You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ge...@apache.org on 2019/04/04 10:52:07 UTC

[lucene-solr] branch master updated: SOLR-13331: Fix AtomicUpdate 'remove' ops in SolrJ

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 9b44fba  SOLR-13331: Fix AtomicUpdate 'remove' ops in SolrJ
9b44fba is described below

commit 9b44fba0d49a25b815698bb163708146fffd7f91
Author: Jason Gerlowski <ge...@apache.org>
AuthorDate: Wed Apr 3 07:06:21 2019 -0400

    SOLR-13331: Fix AtomicUpdate 'remove' ops in SolrJ
    
    The racent change introducing ByteArrayUtf8CharSequence altered the
    NamedLists produced by atomic-update requests so that they include
    instances of this class for requests coming in as javabin.  This is a
    problem for 'remove' atomic-updates, which need to be able to compare
    these ByteArrayUtf8CharSequence instances with existing field values
    represented as Strings.  equals() would always return false, and
    'remove' operations would have no effect.
    
    This commit converts items as necessary to allow atomic-update
    operations to work as expected.
---
 solr/CHANGES.txt                                   |   2 +
 .../org/apache/solr/schema/AbstractEnumField.java  |  12 ++
 .../src/java/org/apache/solr/schema/FieldType.java |   3 +
 .../solr/configsets/cloud-dynamic/conf/schema.xml  |   2 +
 .../processor/AtomicUpdateRemovalJavabinTest.java  | 134 +++++++++++++++++++++
 5 files changed, 153 insertions(+)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index e8f0bf7..43347f2 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -154,6 +154,8 @@ Bug Fixes
 
 * SOLR-13355: 'all' permission ignored by RuleBasedAuthorizationPlugin in most cases (Jason Gerlowski, janhoy)
 
+* SOLR-13331: Atomic Update 'remove' operations broken for certain field types in SolrJ (Thomas Wockinger via Jason Gerlowski)
+
 Improvements
 ----------------------
 
diff --git a/solr/core/src/java/org/apache/solr/schema/AbstractEnumField.java b/solr/core/src/java/org/apache/solr/schema/AbstractEnumField.java
index 10060d7..69b7768 100644
--- a/solr/core/src/java/org/apache/solr/schema/AbstractEnumField.java
+++ b/solr/core/src/java/org/apache/solr/schema/AbstractEnumField.java
@@ -310,4 +310,16 @@ public abstract class AbstractEnumField extends PrimitiveFieldType {
 
     return enumMapping.intValueToStringValue(val.intValue());
   }
