You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by moshebla <gi...@git.apache.org> on 2018/09/17 13:05:49 UTC

[GitHub] lucene-solr pull request #455: SOLR-12638

GitHub user moshebla opened a pull request:

    https://github.com/apache/lucene-solr/pull/455

    SOLR-12638

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/moshebla/lucene-solr SOLR-12638

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/lucene-solr/pull/455.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #455
    
----
commit 34e2b0b3fac2f8fb6c085fe4db23130a0b634924
Author: Moshe <mo...@...>
Date:   2018-08-23T08:11:57Z

    SOLR-12638: first unittest

commit ce09bdb208b5cd8d9fb69a7c22377752d3c19200
Author: Moshe <mo...@...>
Date:   2018-08-30T06:36:32Z

    SOLR-12638: start working on RTG fetch block

commit 1fb442915ed16f6a6d844ebffb89bdfbda641c0d
Author: Moshe <mo...@...>
Date:   2018-09-06T07:13:16Z

    SOLR-12638: tests for child docs atomic updates

commit a9d759546784d2ed3674bcc0a85342466b7cc9ad
Author: Moshe <mo...@...>
Date:   2018-09-16T13:10:10Z

    SOLR-12638: edit tests and delete old block by version if exists

commit 8a840a3e56ffc525250389aceae1149eedf336fc
Author: Moshe <mo...@...>
Date:   2018-09-17T05:16:16Z

    SOLR-12638: minor fixes

commit 5617013725a67002914f59acc064fe09c937c07f
Author: Moshe <mo...@...>
Date:   2018-09-17T11:42:41Z

    SOLR-12368: separate configs for atomic block update tests

commit 06b3436e929165cf7df68643f480995bbe2e5c76
Author: Moshe <mo...@...>
Date:   2018-09-17T13:05:09Z

    SOLR-12368: tidy up for PR

