You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2019/10/06 22:48:44 UTC

[lucene-solr] branch branch_8x updated (67de998 -> 888fe76)

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

noble pushed a change to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git.


    from 67de998  SOLR-13719: introducing SolrClient.ping(collection)
     new bb3d4e4  SOLR-13787: An annotation based system to write v2  APIs
     new 888fe76  String#isBlank() is java 11 API

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 solr/CHANGES.txt                                   |   2 +
 .../src/java/org/apache/solr/api/AnnotatedApi.java | 275 +++++++++++++++++++++
 .../core/src/java/org/apache/solr/api/Command.java |  25 +-
 .../src/java/org/apache/solr/api/EndPoint.java     |  24 +-
 .../solr/handler/admin/TestApiFramework.java       | 136 +++++++++-
 .../java/org/apache/solr/common/util/PathTrie.java |  44 +++-
 .../apache/solr/common/util/ValidatingJsonMap.java |   3 +-
 .../org/apache/solr/common/util/TestPathTrie.java  |  13 +
 8 files changed, 489 insertions(+), 33 deletions(-)
 create mode 100644 solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
 copy lucene/core/src/java/org/apache/lucene/util/SuppressForbidden.java => solr/core/src/java/org/apache/solr/api/Command.java (71%)
 copy lucene/core/src/java/org/apache/lucene/util/SuppressForbidden.java => solr/core/src/java/org/apache/solr/api/EndPoint.java (71%)


[lucene-solr] 01/02: SOLR-13787: An annotation based system to write v2 APIs

Posted by no...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit bb3d4e41f0125c06bf3c5cdc580c1fbe563d48e2
Author: Noble Paul <no...@users.noreply.github.com>
AuthorDate: Mon Oct 7 09:19:57 2019 +1100

    SOLR-13787: An annotation based system to write v2  APIs
    
    This is to make V2 APIs easier to write and less error prone
    * All specs are always in sync with code
    * specs are generated from code
    *  no need to learn and write json schema
---
 solr/CHANGES.txt                                   |   2 +
 .../src/java/org/apache/solr/api/AnnotatedApi.java | 275 +++++++++++++++++++++
 .../core/src/java/org/apache/solr/api/Command.java |  37 +++
 .../src/java/org/apache/solr/api/EndPoint.java     |  36 +++
 .../solr/handler/admin/TestApiFramework.java       | 136 +++++++++-
 .../java/org/apache/solr/common/util/PathTrie.java |  44 +++-
 .../apache/solr/common/util/ValidatingJsonMap.java |   3 +-
 .../org/apache/solr/common/util/TestPathTrie.java  |  13 +
 8 files changed, 535 insertions(+), 11 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 20166dd..167c8d0 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -229,6 +229,8 @@ Other Changes
 * SOLR-13812: Add javadocs, uneven rejection and basic test coverage for the SolrTestCaseJ4.params method.
   (Diego Ceccarelli, Christine Poerschke, Munendra S N)
 
+* SOLR-13787: An annotation based system to write v2 APIs (noble)
+
 ==================  8.2.0 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
