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 2017/01/09 16:10:11 UTC

[21/34] lucene-solr:jira/solr-9856: SOLR-9883: In example schemaless configs' default update chain, move the DUP to after the AddSchemaFields URP (which is now tagged as RunAlways), to avoid invalid buffered tlog entry replays.

SOLR-9883: In example schemaless configs' default update chain, move the DUP to after the AddSchemaFields URP (which is now tagged as RunAlways), to avoid invalid buffered tlog entry replays.


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

Branch: refs/heads/jira/solr-9856
Commit: d817fd43eccd67a5d73c3bbc49561de65d3fc9cb
Parents: 67261d2
Author: Steve Rowe <sa...@apache.org>
Authored: Sat Jan 7 16:39:20 2017 -0500
Committer: Steve Rowe <sa...@apache.org>
Committed: Sat Jan 7 16:39:20 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../java/org/apache/solr/update/UpdateLog.java  |   4 +-
 .../AddSchemaFieldsUpdateProcessorFactory.java  |   3 +-
 ...dd-schema-fields-update-processor-chains.xml |   8 +-
 .../collection1/conf/solrconfig-schemaless.xml  |  45 ++----
 .../schema/TestSchemalessBufferedUpdates.java   | 160 +++++++++++++++++++
 solr/example/files/conf/solrconfig.xml          |   5 +-
 .../basic_configs/conf/solrconfig.xml           |   6 +-
 .../conf/solrconfig.xml                         |   5 +-
 9 files changed, 190 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 1f7f09a..0d61730 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -337,6 +337,9 @@ Bug Fixes
 
 * SOLR-9503: NPE in Replica Placement Rules when using Overseer Role with other rules (Tim Owen via noble)
 