----


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r228174440
  
    --- Diff: solr/core/src/java/org/apache/solr/update/AddUpdateCommand.java ---
    @@ -262,6 +263,11 @@ private void flattenAnonymous(List<SolrInputDocument> unwrappedDocs, SolrInputDo
         flattenAnonymous(unwrappedDocs, currentDoc, false);
       }
     
    +  public String getRouteFieldVal() {
    --- End diff --
    
    What code duplication?
    
    I think we need to standardize/harmonize route a bit.  Notice UpdateCommand has setRoute & getRoute.  Lets initialize route in the constructor here by the presence of \_route\_ in the params.  Then lets not look for _route_ in params later since we know we can get it here.  Then I think `org.apache.solr.update.processor.DistributedUpdateProcessor#setupRequest(java.lang.String, org.apache.solr.common.SolrInputDocument)` can be removed and instead insist everyone call the overloaded version that takes a route, and each caller looks up the route from the command.  It's not clear to me if "null" actually should be passed to route in any circumstance.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r231566214
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java ---
    @@ -259,9 +264,8 @@ public boolean doInPlaceUpdateMerge(AddUpdateCommand cmd, Set<String> updatedFie
         SolrInputDocument oldDocument = RealTimeGetComponent.getInputDocument
           (cmd.getReq().getCore(), idBytes,
            null, // don't want the version to be returned
    -       true, // avoid stored fields from index
            updatedFields,
    -       true); // resolve the full document
    +       RealTimeGetComponent.Resolution.FULL_DOC); // get the partial document for the in-place update
    --- End diff --
    
    it is not a partial document; the original comment was correct.  Now with an enum, we don't even need a comment as the enum has a self explanatory name.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r223747918
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java ---
    @@ -442,5 +442,58 @@ protected void doRemoveRegex(SolrInputDocument toDoc, SolrInputField sif, Object
         }
         return patterns;
       }
    +
    +  private Object getNativeFieldValue(String fieldName, Object val) {
    +    if(isChildDoc(val)) {
    +      return val;
    +    }
    +    SchemaField sf = schema.getField(fieldName);
    +    return sf.getType().toNativeType(val);
    +  }
    +
    +  private static boolean isChildDoc(Object obj) {
    +    if(!(obj instanceof Collection)) {
    +      return obj instanceof SolrDocumentBase;
    +    }
    +    Collection objValues = (Collection) obj;
    +    if(objValues.size() == 0) {
    +      return false;
    +    }
    +    return objValues.iterator().next() instanceof SolrDocumentBase;
    +  }
    +
    +  private void removeObj(Collection original, Object toRemove, String fieldName) {
    +    if(isChildDoc(toRemove)) {
    +      removeChildDoc(original, (SolrInputDocument) toRemove);
    +    } else {
    +      original.remove(getNativeFieldValue(fieldName, toRemove));
    +    }
    +  }
    +
    +  private static void removeChildDoc(Collection original, SolrInputDocument docToRemove) {
    +    for(SolrInputDocument doc: (Collection<SolrInputDocument>) original) {
    +      if(isDerivedFromDoc(doc, docToRemove)) {
    +        original.remove(doc);
    +        return;
    +      }
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param fullDoc the document to be tested
    +   * @param subDoc the sub document that should be a subset of fullDoc
    +   * @return whether subDoc is a subset of fullDoc
    +   */
    +  private static boolean isDerivedFromDoc(SolrInputDocument fullDoc, SolrInputDocument subDoc) {
    +    for(SolrInputField subSif: subDoc) {
    +      String fieldName = subSif.getName();
    +      if(!fullDoc.containsKey(fieldName)) return false;
    --- End diff --
    
    This results in a double-lookup of the values with the next line.  Remove this line and after the next one simply do a null-check.


---

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


[GitHub] lucene-solr issue #455: WIP: SOLR-12638

Posted by moshebla <gi...@git.apache.org>.
Github user moshebla commented on the issue:

    https://github.com/apache/lucene-solr/pull/455
  
    Currently the following update operations are supported for child documents:
    
    - add
    - set
    - remove


---

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


[GitHub] lucene-solr pull request #455: WIP: SOLR-12638

Posted by moshebla <gi...@git.apache.org>.
Github user moshebla commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r223336215
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java ---
    @@ -461,5 +466,33 @@ private static boolean isChildDoc(Object obj) {
         }
         return objValues.iterator().next() instanceof SolrDocumentBase;
       }
    +
    +  private void removeObj(Collection original, Object toRemove, String fieldName) {
    +    if(isChildDoc(toRemove)) {
    +      removeChildDoc(original, (SolrInputDocument) toRemove);
    +    } else {
    +      original.remove(getNativeFieldValue(fieldName, toRemove));
    +    }
    +  }
    +
    +  private static void removeChildDoc(Collection original, SolrInputDocument docToRemove) {
    +    for(SolrInputDocument doc: (Collection<SolrInputDocument>) original) {
    +      if(isDerivedFromDoc(doc, docToRemove)) {
    +        original.remove(doc);
    +        return;
    +      }
    +    }
    +  }
    +
    +  private static boolean isDerivedFromDoc(SolrInputDocument fullDoc, SolrInputDocument subDoc) {
    +    for(SolrInputField subSif: subDoc) {
    +      String fieldName = subSif.getName();
    +      if(!fullDoc.containsKey(fieldName)) return false;
    +      Collection<Object> fieldValues = subDoc.getFieldValues(fieldName);
    --- End diff --
    
    Oh yes you are correct.
    I will get to this and test this ASAP.


---

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


[GitHub] lucene-solr pull request #455: WIP: SOLR-12638

Posted by moshebla <gi...@git.apache.org>.
Github user moshebla commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r223355907
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java ---
    @@ -461,5 +466,33 @@ private static boolean isChildDoc(Object obj) {
         }
         return objValues.iterator().next() instanceof SolrDocumentBase;
       }
    +
    +  private void removeObj(Collection original, Object toRemove, String fieldName) {
    +    if(isChildDoc(toRemove)) {
    +      removeChildDoc(original, (SolrInputDocument) toRemove);
    +    } else {
    +      original.remove(getNativeFieldValue(fieldName, toRemove));
    +    }
    +  }
    +
    +  private static void removeChildDoc(Collection original, SolrInputDocument docToRemove) {
    +    for(SolrInputDocument doc: (Collection<SolrInputDocument>) original) {
    +      if(isDerivedFromDoc(doc, docToRemove)) {
    +        original.remove(doc);
    +        return;
    +      }
    +    }
    +  }
    +
    +  private static boolean isDerivedFromDoc(SolrInputDocument fullDoc, SolrInputDocument subDoc) {
    --- End diff --
    
    Exactly,
    I will add Javadocs for this.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r224130315
  
    --- Diff: solr/core/src/test/org/apache/solr/update/processor/AtomicUpdateBlockTest.java ---
    @@ -0,0 +1,370 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.solr.update.processor;
    +
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.List;
    +
    +import org.apache.lucene.util.BytesRef;
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.core.SolrCore;
    +import org.apache.solr.handler.component.RealTimeGetComponent;
    +import org.junit.AfterClass;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +public class AtomicUpdateBlockTest extends SolrTestCaseJ4 {
    +
    +  private final static String VERSION = "_version_";
    +  private static String PREVIOUS_ENABLE_UPDATE_LOG_VALUE;
    +
    +  @BeforeClass
    +  public static void beforeTests() throws Exception {
    +    PREVIOUS_ENABLE_UPDATE_LOG_VALUE = System.getProperty("enable.update.log");
    +    System.setProperty("enable.update.log", "true");
    +    initCore("solrconfig-block-atomic-update.xml", "schema-nest.xml"); // use "nest" schema
    +  }
    +
    +  @AfterClass
    +  public static void afterTests() throws Exception {
    +    // restore enable.update.log
    +    System.setProperty("enable.update.log", PREVIOUS_ENABLE_UPDATE_LOG_VALUE);
    +  }
    +
    +  @Before
    +  public void before() {
    +    clearIndex();
    +    assertU(commit());
    +  }
    +
    +  @Test
    +  public void testMergeChildDoc() throws Exception {
    +    SolrInputDocument doc = new SolrInputDocument();
    +    doc.setField("id", "1");
    +    doc.setField("cat_ss", new String[]{"aaa", "ccc"});
    +    doc.setField("child", Collections.singletonList(sdoc("id", "2", "cat_ss", "child")));
    +    addDoc(adoc(doc), "nested-rtg");
    +
    +    BytesRef rootDocId = new BytesRef("1");
    +    SolrCore core = h.getCore();
    +    SolrInputDocument block = RealTimeGetComponent.getInputDocument(core, rootDocId, true);
    +    // assert block doc has child docs
    +    assertTrue(block.containsKey("child"));
    +
    +    assertJQ(req("q","id:1")
    +        ,"/response/numFound==0"
    +    );
    +
    +    // commit the changes
    +    assertU(commit());
    +
    +    SolrInputDocument newChildDoc = sdoc("id", "3", "cat_ss", "child");
    +    SolrInputDocument addedDoc = sdoc("id", "1",
    +        "cat_ss", Collections.singletonMap("add", "bbb"),
    +        "child", Collections.singletonMap("add", sdocs(newChildDoc)));
    +    block = RealTimeGetComponent.getInputDocument(core, rootDocId, true);
    +    block.removeField(VERSION);
    +    SolrInputDocument preMergeDoc = new SolrInputDocument(block);
    +    AtomicUpdateDocumentMerger docMerger = new AtomicUpdateDocumentMerger(req());
    +    docMerger.merge(addedDoc, block);
    --- End diff --
    
    It seems the point of this test is to test AtomicUpdateDocumentMerger.merge?  Then why even index anything at all (the first half of this test)?  Simply create the documents directly and call the merge method and test the result.


---

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


[GitHub] lucene-solr pull request #455: WIP: SOLR-12638

Posted by moshebla <gi...@git.apache.org>.
Github user moshebla commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r223359889
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java ---
    @@ -461,5 +466,33 @@ private static boolean isChildDoc(Object obj) {
         }
         return objValues.iterator().next() instanceof SolrDocumentBase;
       }
    +
    +  private void removeObj(Collection original, Object toRemove, String fieldName) {
    +    if(isChildDoc(toRemove)) {
    +      removeChildDoc(original, (SolrInputDocument) toRemove);
    +    } else {
    +      original.remove(getNativeFieldValue(fieldName, toRemove));
    +    }
    +  }
    +
    +  private static void removeChildDoc(Collection original, SolrInputDocument docToRemove) {
    +    for(SolrInputDocument doc: (Collection<SolrInputDocument>) original) {
    +      if(isDerivedFromDoc(doc, docToRemove)) {
    +        original.remove(doc);
    --- End diff --
    
    I was thinking that if doc {"id": "4"} was supplied, then the **whole** child doc with that id(4) is to be removed.
    Or perhaps you have a different scenario in mind, where a field of a a particular child document is to be removed while the rest of the child document is left intact?


---

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


[GitHub] lucene-solr issue #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on the issue:

    https://github.com/apache/lucene-solr/pull/455
  
    As a general point, I feel we should prefer "nested" terminology instead of "block".  If we were purely working within Lucene then I think "block" might be okay but at the Solr layer people see this stuff as "nested" docs.


---

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


[GitHub] lucene-solr issue #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on the issue:

    https://github.com/apache/lucene-solr/pull/455
  
    What does "so _root_ is not hard-coded into childless documents" mean?  If you think SOLR-5211 is missing something, can you please post a patch file to the issue and explain what you're adding?


---

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


[GitHub] lucene-solr pull request #455: WIP: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r223244909
  
    --- Diff: solr/core/src/test/org/apache/solr/update/processor/AtomicUpdateBlockTest.java ---
    @@ -181,6 +182,180 @@ public void testBlockRealTimeGet() throws Exception {
         );
       }
     
    +  @Test
    +  public void testBlockAtomicSet() throws Exception {
    +    SolrInputDocument doc = sdoc("id", "1",
    +        "cat_ss", new String[] {"aaa", "ccc"},
    +        "child1", Collections.singleton(sdoc("id", "2", "cat_ss", "child"))
    +    );
    +    json(doc);
    +    addDoc(adoc(doc), "nested-rtg");
    +
    +    BytesRef rootDocId = new BytesRef("1");
    +    SolrCore core = h.getCore();
    +    SolrInputDocument block = RealTimeGetComponent.getInputDocument(core, rootDocId, true);
    +    // assert block doc has child docs
    +    assertTrue(block.containsKey("child1"));
    +
    +    assertJQ(req("q","id:1")
    +        ,"/response/numFound==0"
    +    );
    +
    +    // commit the changes
    +    assertU(commit());
    +
    +    SolrInputDocument committedBlock = RealTimeGetComponent.getInputDocument(core, rootDocId, true);
    +    BytesRef childDocId = new BytesRef("2");
    +    // ensure the whole block is returned when resolveBlock is true and id of a child doc is provided
    +    assertEquals(committedBlock.toString(), RealTimeGetComponent.getInputDocument(core, childDocId, true).toString());
    +
    +    assertJQ(req("q","id:1")
    +        ,"/response/numFound==1"
    +    );
    +
    +    assertJQ(req("qt","/get", "id","1", "fl","id, cat_ss, child1, [child]")
    +        ,"=={\"doc\":{'id':\"1\"" +
    +            ", cat_ss:[\"aaa\",\"ccc\"], child1:[{\"id\":\"2\",\"cat_ss\":[\"child\"]}]" +
    +            "       }}"
    +    );
    +
    +    assertU(commit());
    +
    +    assertJQ(req("qt","/get", "id","1", "fl","id, cat_ss, child1, [child]")
    +        ,"=={\"doc\":{'id':\"1\"" +
    +            ", cat_ss:[\"aaa\",\"ccc\"], child1:[{\"id\":\"2\",\"cat_ss\":[\"child\"]}]" +
    +            "       }}"
    +    );
    +
    +    doc = sdoc("id", "1",
    +        "cat_ss", ImmutableMap.of("set", Arrays.asList("aaa", "bbb")),
    --- End diff --
    
    Minor: in cases where the JDK has alternatives to Guava, use them.  Here, it's `Collections.singletonMap` I recall.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by moshebla <gi...@git.apache.org>.
Github user moshebla commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r226858107
  
    --- Diff: solr/core/src/test/org/apache/solr/update/processor/AtomicUpdateBlockTest.java ---
    @@ -59,39 +52,147 @@ public void before() {
     
       @Test
       public void testMergeChildDoc() throws Exception {
    -    SolrInputDocument doc = new SolrInputDocument();
    -    doc.setField("id", "1");
    -    doc.setField("cat_ss", new String[]{"aaa", "ccc"});
    -    doc.setField("child", Collections.singletonList(sdoc("id", "2", "cat_ss", "child")));
    +    SolrInputDocument newChildDoc = sdoc("id", "3", "cat_ss", "child");
    +    SolrInputDocument addedDoc = sdoc("id", "1",
    +        "cat_ss", Collections.singletonMap("add", "bbb"),
    +        "child", Collections.singletonMap("add", sdocs(newChildDoc)));
    +
    +    SolrInputDocument dummyBlock = sdoc("id", "1",
    +        "cat_ss", new ArrayList<>(Arrays.asList("aaa", "ccc")),
    +        "_root_", "1", "child", new ArrayList<>(sdocs(addedDoc)));
    +    dummyBlock.removeField(VERSION);
    +
    +    SolrInputDocument preMergeDoc = new SolrInputDocument(dummyBlock);
    +    AtomicUpdateDocumentMerger docMerger = new AtomicUpdateDocumentMerger(req());
    +    docMerger.merge(addedDoc, dummyBlock);
    +    assertEquals("merged document should have the same id", preMergeDoc.getFieldValue("id"), dummyBlock.getFieldValue("id"));
    +    assertDocContainsSubset(preMergeDoc, dummyBlock);
    +    assertDocContainsSubset(addedDoc, dummyBlock);
    +    assertDocContainsSubset(newChildDoc, (SolrInputDocument) ((List) dummyBlock.getFieldValues("child")).get(1));
    +    assertEquals(dummyBlock.getFieldValue("id"), dummyBlock.getFieldValue("id"));
    +  }
    +
    +  @Test
    +  public void testBlockAtomicQuantities() throws Exception {
    +    SolrInputDocument doc = sdoc("id", "1", "string_s", "root");
         addDoc(adoc(doc), "nested-rtg");
     
    -    BytesRef rootDocId = new BytesRef("1");
    -    SolrCore core = h.getCore();
    -    SolrInputDocument block = RealTimeGetComponent.getInputDocument(core, rootDocId, true);
    -    // assert block doc has child docs
    -    assertTrue(block.containsKey("child"));
    +    List<SolrInputDocument> docs = IntStream.range(10, 20).mapToObj(x -> sdoc("id", String.valueOf(x), "string_s", "child")).collect(Collectors.toList());
    +    doc = sdoc("id", "1", "children", Collections.singletonMap("add", docs));
    +    addAndGetVersion(doc, params("update.chain", "nested-rtg", "wt", "json"));
     
    -    assertJQ(req("q","id:1")
    -        ,"/response/numFound==0"
    +    assertU(commit());
    +
    +    assertJQ(req("q", "_root_:1"),
    +        "/response/numFound==11");
    +
    +    assertJQ(req("q", "string_s:child", "fl", "*", "rows", "1000000"),
    +        "/response/numFound==10");
    +
    +    // ensure updates work when block has more than 10 children
    --- End diff --
    
    This is to ensure the limit=-1 is passed to childDocumentTransformer, to retrieve all the block's child documents


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r224133917
  
    --- Diff: solr/core/src/test/org/apache/solr/update/processor/AtomicUpdateBlockTest.java ---
    @@ -0,0 +1,370 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.solr.update.processor;
    +
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.List;
    +
    +import org.apache.lucene.util.BytesRef;
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.core.SolrCore;
    +import org.apache.solr.handler.component.RealTimeGetComponent;
    +import org.junit.AfterClass;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +public class AtomicUpdateBlockTest extends SolrTestCaseJ4 {
    +
    +  private final static String VERSION = "_version_";
    +  private static String PREVIOUS_ENABLE_UPDATE_LOG_VALUE;
    +
    +  @BeforeClass
    +  public static void beforeTests() throws Exception {
    +    PREVIOUS_ENABLE_UPDATE_LOG_VALUE = System.getProperty("enable.update.log");
    +    System.setProperty("enable.update.log", "true");
    +    initCore("solrconfig-block-atomic-update.xml", "schema-nest.xml"); // use "nest" schema
    +  }
    +
    +  @AfterClass
    +  public static void afterTests() throws Exception {
    +    // restore enable.update.log
    +    System.setProperty("enable.update.log", PREVIOUS_ENABLE_UPDATE_LOG_VALUE);
    +  }
    +
    +  @Before
    +  public void before() {
    +    clearIndex();
    +    assertU(commit());
    +  }
    +
    +  @Test
    +  public void testMergeChildDoc() throws Exception {
    +    SolrInputDocument doc = new SolrInputDocument();
    +    doc.setField("id", "1");
    +    doc.setField("cat_ss", new String[]{"aaa", "ccc"});
    +    doc.setField("child", Collections.singletonList(sdoc("id", "2", "cat_ss", "child")));
    +    addDoc(adoc(doc), "nested-rtg");
    +
    +    BytesRef rootDocId = new BytesRef("1");
    +    SolrCore core = h.getCore();
    +    SolrInputDocument block = RealTimeGetComponent.getInputDocument(core, rootDocId, true);
    +    // assert block doc has child docs
    +    assertTrue(block.containsKey("child"));
    +
    +    assertJQ(req("q","id:1")
    +        ,"/response/numFound==0"
    +    );
    +
    +    // commit the changes
    +    assertU(commit());
    +
    +    SolrInputDocument newChildDoc = sdoc("id", "3", "cat_ss", "child");
    +    SolrInputDocument addedDoc = sdoc("id", "1",
    +        "cat_ss", Collections.singletonMap("add", "bbb"),
    +        "child", Collections.singletonMap("add", sdocs(newChildDoc)));
    +    block = RealTimeGetComponent.getInputDocument(core, rootDocId, true);
    +    block.removeField(VERSION);
    +    SolrInputDocument preMergeDoc = new SolrInputDocument(block);
    +    AtomicUpdateDocumentMerger docMerger = new AtomicUpdateDocumentMerger(req());
    +    docMerger.merge(addedDoc, block);
    +    assertEquals("merged document should have the same id", preMergeDoc.getFieldValue("id"), block.getFieldValue("id"));
    +    assertDocContainsSubset(preMergeDoc, block);
    +    assertDocContainsSubset(addedDoc, block);
    +    assertDocContainsSubset(newChildDoc, (SolrInputDocument) ((List) block.getFieldValues("child")).get(1));
    +    assertEquals(doc.getFieldValue("id"), block.getFieldValue("id"));
    +  }
    +
    +  @Test
    +  public void testBlockAtomicAdd() throws Exception {
    +
    +    SolrInputDocument doc = sdoc("id", "1",
    +        "cat_ss", new String[] {"aaa", "ccc"},
    +        "child1", sdoc("id", "2", "cat_ss", "child")
    +    );
    +    json(doc);
    +    addDoc(adoc(doc), "nested-rtg");
    --- End diff --
    
    Maybe the default chain in this config should have those URPs, and therefore we wouldn't need to have the tests refer to the URP chain.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r226709264
  
    --- Diff: solr/core/src/test/org/apache/solr/update/processor/AtomicUpdateBlockTest.java ---
    @@ -59,39 +52,147 @@ public void before() {
     
       @Test
       public void testMergeChildDoc() throws Exception {
    -    SolrInputDocument doc = new SolrInputDocument();
    -    doc.setField("id", "1");
    -    doc.setField("cat_ss", new String[]{"aaa", "ccc"});
    -    doc.setField("child", Collections.singletonList(sdoc("id", "2", "cat_ss", "child")));
    +    SolrInputDocument newChildDoc = sdoc("id", "3", "cat_ss", "child");
    +    SolrInputDocument addedDoc = sdoc("id", "1",
    +        "cat_ss", Collections.singletonMap("add", "bbb"),
    +        "child", Collections.singletonMap("add", sdocs(newChildDoc)));
    +
    +    SolrInputDocument dummyBlock = sdoc("id", "1",
    +        "cat_ss", new ArrayList<>(Arrays.asList("aaa", "ccc")),
    +        "_root_", "1", "child", new ArrayList<>(sdocs(addedDoc)));
    +    dummyBlock.removeField(VERSION);
    +
    +    SolrInputDocument preMergeDoc = new SolrInputDocument(dummyBlock);
    +    AtomicUpdateDocumentMerger docMerger = new AtomicUpdateDocumentMerger(req());
    +    docMerger.merge(addedDoc, dummyBlock);
    +    assertEquals("merged document should have the same id", preMergeDoc.getFieldValue("id"), dummyBlock.getFieldValue("id"));
    +    assertDocContainsSubset(preMergeDoc, dummyBlock);
    +    assertDocContainsSubset(addedDoc, dummyBlock);
    +    assertDocContainsSubset(newChildDoc, (SolrInputDocument) ((List) dummyBlock.getFieldValues("child")).get(1));
    +    assertEquals(dummyBlock.getFieldValue("id"), dummyBlock.getFieldValue("id"));
    +  }
    +
    +  @Test
    +  public void testBlockAtomicQuantities() throws Exception {
    +    SolrInputDocument doc = sdoc("id", "1", "string_s", "root");
         addDoc(adoc(doc), "nested-rtg");
     
    -    BytesRef rootDocId = new BytesRef("1");
    -    SolrCore core = h.getCore();
    -    SolrInputDocument block = RealTimeGetComponent.getInputDocument(core, rootDocId, true);
    -    // assert block doc has child docs
    -    assertTrue(block.containsKey("child"));
    +    List<SolrInputDocument> docs = IntStream.range(10, 20).mapToObj(x -> sdoc("id", String.valueOf(x), "string_s", "child")).collect(Collectors.toList());
    +    doc = sdoc("id", "1", "children", Collections.singletonMap("add", docs));
    +    addAndGetVersion(doc, params("update.chain", "nested-rtg", "wt", "json"));
     
    -    assertJQ(req("q","id:1")
    -        ,"/response/numFound==0"
    +    assertU(commit());
    +
    +    assertJQ(req("q", "_root_:1"),
    +        "/response/numFound==11");
    +
    +    assertJQ(req("q", "string_s:child", "fl", "*", "rows", "1000000"),
    +        "/response/numFound==10");
    +
    +    // ensure updates work when block has more than 10 children
    --- End diff --
    
    Why is 10 special?


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r223747455
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java ---
    @@ -442,5 +442,58 @@ protected void doRemoveRegex(SolrInputDocument toDoc, SolrInputField sif, Object
         }
         return patterns;
       }
    +
    +  private Object getNativeFieldValue(String fieldName, Object val) {
    +    if(isChildDoc(val)) {
    +      return val;
    +    }
    +    SchemaField sf = schema.getField(fieldName);
    +    return sf.getType().toNativeType(val);
    +  }
    +
    +  private static boolean isChildDoc(Object obj) {
    +    if(!(obj instanceof Collection)) {
    +      return obj instanceof SolrDocumentBase;
    +    }
    +    Collection objValues = (Collection) obj;
    +    if(objValues.size() == 0) {
    +      return false;
    +    }
    +    return objValues.iterator().next() instanceof SolrDocumentBase;
    +  }
    +
    +  private void removeObj(Collection original, Object toRemove, String fieldName) {
    +    if(isChildDoc(toRemove)) {
    +      removeChildDoc(original, (SolrInputDocument) toRemove);
    +    } else {
    +      original.remove(getNativeFieldValue(fieldName, toRemove));
    +    }
    +  }
    +
    +  private static void removeChildDoc(Collection original, SolrInputDocument docToRemove) {
    +    for(SolrInputDocument doc: (Collection<SolrInputDocument>) original) {
    +      if(isDerivedFromDoc(doc, docToRemove)) {
    +        original.remove(doc);
    +        return;
    +      }
    +    }
    +  }
    +
    +  /**
    +   *
    +   * @param fullDoc the document to be tested
    +   * @param subDoc the sub document that should be a subset of fullDoc
    +   * @return whether subDoc is a subset of fullDoc
    +   */
    +  private static boolean isDerivedFromDoc(SolrInputDocument fullDoc, SolrInputDocument subDoc) {
    +    for(SolrInputField subSif: subDoc) {
    +      String fieldName = subSif.getName();
    +      if(!fullDoc.containsKey(fieldName)) return false;
    +      Collection<Object> fieldValues = fullDoc.getFieldValues(fieldName);
    +      if(fieldValues.size() < subSif.getValueCount()) return false;
    +      if(!fullDoc.getFieldValues(fieldName).containsAll(subSif.getValues())) return false;
    --- End diff --
    
    `fullDoc.getFieldValues(fieldName)` on this line can be replaced by `fieldValues` since we already have the values.  And the previous line on the count is unnecessary since the containsAll check on this line would fail.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r224300831
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java ---
    @@ -639,12 +650,30 @@ public static SolrInputDocument getInputDocument(SolrCore core, BytesRef idBytes
               sid = new SolrInputDocument();
             } else {
               Document luceneDocument = docFetcher.doc(docid);
    -          sid = toSolrInputDocument(luceneDocument, core.getLatestSchema());
    +          sid = toSolrInputDocument(luceneDocument, schema);
             }
    -        if (onlyTheseNonStoredDVs != null) {
    -          docFetcher.decorateDocValueFields(sid, docid, onlyTheseNonStoredDVs);
    -        } else {
    -          docFetcher.decorateDocValueFields(sid, docid, docFetcher.getNonStoredDVsWithoutCopyTargets());
    +        ensureDocDecorated(onlyTheseNonStoredDVs, sid, docid, docFetcher, resolveBlock || schema.hasExplicitField(IndexSchema.NEST_PATH_FIELD_NAME));
    +        SolrInputField rootField;
    --- End diff --
    
    no big deal to simply initialize rootField up front.  You are doing it as an expression with a side-effect below which is needlessly awkward.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r224306158
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java ---
    @@ -639,12 +650,30 @@ public static SolrInputDocument getInputDocument(SolrCore core, BytesRef idBytes
               sid = new SolrInputDocument();
             } else {
               Document luceneDocument = docFetcher.doc(docid);
    -          sid = toSolrInputDocument(luceneDocument, core.getLatestSchema());
    +          sid = toSolrInputDocument(luceneDocument, schema);
             }
    -        if (onlyTheseNonStoredDVs != null) {
    -          docFetcher.decorateDocValueFields(sid, docid, onlyTheseNonStoredDVs);
    -        } else {
    -          docFetcher.decorateDocValueFields(sid, docid, docFetcher.getNonStoredDVsWithoutCopyTargets());
    +        ensureDocDecorated(onlyTheseNonStoredDVs, sid, docid, docFetcher, resolveBlock || schema.hasExplicitField(IndexSchema.NEST_PATH_FIELD_NAME));
    +        SolrInputField rootField;
    +        if(resolveBlock && schema.isUsableForChildDocs() && (rootField = sid.getField(IndexSchema.ROOT_FIELD_NAME))!=null) {
    +          // doc is part of a nested structure
    +          ModifiableSolrParams params = new ModifiableSolrParams()
    +              .set("q", core.getLatestSchema().getUniqueKeyField().getName()+ ":" +rootField.getFirstValue())
    --- End diff --
    
    It seems the LocalSolrQueryRequest here is a dummy needed to satisfy some of the methods below.  This threw me; there should be comments and/or choice of var names (e.g. dummyReq) to reflect this.  "q" isn't needed; just the "fl".  It seems we don't even need the "fl" here since that can be supplied as the first parameter to SolrReturnFields, which seems better if it works.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r224131263
  
    --- Diff: solr/core/src/test/org/apache/solr/update/processor/AtomicUpdateBlockTest.java ---
    @@ -0,0 +1,370 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.solr.update.processor;
    +
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.List;
    +
    +import org.apache.lucene.util.BytesRef;
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.core.SolrCore;
    +import org.apache.solr.handler.component.RealTimeGetComponent;
    +import org.junit.AfterClass;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +public class AtomicUpdateBlockTest extends SolrTestCaseJ4 {
    +
    +  private final static String VERSION = "_version_";
    +  private static String PREVIOUS_ENABLE_UPDATE_LOG_VALUE;
    +
    +  @BeforeClass
    +  public static void beforeTests() throws Exception {
    +    PREVIOUS_ENABLE_UPDATE_LOG_VALUE = System.getProperty("enable.update.log");
    +    System.setProperty("enable.update.log", "true");
    +    initCore("solrconfig-block-atomic-update.xml", "schema-nest.xml"); // use "nest" schema
    +  }
    +
    +  @AfterClass
    +  public static void afterTests() throws Exception {
    +    // restore enable.update.log
    +    System.setProperty("enable.update.log", PREVIOUS_ENABLE_UPDATE_LOG_VALUE);
    +  }
    +
    +  @Before
    +  public void before() {
    +    clearIndex();
    +    assertU(commit());
    +  }
    +
    +  @Test
    +  public void testMergeChildDoc() throws Exception {
    +    SolrInputDocument doc = new SolrInputDocument();
    +    doc.setField("id", "1");
    +    doc.setField("cat_ss", new String[]{"aaa", "ccc"});
    +    doc.setField("child", Collections.singletonList(sdoc("id", "2", "cat_ss", "child")));
    +    addDoc(adoc(doc), "nested-rtg");
    +
    +    BytesRef rootDocId = new BytesRef("1");
    +    SolrCore core = h.getCore();
    +    SolrInputDocument block = RealTimeGetComponent.getInputDocument(core, rootDocId, true);
    +    // assert block doc has child docs
    +    assertTrue(block.containsKey("child"));
    +
    +    assertJQ(req("q","id:1")
    +        ,"/response/numFound==0"
    +    );
    +
    +    // commit the changes
    +    assertU(commit());
    +
    +    SolrInputDocument newChildDoc = sdoc("id", "3", "cat_ss", "child");
    +    SolrInputDocument addedDoc = sdoc("id", "1",
    +        "cat_ss", Collections.singletonMap("add", "bbb"),
    +        "child", Collections.singletonMap("add", sdocs(newChildDoc)));
    +    block = RealTimeGetComponent.getInputDocument(core, rootDocId, true);
    +    block.removeField(VERSION);
    +    SolrInputDocument preMergeDoc = new SolrInputDocument(block);
    +    AtomicUpdateDocumentMerger docMerger = new AtomicUpdateDocumentMerger(req());
    +    docMerger.merge(addedDoc, block);
    +    assertEquals("merged document should have the same id", preMergeDoc.getFieldValue("id"), block.getFieldValue("id"));
    +    assertDocContainsSubset(preMergeDoc, block);
    +    assertDocContainsSubset(addedDoc, block);
    +    assertDocContainsSubset(newChildDoc, (SolrInputDocument) ((List) block.getFieldValues("child")).get(1));
    +    assertEquals(doc.getFieldValue("id"), block.getFieldValue("id"));
    +  }
    +
    +  @Test
    +  public void testBlockAtomicAdd() throws Exception {
    +
    +    SolrInputDocument doc = sdoc("id", "1",
    +        "cat_ss", new String[] {"aaa", "ccc"},
    +        "child1", sdoc("id", "2", "cat_ss", "child")
    --- End diff --
    
    I think it'd be easier to comprehend tests involving nested documents if the ID for a nested document somehow looked different.  For example, for this child doc, do "1.1" to mean the first child doc of parent doc 1.  Second would be "1.2".  WDYT?


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by moshebla <gi...@git.apache.org>.
Github user moshebla commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r223741723
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java ---
    @@ -461,5 +466,33 @@ private static boolean isChildDoc(Object obj) {
         }
         return objValues.iterator().next() instanceof SolrDocumentBase;
       }
    +
    +  private void removeObj(Collection original, Object toRemove, String fieldName) {
    +    if(isChildDoc(toRemove)) {
    +      removeChildDoc(original, (SolrInputDocument) toRemove);
    +    } else {
    +      original.remove(getNativeFieldValue(fieldName, toRemove));
    +    }
    +  }
    +
    +  private static void removeChildDoc(Collection original, SolrInputDocument docToRemove) {
    +    for(SolrInputDocument doc: (Collection<SolrInputDocument>) original) {
    +      if(isDerivedFromDoc(doc, docToRemove)) {
    +        original.remove(doc);
    +        return;
    +      }
    +    }
    +  }
    +
    +  private static boolean isDerivedFromDoc(SolrInputDocument fullDoc, SolrInputDocument subDoc) {
    --- End diff --
    
    Just pushed a new commit to address your comments.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r224306195
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java ---
    @@ -661,6 +690,21 @@ public static SolrInputDocument getInputDocument(SolrCore core, BytesRef idBytes
         return sid;
       }
     
    +  private static void ensureDocDecorated(Set<String> onlyTheseNonStoredDVs, SolrDocumentBase doc, int docid, SolrDocumentFetcher docFetcher) throws IOException {
    --- End diff --
    
    I suggest renaming these methods `ensureDocDecorated` since it's what it calls.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r231567341
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java ---
    @@ -1164,6 +1225,31 @@ public void processGetUpdates(ResponseBuilder rb) throws IOException
         return new ArrayList<>(versionsToRet);
       }
     
    +  /**
    +   *  <p>
    +   *    Lookup strategy for {@link #getInputDocument(SolrCore, BytesRef, AtomicLong, Set, Resolution)}.
    +   *  </p>
    +   *  <ul>
    +   *    <li>{@link #FULL_DOC}</li>
    +   *    <li>{@link #DOC_CHILDREN}</li>
    +   *    <li>{@link #FULL_HIERARCHY}</li>
    +   *  </ul>
    +   */
    +  public static enum Resolution {
    --- End diff --
    
    Suggested new names:  "DOC", "DOC_WITH_CHILDREN", "ROOT_WITH_CHILDREN".
    Resolution is now only used where there is no "in place" so no need to mention partial documents.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by moshebla <gi...@git.apache.org>.
Github user moshebla commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r228032204
  
    --- Diff: solr/core/src/java/org/apache/solr/update/AddUpdateCommand.java ---
    @@ -262,6 +263,11 @@ private void flattenAnonymous(List<SolrInputDocument> unwrappedDocs, SolrInputDo
         flattenAnonymous(unwrappedDocs, currentDoc, false);
       }
     
    +  public String getRouteFieldVal() {
    --- End diff --
    
    After reading through the code it seems like _route_ is currently only used in delete commands and queries.
    This can be seen in DistributedUpdateProceesor#doDeleteByQuery
    `ModifiableSolrParams outParams = new ModifiableSolrParams(filterParams(req.getParams()));
          outParams.set(DISTRIB_UPDATE_PARAM, DistribPhase.TOLEADER.toString());
          outParams.set(DISTRIB_FROM, ZkCoreNodeProps.getCoreUrl(
              zkController.getBaseUrl(), req.getCore().getName()));
    
          SolrParams params = req.getParams();
          String route = params.get(ShardParams._ROUTE_);
          Collection<Slice> slices = coll.getRouter().getSearchSlices(route, params, coll);
    
          List<Node> leaders =  new ArrayList<>(slices.size());
          for (Slice slice : slices) {
            String sliceName = slice.getName();
            Replica leader;
            try {
              leader = zkController.getZkStateReader().getLeaderRetry(collection, sliceName);
            } catch (InterruptedException e) {
              throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "Exception finding leader for shard " + sliceName, e);
            }`
    Perhaps this should be moved to addUpdateCommand(perhaps even UpdateCommand)
    or to a method that will be accessible and visible so we do not have this confusion in the future?


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r224307768
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java ---
    @@ -1360,24 +1385,47 @@ boolean getUpdatedDocument(AddUpdateCommand cmd, long versionOnUpdate) throws IO
         }
         
         // full (non-inplace) atomic update
    +    final boolean isNestedSchema = req.getSchema().isUsableForChildDocs();
         SolrInputDocument sdoc = cmd.getSolrInputDocument();
         BytesRef id = cmd.getIndexedId();
    -    SolrInputDocument oldDoc = RealTimeGetComponent.getInputDocument(cmd.getReq().getCore(), id);
    +    SolrInputDocument blockDoc = RealTimeGetComponent.getInputDocument(cmd.getReq().getCore(), id, null,
    +        false, null, true, true);
     
    -    if (oldDoc == null) {
    -      // create a new doc by default if an old one wasn't found
    -      if (versionOnUpdate <= 0) {
    -        oldDoc = new SolrInputDocument();
    -      } else {
    +    if (blockDoc == null) {
    +      if (versionOnUpdate > 0) {
             // could just let the optimistic locking throw the error
             throw new SolrException(ErrorCode.CONFLICT, "Document not found for update.  id=" + cmd.getPrintableId());
           }
         } else {
    -      oldDoc.remove(CommonParams.VERSION_FIELD);
    +      blockDoc.remove(CommonParams.VERSION_FIELD);
         }
     
     
    -    cmd.solrDoc = docMerger.merge(sdoc, oldDoc);
    +    SolrInputDocument mergedDoc;
    +    if(idField == null || blockDoc == null) {
    +      // create a new doc by default if an old one wasn't found
    +      mergedDoc = docMerger.merge(sdoc, new SolrInputDocument());
    +    } else {
    +      if(isNestedSchema && req.getSchema().hasExplicitField(IndexSchema.NEST_PATH_FIELD_NAME) &&
    +          blockDoc.containsKey(IndexSchema.ROOT_FIELD_NAME) && !id.utf8ToString().equals(blockDoc.getFieldValue(IndexSchema.ROOT_FIELD_NAME))) {
    +        SolrInputDocument oldDoc = RealTimeGetComponent.getInputDocument(cmd.getReq().getCore(), id, null,
    +            false, null, true, false);
    +        mergedDoc = docMerger.merge(sdoc, oldDoc);
    +        String docPath = (String) mergedDoc.getFieldValue(IndexSchema.NEST_PATH_FIELD_NAME);
    +        List<String> docPaths = StrUtils.splitSmart(docPath, PATH_SEP_CHAR);
    +        SolrInputField replaceDoc = blockDoc.getField(docPaths.remove(0).replaceAll(PATH_SEP_CHAR + "|" + NUM_SEP_CHAR, ""));
    --- End diff --
    
    The logic here (not just this line) is non-obvious to me.  There are no comments.  Please add comments and maybe refactor out a method.  The use of replaceAll with a regexp is suspicious to me.  None of the tests you added triggered a breakpoint within the docPaths loop below.  Needs more testing or possible error.


---

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


[GitHub] lucene-solr pull request #455: WIP: SOLR-12638

Posted by moshebla <gi...@git.apache.org>.
Github user moshebla commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r223355767
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java ---
    @@ -461,5 +466,33 @@ private static boolean isChildDoc(Object obj) {
         }
         return objValues.iterator().next() instanceof SolrDocumentBase;
       }
    +
    +  private void removeObj(Collection original, Object toRemove, String fieldName) {
    +    if(isChildDoc(toRemove)) {
    +      removeChildDoc(original, (SolrInputDocument) toRemove);
    +    } else {
    +      original.remove(getNativeFieldValue(fieldName, toRemove));
    +    }
    +  }
    +
    +  private static void removeChildDoc(Collection original, SolrInputDocument docToRemove) {
    +    for(SolrInputDocument doc: (Collection<SolrInputDocument>) original) {
    +      if(isDerivedFromDoc(doc, docToRemove)) {
    +        original.remove(doc);
    +        return;
    +      }
    +    }
    +  }
    +
    +  private static boolean isDerivedFromDoc(SolrInputDocument fullDoc, SolrInputDocument subDoc) {
    +    for(SolrInputField subSif: subDoc) {
    +      String fieldName = subSif.getName();
    +      if(!fullDoc.containsKey(fieldName)) return false;
    +      Collection<Object> fieldValues = subDoc.getFieldValues(fieldName);
    --- End diff --
    
    fieldValues was supposed to be:
    `fullDoc.getFieldValues(fieldName);`


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r224131571
  
    --- Diff: solr/core/src/test/org/apache/solr/update/processor/AtomicUpdateBlockTest.java ---
    @@ -0,0 +1,370 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.solr.update.processor;
    +
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.List;
    +
    +import org.apache.lucene.util.BytesRef;
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.core.SolrCore;
    +import org.apache.solr.handler.component.RealTimeGetComponent;
    +import org.junit.AfterClass;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +public class AtomicUpdateBlockTest extends SolrTestCaseJ4 {
    +
    +  private final static String VERSION = "_version_";
    +  private static String PREVIOUS_ENABLE_UPDATE_LOG_VALUE;
    +
    +  @BeforeClass
    +  public static void beforeTests() throws Exception {
    +    PREVIOUS_ENABLE_UPDATE_LOG_VALUE = System.getProperty("enable.update.log");
    +    System.setProperty("enable.update.log", "true");
    +    initCore("solrconfig-block-atomic-update.xml", "schema-nest.xml"); // use "nest" schema
    +  }
    +
    +  @AfterClass
    +  public static void afterTests() throws Exception {
    +    // restore enable.update.log
    +    System.setProperty("enable.update.log", PREVIOUS_ENABLE_UPDATE_LOG_VALUE);
    +  }
    +
    +  @Before
    +  public void before() {
    +    clearIndex();
    +    assertU(commit());
    +  }
    +
    +  @Test
    +  public void testMergeChildDoc() throws Exception {
    +    SolrInputDocument doc = new SolrInputDocument();
    +    doc.setField("id", "1");
    +    doc.setField("cat_ss", new String[]{"aaa", "ccc"});
    +    doc.setField("child", Collections.singletonList(sdoc("id", "2", "cat_ss", "child")));
    +    addDoc(adoc(doc), "nested-rtg");
    +
    +    BytesRef rootDocId = new BytesRef("1");
    +    SolrCore core = h.getCore();
    +    SolrInputDocument block = RealTimeGetComponent.getInputDocument(core, rootDocId, true);
    +    // assert block doc has child docs
    +    assertTrue(block.containsKey("child"));
    +
    +    assertJQ(req("q","id:1")
    +        ,"/response/numFound==0"
    +    );
    +
    +    // commit the changes
    +    assertU(commit());
    +
    +    SolrInputDocument newChildDoc = sdoc("id", "3", "cat_ss", "child");
    +    SolrInputDocument addedDoc = sdoc("id", "1",
    +        "cat_ss", Collections.singletonMap("add", "bbb"),
    +        "child", Collections.singletonMap("add", sdocs(newChildDoc)));
    +    block = RealTimeGetComponent.getInputDocument(core, rootDocId, true);
    +    block.removeField(VERSION);
    +    SolrInputDocument preMergeDoc = new SolrInputDocument(block);
    +    AtomicUpdateDocumentMerger docMerger = new AtomicUpdateDocumentMerger(req());
    +    docMerger.merge(addedDoc, block);
    +    assertEquals("merged document should have the same id", preMergeDoc.getFieldValue("id"), block.getFieldValue("id"));
    +    assertDocContainsSubset(preMergeDoc, block);
    +    assertDocContainsSubset(addedDoc, block);
    +    assertDocContainsSubset(newChildDoc, (SolrInputDocument) ((List) block.getFieldValues("child")).get(1));
    +    assertEquals(doc.getFieldValue("id"), block.getFieldValue("id"));
    +  }
    +
    +  @Test
    +  public void testBlockAtomicAdd() throws Exception {
    +
    +    SolrInputDocument doc = sdoc("id", "1",
    +        "cat_ss", new String[] {"aaa", "ccc"},
    +        "child1", sdoc("id", "2", "cat_ss", "child")
    +    );
    +    json(doc);
    --- End diff --
    
    accidentally added this json line?


---

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


[GitHub] lucene-solr pull request #455: WIP: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r223244019
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java ---
    @@ -461,5 +466,33 @@ private static boolean isChildDoc(Object obj) {
         }
         return objValues.iterator().next() instanceof SolrDocumentBase;
       }
    +
    +  private void removeObj(Collection original, Object toRemove, String fieldName) {
    +    if(isChildDoc(toRemove)) {
    +      removeChildDoc(original, (SolrInputDocument) toRemove);
    +    } else {
    +      original.remove(getNativeFieldValue(fieldName, toRemove));
    +    }
    +  }
    +
    +  private static void removeChildDoc(Collection original, SolrInputDocument docToRemove) {
    +    for(SolrInputDocument doc: (Collection<SolrInputDocument>) original) {
    +      if(isDerivedFromDoc(doc, docToRemove)) {
    +        original.remove(doc);
    +        return;
    +      }
    +    }
    +  }
    +
    +  private static boolean isDerivedFromDoc(SolrInputDocument fullDoc, SolrInputDocument subDoc) {
    --- End diff --
    
    Was does it mean to be "derived" here?  Perhaps do you mean that subDoc is a partial update?  Javadocs would help.


---

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


[GitHub] lucene-solr issue #455: WIP: SOLR-12638

Posted by moshebla <gi...@git.apache.org>.
Github user moshebla commented on the issue:

    https://github.com/apache/lucene-solr/pull/455
  
    Currently Some Solr Core tests do not pass.
    I will work on those, since this is a WIP.
    The test that concerns me the most is ConvertedLegacyTest, which fails only if VersionInfo#lookupVersion is called.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r223749656
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java ---
    @@ -111,6 +114,8 @@
       public static final String DISTRIB_FROM = "distrib.from";
       public static final String DISTRIB_INPLACE_PREVVERSION = "distrib.inplace.prevversion";
       private static final String TEST_DISTRIB_SKIP_SERVERS = "test.distrib.skip.servers";
    +  private static final char PATH_SEP_CHAR = '/';
    --- End diff --
    
    Please don't create frivolous constants like this.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by moshebla <gi...@git.apache.org>.
Github user moshebla commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r237364758
  
    --- Diff: solr/core/src/test/org/apache/solr/cloud/AtomicUpdateBlockShardedTest.java ---
    @@ -130,7 +130,7 @@ public void doNestedInplaceUpdateTest() throws Exception {
         // for now,  we know how ranges will be distributed to shards.
         // may have to look it up in clusterstate if that assumption changes.
     
    -    SolrInputDocument doc = sdoc("id", "a", "level_s", "root", "_root_", "a");
    +    SolrInputDocument doc = sdoc("id", "a", "level_s", "root");
    --- End diff --
    
    @dsmiley, this is what I meant by "remove hard coded _root_.
    I added these since my logic depended on chidless documents having _root_.
    When SOLR-5211 was merged, I just rebased on master and removed these hardcoded _root_ fields, which are now automatically generated by Solr.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r231127727
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java ---
    @@ -639,12 +651,32 @@ public static SolrInputDocument getInputDocument(SolrCore core, BytesRef idBytes
               sid = new SolrInputDocument();
             } else {
               Document luceneDocument = docFetcher.doc(docid);
    -          sid = toSolrInputDocument(luceneDocument, core.getLatestSchema());
    +          sid = toSolrInputDocument(luceneDocument, schema);
             }
    -        if (onlyTheseNonStoredDVs != null) {
    -          docFetcher.decorateDocValueFields(sid, docid, onlyTheseNonStoredDVs);
    -        } else {
    -          docFetcher.decorateDocValueFields(sid, docid, docFetcher.getNonStoredDVsWithoutCopyTargets());
    +        ensureDocFieldsDecorated(onlyTheseNonStoredDVs, sid, docid, docFetcher, resolveRootDoc ||
    +            resolveChildren || schema.hasExplicitField(IndexSchema.NEST_PATH_FIELD_NAME));
    +        SolrInputField rootField = sid.getField(IndexSchema.ROOT_FIELD_NAME);
    +        if((resolveChildren || resolveRootDoc) && schema.isUsableForChildDocs() && rootField!=null) {
    +          // doc is part of a nested structure
    +          String id = resolveRootDoc? (String) rootField.getFirstValue(): (String) sid.getField(idField.getName()).getFirstValue();
    +          ModifiableSolrParams params = new ModifiableSolrParams()
    +              .set("fl", "*, _nest_path_, [child]")
    +              .set("limit", "-1");
    +          SolrQueryRequest nestedReq = new LocalSolrQueryRequest(core, params);
    +          final BytesRef rootIdBytes = new BytesRef(id);
    +          final int rootDocId = searcher.getFirstMatch(new Term(idField.getName(), rootIdBytes));
    +          final DocTransformer childDocTransformer = TransformerFactory.defaultFactories.get("child").create("child", params, nestedReq);
    --- End diff --
    
    no, use `core.getTransformerFactory("child")`


---

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


[GitHub] lucene-solr pull request #455: WIP: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r223561565
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java ---
    @@ -461,5 +466,33 @@ private static boolean isChildDoc(Object obj) {
         }
         return objValues.iterator().next() instanceof SolrDocumentBase;
       }
    +
    +  private void removeObj(Collection original, Object toRemove, String fieldName) {
    +    if(isChildDoc(toRemove)) {
    +      removeChildDoc(original, (SolrInputDocument) toRemove);
    +    } else {
    +      original.remove(getNativeFieldValue(fieldName, toRemove));
    +    }
    +  }
    +
    +  private static void removeChildDoc(Collection original, SolrInputDocument docToRemove) {
    +    for(SolrInputDocument doc: (Collection<SolrInputDocument>) original) {
    +      if(isDerivedFromDoc(doc, docToRemove)) {
    +        original.remove(doc);
    +        return;
    +      }
    +    }
    +  }
    +
    +  private static boolean isDerivedFromDoc(SolrInputDocument fullDoc, SolrInputDocument subDoc) {
    --- End diff --
    
    I see the new docs; still it's unclear without the context of our conversation.  Should "subDoc" be "partialDoc"?  How is it that the fullDoc is "the document to be tested" when the return value is "wether subDoc is a subset of fullDoc"?  Isn't the subject of the test subDoc?


---

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


[GitHub] lucene-solr pull request #455: WIP: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r218293921
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java ---
    @@ -1184,7 +1196,16 @@ protected boolean versionAdd(AddUpdateCommand cmd) throws IOException {
     
             // TODO: possibly set checkDeleteByQueries as a flag on the command?
             doLocalAdd(cmd);
    -        
    +
    +        if(lastKnownVersion != null && req.getSchema().isUsableForChildDocs() &&
    --- End diff --
    
    This may very well be right but can you tell me why you added this delete here at this line and why the delete is version-dependent?


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by moshebla <gi...@git.apache.org>.
Github user moshebla commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r228734114
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java ---
    @@ -609,9 +618,11 @@ public static SolrInputDocument getInputDocument(SolrCore core, BytesRef idBytes
        * @param resolveFullDocument In case the document is fetched from the tlog, it could only be a partial document if the last update
        *                  was an in-place update. In that case, should this partial document be resolved to a full document (by following
        *                  back prevPointer/prevVersion)?
    +   * @param resolveRootDoc Check whether the document is part of a nested hierarchy. If so, return the whole hierarchy.
    +   * @param resolveChildren Check whether the document has child documents. If so, return the document including its children.
        */
       public static SolrInputDocument getInputDocument(SolrCore core, BytesRef idBytes, AtomicLong versionReturned, boolean avoidRetrievingStoredFields,
    -      Set<String> onlyTheseNonStoredDVs, boolean resolveFullDocument) throws IOException {
    +      Set<String> onlyTheseNonStoredDVs, boolean resolveFullDocument, boolean resolveRootDoc, boolean resolveChildren) throws IOException {
    --- End diff --
    
    resolveRootDoc -> resolves the whole hierarchy from the root doc regardless of whether the docId provided was the root document.
    resolveChildren -> resolves the specified document's child documents, regardless of whether the document was is a root document or not.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r224127559
  
    --- Diff: solr/core/src/test/org/apache/solr/update/processor/AtomicUpdateBlockTest.java ---
    @@ -0,0 +1,370 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.solr.update.processor;
    +
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.List;
    +
    +import org.apache.lucene.util.BytesRef;
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.core.SolrCore;
    +import org.apache.solr.handler.component.RealTimeGetComponent;
    +import org.junit.AfterClass;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +public class AtomicUpdateBlockTest extends SolrTestCaseJ4 {
    +
    +  private final static String VERSION = "_version_";
    +  private static String PREVIOUS_ENABLE_UPDATE_LOG_VALUE;
    +
    +  @BeforeClass
    +  public static void beforeTests() throws Exception {
    +    PREVIOUS_ENABLE_UPDATE_LOG_VALUE = System.getProperty("enable.update.log");
    +    System.setProperty("enable.update.log", "true");
    --- End diff --
    
    The `solrconfig-block-atomic-update.xml` file is not toggled by this system property (perhaps others are).  Why set this system property?


---

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


[GitHub] lucene-solr issue #455: WIP: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on the issue:

    https://github.com/apache/lucene-solr/pull/455
  
    ConvertedLegacyTest fails in part because line 306 adds with overwrite=false -- a bit of a dubious Solr feature that is probably not properly compatible with the UpdateLog which makes various assumptions about the uniqueKey being unique.  I'll email the dev list to see what others think but I'm inclined to think overwrite=false ought to be explicitly forbidden with an UpdateLog in place.  That ancient legacy test can use a config that doesn't have an updateLog.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r224209409
  
    --- Diff: solr/core/src/test/org/apache/solr/update/processor/AtomicUpdateBlockTest.java ---
    @@ -0,0 +1,370 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.solr.update.processor;
    +
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.List;
    +
    +import org.apache.lucene.util.BytesRef;
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.core.SolrCore;
    +import org.apache.solr.handler.component.RealTimeGetComponent;
    +import org.junit.AfterClass;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Test;
    +
    +public class AtomicUpdateBlockTest extends SolrTestCaseJ4 {
    +
    +  private final static String VERSION = "_version_";
    +  private static String PREVIOUS_ENABLE_UPDATE_LOG_VALUE;
    +
    +  @BeforeClass
    +  public static void beforeTests() throws Exception {
    +    PREVIOUS_ENABLE_UPDATE_LOG_VALUE = System.getProperty("enable.update.log");
    +    System.setProperty("enable.update.log", "true");
    +    initCore("solrconfig-block-atomic-update.xml", "schema-nest.xml"); // use "nest" schema
    +  }
    +
    +  @AfterClass
    +  public static void afterTests() throws Exception {
    +    // restore enable.update.log
    +    System.setProperty("enable.update.log", PREVIOUS_ENABLE_UPDATE_LOG_VALUE);
    +  }
    +
    +  @Before
    +  public void before() {
    +    clearIndex();
    +    assertU(commit());
    +  }
    +
    +  @Test
    +  public void testMergeChildDoc() throws Exception {
    +    SolrInputDocument doc = new SolrInputDocument();
    +    doc.setField("id", "1");
    +    doc.setField("cat_ss", new String[]{"aaa", "ccc"});
    +    doc.setField("child", Collections.singletonList(sdoc("id", "2", "cat_ss", "child")));
    +    addDoc(adoc(doc), "nested-rtg");
    +
    +    BytesRef rootDocId = new BytesRef("1");
    +    SolrCore core = h.getCore();
    +    SolrInputDocument block = RealTimeGetComponent.getInputDocument(core, rootDocId, true);
    +    // assert block doc has child docs
    +    assertTrue(block.containsKey("child"));
    +
    +    assertJQ(req("q","id:1")
    +        ,"/response/numFound==0"
    +    );
    +
    +    // commit the changes
    +    assertU(commit());
    +
    +    SolrInputDocument newChildDoc = sdoc("id", "3", "cat_ss", "child");
    +    SolrInputDocument addedDoc = sdoc("id", "1",
    +        "cat_ss", Collections.singletonMap("add", "bbb"),
    +        "child", Collections.singletonMap("add", sdocs(newChildDoc)));
    +    block = RealTimeGetComponent.getInputDocument(core, rootDocId, true);
    +    block.removeField(VERSION);
    +    SolrInputDocument preMergeDoc = new SolrInputDocument(block);
    +    AtomicUpdateDocumentMerger docMerger = new AtomicUpdateDocumentMerger(req());
    +    docMerger.merge(addedDoc, block);
    +    assertEquals("merged document should have the same id", preMergeDoc.getFieldValue("id"), block.getFieldValue("id"));
    +    assertDocContainsSubset(preMergeDoc, block);
    +    assertDocContainsSubset(addedDoc, block);
    +    assertDocContainsSubset(newChildDoc, (SolrInputDocument) ((List) block.getFieldValues("child")).get(1));
    +    assertEquals(doc.getFieldValue("id"), block.getFieldValue("id"));
    +  }
    +
    +  @Test
    +  public void testBlockAtomicAdd() throws Exception {
    +
    +    SolrInputDocument doc = sdoc("id", "1",
    +        "cat_ss", new String[] {"aaa", "ccc"},
    +        "child1", sdoc("id", "2", "cat_ss", "child")
    +    );
    +    json(doc);
    +    addDoc(adoc(doc), "nested-rtg");
    +
    +    BytesRef rootDocId = new BytesRef("1");
    +    SolrCore core = h.getCore();
    +    SolrInputDocument block = RealTimeGetComponent.getInputDocument(core, rootDocId, true);
    +    // assert block doc has child docs
    +    assertTrue(block.containsKey("child1"));
    +
    +    assertJQ(req("q","id:1")
    +        ,"/response/numFound==0"
    +    );
    +
    +    // commit the changes
    +    assertU(commit());
    +
    +    SolrInputDocument committedBlock = RealTimeGetComponent.getInputDocument(core, rootDocId, true);
    +    BytesRef childDocId = new BytesRef("2");
    +    // ensure the whole block is returned when resolveBlock is true and id of a child doc is provided
    +    assertEquals(committedBlock.toString(), RealTimeGetComponent.getInputDocument(core, childDocId, true).toString());
    --- End diff --
    
    Can we avoid the low-level API use and instead stay at the top level (SolrJ) / integration test?  More specifically, can we avoid access to RealTimeGetComponent completely from this test method and the others here?  It's _appears_ we wouldn't lose coverage.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r226709714
  
    --- Diff: solr/core/src/test/org/apache/solr/update/processor/AtomicUpdateBlockTest.java ---
    @@ -59,39 +52,147 @@ public void before() {
     
       @Test
       public void testMergeChildDoc() throws Exception {
    -    SolrInputDocument doc = new SolrInputDocument();
    -    doc.setField("id", "1");
    -    doc.setField("cat_ss", new String[]{"aaa", "ccc"});
    -    doc.setField("child", Collections.singletonList(sdoc("id", "2", "cat_ss", "child")));
    +    SolrInputDocument newChildDoc = sdoc("id", "3", "cat_ss", "child");
    +    SolrInputDocument addedDoc = sdoc("id", "1",
    +        "cat_ss", Collections.singletonMap("add", "bbb"),
    +        "child", Collections.singletonMap("add", sdocs(newChildDoc)));
    +
    +    SolrInputDocument dummyBlock = sdoc("id", "1",
    +        "cat_ss", new ArrayList<>(Arrays.asList("aaa", "ccc")),
    +        "_root_", "1", "child", new ArrayList<>(sdocs(addedDoc)));
    +    dummyBlock.removeField(VERSION);
    +
    +    SolrInputDocument preMergeDoc = new SolrInputDocument(dummyBlock);
    +    AtomicUpdateDocumentMerger docMerger = new AtomicUpdateDocumentMerger(req());
    +    docMerger.merge(addedDoc, dummyBlock);
    +    assertEquals("merged document should have the same id", preMergeDoc.getFieldValue("id"), dummyBlock.getFieldValue("id"));
    +    assertDocContainsSubset(preMergeDoc, dummyBlock);
    +    assertDocContainsSubset(addedDoc, dummyBlock);
    +    assertDocContainsSubset(newChildDoc, (SolrInputDocument) ((List) dummyBlock.getFieldValues("child")).get(1));
    +    assertEquals(dummyBlock.getFieldValue("id"), dummyBlock.getFieldValue("id"));
    +  }
    +
    +  @Test
    +  public void testBlockAtomicQuantities() throws Exception {
    +    SolrInputDocument doc = sdoc("id", "1", "string_s", "root");
         addDoc(adoc(doc), "nested-rtg");
     
    -    BytesRef rootDocId = new BytesRef("1");
    -    SolrCore core = h.getCore();
    -    SolrInputDocument block = RealTimeGetComponent.getInputDocument(core, rootDocId, true);
    -    // assert block doc has child docs
    -    assertTrue(block.containsKey("child"));
    +    List<SolrInputDocument> docs = IntStream.range(10, 20).mapToObj(x -> sdoc("id", String.valueOf(x), "string_s", "child")).collect(Collectors.toList());
    +    doc = sdoc("id", "1", "children", Collections.singletonMap("add", docs));
    +    addAndGetVersion(doc, params("update.chain", "nested-rtg", "wt", "json"));
     
    -    assertJQ(req("q","id:1")
    -        ,"/response/numFound==0"
    +    assertU(commit());
    +
    +    assertJQ(req("q", "_root_:1"),
    +        "/response/numFound==11");
    +
    +    assertJQ(req("q", "string_s:child", "fl", "*", "rows", "1000000"),
    +        "/response/numFound==10");
    +
    +    // ensure updates work when block has more than 10 children
    +    for(int i = 10; i < 20; ++i) {
    +      System.out.println("indexing " + i);
    +      docs = IntStream.range(i * 10, (i * 10) + 5).mapToObj(x -> sdoc("id", String.valueOf(x), "string_s", "grandChild")).collect(Collectors.toList());
    +      doc = sdoc("id", String.valueOf(i), "grandChildren", Collections.singletonMap("add", docs));
    +      addAndGetVersion(doc, params("update.chain", "nested-rtg", "wt", "json"));
    +      assertU(commit());
    +    }
    +
    +    assertJQ(req("q", "id:114", "fl", "*", "rows", "1000000"),
    --- End diff --
    
    Why set the "fl" or "rows" in these queries?  Your assertion only checks numFound and not the content of those that were found.


---

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


[GitHub] lucene-solr pull request #455: WIP: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r223244186
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java ---
    @@ -461,5 +466,33 @@ private static boolean isChildDoc(Object obj) {
         }
         return objValues.iterator().next() instanceof SolrDocumentBase;
       }
    +
    +  private void removeObj(Collection original, Object toRemove, String fieldName) {
    +    if(isChildDoc(toRemove)) {
    +      removeChildDoc(original, (SolrInputDocument) toRemove);
    +    } else {
    +      original.remove(getNativeFieldValue(fieldName, toRemove));
    +    }
    +  }
    +
    +  private static void removeChildDoc(Collection original, SolrInputDocument docToRemove) {
    +    for(SolrInputDocument doc: (Collection<SolrInputDocument>) original) {
    +      if(isDerivedFromDoc(doc, docToRemove)) {
    +        original.remove(doc);
    +        return;
    +      }
    +    }
    +  }
    +
    +  private static boolean isDerivedFromDoc(SolrInputDocument fullDoc, SolrInputDocument subDoc) {
    +    for(SolrInputField subSif: subDoc) {
    +      String fieldName = subSif.getName();
    +      if(!fullDoc.containsKey(fieldName)) return false;
    +      Collection<Object> fieldValues = subDoc.getFieldValues(fieldName);
    --- End diff --
    
    Can't we get this directly off the subSif via subSif.getValues()?  And why would subSif.getValueCount ever be different than fieldValues.size()?  Comments would help.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r224283996
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java ---
    @@ -609,9 +618,10 @@ public static SolrInputDocument getInputDocument(SolrCore core, BytesRef idBytes
        * @param resolveFullDocument In case the document is fetched from the tlog, it could only be a partial document if the last update
        *                  was an in-place update. In that case, should this partial document be resolved to a full document (by following
        *                  back prevPointer/prevVersion)?
    +   * @param resolveBlock Check whether the document is part of a block. If so, return the whole block.
        */
       public static SolrInputDocument getInputDocument(SolrCore core, BytesRef idBytes, AtomicLong versionReturned, boolean avoidRetrievingStoredFields,
    -      Set<String> onlyTheseNonStoredDVs, boolean resolveFullDocument) throws IOException {
    +      Set<String> onlyTheseNonStoredDVs, boolean resolveFullDocument, boolean resolveBlock) throws IOException {
         SolrInputDocument sid = null;
    --- End diff --
    
    It would be helpful to add a javadoc comment to say that if the id refers to a nested document (which isn't known up-front), then it'll never be found in the tlog (at least if you follow the rules of nested docs).  Also, perhaps the parameter "resolveBlock" should be "resolveToRootDocument" since I think the "root" terminology may be more widely used as it's even in the schema, whereas "block" is I think not so much.  If you disagree, a compromise may be to use both "root" and "Block" together -- "resolveRootBlock".


---

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


[GitHub] lucene-solr pull request #455: WIP: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r223243871
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/AtomicUpdateDocumentMerger.java ---
    @@ -461,5 +466,33 @@ private static boolean isChildDoc(Object obj) {
         }
         return objValues.iterator().next() instanceof SolrDocumentBase;
       }
    +
    +  private void removeObj(Collection original, Object toRemove, String fieldName) {
    +    if(isChildDoc(toRemove)) {
    +      removeChildDoc(original, (SolrInputDocument) toRemove);
    +    } else {
    +      original.remove(getNativeFieldValue(fieldName, toRemove));
    +    }
    +  }
    +
    +  private static void removeChildDoc(Collection original, SolrInputDocument docToRemove) {
    +    for(SolrInputDocument doc: (Collection<SolrInputDocument>) original) {
    +      if(isDerivedFromDoc(doc, docToRemove)) {
    +        original.remove(doc);
    --- End diff --
    
    eh... I'm not sure if it's as simple as `original.remove(doc)` since if it was, we wouldn't need the method `isDerivedFromDoc`.  Assuming we do need that "derived" predicate method, then we probably need to call remove on an iterator here.


---

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


[GitHub] lucene-solr issue #455: SOLR-12638

Posted by moshebla <gi...@git.apache.org>.
Github user moshebla commented on the issue:

    https://github.com/apache/lucene-solr/pull/455
  
    I rebased using the latest master.
    I also changed the tests so _root_ is not hard-coded into childless documents, since SOLR-5211 was committed to master and ensures this behaviour.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r228026284
  
    --- Diff: solr/core/src/java/org/apache/solr/update/AddUpdateCommand.java ---
    @@ -262,6 +263,11 @@ private void flattenAnonymous(List<SolrInputDocument> unwrappedDocs, SolrInputDo
         flattenAnonymous(unwrappedDocs, currentDoc, false);
       }
     
    +  public String getRouteFieldVal() {
    --- End diff --
    
    I'm skeptical of this method.  It's name seems innocent enough looking at the code here.  But then also consider some collections have a "router.field" and this method is named in such a way that one would think this method returns that field's value... yet it does not.  Some callers put this into a variable named "id" or similar.  Given that, I propose you remove it but incorporate the logic into getHashableId which seems the proper place for it.  It _is_ the hashable Id... the hashable ID of a nested doc is it's root.
    
    But... I do also wonder if we need this at all.  Somewhere Solr already has code that looks at \_route\_ and acts on that if present.  Perhaps the code path for an atomic update isn't doing this yet but should do it?  Then we wouldn't need this change to AddUpdateCommand.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r228197896
  
    --- Diff: solr/core/src/java/org/apache/solr/handler/component/RealTimeGetComponent.java ---
    @@ -609,9 +618,10 @@ public static SolrInputDocument getInputDocument(SolrCore core, BytesRef idBytes
        * @param resolveFullDocument In case the document is fetched from the tlog, it could only be a partial document if the last update
        *                  was an in-place update. In that case, should this partial document be resolved to a full document (by following
        *                  back prevPointer/prevVersion)?
    +   * @param resolveBlock Check whether the document is part of a block. If so, return the whole block.
        */
       public static SolrInputDocument getInputDocument(SolrCore core, BytesRef idBytes, AtomicLong versionReturned, boolean avoidRetrievingStoredFields,
    -      Set<String> onlyTheseNonStoredDVs, boolean resolveFullDocument) throws IOException {
    +      Set<String> onlyTheseNonStoredDVs, boolean resolveFullDocument, boolean resolveBlock) throws IOException {
         SolrInputDocument sid = null;
    --- End diff --
    
    @moshebla I'm concerned about this method having so many parameters... it's a code smell.
    resolveBlock & resolveChildren booleans... not sure how they differ.  Isn't resolveChildren enough?  if resolveChildren true, then isn't effectively resolveFullDocument also true?  (if so the constraint could be documented and enforced with an assertion).  
    
    versionReturned is a dubious parameter I'm skeptical needs to be here.  I'm aware you didn't add it, but the number of parameters is getting troublingly long with your changes; it's good to review what's needed.


---

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


[GitHub] lucene-solr pull request #455: SOLR-12638

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r224306892
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java ---
    @@ -1360,24 +1385,47 @@ boolean getUpdatedDocument(AddUpdateCommand cmd, long versionOnUpdate) throws IO
         }
         
         // full (non-inplace) atomic update
    +    final boolean isNestedSchema = req.getSchema().isUsableForChildDocs();
         SolrInputDocument sdoc = cmd.getSolrInputDocument();
         BytesRef id = cmd.getIndexedId();
    -    SolrInputDocument oldDoc = RealTimeGetComponent.getInputDocument(cmd.getReq().getCore(), id);
    +    SolrInputDocument blockDoc = RealTimeGetComponent.getInputDocument(cmd.getReq().getCore(), id, null,
    +        false, null, true, true);
     
    -    if (oldDoc == null) {
    -      // create a new doc by default if an old one wasn't found
    -      if (versionOnUpdate <= 0) {
    -        oldDoc = new SolrInputDocument();
    -      } else {
    +    if (blockDoc == null) {
    +      if (versionOnUpdate > 0) {
             // could just let the optimistic locking throw the error
             throw new SolrException(ErrorCode.CONFLICT, "Document not found for update.  id=" + cmd.getPrintableId());
           }
         } else {
    -      oldDoc.remove(CommonParams.VERSION_FIELD);
    +      blockDoc.remove(CommonParams.VERSION_FIELD);
         }
     
     
    -    cmd.solrDoc = docMerger.merge(sdoc, oldDoc);
    +    SolrInputDocument mergedDoc;
    +    if(idField == null || blockDoc == null) {
    +      // create a new doc by default if an old one wasn't found
    +      mergedDoc = docMerger.merge(sdoc, new SolrInputDocument());
    +    } else {
    +      if(isNestedSchema && req.getSchema().hasExplicitField(IndexSchema.NEST_PATH_FIELD_NAME) &&
    +          blockDoc.containsKey(IndexSchema.ROOT_FIELD_NAME) && !id.utf8ToString().equals(blockDoc.getFieldValue(IndexSchema.ROOT_FIELD_NAME))) {
    --- End diff --
    
    I don't think we can assume id.utf8ToString() is correct.  I think we have to consult the corresponding FieldType to get the "external value".  Also, cast blockDoc.getFieldValue as a String to make it clear we expected it to be one.


---

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


[GitHub] lucene-solr pull request #455: WIP: SOLR-12638

Posted by moshebla <gi...@git.apache.org>.
Github user moshebla commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/455#discussion_r219034542
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java ---
    @@ -1184,7 +1196,16 @@ protected boolean versionAdd(AddUpdateCommand cmd) throws IOException {
     
             // TODO: possibly set checkDeleteByQueries as a flag on the command?
             doLocalAdd(cmd);
    -        
    +
    +        if(lastKnownVersion != null && req.getSchema().isUsableForChildDocs() &&
    --- End diff --
    
    This delete ensures the updated block is the latest block, deleting all previous ones.


---

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