You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sa...@apache.org on 2014/06/18 00:24:29 UTC

svn commit: r1603300 - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/schema/ core/src/test/org/apache/solr/schema/

Author: sarowe
Date: Tue Jun 17 22:24:29 2014
New Revision: 1603300

URL: http://svn.apache.org/r1603300
Log:
SOLR-6164: Copy Fields Schema additions are not distributed to other nodes

Added:
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaConcurrent.java
Removed:
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaAddField.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaAddFields.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaCopyFields.java
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1603300&r1=1603299&r2=1603300&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Tue Jun 17 22:24:29 2014
@@ -197,6 +197,9 @@ Bug Fixes
 
 * SOLR-6129: DateFormatTransformer doesn't resolve dateTimeFormat. (Aaron LaBella via shalin)
 
+* SOLR-6164: Copy Fields Schema additions are not distributed to other nodes.
+  (Gregory Chanan via Steve Rowe)
+
 Other Changes
 ---------------------
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/IndexSchema.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/IndexSchema.java?rev=1603300&r1=1603299&r2=1603300&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/IndexSchema.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/IndexSchema.java Tue Jun 17 22:24:29 2014
@@ -591,44 +591,7 @@ public class IndexSchema {
       // expression = "/schema/copyField";
     
       dynamicCopyFields = new DynamicCopy[] {};
-      expression = "//" + COPY_FIELD;
-      nodes = (NodeList) xpath.evaluate(expression, document, XPathConstants.NODESET);
-
-      for (int i=0; i<nodes.getLength(); i++) {
-        node = nodes.item(i);
-        NamedNodeMap attrs = node.getAttributes();
-
-        String source = DOMUtil.getAttr(attrs, SOURCE, COPY_FIELD + " definition");
-        String dest   = DOMUtil.getAttr(attrs, DESTINATION,  COPY_FIELD + " definition");
-        String maxChars = DOMUtil.getAttr(attrs, MAX_CHARS);
-        int maxCharsInt = CopyField.UNLIMITED;
-        if (maxChars != null) {
-          try {
-            maxCharsInt = Integer.parseInt(maxChars);
-          } catch (NumberFormatException e) {
-            log.warn("Couldn't parse " + MAX_CHARS + " attribute for " + COPY_FIELD + " from "
-                    + source + " to " + dest + " as integer. The whole field will be copied.");
-          }
-        }
-
-        if (dest.equals(uniqueKeyFieldName)) {
-          String msg = UNIQUE_KEY + " field ("+uniqueKeyFieldName+
-            ") can not be the " + DESTINATION + " of a " + COPY_FIELD + "(" + SOURCE + "=" +source+")";
-          log.error(msg);
-          throw new SolrException(ErrorCode.SERVER_ERROR, msg);
-          
-        }
-
-        registerCopyField(source, dest, maxCharsInt);
-      }
-      
-      for (Map.Entry<SchemaField, Integer> entry : copyFieldTargetCounts.entrySet()) {
-        if (entry.getValue() > 1 && !entry.getKey().multiValued())  {
-          log.warn("Field " + entry.getKey().name + " is not multivalued "+
-              "and destination for multiple " + COPY_FIELDS + " ("+
-              entry.getValue()+")");
-        }
-      }
+      loadCopyFields(document, xpath);
 
       //Run the callbacks on SchemaAware now that everything else is done
       for (SchemaAware aware : schemaAware) {
@@ -747,6 +710,50 @@ public class IndexSchema {
   }
 
   /**
+   * Loads the copy fields
+   */
+  protected synchronized void loadCopyFields(Document document, XPath xpath) throws XPathExpressionException {
+    String expression = "//" + COPY_FIELD;
+    NodeList nodes = (NodeList)xpath.evaluate(expression, document, XPathConstants.NODESET);
+
+    for (int i=0; i<nodes.getLength(); i++) {
+      Node node = nodes.item(i);
+      NamedNodeMap attrs = node.getAttributes();
+
+      String source = DOMUtil.getAttr(attrs, SOURCE, COPY_FIELD + " definition");
+      String dest   = DOMUtil.getAttr(attrs, DESTINATION,  COPY_FIELD + " definition");
+      String maxChars = DOMUtil.getAttr(attrs, MAX_CHARS);
+
+      int maxCharsInt = CopyField.UNLIMITED;
+      if (maxChars != null) {
+        try {
+          maxCharsInt = Integer.parseInt(maxChars);
+        } catch (NumberFormatException e) {
+          log.warn("Couldn't parse " + MAX_CHARS + " attribute for " + COPY_FIELD + " from "
+                  + source + " to " + dest + " as integer. The whole field will be copied.");
+        }
+      }
+
+      if (dest.equals(uniqueKeyFieldName)) {
+        String msg = UNIQUE_KEY + " field ("+uniqueKeyFieldName+
+          ") can not be the " + DESTINATION + " of a " + COPY_FIELD + "(" + SOURCE + "=" +source+")";
+        log.error(msg);
+        throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+      }
+      
+      registerCopyField(source, dest, maxCharsInt);
+    }
+      
+    for (Map.Entry<SchemaField, Integer> entry : copyFieldTargetCounts.entrySet()) {
+      if (entry.getValue() > 1 && !entry.getKey().multiValued())  {
+        log.warn("Field " + entry.getKey().name + " is not multivalued "+
+            "and destination for multiple " + COPY_FIELDS + " ("+
+            entry.getValue()+")");
+      }
+    }
+  }
+
+  /**
    * Converts a sequence of path steps into a rooted path, by inserting slashes in front of each step.
    * @param steps The steps to join with slashes to form a path
    * @return a rooted path: a leading slash followed by the given steps joined with slashes

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java?rev=1603300&r1=1603299&r2=1603300&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java Tue Jun 17 22:24:29 2014
@@ -43,6 +43,7 @@ import java.nio.charset.StandardCharsets
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Map;
 
 /** Solr-managed schema - non-user-editable, but can be mutable via internal and external REST API requests. */
@@ -338,6 +339,13 @@ public final class ManagedIndexSchema ex
       Document document = schemaConf.getDocument();
       final XPath xpath = schemaConf.getXPath();
       newSchema.loadFields(document, xpath);
+      // let's completely rebuild the copy fields from the schema in ZK.
+      // create new copyField-related objects so we don't affect the
+      // old schema
+      newSchema.copyFieldsMap = new HashMap<>();
+      newSchema.dynamicCopyFields = null;
+      newSchema.copyFieldTargetCounts = new HashMap<>();
+      newSchema.loadCopyFields(document, xpath);
       if (null != uniqueKeyField) {
         newSchema.requiredFields.add(uniqueKeyField);
       }

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaConcurrent.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaConcurrent.java?rev=1603300&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaConcurrent.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaConcurrent.java Tue Jun 17 22:24:29 2014
@@ -0,0 +1,251 @@
+package org.apache.solr.schema;
+/*
+ * 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.
+ */
+
+import org.apache.solr.client.solrj.SolrServer;
+import org.apache.solr.client.solrj.impl.HttpSolrServer;
+import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
+import org.apache.solr.util.BaseTestHarness;
+import org.apache.solr.util.RESTfulServerProvider;
+import org.apache.solr.util.RestTestHarness;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.restlet.ext.servlet.ServerServlet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.junit.BeforeClass;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+
+public class TestCloudManagedSchemaConcurrent extends AbstractFullDistribZkTestBase {
+  private static final Logger log = LoggerFactory.getLogger(TestCloudManagedSchemaConcurrent.class);
+  private static final String SUCCESS_XPATH = "/response/lst[@name='responseHeader']/int[@name='status'][.='0']";
+
+  public TestCloudManagedSchemaConcurrent() {
+    super();
+    fixShardCount = true;
+
+    sliceCount = 4;
+    shardCount = 8;
+  }
+
+  @BeforeClass
+  public static void initSysProperties() {
+    System.setProperty("managed.schema.mutable", "true");
+    System.setProperty("enable.update.log", "true");
+  }
+
+  @Override
+  protected String getCloudSolrConfig() {
+    return "solrconfig-managed-schema.xml";
+  }
+
+  @Override
+  public SortedMap<ServletHolder,String> getExtraServlets() {
+    final SortedMap<ServletHolder,String> extraServlets = new TreeMap<>();
+    final ServletHolder solrRestApi = new ServletHolder("SolrSchemaRestApi", ServerServlet.class);
+    solrRestApi.setInitParameter("org.restlet.application", "org.apache.solr.rest.SolrSchemaRestApi");
+    extraServlets.put(solrRestApi, "/schema/*");  // '/schema/*' matches '/schema', '/schema/', and '/schema/whatever...'
+    return extraServlets;
+  }
+  
+  private List<RestTestHarness> restTestHarnesses = new ArrayList<>();
+  
+  private void setupHarnesses() {
+    for (final SolrServer client : clients) {
+      RestTestHarness harness = new RestTestHarness(new RESTfulServerProvider() {
+        @Override
+        public String getBaseURL() {
+          return ((HttpSolrServer)client).getBaseURL();
+        }
+      });
+      restTestHarnesses.add(harness);
+    }
+  }
+
+  private void verifySuccess(String request, String response) throws Exception {
+    String result = BaseTestHarness.validateXPath(response, SUCCESS_XPATH);
+    if (null != result) {
+      String msg = "QUERY FAILED: xpath=" + result + "  request=" + request + "  response=" + response;
+      log.error(msg);
+      fail(msg);
+    }
+  }
+
+  private void addFieldPut(RestTestHarness publisher, String fieldName) throws Exception {
+    final String content = "{\"type\":\"text\",\"stored\":\"false\"}";
+    String request = "/schema/fields/" + fieldName + "?wt=xml";
+    String response = publisher.put(request, content);
+    verifySuccess(request, response);
+  }
+
+  private void addFieldPost(RestTestHarness publisher, String fieldName) throws Exception {
+    final String content = "[{\"name\":\""+fieldName+"\",\"type\":\"text\",\"stored\":\"false\"}]";
+    String request = "/schema/fields/?wt=xml";
+    String response = publisher.post(request, content);
+    verifySuccess(request, response);
+  }
+
+  private void copyField(RestTestHarness publisher, String source, String dest) throws Exception {
+    final String content = "[{\"source\":\""+source+"\",\"dest\":[\""+dest+"\"]}]";
+    String request = "/schema/copyfields/?wt=xml";
+    String response = publisher.post(request, content);
+    verifySuccess(request, response);
+  }
+
+  private String[] getExpectedFieldResponses(int numAddFieldPuts, int numAddFieldPosts) {
+    String[] expectedAddFields = new String[1 + numAddFieldPuts + numAddFieldPosts];
+    expectedAddFields[0] = SUCCESS_XPATH;
+
+    for (int i = 0; i < numAddFieldPuts; ++i) {
+      String newFieldName = "newfieldPut" + i;
+      expectedAddFields[1 + i] 
+          = "/response/arr[@name='fields']/lst/str[@name='name'][.='" + newFieldName + "']";
+    }
+
+    for (int i = 0; i < numAddFieldPosts; ++i) {
+      String newFieldName = "newfieldPost" + i;
+      expectedAddFields[1 + numAddFieldPuts + i]
+          = "/response/arr[@name='fields']/lst/str[@name='name'][.='" + newFieldName + "']";
+    }
+
+    return expectedAddFields;
+  }
+
+  private String[] getExpectedCopyFieldResponses(List<CopyFieldInfo> copyFields) {
+    ArrayList<String> expectedCopyFields = new ArrayList<>();
+    expectedCopyFields.add(SUCCESS_XPATH);
+    for (CopyFieldInfo cpi : copyFields) {
+      String expectedSourceName = cpi.getSourceField();
+      expectedCopyFields.add
+          ("/response/arr[@name='copyFields']/lst/str[@name='source'][.='" + expectedSourceName + "']");
+      String expectedDestName = cpi.getDestField();
+      expectedCopyFields.add
+          ("/response/arr[@name='copyFields']/lst/str[@name='dest'][.='" + expectedDestName + "']");
+    }
+
+    return expectedCopyFields.toArray(new String[expectedCopyFields.size()]);
+  }
+
+  @Override
+  public void doTest() throws Exception {
+    setupHarnesses();
+    
+    // First, add a bunch of fields via PUT and POST, as well as copyFields,
+    // but do it fast enough and verify shards' schemas after all of them are added
+    int numFields = 100;
+    int numAddFieldPuts = 0;
+    int numAddFieldPosts = 0;
+    List<CopyFieldInfo> copyFields = new ArrayList<>();
+
+    for (int i = 0; i <= numFields ; ++i) {
+      RestTestHarness publisher = restTestHarnesses.get(r.nextInt(restTestHarnesses.size()));
+
+      int type = random().nextInt(3);
+      if (type == 0) { // send an add field via PUT
+        addFieldPut(publisher, "newfieldPut" + numAddFieldPuts++);
+      }
+      else if (type == 1) { // send an add field via POST
+        addFieldPost(publisher, "newfieldPost" + numAddFieldPosts++);
+      }
+      else if (type == 2) { // send a copy field
+        String sourceField = null;
+        String destField = null;
+
+        int sourceType = random().nextInt(3);
+        if (sourceType == 0) {  // existing
+          sourceField = "name";
+        } else if (sourceType == 1) { // newly created
+          sourceField = "copySource" + i;
+          addFieldPut(publisher, sourceField);
+        } else { // dynamic
+          sourceField = "*_dynamicSource" + i + "_t";
+          // * only supported if both src and dst use it
+          destField = "*_dynamicDest" + i + "_t";
+        }
+        
+        if (destField == null) {
+          int destType = random().nextInt(2);
+          if (destType == 0) {  // existing
+            destField = "title";
+          } else { // newly created
+            destField = "copyDest" + i;
+            addFieldPut(publisher, destField);
+          }
+        }
+        copyField(publisher, sourceField, destField);
+        copyFields.add(new CopyFieldInfo(sourceField, destField));
+      }
+    }
+
+    String[] expectedAddFields = getExpectedFieldResponses(numAddFieldPuts, numAddFieldPosts);
+    String[] expectedCopyFields = getExpectedCopyFieldResponses(copyFields);
+
+    boolean success = false;
+    long maxTimeoutMillis = 100000;
+    long startTime = System.nanoTime();
+    String request = null;
+    String response = null;
+    String result = null;
+
+    while ( ! success 
+           && TimeUnit.MILLISECONDS.convert(System.nanoTime() - startTime, TimeUnit.NANOSECONDS) < maxTimeoutMillis) {
+      Thread.sleep(100);
+
+      for (RestTestHarness client : restTestHarnesses) {
+        // verify addFieldPuts and addFieldPosts
+        request = "/schema/fields?wt=xml";
+        response = client.query(request);
+        result = BaseTestHarness.validateXPath(response, expectedAddFields);
+        if (result != null) {
+          break;
+        }
+
+        // verify copyFields
+        request = "/schema/copyfields?wt=xml";
+        response = client.query(request);
+        result = BaseTestHarness.validateXPath(response, expectedCopyFields);
+        if (result != null) {
+          break;
+        }
+      }
+      success = (result == null);
+    }
+    if ( ! success) {
+      String msg = "QUERY FAILED: xpath=" + result + "  request=" + request + "  response=" + response;
+      log.error(msg);
+      fail(msg);
+    }
+  }
+
+  private static class CopyFieldInfo {
+    private String sourceField;
+    private String destField;
+
+    public CopyFieldInfo(String sourceField, String destField) {
+      this.sourceField = sourceField;
+      this.destField = destField;
+    }
+
+    public String getSourceField() { return sourceField; }
+    public String getDestField() { return destField; }
+  }
+}