+  
+  @Override
+  public Object toNativeType(Object val) {
+    if (val instanceof CharSequence || val instanceof String) {
+      final String str = val.toString();
+      return new EnumFieldValue(enumMapping.enumStringToIntMap.get(str), str);
+    } else if(val instanceof Number) {
+      final int num = ((Number)val).intValue();
+      return new EnumFieldValue(num, enumMapping.enumIntToStringMap.get(num));
+    }
+    return super.toNativeType(val);
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldType.java b/solr/core/src/java/org/apache/solr/schema/FieldType.java
index f960d8e..69ef981 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java
@@ -1184,6 +1184,9 @@ public abstract class FieldType extends FieldProperties {
   /**Converts any Object to a java Object native to this field type
    */
   public Object toNativeType(Object val) {
+    if (val instanceof CharSequence) {
+      return ((CharSequence) val).toString();
+    }
     return val;
   }
   
diff --git a/solr/core/src/test-files/solr/configsets/cloud-dynamic/conf/schema.xml b/solr/core/src/test-files/solr/configsets/cloud-dynamic/conf/schema.xml
index 1d18671..5e2719d 100644
--- a/solr/core/src/test-files/solr/configsets/cloud-dynamic/conf/schema.xml
+++ b/solr/core/src/test-files/solr/configsets/cloud-dynamic/conf/schema.xml
@@ -268,7 +268,9 @@
   <dynamicField name="*_b" type="boolean" indexed="true" stored="true"/>
   <dynamicField name="*_f" type="float" indexed="true" stored="true"/>
   <dynamicField name="*_d" type="double" indexed="true" stored="true"/>
+  <dynamicField name="*_md" type="double" indexed="true" stored="true" multiValued="true"/>
   <dynamicField name="*_dt" type="date" indexed="true" stored="true"/>
+  <dynamicField name="*_mdt" type="date" indexed="true" stored="true" multiValued="true"/>
 
   <!-- some trie-coded dynamic fields for faster range queries -->
   <dynamicField name="*_ti" type="tint" indexed="true" stored="true"/>
diff --git a/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdateRemovalJavabinTest.java b/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdateRemovalJavabinTest.java
new file mode 100644
index 0000000..61a94f5
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/update/processor/AtomicUpdateRemovalJavabinTest.java
@@ -0,0 +1,134 @@
+/*
+ * 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.time.Instant;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrInputDocument;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Tests Solr's atomic-update functionality using requests sent through SolrJ using wt=javabin
+ *
+ * {@link AtomicUpdatesTest} covers some of the same functionality, but does so by making xml-based requests.  Recent
+ * changes to Solr have made it possible for the same data sent with different formats to result in different NamedLists
+ * after unmarshalling, so the test duplication is now necessary.  See SOLR-13331 for an example.
+ */
+public class AtomicUpdateRemovalJavabinTest extends SolrCloudTestCase {
+  private static final String COLLECTION = "collection1";
+  private static final int NUM_SHARDS = 1;
+  private static final int NUM_REPLICAS = 1;
+  private static final int MAX_SHARDS_PER_NODE = 1;
+  private static final Date DATE_1 = new Date();
+  private static final Date DATE_2 = Date.from(Instant.ofEpochSecond(1554243909));
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(1)
+        .addConfig("conf", configset("cloud-dynamic"))
+        .configure();
+
+    CollectionAdminRequest.createCollection(COLLECTION, "conf", NUM_SHARDS, NUM_REPLICAS)
+        .setMaxShardsPerNode(MAX_SHARDS_PER_NODE)
+        .process(cluster.getSolrClient());
+
+    cluster.waitForActiveCollection(COLLECTION, 1, 1);
+
+    final SolrInputDocument doc1 = sdoc(
+        "id", "1",
+        "title_s", "title_1", "title_s", "title_2",
+        "tv_mv_text", "text_1", "tv_mv_text", "text_2",
+        "count_is", 1, "count_is", 2,
+        "count_md", 1.0, "count_md", 2.0,
+        "timestamps_mdt", DATE_1, "timestamps_mdt", DATE_2);
+    final UpdateRequest req = new UpdateRequest()
+        .add(doc1);
+    req.commit(cluster.getSolrClient(), COLLECTION);
+  }
+
+  @Test
+  public void testAtomicUpdateRemovalOfStrField() throws Exception {
+    ensureFieldHasValues("1", "title_s", "title_1", "title_2");
+    atomicRemoveValueFromField("1", "title_s", "title_1");
+    ensureFieldHasValues("1", "title_s", "title_2");
+  }
+
+  @Test
+  public void testAtomicUpdateRemovalOfTextField() throws Exception {
+    ensureFieldHasValues("1", "tv_mv_text", "text_1", "text_2");
+    atomicRemoveValueFromField("1", "tv_mv_text", "text_1");
+    ensureFieldHasValues("1", "tv_mv_text", "text_2");
+  }
+
+  @Test
+  public void testAtomicUpdateRemovalOfIntField() throws Exception {
+    ensureFieldHasValues("1", "count_is", 1, 2);
+    atomicRemoveValueFromField("1", "count_is", 1);
+    ensureFieldHasValues("1", "count_is", 2);
+  }
+
+  @Test
+  public void testAtomicUpdateRemovalOfDoubleField() throws Exception {
+    ensureFieldHasValues("1", "count_md", 1.0, 2.0);
+    atomicRemoveValueFromField("1", "count_md", 1.0);
+    ensureFieldHasValues("1", "count_md", 2.0);
+  }
+
+  @Test
+  public void testAtomicUpdateRemovalOfDateField() throws Exception {
+    ensureFieldHasValues("1", "timestamps_mdt", DATE_1, DATE_2);
+    atomicRemoveValueFromField("1", "timestamps_mdt", DATE_1);
+    ensureFieldHasValues("1", "timestamps_mdt", DATE_2);
+  }
+
+  private void atomicRemoveValueFromField(String docId, String fieldName, Object value) throws Exception {
+    final SolrInputDocument doc = new SolrInputDocument();
+    doc.setField("id", docId);
+    Map<String, Object> atomicUpdateRemoval = new HashMap<>(1);
+    atomicUpdateRemoval.put("remove", value);
+    doc.setField(fieldName, atomicUpdateRemoval);
+
+    cluster.getSolrClient().add(COLLECTION, doc);
+    cluster.getSolrClient().commit(COLLECTION);
+  }
+
+  private void ensureFieldHasValues(String identifyingDocId, String fieldName, Object... expectedValues) throws Exception {
+    final SolrClient client = cluster.getSolrClient();
+
+    final QueryResponse response = client.query(COLLECTION, new SolrQuery("id:" + identifyingDocId));
+    final SolrDocumentList docs = response.getResults();
+    assertEquals(1, docs.getNumFound());
+    final Collection<Object> valuesAfterUpdate = docs.get(0).getFieldValues(fieldName);
+    assertEquals(expectedValues.length, valuesAfterUpdate.size());
+    for (Object expectedValue: expectedValues) {
+      assertTrue("Expected value [" + expectedValue + "] was not found in field", valuesAfterUpdate.contains(expectedValue));
+    }
+  }
+}