You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2018/10/08 02:30:25 UTC

[03/30] lucene-solr:jira/http2: SOLR-12824: NamedList to implement MapWriter interface

SOLR-12824: NamedList to implement MapWriter interface


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

Branch: refs/heads/jira/http2
Commit: 961d565955ed9b2acd8f44319c16b627d6192714
Parents: fe844c7
Author: Noble Paul <no...@apache.org>
Authored: Tue Oct 2 17:34:58 2018 +1000
Committer: Noble Paul <no...@apache.org>
Committed: Tue Oct 2 17:34:58 2018 +1000

----------------------------------------------------------------------
 .../solr/cloud/CollectionsAPISolrJTest.java     | 11 ++--
 .../solr/cloud/autoscaling/CapturedEvent.java   |  2 +-
 .../SearchRateTriggerIntegrationTest.java       |  7 +-
 .../test/org/apache/solr/util/TestUtils.java    | 67 +++++++++++++++++---
 .../client/solrj/response/SolrResponseBase.java | 12 +++-
 .../java/org/apache/solr/common/MapWriter.java  |  7 +-
 .../org/apache/solr/common/MapWriterMap.java    | 39 ++++++++++++
 .../java/org/apache/solr/common/util/Utils.java | 49 ++++++++++----
 8 files changed, 162 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/961d5659/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