diff --git a/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java b/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
new file mode 100644
index 0000000..b1be461
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
@@ -0,0 +1,275 @@
+/*
+ * 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.api;
+
+
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SpecProvider;
+import org.apache.solr.common.util.CommandOperation;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.common.util.ValidatingJsonMap;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.security.AuthorizationContext;
+import org.apache.solr.security.PermissionNameProvider;
+
+/**This class implements an Api just from  an annotated java class
+ * The class must have an annotation {@link EndPoint}
+ * Each method must have an annotation {@link Command}
+ * The methods that implement a command should have the first 2 parameters
+ * {@link SolrQueryRequest} and {@link SolrQueryResponse} or it may optionally
+ * have a third parameter which could be a java class annotated with jackson annotations.
+ * The third parameter is only valid if it is using a json command payload
+ *
+ */
+
+public class AnnotatedApi extends Api implements PermissionNameProvider {
+  private EndPoint endPoint;
+  private Map<String, Cmd> commands = new HashMap<>();
+  private final Api fallback;
+
+  public AnnotatedApi(Object obj) {
+    this(obj, null);
+
+  }
+
+  public AnnotatedApi(Object obj, Api fallback) {
+    super(readSpec(obj.getClass()));
+    this.fallback = fallback;
+    Class<?> klas = obj.getClass();
+    if (!Modifier.isPublic(klas.getModifiers())) {
+      throw new RuntimeException(obj.getClass().getName() + " is not public");
+    }
+
+    endPoint = klas.getAnnotation(EndPoint.class);
+
+    for (Method m : klas.getDeclaredMethods()) {
+      Command command = m.getAnnotation(Command.class);
+      if (command == null) continue;
+
+      if (commands.containsKey(command.name())) {
+        throw new RuntimeException("Duplicate commands " + command.name());
+      }
+      commands.put(command.name(), new Cmd(command, obj, m));
+    }
+
+  }
+
+  @Override
+  public Name getPermissionName(AuthorizationContext request) {
+    return endPoint.permission();
+  }
+
+  private static SpecProvider readSpec(Class klas) {
+    EndPoint endPoint = (EndPoint) klas.getAnnotation(EndPoint.class);
+    if (endPoint == null) throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Invalid class :  "+ klas.getName());
+    EndPoint endPoint1 = (EndPoint) klas.getAnnotation(EndPoint.class);
+    return () -> {
+      Map map = new LinkedHashMap();
+      List<String> methods = new ArrayList<>();
+      for (SolrRequest.METHOD method : endPoint1.method()) {
+        methods.add(method.name());
+      }
+      map.put("methods", methods);
+      map.put("url", new ValidatingJsonMap(Collections.singletonMap("paths", Arrays.asList(endPoint1.path()))));
+      Map<String, Object> cmds = new HashMap<>();
+
+      for (Method method : klas.getMethods()) {
+        Command command = method.getAnnotation(Command.class);
+        if (command != null && !command.name().isBlank()) {
+          cmds.put(command.name(), AnnotatedApi.createSchema(method));
+        }
+      }
+      if (!cmds.isEmpty()) {
+        map.put("commands", cmds);
+      }
+      return new ValidatingJsonMap(map);
+    };
+
+
+  }
+
+
+  @Override
+  public void call(SolrQueryRequest req, SolrQueryResponse rsp) {
+    if (commands.size() == 1) {
+      Cmd cmd = commands.get("");
+      if (cmd != null) {
+        cmd.invoke(req, rsp, null);
+        return;
+      }
+    }
+
+    List<CommandOperation> cmds = req.getCommands(true);
+    boolean allExists = true;
+    for (CommandOperation cmd : cmds) {
+      if (!commands.containsKey(cmd.name)) {
+        cmd.addError("No such command supported: " + cmd.name);
+        allExists = false;
+      }
+    }
+    if (!allExists) {
+      if (fallback != null) {
+        fallback.call(req, rsp);
+        return;
+      } else {
+        throw new ApiBag.ExceptionWithErrObject(SolrException.ErrorCode.BAD_REQUEST, "Error processing commands",
+            CommandOperation.captureErrors(cmds));
+      }
+    }
+
+    for (CommandOperation cmd : cmds) {
+      commands.get(cmd.name).invoke(req, rsp, cmd);
+    }
+
+    List<Map> errs = CommandOperation.captureErrors(cmds);
+    if (!errs.isEmpty()) {
+      throw new ApiBag.ExceptionWithErrObject(SolrException.ErrorCode.BAD_REQUEST, "Error in executing commands", errs);
+    }
+
+  }
+
+  class Cmd {
+    final Command command;
+    final Method method;
+    final Object obj;
+    ObjectMapper mapper = new ObjectMapper();
+    int paramsCount;
+    Class c;
+
+
+    Cmd(Command command, Object obj, Method method) {
+      if (Modifier.isPublic(method.getModifiers())) {
+        this.command = command;
+        this.obj = obj;
+        this.method = method;
+        Class<?>[] parameterTypes = method.getParameterTypes();
+        paramsCount = parameterTypes.length;
+        if (parameterTypes[0] != SolrQueryRequest.class || parameterTypes[1] != SolrQueryResponse.class) {
+          throw new RuntimeException("Invalid params for method " + method);
+        }
+        if (parameterTypes.length == 3) {
+          c = parameterTypes[2];
+        }
+        if (parameterTypes.length > 3) {
+          throw new RuntimeException("Invalid params count for method " + method);
+
+        }
+      } else {
+        throw new RuntimeException(method.toString() + " is not a public static method");
+      }
+
+    }
+
+    void invoke(SolrQueryRequest req, SolrQueryResponse rsp, CommandOperation cmd) {
+      try {
+
+        if (paramsCount == 2) {
+          method.invoke(obj, req, rsp);
+        } else {
+          Object o = cmd.getCommandData();
+          if (o instanceof Map && c != null) {
+            o = mapper.readValue(Utils.toJSONString(o), c);
+          }
+          method.invoke(obj, req, rsp, o);
+        }
+
+      } catch (SolrException se) {
+        throw se;
+      } catch (InvocationTargetException ite) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, ite.getCause());
+      } catch (Exception e) {
+      }
+
+    }
+  }
+
+  private static final Map<Class, String> primitives = new HashMap<>();
+
+  static {
+    primitives.put(String.class, "string");
+    primitives.put(Integer.class, "integer");
+    primitives.put(int.class, "integer");
+    primitives.put(Float.class, "number");
+    primitives.put(float.class, "number");
+    primitives.put(Double.class, "number");
+    primitives.put(double.class, "number");
+    primitives.put(Boolean.class, "boolean");
+    primitives.put(List.class, "array");
+  }
+
+
+  public static Map<String, Object> createSchema(Method m) {
+    Type[] types = m.getGenericParameterTypes();
+    Map<String, Object> result;
+    if (types.length == 3) {
+      return createSchemaFromType(types[2]);
+
+    }
+    return null;
+  }
+
+  private static Map<String, Object> createSchemaFromType(Type t) {
+    Map<String, Object> map = new LinkedHashMap<>();
+
+    if (primitives.containsKey(t)) {
+      map.put("type", primitives.get(t));
+    } else if (t == List.class) {
+
+    } else if (t instanceof ParameterizedType && ((ParameterizedType) t).getRawType() == List.class) {
+      Type typ = ((ParameterizedType) t).getActualTypeArguments()[0];
+      map.put("type", "array");
+      map.put("items", createSchemaFromType(typ));
+    } else {
+      createObjectSchema((Class) t, map);
+    }
+    return map;
+  }
+
+  private static void createObjectSchema(Class klas, Map<String, Object> map) {
+    map.put("type", "object");
+    Map<String, Object> props = new HashMap<>();
+    map.put("properties", props);
+    for (Field fld : klas.getDeclaredFields()) {
+      JsonProperty p = fld.getAnnotation(JsonProperty.class);
+      if (p == null) continue;
+      props.put(p.value(), createSchemaFromType(fld.getGenericType()));
+
+
+    }
+  }
+
+
+}
diff --git a/solr/core/src/java/org/apache/solr/api/Command.java b/solr/core/src/java/org/apache/solr/api/Command.java
new file mode 100644
index 0000000..d18d064
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/Command.java
@@ -0,0 +1,37 @@
+/*
+ * 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.api;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.METHOD)
+public @interface Command {
+  /**if this is not a json command , leave it empty.
+   * Keep in mind that you cannot have duplicates.
+   * Only one method per name
+   *
+   */
+  String name() default "";
+
+  String jsonSchema() default "";
+
+}
diff --git a/solr/core/src/java/org/apache/solr/api/EndPoint.java b/solr/core/src/java/org/apache/solr/api/EndPoint.java
new file mode 100644
index 0000000..6cbe500
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/api/EndPoint.java
@@ -0,0 +1,36 @@
+/*
+ * 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.api;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.security.PermissionNameProvider;
+
+@Retention(RetentionPolicy.RUNTIME)
+@Target({ElementType.TYPE})
+public @interface EndPoint {
+  SolrRequest.METHOD[] method();
+
+  String[] path();
+
+  PermissionNameProvider.Name permission();
+}
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java b/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
index 1762ec6..933b862 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestApiFramework.java
@@ -17,6 +17,9 @@
 
 package org.apache.solr.handler.admin;
 
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -24,14 +27,23 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.api.AnnotatedApi;
 import org.apache.solr.api.Api;
 import org.apache.solr.api.ApiBag;
+import org.apache.solr.api.Command;
+import org.apache.solr.api.EndPoint;
 import org.apache.solr.api.V2HttpCall;
 import org.apache.solr.api.V2HttpCall.CompositeApi;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.CommandOperation;
+import org.apache.solr.common.util.ContentStream;
+import org.apache.solr.common.util.ContentStreamBase;
+import org.apache.solr.common.util.JsonSchemaValidator;
 import org.apache.solr.common.util.PathTrie;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
@@ -43,11 +55,15 @@ import org.apache.solr.handler.SchemaHandler;
 import org.apache.solr.handler.SolrConfigHandler;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrQueryRequestBase;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.security.PermissionNameProvider;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.solr.api.ApiBag.EMPTY_SPEC;
 import static org.apache.solr.client.solrj.SolrRequest.METHOD.GET;
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
 import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
 import static org.apache.solr.common.params.CommonParams.CONFIGSETS_HANDLER_PATH;
 import static org.apache.solr.common.params.CommonParams.CORES_HANDLER_PATH;
@@ -152,6 +168,124 @@ public class TestApiFramework extends SolrTestCaseJ4 {
 
   }
 
+  public void testPayload() throws IOException {
+    String json = "{package:pkg1, version: '0.1', files  :[a.jar, b.jar]}";
+    Utils.fromJSONString(json);
+
+    ApiBag apiBag = new ApiBag(false);
+    AnnotatedApi api = new AnnotatedApi(new ApiTest());
+    apiBag.register(api, Collections.emptyMap());
+
+    ValidatingJsonMap spec = api.getSpec();
+
+    assertEquals("POST", spec._getStr("/methods[0]",null) );
+    assertEquals("POST", spec._getStr("/methods[0]",null) );
+    assertEquals("/cluster/package", spec._getStr("/url/paths[0]",null) );
+    assertEquals("string", spec._getStr("/commands/add/properties/package/type",null) );
+    assertEquals("array", spec._getStr("/commands/add/properties/files/type",null) );
+    assertEquals("string", spec._getStr("/commands/add/properties/files/items/type",null) );
+    assertEquals("string", spec._getStr("/commands/delete/items/type",null) );
+    SolrQueryResponse rsp = v2ApiInvoke(apiBag, "/cluster/package", "POST", new ModifiableSolrParams(),
+        new ByteArrayInputStream("{add:{package:mypkg, version: '1.0', files : [a.jar, b.jar]}}".getBytes(UTF_8)));
+
+
+    AddVersion addversion = (AddVersion) rsp.getValues().get("add");
+    assertEquals("mypkg", addversion.pkg);
+    assertEquals("1.0", addversion.version);
+    assertEquals("a.jar", addversion.files.get(0));
+    assertEquals("b.jar", addversion.files.get(1));
+
+
+
+  }
+
+  @EndPoint(method = POST, path = "/cluster/package", permission = PermissionNameProvider.Name.ALL)
+  public static class ApiTest {
+    @Command(name = "add")
+    public void add(SolrQueryRequest req, SolrQueryResponse rsp, AddVersion addVersion) {
+      rsp.add("add", addVersion);
+
+    }
+
+    @Command(name = "delete")
+    public void del(SolrQueryRequest req, SolrQueryResponse rsp, List<String> names) {
+      rsp.add("delete",names);
+
+    }
+
+
+
+  }
+
+  public static class AddVersion {
+    @JsonProperty(value = "package", required = true)
+    public String pkg;
+    @JsonProperty(value = "version", required = true)
+    public String version;
+    @JsonProperty(value = "files", required = true)
+    public List<String> files;
+  }
+
+  public void testAnnotatedApi() {
+    ApiBag apiBag = new ApiBag(false);
+    apiBag.register(new AnnotatedApi(new DummyTest()), Collections.emptyMap());
+    SolrQueryResponse rsp = v2ApiInvoke(apiBag, "/node/filestore/package/mypkg/jar1.jar", "GET",
+        new ModifiableSolrParams(), null);
+    assertEquals("/package/mypkg/jar1.jar", rsp.getValues().get("path"));
+  }
+
+  @EndPoint(
+      path = "/node/filestore/*",
+      method = SolrRequest.METHOD.GET,
+      permission = PermissionNameProvider.Name.ALL)
+  public class DummyTest {
+    @Command
+    public void read(SolrQueryRequest req, SolrQueryResponse rsp) {
+      rsp.add("FSRead.called", "true");
+      rsp.add("path", req.getPathTemplateValues().get("*"));
+    }
+  }
+
+  private static SolrQueryResponse v2ApiInvoke(ApiBag bag, String uri, String method, SolrParams params, InputStream payload) {
+    if (params == null) params = new ModifiableSolrParams();
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    HashMap<String, String> templateVals = new HashMap<>();
+    Api[] currentApi = new Api[1];
+
+    SolrQueryRequestBase req = new SolrQueryRequestBase(null, params) {
+
+      @Override
+      public Map<String, String> getPathTemplateValues() {
+        return templateVals;
+      }
+
+      @Override
+      protected Map<String, JsonSchemaValidator> getValidators() {
+        return currentApi[0] == null?
+            Collections.emptyMap():
+            currentApi[0].getCommandSchema();
+      }
+
+      @Override
+      public Iterable<ContentStream> getContentStreams() {
+        return Collections.singletonList(new ContentStreamBase() {
+          @Override
+          public InputStream getStream() throws IOException {
+            return payload;
+          }
+        });
+
+      }
+    };
+    Api api = bag.lookup(uri, method, templateVals);
+    currentApi[0] = api;
+
+
+    api.call(req, rsp);
+    return rsp;
+
+  }
+
   public void testTrailingTemplatePaths() {
     PathTrie<Api> registry = new PathTrie<>();
     Api api = new Api(EMPTY_SPEC) {
@@ -204,7 +338,7 @@ public class TestApiFramework extends SolrTestCaseJ4 {
   }
 
 
-  private void assertConditions(Map root, Map conditions) {
+  public static void assertConditions(Map root, Map conditions) {
     for (Object o : conditions.entrySet()) {
       Map.Entry e = (Map.Entry) o;
       String path = (String) e.getKey();
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/PathTrie.java b/solr/solrj/src/java/org/apache/solr/common/util/PathTrie.java
index 1d64834..742c59d 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/PathTrie.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/PathTrie.java
@@ -26,24 +26,25 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import static java.util.Collections.emptyList;
 
-/**A utility class to efficiently parse/store/lookup hierarchical paths which are templatized
+/**
+ * A utility class to efficiently parse/store/lookup hierarchical paths which are templatized
  * like /collections/{collection}/shards/{shard}/{replica}
  */
 public class PathTrie<T> {
   private final Set<String> reserved = new HashSet<>();
   Node root = new Node(emptyList(), null);
 
-  public PathTrie() { }
+  public PathTrie() {
+  }
 
   public PathTrie(Set<String> reserved) {
     this.reserved.addAll(reserved);
   }
 
 
-
   public void insert(String path, Map<String, String> replacements, T o) {
     List<String> parts = getPathSegments(path);
-    insert(parts,replacements, o);
+    insert(parts, replacements, o);
   }
 
   public void insert(List<String> parts, Map<String, String> replacements, T o) {
@@ -122,6 +123,9 @@ public class PathTrie<T> {
     private synchronized void insert(List<String> path, T o) {
       String part = path.get(0);
       Node matchedChild = null;
+      if ("*".equals(name)) {
+        return;
+      }
       if (children == null) children = new ConcurrentHashMap<>();
 
       String varName = templateName(part);
@@ -169,9 +173,8 @@ public class PathTrie<T> {
     }
 
     /**
-     *
-     * @param pathSegments pieces in the url /a/b/c has pieces as 'a' , 'b' , 'c'
-     * @param index current index of the pieces that we are looking at in /a/b/c 0='a' and 1='b'
+     * @param pathSegments      pieces in the url /a/b/c has pieces as 'a' , 'b' , 'c'
+     * @param index             current index of the pieces that we are looking at in /a/b/c 0='a' and 1='b'
      * @param templateVariables The mapping of template variable to its value
      * @param availableSubPaths If not null , available sub paths will be returned in this set
      */
@@ -179,13 +182,36 @@ public class PathTrie<T> {
       if (templateName != null) templateVariables.put(templateName, pathSegments.get(index - 1));
       if (pathSegments.size() < index + 1) {
         findAvailableChildren("", availableSubPaths);
+        if (obj == null) {//this is not a leaf node
+          Node n = children.get("*");
+          if (n != null) {
+            return n.obj;
+          }
+
+        }
         return obj;
       }
       String piece = pathSegments.get(index);
-      if (children == null) return null;
+      if (children == null) {
+        return null;
+      }
       Node n = children.get(piece);
       if (n == null && !reserved.contains(piece)) n = children.get("");
-      if (n == null) return null;
+      if (n == null) {
+        n = children.get("*");
+        if (n != null) {
+          StringBuffer sb = new StringBuffer();
+          for (int i = index; i < pathSegments.size(); i++) {
+            sb.append("/").append(pathSegments.get(i));
+          }
+          templateVariables.put("*", sb.toString());
+          return n.obj;
+
+        }
+      }
+      if (n == null) {
+        return null;
+      }
       return n.lookup(pathSegments, index + 1, templateVariables, availableSubPaths);
     }
   }
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/ValidatingJsonMap.java b/solr/solrj/src/java/org/apache/solr/common/util/ValidatingJsonMap.java
index 28c0019..b537536 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/ValidatingJsonMap.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/ValidatingJsonMap.java
@@ -31,6 +31,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.solr.common.NavigableObject;
 import org.apache.solr.common.SolrException;
 import org.noggit.JSONParser;
 import org.noggit.ObjectBuilder;
@@ -39,7 +40,7 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.Collections.unmodifiableList;
 import static java.util.Collections.unmodifiableSet;
 
-public class ValidatingJsonMap implements Map<String, Object> {
+public class ValidatingJsonMap implements Map<String, Object>, NavigableObject {
 
   private static final String INCLUDE = "#include";
   private static final String RESOURCE_EXTENSION = ".json";
diff --git a/solr/solrj/src/test/org/apache/solr/common/util/TestPathTrie.java b/solr/solrj/src/test/org/apache/solr/common/util/TestPathTrie.java
index 147535a..52a661f 100644
--- a/solr/solrj/src/test/org/apache/solr/common/util/TestPathTrie.java
+++ b/solr/solrj/src/test/org/apache/solr/common/util/TestPathTrie.java
@@ -55,6 +55,19 @@ public class TestPathTrie extends SolrTestCaseJ4 {
     pathTrie.lookup("/aa",templateValues, subPaths);
     assertEquals(3, subPaths.size());
 
+    pathTrie = new PathTrie<>(ImmutableSet.of("_introspect"));
+    pathTrie.insert("/aa/bb/{cc}/tt/*", emptyMap(), "W");
 
+    templateValues.clear();
+    assertEquals("W" ,pathTrie.lookup("/aa/bb/somepart/tt/hello", templateValues));
+    assertEquals(templateValues.get("*"), "/hello");
+
+    templateValues.clear();
+    assertEquals("W" ,pathTrie.lookup("/aa/bb/somepart/tt", templateValues));
+    assertEquals(templateValues.get("*"), null);
+
+    templateValues.clear();
+    assertEquals("W" ,pathTrie.lookup("/aa/bb/somepart/tt/hello/world/from/solr", templateValues));
+    assertEquals(templateValues.get("*"), "/hello/world/from/solr");
   }
 }


[lucene-solr] 02/02: String#isBlank() is java 11 API

Posted by no...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 888fe76a09e1b00ee99f024ad2dbe1e55e65fe42
Author: noble <no...@apache.org>
AuthorDate: Mon Oct 7 09:47:23 2019 +1100

    String#isBlank() is java 11 API
---
 solr/core/src/java/org/apache/solr/api/AnnotatedApi.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java b/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
index b1be461..cda2d64 100644
--- a/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
+++ b/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
@@ -108,7 +108,7 @@ public class AnnotatedApi extends Api implements PermissionNameProvider {
 
       for (Method method : klas.getMethods()) {
         Command command = method.getAnnotation(Command.class);
-        if (command != null && !command.name().isBlank()) {
+        if (command != null && !command.name().isEmpty()) {
           cmds.put(command.name(), AnnotatedApi.createSchema(method));
         }
       }