You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2019/08/09 04:41:44 UTC

[lucene-solr] branch jira/SOLR-13682 updated: SOLR-13682: refactored and cleaned up

This is an automated email from the ASF dual-hosted git repository.

noble pushed a commit to branch jira/SOLR-13682
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/jira/SOLR-13682 by this push:
     new acc3e47  SOLR-13682: refactored and cleaned up
acc3e47 is described below

commit acc3e47218c11a26839ad912e600294db2d0fda8
Author: noble <no...@apache.org>
AuthorDate: Fri Aug 9 14:41:25 2019 +1000

    SOLR-13682: refactored and cleaned up
---
 .../src/java/org/apache/solr/util/ExportTool.java  | 254 ++++++++++++---------
 .../org/apache/solr/common/SolrInputDocument.java  |  48 ++--
 2 files changed, 175 insertions(+), 127 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/util/ExportTool.java b/solr/core/src/java/org/apache/solr/util/ExportTool.java
index 9e2f109..4c75791 100644
--- a/solr/core/src/java/org/apache/solr/util/ExportTool.java
+++ b/solr/core/src/java/org/apache/solr/util/ExportTool.java
@@ -20,14 +20,14 @@ package org.apache.solr.util;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
-import java.io.Writer;
 import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
-import java.util.function.Consumer;
+import java.util.Set;
 
+import com.google.common.collect.ImmutableSet;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
@@ -74,11 +74,16 @@ public class ExportTool extends SolrCLI.ToolBase {
 
   }
 
