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/11/02 04:06:40 UTC

[lucene-solr] branch jira/solr13841 created (now 8f07cb2)

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

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


      at 8f07cb2  wrong import

This branch includes the following new commits:

     new 240a285  Removed jackson annottaions dependency from SolrJ. Provided a mapping from our annotation to jackson annotation
     new 8f07cb2  wrong import

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.



[lucene-solr] 01/02: Removed jackson annottaions dependency from SolrJ. Provided a mapping from our annotation to jackson annotation

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

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

commit 240a2859296f70dff44b7b67eaff779106050071
Author: noble <no...@apache.org>
AuthorDate: Sat Nov 2 15:05:01 2019 +1100

    Removed jackson annottaions dependency from SolrJ. Provided a mapping from our annotation to jackson annotation
---
 .../src/java/org/apache/solr/api/AnnotatedApi.java | 68 ++++------------
 .../src/java/org/apache/solr/pkg/PackageAPI.java   | 15 ++--
 .../solr/util/SolrPropertyAnnotationInspector.java | 83 ++++++++++++++++++++
 .../solr/util/TestSolrJacksonAnnotation.java       | 90 ++++++++++++++++++++++
 solr/solrj/ivy.xml                                 |  2 -
 .../solr/client/solrj/request/beans/Package.java   | 12 +--
 .../annotation/Property.java}                      | 34 +++-----
 .../apache/solr/common/util/JsonSchemaCreator.java | 81 +++++++++++++++++++
 .../apache/solr/common/util/ReflectMapWriter.java  |  5 +-
 9 files changed, 298 insertions(+), 92 deletions(-)

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 a6ae1be..4ec0012 100644
--- a/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
+++ b/solr/core/src/java/org/apache/solr/api/AnnotatedApi.java
@@ -19,7 +19,6 @@ package org.apache.solr.api;
 
 
 import java.lang.invoke.MethodHandles;
-import java.lang.reflect.Field;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
@@ -33,18 +32,19 @@ 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.JsonSchemaCreator;
 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;
+import org.apache.solr.util.SolrPropertyAnnotationInspector;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -61,7 +61,7 @@ import org.slf4j.LoggerFactory;
 public class AnnotatedApi extends Api implements PermissionNameProvider {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public static final String ERR ="Error executing commands :";
+  public static final String ERR = "Error executing commands :";
   private EndPoint endPoint;
   private Map<String, Cmd> commands = new HashMap<>();
   private final Api fallback;
@@ -161,8 +161,8 @@ public class AnnotatedApi extends Api implements PermissionNameProvider {
 
     List<Map> errs = CommandOperation.captureErrors(cmds);
     if (!errs.isEmpty()) {
-      log.error(ERR+ Utils.toJSONString(errs));
-      throw new ApiBag.ExceptionWithErrObject(SolrException.ErrorCode.BAD_REQUEST, ERR , errs);
+      log.error(ERR + Utils.toJSONString(errs));
+      throw new ApiBag.ExceptionWithErrObject(SolrException.ErrorCode.BAD_REQUEST, ERR, errs);
     }
 
   }
@@ -178,6 +178,7 @@ public class AnnotatedApi extends Api implements PermissionNameProvider {
 
 
     Cmd(Command command, Object obj, Method method) {
+      mapper.setAnnotationIntrospector(SolrPropertyAnnotationInspector.INSTANCE);
       if (Modifier.isPublic(method.getModifiers())) {
         this.command = command;
         this.obj = obj;
@@ -253,61 +254,20 @@ public class AnnotatedApi extends Api implements PermissionNameProvider {
     }
   }
 
-  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();
     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 (t instanceof ParameterizedType) {
-      ParameterizedType typ = (ParameterizedType) t;
-      if (typ.getRawType() == PayloadObj.class) {
-        t = typ.getActualTypeArguments()[0];
+      Type t = types[2];
+      if (t instanceof ParameterizedType) {
+        ParameterizedType typ = (ParameterizedType) t;
+        if (typ.getRawType() == PayloadObj.class) {
+          t = typ.getActualTypeArguments()[0];
+        }
       }
-    }
+      return JsonSchemaCreator.getSchema(t);
 
-    if (primitives.containsKey(t)) {
-      map.put("type", primitives.get(t));
-    } 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()));
     }
+    return null;
   }
 