index 5390704..cd9087d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
@@ -189,15 +189,14 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
 
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
-    String nodeName = ((NamedList) response.getResponse().get("success")).getName(0);
-    String corename = (String) response.getResponse()._get(asList("success", nodeName,"core"),null);
+    String nodeName = (String) response._get("success[0]/key", null);
+    String corename = (String) response._get(asList("success", nodeName, "core"), null);
 
     try (HttpSolrClient coreclient = getHttpSolrClient(cluster.getSolrClient().getZkStateReader().getBaseUrlForNodeName(nodeName))) {
       CoreAdminResponse status = CoreAdminRequest.getStatus(corename, coreclient);
-      NamedList m = status.getResponse();
-      assertEquals(collectionName, m._get(asList("status", corename, "cloud", "collection"), null));
-      assertNotNull(m._get(asList("status", corename, "cloud", "shard"), null));
-      assertNotNull(m._get(asList("status", corename, "cloud", "replica"), null));
+      assertEquals(collectionName, status._get(asList("status", corename, "cloud", "collection"), null));
+      assertNotNull(status._get(asList("status", corename, "cloud", "shard"), null));
+      assertNotNull(status._get(asList("status", corename, "cloud", "replica"), null));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/961d5659/solr/core/src/test/org/apache/solr/cloud/autoscaling/CapturedEvent.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/CapturedEvent.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/CapturedEvent.java
index 462e948..d5c3127 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/CapturedEvent.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/CapturedEvent.java
@@ -38,7 +38,7 @@ public class CapturedEvent {
   public CapturedEvent(long timestamp, ActionContext context, AutoScalingConfig.TriggerListenerConfig config, TriggerEventProcessorStage stage, String actionName,
                        TriggerEvent event, String message) {
     if (context != null) {
-      context.toMap(this.context);
+      context._forEachEntry((o, o2) -> CapturedEvent.this.context.put((String) o, o2));
       TriggerEvent.fixOps("properties." + TriggerEvent.REQUESTED_OPS, this.context);
       TriggerEvent.fixOps("properties." + TriggerEvent.UNSUPPORTED_OPS, this.context);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/961d5659/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java
index 5d7ce99..106ec17 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java
@@ -38,6 +38,7 @@ import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.cloud.CloudTestUtils;
 import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -264,11 +265,11 @@ public class SearchRateTriggerIntegrationTest extends SolrCloudTestCase {
     assertEquals(collectionRate, totalReplicaRate.get(), 5.0);
 
     // check operations
-    List<Map<String, Object>> ops = (List<Map<String, Object>>) ev.context.get("properties.operations");
+    List<MapWriter> ops = (List<MapWriter>) ev.context.get("properties.operations");
     assertNotNull(ops);
     assertTrue(ops.size() > 1);
-    for (Map<String, Object> m : ops) {
-      assertEquals("ADDREPLICA", m.get("params.action"));
+    for (MapWriter m : ops) {
+      assertEquals("ADDREPLICA", m._get("params.action",null));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/961d5659/solr/core/src/test/org/apache/solr/util/TestUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/util/TestUtils.java b/solr/core/src/test/org/apache/solr/util/TestUtils.java
index 88dbba2..27db2d6 100644
--- a/solr/core/src/test/org/apache/solr/util/TestUtils.java
+++ b/solr/core/src/test/org/apache/solr/util/TestUtils.java
@@ -19,7 +19,6 @@ package org.apache.solr.util;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.StringReader;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -40,6 +39,7 @@ import org.apache.solr.common.util.Utils;
 import org.junit.Assert;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.Arrays.asList;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_DEF;
 import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.NUM_SHARDS_PROP;
@@ -51,12 +51,12 @@ import static org.apache.solr.common.util.Utils.fromJSONString;
 public class TestUtils extends SolrTestCaseJ4 {
   
   public void testJoin() {
-    assertEquals("a|b|c",   StrUtils.join(Arrays.asList("a","b","c"), '|'));
-    assertEquals("a,b,c",   StrUtils.join(Arrays.asList("a","b","c"), ','));
-    assertEquals("a\\,b,c", StrUtils.join(Arrays.asList("a,b","c"), ','));
-    assertEquals("a,b|c",   StrUtils.join(Arrays.asList("a,b","c"), '|'));
+    assertEquals("a|b|c",   StrUtils.join(asList("a","b","c"), '|'));
+    assertEquals("a,b,c",   StrUtils.join(asList("a","b","c"), ','));
+    assertEquals("a\\,b,c", StrUtils.join(asList("a,b","c"), ','));
+    assertEquals("a,b|c",   StrUtils.join(asList("a,b","c"), '|'));
 
-    assertEquals("a\\\\b|c",   StrUtils.join(Arrays.asList("a\\b","c"), '|'));
+    assertEquals("a\\\\b|c",   StrUtils.join(asList("a\\b","c"), '|'));
   }
 
   public void testEscapeTextWithSeparator() {
@@ -191,8 +191,8 @@ public class TestUtils extends SolrTestCaseJ4 {
       jbc.marshal((MapWriter) ew -> {
         ew.put("set-user", fromJSONString("{x:y}"));
         ew.put("set-user", fromJSONString("{x:y,x1:y1}"));
-        ew.put("single", Arrays.asList(fromJSONString("[{x:y,x1:y1},{x2:y2}]"), fromJSONString( "{x2:y2}")));
-        ew.put("multi", Arrays.asList(fromJSONString("{x:y,x1:y1}"), fromJSONString( "{x2:y2}")));
+        ew.put("single", asList(fromJSONString("[{x:y,x1:y1},{x2:y2}]"), fromJSONString( "{x2:y2}")));
+        ew.put("multi", asList(fromJSONString("{x:y,x1:y1}"), fromJSONString( "{x2:y2}")));
       }, baos);
     }
 
@@ -252,6 +252,57 @@ public class TestUtils extends SolrTestCaseJ4 {
     assertEquals("x-update", Utils.getObjectByPath(m,false, "authorization/permissions[1]/name"));
 
   }
+  
+  public void testMapWriterIdx(){
+    String json = "{" +
+        "  'responseHeader':{" +
+        "    'status':0," +
+        "    'QTime':6752}," +
+        "  'success':{" +
+        "    '127.0.0.1:56443_solr':{" +
+        "      'responseHeader':{" +
+        "        'status':0," +
+        "        'QTime':4276}," +
+        "      'core':'corestatus_test_shard2_replica_n5'}," +
+        "    '127.0.0.1:56445_solr':{" +
+        "      'responseHeader':{" +
+        "        'status':0," +
+        "        'QTime':4271}," +
+        "      'core':'corestatus_test_shard1_replica_n1'}," +
+        "    '127.0.0.1:56446_solr':{" +
+        "      'responseHeader':{" +
+        "        'status':0," +
+        "        'QTime':5015}," +
+        "      'core':'corestatus_test_shard1_replica_n2'}," +
+        "    '127.0.0.1:56444_solr':{" +
+        "      'responseHeader':{" +
+        "        'status':0," +
+        "        'QTime':5033}," +
+        "      'core':'corestatus_test_shard2_replica_n3'}}}";
+    Map m = (Map) fromJSONString(json);
+
+    assertEquals("127.0.0.1:56443_solr", Utils.getObjectByPath(m,false, "success[0]/key"));
+    assertEquals("corestatus_test_shard2_replica_n5", Utils.getObjectByPath(m, false,asList("success[0]", "value", "core") ));
+    assertEquals(4276L, Utils.getObjectByPath(m, false,asList("success[0]", "value", "responseHeader", "QTime") ));
+
+    assertEquals("127.0.0.1:56444_solr", Utils.getObjectByPath(m,false, "success[3]/key"));
+    assertEquals("corestatus_test_shard2_replica_n3", Utils.getObjectByPath(m, false,asList("success[3]", "value", "core") ));
+    assertEquals(5033L, Utils.getObjectByPath(m, false,asList("success[3]", "value", "responseHeader", "QTime") ));
+
+    Map nodes = (Map) m.get("success");
+    m.put("success", (MapWriter) ew -> nodes.forEach((o, o2) -> ew.putNoEx((String) o,o2)));
+    assertEquals("127.0.0.1:56443_solr", Utils.getObjectByPath(m,false, "success[0]/key"));
+    assertEquals("corestatus_test_shard2_replica_n5", Utils.getObjectByPath(m, false,asList("success[0]", "value", "core") ));
+    assertEquals(4276L, Utils.getObjectByPath(m, false,asList("success[0]", "value", "responseHeader", "QTime") ));
+
+    assertEquals("127.0.0.1:56444_solr", Utils.getObjectByPath(m,false, "success[3]/key"));
+    assertEquals("corestatus_test_shard2_replica_n3", Utils.getObjectByPath(m, false,asList("success[3]", "value", "core") ));
+    assertEquals(5033L, Utils.getObjectByPath(m, false,asList("success[3]", "value", "responseHeader", "QTime") ));
+    final int[] count = {0};
+    NamedList nl = new NamedList(m);
+    nl._forEachEntry("success", (o, o2) -> count[0]++);
+    assertEquals(count[0], 4);
+  }
 
   public void testMergeJson() {
     Map<String, Object> sink = (Map<String, Object>) Utils.fromJSONString("{k2:v2, k1: {a:b, p:r, k21:{xx:yy}}}");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/961d5659/solr/solrj/src/java/org/apache/solr/client/solrj/response/SolrResponseBase.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/response/SolrResponseBase.java b/solr/solrj/src/java/org/apache/solr/client/solrj/response/SolrResponseBase.java
index d20481c..ffadb38 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/response/SolrResponseBase.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/response/SolrResponseBase.java
@@ -16,7 +16,10 @@
  */
 package org.apache.solr.client.solrj.response;
 
+import java.io.IOException;
+
 import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.util.NamedList;
 
 /**
@@ -24,12 +27,17 @@ import org.apache.solr.common.util.NamedList;
  *
  * @since solr 1.3
  */
-public class SolrResponseBase extends SolrResponse
+public class SolrResponseBase extends SolrResponse implements MapWriter
 {
   private long elapsedTime = -1;
   private NamedList<Object> response = null;
   private String requestUrl = null;
-  
+
+  @Override
+  public void writeMap(EntryWriter ew) throws IOException {
+    if (response != null) response.writeMap(ew);
+  }
+
   @Override
   public long getElapsedTime() {
     return elapsedTime;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/961d5659/solr/solrj/src/java/org/apache/solr/common/MapWriter.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/MapWriter.java b/solr/solrj/src/java/org/apache/solr/common/MapWriter.java
index 3ec37e7..3256b6a 100644
--- a/solr/solrj/src/java/org/apache/solr/common/MapWriter.java
+++ b/solr/solrj/src/java/org/apache/solr/common/MapWriter.java
@@ -91,12 +91,17 @@ public interface MapWriter extends MapSerializable {
     Object v = Utils.getObjectByPath(this, false, path);
     return v == null ? def : v;
   }
+
   default void _forEachEntry(String path, BiConsumer fun) {
     Utils.forEachMapEntry(this, path, fun);
   }
 
+  default void _forEachEntry(List<String> path, BiConsumer fun) {
+    Utils.forEachMapEntry(this, path, fun);
+  }
+
   default void _forEachEntry(BiConsumer fun) {
-    Utils.forEachMapEntry(this, null, fun);
+    Utils.forEachMapEntry(this, fun);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/961d5659/solr/solrj/src/java/org/apache/solr/common/MapWriterMap.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/MapWriterMap.java b/solr/solrj/src/java/org/apache/solr/common/MapWriterMap.java
new file mode 100644
index 0000000..f6580b2
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/MapWriterMap.java
@@ -0,0 +1,39 @@
+/*
+ * 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.common;
+
+import java.io.IOException;
+import java.util.Map;
+
+public class MapWriterMap implements MapWriter {
+  private final Map delegate;
+
+  public MapWriterMap(Map delegate) {
+    this.delegate = delegate;
+  }
+
+  @Override
+  public void writeMap(EntryWriter ew) throws IOException {
+    delegate.forEach((k, v) -> ew.putNoEx(k == null ? null : k.toString(), v));
+  }
+
+  @Override
+  public Map toMap(Map<String, Object> map) {
+    return delegate;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/961d5659/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/Utils.java b/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
index 2d1df42..c8274f3 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
@@ -30,6 +30,7 @@ import java.net.URL;
 import java.net.URLDecoder;
 import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
+import java.util.AbstractMap;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -54,6 +55,7 @@ import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
 import org.apache.solr.client.solrj.impl.BinaryRequestWriter;
 import org.apache.solr.common.IteratorWriter;
 import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.MapWriterMap;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SpecProvider;
 import org.apache.solr.common.cloud.SolrZkClient;
@@ -104,7 +106,16 @@ public class Utils {
   }
 
   public static void forEachMapEntry(MapWriter mw, String path, BiConsumer fun) {
-    Object o = path == null ? mw : Utils.getObjectByPath(mw, false, path);
+    Object o = Utils.getObjectByPath(mw, false, path);
+    forEachMapEntry(o, fun);
+  }
+
+  public static void forEachMapEntry(MapWriter mw, List<String> path, BiConsumer fun) {
+    Object o = Utils.getObjectByPath(mw, false, path);
+    forEachMapEntry(o, fun);
+  }
+
+  public static void forEachMapEntry(Object o, BiConsumer fun) {
     if (o instanceof MapWriter) {
       MapWriter m = (MapWriter) o;
       try {
@@ -365,7 +376,7 @@ public class Utils {
     for (int i = 0; i < hierarchy.size(); i++) {
       int idx = -1;
       String s = hierarchy.get(i);
-      if (s.endsWith("]")) {
+      if (s != null && s.endsWith("]")) {
         Matcher matcher = ARRAY_ELEMENT_INDEX.matcher(s);
         if (matcher.find()) {
           s = matcher.group(1);
@@ -376,8 +387,14 @@ public class Utils {
         Object o = getVal(obj, s, -1);
         if (o == null) return null;
         if (idx > -1) {
-          List l = (List) o;
-          o = idx < l.size() ? l.get(idx) : null;
+          if (o instanceof MapWriter) {
+            o = getVal(o, null, idx);
+          } else if (o instanceof Map) {
+            o = getVal(new MapWriterMap((Map) o), null, idx);
+          } else {
+            List l = (List) o;
+            o = idx < l.size() ? l.get(idx) : null;
+          }
         }
         if (!isMapLike(o)) return null;
         obj = o;
@@ -385,10 +402,7 @@ public class Utils {
         Object val = getVal(obj, s, -1);
         if (val == null) return null;
         if (idx > -1) {
-          if (val instanceof MapWriter) {
-            val = getVal((MapWriter) val, null, idx);
-
-          } else if (val instanceof IteratorWriter) {
+          if (val instanceof IteratorWriter) {
             val = getValueAt((IteratorWriter) val, idx);
           } else {
             List l = (List) val;
@@ -427,6 +441,19 @@ public class Utils {
 
   }
 
+  static class MapWriterEntry<V> extends AbstractMap.SimpleEntry<String, V> implements MapWriter, Map.Entry<String, V> {
+    MapWriterEntry(String key, V value) {
+      super(key, value);
+    }
+
+    @Override
+    public void writeMap(EntryWriter ew) throws IOException {
+      ew.put("key", getKey());
+      ew.put("value", getValue());
+    }
+
+  }
+
   private static boolean isMapLike(Object o) {
     return o instanceof Map || o instanceof NamedList || o instanceof MapWriter;
   }
@@ -440,10 +467,10 @@ public class Utils {
           @Override
           public MapWriter.EntryWriter put(String k, Object v) {
             if (result[0] != null) return this;
-            if (k != null) {
-              if (key.equals(k)) result[0] = v;
+            if (idx < 0) {
+              if (k.equals(key)) result[0] = v;
             } else {
-              if (++count == idx) result[0] = v;
+              if (++count == idx) result[0] = new MapWriterEntry(k, v);
             }
             return this;
           }