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 2021/04/12 01:08:06 UTC

[lucene-solr] branch branch_8x updated: optimizations

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 979e0bb  optimizations
979e0bb is described below

commit 979e0bbdc6841fe6d45b4586db7aaee41879facd
Author: Noble Paul <no...@gmail.com>
AuthorDate: Mon Apr 12 11:07:30 2021 +1000

    optimizations
---
 .../java/org/apache/solr/loader/FastBinLoader.java | 123 +++++++++++++++++++++
 .../org/apache/solr/loader/FastFieldReader.java    |  46 ++++++++
 .../java/org/apache/solr/loader/PackedBytes.java   |  58 ++++++++++
 .../java/org/apache/solr/schema/SchemaField.java   |   8 ++
 4 files changed, 235 insertions(+)

diff --git a/solr/core/src/java/org/apache/solr/loader/FastBinLoader.java b/solr/core/src/java/org/apache/solr/loader/FastBinLoader.java
new file mode 100644
index 0000000..4e80400
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/loader/FastBinLoader.java
@@ -0,0 +1,123 @@
+/*
+ * 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.loader;
+
+import java.io.IOException;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexableField;
+import org.apache.solr.common.util.DataEntry;
+import org.apache.solr.common.util.FastJavaBinDecoder;
+import org.apache.solr.handler.RequestHandlerBase;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.util.RefCounted;
+
+import static org.apache.solr.common.util.DataEntry.Type.ENTRY_ITER;
+import static org.apache.solr.common.util.DataEntry.Type.KEYVAL_ITER;
+
+public class FastBinLoader extends RequestHandlerBase {
+  @Override
+  public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
+    RefCounted<IndexWriter> ref = req.getCore().getSolrCoreState().getIndexWriter(req.getCore());
+    FastFieldReader.Ctx ctx= new FastFieldReader.Ctx();
+    ctx.iw = ref.get();
+    ctx.document = new Document();
+    ctx.schema = req.getSchema();
+    FieldsListener fieldsListener = new FieldsListener(ctx);
+    try {
+      DataEntry.FastDecoder decoder = new FastJavaBinDecoder();
+      decoder
+          .withInputStream(req.getContentStreams().iterator().next().getStream())
+          .decode(e -> {
+            if(e.type() == ENTRY_ITER) {// a list of docs
+              e.listenContainer(null, de -> {
+                if(de.type() == KEYVAL_ITER) {
+                  ctx.document.clear();
+                  de.listenContainer(null, fieldsListener);
+                }
+              });
+            }// we ignore everything else
+          });
+
+    } finally {
+      ref.decref();
+    }
+
+  }
+
+  @Override
+  public String getDescription() {
+    return "FastBinLoader";
+  }
+
+  private static class FieldsListener implements DataEntry.EntryListener {
+   FastFieldReader.Ctx ctx;
+
+    public FieldsListener(FastFieldReader.Ctx ctx) {
+      this.ctx = ctx;
+    }
+
+    @Override
+    public void end(DataEntry e) {
+      try {
+        ctx.iw.addDocument(ctx.document);
+        ctx.document.clear();
+
+        ctx.packedBytes.clear();
+      } catch (IOException ioException) {
+        throw new RuntimeException(ioException);
+      }
+    }
+
+    @Override
+    public void entry(DataEntry e) {
+      CharSequence name = e.name();
+      SchemaField sf = null;
+      if (name != null &&
+          (sf = ctx.schema.getField(name.toString())) != null) {
+        FastFieldReader ffr = sf.getFastFieldReader();
+        if (ffr != null) {
+          //this type is optimized to be indexed live
+          ffr.addFields(ctx);
+        } else {
+          for (IndexableField f : sf.getType().createFields(sf, e.val())) {
+            if (f != null) { // null fields are not added
+              // HACK: workaround for SOLR-9809
+              // even though at this point in the code we know the field is single valued and DV only
+              // TrieField.createFields() may still return (usless) IndexableField instances that are not
+              // NumericDocValuesField instances.
+              //
+              // once SOLR-9809 is resolved, we should be able to replace this conditional with...
+              //    assert f instanceof NumericDocValuesField
+             /* if (forInPlaceUpdate) {
+                if (f instanceof NumericDocValuesField) {
+                  doc.add(f);
+                }
+              } else {*/
+                ctx.document.add(f);
+              }/**/
+            }
+          }
+        }
+      }
+      //else , not present in schema, ignore
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/loader/FastFieldReader.java b/solr/core/src/java/org/apache/solr/loader/FastFieldReader.java
new file mode 100644
index 0000000..b4330d6
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/loader/FastFieldReader.java
@@ -0,0 +1,46 @@
+/*
+ * 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.loader;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.solr.common.util.DataEntry;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.SchemaField;
+
+/**An interface that field types should implement that can optimally read data from a stream of bytes
+ * and write to lucene index
+ *
+ */
+public interface FastFieldReader {
+
+  void addFields(Ctx ctx);
+  class Ctx {
+    final PackedBytes packedBytes = new PackedBytes();
+    SchemaField field;
+    DataEntry data;
+    Document document;
+    IndexWriter iw;
+    IndexSchema schema;
+
+
+    public SchemaField getSchemaField(){return field;}
+    public DataEntry getDataEntry(){return data;}
+    public Document getDoc(){return document;}
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/loader/PackedBytes.java b/solr/core/src/java/org/apache/solr/loader/PackedBytes.java
new file mode 100644
index 0000000..91b4858
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/loader/PackedBytes.java
@@ -0,0 +1,58 @@
+/*
+ * 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.loader;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.solr.common.util.ByteArrayUtf8CharSequence;
+
+public class PackedBytes {
+  public byte[] buf;
+  public int pos = 0;
+  private static final int INCR = 1024;
+
+  public PackedBytes() {
+    this.buf = new byte[INCR];
+  }
+
+  public void clear() {
+    pos = 0;
+  }
+
+  public PackedBytes(byte[] buf) {
+    this.buf = buf;
+  }
+
+  public BytesRef add(ByteArrayUtf8CharSequence utf8) {
+    return add(utf8.getBuf(), utf8.offset(), utf8.size());
+  }
+
+  public BytesRef add(byte[] buf, int offset, int len) {
+    ensureCapacity(len);
+    System.arraycopy(buf, offset, this.buf, pos, len);
+    pos += len;
+    return new BytesRef(buf, pos - len, len);
+  }
+
+  private void ensureCapacity(int len) {
+    if (buf.length < pos + len) return;
+    byte[] newBytes = new byte[pos + len + INCR];
+    System.arraycopy(buf, 0, newBytes, 0, pos);
+    buf = newBytes;
+  }
+
+}
diff --git a/solr/core/src/java/org/apache/solr/schema/SchemaField.java b/solr/core/src/java/org/apache/solr/schema/SchemaField.java
index 2f60ccc..f87ac5a 100644
--- a/solr/core/src/java/org/apache/solr/schema/SchemaField.java
+++ b/solr/core/src/java/org/apache/solr/schema/SchemaField.java
@@ -29,6 +29,8 @@ import org.apache.lucene.index.IndexableFieldType;
 import org.apache.lucene.search.SortField;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.loader.FastFieldReader;
+import org.apache.solr.loader.FastFieldReaders;
 import org.apache.solr.response.TextResponseWriter;
 
 /**
@@ -46,6 +48,7 @@ public final class SchemaField extends FieldProperties implements IndexableField
   final int properties;
   final String defaultValue;
   boolean required = false;  // this can't be final since it may be changed dynamically
+  private FastFieldReader fastFieldReader;
   
   /** Declared field property overrides */
   Map<String,Object> args = Collections.emptyMap();
@@ -56,6 +59,7 @@ public final class SchemaField extends FieldProperties implements IndexableField
    */
   public SchemaField(String name, FieldType type) {
     this(name, type, type.properties, null);
+    this.fastFieldReader = FastFieldReaders.getInst(type);
   }
 
   /** Create a new SchemaField from an existing one by using all
@@ -451,4 +455,8 @@ public final class SchemaField extends FieldProperties implements IndexableField
   public Map<String, String> getAttributes() {
     return null;
   }
+
+  public FastFieldReader getFastFieldReader() {
+    return fastFieldReader;
+  }
 }