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/06/25 08:43:18 UTC

[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

GitHub user moshebla opened a pull request:

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

    SOLR-12441: add deeply nested URP for nested documents metadata

    

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

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

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

    https://github.com/apache/lucene-solr/pull/410.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 #410
    
----
commit 6da7b4b2447f2dd20dbca031974e79f69189f953
Author: user <us...@...>
Date:   2018-06-05T08:58:10Z

    first tests

commit 6ba4debda4e77198330a5e157c85564cdd8981dd
Author: user <us...@...>
Date:   2018-06-06T00:31:47Z

    only index fields from conf

commit 15deec89f7b6633b4655aecc2e061964b4ffce84
Author: Moshe <mo...@...>
Date:   2018-06-06T13:36:31Z

    SOLR-12441: tests with query

commit 76dd0d7155c94b8c6397032a25cf933abe4579e2
Author: Moshe <mo...@...>
Date:   2018-06-11T07:54:56Z

    SOLR-12441: use EnumSet for conf

----


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r200230960
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessorFactory.java ---
    @@ -0,0 +1,120 @@
    +/*
    + * 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.io.IOException;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessorFactory extends UpdateRequestProcessorFactory {
    +
    +  public UpdateRequestProcessor getInstance(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next ) {
    +    boolean storeParent = shouldStoreDocParent(req.getSchema());
    +    boolean storePath = shouldStoreDocPath(req.getSchema());
    +    if(!(storeParent || storePath)) {
    +      return next;
    +    }
    +    return new NestedUpdateProcessor(req, rsp, shouldStoreDocParent(req.getSchema()), shouldStoreDocPath(req.getSchema()), next);
    +  }
    +
    +  private static boolean shouldStoreDocParent(IndexSchema schema) {
    +    return schema.getFields().containsKey(IndexSchema.PARENT_FIELD_NAME);
    +  }
    +
    +  private static boolean shouldStoreDocPath(IndexSchema schema) {
    +    return schema.getFields().containsKey(IndexSchema.PATH_FIELD_NAME);
    +  }
    +}
    +
    +class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  private static final String PATH_SEP_CHAR = "/";
    +  private boolean storePath;
    +  private boolean storeParent;
    +  private String uniqueKeyFieldName;
    +
    +
    +  protected NestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, boolean storeParent, boolean storePath, UpdateRequestProcessor next) {
    +    super(next);
    +    this.storeParent = storeParent;
    +    this.storePath = storePath;
    +    this.uniqueKeyFieldName = req.getSchema().getUniqueKeyField().getName();
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    processDocChildren(doc, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String fullPath) {
    +    int childNum = 0;
    +    for(SolrInputField field: doc.values()) {
    +      for(Object val: field) {
    +        if(!(val instanceof SolrInputDocument)) {
    +          // either all collection items are child docs or none are.
    +          break;
    +        }
    +        final String fieldName = field.getName();
    +
    +        if(fieldName.contains(PATH_SEP_CHAR)) {
    +          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field name: '" + fieldName
    +              + "' contains: '" + PATH_SEP_CHAR + "' , which is reserved for the nested URP");
    +        }
    +        final String jointPath = fullPath == null ? fieldName: fullPath + PATH_SEP_CHAR + fieldName;
    +        SolrInputDocument cDoc = (SolrInputDocument) val;
    +        if(!cDoc.containsKey(uniqueKeyFieldName)) {
    +          String parentDocId = doc.getField(uniqueKeyFieldName).getFirstValue().toString();
    +          cDoc.setField(uniqueKeyFieldName, generateChildUniqueId(parentDocId, fieldName, childNum));
    +        }
    +        processChildDoc((SolrInputDocument) val, doc, jointPath);
    +        ++childNum;
    +      }
    +    }
    +  }
    +
    +  private void processChildDoc(SolrInputDocument sdoc, SolrInputDocument parent, String fullPath) {
    +    if(storePath) {
    +      setPathField(sdoc, fullPath);
    +    }
    +    if (storeParent) {
    +      setParentKey(sdoc, parent);
    +    }
    +    processDocChildren(sdoc, fullPath);
    +  }
    +
    +  private String generateChildUniqueId(String parentId, String childKey, int childNum) {
    --- End diff --
    
    It occurred to me that this feature only occurs for labelled child docs, not anonymous ones.  And furthermore only if these two special fields are in the schema.  We'll have to remember to document that in the ref guide when we get to that point.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r197868389
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/DeeplyNestedUpdateProcessor.java ---
    @@ -0,0 +1,101 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Collection;
    +import java.util.EnumSet;
    +import java.util.Locale;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +import static org.apache.solr.update.processor.DeeplyNestedUpdateProcessorFactory.NestedFlag;
    +
    +public class DeeplyNestedUpdateProcessor extends UpdateRequestProcessor {
    --- End diff --
    
    I wonder if we want to call this as-such.  Can this be used if the nesting is "shallow"?  Perhaps remove the "Deeply" part.  Perhaps this URP should handle "_root_" as well to thus unify where nested child doc fields are added?  Although that raises other questions.  With 8.0 on the horizon this fall, we can think of the best solution that isn't necessarily fully backwards compatible.  This conversation should probably be taken to the JIRA as it's a bigger question and not a code detail.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r200871822
  
    --- Diff: solr/core/src/test/org/apache/solr/update/TestNestedUpdateProcessor.java ---
    @@ -0,0 +1,171 @@
    +/*
    + * 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;
    +
    +import java.util.List;
    +
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.processor.NestedUpdateProcessorFactory;
    +import org.apache.solr.update.processor.UpdateRequestProcessor;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Rule;
    +import org.junit.Test;
    +import org.junit.rules.ExpectedException;
    +
    +public class TestNestedUpdateProcessor extends SolrTestCaseJ4 {
    +
    +  private static final char PATH_SEP_CHAR = '/';
    +  private static final String[] childrenIds = { "2", "3" };
    +  private static final String grandChildId = "4";
    +  private static final String jDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String noIdChildren = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String errDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children" + PATH_SEP_CHAR + "a\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  @Rule
    +  public ExpectedException thrown = ExpectedException.none();
    +
    +  @BeforeClass
    +  public static void beforeClass() throws Exception {
    +    initCore("solrconfig-update-processor-chains.xml", "schema15.xml");
    +  }
    +
    +  @Before
    +  public void before() throws Exception {
    +    assertU(delQ("*:*"));
    +    assertU(commit());
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPGrandChild() throws Exception {
    +    indexSampleData(jDoc);
    +
    +    assertJQ(req("q", IndexSchema.PATH_FIELD_NAME + ":*" + PATH_SEP_CHAR + "grandChild",
    +        "fl","*",
    +        "sort","id desc",
    +        "wt","json"),
    +        "/response/docs/[0]/id=='" + grandChildId + "'");
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPChildren() throws Exception {
    +    final String[] childrenTests = {"/response/docs/[0]/id=='" + childrenIds[0] + "'", "/response/docs/[1]/id=='" + childrenIds[1] + "'"};
    +    indexSampleData(jDoc);
    +
    +    assertJQ(req("q", IndexSchema.PATH_FIELD_NAME + ":children",
    +        "fl","*",
    +        "sort","id asc",
    +        "wt","json"),
    +        childrenTests);
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPChildrenWoId() throws Exception {
    +    final String parentId = "3";
    +    final String childKey = "grandChild";
    +    final String expectedId = parentId + PATH_SEP_CHAR + childKey + PATH_SEP_CHAR + "0";
    +    SolrInputDocument noIdChildren = sdoc("id", "1", "children", sdocs(sdoc("id", "2", "name_s", "Yaz"), sdoc("id", parentId, "name_s", "Jazz", childKey, sdoc("names_s", "Gaz"))));
    +    UpdateRequestProcessor nestedUpdate = new NestedUpdateProcessorFactory().getInstance(req(), null, null);
    +    AddUpdateCommand cmd = new AddUpdateCommand(req());
    +    cmd.solrDoc = noIdChildren;
    +    nestedUpdate.processAdd(cmd);
    +    cmd.clear();
    +    List children = (List) noIdChildren.get("children").getValues();
    +    SolrInputDocument idLessChild = (SolrInputDocument)((SolrInputDocument) children.get(1)).get(childKey).getValue();
    +    assertTrue("Id less child did not get an Id", idLessChild.containsKey("id"));
    --- End diff --
    
    come to think of it, in this whole test file I don't see a test for what the `_NEST_PATH_` field is supposed to look like?  That's kinda important!  If you modify this to test via doc.toString it'd cover it.  Granted `testDeeplyNestedURPChildren()` or some new method that ought to test this as well.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

Posted by moshebla <gi...@git.apache.org>.
Github user moshebla closed the pull request at:

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


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r199383354
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessor.java ---
    @@ -0,0 +1,96 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  public static final String PATH_SEP_CHAR = "/";
    +  private boolean storePath;
    +  private boolean storeParent;
    +  SolrQueryRequest req;
    +
    +
    +  protected NestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, boolean storeParent, boolean storePath, UpdateRequestProcessor next) {
    +    super(next);
    +    this.req = req;
    +    this.storeParent = storeParent;
    +    this.storePath = storePath;
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    String rootId = doc.getField(req.getSchema().getUniqueKeyField().getName()).getFirstValue().toString();
    +    processDocChildren(doc, rootId, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String rootId, String fullPath) {
    +    int childNum = 0;
    +    for(SolrInputField field: doc.values()) {
    +      for(Object val: field) {
    +        if(val instanceof SolrInputDocument) {
    +          if(field.getName().contains(PATH_SEP_CHAR)) {
    +            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field name: '" + field.getName()
    +                + "' contains: '" + PATH_SEP_CHAR + "' , which is reserved for the nested URP");
    +          }
    +          final String jointPath = Objects.isNull(fullPath) ? field.getName(): String.join(PATH_SEP_CHAR, fullPath, field.getName());
    --- End diff --
    
    And do we really need to call join() with exactly 2 args?  It's a roundabout way of something simpler.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r198150106
  
    --- Diff: solr/core/src/test/org/apache/solr/update/TestNestedUpdateProcessor.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.util.ContentStream;
    +import org.apache.solr.common.util.ContentStreamBase;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.request.SolrRequestInfo;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.servlet.SolrRequestParsers;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Rule;
    +import org.junit.Test;
    +import org.junit.rules.ExpectedException;
    +
    +import static org.apache.solr.update.processor.NestedUpdateProcessor.splitChar;
    --- End diff --
    
    After Yonik gave some sage advice on this topic once, I now think tests ought not to refer to constants in the tested code.  This way in the future if we change our minds on what those constants refer to, we're then realizing we may break back-compat.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r197869098
  
    --- Diff: solr/core/src/java/org/apache/solr/schema/IndexSchema.java ---
    @@ -104,9 +104,12 @@
       public static final String FIELD_TYPE = "fieldType";
       public static final String FIELD_TYPES = FIELD_TYPE + "s";
       public static final String INTERNAL_POLY_FIELD_PREFIX = "*" + FieldType.POLY_FIELD_SEPARATOR;
    +  public static final String LEVEL_FIELD_NAME = "_nestLevel_";
    --- End diff --
    
    I think these should nest ones should start with "NEST_"


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r201059845
  
    --- Diff: solr/core/src/test/org/apache/solr/update/TestNestedUpdateProcessor.java ---
    @@ -0,0 +1,171 @@
    +/*
    + * 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;
    +
    +import java.util.List;
    +
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.processor.NestedUpdateProcessorFactory;
    +import org.apache.solr.update.processor.UpdateRequestProcessor;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Rule;
    +import org.junit.Test;
    +import org.junit.rules.ExpectedException;
    +
    +public class TestNestedUpdateProcessor extends SolrTestCaseJ4 {
    +
    +  private static final char PATH_SEP_CHAR = '/';
    +  private static final String[] childrenIds = { "2", "3" };
    +  private static final String grandChildId = "4";
    +  private static final String jDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String noIdChildren = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String errDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children" + PATH_SEP_CHAR + "a\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  @Rule
    +  public ExpectedException thrown = ExpectedException.none();
    +
    +  @BeforeClass
    +  public static void beforeClass() throws Exception {
    +    initCore("solrconfig-update-processor-chains.xml", "schema15.xml");
    +  }
    +
    +  @Before
    +  public void before() throws Exception {
    +    assertU(delQ("*:*"));
    +    assertU(commit());
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPGrandChild() throws Exception {
    +    indexSampleData(jDoc);
    +
    +    assertJQ(req("q", IndexSchema.PATH_FIELD_NAME + ":*" + PATH_SEP_CHAR + "grandChild",
    +        "fl","*",
    +        "sort","id desc",
    +        "wt","json"),
    +        "/response/docs/[0]/id=='" + grandChildId + "'");
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPChildren() throws Exception {
    +    final String[] childrenTests = {"/response/docs/[0]/id=='" + childrenIds[0] + "'", "/response/docs/[1]/id=='" + childrenIds[1] + "'"};
    +    indexSampleData(jDoc);
    +
    +    assertJQ(req("q", IndexSchema.PATH_FIELD_NAME + ":children",
    +        "fl","*",
    +        "sort","id asc",
    +        "wt","json"),
    +        childrenTests);
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPChildrenWoId() throws Exception {
    +    final String parentId = "3";
    +    final String childKey = "grandChild";
    +    final String expectedId = parentId + PATH_SEP_CHAR + childKey + PATH_SEP_CHAR + "0";
    +    SolrInputDocument noIdChildren = sdoc("id", "1", "children", sdocs(sdoc("id", "2", "name_s", "Yaz"), sdoc("id", parentId, "name_s", "Jazz", childKey, sdoc("names_s", "Gaz"))));
    +    UpdateRequestProcessor nestedUpdate = new NestedUpdateProcessorFactory().getInstance(req(), null, null);
    +    AddUpdateCommand cmd = new AddUpdateCommand(req());
    +    cmd.solrDoc = noIdChildren;
    +    nestedUpdate.processAdd(cmd);
    +    cmd.clear();
    +    List children = (List) noIdChildren.get("children").getValues();
    +    SolrInputDocument idLessChild = (SolrInputDocument)((SolrInputDocument) children.get(1)).get(childKey).getValue();
    +    assertTrue("Id less child did not get an Id", idLessChild.containsKey("id"));
    --- End diff --
    
    I have just added a new commit with tests to ensure 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 #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r199383693
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessor.java ---
    @@ -0,0 +1,96 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  public static final String PATH_SEP_CHAR = "/";
    +  private boolean storePath;
    +  private boolean storeParent;
    +  SolrQueryRequest req;
    +
    +
    +  protected NestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, boolean storeParent, boolean storePath, UpdateRequestProcessor next) {
    +    super(next);
    +    this.req = req;
    +    this.storeParent = storeParent;
    +    this.storePath = storePath;
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    String rootId = doc.getField(req.getSchema().getUniqueKeyField().getName()).getFirstValue().toString();
    +    processDocChildren(doc, rootId, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String rootId, String fullPath) {
    +    int childNum = 0;
    +    for(SolrInputField field: doc.values()) {
    +      for(Object val: field) {
    +        if(val instanceof SolrInputDocument) {
    +          if(field.getName().contains(PATH_SEP_CHAR)) {
    +            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field name: '" + field.getName()
    +                + "' contains: '" + PATH_SEP_CHAR + "' , which is reserved for the nested URP");
    +          }
    +          final String jointPath = Objects.isNull(fullPath) ? field.getName(): String.join(PATH_SEP_CHAR, fullPath, field.getName());
    +          SolrInputDocument cDoc = (SolrInputDocument) val;
    +          if(!cDoc.containsKey(req.getSchema().getUniqueKeyField().getName())) {
    +            cDoc.setField(req.getSchema().getUniqueKeyField().getName(), generateChildUniqueId(rootId, jointPath, childNum));
    +          }
    +          processChildDoc((SolrInputDocument) val, doc, rootId, jointPath);
    +        }
    +        ++childNum;
    +      }
    --- End diff --
    
    else break the loop?  Either they all will be a child doc or none will be.  It'd be a shame to go looping on some many-multiValue field of integers wasting time.  Alternatively and simpler I think would be to grab only the first value before doing value looping and check the instance of that; then we needn't check all the values in the loop; we'd just cast.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r199383124
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessor.java ---
    @@ -0,0 +1,96 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  public static final String PATH_SEP_CHAR = "/";
    +  private boolean storePath;
    +  private boolean storeParent;
    +  SolrQueryRequest req;
    +
    +
    +  protected NestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, boolean storeParent, boolean storePath, UpdateRequestProcessor next) {
    +    super(next);
    +    this.req = req;
    +    this.storeParent = storeParent;
    +    this.storePath = storePath;
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    String rootId = doc.getField(req.getSchema().getUniqueKeyField().getName()).getFirstValue().toString();
    --- End diff --
    
    I've seen `req.getSchema().getUniqueKeyField().getName()` 3+ times in this class; lets put it on a field.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r200373726
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessorFactory.java ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Collection;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessorFactory extends UpdateRequestProcessorFactory {
    +
    +  public UpdateRequestProcessor getInstance(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next ) {
    +    boolean storeParent = shouldStoreDocParent(req.getSchema());
    +    boolean storePath = shouldStoreDocPath(req.getSchema());
    +    if(!(storeParent || storePath)) {
    +      return next;
    +    }
    +    return new NestedUpdateProcessor(req, rsp, shouldStoreDocParent(req.getSchema()), shouldStoreDocPath(req.getSchema()), next);
    +  }
    +
    +  private static boolean shouldStoreDocParent(IndexSchema schema) {
    +    return schema.getFields().containsKey(IndexSchema.NEST_PARENT_FIELD_NAME);
    +  }
    +
    +  private static boolean shouldStoreDocPath(IndexSchema schema) {
    +    return schema.getFields().containsKey(IndexSchema.NEST_PATH_FIELD_NAME);
    +  }
    +}
    +
    +class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  private static final String PATH_SEP_CHAR = "/";
    +  private static final String NUM_SEP_CHAR = ",";
    +  private static final String SINGLE_VALUE_CHAR = "s";
    +  private boolean storePath;
    +  private boolean storeParent;
    +  private String uniqueKeyFieldName;
    +
    +
    +  protected NestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, boolean storeParent, boolean storePath, UpdateRequestProcessor next) {
    +    super(next);
    +    this.storeParent = storeParent;
    +    this.storePath = storePath;
    +    this.uniqueKeyFieldName = req.getSchema().getUniqueKeyField().getName();
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    processDocChildren(doc, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String fullPath) {
    +    int childNum = 0;
    +    for(SolrInputField field: doc.values()) {
    +      boolean isSingleVal = !(field.getValue() instanceof Collection);
    +      for(Object val: field) {
    +        if(!(val instanceof SolrInputDocument)) {
    +          // either all collection items are child docs or none are.
    +          break;
    +        }
    +        final String fieldName = field.getName();
    +
    +        if(fieldName.contains(PATH_SEP_CHAR)) {
    +          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field name: '" + fieldName
    +              + "' contains: '" + PATH_SEP_CHAR + "' , which is reserved for the nested URP");
    +        }
    +        final String sChildNum = isSingleVal ? SINGLE_VALUE_CHAR: String.valueOf(childNum);
    +        final String lastPath = fieldName + NUM_SEP_CHAR + sChildNum + NUM_SEP_CHAR;
    +        final String jointPath = fullPath == null ? lastPath : fullPath + PATH_SEP_CHAR + lastPath;
    +        SolrInputDocument cDoc = (SolrInputDocument) val;
    +        if(!cDoc.containsKey(uniqueKeyFieldName)) {
    +          String parentDocId = doc.getField(uniqueKeyFieldName).getFirstValue().toString();
    +          cDoc.setField(uniqueKeyFieldName, generateChildUniqueId(parentDocId, fieldName, sChildNum));
    +        }
    +        processChildDoc((SolrInputDocument) val, doc, jointPath);
    +        ++childNum;
    +      }
    +    }
    +  }
    +
    +  private void processChildDoc(SolrInputDocument sdoc, SolrInputDocument parent, String fullPath) {
    +    if(storePath) {
    +      setPathField(sdoc, fullPath);
    +    }
    +    if (storeParent) {
    +      setParentKey(sdoc, parent);
    +    }
    +    processDocChildren(sdoc, fullPath);
    +  }
    +
    +  private String generateChildUniqueId(String parentId, String childKey, String childNum) {
    +    // combines parentId with the child's key and childNum. e.g. "10/footnote/1"
    --- End diff --
    
    the comment is obsolete again ;-)


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r201222488
  
    --- Diff: solr/core/src/test/org/apache/solr/update/TestNestedUpdateProcessor.java ---
    @@ -120,25 +122,41 @@ public void before() throws Exception {
     
       @Test
       public void testDeeplyNestedURPGrandChild() throws Exception {
    +    final String[] tests = {
    +        "/response/docs/[0]/id=='" + grandChildId + "'",
    +        "/response/docs/[0]/" + IndexSchema.NEST_PATH_FIELD_NAME + "=='children#0/grandChild#'"
    +    };
         indexSampleData(jDoc);
     
    -    assertJQ(req("q", IndexSchema.NEST_PATH_FIELD_NAME + ":*" + PATH_SEP_CHAR + "grandChild" + NUM_SEP_CHAR + "*" + NUM_SEP_CHAR,
    +    assertJQ(req("q", IndexSchema.NEST_PATH_FIELD_NAME + ":*" + PATH_SEP_CHAR + "grandChild" + NUM_SEP_CHAR + "*",
             "fl","*",
             "sort","id desc",
             "wt","json"),
    -        "/response/docs/[0]/id=='" + grandChildId + "'");
    +        tests);
       }
     
       @Test
       public void testDeeplyNestedURPChildren() throws Exception {
    --- End diff --
    
    I added a new sanity unit test for this URP


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r198158464
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessor.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * 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.io.IOException;
    +import java.util.EnumSet;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +import static org.apache.solr.update.processor.NestedUpdateProcessorFactory.NestedFlag;
    +
    +public class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  public static final String splitChar = ".";
    +  private EnumSet<NestedFlag> fields;
    --- End diff --
    
    Yes, though in the future if more meta-data is to be added it will be a lot more painful to add more fields


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r199678334
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessor.java ---
    @@ -0,0 +1,96 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  public static final String PATH_SEP_CHAR = "/";
    +  private boolean storePath;
    +  private boolean storeParent;
    +  SolrQueryRequest req;
    +
    +
    +  protected NestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, boolean storeParent, boolean storePath, UpdateRequestProcessor next) {
    +    super(next);
    +    this.req = req;
    +    this.storeParent = storeParent;
    +    this.storePath = storePath;
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    String rootId = doc.getField(req.getSchema().getUniqueKeyField().getName()).getFirstValue().toString();
    +    processDocChildren(doc, rootId, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String rootId, String fullPath) {
    +    int childNum = 0;
    +    for(SolrInputField field: doc.values()) {
    +      for(Object val: field) {
    +        if(val instanceof SolrInputDocument) {
    +          if(field.getName().contains(PATH_SEP_CHAR)) {
    +            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field name: '" + field.getName()
    +                + "' contains: '" + PATH_SEP_CHAR + "' , which is reserved for the nested URP");
    +          }
    +          final String jointPath = Objects.isNull(fullPath) ? field.getName(): String.join(PATH_SEP_CHAR, fullPath, field.getName());
    +          SolrInputDocument cDoc = (SolrInputDocument) val;
    +          if(!cDoc.containsKey(req.getSchema().getUniqueKeyField().getName())) {
    +            cDoc.setField(req.getSchema().getUniqueKeyField().getName(), generateChildUniqueId(rootId, jointPath, childNum));
    +          }
    +          processChildDoc((SolrInputDocument) val, doc, rootId, jointPath);
    +        }
    +        ++childNum;
    +      }
    +    }
    +  }
    +
    +  private void processChildDoc(SolrInputDocument sdoc, SolrInputDocument parent, String rootId, String fullPath) {
    +    if(storePath) {
    +      setPathField(sdoc, fullPath);
    +    }
    +    if (storeParent) {
    +      setParentKey(sdoc, parent);
    +    }
    +    processDocChildren(sdoc, rootId, fullPath);
    +  }
    +
    +  private String generateChildUniqueId(String rootId, String childPath, int childNum) {
    +    return String.join(PATH_SEP_CHAR, rootId, childPath, Integer.toString(childNum));
    --- End diff --
    
    See my point?


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r197861078
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/DeeplyNestedUpdateProcessor.java ---
    @@ -0,0 +1,101 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Collection;
    +import java.util.EnumSet;
    +import java.util.Locale;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +import static org.apache.solr.update.processor.DeeplyNestedUpdateProcessorFactory.NestedFlag;
    +
    +public class DeeplyNestedUpdateProcessor extends UpdateRequestProcessor {
    +  private EnumSet<NestedFlag> fields;
    +  SolrQueryRequest req;
    +
    +
    +  protected DeeplyNestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, EnumSet<NestedFlag> fields, UpdateRequestProcessor next) {
    +    super(next);
    +    this.req = req;
    +    this.fields = fields;
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    processDocChildren(doc, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String fullPath) {
    +    for(SolrInputField field: doc.values()) {
    +      if(field.getFirstValue() instanceof SolrInputDocument) {
    +        Object val = field.getValue();
    +        fullPath = Objects.isNull(fullPath) ? field.getName(): String.format(Locale.ROOT,"%s.%s", fullPath, field.getName());
    +        if (val instanceof Collection) {
    +          for(Object childDoc: (Collection) val) {
    +            if(childDoc instanceof SolrInputDocument) {
    +              processChildDoc((SolrInputDocument) childDoc, doc, fullPath);
    +            }
    +          }
    +        } else {
    +          processChildDoc((SolrInputDocument) val, doc, fullPath);
    +        }
    +      }
    +    }
    +  }
    +
    +  private void processChildDoc(SolrInputDocument sdoc, SolrInputDocument parent, String fullPath) {
    +    if(fields == NestedFlag.ALL) {
    --- End diff --
    
    I don't think we need special-case ALL


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r197866818
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/DeeplyNestedUpdateProcessorFactory.java ---
    @@ -0,0 +1,74 @@
    +/*
    + * 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.EnumSet;
    +import java.util.List;
    +import java.util.Locale;
    +import java.util.stream.Collectors;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.solr.common.util.NamedList;
    +import org.apache.solr.common.util.StrUtils;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +
    +import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
    +
    +public class DeeplyNestedUpdateProcessorFactory extends UpdateRequestProcessorFactory {
    +
    +  private EnumSet<NestedFlag> fields;
    +  private static final List<String> allowedConfFields = NestedFlag.ALL.stream().map(e -> e.toString().toLowerCase(Locale.ROOT)).collect(Collectors.toList());
    +
    +  public UpdateRequestProcessor getInstance(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next ) {
    +    return new DeeplyNestedUpdateProcessor(req, rsp, fields, next);
    +  }
    +
    +  @Override
    +  public void init( NamedList args )
    +  {
    +    Object tmp = args.remove("fields");
    +    if (null == tmp) {
    +      throw new SolrException(SERVER_ERROR,
    +          "'versionField' must be configured");
    +    }
    +    if (! (tmp instanceof String) ) {
    +      throw new SolrException(SERVER_ERROR,
    +          "'versionField' must be configured as a <str>");
    +    }
    +    List<String> fields = StrUtils.splitSmart((String)tmp, ',');
    +    if(!allowedConfFields.containsAll(fields)) {
    +      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Deeply Nested URP may only contain: " + StringUtils.join(allowedConfFields, ", ") +
    +      " got: " + StringUtils.join(fields, ", ") + " instead");
    +    }
    +    this.fields = fields.size() == NestedFlag.values().length ? NestedFlag.ALL: generateNestedFlags(fields);
    +  }
    +
    +  private static EnumSet<NestedFlag> generateNestedFlags(List<String> fields) {
    +    return EnumSet.copyOf(fields.stream().map(e -> NestedFlag.valueOf(e.toUpperCase(Locale.ROOT))).collect(Collectors.toList()));
    +  }
    +
    +  public enum NestedFlag {
    +    PATH,
    +    PARENT,
    +    LEVEL;
    +
    +    public static final EnumSet<NestedFlag> ALL = EnumSet.allOf(NestedFlag.class);
    --- End diff --
    
    Nice


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r200230404
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessorFactory.java ---
    @@ -0,0 +1,120 @@
    +/*
    + * 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.io.IOException;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessorFactory extends UpdateRequestProcessorFactory {
    +
    +  public UpdateRequestProcessor getInstance(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next ) {
    +    boolean storeParent = shouldStoreDocParent(req.getSchema());
    +    boolean storePath = shouldStoreDocPath(req.getSchema());
    +    if(!(storeParent || storePath)) {
    +      return next;
    +    }
    +    return new NestedUpdateProcessor(req, rsp, shouldStoreDocParent(req.getSchema()), shouldStoreDocPath(req.getSchema()), next);
    +  }
    +
    +  private static boolean shouldStoreDocParent(IndexSchema schema) {
    +    return schema.getFields().containsKey(IndexSchema.PARENT_FIELD_NAME);
    +  }
    +
    +  private static boolean shouldStoreDocPath(IndexSchema schema) {
    +    return schema.getFields().containsKey(IndexSchema.PATH_FIELD_NAME);
    +  }
    +}
    +
    +class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  private static final String PATH_SEP_CHAR = "/";
    +  private boolean storePath;
    +  private boolean storeParent;
    +  private String uniqueKeyFieldName;
    +
    +
    +  protected NestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, boolean storeParent, boolean storePath, UpdateRequestProcessor next) {
    +    super(next);
    +    this.storeParent = storeParent;
    +    this.storePath = storePath;
    +    this.uniqueKeyFieldName = req.getSchema().getUniqueKeyField().getName();
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    processDocChildren(doc, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String fullPath) {
    +    int childNum = 0;
    +    for(SolrInputField field: doc.values()) {
    +      for(Object val: field) {
    +        if(!(val instanceof SolrInputDocument)) {
    +          // either all collection items are child docs or none are.
    +          break;
    +        }
    +        final String fieldName = field.getName();
    +
    +        if(fieldName.contains(PATH_SEP_CHAR)) {
    +          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field name: '" + fieldName
    +              + "' contains: '" + PATH_SEP_CHAR + "' , which is reserved for the nested URP");
    +        }
    +        final String jointPath = fullPath == null ? fieldName: fullPath + PATH_SEP_CHAR + fieldName;
    +        SolrInputDocument cDoc = (SolrInputDocument) val;
    +        if(!cDoc.containsKey(uniqueKeyFieldName)) {
    +          String parentDocId = doc.getField(uniqueKeyFieldName).getFirstValue().toString();
    +          cDoc.setField(uniqueKeyFieldName, generateChildUniqueId(parentDocId, fieldName, childNum));
    +        }
    +        processChildDoc((SolrInputDocument) val, doc, jointPath);
    +        ++childNum;
    +      }
    +    }
    +  }
    +
    +  private void processChildDoc(SolrInputDocument sdoc, SolrInputDocument parent, String fullPath) {
    +    if(storePath) {
    +      setPathField(sdoc, fullPath);
    +    }
    +    if (storeParent) {
    +      setParentKey(sdoc, parent);
    +    }
    +    processDocChildren(sdoc, fullPath);
    +  }
    +
    +  private String generateChildUniqueId(String parentId, String childKey, int childNum) {
    +    // combines parentId with the child's key and childNum. e.g. "10/footnote/1"
    +    return String.join(PATH_SEP_CHAR, parentId, childKey, Integer.toString(childNum));
    --- End diff --
    
    Glad you fixed the comment.
    Another dubious use of of String.join
    What did you think of my suggestion to use some other separator char (not PATH_SEP_CHAR) over here.  My first suggestion was a pound symbol.  Though it wouldn't show when in a URL... maybe a comma.  But we could stay with a '/'; I just thought it might be nice to make a distinction between the separator between parent/child and the separator to append the child sequence/num.
    We ought to surface this to the JIRA as it's likely to get input.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r199383206
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessor.java ---
    @@ -0,0 +1,96 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  public static final String PATH_SEP_CHAR = "/";
    +  private boolean storePath;
    +  private boolean storeParent;
    +  SolrQueryRequest req;
    +
    +
    +  protected NestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, boolean storeParent, boolean storePath, UpdateRequestProcessor next) {
    +    super(next);
    +    this.req = req;
    +    this.storeParent = storeParent;
    +    this.storePath = storePath;
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    String rootId = doc.getField(req.getSchema().getUniqueKeyField().getName()).getFirstValue().toString();
    +    processDocChildren(doc, rootId, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String rootId, String fullPath) {
    +    int childNum = 0;
    +    for(SolrInputField field: doc.values()) {
    +      for(Object val: field) {
    +        if(val instanceof SolrInputDocument) {
    +          if(field.getName().contains(PATH_SEP_CHAR)) {
    +            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field name: '" + field.getName()
    +                + "' contains: '" + PATH_SEP_CHAR + "' , which is reserved for the nested URP");
    +          }
    +          final String jointPath = Objects.isNull(fullPath) ? field.getName(): String.join(PATH_SEP_CHAR, fullPath, field.getName());
    --- End diff --
    
    Objects.isNull is a more verbose version of == null.  It mainly exists for the Predicate API.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r197862734
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/DeeplyNestedUpdateProcessor.java ---
    @@ -0,0 +1,101 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Collection;
    +import java.util.EnumSet;
    +import java.util.Locale;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +import static org.apache.solr.update.processor.DeeplyNestedUpdateProcessorFactory.NestedFlag;
    +
    +public class DeeplyNestedUpdateProcessor extends UpdateRequestProcessor {
    +  private EnumSet<NestedFlag> fields;
    +  SolrQueryRequest req;
    +
    +
    +  protected DeeplyNestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, EnumSet<NestedFlag> fields, UpdateRequestProcessor next) {
    +    super(next);
    +    this.req = req;
    +    this.fields = fields;
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    processDocChildren(doc, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String fullPath) {
    +    for(SolrInputField field: doc.values()) {
    +      if(field.getFirstValue() instanceof SolrInputDocument) {
    +        Object val = field.getValue();
    +        fullPath = Objects.isNull(fullPath) ? field.getName(): String.format(Locale.ROOT,"%s.%s", fullPath, field.getName());
    +        if (val instanceof Collection) {
    +          for(Object childDoc: (Collection) val) {
    +            if(childDoc instanceof SolrInputDocument) {
    +              processChildDoc((SolrInputDocument) childDoc, doc, fullPath);
    +            }
    +          }
    +        } else {
    +          processChildDoc((SolrInputDocument) val, doc, fullPath);
    +        }
    +      }
    +    }
    +  }
    +
    +  private void processChildDoc(SolrInputDocument sdoc, SolrInputDocument parent, String fullPath) {
    +    if(fields == NestedFlag.ALL) {
    +      setPathField(sdoc, fullPath);
    +      setParentKey(sdoc, parent);
    +      setLevelKey(sdoc, fullPath);
    +    } else {
    +      if(fields.contains(NestedFlag.PATH)) {
    +        setPathField(sdoc, fullPath);
    +      }
    +      if (fields.contains(NestedFlag.PARENT)) {
    +        setParentKey(sdoc, parent);
    +      }
    +      if(fields.contains(NestedFlag.LEVEL)) {
    +        setLevelKey(sdoc, fullPath);
    +      }
    +    }
    +    processDocChildren(sdoc, fullPath);
    +  }
    +
    +  private void setLevelKey(SolrInputDocument sdoc, String fullPath) {
    +    sdoc.addField(IndexSchema.LEVEL_FIELD_NAME, fullPath.split("\\.").length);
    --- End diff --
    
    yuck; sorry.  Perhaps instead of pre-concatenating the full path, we instead pass full path around as a List<String> to the various arguments and then we only materialize it in setPathField?  This may also ameliorate the O(N^2) problem although we would be re-concatenating the earlier part of the string but that doesn't seem like a big deal.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r199383053
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessor.java ---
    @@ -0,0 +1,96 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  public static final String PATH_SEP_CHAR = "/";
    +  private boolean storePath;
    +  private boolean storeParent;
    +  SolrQueryRequest req;
    --- End diff --
    
    I think we don't need this field since AddUpdateCommand.getReq() has it.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r199677374
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessor.java ---
    @@ -0,0 +1,96 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  public static final String PATH_SEP_CHAR = "/";
    +  private boolean storePath;
    +  private boolean storeParent;
    +  SolrQueryRequest req;
    +
    +
    +  protected NestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, boolean storeParent, boolean storePath, UpdateRequestProcessor next) {
    +    super(next);
    +    this.req = req;
    +    this.storeParent = storeParent;
    +    this.storePath = storePath;
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    String rootId = doc.getField(req.getSchema().getUniqueKeyField().getName()).getFirstValue().toString();
    +    processDocChildren(doc, rootId, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String rootId, String fullPath) {
    +    int childNum = 0;
    +    for(SolrInputField field: doc.values()) {
    +      for(Object val: field) {
    +        if(val instanceof SolrInputDocument) {
    +          if(field.getName().contains(PATH_SEP_CHAR)) {
    +            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field name: '" + field.getName()
    +                + "' contains: '" + PATH_SEP_CHAR + "' , which is reserved for the nested URP");
    +          }
    +          final String jointPath = Objects.isNull(fullPath) ? field.getName(): String.join(PATH_SEP_CHAR, fullPath, field.getName());
    --- End diff --
    
    Simply use Java's String concatenation syntax:  `fullPath + PATH_SEP_CHAR + field.getName()`  BTW this translates internally to a StringBuilder though that's an implementation detail.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r198148078
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessor.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * 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.io.IOException;
    +import java.util.EnumSet;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +import static org.apache.solr.update.processor.NestedUpdateProcessorFactory.NestedFlag;
    +
    +public class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  public static final String splitChar = ".";
    --- End diff --
    
    nitpick: "splitChar" name suggests we are going to "split" on something but we don't.  I think "PATH_SEP_CHAR" is a better constant name.  If it suits the usage details it could be of type character but if not leave as String.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r200372947
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessorFactory.java ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Collection;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessorFactory extends UpdateRequestProcessorFactory {
    +
    +  public UpdateRequestProcessor getInstance(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next ) {
    +    boolean storeParent = shouldStoreDocParent(req.getSchema());
    +    boolean storePath = shouldStoreDocPath(req.getSchema());
    +    if(!(storeParent || storePath)) {
    +      return next;
    +    }
    +    return new NestedUpdateProcessor(req, rsp, shouldStoreDocParent(req.getSchema()), shouldStoreDocPath(req.getSchema()), next);
    +  }
    +
    +  private static boolean shouldStoreDocParent(IndexSchema schema) {
    +    return schema.getFields().containsKey(IndexSchema.NEST_PARENT_FIELD_NAME);
    +  }
    +
    +  private static boolean shouldStoreDocPath(IndexSchema schema) {
    +    return schema.getFields().containsKey(IndexSchema.NEST_PATH_FIELD_NAME);
    +  }
    +}
    +
    +class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  private static final String PATH_SEP_CHAR = "/";
    +  private static final String NUM_SEP_CHAR = ",";
    +  private static final String SINGLE_VALUE_CHAR = "s";
    +  private boolean storePath;
    +  private boolean storeParent;
    +  private String uniqueKeyFieldName;
    +
    +
    +  protected NestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, boolean storeParent, boolean storePath, UpdateRequestProcessor next) {
    +    super(next);
    +    this.storeParent = storeParent;
    +    this.storePath = storePath;
    +    this.uniqueKeyFieldName = req.getSchema().getUniqueKeyField().getName();
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    processDocChildren(doc, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String fullPath) {
    +    int childNum = 0;
    +    for(SolrInputField field: doc.values()) {
    +      boolean isSingleVal = !(field.getValue() instanceof Collection);
    +      for(Object val: field) {
    +        if(!(val instanceof SolrInputDocument)) {
    +          // either all collection items are child docs or none are.
    +          break;
    +        }
    +        final String fieldName = field.getName();
    +
    +        if(fieldName.contains(PATH_SEP_CHAR)) {
    +          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field name: '" + fieldName
    +              + "' contains: '" + PATH_SEP_CHAR + "' , which is reserved for the nested URP");
    +        }
    +        final String sChildNum = isSingleVal ? SINGLE_VALUE_CHAR: String.valueOf(childNum);
    +        final String lastPath = fieldName + NUM_SEP_CHAR + sChildNum + NUM_SEP_CHAR;
    --- End diff --
    
    I'm a bit confused why SINGLE_VALUE_CHAR was introduced.  Is this so that a DocTransformer can differentiates between a one-ary array and a single doc?  I suppose so.  The choice of 's' was confusing as I thought it was an attempt to pluralize something at first but now I think you intended an abbreviation for "singular".  Could we simply use the empty string in this case?
    
    Also, please define lastPath & jointPath later closer to when they are used, which is immediately prior to calling processChildDoc.
    
    Why the trailing NUM_SEP_CHAR in lastPath?


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r200233323
  
    --- Diff: solr/core/src/test/org/apache/solr/update/TestNestedUpdateProcessor.java ---
    @@ -0,0 +1,171 @@
    +/*
    + * 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;
    +
    +import java.util.List;
    +
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.processor.NestedUpdateProcessorFactory;
    +import org.apache.solr.update.processor.UpdateRequestProcessor;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Rule;
    +import org.junit.Test;
    +import org.junit.rules.ExpectedException;
    +
    +public class TestNestedUpdateProcessor extends SolrTestCaseJ4 {
    +
    +  private static final char PATH_SEP_CHAR = '/';
    +  private static final String[] childrenIds = { "2", "3" };
    +  private static final String grandChildId = "4";
    +  private static final String jDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String noIdChildren = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String errDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children" + PATH_SEP_CHAR + "a\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  @Rule
    +  public ExpectedException thrown = ExpectedException.none();
    +
    +  @BeforeClass
    +  public static void beforeClass() throws Exception {
    +    initCore("solrconfig-update-processor-chains.xml", "schema15.xml");
    +  }
    +
    +  @Before
    +  public void before() throws Exception {
    +    assertU(delQ("*:*"));
    +    assertU(commit());
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPGrandChild() throws Exception {
    +    indexSampleData(jDoc);
    +
    +    assertJQ(req("q", IndexSchema.PATH_FIELD_NAME + ":*" + PATH_SEP_CHAR + "grandChild",
    +        "fl","*",
    +        "sort","id desc",
    +        "wt","json"),
    +        "/response/docs/[0]/id=='" + grandChildId + "'");
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPChildren() throws Exception {
    +    final String[] childrenTests = {"/response/docs/[0]/id=='" + childrenIds[0] + "'", "/response/docs/[1]/id=='" + childrenIds[1] + "'"};
    +    indexSampleData(jDoc);
    +
    +    assertJQ(req("q", IndexSchema.PATH_FIELD_NAME + ":children",
    +        "fl","*",
    +        "sort","id asc",
    +        "wt","json"),
    +        childrenTests);
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPChildrenWoId() throws Exception {
    +    final String parentId = "3";
    --- End diff --
    
    Sure thing.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r198151910
  
    --- Diff: solr/core/src/test/org/apache/solr/update/TestNestedUpdateProcessor.java ---
    @@ -0,0 +1,158 @@
    +/*
    + * 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;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.util.ContentStream;
    +import org.apache.solr.common.util.ContentStreamBase;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.request.SolrRequestInfo;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.servlet.SolrRequestParsers;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Rule;
    +import org.junit.Test;
    +import org.junit.rules.ExpectedException;
    +
    +import static org.apache.solr.update.processor.NestedUpdateProcessor.splitChar;
    +
    +public class TestNestedUpdateProcessor extends SolrTestCaseJ4 {
    +
    +  private static final String[] childrenIds = { "2", "3" };
    +  private static final String grandChildId = "4";
    +  private static final String jDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String errDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children" + splitChar + "a\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  @Rule
    +  public ExpectedException thrown = ExpectedException.none();
    +
    +  @BeforeClass
    +  public static void beforeClass() throws Exception {
    +    initCore("solrconfig-update-processor-chains.xml", "schema15.xml");
    +  }
    +
    +  @Before
    +  public void before() throws Exception {
    +    assertU(delQ("*:*"));
    +    assertU(commit());
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPGrandChild() throws Exception {
    +    indexSampleData(jDoc);
    +
    +    assertJQ(req("q", IndexSchema.PATH_FIELD_NAME + ":*.grandChild",
    +        "fl","*",
    +        "sort","id desc",
    +        "wt","json"),
    +        "/response/docs/[0]/id=='" + grandChildId + "'");
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPChildren() throws Exception {
    +    final String[] childrenTests = {"/response/docs/[0]/id=='" + childrenIds[0] + "'", "/response/docs/[1]/id=='" + childrenIds[1] + "'"};
    +    indexSampleData(jDoc);
    +
    +    assertJQ(req("q", IndexSchema.PATH_FIELD_NAME + ":children",
    +        "fl","*",
    +        "sort","id asc",
    +        "wt","json"),
    +        childrenTests);
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPFieldNameException() throws Exception {
    +    final String errMsg = "contains: '" + splitChar + "' , which is reserved for the nested URP";
    --- End diff --
    
    Lets consider rewording the error message so that the user is able to understand it without having to know about URPs.  I suggest: "The field/relationship name 'children.a' to a nested document cannot have a '.' as it interferes with internal processing".
    
    Come to think of it, lets use '/' as this is not a valid field name char whereas '.' is?


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r200231147
  
    --- Diff: solr/core/src/test/org/apache/solr/update/TestNestedUpdateProcessor.java ---
    @@ -0,0 +1,171 @@
    +/*
    + * 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;
    +
    +import java.util.List;
    +
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.processor.NestedUpdateProcessorFactory;
    +import org.apache.solr.update.processor.UpdateRequestProcessor;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Rule;
    +import org.junit.Test;
    +import org.junit.rules.ExpectedException;
    +
    +public class TestNestedUpdateProcessor extends SolrTestCaseJ4 {
    +
    +  private static final char PATH_SEP_CHAR = '/';
    +  private static final String[] childrenIds = { "2", "3" };
    +  private static final String grandChildId = "4";
    +  private static final String jDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String noIdChildren = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String errDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children" + PATH_SEP_CHAR + "a\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  @Rule
    +  public ExpectedException thrown = ExpectedException.none();
    +
    +  @BeforeClass
    +  public static void beforeClass() throws Exception {
    +    initCore("solrconfig-update-processor-chains.xml", "schema15.xml");
    +  }
    +
    +  @Before
    +  public void before() throws Exception {
    +    assertU(delQ("*:*"));
    +    assertU(commit());
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPGrandChild() throws Exception {
    +    indexSampleData(jDoc);
    +
    +    assertJQ(req("q", IndexSchema.PATH_FIELD_NAME + ":*" + PATH_SEP_CHAR + "grandChild",
    +        "fl","*",
    +        "sort","id desc",
    +        "wt","json"),
    +        "/response/docs/[0]/id=='" + grandChildId + "'");
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPChildren() throws Exception {
    +    final String[] childrenTests = {"/response/docs/[0]/id=='" + childrenIds[0] + "'", "/response/docs/[1]/id=='" + childrenIds[1] + "'"};
    +    indexSampleData(jDoc);
    +
    +    assertJQ(req("q", IndexSchema.PATH_FIELD_NAME + ":children",
    +        "fl","*",
    +        "sort","id asc",
    +        "wt","json"),
    +        childrenTests);
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPChildrenWoId() throws Exception {
    +    final String parentId = "3";
    +    final String childKey = "grandChild";
    +    final String expectedId = parentId + PATH_SEP_CHAR + childKey + PATH_SEP_CHAR + "0";
    +    SolrInputDocument noIdChildren = sdoc("id", "1", "children", sdocs(sdoc("id", "2", "name_s", "Yaz"), sdoc("id", parentId, "name_s", "Jazz", childKey, sdoc("names_s", "Gaz"))));
    +    UpdateRequestProcessor nestedUpdate = new NestedUpdateProcessorFactory().getInstance(req(), null, null);
    +    AddUpdateCommand cmd = new AddUpdateCommand(req());
    +    cmd.solrDoc = noIdChildren;
    +    nestedUpdate.processAdd(cmd);
    +    cmd.clear();
    +    List children = (List) noIdChildren.get("children").getValues();
    +    SolrInputDocument idLessChild = (SolrInputDocument)((SolrInputDocument) children.get(1)).get(childKey).getValue();
    +    assertTrue("Id less child did not get an Id", idLessChild.containsKey("id"));
    --- End diff --
    
    This is fine but FYI doing a doc.toString() and doing a simple assertEquals on that String is often more thorough and easy to read and less code.  That said, sometimes it can be brittle to unrelated changes.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r200373375
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessorFactory.java ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Collection;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessorFactory extends UpdateRequestProcessorFactory {
    +
    +  public UpdateRequestProcessor getInstance(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next ) {
    +    boolean storeParent = shouldStoreDocParent(req.getSchema());
    +    boolean storePath = shouldStoreDocPath(req.getSchema());
    +    if(!(storeParent || storePath)) {
    +      return next;
    +    }
    +    return new NestedUpdateProcessor(req, rsp, shouldStoreDocParent(req.getSchema()), shouldStoreDocPath(req.getSchema()), next);
    +  }
    +
    +  private static boolean shouldStoreDocParent(IndexSchema schema) {
    +    return schema.getFields().containsKey(IndexSchema.NEST_PARENT_FIELD_NAME);
    +  }
    +
    +  private static boolean shouldStoreDocPath(IndexSchema schema) {
    +    return schema.getFields().containsKey(IndexSchema.NEST_PATH_FIELD_NAME);
    +  }
    +}
    +
    +class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  private static final String PATH_SEP_CHAR = "/";
    +  private static final String NUM_SEP_CHAR = ",";
    --- End diff --
    
    It appears most any special char we use would be URL encoded so I think might as well use '#' for NUM_SEP_CHAR, which seems like the ideal choice.


---

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


[GitHub] lucene-solr issue #410: SOLR-12441: add deeply nested URP for nested documen...

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

    https://github.com/apache/lucene-solr/pull/410
  
    Closing this as it was merged.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r199394431
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessor.java ---
    @@ -0,0 +1,96 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  public static final String PATH_SEP_CHAR = "/";
    +  private boolean storePath;
    +  private boolean storeParent;
    +  SolrQueryRequest req;
    +
    +
    +  protected NestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, boolean storeParent, boolean storePath, UpdateRequestProcessor next) {
    +    super(next);
    +    this.req = req;
    +    this.storeParent = storeParent;
    +    this.storePath = storePath;
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    String rootId = doc.getField(req.getSchema().getUniqueKeyField().getName()).getFirstValue().toString();
    +    processDocChildren(doc, rootId, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String rootId, String fullPath) {
    +    int childNum = 0;
    +    for(SolrInputField field: doc.values()) {
    +      for(Object val: field) {
    +        if(val instanceof SolrInputDocument) {
    +          if(field.getName().contains(PATH_SEP_CHAR)) {
    +            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field name: '" + field.getName()
    +                + "' contains: '" + PATH_SEP_CHAR + "' , which is reserved for the nested URP");
    +          }
    +          final String jointPath = Objects.isNull(fullPath) ? field.getName(): String.join(PATH_SEP_CHAR, fullPath, field.getName());
    --- End diff --
    
    Do you suggest using StringBuilder instead?


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r200841187
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessorFactory.java ---
    @@ -72,8 +72,8 @@ public void processAdd(AddUpdateCommand cmd) throws IOException {
       }
     
       private void processDocChildren(SolrInputDocument doc, String fullPath) {
    -    int childNum = 0;
         for(SolrInputField field: doc.values()) {
    +      int childNum = 0;
    --- End diff --
    
    If the test did not show this bug, lets tweak the test to show it.  Yeah?


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r200115050
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessor.java ---
    @@ -0,0 +1,96 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  public static final String PATH_SEP_CHAR = "/";
    +  private boolean storePath;
    +  private boolean storeParent;
    +  SolrQueryRequest req;
    +
    +
    +  protected NestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, boolean storeParent, boolean storePath, UpdateRequestProcessor next) {
    +    super(next);
    +    this.req = req;
    +    this.storeParent = storeParent;
    +    this.storePath = storePath;
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    String rootId = doc.getField(req.getSchema().getUniqueKeyField().getName()).getFirstValue().toString();
    +    processDocChildren(doc, rootId, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String rootId, String fullPath) {
    +    int childNum = 0;
    +    for(SolrInputField field: doc.values()) {
    +      for(Object val: field) {
    +        if(val instanceof SolrInputDocument) {
    +          if(field.getName().contains(PATH_SEP_CHAR)) {
    +            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field name: '" + field.getName()
    +                + "' contains: '" + PATH_SEP_CHAR + "' , which is reserved for the nested URP");
    +          }
    +          final String jointPath = Objects.isNull(fullPath) ? field.getName(): String.join(PATH_SEP_CHAR, fullPath, field.getName());
    +          SolrInputDocument cDoc = (SolrInputDocument) val;
    +          if(!cDoc.containsKey(req.getSchema().getUniqueKeyField().getName())) {
    +            cDoc.setField(req.getSchema().getUniqueKeyField().getName(), generateChildUniqueId(rootId, jointPath, childNum));
    +          }
    +          processChildDoc((SolrInputDocument) val, doc, rootId, jointPath);
    +        }
    +        ++childNum;
    +      }
    +    }
    +  }
    +
    +  private void processChildDoc(SolrInputDocument sdoc, SolrInputDocument parent, String rootId, String fullPath) {
    +    if(storePath) {
    +      setPathField(sdoc, fullPath);
    +    }
    +    if (storeParent) {
    +      setParentKey(sdoc, parent);
    +    }
    +    processDocChildren(sdoc, rootId, fullPath);
    +  }
    +
    +  private String generateChildUniqueId(String rootId, String childPath, int childNum) {
    +    return String.join(PATH_SEP_CHAR, rootId, childPath, Integer.toString(childNum));
    --- End diff --
    
    Would label be the key of the document?


---

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


[GitHub] lucene-solr issue #410: SOLR-12441: add deeply nested URP for nested documen...

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

    https://github.com/apache/lucene-solr/pull/410
  
    I added a test for "id"less children, which is currently because the JSON loader does not support it yet, as discussed in [SOLR-12362](https://issues.apache.org/jira/browse/SOLR-12362)


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r200841295
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessor.java ---
    @@ -0,0 +1,96 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  public static final String PATH_SEP_CHAR = "/";
    +  private boolean storePath;
    +  private boolean storeParent;
    +  SolrQueryRequest req;
    +
    +
    +  protected NestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, boolean storeParent, boolean storePath, UpdateRequestProcessor next) {
    +    super(next);
    +    this.req = req;
    +    this.storeParent = storeParent;
    +    this.storePath = storePath;
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    String rootId = doc.getField(req.getSchema().getUniqueKeyField().getName()).getFirstValue().toString();
    +    processDocChildren(doc, rootId, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String rootId, String fullPath) {
    +    int childNum = 0;
    +    for(SolrInputField field: doc.values()) {
    +      for(Object val: field) {
    +        if(val instanceof SolrInputDocument) {
    +          if(field.getName().contains(PATH_SEP_CHAR)) {
    +            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field name: '" + field.getName()
    +                + "' contains: '" + PATH_SEP_CHAR + "' , which is reserved for the nested URP");
    +          }
    +          final String jointPath = Objects.isNull(fullPath) ? field.getName(): String.join(PATH_SEP_CHAR, fullPath, field.getName());
    +          SolrInputDocument cDoc = (SolrInputDocument) val;
    +          if(!cDoc.containsKey(req.getSchema().getUniqueKeyField().getName())) {
    +            cDoc.setField(req.getSchema().getUniqueKeyField().getName(), generateChildUniqueId(rootId, jointPath, childNum));
    +          }
    +          processChildDoc((SolrInputDocument) val, doc, rootId, jointPath);
    +        }
    +        ++childNum;
    +      }
    +    }
    +  }
    +
    +  private void processChildDoc(SolrInputDocument sdoc, SolrInputDocument parent, String rootId, String fullPath) {
    +    if(storePath) {
    +      setPathField(sdoc, fullPath);
    +    }
    +    if (storeParent) {
    +      setParentKey(sdoc, parent);
    +    }
    +    processDocChildren(sdoc, rootId, fullPath);
    +  }
    +
    +  private String generateChildUniqueId(String rootId, String childPath, int childNum) {
    +    return String.join(PATH_SEP_CHAR, rootId, childPath, Integer.toString(childNum));
    --- End diff --
    
    Yes; my use of "label" means the key of the doc, aka the pseudo-fieldname linking the parent to 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 #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r201076987
  
    --- Diff: solr/core/src/test/org/apache/solr/update/TestNestedUpdateProcessor.java ---
    @@ -120,25 +122,41 @@ public void before() throws Exception {
     
       @Test
       public void testDeeplyNestedURPGrandChild() throws Exception {
    +    final String[] tests = {
    +        "/response/docs/[0]/id=='" + grandChildId + "'",
    +        "/response/docs/[0]/" + IndexSchema.NEST_PATH_FIELD_NAME + "=='children#0/grandChild#'"
    +    };
         indexSampleData(jDoc);
     
    -    assertJQ(req("q", IndexSchema.NEST_PATH_FIELD_NAME + ":*" + PATH_SEP_CHAR + "grandChild" + NUM_SEP_CHAR + "*" + NUM_SEP_CHAR,
    +    assertJQ(req("q", IndexSchema.NEST_PATH_FIELD_NAME + ":*" + PATH_SEP_CHAR + "grandChild" + NUM_SEP_CHAR + "*",
             "fl","*",
             "sort","id desc",
             "wt","json"),
    -        "/response/docs/[0]/id=='" + grandChildId + "'");
    +        tests);
       }
     
       @Test
       public void testDeeplyNestedURPChildren() throws Exception {
    --- End diff --
    
    This test tests the search behavior more so than literally what the URP is doing.  Can you make this more of a unit test around the result of the URP without actually indexing/searching anything?  And I would much prefer simpler test assertions that check a complete string value instead of making reference to many variables/constants that need to be concatenated.  This makes it plainly clear what the nest path will be; no mental gymnastics are needed to chase down vars/constants to figure it out.  I've mentioned before Yonik's advise on avoiding some constants in tests as it helps tests make us aware if in the future we might have a backwards-breaking change; so there are virtues to this way of thinking.  It would make this easier to review too.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r198148926
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessorFactory.java ---
    @@ -0,0 +1,72 @@
    +/*
    + * 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.EnumSet;
    +import java.util.List;
    +import java.util.Locale;
    +import java.util.stream.Collectors;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.solr.common.util.NamedList;
    +import org.apache.solr.common.util.StrUtils;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +
    +import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
    +
    +public class NestedUpdateProcessorFactory extends UpdateRequestProcessorFactory {
    +
    +  private EnumSet<NestedFlag> fields;
    +  private static final List<String> allowedConfFields = Arrays.stream(NestedFlag.values()).map(e -> e.toString().toLowerCase(Locale.ROOT)).collect(Collectors.toList());
    +
    +  public UpdateRequestProcessor getInstance(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next ) {
    +    return new NestedUpdateProcessor(req, rsp, fields, next);
    +  }
    +
    +  @Override
    +  public void init( NamedList args )
    --- End diff --
    
    Maybe lets not have any configuration at all -- let the schema be the guide.  If the schema contains `IndexSchema.PATH_FIELD_NAME` then we should populate it, if not then don't.  Ditto for `PARENT_FIELD_NAME`.  The getInstance method of the factory can detect if there is any work to do and if none then return "next", thus avoiding overhead.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r200230779
  
    --- Diff: solr/core/src/java/org/apache/solr/schema/IndexSchema.java ---
    @@ -107,6 +107,8 @@
       public static final String LUCENE_MATCH_VERSION_PARAM = "luceneMatchVersion";
       public static final String MAX_CHARS = "maxChars";
       public static final String NAME = "name";
    +  public static final String PARENT_FIELD_NAME = "_NEST_PARENT_";
    --- End diff --
    
    In one of my earliest review comments I suggested renaming both constants to start with `_NEST_` so that they are more clearly grouped together in their purpose.  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 #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r199386196
  
    --- Diff: solr/core/src/test/org/apache/solr/update/TestNestedUpdateProcessor.java ---
    @@ -0,0 +1,189 @@
    +/*
    + * 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;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.util.ContentStream;
    +import org.apache.solr.common.util.ContentStreamBase;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.request.SolrRequestInfo;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.servlet.SolrRequestParsers;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Rule;
    +import org.junit.Test;
    +import org.junit.rules.ExpectedException;
    +
    +public class TestNestedUpdateProcessor extends SolrTestCaseJ4 {
    +
    +  private static final char PATH_SEP_CHAR = '/';
    +  private static final String[] childrenIds = { "2", "3" };
    +  private static final String grandChildId = "4";
    +  private static final String jDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String noIdChildren = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String errDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children" + PATH_SEP_CHAR + "a\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  @Rule
    +  public ExpectedException thrown = ExpectedException.none();
    --- End diff --
    
    This is neat; I didn't know about this utility.  But take note of `SolrTestCaseJ4.assertQEx(...)` especially the one that takes the exception message.  You could use that?


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r200996646
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessorFactory.java ---
    @@ -72,8 +72,8 @@ public void processAdd(AddUpdateCommand cmd) throws IOException {
       }
     
       private void processDocChildren(SolrInputDocument doc, String fullPath) {
    -    int childNum = 0;
         for(SolrInputField field: doc.values()) {
    +      int childNum = 0;
    --- End diff --
    
    I'll add a test with another key holding childDocs to ensure this bug does not resurface


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r199384878
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessor.java ---
    @@ -0,0 +1,96 @@
    +/*
    --- End diff --
    
    Maybe move this as an inner class of it's factory?  It's not a particularly long class.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r199773120
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessor.java ---
    @@ -0,0 +1,96 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  public static final String PATH_SEP_CHAR = "/";
    +  private boolean storePath;
    +  private boolean storeParent;
    +  SolrQueryRequest req;
    +
    +
    +  protected NestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, boolean storeParent, boolean storePath, UpdateRequestProcessor next) {
    +    super(next);
    +    this.req = req;
    +    this.storeParent = storeParent;
    +    this.storePath = storePath;
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    String rootId = doc.getField(req.getSchema().getUniqueKeyField().getName()).getFirstValue().toString();
    +    processDocChildren(doc, rootId, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String rootId, String fullPath) {
    +    int childNum = 0;
    +    for(SolrInputField field: doc.values()) {
    +      for(Object val: field) {
    +        if(val instanceof SolrInputDocument) {
    +          if(field.getName().contains(PATH_SEP_CHAR)) {
    +            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field name: '" + field.getName()
    +                + "' contains: '" + PATH_SEP_CHAR + "' , which is reserved for the nested URP");
    +          }
    +          final String jointPath = Objects.isNull(fullPath) ? field.getName(): String.join(PATH_SEP_CHAR, fullPath, field.getName());
    --- End diff --
    
    Oh nice,
    changed it


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r198175722
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessor.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * 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.io.IOException;
    +import java.util.EnumSet;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +import static org.apache.solr.update.processor.NestedUpdateProcessorFactory.NestedFlag;
    +
    +public class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  public static final String splitChar = ".";
    +  private EnumSet<NestedFlag> fields;
    --- End diff --
    
    Replaced the EnumSet with two booleans as you proposed.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r199384693
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessor.java ---
    @@ -0,0 +1,96 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  public static final String PATH_SEP_CHAR = "/";
    +  private boolean storePath;
    +  private boolean storeParent;
    +  SolrQueryRequest req;
    +
    +
    +  protected NestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, boolean storeParent, boolean storePath, UpdateRequestProcessor next) {
    +    super(next);
    +    this.req = req;
    +    this.storeParent = storeParent;
    +    this.storePath = storePath;
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    String rootId = doc.getField(req.getSchema().getUniqueKeyField().getName()).getFirstValue().toString();
    +    processDocChildren(doc, rootId, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String rootId, String fullPath) {
    +    int childNum = 0;
    +    for(SolrInputField field: doc.values()) {
    +      for(Object val: field) {
    +        if(val instanceof SolrInputDocument) {
    +          if(field.getName().contains(PATH_SEP_CHAR)) {
    +            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field name: '" + field.getName()
    +                + "' contains: '" + PATH_SEP_CHAR + "' , which is reserved for the nested URP");
    +          }
    +          final String jointPath = Objects.isNull(fullPath) ? field.getName(): String.join(PATH_SEP_CHAR, fullPath, field.getName());
    +          SolrInputDocument cDoc = (SolrInputDocument) val;
    +          if(!cDoc.containsKey(req.getSchema().getUniqueKeyField().getName())) {
    +            cDoc.setField(req.getSchema().getUniqueKeyField().getName(), generateChildUniqueId(rootId, jointPath, childNum));
    +          }
    +          processChildDoc((SolrInputDocument) val, doc, rootId, jointPath);
    +        }
    +        ++childNum;
    +      }
    +    }
    +  }
    +
    +  private void processChildDoc(SolrInputDocument sdoc, SolrInputDocument parent, String rootId, String fullPath) {
    +    if(storePath) {
    +      setPathField(sdoc, fullPath);
    +    }
    +    if (storeParent) {
    +      setParentKey(sdoc, parent);
    +    }
    +    processDocChildren(sdoc, rootId, fullPath);
    +  }
    +
    +  private String generateChildUniqueId(String rootId, String childPath, int childNum) {
    +    return String.join(PATH_SEP_CHAR, rootId, childPath, Integer.toString(childNum));
    --- End diff --
    
    This will not necessarily generate a uniqueId.  Imagine their are two "page" children on the top book, and both pages have a "footnote" child.  They would both get the same ID?  BTW I find it really helpful for such methods like this if it has a comment showing an example ID.
    
    Perhaps the ID should be simply "/{label}#{childNum}" appended to whatever the ID is of the the parent of this doc is?


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r197859682
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/DeeplyNestedUpdateProcessor.java ---
    @@ -0,0 +1,101 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Collection;
    +import java.util.EnumSet;
    +import java.util.Locale;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +import static org.apache.solr.update.processor.DeeplyNestedUpdateProcessorFactory.NestedFlag;
    +
    +public class DeeplyNestedUpdateProcessor extends UpdateRequestProcessor {
    +  private EnumSet<NestedFlag> fields;
    +  SolrQueryRequest req;
    +
    +
    +  protected DeeplyNestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, EnumSet<NestedFlag> fields, UpdateRequestProcessor next) {
    +    super(next);
    +    this.req = req;
    +    this.fields = fields;
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    processDocChildren(doc, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String fullPath) {
    +    for(SolrInputField field: doc.values()) {
    +      if(field.getFirstValue() instanceof SolrInputDocument) {
    +        Object val = field.getValue();
    +        fullPath = Objects.isNull(fullPath) ? field.getName(): String.format(Locale.ROOT,"%s.%s", fullPath, field.getName());
    --- End diff --
    
    I can't say I'm a fan of String.format -- it usually is just a more verbose & complex way to do trivial string concatenation.  Granted String.format _sometimes_ has value (e.g. formatting floating point); it isn't exercised here.
    
    Shouldn't we throw a BAD_REQUEST exception if field.getName() contains the character we're using to join ('.')?
    
    There's an O(N^2) string concatenation here where N is the depth of the tree but hopefully the docs won't be *that* nested to worry about this.  We could add a comment to at least acknowledge 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 #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r197865187
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/DeeplyNestedUpdateProcessor.java ---
    @@ -0,0 +1,101 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Collection;
    +import java.util.EnumSet;
    +import java.util.Locale;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +import static org.apache.solr.update.processor.DeeplyNestedUpdateProcessorFactory.NestedFlag;
    +
    +public class DeeplyNestedUpdateProcessor extends UpdateRequestProcessor {
    +  private EnumSet<NestedFlag> fields;
    +  SolrQueryRequest req;
    +
    +
    +  protected DeeplyNestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, EnumSet<NestedFlag> fields, UpdateRequestProcessor next) {
    +    super(next);
    +    this.req = req;
    +    this.fields = fields;
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    processDocChildren(doc, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String fullPath) {
    +    for(SolrInputField field: doc.values()) {
    +      if(field.getFirstValue() instanceof SolrInputDocument) {
    +        Object val = field.getValue();
    +        fullPath = Objects.isNull(fullPath) ? field.getName(): String.format(Locale.ROOT,"%s.%s", fullPath, field.getName());
    +        if (val instanceof Collection) {
    +          for(Object childDoc: (Collection) val) {
    +            if(childDoc instanceof SolrInputDocument) {
    +              processChildDoc((SolrInputDocument) childDoc, doc, fullPath);
    +            }
    +          }
    +        } else {
    +          processChildDoc((SolrInputDocument) val, doc, fullPath);
    +        }
    +      }
    +    }
    +  }
    +
    +  private void processChildDoc(SolrInputDocument sdoc, SolrInputDocument parent, String fullPath) {
    +    if(fields == NestedFlag.ALL) {
    +      setPathField(sdoc, fullPath);
    +      setParentKey(sdoc, parent);
    +      setLevelKey(sdoc, fullPath);
    +    } else {
    +      if(fields.contains(NestedFlag.PATH)) {
    +        setPathField(sdoc, fullPath);
    +      }
    +      if (fields.contains(NestedFlag.PARENT)) {
    +        setParentKey(sdoc, parent);
    +      }
    +      if(fields.contains(NestedFlag.LEVEL)) {
    +        setLevelKey(sdoc, fullPath);
    +      }
    +    }
    +    processDocChildren(sdoc, fullPath);
    +  }
    +
    +  private void setLevelKey(SolrInputDocument sdoc, String fullPath) {
    +    sdoc.addField(IndexSchema.LEVEL_FIELD_NAME, fullPath.split("\\.").length);
    +  }
    +
    +  private void setParentKey(SolrInputDocument sdoc, SolrInputDocument parent) {
    +    sdoc.addField(IndexSchema.PARENT_FIELD_NAME, parent.getFieldValue(req.getSchema().getUniqueKeyField().getName()));
    --- End diff --
    
    For these "set" methods, I think we want to be calling *setField* and not *addField*; no?  Granted it'd be weird if there was an existing value... and if there was, maybe we shouldn't overwrite it?  Probably shouldn't.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r198147106
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessor.java ---
    @@ -0,0 +1,85 @@
    +/*
    + * 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.io.IOException;
    +import java.util.EnumSet;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +import static org.apache.solr.update.processor.NestedUpdateProcessorFactory.NestedFlag;
    +
    +public class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  public static final String splitChar = ".";
    +  private EnumSet<NestedFlag> fields;
    --- End diff --
    
    Although I do think EnumSet / enums are pretty nifty JDK utilities, in this case two booleans would be more clear?


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r199401847
  
    --- Diff: solr/core/src/test/org/apache/solr/update/TestNestedUpdateProcessor.java ---
    @@ -0,0 +1,189 @@
    +/*
    + * 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;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.util.ContentStream;
    +import org.apache.solr.common.util.ContentStreamBase;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.request.SolrRequestInfo;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.servlet.SolrRequestParsers;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Rule;
    +import org.junit.Test;
    +import org.junit.rules.ExpectedException;
    +
    +public class TestNestedUpdateProcessor extends SolrTestCaseJ4 {
    +
    +  private static final char PATH_SEP_CHAR = '/';
    +  private static final String[] childrenIds = { "2", "3" };
    +  private static final String grandChildId = "4";
    +  private static final String jDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String noIdChildren = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String errDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children" + PATH_SEP_CHAR + "a\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  @Rule
    +  public ExpectedException thrown = ExpectedException.none();
    +
    +  @BeforeClass
    +  public static void beforeClass() throws Exception {
    +    initCore("solrconfig-update-processor-chains.xml", "schema15.xml");
    +  }
    +
    +  @Before
    +  public void before() throws Exception {
    +    assertU(delQ("*:*"));
    +    assertU(commit());
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPGrandChild() throws Exception {
    +    indexSampleData(jDoc);
    +
    +    assertJQ(req("q", IndexSchema.PATH_FIELD_NAME + ":*" + PATH_SEP_CHAR + "grandChild",
    +        "fl","*",
    +        "sort","id desc",
    +        "wt","json"),
    +        "/response/docs/[0]/id=='" + grandChildId + "'");
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPChildren() throws Exception {
    +    final String[] childrenTests = {"/response/docs/[0]/id=='" + childrenIds[0] + "'", "/response/docs/[1]/id=='" + childrenIds[1] + "'"};
    +    indexSampleData(jDoc);
    +
    +    assertJQ(req("q", IndexSchema.PATH_FIELD_NAME + ":children",
    +        "fl","*",
    +        "sort","id asc",
    +        "wt","json"),
    +        childrenTests);
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPChildrenWoId() throws Exception {
    +    final String[] childrenTests = {"/response/docs/[0]/id=='1" + PATH_SEP_CHAR + "children" + PATH_SEP_CHAR + "2'"};
    +    indexSampleData(noIdChildren);
    +
    +    assertJQ(req("q", "foo_s:Bar",
    +        "fl","*",
    +        "sort","id asc",
    +        "wt","json"),
    +        childrenTests);
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPFieldNameException() throws Exception {
    +    final String errMsg = "contains: '" + PATH_SEP_CHAR + "' , which is reserved for the nested URP";
    +    thrown.expect(SolrException.class);
    +    indexSampleData(errDoc);
    +    thrown.expectMessage(errMsg);
    +  }
    +
    +  private void indexSampleData(String cmd) throws Exception {
    --- End diff --
    
    Sorry, kind of missed that method.
    My bad.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r200231503
  
    --- Diff: solr/core/src/test/org/apache/solr/update/TestNestedUpdateProcessor.java ---
    @@ -0,0 +1,171 @@
    +/*
    + * 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;
    +
    +import java.util.List;
    +
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.processor.NestedUpdateProcessorFactory;
    +import org.apache.solr.update.processor.UpdateRequestProcessor;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Rule;
    +import org.junit.Test;
    +import org.junit.rules.ExpectedException;
    +
    +public class TestNestedUpdateProcessor extends SolrTestCaseJ4 {
    +
    +  private static final char PATH_SEP_CHAR = '/';
    +  private static final String[] childrenIds = { "2", "3" };
    +  private static final String grandChildId = "4";
    +  private static final String jDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String noIdChildren = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String errDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children" + PATH_SEP_CHAR + "a\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  @Rule
    +  public ExpectedException thrown = ExpectedException.none();
    +
    +  @BeforeClass
    +  public static void beforeClass() throws Exception {
    +    initCore("solrconfig-update-processor-chains.xml", "schema15.xml");
    +  }
    +
    +  @Before
    +  public void before() throws Exception {
    +    assertU(delQ("*:*"));
    +    assertU(commit());
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPGrandChild() throws Exception {
    +    indexSampleData(jDoc);
    +
    +    assertJQ(req("q", IndexSchema.PATH_FIELD_NAME + ":*" + PATH_SEP_CHAR + "grandChild",
    +        "fl","*",
    +        "sort","id desc",
    +        "wt","json"),
    +        "/response/docs/[0]/id=='" + grandChildId + "'");
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPChildren() throws Exception {
    +    final String[] childrenTests = {"/response/docs/[0]/id=='" + childrenIds[0] + "'", "/response/docs/[1]/id=='" + childrenIds[1] + "'"};
    +    indexSampleData(jDoc);
    +
    +    assertJQ(req("q", IndexSchema.PATH_FIELD_NAME + ":children",
    +        "fl","*",
    +        "sort","id asc",
    +        "wt","json"),
    +        childrenTests);
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPChildrenWoId() throws Exception {
    +    final String parentId = "3";
    --- End diff --
    
    I think this doesn't test the recursive/layered nature of the ID generation.  Change the test to only have an ID at the root, then lets see that the rest of the IDs are what they ought to be?


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r199399224
  
    --- Diff: solr/core/src/test/org/apache/solr/update/TestNestedUpdateProcessor.java ---
    @@ -0,0 +1,189 @@
    +/*
    + * 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;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.util.ContentStream;
    +import org.apache.solr.common.util.ContentStreamBase;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.request.SolrRequestInfo;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.servlet.SolrRequestParsers;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Rule;
    +import org.junit.Test;
    +import org.junit.rules.ExpectedException;
    +
    +public class TestNestedUpdateProcessor extends SolrTestCaseJ4 {
    +
    +  private static final char PATH_SEP_CHAR = '/';
    +  private static final String[] childrenIds = { "2", "3" };
    +  private static final String grandChildId = "4";
    +  private static final String jDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String noIdChildren = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String errDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children" + PATH_SEP_CHAR + "a\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  @Rule
    +  public ExpectedException thrown = ExpectedException.none();
    --- End diff --
    
    It seems like if I use updateJ(), I do not build the SolrRequest have manually any more, which will only complicate the test. You will probably be able to see it when I push the new commits.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r197857625
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/DeeplyNestedUpdateProcessor.java ---
    @@ -0,0 +1,101 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Collection;
    +import java.util.EnumSet;
    +import java.util.Locale;
    +import java.util.Objects;
    +
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +import static org.apache.solr.update.processor.DeeplyNestedUpdateProcessorFactory.NestedFlag;
    +
    +public class DeeplyNestedUpdateProcessor extends UpdateRequestProcessor {
    +  private EnumSet<NestedFlag> fields;
    +  SolrQueryRequest req;
    +
    +
    +  protected DeeplyNestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, EnumSet<NestedFlag> fields, UpdateRequestProcessor next) {
    +    super(next);
    +    this.req = req;
    +    this.fields = fields;
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    processDocChildren(doc, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String fullPath) {
    +    for(SolrInputField field: doc.values()) {
    +      if(field.getFirstValue() instanceof SolrInputDocument) {
    +        Object val = field.getValue();
    +        fullPath = Objects.isNull(fullPath) ? field.getName(): String.format(Locale.ROOT,"%s.%s", fullPath, field.getName());
    +        if (val instanceof Collection) {
    --- End diff --
    
    SolrInputField is directly Iterable over its values, so you needn't call getValue and check its type.


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r200233243
  
    --- Diff: solr/core/src/java/org/apache/solr/schema/IndexSchema.java ---
    @@ -107,6 +107,8 @@
       public static final String LUCENE_MATCH_VERSION_PARAM = "luceneMatchVersion";
       public static final String MAX_CHARS = "maxChars";
       public static final String NAME = "name";
    +  public static final String PARENT_FIELD_NAME = "_NEST_PARENT_";
    --- End diff --
    
    Sounds good to me :+1: 


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r200829771
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessorFactory.java ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Collection;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessorFactory extends UpdateRequestProcessorFactory {
    +
    +  public UpdateRequestProcessor getInstance(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next ) {
    +    boolean storeParent = shouldStoreDocParent(req.getSchema());
    +    boolean storePath = shouldStoreDocPath(req.getSchema());
    +    if(!(storeParent || storePath)) {
    +      return next;
    +    }
    +    return new NestedUpdateProcessor(req, rsp, shouldStoreDocParent(req.getSchema()), shouldStoreDocPath(req.getSchema()), next);
    +  }
    +
    +  private static boolean shouldStoreDocParent(IndexSchema schema) {
    +    return schema.getFields().containsKey(IndexSchema.NEST_PARENT_FIELD_NAME);
    +  }
    +
    +  private static boolean shouldStoreDocPath(IndexSchema schema) {
    +    return schema.getFields().containsKey(IndexSchema.NEST_PATH_FIELD_NAME);
    +  }
    +}
    +
    +class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  private static final String PATH_SEP_CHAR = "/";
    +  private static final String NUM_SEP_CHAR = ",";
    +  private static final String SINGLE_VALUE_CHAR = "s";
    +  private boolean storePath;
    +  private boolean storeParent;
    +  private String uniqueKeyFieldName;
    +
    +
    +  protected NestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, boolean storeParent, boolean storePath, UpdateRequestProcessor next) {
    +    super(next);
    +    this.storeParent = storeParent;
    +    this.storePath = storePath;
    +    this.uniqueKeyFieldName = req.getSchema().getUniqueKeyField().getName();
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    processDocChildren(doc, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String fullPath) {
    +    int childNum = 0;
    +    for(SolrInputField field: doc.values()) {
    +      boolean isSingleVal = !(field.getValue() instanceof Collection);
    +      for(Object val: field) {
    +        if(!(val instanceof SolrInputDocument)) {
    +          // either all collection items are child docs or none are.
    +          break;
    +        }
    +        final String fieldName = field.getName();
    +
    +        if(fieldName.contains(PATH_SEP_CHAR)) {
    +          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field name: '" + fieldName
    +              + "' contains: '" + PATH_SEP_CHAR + "' , which is reserved for the nested URP");
    +        }
    +        final String sChildNum = isSingleVal ? SINGLE_VALUE_CHAR: String.valueOf(childNum);
    +        final String lastPath = fieldName + NUM_SEP_CHAR + sChildNum + NUM_SEP_CHAR;
    --- End diff --
    
    The trailing num sep char is there so we can query all documents in the specified _NEST_PATH_: path#*#


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r199386009
  
    --- Diff: solr/core/src/test/org/apache/solr/update/TestNestedUpdateProcessor.java ---
    @@ -0,0 +1,189 @@
    +/*
    + * 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;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.solr.SolrTestCaseJ4;
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.util.ContentStream;
    +import org.apache.solr.common.util.ContentStreamBase;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.request.SolrRequestInfo;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.servlet.SolrRequestParsers;
    +import org.junit.Before;
    +import org.junit.BeforeClass;
    +import org.junit.Rule;
    +import org.junit.Test;
    +import org.junit.rules.ExpectedException;
    +
    +public class TestNestedUpdateProcessor extends SolrTestCaseJ4 {
    +
    +  private static final char PATH_SEP_CHAR = '/';
    +  private static final String[] childrenIds = { "2", "3" };
    +  private static final String grandChildId = "4";
    +  private static final String jDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String noIdChildren = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children\": [\n" +
    +      "                {\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  private static final String errDoc = "{\n" +
    +      "    \"add\": {\n" +
    +      "        \"doc\": {\n" +
    +      "            \"id\": \"1\",\n" +
    +      "            \"children" + PATH_SEP_CHAR + "a\": [\n" +
    +      "                {\n" +
    +      "                    \"id\": \"2\",\n" +
    +      "                    \"foo_s\": \"Yaz\"\n" +
    +      "                    \"grandChild\": \n" +
    +      "                          {\n" +
    +      "                             \"id\": \""+ grandChildId + "\",\n" +
    +      "                             \"foo_s\": \"Jazz\"\n" +
    +      "                          },\n" +
    +      "                },\n" +
    +      "                {\n" +
    +      "                    \"id\": \"3\",\n" +
    +      "                    \"foo_s\": \"Bar\"\n" +
    +      "                }\n" +
    +      "            ]\n" +
    +      "        }\n" +
    +      "    }\n" +
    +      "}";
    +
    +  @Rule
    +  public ExpectedException thrown = ExpectedException.none();
    +
    +  @BeforeClass
    +  public static void beforeClass() throws Exception {
    +    initCore("solrconfig-update-processor-chains.xml", "schema15.xml");
    +  }
    +
    +  @Before
    +  public void before() throws Exception {
    +    assertU(delQ("*:*"));
    +    assertU(commit());
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPGrandChild() throws Exception {
    +    indexSampleData(jDoc);
    +
    +    assertJQ(req("q", IndexSchema.PATH_FIELD_NAME + ":*" + PATH_SEP_CHAR + "grandChild",
    +        "fl","*",
    +        "sort","id desc",
    +        "wt","json"),
    +        "/response/docs/[0]/id=='" + grandChildId + "'");
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPChildren() throws Exception {
    +    final String[] childrenTests = {"/response/docs/[0]/id=='" + childrenIds[0] + "'", "/response/docs/[1]/id=='" + childrenIds[1] + "'"};
    +    indexSampleData(jDoc);
    +
    +    assertJQ(req("q", IndexSchema.PATH_FIELD_NAME + ":children",
    +        "fl","*",
    +        "sort","id asc",
    +        "wt","json"),
    +        childrenTests);
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPChildrenWoId() throws Exception {
    +    final String[] childrenTests = {"/response/docs/[0]/id=='1" + PATH_SEP_CHAR + "children" + PATH_SEP_CHAR + "2'"};
    +    indexSampleData(noIdChildren);
    +
    +    assertJQ(req("q", "foo_s:Bar",
    +        "fl","*",
    +        "sort","id asc",
    +        "wt","json"),
    +        childrenTests);
    +  }
    +
    +  @Test
    +  public void testDeeplyNestedURPFieldNameException() throws Exception {
    +    final String errMsg = "contains: '" + PATH_SEP_CHAR + "' , which is reserved for the nested URP";
    +    thrown.expect(SolrException.class);
    +    indexSampleData(errDoc);
    +    thrown.expectMessage(errMsg);
    +  }
    +
    +  private void indexSampleData(String cmd) throws Exception {
    --- End diff --
    
    I'm skeptical we need to write this much code for something routine.  See `updateJ()`; can you just call that?


---

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


[GitHub] lucene-solr pull request #410: SOLR-12441: add deeply nested URP for nested ...

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/410#discussion_r200871182
  
    --- Diff: solr/core/src/java/org/apache/solr/update/processor/NestedUpdateProcessorFactory.java ---
    @@ -0,0 +1,126 @@
    +/*
    + * 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.io.IOException;
    +import java.util.Collection;
    +
    +import org.apache.solr.common.SolrException;
    +import org.apache.solr.common.SolrInputDocument;
    +import org.apache.solr.common.SolrInputField;
    +import org.apache.solr.request.SolrQueryRequest;
    +import org.apache.solr.response.SolrQueryResponse;
    +import org.apache.solr.schema.IndexSchema;
    +import org.apache.solr.update.AddUpdateCommand;
    +
    +public class NestedUpdateProcessorFactory extends UpdateRequestProcessorFactory {
    +
    +  public UpdateRequestProcessor getInstance(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next ) {
    +    boolean storeParent = shouldStoreDocParent(req.getSchema());
    +    boolean storePath = shouldStoreDocPath(req.getSchema());
    +    if(!(storeParent || storePath)) {
    +      return next;
    +    }
    +    return new NestedUpdateProcessor(req, rsp, shouldStoreDocParent(req.getSchema()), shouldStoreDocPath(req.getSchema()), next);
    +  }
    +
    +  private static boolean shouldStoreDocParent(IndexSchema schema) {
    +    return schema.getFields().containsKey(IndexSchema.NEST_PARENT_FIELD_NAME);
    +  }
    +
    +  private static boolean shouldStoreDocPath(IndexSchema schema) {
    +    return schema.getFields().containsKey(IndexSchema.NEST_PATH_FIELD_NAME);
    +  }
    +}
    +
    +class NestedUpdateProcessor extends UpdateRequestProcessor {
    +  private static final String PATH_SEP_CHAR = "/";
    +  private static final String NUM_SEP_CHAR = "#";
    +  private static final String SINGULAR_VALUE_CHAR = " ";
    +  private boolean storePath;
    +  private boolean storeParent;
    +  private String uniqueKeyFieldName;
    +
    +
    +  protected NestedUpdateProcessor(SolrQueryRequest req, SolrQueryResponse rsp, boolean storeParent, boolean storePath, UpdateRequestProcessor next) {
    +    super(next);
    +    this.storeParent = storeParent;
    +    this.storePath = storePath;
    +    this.uniqueKeyFieldName = req.getSchema().getUniqueKeyField().getName();
    +  }
    +
    +  @Override
    +  public void processAdd(AddUpdateCommand cmd) throws IOException {
    +    SolrInputDocument doc = cmd.getSolrInputDocument();
    +    processDocChildren(doc, null);
    +    super.processAdd(cmd);
    +  }
    +
    +  private void processDocChildren(SolrInputDocument doc, String fullPath) {
    +    for(SolrInputField field: doc.values()) {
    +      int childNum = 0;
    +      boolean isSingleVal = !(field.getValue() instanceof Collection);
    +      for(Object val: field) {
    +        if(!(val instanceof SolrInputDocument)) {
    +          // either all collection items are child docs or none are.
    +          break;
    +        }
    +        final String fieldName = field.getName();
    +
    +        if(fieldName.contains(PATH_SEP_CHAR)) {
    +          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Field name: '" + fieldName
    +              + "' contains: '" + PATH_SEP_CHAR + "' , which is reserved for the nested URP");
    +        }
    +        final String sChildNum = isSingleVal ? SINGULAR_VALUE_CHAR : String.valueOf(childNum);
    +        SolrInputDocument cDoc = (SolrInputDocument) val;
    +        if(!cDoc.containsKey(uniqueKeyFieldName)) {
    +          String parentDocId = doc.getField(uniqueKeyFieldName).getFirstValue().toString();
    +          cDoc.setField(uniqueKeyFieldName, generateChildUniqueId(parentDocId, fieldName, sChildNum));
    +        }
    +        final String lastPath = fieldName + NUM_SEP_CHAR + sChildNum + NUM_SEP_CHAR;
    +        final String jointPath = fullPath == null ? lastPath : fullPath + PATH_SEP_CHAR + lastPath;
    +        processChildDoc((SolrInputDocument) val, doc, jointPath);
    --- End diff --
    
    Can you please add a comment showing what an example multi-layer (child of child of root) might look like?  Also FWIW "jointPath" as a variable name confuses me; I just don't know what the origin of that choice is.  Perhaps childDocPath?


---

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