You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2018/05/08 21:15:04 UTC

[18/50] [abbrv] lucene-solr:jira/solr-11779: SOLR-12278: Add IgnoreLargeDocumentProcessFactory

SOLR-12278: Add IgnoreLargeDocumentProcessFactory


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/555b7ef2
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/555b7ef2
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/555b7ef2

Branch: refs/heads/jira/solr-11779
Commit: 555b7ef27049633eb91b7c0cf1ec08b24b2f8e41
Parents: ee2198d
Author: Cao Manh Dat <da...@apache.org>
Authored: Wed May 2 17:43:29 2018 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Wed May 2 17:43:29 2018 +0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../IgnoreLargeDocumentProcessorFactory.java    | 174 +++++++++++++++++++
 ...IgnoreLargeDocumentProcessorFactoryTest.java | 102 +++++++++++
 3 files changed, 278 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/555b7ef2/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a3a6c37..f8261ea 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -108,6 +108,8 @@ New Features
 * SOLR-8998: introducing uniqueBlock(_root_) aggregation as faster alternative to unique(_root_) for counting
   child value facets in parents via json.facet on block index (Dr Oleg Savrasov, Mikhail Khludnev)
 
+* SOLR-11278: Add IgnoreLargeDocumentProcessFactory (Cao Manh Dat, David Smiley)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/555b7ef2/solr/core/src/java/org/apache/solr/update/processor/IgnoreLargeDocumentProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/IgnoreLargeDocumentProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/IgnoreLargeDocumentProcessorFactory.java
new file mode 100644
index 0000000..1782438
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/update/processor/IgnoreLargeDocumentProcessorFactory.java
@@ -0,0 +1,174 @@
+/*
+ * 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.IdentityHashMap;
+import java.util.Map;
+
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.AddUpdateCommand;
+
+import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
+import static org.apache.solr.common.SolrException.ErrorCode.SERVER_ERROR;
+
+/**
+ * <p>
+ * Gives system administrators a way to ignore very large update from clients.
+ * When an update goes through processors its size can change
+ * therefore this processor should be the last processor of the chain.
+ * </p>
+ * @since 7.4.0
+ */
+public class IgnoreLargeDocumentProcessorFactory extends UpdateRequestProcessorFactory {
+  public static final String LIMIT_SIZE_PARAM = "limit";
+
+  // limit of a SolrInputDocument size (in kb)
+  private long maxDocumentSize = 1024 * 1024;
+
+  @Override
+  public void init(NamedList args) {
+    maxDocumentSize = args.toSolrParams().required().getLong(LIMIT_SIZE_PARAM);
+    args.remove(LIMIT_SIZE_PARAM);
+
+    if (args.size() > 0) {
+      throw new SolrException(SERVER_ERROR,
+          "Unexpected init param(s): '" +
+              args.getName(0) + "'");
+    }
+  }
+
+  @Override
+  public UpdateRequestProcessor getInstance(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next) {
+    return new UpdateRequestProcessor(next) {
+      @Override
+      public void processAdd(AddUpdateCommand cmd) throws IOException {
+        long docSize = ObjectSizeEstimator.fastEstimate(cmd.getSolrInputDocument());
+        if (docSize / 1024 > maxDocumentSize) {
+          throw new SolrException(BAD_REQUEST, "Size of the document "+cmd.getPrintableId()+" is too large, around:"+docSize);
+        }
+        super.processAdd(cmd);
+      }
+    };
+  }
+
+  /**
+   * Util class for quickly estimate size of a {@link org.apache.solr.common.SolrInputDocument}
+   * Compare to {@link org.apache.lucene.util.RamUsageEstimator}, this class have some pros
+   * <ul>
+   * <li>does not use reflection
+   * <li>go as deep as needed to compute size of all {@link org.apache.solr.common.SolrInputField} and
+   * all {@link org.apache.solr.common.SolrInputDocument} children
+   * <li>compute size of String based on its length
+   * <li>fast estimate size of a {@link java.util.Map} or a {@link java.util.Collection}
+   * </ul>
+   */
+  // package private for testing
+  static class ObjectSizeEstimator {
+    /**
+     * Sizes of primitive classes.
+     */
+    private static final Map<Class<?>,Integer> primitiveSizes = new IdentityHashMap<>();
+    static {
+      primitiveSizes.put(boolean.class, 1);
+      primitiveSizes.put(Boolean.class, 1);
+      primitiveSizes.put(byte.class, 1);
+      primitiveSizes.put(Byte.class, 1);
+      primitiveSizes.put(char.class, Character.BYTES);
+      primitiveSizes.put(Character.class, Character.BYTES);
+      primitiveSizes.put(short.class, Short.BYTES);
+      primitiveSizes.put(Short.class, Short.BYTES);
+      primitiveSizes.put(int.class, Integer.BYTES);
+      primitiveSizes.put(Integer.class, Integer.BYTES);
+      primitiveSizes.put(float.class, Float.BYTES);
+      primitiveSizes.put(Float.class, Float.BYTES);
+      primitiveSizes.put(double.class, Double.BYTES);
+      primitiveSizes.put(Double.class, Double.BYTES);
+      primitiveSizes.put(long.class, Long.BYTES);
+      primitiveSizes.put(Long.class, Long.BYTES);
+    }
+
+    static long fastEstimate(SolrInputDocument doc) {
+      if (doc == null) return 0L;
+      long size = 0;
+      if (doc.getFieldNames() != null) {
+        for (String fieldName : doc.getFieldNames()) {
+          size += fastEstimate(fieldName) + fastEstimate(doc.getField(fieldName).getValue());
+        }
+      }
+      if (doc.hasChildDocuments()) {
+        for (SolrInputDocument childDoc : doc.getChildDocuments()) {
+          size += fastEstimate(childDoc);
+        }
+      }
+      return size;
+    }
+
+    static long fastEstimate(Object obj) {
+      if (obj == null) return 0;
+
+      long size = primitiveEstimate(obj, -1);
+      if (size != -1) return size;
+
+      if (obj instanceof Map) {
+        return fastEstimate((Map) obj);
+      }
+
+      if (obj instanceof Collection) {
+        return fastEstimate((Collection) obj);
+      }
+
+      return 0L;
+    }
+
+    private static long primitiveEstimate(Object obj, long def) {
+      Class<?> clazz = obj.getClass();
+      if (clazz.isPrimitive()) {
+        return primitiveSizes.get(clazz);
+      }
+      if (obj instanceof String) {
+        return ((String) obj).length() * Character.BYTES;
+      }
+      return def;
+    }
+
+    private static long fastEstimate(Map<Object, Object> map) {
+      if (map.isEmpty()) return 0;
+      long size = 0;
+      for (Map.Entry<Object, Object> entry : map.entrySet()) {
+        size += primitiveEstimate(entry.getKey(), 0L) + primitiveEstimate(entry.getValue(), 0L);
+      }
+      return size;
+    }
+
+    private static long fastEstimate(Collection collection) {
+      if (collection.isEmpty()) return 0;
+      long size = 0;
+      for (Object obj : collection) {
+        size += primitiveEstimate(obj, 0L);
+      }
+      return size;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/555b7ef2/solr/core/src/test/org/apache/solr/update/processor/IgnoreLargeDocumentProcessorFactoryTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/IgnoreLargeDocumentProcessorFactoryTest.java b/solr/core/src/test/org/apache/solr/update/processor/IgnoreLargeDocumentProcessorFactoryTest.java
new file mode 100644
index 0000000..da70fc6
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/update/processor/IgnoreLargeDocumentProcessorFactoryTest.java
@@ -0,0 +1,102 @@
+/*
+ * 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.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.update.AddUpdateCommand;
+import org.junit.Test;
+
+import static org.apache.solr.update.processor.IgnoreLargeDocumentProcessorFactory.ObjectSizeEstimator.fastEstimate;
+
+public class IgnoreLargeDocumentProcessorFactoryTest extends LuceneTestCase {
+
+  @Test
+  public void testProcessor() throws IOException {
+    NamedList args = new NamedList();
+    args.add(IgnoreLargeDocumentProcessorFactory.LIMIT_SIZE_PARAM, 1);
+
+    IgnoreLargeDocumentProcessorFactory factory = new IgnoreLargeDocumentProcessorFactory();
+    factory.init(args);
+    try {
+      UpdateRequestProcessor processor = factory.getInstance(null, null, null);
+      processor.processAdd(getUpdate(1024));
+      fail("Expected processor to ignore the update");
+    } catch (SolrException e) {
+      //expected
+    }
+
+    args = new NamedList();
+    args.add(IgnoreLargeDocumentProcessorFactory.LIMIT_SIZE_PARAM, 2);
+    factory = new IgnoreLargeDocumentProcessorFactory();
+    factory.init(args);
+    UpdateRequestProcessor processor = factory.getInstance(null, null, null);
+    processor.processAdd(getUpdate(1024));
+
+  }
+
+  public AddUpdateCommand getUpdate(int size) {
+    SolrInputDocument document = new SolrInputDocument();
+    document.addField(new String(new byte[size], Charset.defaultCharset()), 1L);
+    assertTrue(fastEstimate(document) > size);
+
+    AddUpdateCommand cmd = new AddUpdateCommand(null);
+    cmd.solrDoc = document;
+    return cmd;
+  }
+
+  @Test
+  public void testEstimateObjectSize() {
+    assertEquals(fastEstimate("abc"), 6);
+    assertEquals(fastEstimate("abcdefgh"), 16);
+    List<String> keys = Arrays.asList("int", "long", "double", "float", "str");
+    assertEquals(fastEstimate(keys), 42);
+    List<Object> values = Arrays.asList(12, 5L, 12.0, 5.0, "duck");
+    assertEquals(fastEstimate(values), 8);
+
+    Map<String, Object> map = new HashMap<>();
+    map.put("int", 12);
+    map.put("long", 5L);
+    map.put("double", 12.0);
+    map.put("float", 5.0f);
+    map.put("str", "duck");
+    assertEquals(fastEstimate(map), 50);
+
+    SolrInputDocument document = new SolrInputDocument();
+    for (Map.Entry<String, Object> entry : map.entrySet()) {
+      document.addField(entry.getKey(), entry.getValue());
+    }
+    assertEquals(fastEstimate(document), fastEstimate(map));
+
+    SolrInputDocument childDocument = new SolrInputDocument();
+    for (Map.Entry<String, Object> entry : map.entrySet()) {
+      childDocument.addField(entry.getKey(), entry.getValue());
+    }
+    document.addChildDocument(childDocument);
+    assertEquals(fastEstimate(document), fastEstimate(map) * 2);
+  }
+}