+* SOLR-9883: Example schemaless solr config files can lead to invalid tlog replays: when updates are buffered,
+  update processors ordered before DistributedUpdateProcessor, e.g. field normalization, are never run. (Steve Rowe)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/core/src/java/org/apache/solr/update/UpdateLog.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLog.java b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
index 9c0f1cf..b79290d 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
@@ -1429,7 +1429,7 @@ public class UpdateLog implements PluginInfoInitialized {
             }
           } catch (IOException ex) {
             recoveryInfo.errors++;
-            loglog.warn("REYPLAY_ERR: IOException reading log", ex);
+            loglog.warn("REPLAY_ERR: IOException reading log", ex);
             // could be caused by an incomplete flush if recovering from log
           } catch (ClassCastException cl) {
             recoveryInfo.errors++;
@@ -1440,7 +1440,7 @@ public class UpdateLog implements PluginInfoInitialized {
               throw ex;
             }
             recoveryInfo.errors++;
-            loglog.warn("REYPLAY_ERR: IOException reading log", ex);
+            loglog.warn("REPLAY_ERR: IOException reading log", ex);
             // could be caused by an incomplete flush if recovering from log
           } catch (Exception ex) {
             recoveryInfo.errors++;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java
index 4f68bcc..4758972 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java
@@ -120,7 +120,8 @@ import static org.apache.solr.core.ConfigSetProperties.IMMUTABLE_CONFIGSET_ARG;
  *   &lt;/lst&gt;
  * &lt;/processor&gt;</pre>
  */
-public class AddSchemaFieldsUpdateProcessorFactory extends UpdateRequestProcessorFactory implements SolrCoreAware {
+public class AddSchemaFieldsUpdateProcessorFactory extends UpdateRequestProcessorFactory
+    implements SolrCoreAware, UpdateRequestProcessorFactory.RunAlways {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private static final String TYPE_MAPPING_PARAM = "typeMapping";

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml
index 8d91d28..e574575 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml
@@ -66,8 +66,6 @@
   </updateRequestProcessorChain>
 
   <updateRequestProcessorChain name="add-fields">
-    <processor class="solr.LogUpdateProcessorFactory" />
-    <processor class="solr.DistributedUpdateProcessorFactory" />
     <processor class="solr.AddSchemaFieldsUpdateProcessorFactory">
       <str name="defaultFieldType">text</str>
       <lst name="typeMapping">
@@ -96,12 +94,12 @@
         <str name="fieldType">tdouble</str>
       </lst>
     </processor>
+    <processor class="solr.LogUpdateProcessorFactory" />
+    <processor class="solr.DistributedUpdateProcessorFactory" />
     <processor class="solr.RunUpdateProcessorFactory" />
   </updateRequestProcessorChain>
 
   <updateRequestProcessorChain name="parse-and-add-fields">
-    <processor class="solr.LogUpdateProcessorFactory" />
-    <processor class="solr.DistributedUpdateProcessorFactory" />
     <processor class="solr.ParseBooleanFieldUpdateProcessorFactory"/>
     <processor class="solr.ParseLongFieldUpdateProcessorFactory"/>
     <processor class="solr.ParseDoubleFieldUpdateProcessorFactory"/>
@@ -154,6 +152,8 @@
         <str name="fieldType">tdouble</str>
       </lst>
     </processor>
+    <processor class="solr.LogUpdateProcessorFactory" />
+    <processor class="solr.DistributedUpdateProcessorFactory" />
     <processor class="solr.RunUpdateProcessorFactory" />
   </updateRequestProcessorChain>
 </config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml
index 46aa2a4..8247d69 100644
--- a/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml
@@ -1,5 +1,4 @@
-<?xml version="1.0" ?>
-
+<?xml version="1.0" encoding="UTF-8" ?>
 <!--
  Licensed to the Apache Software Foundation (ASF) under one or more
  contributor license agreements.  See the NOTICE file distributed with
@@ -16,7 +15,7 @@
  See the License for the specific language governing permissions and
  limitations under the License.
 -->
-                                                           
+
 <config>
   <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
 
@@ -33,47 +32,26 @@
   <updateHandler>
     <updateLog enable="true">
       <str name="dir">${solr.ulog.dir:}</str>
-    </updateLog> 
+    </updateLog>
   </updateHandler>
 
   <requestHandler name="standard" class="solr.StandardRequestHandler">
     <bool name="httpCaching">true</bool>
   </requestHandler>
 
-
-  <requestHandler name="/update" class="solr.UpdateRequestHandler">
+  <initParams path="/update/**">
     <lst name="defaults">
       <str name="update.chain">add-unknown-fields-to-the-schema</str>
     </lst>
-  </requestHandler>
-
-  <query>
-    <!-- custom cache currently used by block join -->
-    <cache name="perSegFilter"
-           class="solr.search.LRUCache"
-           size="10"
-           initialSize="0"
-           autowarmCount="10"
-           regenerator="solr.NoOpRegenerator" />
-  </query>
+  </initParams>
 
-  <!-- Add unknown fields to the schema
-  
-       An example field type guessing update processor that will
-       attempt to parse string-typed field values as Booleans, Longs,
-       Doubles, or Dates, and then add schema fields with the guessed
-       field types.  
-       
-       This requires that the schema is both managed and mutable, by
-       declaring schemaFactory as ManagedIndexSchemaFactory, with
-       mutable specified as true. 
-       
-       See http://wiki.apache.org/solr/GuessingFieldTypes
-    -->
   <updateRequestProcessorChain name="add-unknown-fields-to-the-schema">
-    <processor class="solr.LogUpdateProcessorFactory"/>
-    <processor class="solr.DistributedUpdateProcessorFactory" />
+    <processor class="solr.UUIDUpdateProcessorFactory" />
     <processor class="solr.RemoveBlankFieldUpdateProcessorFactory"/>
+    <processor class="solr.FieldNameMutatingUpdateProcessorFactory">
+      <str name="pattern">[^\w-\.]</str>
+      <str name="replacement">_</str>
+    </processor>
     <processor class="solr.ParseBooleanFieldUpdateProcessorFactory"/>
     <processor class="solr.ParseLongFieldUpdateProcessorFactory"/>
     <processor class="solr.ParseDoubleFieldUpdateProcessorFactory"/>
@@ -119,7 +97,8 @@
         <str name="fieldType">tdouble</str>
       </lst>
     </processor>
+    <processor class="solr.LogUpdateProcessorFactory"/>
+    <processor class="solr.DistributedUpdateProcessorFactory" />
     <processor class="solr.RunUpdateProcessorFactory"/>
   </updateRequestProcessorChain>
-
 </config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/core/src/test/org/apache/solr/schema/TestSchemalessBufferedUpdates.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/schema/TestSchemalessBufferedUpdates.java b/solr/core/src/test/org/apache/solr/schema/TestSchemalessBufferedUpdates.java
new file mode 100644
index 0000000..c2e8b2e
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/schema/TestSchemalessBufferedUpdates.java
@@ -0,0 +1,160 @@
+/*
+ * 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.schema;
+
+import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrRequestInfo;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.DirectUpdateHandler2;
+import org.apache.solr.update.UpdateLog;
+import org.apache.solr.update.UpdateHandler;
+import org.apache.solr.update.processor.DistributedUpdateProcessorFactory;
+import org.apache.solr.update.processor.UpdateRequestProcessor;
+import org.apache.solr.update.processor.UpdateRequestProcessorChain;
+import org.apache.solr.update.processor.UpdateRequestProcessorFactory;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.update.processor.DistributedUpdateProcessor.DistribPhase;
+
+public class TestSchemalessBufferedUpdates extends SolrTestCaseJ4 {
+
+  // means that we've seen the leader and have version info (i.e. we are a non-leader replica)
+  private static final String FROM_LEADER = DistribPhase.FROMLEADER.toString();
+  private static final String UPDATE_CHAIN = "add-unknown-fields-to-the-schema";
+  private static final int TIMEOUT = 10;
+
+  private static final String collection = "collection1";
+  private static final String confDir = collection + "/conf";
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    File tmpSolrHome = createTempDir().toFile();
+    File tmpConfDir = new File(tmpSolrHome, confDir);
+    File testHomeConfDir = new File(TEST_HOME(), confDir);
+    FileUtils.copyFileToDirectory(new File(testHomeConfDir, "solrconfig-schemaless.xml"), tmpConfDir);
+    FileUtils.copyFileToDirectory(new File(testHomeConfDir, "schema-add-schema-fields-update-processor.xml"), tmpConfDir);
+    FileUtils.copyFileToDirectory(new File(testHomeConfDir, "solrconfig.snippet.randomindexconfig.xml"), tmpConfDir);
+    initCore("solrconfig-schemaless.xml", "schema-add-schema-fields-update-processor.xml", tmpSolrHome.getPath());
+  }
+
+  @Test
+  public void test() throws Exception {
+    DirectUpdateHandler2.commitOnClose = false;
+    final Semaphore logReplay = new Semaphore(0);
+    final Semaphore logReplayFinish = new Semaphore(0);
+    UpdateLog.testing_logReplayHook = () -> {
+      try {
+        assertTrue(logReplay.tryAcquire(TIMEOUT, TimeUnit.SECONDS));
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    };
+    UpdateLog.testing_logReplayFinishHook = logReplayFinish::release;
+
+    SolrQueryRequest req = req();
+    UpdateHandler uhandler = req.getCore().getUpdateHandler();
+    UpdateLog ulog = uhandler.getUpdateLog();
+
+    try {
+      assertEquals(UpdateLog.State.ACTIVE, ulog.getState());
+
+      // Invalid date will be normalized by ParseDateField URP
+      updateJ(jsonAdd(processAdd(sdoc("id","1", "f_dt","2017-01-04"))), params(DISTRIB_UPDATE_PARAM,FROM_LEADER));
+      assertU(commit());
+      assertJQ(req("q", "*:*"), "/response/numFound==1");
+
+      ulog.bufferUpdates();
+      assertEquals(UpdateLog.State.BUFFERING, ulog.getState());
+
+      // If the ParseDateField URP isn't ahead of the DUP, then the date won't be normalized in the buffered tlog entry,
+      // and the doc won't be indexed on the replaying replica - a warning is logged as follows:
+      // WARN [...] o.a.s.u.UpdateLog REYPLAY_ERR: IOException reading log
+      //            org.apache.solr.common.SolrException: Invalid Date String:'2017-01-05'
+      //              at org.apache.solr.util.DateMathParser.parseMath(DateMathParser.java:234)
+      //              at org.apache.solr.schema.TrieField.createField(TrieField.java:725) [...]
+      updateJ(jsonAdd(processAdd(sdoc("id","2", "f_dt","2017-01-05"))), params(DISTRIB_UPDATE_PARAM,FROM_LEADER));
+
+      Future<UpdateLog.RecoveryInfo> rinfoFuture = ulog.applyBufferedUpdates();
+
+      assertTrue(rinfoFuture != null);
+
+      assertEquals(UpdateLog.State.APPLYING_BUFFERED, ulog.getState());
+
+      logReplay.release(1000);
+
+      UpdateLog.RecoveryInfo rinfo = rinfoFuture.get();
+      assertEquals(UpdateLog.State.ACTIVE, ulog.getState());
+
+      assertU(commit());
+      assertJQ(req("q", "*:*"), "/response/numFound==2");
+    } finally {
+      DirectUpdateHandler2.commitOnClose = true;
+      UpdateLog.testing_logReplayHook = null;
+      UpdateLog.testing_logReplayFinishHook = null;
+      req().close();
+    }
+  }
+
+  private SolrInputDocument processAdd(final SolrInputDocument docIn) throws IOException {
+    UpdateRequestProcessorChain processorChain = h.getCore().getUpdateProcessingChain(UPDATE_CHAIN);
+    assertNotNull("Undefined URP chain '" + UPDATE_CHAIN + "'", processorChain);
+    List <UpdateRequestProcessorFactory> factoriesUpToDUP = new ArrayList<>();
+    for (UpdateRequestProcessorFactory urpFactory : processorChain.getProcessors()) {
+      factoriesUpToDUP.add(urpFactory);
+      if (urpFactory.getClass().equals(DistributedUpdateProcessorFactory.class)) 
+        break;
+    }
+    UpdateRequestProcessorChain chainUpToDUP = new UpdateRequestProcessorChain(factoriesUpToDUP, h.getCore());
+    assertNotNull("URP chain '" + UPDATE_CHAIN + "'", chainUpToDUP);
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    SolrQueryRequest req = req();
+    try {
+      SolrRequestInfo.setRequestInfo(new SolrRequestInfo(req, rsp));
+      AddUpdateCommand cmd = new AddUpdateCommand(req);
+      cmd.solrDoc = docIn;
+      UpdateRequestProcessor processor = chainUpToDUP.createProcessor(req, rsp);
+      processor.processAdd(cmd);
+      if (cmd.solrDoc.get("f_dt").getValue() instanceof Date) {
+        // Non-JSON types (Date in this case) aren't handled properly in noggit-0.6.  Although this is fixed in 
+        // https://github.com/yonik/noggit/commit/ec3e732af7c9425e8f40297463cbe294154682b1 to call obj.toString(), 
+        // Date::toString produces a Date representation that Solr doesn't like, so we convert using Instant::toString
+        cmd.solrDoc.get("f_dt").setValue(((Date) cmd.solrDoc.get("f_dt").getValue()).toInstant().toString(), 1.0f);
+      }
+      return cmd.solrDoc;
+    } finally {
+      SolrRequestInfo.clearRequestInfo();
+      req.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/example/files/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/example/files/conf/solrconfig.xml b/solr/example/files/conf/solrconfig.xml
index 3fd825e..f83c235 100644
--- a/solr/example/files/conf/solrconfig.xml
+++ b/solr/example/files/conf/solrconfig.xml
@@ -1193,9 +1193,6 @@
   <updateRequestProcessorChain name="files-update-processor">
     <!-- UUIDUpdateProcessorFactory will generate an id if none is present in the incoming document -->
     <processor class="solr.UUIDUpdateProcessorFactory" />
-
-    <processor class="solr.LogUpdateProcessorFactory"/>
-    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RemoveBlankFieldUpdateProcessorFactory"/>
     <processor class="solr.FieldNameMutatingUpdateProcessorFactory">
       <str name="pattern">[^\w-\.]</str>
@@ -1261,6 +1258,8 @@
       <!--</lst>-->
     </processor>
 
+    <processor class="solr.LogUpdateProcessorFactory"/>
+    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RunUpdateProcessorFactory"/>
   </updateRequestProcessorChain>
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml b/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
index 4ef902f..e009aab 100644
--- a/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/basic_configs/conf/solrconfig.xml
@@ -1187,9 +1187,6 @@
   <updateRequestProcessorChain name="add-unknown-fields-to-the-schema">
     <!-- UUIDUpdateProcessorFactory will generate an id if none is present in the incoming document -->
     <processor class="solr.UUIDUpdateProcessorFactory" />
-
-    <processor class="solr.LogUpdateProcessorFactory"/>
-    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RemoveBlankFieldUpdateProcessorFactory"/>
     <processor class="solr.FieldNameMutatingUpdateProcessorFactory">
       <str name="pattern">[^\w-\.]</str>
@@ -1239,6 +1236,9 @@
         <str name="fieldType">tdoubles</str>
       </lst>
     </processor>
+
+    <processor class="solr.LogUpdateProcessorFactory"/>
+    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RunUpdateProcessorFactory"/>
   </updateRequestProcessorChain>
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d817fd43/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml b/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
index 4b0899b..2ca1b7f 100644
--- a/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
+++ b/solr/server/solr/configsets/data_driven_schema_configs/conf/solrconfig.xml
@@ -1186,9 +1186,6 @@
   <updateRequestProcessorChain name="add-unknown-fields-to-the-schema">
     <!-- UUIDUpdateProcessorFactory will generate an id if none is present in the incoming document -->
     <processor class="solr.UUIDUpdateProcessorFactory" />
-
-    <processor class="solr.LogUpdateProcessorFactory"/>
-    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RemoveBlankFieldUpdateProcessorFactory"/>
     <processor class="solr.FieldNameMutatingUpdateProcessorFactory">
       <str name="pattern">[^\w-\.]</str>
@@ -1238,6 +1235,8 @@
         <str name="fieldType">tdoubles</str>
       </lst>
     </processor>
+    <processor class="solr.LogUpdateProcessorFactory"/>
+    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RunUpdateProcessorFactory"/>
   </updateRequestProcessorChain>