You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2021/02/11 18:44:53 UTC

[GitHub] [lucene-solr] epugh opened a new pull request #2356: SOLR-15152: Export Tool should export nested docs cleanly in .json, .jsonl, and javabin

epugh opened a new pull request #2356:
URL: https://github.com/apache/lucene-solr/pull/2356


   # Description
   
   Export tool says it uses json, but it's actually a json lines format.   It ignores anonymous and nested docs.
   
   # Solution
   
   * Tweaked the writer to properly handle anonymous and regular nested docs when exporting data.
   * Renamed the existing `json` format to `jsonl`, and introduced a proper `json` format.
   * Introduce explicit DocSinks per format, `json`, `jsonl`, and `javabin`.
   
   Now, with the `json` format you can export and then reimport the Solr docs, including with child docs.
   
   # Tests
   
   I've added a new `TestExportToolWithNestedDocs`, and extended the existing `TestExportTool` tests.  The setup for the tests was quite different, so I didn't make them all one file.
   
   # Checklist
   
   Please review the following and check all that apply:
   
   - [ X] I have reviewed the guidelines for [How to Contribute](https://wiki.apache.org/solr/HowToContribute) and my code conforms to the standards described there to the best of my ability.
   - [ X] I have created a Jira issue and added the issue ID to my pull request title.
   - [ X] I have given Solr maintainers [access](https://help.github.com/en/articles/allowing-changes-to-a-pull-request-branch-created-from-a-fork) to contribute to my PR branch. (optional but recommended)
   - [ X] I have developed this patch against the `master` branch.
   - [ ] I have run `./gradlew check`.
   - [ X] I have added tests for my changes.
   - [ ] I have added documentation for the [Ref Guide](https://github.com/apache/lucene-solr/tree/master/solr/solr-ref-guide) (for Solr changes only).
   


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

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



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


[GitHub] [lucene-solr] epugh commented on pull request #2356: SOLR-15152: Export Tool should export nested docs cleanly in .json, .jsonl, and javabin

Posted by GitBox <gi...@apache.org>.
epugh commented on pull request #2356:
URL: https://github.com/apache/lucene-solr/pull/2356#issuecomment-778497757


   In more testing, I don't have the `javabin` format working yet...   It's not roundtripping exports with the javabin format back into solr.


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

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



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


[GitHub] [lucene-solr] epugh commented on a change in pull request #2356: SOLR-15152: Export Tool should export nested docs cleanly in .json, .jsonl, and javabin

Posted by GitBox <gi...@apache.org>.
epugh commented on a change in pull request #2356:
URL: https://github.com/apache/lucene-solr/pull/2356#discussion_r575534307



##########
File path: solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformerFactory.java
##########
@@ -160,8 +161,18 @@ protected static String processPathHierarchyQueryString(String queryString) {
     int indexOfLastPathSepChar = queryString.lastIndexOf(PATH_SEP_CHAR, indexOfFirstColon);
     if (indexOfLastPathSepChar < 0) {
       // regular filter, not hierarchy based.
-      return ClientUtils.escapeQueryChars(queryString.substring(0, indexOfFirstColon))

Review comment:
       @dsmiley I need your help on this!  I am struggling on childFilters...   `level_i:1` works great with the escaping, but queries like `level_i:[1 TO 3]` fail due to the escaping, I get werid "can parse number" errors.   Same if I have `type_s:Chocolate OR type_s:Regular`, but if I skip that escaping then things work.   
   
   This is definitly at the very outer edges of my knowledge. Would love to chat about how to fix this.




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

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



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


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #2356: SOLR-15152: Export Tool should export nested docs cleanly in .json, .jsonl, and javabin

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #2356:
URL: https://github.com/apache/lucene-solr/pull/2356#discussion_r575699438



##########
File path: solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java
##########
@@ -1935,21 +1935,22 @@ public void testChildDoctransformer() throws IOException, SolrServerException {
     Map<String,SolrInputDocument> allDocs = new HashMap<>();
 
     for (int i =0; i < numRootDocs; i++) {
-      client.add(genNestedDocuments(allDocs, 0, maxDepth));
+      client.add(generateNestedDocuments(allDocs, 0, maxDepth));
     }
 
     client.commit();
 
     // sanity check
-    SolrQuery q = new SolrQuery("q", "*:*", "indent", "true");
+    SolrQuery q = new SolrQuery("q", "*:*", "rows", "0");
     QueryResponse resp = client.query(q);
     assertEquals("Doc count does not match",
         allDocs.size(), resp.getResults().getNumFound());
 
 
-    // base check - we know there is an exact number of these root docs
-    q = new SolrQuery("q","level_i:0", "indent", "true");
-    q.setFields("*", "[child parentFilter=\"level_i:0\"]");
+    // base check - we know there the exact number of these root docs
+    q = new SolrQuery("{!parent which=\"*:* -_nest_path_:*\"}");
+    q.addField("*,[child limit=\"-1\"]");

Review comment:
       I think it was clearer before with setFields (plural) as you are adding two fields

##########
File path: solr/core/src/java/org/apache/solr/response/transform/ChildDocTransformerFactory.java
##########
@@ -160,8 +161,18 @@ protected static String processPathHierarchyQueryString(String queryString) {
     int indexOfLastPathSepChar = queryString.lastIndexOf(PATH_SEP_CHAR, indexOfFirstColon);
     if (indexOfLastPathSepChar < 0) {
       // regular filter, not hierarchy based.
-      return ClientUtils.escapeQueryChars(queryString.substring(0, indexOfFirstColon))

Review comment:
       I think there should **not** have been escaping here in the first place.  Unless the user is using the special syntax, we should handle it plainly like we do everywhere else -- simple.  It may mean the user has to escape characters like '/' but that's normal.  In a Java String that has an embedded local-params for the child doc transformer, that winds up being four back-slashes.  So be it.  It may seem like we're trying to do the user a favor by escaping what we see but I think it's actually more confusing to reason about because it's inconsistent.  Besides, there are other ways of constructing the query syntax that results in zero escaping.  like `childFilter='{!field f=_nest_path_}/toppings'`  So that's longer but no escaping at least.  Up to the user's prerogative.
   CC @moshebla
   
   I'll fix this in your PR momentarily.




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

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



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


[GitHub] [lucene-solr] epugh commented on pull request #2356: SOLR-15152: Export Tool should export nested docs cleanly in .json, .jsonl, and javabin

Posted by GitBox <gi...@apache.org>.
epugh commented on pull request #2356:
URL: https://github.com/apache/lucene-solr/pull/2356#issuecomment-802415285


   closing in favour of https://github.com/apache/solr/pull/33


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

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



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


[GitHub] [lucene-solr] epugh closed pull request #2356: SOLR-15152: Export Tool should export nested docs cleanly in .json, .jsonl, and javabin

Posted by GitBox <gi...@apache.org>.
epugh closed pull request #2356:
URL: https://github.com/apache/lucene-solr/pull/2356


   


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

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



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


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #2356: SOLR-15152: Export Tool should export nested docs cleanly in .json, .jsonl, and javabin

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #2356:
URL: https://github.com/apache/lucene-solr/pull/2356#discussion_r579854438



##########
File path: solr/core/src/java/org/apache/solr/util/ExportTool.java
##########
@@ -319,6 +369,91 @@ private Object constructDateStr(Object field) {
       return field;
     }
   }
+  
+  static class JsonlSink extends DocsSink {
+    private CharArr charArr = new CharArr(1024 * 2);
+    JSONWriter jsonWriter = new JSONWriter(charArr, -1);
+    private Writer writer;
+
+    public JsonlSink(Info info) {
+      this.info = info;
+    }
+
+    @Override
+    public void start() throws IOException {
+      fos = new FileOutputStream(info.out);
+      if(info.out.endsWith(".jsonl.gz")) {
+        fos = new GZIPOutputStream(fos);
+      }
+      if (info.bufferSize > 0) {
+        fos = new BufferedOutputStream(fos, info.bufferSize);
+      }
+      writer = new OutputStreamWriter(fos, StandardCharsets.UTF_8);
+
+    }
+
+    @Override
+    public void end() throws IOException {
+      writer.flush();
+      fos.flush();
+      fos.close();
+    }
+
+    @Override
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public synchronized void accept(SolrDocument doc) throws IOException {
+      charArr.reset();
+      int mapSize = doc._size();
+      if(doc.hasChildDocuments()) {
+        mapSize ++;
+      }
+      Map m = new LinkedHashMap(mapSize);
+      
+      doc.forEach((s, field) -> {
+        if (s.equals("_version_") || s.equals("_root_")) return;
+        if (field instanceof List) {
+          if (((List) field).size() == 1) {
+            field = ((List) field).get(0);
+          }
+        }
+        field = constructDateStr(field);
+        if (field instanceof List) {
+          List list = (List) field;
+          if (hasdate(list)) {
+            ArrayList<Object> listCopy = new ArrayList<>(list.size());
+            for (Object o : list) listCopy.add(constructDateStr(o));
+            field = listCopy;
+          }
+        }
+        m.put(s, field);
+      });
+      if (doc.hasChildDocuments()) {
+        m.put("_childDocuments_", doc.getChildDocuments());
+      }
+      jsonWriter.write(m);
+      writer.write(charArr.getArray(), charArr.getStart(), charArr.getEnd());
+      writer.append('\n');
+      super.accept(doc);
+    }
+
+    private boolean hasdate(@SuppressWarnings({"rawtypes"})List list) {
+      boolean hasDate = false;
+      for (Object o : list) {
+        if(o instanceof Date){
+          hasDate = true;
+          break;
+        }
+      }
+      return hasDate;
+    }
+
+    private Object constructDateStr(Object field) {
+      if (field instanceof Date) {
+        field = DateTimeFormatter.ISO_INSTANT.format(Instant.ofEpochMilli(((Date) field).getTime()));

Review comment:
       date.toInstant().toString() is equivalent.
   Oh yeah; this is copy-pasted code :-/  change or not as you wish.

##########
File path: solr/core/src/java/org/apache/solr/util/ExportTool.java
##########
@@ -129,23 +130,51 @@ public void setLimit(String maxDocsStr) {
     }
 
     public void setOutFormat(String out, String format) {
-      this.format = format;
-      if (format == null) format = "jsonl";
+      if (format == null) {
+        format = "json";
+      }      
       if (!formats.contains(format)) {
         throw new IllegalArgumentException("format must be one of :" + formats);
       }
+      this.format = format;
 
       this.out = out;
       if (this.out == null) {
-        this.out = JAVABIN.equals(format) ?
-            coll + ".javabin" :
-            coll + ".json";
+        this.out = coll + getOutExtension();
       }
 
     }
+    
+    String getOutExtension() {
+      String extension = null;
+      switch (format) {
+        case JAVABIN: 
+          extension = ".javabin";
+          break;
+        case JSON: 
+          extension = ".json";
+          break;
+        case "jsonl":
+          extension = ".jsonl";
+          break;
+      }
+      return extension;
+    }
 
     DocsSink getSink() {
-      return JAVABIN.equals(format) ? new JavabinSink(this) : new JsonSink(this);
+      DocsSink docSink = null;
+      switch (format) {
+        case JAVABIN: 

Review comment:
       I think I'd prefer that these "case" values be consistently represented -- either string literals or references to something.  It's okay if they are all string literals... I'm not a believer in the school of thought that all literals must be constantly defined; it's hard to read such.

##########
File path: solr/solr-ref-guide/src/solr-control-script-reference.adoc
##########
@@ -876,8 +876,8 @@ Examples of this command:
 
 == Exporting Documents to a File
 
-The `export` command will allow you to export documents from a collection in either JSON or Javabin format.
-All documents can be exported, or only those that match a query.
+The `export` command will allow you to export documents from a collection in JSON, https://jsonlines.org/[JSON Lines], or Javabin format.

Review comment:
       "Javabin" us unlikely to be known by the user.

##########
File path: solr/solrj/src/test-files/solrj/solr/configsets/nested/anonymous/conf/solrconfig.xml
##########
@@ -0,0 +1,46 @@
+<?xml version="1.0" ?>
+
+<!--
+ 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.
+-->
+
+<!-- Minimal solrconfig.xml with /select, /admin and /update only -->

Review comment:
       It'd be nice to use a copy of solrconfig-minimal.xml instead.  It's a shame to need/want to copy it at all :-/  

##########
File path: solr/core/src/test/org/apache/solr/util/TestExportToolWithNestedDocs.java
##########
@@ -0,0 +1,293 @@
+/*
+ * 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.util;
+
+import java.io.File;
+import java.nio.file.Files;
+import java.util.Arrays;
+
+import org.apache.solr.SolrTestCaseHS;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrInputDocument;
+import org.junit.After;
+import org.junit.BeforeClass;
+
+@SolrTestCaseJ4.SuppressSSL
+public class TestExportToolWithNestedDocs extends SolrCloudTestCase {
+  
+  public static final String ANON_KIDS_CONFIG = "anon_kids_configset";
+  public static final String NESTED_KIDS_CONFIG = "nested_kids_configset";
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1)
+      .addConfig(ANON_KIDS_CONFIG, configset("nested/anonymous"))
+      .addConfig(NESTED_KIDS_CONFIG, configset("nested/regular"))
+      .configure();
+  }
+
+  @After
+  public void cleanCollections() throws Exception {
+    cluster.deleteAllCollections();
+  }
+
+  /**
+   * This test is inspired by the IndexingNestedDocuments.java unit test that 
+   * demonstrates creating Anonymous Children docs, and then confirming the
+   * export format.
+   */
+  public void testIndexingAnonKids() throws Exception {
+    final String collection = "test_anon";
+    CollectionAdminRequest.createCollection(collection, ANON_KIDS_CONFIG, 1, 1)
+        .setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
+        .process(cluster.getSolrClient());
+    cluster.getSolrClient().setDefaultCollection(collection);
+    
+    final SolrClient client = cluster.getSolrClient();
+
+    final SolrInputDocument p1 = new SolrInputDocument();
+    p1.setField("id", "P11!prod");
+    p1.setField("type_s", "PRODUCT");
+    p1.setField("name_s", "Swingline Stapler");
+    p1.setField("description_t", "The Cadillac of office staplers ...");
+    {
+      final SolrInputDocument s1 = new SolrInputDocument();
+      s1.setField("id", "P11!S21");
+      s1.setField("type_s", "SKU");
+      s1.setField("color_s", "RED");
+      s1.setField("price_i", 42);
+      { 
+        final SolrInputDocument m1 = new SolrInputDocument();
+        m1.setField("id", "P11!D41");
+        m1.setField("type_s", "MANUAL");
+        m1.setField("name_s", "Red Swingline Brochure");
+        m1.setField("pages_i", 1);
+        m1.setField("content_t", "...");
+
+        s1.addChildDocument(m1);
+      }
+
+      final SolrInputDocument s2 = new SolrInputDocument();
+      s2.setField("id", "P11!S31");
+      s2.setField("type_s", "SKU");
+      s2.setField("color_s", "BLACK");
+      s2.setField("price_i", 3);
+
+      final SolrInputDocument m1 = new SolrInputDocument();
+      m1.setField("id", "P11!D51");
+      m1.setField("type_s", "MANUAL");
+      m1.setField("name_s", "Quick Reference Guide");
+      m1.setField("pages_i", 1);
+      m1.setField("content_t", "How to use your stapler ...");
+
+      final SolrInputDocument m2 = new SolrInputDocument();
+      m2.setField("id", "P11!D61");
+      m2.setField("type_s", "MANUAL");
+      m2.setField("name_s", "Warranty Details");
+      m2.setField("pages_i", 42);
+      m2.setField("content_t", "... lifetime guarantee ...");
+
+      p1.addChildDocuments(Arrays.asList(s1, s2, m1, m2));
+    }
+    
+    client.add(p1);
+ 
+    client.commit();
+
+    String url = cluster.getRandomJetty(random()).getBaseUrl() + "/" + collection;
+    
+    String tmpFileLoc = new File(cluster.getBaseDir().toFile().getAbsolutePath() +
+        File.separator).getPath();
+    
+    ExportTool.Info info = new ExportTool.MultiThreadedRunner(url);
+    String absolutePath = tmpFileLoc + collection + random().nextInt(100000) + ".jsonl";
+    info.setOutFormat(absolutePath, "jsonl");
+    info.setLimit("-1");
+    info.query = "description_t:Cadillac";
+    info.fields = "*,[child parentFilter='type_s:PRODUCT']";
+    info.exportDocs();
+    
+    assertEquals(1, info.docsWritten.get());
+    
+    String jsonOutput = Files.readString(new File(info.out).toPath());

Review comment:
       Use Path instead of File please.  Here, just use Path.of(info.out).  Ideally info.out would already be a Path but that's maybe too much refactoring to do here now.

##########
File path: solr/core/src/java/org/apache/solr/util/ExportTool.java
##########
@@ -319,6 +369,91 @@ private Object constructDateStr(Object field) {
       return field;
     }
   }
+  
+  static class JsonlSink extends DocsSink {
+    private CharArr charArr = new CharArr(1024 * 2);
+    JSONWriter jsonWriter = new JSONWriter(charArr, -1);
+    private Writer writer;
+
+    public JsonlSink(Info info) {
+      this.info = info;
+    }
+
+    @Override
+    public void start() throws IOException {
+      fos = new FileOutputStream(info.out);
+      if(info.out.endsWith(".jsonl.gz")) {
+        fos = new GZIPOutputStream(fos);
+      }
+      if (info.bufferSize > 0) {
+        fos = new BufferedOutputStream(fos, info.bufferSize);
+      }
+      writer = new OutputStreamWriter(fos, StandardCharsets.UTF_8);
+
+    }
+
+    @Override
+    public void end() throws IOException {
+      writer.flush();
+      fos.flush();
+      fos.close();
+    }
+
+    @Override
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public synchronized void accept(SolrDocument doc) throws IOException {
+      charArr.reset();
+      int mapSize = doc._size();
+      if(doc.hasChildDocuments()) {
+        mapSize ++;
+      }
+      Map m = new LinkedHashMap(mapSize);
+      
+      doc.forEach((s, field) -> {

Review comment:
       as a matter of taste perhaps, I don't use this forEach thing unless the lambda I'm passing is very short; otherwise I use standard for loops.

##########
File path: solr/core/src/java/org/apache/solr/util/ExportTool.java
##########
@@ -319,6 +369,91 @@ private Object constructDateStr(Object field) {
       return field;
     }
   }
+  
+  static class JsonlSink extends DocsSink {

Review comment:
       There's too much duplication here with the Json format.  Please either extend JsonSink for give that format this option in a toggle-able way.  I suppose the latter may be easier/cleaner.

##########
File path: solr/core/src/java/org/apache/solr/util/ExportTool.java
##########
@@ -257,16 +287,20 @@ public JsonSink(Info info) {
     @Override
     public void start() throws IOException {
       fos = new FileOutputStream(info.out);
-      if(info.out.endsWith(".json.gz") || info.out.endsWith(".json.")) fos = new GZIPOutputStream(fos);
+      if(info.out.endsWith(".json.gz")) {

Review comment:
       Why not just check the ".gz" part in case we want to support jsonnl.gz or others?

##########
File path: solr/core/src/test/org/apache/solr/util/TestExportToolWithNestedDocs.java
##########
@@ -0,0 +1,293 @@
+/*
+ * 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.util;
+
+import java.io.File;
+import java.nio.file.Files;
+import java.util.Arrays;
+
+import org.apache.solr.SolrTestCaseHS;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrInputDocument;
+import org.junit.After;
+import org.junit.BeforeClass;
+
+@SolrTestCaseJ4.SuppressSSL
+public class TestExportToolWithNestedDocs extends SolrCloudTestCase {
+  
+  public static final String ANON_KIDS_CONFIG = "anon_kids_configset";
+  public static final String NESTED_KIDS_CONFIG = "nested_kids_configset";
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1)
+      .addConfig(ANON_KIDS_CONFIG, configset("nested/anonymous"))
+      .addConfig(NESTED_KIDS_CONFIG, configset("nested/regular"))
+      .configure();
+  }
+
+  @After
+  public void cleanCollections() throws Exception {
+    cluster.deleteAllCollections();
+  }
+
+  /**
+   * This test is inspired by the IndexingNestedDocuments.java unit test that 
+   * demonstrates creating Anonymous Children docs, and then confirming the
+   * export format.
+   */
+  public void testIndexingAnonKids() throws Exception {
+    final String collection = "test_anon";
+    CollectionAdminRequest.createCollection(collection, ANON_KIDS_CONFIG, 1, 1)
+        .setPerReplicaState(SolrCloudTestCase.USE_PER_REPLICA_STATE)
+        .process(cluster.getSolrClient());
+    cluster.getSolrClient().setDefaultCollection(collection);
+    
+    final SolrClient client = cluster.getSolrClient();
+
+    final SolrInputDocument p1 = new SolrInputDocument();
+    p1.setField("id", "P11!prod");
+    p1.setField("type_s", "PRODUCT");
+    p1.setField("name_s", "Swingline Stapler");
+    p1.setField("description_t", "The Cadillac of office staplers ...");
+    {
+      final SolrInputDocument s1 = new SolrInputDocument();
+      s1.setField("id", "P11!S21");
+      s1.setField("type_s", "SKU");
+      s1.setField("color_s", "RED");
+      s1.setField("price_i", 42);
+      { 
+        final SolrInputDocument m1 = new SolrInputDocument();
+        m1.setField("id", "P11!D41");
+        m1.setField("type_s", "MANUAL");
+        m1.setField("name_s", "Red Swingline Brochure");
+        m1.setField("pages_i", 1);
+        m1.setField("content_t", "...");
+
+        s1.addChildDocument(m1);
+      }
+
+      final SolrInputDocument s2 = new SolrInputDocument();
+      s2.setField("id", "P11!S31");
+      s2.setField("type_s", "SKU");
+      s2.setField("color_s", "BLACK");
+      s2.setField("price_i", 3);
+
+      final SolrInputDocument m1 = new SolrInputDocument();
+      m1.setField("id", "P11!D51");
+      m1.setField("type_s", "MANUAL");
+      m1.setField("name_s", "Quick Reference Guide");
+      m1.setField("pages_i", 1);
+      m1.setField("content_t", "How to use your stapler ...");
+
+      final SolrInputDocument m2 = new SolrInputDocument();
+      m2.setField("id", "P11!D61");
+      m2.setField("type_s", "MANUAL");
+      m2.setField("name_s", "Warranty Details");
+      m2.setField("pages_i", 42);
+      m2.setField("content_t", "... lifetime guarantee ...");
+
+      p1.addChildDocuments(Arrays.asList(s1, s2, m1, m2));
+    }
+    
+    client.add(p1);
+ 
+    client.commit();
+
+    String url = cluster.getRandomJetty(random()).getBaseUrl() + "/" + collection;
+    
+    String tmpFileLoc = new File(cluster.getBaseDir().toFile().getAbsolutePath() +

Review comment:
       Why use "File" class?  Is getting the absolute path necessary?

##########
File path: solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/IndexingNestedDocuments.java
##########
@@ -40,12 +39,13 @@
  */
 public class IndexingNestedDocuments extends SolrCloudTestCase {
   public static final String ANON_KIDS_CONFIG = "anon_kids_configset";
+  public static final String NESTED_KIDS_CONFIG = "nested_kids_configset";
   @BeforeClass
   public static void setupCluster() throws Exception {
+    Path solrJConfigsets = getFile("solrj").toPath().resolve("solr/configsets");

Review comment:
       ```suggestion
       Path solrJConfigsets = getFile("solrj").toPath().resolve("solr", "configsets");
   ```

##########
File path: solr/core/src/java/org/apache/solr/util/ExportTool.java
##########
@@ -276,9 +310,13 @@ public void end() throws IOException {
     @SuppressWarnings({"unchecked", "rawtypes"})
     public synchronized void accept(SolrDocument doc) throws IOException {
       charArr.reset();
-      Map m = new LinkedHashMap(doc.size());
+      int mapSize = doc._size();
+      if(doc.hasChildDocuments()) {
+        mapSize ++;
+      }
+      Map m = new LinkedHashMap(mapSize);
       doc.forEach((s, field) -> {
-        if (s.equals("_version_") || s.equals("_roor_")) return;
+        if (s.equals("_version_") || s.equals("_root_")) return;

Review comment:
       This mistake would have been caught if IndexSchema constants had been used.

##########
File path: solr/solrj/src/test/org/apache/solr/client/ref_guide_examples/IndexingNestedDocuments.java
##########
@@ -40,12 +39,13 @@
  */
 public class IndexingNestedDocuments extends SolrCloudTestCase {
   public static final String ANON_KIDS_CONFIG = "anon_kids_configset";
+  public static final String NESTED_KIDS_CONFIG = "nested_kids_configset";
   @BeforeClass
   public static void setupCluster() throws Exception {
+    Path solrJConfigsets = getFile("solrj").toPath().resolve("solr/configsets");
     configureCluster(1)
-      // when indexing 'anonymous' kids, we need a schema that doesn't use _nest_path_ so
-      // that we can use [child] transformer with a parentFilter...
-      .addConfig(ANON_KIDS_CONFIG, new File(ExternalPaths.TECHPRODUCTS_CONFIGSET).toPath())
+      .addConfig(ANON_KIDS_CONFIG, solrJConfigsets.resolve("nested/anonymous/conf"))

Review comment:
       More path changes needed here.

##########
File path: solr/solrj/src/test/org/apache/solr/client/solrj/SolrExampleTests.java
##########
@@ -2065,9 +2074,9 @@ public void testChildDoctransformer() throws IOException, SolrServerException {
         q = new SolrQuery("q", "name:" + name, "indent", "true");
       }
       q.setFilterQueries(parentFilter);
-      q.setFields("id, level_i, [child parentFilter=\"" + parentFilter +
-                  "\" childFilter=\"" + childFilter + 
-                  "\" limit=\"" + maxKidCount + "\"],name");
+      q.setFields("id, level_i, [child " +

Review comment:
       Can you add newlines at the comma of fields instead of in the middle of the transformer?
   You could call q.addField multiple times instead of setFields; which would look nicer?

##########
File path: solr/solr-ref-guide/src/solr-control-script-reference.adoc
##########
@@ -902,25 +906,30 @@ The number of documents to export. The default is `100`. The value `-1` will exp
 
 *Examples*
 
-Export all documents from a collection `gettingstarted`:
+Export all documents from a collection `gettingstarted` into a file called `gettingstarted.json`:
 
 [source,bash]
-bin/solr export -url http://localhost:8983/solr/gettingstarted limit -1
+bin/solr export -url http://localhost:8983/solr/gettingstarted -limit -1
 
 Export all documents of collection `gettingstarted` into a file called `1MDocs.json.gz` as a zipped JSON file:
 
 [source,bash]
 ----
-bin/solr export -url http://localhost:8983/solr/gettingstarted -1 -out 1MDocs.json.gz
+bin/solr export -url http://localhost:8983/solr/gettingstarted -limit -1 -out 1MDocs.json.gz
 ----
 
 === Importing Documents to a Collection
 
-Once you have exported documents in a file, you can use the <<updatehandlers-in-solrconfig.adoc#,/update request handler>> to import them to a new Solr collection.
+Once you have exported documents in a file, you can use the <<updatehandlers-in-solrconfig.adoc#updatehandlers-in-solrconfig,/update request handler>> to import them to a new Solr collection.
+Notice the different endpoints used depending on the format.  
+
+*Example: import `json` files*
+
+`curl -X POST --header "Content-Type: application/json" -d @gettingstarted.json http://localhost:8983/solr/gettingstarted/update/json?commit=true`
 
 *Example: import `jsonl` files*
 
-`curl -X POST -d @gettingstarted.json http://localhost:8983/solr/gettingstarted/update/json/docs?commit=true`
+`curl -X POST --header "Content-Type: application/json" -d @gettingstarted.json http://localhost:8983/solr/gettingstarted/update/json/docs?commit=true`

Review comment:
       Just an FYI... Solr's very own "bin/post" actually has advantages to curl.  https://issues.apache.org/jira/browse/SOLR-8937  Looking back at that old-ish JIRA, I see that was the stdin case.  I'm not sure if curl is smart enough to not put the whole thing in RAM.  I think "bin/post" is smart about it though.




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

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



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


[GitHub] [lucene-solr] epugh edited a comment on pull request #2356: SOLR-15152: Export Tool should export nested docs cleanly in .json, .jsonl, and javabin

Posted by GitBox <gi...@apache.org>.
epugh edited a comment on pull request #2356:
URL: https://github.com/apache/lucene-solr/pull/2356#issuecomment-778497757


   In more testing, I don't have the `javabin` format working yet with nested children...   It's not roundtripping exports with the javabin format back into solr.  It does work for non nested children.


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

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



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


[GitHub] [lucene-solr] dsmiley commented on pull request #2356: SOLR-15152: Export Tool should export nested docs cleanly in .json, .jsonl, and javabin

Posted by GitBox <gi...@apache.org>.
dsmiley commented on pull request #2356:
URL: https://github.com/apache/lucene-solr/pull/2356#issuecomment-784291944


   >  "JSON is just JSONL with more delimiters"
   
   True!
   
   RE "Decorator" -- implies delegation.  I think this is more simply done by either subclassing or adding a toggle in a single format.


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

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



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


[GitHub] [lucene-solr] epugh commented on pull request #2356: SOLR-15152: Export Tool should export nested docs cleanly in .json, .jsonl, and javabin

Posted by GitBox <gi...@apache.org>.
epugh commented on pull request #2356:
URL: https://github.com/apache/lucene-solr/pull/2356#issuecomment-784182719


   I'm going to pick this up later this week.  My shower thought was "JSON is just JSONL with more delimiters" based on how we wrap each doc.   So maybe a Decorater around the JSONL that adds the appropirate `[`, `,`], and `]` delimiters?


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

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



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


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #2356: SOLR-15152: Export Tool should export nested docs cleanly in .json, .jsonl, and javabin

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #2356:
URL: https://github.com/apache/lucene-solr/pull/2356#discussion_r579854153



##########
File path: solr/core/src/java/org/apache/solr/util/ExportTool.java
##########
@@ -319,6 +369,91 @@ private Object constructDateStr(Object field) {
       return field;
     }
   }
+  
+  static class JsonlSink extends DocsSink {
+    private CharArr charArr = new CharArr(1024 * 2);
+    JSONWriter jsonWriter = new JSONWriter(charArr, -1);
+    private Writer writer;
+
+    public JsonlSink(Info info) {
+      this.info = info;
+    }
+
+    @Override
+    public void start() throws IOException {
+      fos = new FileOutputStream(info.out);
+      if(info.out.endsWith(".jsonl.gz")) {
+        fos = new GZIPOutputStream(fos);
+      }
+      if (info.bufferSize > 0) {
+        fos = new BufferedOutputStream(fos, info.bufferSize);
+      }
+      writer = new OutputStreamWriter(fos, StandardCharsets.UTF_8);
+
+    }
+
+    @Override
+    public void end() throws IOException {
+      writer.flush();
+      fos.flush();
+      fos.close();
+    }
+
+    @Override
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public synchronized void accept(SolrDocument doc) throws IOException {
+      charArr.reset();
+      int mapSize = doc._size();
+      if(doc.hasChildDocuments()) {
+        mapSize ++;
+      }
+      Map m = new LinkedHashMap(mapSize);
+      
+      doc.forEach((s, field) -> {
+        if (s.equals("_version_") || s.equals("_root_")) return;

Review comment:
       constants on IndexSchema




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

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



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


[GitHub] [lucene-solr] dsmiley commented on pull request #2356: SOLR-15152: Export Tool should export nested docs cleanly in .json, .jsonl, and javabin

Posted by GitBox <gi...@apache.org>.
dsmiley commented on pull request #2356:
URL: https://github.com/apache/lucene-solr/pull/2356#issuecomment-778937791


   I filed https://issues.apache.org/jira/browse/SOLR-15156 to keep that separate.


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

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



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


[GitHub] [lucene-solr] dsmiley commented on a change in pull request #2356: SOLR-15152: Export Tool should export nested docs cleanly in .json, .jsonl, and javabin

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #2356:
URL: https://github.com/apache/lucene-solr/pull/2356#discussion_r579993777



##########
File path: solr/solr-ref-guide/src/solr-control-script-reference.adoc
##########
@@ -902,25 +906,30 @@ The number of documents to export. The default is `100`. The value `-1` will exp
 
 *Examples*
 
-Export all documents from a collection `gettingstarted`:
+Export all documents from a collection `gettingstarted` into a file called `gettingstarted.json`:
 
 [source,bash]
-bin/solr export -url http://localhost:8983/solr/gettingstarted limit -1
+bin/solr export -url http://localhost:8983/solr/gettingstarted -limit -1
 
 Export all documents of collection `gettingstarted` into a file called `1MDocs.json.gz` as a zipped JSON file:
 
 [source,bash]
 ----
-bin/solr export -url http://localhost:8983/solr/gettingstarted -1 -out 1MDocs.json.gz
+bin/solr export -url http://localhost:8983/solr/gettingstarted -limit -1 -out 1MDocs.json.gz
 ----
 
 === Importing Documents to a Collection
 
-Once you have exported documents in a file, you can use the <<updatehandlers-in-solrconfig.adoc#,/update request handler>> to import them to a new Solr collection.
+Once you have exported documents in a file, you can use the <<updatehandlers-in-solrconfig.adoc#updatehandlers-in-solrconfig,/update request handler>> to import them to a new Solr collection.
+Notice the different endpoints used depending on the format.  
+
+*Example: import `json` files*
+
+`curl -X POST --header "Content-Type: application/json" -d @gettingstarted.json http://localhost:8983/solr/gettingstarted/update/json?commit=true`
 
 *Example: import `jsonl` files*
 
-`curl -X POST -d @gettingstarted.json http://localhost:8983/solr/gettingstarted/update/json/docs?commit=true`
+`curl -X POST --header "Content-Type: application/json" -d @gettingstarted.json http://localhost:8983/solr/gettingstarted/update/json/docs?commit=true`

Review comment:
       Just an FYI... Solr's very own "bin/post" actually has advantages to curl.  https://issues.apache.org/jira/browse/SOLR-8937  Looking back at that old-ish JIRA, I see that was the stdin case.  I'm not sure if curl is smart enough to not put the whole file in RAM for non-stdin.  I think "bin/post" is smart about it though.




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

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



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


[GitHub] [lucene-solr] noblepaul commented on pull request #2356: SOLR-15152: Export Tool should export nested docs cleanly in .json, .jsonl, and javabin

Posted by GitBox <gi...@apache.org>.
noblepaul commented on pull request #2356:
URL: https://github.com/apache/lucene-solr/pull/2356#issuecomment-784171132


   Thanks @dsmiley for the reviews. I missed this.


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

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



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