-
 }
diff --git a/solr/core/src/java/org/apache/solr/pkg/PackageAPI.java b/solr/core/src/java/org/apache/solr/pkg/PackageAPI.java
index f202503..be16f35 100644
--- a/solr/core/src/java/org/apache/solr/pkg/PackageAPI.java
+++ b/solr/core/src/java/org/apache/solr/pkg/PackageAPI.java
@@ -34,6 +34,7 @@ import org.apache.solr.api.PayloadObj;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.request.beans.Package;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.annotation.Property;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.cloud.ZooKeeperException;
@@ -44,6 +45,7 @@ import org.apache.solr.core.CoreContainer;
 import org.apache.solr.filestore.PackageStoreAPI;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.util.SolrPropertyAnnotationInspector;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
@@ -61,7 +63,7 @@ public class PackageAPI {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   final CoreContainer coreContainer;
-  private ObjectMapper mapper = new ObjectMapper();
+  private final ObjectMapper mapper = new ObjectMapper();
   private final PackageLoader packageLoader;
   Packages pkgs;
 
@@ -71,6 +73,7 @@ public class PackageAPI {
   public PackageAPI(CoreContainer coreContainer, PackageLoader loader) {
     this.coreContainer = coreContainer;
     this.packageLoader = loader;
+    mapper.setAnnotationIntrospector(SolrPropertyAnnotationInspector.INSTANCE);
     pkgs = new Packages();
     SolrZkClient zkClient = coreContainer.getZkController().getZkClient();
     try {
@@ -145,10 +148,10 @@ public class PackageAPI {
 
 
   public static class Packages implements ReflectMapWriter {
-    @JsonProperty
+    @Property
     public int znodeVersion = -1;
 
-    @JsonProperty
+    @Property
     public Map<String, List<PkgVersion>> packages = new LinkedHashMap<>();
 
 
@@ -164,10 +167,10 @@ public class PackageAPI {
 
   public static class PkgVersion implements ReflectMapWriter {
 
-    @JsonProperty
+    @Property
     public String version;
 
-    @JsonProperty
+    @Property
     public List<String> files;
 
     public PkgVersion() {
@@ -243,7 +246,7 @@ public class PackageAPI {
             packages = new Packages();
           }
           packages.packages.computeIfAbsent(add.pkg, Utils.NEW_ARRAYLIST_FUN).add(new PkgVersion(add));
-          packages.znodeVersion = stat.getVersion() + 1;
+          packages.znodeVersion = stat.getVersion() ;
           finalState[0] = packages;
           return Utils.toJSON(packages);
         });
diff --git a/solr/core/src/java/org/apache/solr/util/SolrPropertyAnnotationInspector.java b/solr/core/src/java/org/apache/solr/util/SolrPropertyAnnotationInspector.java
new file mode 100644
index 0000000..59b186f
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/util/SolrPropertyAnnotationInspector.java
@@ -0,0 +1,83 @@
+/*
+ * 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.util;
+
+import com.fasterxml.jackson.core.Version;
+import com.fasterxml.jackson.databind.AnnotationIntrospector;
+import com.fasterxml.jackson.databind.PropertyName;
+import com.fasterxml.jackson.databind.introspect.Annotated;
+import com.fasterxml.jackson.databind.introspect.AnnotatedField;
+import com.fasterxml.jackson.databind.introspect.AnnotatedMember;
+import com.fasterxml.jackson.databind.introspect.AnnotatedMethod;
+import com.fasterxml.jackson.databind.util.BeanUtil;
+import org.apache.solr.common.annotation.Property;
+import org.apache.solr.search.SolrCache;
+//this class provides a maping between jackson's JsonProperty Annotation to Solr's Property annotation
+// see SOLR-13841 for more details
+public class SolrPropertyAnnotationInspector extends AnnotationIntrospector {
+  public static final SolrPropertyAnnotationInspector INSTANCE = new SolrPropertyAnnotationInspector();
+
+  @Override
+  public Version version() {
+    return Version.unknownVersion();
+  }
+
+  @Override
+  public PropertyName findNameForSerialization(Annotated a) {
+    if (a instanceof AnnotatedMethod) {
+      AnnotatedMethod am = (AnnotatedMethod) a;
+      Property prop = am.getAnnotation(Property.class);
+      if (prop == null) return null;
+      if (prop.value().isEmpty()) {
+        return new PropertyName(BeanUtil.okNameForGetter(am, true));
+      } else {
+        return new PropertyName(prop.value());
+      }
+
+    }
+    if (a instanceof AnnotatedField) {
+      AnnotatedField af = (AnnotatedField) a;
+      Property prop = af.getAnnotation(Property.class);
+      if (prop == null) return null;
+      return prop.value().isEmpty() ?
+          new PropertyName(af.getName()) :
+          new PropertyName(prop.value());
+    }
+    return null;
+  }
+
+  @Override
+  public Boolean hasRequiredMarker(AnnotatedMember m) {
+    Property prop = m.getAnnotation(Property.class);
+    if (prop == null) return Boolean.FALSE;
+    return prop.required();
+  }
+
+  @Override
+  public String findPropertyDefaultValue(Annotated m) {
+    Property prop = m.getAnnotation(Property.class);
+    if (prop == null) return "";
+    return prop.defaultValue();
+  }
+
+  @Override
+  public PropertyName findNameForDeserialization(Annotated a) {
+    return findNameForSerialization(a);
+  }
+
+}
diff --git a/solr/core/src/test/org/apache/solr/util/TestSolrJacksonAnnotation.java b/solr/core/src/test/org/apache/solr/util/TestSolrJacksonAnnotation.java
new file mode 100644
index 0000000..9693b0d
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/util/TestSolrJacksonAnnotation.java
@@ -0,0 +1,90 @@
+/*
+ * 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.util;
+
+import java.util.List;
+import java.util.Map;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.module.jsonSchema.JsonSchema;
+import com.fasterxml.jackson.module.jsonSchema.factories.SchemaFactoryWrapper;
+import org.apache.solr.SolrTestCase;
+import org.apache.solr.common.annotation.Property;
+import org.apache.solr.common.util.JsonSchemaCreator;
+import org.apache.solr.common.util.JsonSchemaValidator;
+import org.apache.solr.common.util.Utils;
+
+public class TestSolrJacksonAnnotation extends SolrTestCase {
+
+  public void testSerDe() throws Exception {
+    ObjectMapper mapper = new ObjectMapper();
+    mapper.setAnnotationIntrospector(new SolrPropertyAnnotationInspector());
+
+    TestObj o = new TestObj();
+    o.field = "v1";
+    o.f2 = "v2";
+    o.ifld = 1234;
+    String json = mapper.writeValueAsString(o);
+
+    Map m = (Map) Utils.fromJSONString(json);
+    assertEquals("v1",  m.get("field"));
+    assertEquals("v2",  m.get("friendlyName"));
+    assertEquals("1234",  String.valueOf(m.get("friendlyIntFld")));
+    TestObj o1 = mapper.readValue(json, TestObj.class);
+
+    assertEquals("v1", o1.field);
+    assertEquals("v2", o1.f2);
+    assertEquals(1234, o1.ifld);
+
+    Map<String, Object> schema = JsonSchemaCreator.getSchema(TestObj.class);
+    assertEquals("string", Utils.getObjectByPath(schema,true,"/properties/friendlyName/type"));
+    assertEquals("integer", Utils.getObjectByPath(schema,true,"/properties/friendlyIntFld/type"));
+    assertEquals("friendlyName", Utils.getObjectByPath(schema,true,"/required[0]"));
+
+
+    JsonSchemaValidator validator = new JsonSchemaValidator(schema);
+    List<String> errs = validator.validateJson(m);
+    assertNull(errs);
+    m.remove("friendlyName");
+    errs = validator.validateJson(m);
+    assertFalse(errs.isEmpty());
+    assertTrue(errs.get(0).contains("Missing required attribute"));
+    m.put("friendlyIntFld", Boolean.TRUE);
+    errs = validator.validateJson(m);
+    m.put("friendlyIntFld", "another String");
+    assertTrue(errs.get(0).contains("Value is not valid"));
+  }
+
+  public String getSchema(ObjectMapper mapper, Class c)  throws Exception{
+    SchemaFactoryWrapper visitor = new SchemaFactoryWrapper();
+    mapper.acceptJsonFormatVisitor(mapper.constructType(c), visitor);
+    JsonSchema jsonSchema = visitor.finalSchema();
+    return mapper.writerWithDefaultPrettyPrinter().writeValueAsString(jsonSchema);
+  }
+
+
+
+  public static class TestObj {
+    @Property()
+    public String field;
+    @Property(value = "friendlyName" ,required = true)
+    public String f2;
+    @Property("friendlyIntFld")
+    public int ifld;
+  }
+}
diff --git a/solr/solrj/ivy.xml b/solr/solrj/ivy.xml
index b1871fe..5312c42 100644
--- a/solr/solrj/ivy.xml
+++ b/solr/solrj/ivy.xml
@@ -61,8 +61,6 @@
     <dependency org="io.netty" name="netty-transport-native-epoll" rev="${/io.netty/netty-transport-native-epoll}" conf="compile"/>
     <dependency org="io.netty" name="netty-transport-native-unix-common" rev="${/io.netty/netty-transport-native-unix-common}" conf="compile"/>
 
-    <dependency org="com.fasterxml.jackson.core" name="jackson-annotations" rev="${/com.fasterxml.jackson.core/jackson-annotations}" conf="compile"/>
-
     <dependency org="org.apache.logging.log4j" name="log4j-slf4j-impl" rev="${/org.apache.logging.log4j/log4j-slf4j-impl}" conf="test"/>
 
     <dependency org="org.mockito" name="mockito-core" rev="${/org.mockito/mockito-core}" conf="test"/>
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/Package.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/Package.java
index 62bf6ec..f725a61 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/Package.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/Package.java
@@ -19,7 +19,7 @@ package org.apache.solr.client.solrj.request.beans;
 
 import java.util.List;
 
-import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.solr.common.annotation.Property;
 import org.apache.solr.common.util.ReflectMapWriter;
 
 /**Just a container class for POJOs used in Package APIs
@@ -27,19 +27,19 @@ import org.apache.solr.common.util.ReflectMapWriter;
  */
 public class Package {
   public static class AddVersion implements ReflectMapWriter {
-    @JsonProperty(value = "package", required = true)
+    @Property(value = "package", required = true)
     public String pkg;
-    @JsonProperty(required = true)
+    @Property(required = true)
     public String version;
-    @JsonProperty(required = true)
+    @Property(required = true)
     public List<String> files;
 
   }
 
   public static class DelVersion implements ReflectMapWriter {
-    @JsonProperty(value = "package", required = true)
+    @Property(value = "package", required = true)
     public String pkg;
-    @JsonProperty(required = true)
+    @Property(required = true)
     public String version;
 
   }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/Package.java b/solr/solrj/src/java/org/apache/solr/common/annotation/Property.java
similarity index 52%
copy from solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/Package.java
copy to solr/solrj/src/java/org/apache/solr/common/annotation/Property.java
index 62bf6ec..46db8a6 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/beans/Package.java
+++ b/solr/solrj/src/java/org/apache/solr/common/annotation/Property.java
@@ -15,32 +15,22 @@
  * limitations under the License.
  */
 
-package org.apache.solr.client.solrj.request.beans;
+package org.apache.solr.common.annotation;
 
-import java.util.List;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
 
-import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.solr.common.util.ReflectMapWriter;
+@Target({ElementType.FIELD, ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+public @interface Property {
 
-/**Just a container class for POJOs used in Package APIs
- *
- */
-public class Package {
-  public static class AddVersion implements ReflectMapWriter {
-    @JsonProperty(value = "package", required = true)
-    public String pkg;
-    @JsonProperty(required = true)
-    public String version;
-    @JsonProperty(required = true)
-    public List<String> files;
+  String value() default "";
 
-  }
+  boolean required() default false;
 
-  public static class DelVersion implements ReflectMapWriter {
-    @JsonProperty(value = "package", required = true)
-    public String pkg;
-    @JsonProperty(required = true)
-    public String version;
+//  int index() default -1;
 
-  }
+  String defaultValue() default "";
 }
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaCreator.java b/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaCreator.java
new file mode 100644
index 0000000..ab09ec9
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaCreator.java
@@ -0,0 +1,81 @@
+/*
+ * 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.util;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.solr.common.annotation.Property;
+
+public class JsonSchemaCreator {
+  public static final Map<Class, String> natives = new HashMap<>();
+
+  static {
+    natives.put(String.class, "string");
+    natives.put(Integer.class, "integer");
+    natives.put(int.class, "integer");
+    natives.put(Float.class, "number");
+    natives.put(float.class, "number");
+    natives.put(Double.class, "number");
+    natives.put(double.class, "number");
+    natives.put(Boolean.class, "boolean");
+    natives.put(List.class, "array");
+  }
+
+  public static Map<String, Object> getSchema(java.lang.reflect.Type t) {
+    return createSchemaFromType(t, new LinkedHashMap<>());
+  }
+
+  private static Map<String, Object> createSchemaFromType(java.lang.reflect.Type t, Map<String, Object> map) {
+    if (natives.containsKey(t)) {
+      map.put("type", natives.get(t));
+    } else if (t instanceof ParameterizedType && ((ParameterizedType) t).getRawType() == List.class) {
+      Type typ = ((ParameterizedType) t).getActualTypeArguments()[0];
+      map.put("type", "array");
+      map.put("items", getSchema(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);
+    Set<String>  required = new HashSet<>();
+    for (Field fld : klas.getDeclaredFields()) {
+      Property p = fld.getAnnotation(Property.class);
+      if (p == null) continue;
+      String name = p.value().isEmpty() ? fld.getName() : p.value();
+      props.put(name, getSchema(fld.getGenericType()));
+      if(p.required()) required.add(name);
+    }
+    if(!required.isEmpty()) map.put("required", new ArrayList<>(required));
+
+  }
+}
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/ReflectMapWriter.java b/solr/solrj/src/java/org/apache/solr/common/util/ReflectMapWriter.java
index 3a78891..0eac7bb 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/ReflectMapWriter.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/ReflectMapWriter.java
@@ -21,15 +21,16 @@ import java.io.IOException;
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
 
-import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.annotation.Property;
+
 // An implementation of MapWriter which is annotated with Jackson annotations
 public interface ReflectMapWriter extends MapWriter {
 
   @Override
   default void writeMap(EntryWriter ew) throws IOException {
     for (Field field : this.getClass().getDeclaredFields()) {
-      JsonProperty prop = field.getAnnotation(JsonProperty.class);
+      Property prop = field.getAnnotation(Property.class);
       if (prop == null) continue;
       int modifiers = field.getModifiers();
       if (Modifier.isPublic(modifiers) && !Modifier.isStatic(modifiers)) {


[lucene-solr] 02/02: wrong import

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

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

commit 8f07cb2e08530a06304ac713e2c370c376417a4a
Author: noble <no...@apache.org>
AuthorDate: Sat Nov 2 15:05:59 2019 +1100

    wrong import
---
 solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaCreator.java | 1 -
 1 file changed, 1 deletion(-)

diff --git a/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaCreator.java b/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaCreator.java
index ab09ec9..fc932df 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaCreator.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaCreator.java
@@ -28,7 +28,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.solr.common.annotation.Property;
 
 public class JsonSchemaCreator {