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 2017/05/18 04:25:09 UTC

[1/2] lucene-solr:master: SOLR-10413: v2 API: parsed JSON type should be coerced to expected type

Repository: lucene-solr
Updated Branches:
  refs/heads/master 523590310 -> 896270d13


SOLR-10413: v2 API: parsed JSON type should be coerced to expected type


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

Branch: refs/heads/master
Commit: c93409392d780892542ba736739099970a26632f
Parents: 9dc952a
Author: Cao Manh Dat <da...@apache.org>
Authored: Thu May 18 11:24:50 2017 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Thu May 18 11:24:50 2017 +0700

----------------------------------------------------------------------
 .../apache/solr/util/JsonSchemaValidator.java   | 479 ++++++++-----------
 .../src/resources/apispec/cluster.Commands.json |   2 +-
 .../solr/handler/V2ApiIntegrationTest.java      |  11 +
 .../org/apache/solr/util/JsonValidatorTest.java |  27 +-
 4 files changed, 222 insertions(+), 297 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9340939/solr/core/src/java/org/apache/solr/util/JsonSchemaValidator.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/util/JsonSchemaValidator.java b/solr/core/src/java/org/apache/solr/util/JsonSchemaValidator.java
index 1074ed8..8a0a09f 100644
--- a/solr/core/src/java/org/apache/solr/util/JsonSchemaValidator.java
+++ b/solr/core/src/java/org/apache/solr/util/JsonSchemaValidator.java
@@ -17,354 +17,261 @@
 
 package org.apache.solr.util;
 
-import java.util.Collections;
+import java.util.Arrays;
+import java.util.HashMap;
 import java.util.HashSet;
-import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
 
-import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.common.util.Pair;
 import org.apache.solr.common.util.Utils;
 
-import static java.util.Arrays.asList;
-import static java.util.Collections.unmodifiableMap;
-import static java.util.function.Function.identity;
-import static java.util.stream.Collectors.toMap;
-
-/**A very basic and lightweight json schema parsing and data validation tool. This custom tool is created
+/**
+ * A very basic and lightweight json schema parsing and data validation tool. This custom tool is created
  * because a) we need to support non json inputs b) to avoiding double parsing (this accepts an already parsed json as a map)
  * It validates most aspects of json schema but it is NOT A FULLY COMPLIANT JSON schema parser or validator.
- * What is supported ?
- * a) all types and their validation (string, boolean, array, enum,object, integer, number)
- * b) 'required' properties, 'additionalProperties'
- *
- *
+ * This validator borrow some design's idea from https://github.com/networknt/json-schema-validator
  */