+  static Set<String> formats = ImmutableSet.of(JAVABIN, "jsonl");
+
   @Override
   protected void runImpl(CommandLine cli) throws Exception {
     Info info = new Info();
     String url = cli.getOptionValue("url");
     info.format = cli.getOptionValue("format", "jsonl");
+    if (!formats.contains(info.format)) {
+      throw new IllegalArgumentException("format must be one of :" + formats);
+    }
     info.query = cli.getOptionValue("query", "*:*");
     info.fields = cli.getOptionValue("fields");
     int idx = url.lastIndexOf('/');
@@ -86,129 +91,74 @@ public class ExportTool extends SolrCLI.ToolBase {
     info.coll = url.substring(idx + 1);
     info.out = cli.getOptionValue("out",
         JAVABIN.equals(info.format) ? info.coll + ".javabin" : info.coll + ".json");
+
     String maxDocsStr = cli.getOptionValue("limit", "100");
     info.limit = Long.parseLong(maxDocsStr);
     if (info.limit == -1) info.limit = Long.MAX_VALUE;
-    if (JAVABIN.equals(info.format)) {
-      writeJavabinDocs(info);
-    } else {
-      writeJsonLDocs(info);
-    }
+    DocsSink sink = JAVABIN.equals(info.format) ? new JavabinSink(info) : new JsonSink(info);
+    streamDocsWithCursorMark(info, sink);
   }
 
-  public static void writeJsonLDocs(Info info) throws IOException, SolrServerException {
-    SolrClient solrClient = new CloudSolrClient.Builder(Collections.singletonList(info.baseurl)).build();
-    FileOutputStream fos = new FileOutputStream(info.out);
-    Writer writer = FastWriter.wrap(new OutputStreamWriter(fos));
-
-    SolrJSONWriter jsonw = new SolrJSONWriter(writer);
-    jsonw.setIndent(false);
-    Consumer<SolrDocument> consumer = doc -> {
-      try {
-        Map m = new LinkedHashMap(doc.size());
-        doc.forEach((s, field) -> {
-          if (s.equals("_version_")) return;
-          if (field instanceof List) {
-            if (((List) field).size() == 1) {
-              field = ((List) field).get(0);
-            }
-          }
-          m.put(s, field);
-        });
-        jsonw.writeObj(m);
-        writer.flush();
-        writer.append('\n');
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    };
-
-    try {
-      streamDocsWithCursorMark(info, solrClient, consumer);
-    } finally {
-      jsonw.close();
-      solrClient.close();
-      fos.close();
-    }
-  }
+  interface DocsSink {
+    void start() throws IOException;
 
-  public static void writeJavabinDocs(Info info) throws IOException, SolrServerException {
-    SolrClient solrClient = new CloudSolrClient.Builder(Collections.singletonList(info.baseurl)).build();
-    FileOutputStream fos = new FileOutputStream(info.out);
-    JavaBinCodec codec = new JavaBinCodec(fos, null);
-    codec.writeTag(JavaBinCodec.NAMED_LST, 2);
-    codec.writeStr("params");
-    codec.writeNamedList(new NamedList<>());
-    codec.writeStr("docs");
-    codec.writeTag(JavaBinCodec.ITERATOR);
-    long[] docsWritten = new long[1];
-    docsWritten[0] = 0;
-    Consumer<SolrDocument> consumer = doc -> {
-      try {
-        SolrInputDocument document = new SolrInputDocument();
-        doc.forEach((s, o) -> {
-          if (s.equals("_version_")) return;
-          if (o instanceof List) {
-            if (((List) o).size() == 1) o = ((List) o).get(0);
-          }
-          document.addField(s, o);
-        });
+    void accept(SolrDocument document) throws IOException;
 
-        codec.writeSolrInputDocument(document);
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    };
-    try {
-      streamDocsWithCursorMark(info, solrClient, consumer);
-    } finally {
-      codec.writeTag(JavaBinCodec.END);
-      solrClient.close();
-      codec.close();
-      fos.close();
-    }
+    void end() throws IOException;
   }
 
-  private static void streamDocsWithCursorMark(Info info, SolrClient solrClient,
-                                               Consumer<SolrDocument> consumer) throws SolrServerException, IOException {
+  private static void streamDocsWithCursorMark(Info info,
+                                               DocsSink sink) throws SolrServerException, IOException {
+    SolrClient solrClient = new CloudSolrClient.Builder(Collections.singletonList(info.baseurl)).build();
     long[] docsWritten = new long[]{0};
     NamedList<Object> rsp1 = solrClient.request(new GenericSolrRequest(SolrRequest.METHOD.GET, "/schema/uniquekey",
         new MapSolrParams(Collections.singletonMap("collection", info.coll))));
     String uniqueKey = (String) rsp1.get("uniqueKey");
-    NamedList<Object> rsp;
-    SolrQuery q = (new SolrQuery(info.query))
-        .setParam("collection", info.coll)
-        .setRows(100)
-        .setSort(SolrQuery.SortClause.asc(uniqueKey));
-    if (info.fields != null) {
-      q.setParam(FL, info.fields);
-    }
 
-    String cursorMark = CursorMarkParams.CURSOR_MARK_START;
-    boolean done = false;
-    while (!done) {
-      if (docsWritten[0] >= info.limit) break;
-      QueryRequest request = new QueryRequest(q);
-
-      request.setResponseParser(new StreamingBinaryResponseParser(new StreamingResponseCallback() {
-        @Override
-        public void streamSolrDocument(SolrDocument doc) {
-          consumer.accept(doc);
-          docsWritten[0]++;
-        }
+    sink.start();
+    try {
+      NamedList<Object> rsp;
+      SolrQuery q = (new SolrQuery(info.query))
+          .setParam("collection", info.coll)
+          .setRows(100)
+          .setSort(SolrQuery.SortClause.asc(uniqueKey));
+      if (info.fields != null) {
+        q.setParam(FL, info.fields);
+      }
+
+      String cursorMark = CursorMarkParams.CURSOR_MARK_START;
+      boolean done = false;
+      while (!done) {
+        if (docsWritten[0] >= info.limit) break;
+        QueryRequest request = new QueryRequest(q);
 
-        @Override
-        public void streamDocListInfo(long numFound, long start, Float maxScore) {
+        request.setResponseParser(new StreamingBinaryResponseParser(new StreamingResponseCallback() {
+          @Override
+          public void streamSolrDocument(SolrDocument doc) {
+            try {
+              sink.accept(doc);
+            } catch (IOException e) {
+              throw new RuntimeException(e);
+            }
+            docsWritten[0]++;
+          }
 
+          @Override
+          public void streamDocListInfo(long numFound, long start, Float maxScore) {
+
+          }
+        }));
+        q.set(CursorMarkParams.CURSOR_MARK_PARAM, cursorMark);
+        rsp = solrClient.request(request);
+        String nextCursorMark = (String) rsp.get(CursorMarkParams.CURSOR_MARK_NEXT);
+        if (nextCursorMark == null || Objects.equals(cursorMark, nextCursorMark)) {
+          break;
         }
-      }));
-      q.set(CursorMarkParams.CURSOR_MARK_PARAM, cursorMark);
-      rsp = solrClient.request(request);
-      String nextCursorMark = (String) rsp.get(CursorMarkParams.CURSOR_MARK_NEXT);
-      if (nextCursorMark == null || Objects.equals(cursorMark, nextCursorMark)) {
-        break;
+        cursorMark = nextCursorMark;
       }
-      cursorMark = nextCursorMark;
+    } finally {
+      sink.end();
+
     }
   }
 
@@ -244,4 +194,94 @@ public class ExportTool extends SolrCLI.ToolBase {
           .withDescription("Comma separated fields. By default all fields are fetched")
           .create("fields")
   };
+
+  private static class JsonSink implements DocsSink {
+    private final Info info;
+    private SolrJSONWriter jsonw;
+    private FastWriter writer;
+    private FileOutputStream fos;
+
+    public JsonSink(Info info) {
+      this.info = info;
+    }
+
+    @Override
+    public void start() throws IOException {
+      fos = new FileOutputStream(info.out);
+      writer = FastWriter.wrap(new OutputStreamWriter(fos));
+      jsonw = new SolrJSONWriter(writer);
+      jsonw.setIndent(false);
+
+    }
+
+    @Override
+    public void end() throws IOException {
+      jsonw.close();
+      fos.close();
+
+    }
+
+    @Override
+    public void accept(SolrDocument doc) throws IOException {
+      Map m = new LinkedHashMap(doc.size());
+      doc.forEach((s, field) -> {
+        if (s.equals("_version_")) return;
+        if (field instanceof List) {
+          if (((List) field).size() == 1) {
+            field = ((List) field).get(0);
+          }
+        }
+        m.put(s, field);
+      });
+      jsonw.writeObj(m);
+      writer.flush();
+      writer.append('\n');
+
+    }
+  }
+
+  private static class JavabinSink implements DocsSink {
+    private final Info info;
+    JavaBinCodec codec;
+    FileOutputStream fos;
+
+    public JavabinSink(Info info) {
+      this.info = info;
+    }
+
+    @Override
+    public void start() throws IOException {
+      fos = new FileOutputStream(info.out);
+      codec = new JavaBinCodec(fos, null);
+      codec.writeTag(JavaBinCodec.NAMED_LST, 2);
+      codec.writeStr("params");
+      codec.writeNamedList(new NamedList<>());
+      codec.writeStr("docs");
+      codec.writeTag(JavaBinCodec.ITERATOR);
+
+    }
+
+    @Override
+    public void end() throws IOException {
+      codec.writeTag(JavaBinCodec.END);
+      codec.close();
+      fos.close();
+
+    }
+
+    @Override
+    public void accept(SolrDocument doc) throws IOException {
+      SolrInputDocument document = new SolrInputDocument();
+      doc.forEach((s, o) -> {
+        if (s.equals("_version_")) return;
+        if (o instanceof List) {
+          if (((List) o).size() == 1) o = ((List) o).get(0);
+        }
+        document.addField(s, o);
+      });
+
+      codec.writeSolrInputDocument(document);
+
+    }
+  }
 }
diff --git a/solr/solrj/src/java/org/apache/solr/common/SolrInputDocument.java b/solr/solrj/src/java/org/apache/solr/common/SolrInputDocument.java
index e1fa311..0c86817 100644
--- a/solr/solrj/src/java/org/apache/solr/common/SolrInputDocument.java
+++ b/solr/solrj/src/java/org/apache/solr/common/SolrInputDocument.java
@@ -24,11 +24,12 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.function.BiConsumer;
 
 /**
  * Represent the field-value information needed to construct and index
  * a Lucene Document.  Like the SolrDocument, the field values should
- * match those specified in schema.xml 
+ * match those specified in schema.xml
  *
  *
  * @since solr 1.3
@@ -37,7 +38,7 @@ public class SolrInputDocument extends SolrDocumentBase<SolrInputField, SolrInpu
 {
   private final Map<String,SolrInputField> _fields;
   private List<SolrInputDocument> _childDocuments;
-  
+
   public SolrInputDocument(String... fields) {
     _fields = new LinkedHashMap<>();
     assert fields.length % 2 == 0;
@@ -48,7 +49,14 @@ public class SolrInputDocument extends SolrDocumentBase<SolrInputField, SolrInpu
 
   @Override
   public void writeMap(EntryWriter ew) throws IOException {
-    _fields.forEach(ew.getBiConsumer());
+    BiConsumer<CharSequence, Object> bc = ew.getBiConsumer();
+    BiConsumer<CharSequence, Object> wrapper = (k, o) -> {
+      if (o instanceof SolrInputField) {
+        o = ((SolrInputField) o).getValue();
+      }
+      bc.accept(k, o);
+    };
+    _fields.forEach(wrapper);
   }
 
   public SolrInputDocument(Map<String,SolrInputField> fields) {
@@ -62,7 +70,7 @@ public class SolrInputDocument extends SolrDocumentBase<SolrInputField, SolrInpu
   public void clear()
   {
     if( _fields != null ) {
-      _fields.clear();      
+      _fields.clear();
     }
     _childDocuments = null;
   }
@@ -104,38 +112,38 @@ public class SolrInputDocument extends SolrDocumentBase<SolrInputField, SolrInpu
     if (field!=null) o = field.getFirstValue();
     return o;
   }
-  
+
   /** Get all the values for a field.
-   * 
+   *
    * @param name name of the field to fetch
    * @return value of the field or null if not set
    */
   @Override
-  public Collection<Object> getFieldValues(String name) 
+  public Collection<Object> getFieldValues(String name)
   {
     SolrInputField field = getField(name);
     if (field!=null) {
       return field.getValues();
     }
     return null;
-  } 
-  
+  }
+
   /** Get all field names.
-   * 
+   *
    * @return Set of all field names.
    */
   @Override
-  public Collection<String> getFieldNames() 
+  public Collection<String> getFieldNames()
   {
     return _fields.keySet();
   }
-  
+
   /** Set a field value; replacing the existing value if present.
-   * 
+   *
    * @param name name of the field to set
    * @param value value of the field
    */
-  public void setField(String name, Object value ) 
+  public void setField(String name, Object value )
   {
     SolrInputField field = new SolrInputField( name );
     _fields.put( name, field );
@@ -144,7 +152,7 @@ public class SolrInputDocument extends SolrDocumentBase<SolrInputField, SolrInpu
 
   /**
    * Remove a field from the document
-   * 
+   *
    * @param name The field name whose field is to be removed from the document
    * @return the previous field with <tt>name</tt>, or
    *         <tt>null</tt> if there was no field for <tt>key</tt>.
@@ -166,7 +174,7 @@ public class SolrInputDocument extends SolrDocumentBase<SolrInputField, SolrInpu
   public Iterator<SolrInputField> iterator() {
     return _fields.values().iterator();
   }
-  
+
   @Override
   public String toString()
   {
@@ -174,7 +182,7 @@ public class SolrInputDocument extends SolrDocumentBase<SolrInputField, SolrInpu
         + ( _childDocuments == null ? "" : (", children: " + _childDocuments) )
         + ")";
   }
-  
+
   public SolrInputDocument deepCopy() {
     SolrInputDocument clone = new SolrInputDocument();
     Set<Entry<String,SolrInputField>> entries = _fields.entrySet();
@@ -188,7 +196,7 @@ public class SolrInputDocument extends SolrDocumentBase<SolrInputField, SolrInpu
         clone._childDocuments.add(child.deepCopy());
       }
     }
-    
+
     return clone;
   }
 
@@ -258,7 +266,7 @@ public class SolrInputDocument extends SolrDocumentBase<SolrInputField, SolrInpu
    }
     _childDocuments.add(child);
   }
-  
+
   public void addChildDocuments(Collection<SolrInputDocument> children) {
     for (SolrInputDocument child : children) {
       addChildDocument(child);
@@ -269,7 +277,7 @@ public class SolrInputDocument extends SolrDocumentBase<SolrInputField, SolrInpu
   public List<SolrInputDocument> getChildDocuments() {
     return _childDocuments;
   }
-  
+
   public boolean hasChildDocuments() {
     boolean isEmpty = (_childDocuments == null || _childDocuments.isEmpty());
     return !isEmpty;