-
 public class JsonSchemaValidator {
-  private final SchemaNode root;
+
+  private List<Validator> validators;
+  private static Set<String> KNOWN_FNAMES = new HashSet<>(Arrays.asList(
+      "description","documentation","default","additionalProperties"));
+
 
   public JsonSchemaValidator(String jsonString) {
     this((Map) Utils.fromJSONString(jsonString));
   }
-  public JsonSchemaValidator(Map jsonSchema) {
-    root = new SchemaNode(null);
-    root.isRequired = true;
-    List<String> errs = new LinkedList<>();
-    root.validateSchema(jsonSchema, errs);
-    if(!errs.isEmpty()){
-      throw new RuntimeException("Invalid schema. "+ StrUtils.join(errs,'|'));
-    }
-  }
 
-  private static class SchemaNode {
-    final SchemaNode parent;
-    Type type;
-    Type elementType;
-    boolean isRequired = false;
-    Object validationInfo;
-    Boolean additionalProperties;
-    Map<String, SchemaNode> children;
-
-    private SchemaNode(SchemaNode parent) {
-      this.parent = parent;
-    }
-
-    private void validateSchema(Map jsonSchema, List<String> errs) {
-      Object typeStr = jsonSchema.get("type");
-      if (typeStr == null) {
-        errs.add("'type' is missing ");
-      }
-      Type type = Type.get(typeStr);
-      if (type == null) {
-        errs.add ("Unknown type " + typeStr + " in object "+ Utils.toJSONString(jsonSchema));
-        return;
-      }
-      this.type = type;
+  public JsonSchemaValidator(Map jsonSchema) {
+    this.validators = new LinkedList<>();
+    for (Object fname : jsonSchema.keySet()) {
+      if (KNOWN_FNAMES.contains(fname.toString())) continue;
 
-      for (SchemaAttribute schemaAttribute : SchemaAttribute.values()) {
-        schemaAttribute.validateSchema(jsonSchema, this, errs);
-      }
-      jsonSchema.keySet().forEach(o -> {
-        if (!knownAttributes.containsKey(o)) errs.add("Unknown key : " + o);
-      });
-      if (!errs.isEmpty()) return;
-
-      if (type == Type.OBJECT) {
-        Map m = (Map) jsonSchema.get("properties");
-        if (m != null) {
-          for (Object o : m.entrySet()) {
-            Map.Entry e = (Map.Entry) o;
-            if (e.getValue() instanceof Map) {
-              Map od = (Map) e.getValue();
-              if (children == null) children = new LinkedHashMap<>();
-              SchemaNode child = new SchemaNode(this);
-              children.put((String) e.getKey(), child);
-              child.validateSchema(od, errs);
-            } else {
-              errs.add("Invalid Object definition for field " + e.getKey());
-            }
-          }
-        } else {
-          additionalProperties = Boolean.TRUE;
-        }
-      }
-      for (SchemaAttribute attr : SchemaAttribute.values()) {
-        attr.postValidateSchema(jsonSchema, this, errs);
-      }
+      Function<Pair<Map, Object>, Validator> initializeFunction = VALIDATORS.get(fname.toString());
+      if (initializeFunction == null) throw new RuntimeException("Unknown key : " + fname);
 
+      this.validators.add(initializeFunction.apply(new Pair<>(jsonSchema, jsonSchema.get(fname))));
     }
+  }
 
-    private void validate(String key, Object data, List<String> errs) {
-      if (data == null) {
-        if (isRequired) {
-          errs.add("Missing field '" + key+"'");
-          return;
-        }
-      } else {
-        type.validateData(key, data, this, errs);
-        if(!errs.isEmpty()) return;
-        if (children != null && type == Type.OBJECT) {
-          for (Map.Entry<String, SchemaNode> e : children.entrySet()) {
-            e.getValue().validate(e.getKey(), ((Map) data).get(e.getKey()), errs);
-          }
-          if (Boolean.TRUE != additionalProperties) {
-            for (Object o : ((Map) data).keySet()) {
-              if (!children.containsKey(o)) {
-                errs.add("Unknown field '" + o + "' in object : " + Utils.toJSONString(data));
-              }
-            }
-          }
-        }
-      }
-    }
+  static final Map<String, Function<Pair<Map,Object>, Validator>> VALIDATORS = new HashMap<>();
 
+  static {
+    VALIDATORS.put("items", pair -> new ItemsValidator(pair.first(), (Map) pair.second()));
+    VALIDATORS.put("enum", pair -> new EnumValidator(pair.first(), (List) pair.second()));
+    VALIDATORS.put("properties", pair -> new PropertiesValidator(pair.first(), (Map) pair.second()));
+    VALIDATORS.put("type", pair -> new TypeValidator(pair.first(), pair.second()));
+    VALIDATORS.put("required", pair -> new RequiredValidator(pair.first(), (List)pair.second()));
+    VALIDATORS.put("oneOf", pair -> new OneOfValidator(pair.first(), (List)pair.second()));
   }
 
   public List<String> validateJson(Object data) {
     List<String> errs = new LinkedList<>();
-    root.validate(null, data, errs);
+    validate(data, errs);
     return errs.isEmpty() ? null : errs;
   }
 
-  /**represents an attribute in the schema definition
-   *
-   */
-  enum SchemaAttribute {
-    type(true, Type.STRING),
-    properties(false, Type.OBJECT) {
-      @Override
-      public void validateSchema(Map attrSchema, SchemaNode schemaNode, List<String> errors) {
-        super.validateSchema(attrSchema, schemaNode, errors);
-        if (schemaNode.type != Type.OBJECT) return;
-        Object val = attrSchema.get(key);
-        if (val == null) {
-          Object additional = attrSchema.get(additionalProperties.key);
-          if (Boolean.TRUE.equals(additional)) schemaNode.additionalProperties =  Boolean.TRUE;
-        }
-      }
-    },
-    additionalProperties(false, Type.BOOLEAN),
-    items(false, Type.OBJECT) {
-      @Override
-      public void validateSchema(Map attrSchema, SchemaNode schemaNode, List<String> errors) {
-        super.validateSchema(attrSchema, schemaNode, errors);
-        Object itemsVal = attrSchema.get(key);
-        if (itemsVal != null) {
-          if (schemaNode.type != Type.ARRAY) {
-            errors.add("Only 'array' can have 'items'");
-            return;
-          } else {
-            if (itemsVal instanceof Map) {
-              Map val = (Map) itemsVal;
-              Object value = val.get(type.key);
-              Type t = Type.get(String.valueOf(value));
-              if (t == null) {
-                errors.add("Unknown array type " + Utils.toJSONString(attrSchema));
-              } else {
-                schemaNode.elementType = t;
-              }
-            }
-          }
-        }
-      }
-    },
-    __default(false,Type.UNKNOWN),
-    description(false, Type.STRING),
-    documentation(false, Type.STRING),
-    oneOf(false, Type.ARRAY),
-    __enum(false, Type.ARRAY) {
-      @Override
-      void validateSchema(Map attrSchema, SchemaNode schemaNode, List<String> errors) {
-        if (attrSchema.get(Type.ENUM._name) != null) {
-          schemaNode.elementType = schemaNode.type;
-          schemaNode.type = Type.ENUM;
-        }
+  boolean validate(Object data, List<String> errs){
+    for (Validator validator : validators) {
+      if(!validator.validate(data, errs)) {
+        return false;
       }
+    }
+    return true;
+  }
 
-      @Override
-      void postValidateSchema(Map attrSchema, SchemaNode schemaNode, List<String> errs) {
-        Object val = attrSchema.get(key);
-        if (val == null) return;
-        if (val instanceof List) {
-          List list = (List) val;
-          for (Object o : list) {
-            if (!schemaNode.elementType.validate(o)) {
-              errs.add("Invalid value : " + o + " Expected type : " + schemaNode.elementType._name);
-            }
-          }
-          if (!errs.isEmpty()) return;
-          schemaNode.validationInfo = new HashSet(list);
-        } else {
-          errs.add("'enum' should have a an array as value in Object " + Utils.toJSONString(attrSchema));
-        }
-      }
-    },
-    id(false, Type.STRING),
-    _ref(false, Type.STRING),
-    _schema(false, Type.STRING),
-    required(false, Type.ARRAY) {
-      @Override
-      public void postValidateSchema(Map attrSchema, SchemaNode attr, List<String> errors) {
-        Object val = attrSchema.get(key);
-        if (val instanceof List) {
-          List list = (List) val;
-          if (attr.children != null) {
-            for (Map.Entry<String, SchemaNode> e : attr.children.entrySet()) {
-              if (list.contains(e.getKey())) e.getValue().isRequired = true;
-            }
-          }
-        }
-      }
-    };
+}
 
-    final String key;
-    final boolean _required;
-    final Type typ;
+abstract class Validator<T> {
+  @SuppressWarnings("unused")
+  Validator(Map schema, T properties) {};
+  abstract boolean validate(Object o, List<String> errs);
+}
 
-    public String getKey() {
-      return key;
-    }
+enum Type {
+  STRING(String.class),
+  ARRAY(List.class),
+  NUMBER(Number.class),
+  INTEGER(Long.class),
+  BOOLEAN(Boolean.class),
+  ENUM(List.class),
+  OBJECT(Map.class),
+  NULL(null),
+  UNKNOWN(Object.class);
+
+  Class type;
+
+  Type(Class type) {
+    this.type = type;
+  }
 
-    void validateSchema(Map attrSchema, SchemaNode schemaNode, List<String> errors) {
-      Object val = attrSchema.get(key);
-      if (val == null) {
-        if (_required)
-          errors.add("Missing required attribute '" + key + "' in object " + Utils.toJSONString(attrSchema));
-      } else {
-        if (!typ.validate(val)) errors.add(key + " should be of type " + typ._name);
+  boolean isValid(Object o) {
+    if (type == null) return o == null;
+    return type.isInstance(o);
+  }
+}
+
+class TypeValidator extends Validator<Object> {
+  private Set<Type> types;
+
+  TypeValidator(Map schema, Object type) {
+    super(schema, type);
+    types = new HashSet<>(1);
+    if (type instanceof List) {
+      for (Object t : (List)type) {
+        types.add(getType(t.toString()));
       }
+    } else {
+      types.add(getType(type.toString()));
     }
+  }
 
-    void postValidateSchema(Map attrSchema, SchemaNode schemaNode, List<String> errs) {
+  private Type getType(String typeStr) {
+    try {
+      return Type.valueOf(typeStr.toUpperCase(Locale.ROOT));
+    } catch (IllegalArgumentException e) {
+      throw new IllegalArgumentException("Unknown type " + typeStr);
     }
+  }
 
-    SchemaAttribute(boolean required, Type type) {
-      this.key = name().replaceAll("__","").replace('_', '$');
-      this._required = required;
-      this.typ = type;
+  @Override
+  boolean validate(Object o, List<String> errs) {
+    for (Type type: types) {
+      if (type.isValid(o)) return true;
     }
+    errs.add("Value is not valid, expected one of: " + types + ", found: " + o.getClass().getSimpleName());
+    return false;
   }
+}
 
-  interface TypeValidator {
-    void validateData(String key, Object o, SchemaNode schemaNode, List<String> errs);
+class ItemsValidator extends Validator<Map> {
+  private JsonSchemaValidator validator;
+  ItemsValidator(Map schema, Map properties) {
+    super(schema, properties);
+    validator = new JsonSchemaValidator(properties);
   }
 
-  /**represents a type in json
-   *
-   */
-  enum Type {
-    STRING(o -> o instanceof String),
-    ARRAY(o -> o instanceof List, (key, o, schemaNode, errs) -> {
-      List l = o instanceof List ? (List) o : Collections.singletonList(o);
-      if (schemaNode.elementType != null) {
-        for (Object elem : l) {
-          if (!schemaNode.elementType.validate(elem)) {
-            errs.add("Expected elements of type : " + key + " but found : " + Utils.toJSONString(o));
-            break;
-          }
+  @Override
+  boolean validate(Object o, List<String> errs) {
+    if (o instanceof List) {
+      for (Object o2 : (List) o) {
+        if (!validator.validate(o2, errs)) {
+          errs.add("Items not valid");
+          return false;
         }
       }
-    }),
-    NUMBER(o -> o instanceof Number, (key, o, schemaNode, errs) -> {
-      if (o instanceof String) {
-        try {
-          Double.parseDouble((String) o);
-        } catch (NumberFormatException e) {
-          errs.add(e.getClass().getName() + " " + e.getMessage());
-        }
+      return true;
+    }
+    return false;
+  }
+}
 
-      }
+class EnumValidator extends Validator<List<String>> {
 
-    }),
-    INTEGER(o -> o instanceof Integer, (key, o, schemaNode, errs) -> {
-      if (o instanceof String) {
-        try {
-          Integer.parseInt((String) o);
-        } catch (NumberFormatException e) {
-          errs.add(e.getClass().getName() + " " + e.getMessage());
-        }
-      }
-    }),
-    BOOLEAN(o -> o instanceof Boolean, (key, o, schemaNode, errs) -> {
-      if (o instanceof String) {
-        try {
-          Boolean.parseBoolean((String) o);
-        } catch (Exception e) {
-          errs.add(e.getClass().getName() + " " + e.getMessage());
-        }
-      }
-    }),
-    ENUM(o -> o instanceof List, (key, o, schemaNode, errs) -> {
-      if (schemaNode.validationInfo instanceof HashSet) {
-        HashSet enumVals = (HashSet) schemaNode.validationInfo;
-        if (!enumVals.contains(o)) {
-          errs.add("value of enum " + key + " must be one of" + enumVals);
-        }
-      }
-    }),
-    OBJECT(o -> o instanceof Map),
-    UNKNOWN((o -> true));
-    final String _name;
+  private Set<String> enumVals;
 
-    final java.util.function.Predicate typeValidator;
-    private final TypeValidator validator;
+  EnumValidator(Map schema, List<String> properties) {
+    super(schema, properties);
+    enumVals = new HashSet<>(properties);
 
-    Type(java.util.function.Predicate validator) {
-      this(validator, null);
+  }
 
+  @Override
+  boolean validate(Object o, List<String> errs) {
+    if (o instanceof String) {
+      if(!enumVals.contains(o)) {
+        errs.add("Value of enum must be one of " + enumVals);
+        return false;
+      }
+      return true;
     }
+    return false;
+  }
+}
 
-    Type(java.util.function.Predicate validator, TypeValidator v) {
-      _name = this.name().toLowerCase(Locale.ROOT);
-      this.typeValidator = validator;
-      this.validator = v;
-    }
+class RequiredValidator extends Validator<List<String>> {
 
-    boolean validate(Object o) {
-      return typeValidator.test(o);
-    }
+  private Set<String> requiredProps;
 
-    void validateData(String key, Object o, SchemaNode attr, List<String> errs) {
-      if (validator != null) {
-        validator.validateData(key, o, attr, errs);
-        return;
+  RequiredValidator(Map schema, List<String> requiredProps) {
+    super(schema, requiredProps);
+    this.requiredProps = new HashSet<>(requiredProps);
+  }
+
+  @Override
+  boolean validate(Object o, List<String> errs) {
+    if (o instanceof Map) {
+      Set fnames = ((Map) o).keySet();
+      for (String requiredProp : requiredProps) {
+        if (!fnames.contains(requiredProp)) {
+          errs.add("Missing required attribute '" + requiredProp + "' in object " + Utils.toJSONString(o));
+          return false;
+        }
       }
-      if (!typeValidator.test(o))
-        errs.add("Expected type : " + _name + " but found : " + o + "in object : " + Utils.toJSONString(o));
+      return true;
+    }
+    return false;
+  }
+}
+
+class PropertiesValidator extends Validator<Map<String, Map>> {
+  private Map<String, JsonSchemaValidator> jsonSchemas;
+  private boolean additionalProperties;
+
+  PropertiesValidator(Map schema, Map<String, Map> properties) {
+    super(schema, properties);
+    jsonSchemas = new HashMap<>();
+    this.additionalProperties = (boolean) schema.getOrDefault("additionalProperties", false);
+    for (Map.Entry<String, Map> entry : properties.entrySet()) {
+      jsonSchemas.put(entry.getKey(), new JsonSchemaValidator(entry.getValue()));
     }
+  }
 
-    static Type get(Object type) {
-      for (Type t : Type.values()) {
-        if (t._name.equals(type)) return t;
+  @Override
+  boolean validate(Object o, List<String> errs) {
+    if (o instanceof Map) {
+      Map map = (Map) o;
+      for (Object key : map.keySet()) {
+        JsonSchemaValidator jsonSchema = jsonSchemas.get(key.toString());
+        if (jsonSchema == null && !additionalProperties) {
+          errs.add("Unknown field '" + key + "' in object : " + Utils.toJSONString(o));
+          return false;
+        }
+        if (jsonSchema != null && !jsonSchema.validate(map.get(key), errs)) {
+          return false;
+        }
       }
-      return null;
+      return true;
     }
+    return false;
   }
+}
+
+class OneOfValidator extends Validator<List<String>> {
 
+  private Set<String> oneOfProps;
 
-  static final Map<String, SchemaAttribute> knownAttributes = unmodifiableMap(asList(SchemaAttribute.values()).stream().collect(toMap(SchemaAttribute::getKey, identity())));
+  OneOfValidator(Map schema, List<String> oneOfProps) {
+    super(schema, oneOfProps);
+    this.oneOfProps = new HashSet<>(oneOfProps);
+  }
 
+  @Override
+  boolean validate(Object o, List<String> errs) {
+    if (o instanceof Map) {
+      Map map = (Map) o;
+      for (Object key : map.keySet()) {
+        if (oneOfProps.contains(key.toString())) return true;
+      }
+      errs.add("One of fields :"  + oneOfProps + " is not presented in object : " + Utils.toJSONString(o));
+      return false;
+    }
+
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9340939/solr/core/src/resources/apispec/cluster.Commands.json
----------------------------------------------------------------------
diff --git a/solr/core/src/resources/apispec/cluster.Commands.json b/solr/core/src/resources/apispec/cluster.Commands.json
index 8983964..88f8c06 100644
--- a/solr/core/src/resources/apispec/cluster.Commands.json
+++ b/solr/core/src/resources/apispec/cluster.Commands.json
@@ -61,7 +61,7 @@
           "description": "The name of the property"
         },
         "val": {
-          "type": "string",
+          "type": ["string","boolean","null"],
           "description": "The value of the property. If the value is empty or null, the property is unset."
         }
       },

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9340939/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java b/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java
index 57ab9fc..70b7f8a 100644
--- a/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/V2ApiIntegrationTest.java
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
@@ -70,6 +71,16 @@ public class V2ApiIntegrationTest extends SolrCloudTestCase {
   }
 
   @Test
+  public void testSetPropertyValidationOfCluster() throws IOException, SolrServerException {
+    NamedList resp = cluster.getSolrClient().request(
+      new V2Request.Builder("/cluster").withMethod(SolrRequest.METHOD.POST).withPayload("{set-property: {name: autoAddReplicas, val:false}}").build());
+    assertTrue(resp.toString().contains("status=0"));
+    resp = cluster.getSolrClient().request(
+        new V2Request.Builder("/cluster").withMethod(SolrRequest.METHOD.POST).withPayload("{set-property: {name: autoAddReplicas, val:null}}").build());
+    assertTrue(resp.toString().contains("status=0"));
+  }
+
+  @Test
   public void testCollectionsApi() throws Exception {
     CloudSolrClient client = cluster.getSolrClient();
     Map result = resAsMap(client, new V2Request.Builder("/c/"+COLL_NAME+"/get/_introspect").build());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c9340939/solr/core/src/test/org/apache/solr/util/JsonValidatorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/util/JsonValidatorTest.java b/solr/core/src/test/org/apache/solr/util/JsonValidatorTest.java
index 4492586..02e95f7 100644
--- a/solr/core/src/test/org/apache/solr/util/JsonValidatorTest.java
+++ b/solr/core/src/test/org/apache/solr/util/JsonValidatorTest.java
@@ -53,21 +53,19 @@ public class JsonValidatorTest extends SolrTestCaseJ4 {
     JsonSchemaValidator validator = new JsonSchemaValidator(createSchema);
     List<String> errs = validator.validateJson(Utils.fromJSONString("{name : x, collections: [ c1 , c2]}"));
     assertNull(toJSONString(errs), errs);
-    errs = validator.validateJson(Utils.fromJSONString("{name : x, collections: c1 }"));
+    errs = validator.validateJson(Utils.fromJSONString("{name : x, collections: [c1] }"));
     assertNull(toJSONString(errs), errs);
     errs = validator.validateJson(Utils.fromJSONString("{name : x, x:y, collections: [ c1 , c2]}"));
     assertNotNull(toJSONString(errs), errs);
     assertTrue(toJSONString(errs), errs.get(0).contains("Unknown"));
     errs = validator.validateJson(Utils.fromJSONString("{name : 123, collections: c1 }"));
     assertNotNull(toJSONString(errs), errs);
-    assertTrue(toJSONString(errs), errs.get(0).contains("Expected type"));
+    assertTrue(toJSONString(errs), errs.get(0).contains("expected"));
     errs = validator.validateJson(Utils.fromJSONString("{x:y, collections: [ c1 , c2]}"));
-    assertEquals(toJSONString(errs), 2, errs.size());
-    assertTrue(toJSONString(errs), StrUtils.join(errs, '|').contains("Missing field"));
     assertTrue(toJSONString(errs), StrUtils.join(errs, '|').contains("Unknown"));
     errs = validator.validateJson(Utils.fromJSONString("{name : x, collections: [ 1 , 2]}"));
     assertFalse(toJSONString(errs), errs.isEmpty());
-    assertTrue(toJSONString(errs), errs.get(0).contains("Expected elements of type"));
+    assertTrue(toJSONString(errs), errs.get(0).contains("expected"));
     validator = new JsonSchemaValidator("{" +
         "  type:object," +
         "  properties: {" +
@@ -77,7 +75,7 @@ public class JsonValidatorTest extends SolrTestCaseJ4 {
     errs = validator.validateJson(Utils.fromJSONString("{name:x, age:21, adult:true}"));
     assertNull(errs);
     errs = validator.validateJson(Utils.fromJSONString("{name:x, age:'21', adult:'true'}"));
-    assertNull(errs);
+    assertNotNull(errs);
 
     errs = validator.validateJson(Utils.fromJSONString("{name:x, age:'x21', adult:'true'}"));
     assertEquals(1, errs.size());
@@ -128,8 +126,8 @@ public class JsonValidatorTest extends SolrTestCaseJ4 {
     assertNull("errs are " + errs, errs);
     errs = validator.validateJson(Utils.fromJSONString("{name: 'Joe Average' , sex:m}"));
     assertEquals(1, errs.size());
-    assertTrue(errs.get(0).contains("value of enum"));
-    
+    assertTrue(errs.get(0).contains("Value of enum"));
+
     String schema = "{\n" +
         "  'type': 'object',\n" +
         "  'properties': {\n" +
@@ -167,9 +165,18 @@ public class JsonValidatorTest extends SolrTestCaseJ4 {
         "    }\n" +
         "  ]\n" +
         "}"));
-    
-
 
+    schema = "{\n" +
+        "'type' : 'object',\n" +
+        "'oneOf' : ['a', 'b']\n" +
+        "}";
+    validator = new JsonSchemaValidator(schema);
+    errs = validator.validateJson(Utils.fromJSONString("" +
+        "{'c':'val'}"));
+    assertNotNull(errs);
+    errs = validator.validateJson(Utils.fromJSONString("" +
+        "{'a':'val'}"));
+    assertNull(errs);
 
   }
 


[2/2] lucene-solr:master: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/lucene-solr

Posted by da...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/lucene-solr


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

Branch: refs/heads/master
Commit: 896270d1321296f5851cd39671328619225eb5a6
Parents: c934093 5235903
Author: Cao Manh Dat <da...@apache.org>
Authored: Thu May 18 11:24:58 2017 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Thu May 18 11:24:58 2017 +0700

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   6 +-
 .../org/apache/lucene/codecs/CodecUtil.java     |   6 +
 .../org/apache/lucene/codecs/TestCodecUtil.java |  13 +
 solr/CHANGES.txt                                |  30 +-
 solr/bin/install_solr_service.sh                |   8 +-
 solr/bin/solr                                   | 198 +++++++++--
 solr/bin/solr.cmd                               | 106 ++++--
 solr/bin/solr.in.cmd                            |   3 +
 solr/bin/solr.in.sh                             |   3 +
 .../org/apache/solr/cloud/RecoveryStrategy.java |   5 +-
 .../org/apache/solr/handler/SchemaHandler.java  |  10 -
 .../org/apache/solr/handler/StreamHandler.java  |   7 +
 .../apache/solr/response/SchemaXmlWriter.java   |   4 -
 .../org/apache/solr/schema/IndexSchema.java     |  26 +-
 .../apache/solr/schema/ManagedIndexSchema.java  |   2 -
 .../solr/search/ComplexPhraseQParserPlugin.java |   2 +-
 .../org/apache/solr/search/DisMaxQParser.java   |   5 +-
 .../solr/search/ExtendedDismaxQParser.java      |   3 +-
 .../org/apache/solr/search/LuceneQParser.java   |   4 +-
 .../org/apache/solr/search/QueryParsing.java    |  27 +-
 .../apache/solr/search/SimpleQParserPlugin.java |   2 +-
 .../apache/solr/servlet/SolrDispatchFilter.java |   2 +
 .../solr/update/DefaultSolrCoreState.java       |   4 +-
 .../src/java/org/apache/solr/util/SolrCLI.java  | 330 +++++++++++--------
 .../util/configuration/SSLConfigurations.java   |  78 +++++
 .../configuration/SSLConfigurationsFactory.java |  49 +++
 .../solr/util/configuration/package-info.java   |  23 ++
 .../src/resources/apispec/core.SchemaRead.json  |   3 +-
 .../conf/bad-schema-default-operator.xml        |  26 ++
 ...tSolrQueryParserDefaultOperatorResource.java |  29 --
 .../schema/TestSolrQueryParserResource.java     |  30 --
 .../apache/solr/schema/BadIndexSchemaTest.java  |   6 +-
 .../solr/schema/TestUseDocValuesAsStored.java   |   2 +-
 .../configuration/SSLConfigurationsTest.java    | 121 +++++++
 solr/server/etc/jetty-ssl.xml                   |   4 +-
 solr/solr-ref-guide/src/404.md                  |   6 -
 solr/solr-ref-guide/src/_config.yml.template    |  12 -
 solr/solr-ref-guide/src/_includes/archive.html  |  15 -
 solr/solr-ref-guide/src/_includes/feedback.html |  16 -
 solr/solr-ref-guide/src/_includes/footer.html   |   2 +-
 solr/solr-ref-guide/src/_includes/image.html    |   1 -
 .../src/_includes/inline_image.html             |   1 -
 solr/solr-ref-guide/src/_includes/links.html    |  44 ---
 solr/solr-ref-guide/src/_includes/topnav.html   |   3 -
 solr/solr-ref-guide/src/_layouts/post.html      |  41 ---
 solr/solr-ref-guide/src/css/ref-guide.css       |   2 +-
 solr/solr-ref-guide/src/draft-background.png    | Bin 5391 -> 0 bytes
 .../src/images/draft-background.png             | Bin 0 -> 5391 bytes
 .../src/images/solr-sunOnly-small.png           | Bin 0 -> 7528 bytes
 .../src/other-schema-elements.adoc              |   8 +-
 .../src/pdf/themes/refguide-theme.yml           |   2 +-
 solr/solr-ref-guide/src/schema-api.adoc         |  58 ----
 solr/solr-ref-guide/src/solr-sunOnly-small.png  | Bin 7528 -> 0 bytes
 .../src/the-dismax-query-parser.adoc            |   2 +-
 .../client/solrj/io/stream/CopyOfEvaluator.java |  84 +++++
 .../solrj/io/stream/DistanceEvaluator.java      |  77 +++++
 .../stream/EmpiricalDistributionEvaluator.java  | 129 ++++++++
 .../client/solrj/io/stream/LengthEvaluator.java |  60 ++++
 .../solrj/io/stream/PercentileEvaluator.java    |  68 ++++
 .../client/solrj/io/stream/RankEvaluator.java   |  75 +++++
 .../client/solrj/io/stream/ScaleEvaluator.java  |  78 +++++
 .../solrj/request/schema/SchemaRequest.java     |  19 --
 .../response/schema/SchemaRepresentation.java   |  10 -
 .../solrj/response/schema/SchemaResponse.java   |  28 --
 .../solrj/io/stream/StreamExpressionTest.java   | 330 +++++++++++++++++++
 .../solr/client/solrj/request/SchemaTest.java   |  10 -
 66 files changed, 1741 insertions(+), 617 deletions(-)
----------------------------------------------------------------------