You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@unomi.apache.org by dg...@apache.org on 2022/04/22 07:03:01 UTC

[unomi] 01/01: DMF-5359 : - Add schema to contextRequest object - Improve schema validation by using custom deserializers for contextRequest and eventCollectorRequest

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

dgriffon pushed a commit to branch add-schema-validators
in repository https://gitbox.apache.org/repos/asf/unomi.git

commit a472c79c08a221836a60d055804d0e01c462722c
Author: David Griffon <dg...@jahia.com>
AuthorDate: Fri Apr 22 09:02:47 2022 +0200

    DMF-5359 :
    - Add schema to contextRequest object
    - Improve schema validation by using custom deserializers for contextRequest and eventCollectorRequest
---
 api/pom.xml                                        |   5 +
 .../apache/unomi/api/services/SchemaRegistry.java  |   3 +-
 .../unomi/persistence/spi/CustomObjectMapper.java  |   8 ++
 .../deserializers/ContextRequestDeserializer.java  | 145 +++++++++++++++++++++
 .../EventCollectorRequestDeserializer.java         |  65 +++++++++
 .../org/apache/unomi/rest/server/RestServer.java   |  25 +++-
 .../rest/service/impl/RestServiceUtilsImpl.java    |   5 -
 .../services/impl/events/EventServiceImpl.java     |   5 +-
 .../services/impl/schemas/SchemaRegistryImpl.java  |   4 +-
 .../META-INF/cxs/schemas/contextrequest.json       |  64 +++++++++
 .../main/resources/META-INF/cxs/schemas/event.json |   8 +-
 .../main/resources/META-INF/cxs/schemas/item.json  |   8 +-
 .../cxs/schemas/personalization/filter.json        |  21 +++
 .../personalization/personalizationrequest.json    |  25 ++++
 .../personalization/personalizedcontent.json       |  22 ++++
 .../cxs/schemas/personalization/target.json        |  17 +++
 .../META-INF/cxs/schemas/timestampeditem.json      |   2 +-
 17 files changed, 408 insertions(+), 24 deletions(-)

diff --git a/api/pom.xml b/api/pom.xml
index 75da098d3..fa706a4dd 100644
--- a/api/pom.xml
+++ b/api/pom.xml
@@ -57,6 +57,11 @@
             <groupId>commons-collections</groupId>
             <artifactId>commons-collections</artifactId>
         </dependency>
+        <dependency>
+            <groupId>com.fasterxml.jackson.core</groupId>
+            <artifactId>jackson-databind</artifactId>
+            <scope>provided</scope>
+        </dependency>
     </dependencies>
 
     <reporting>
diff --git a/api/src/main/java/org/apache/unomi/api/services/SchemaRegistry.java b/api/src/main/java/org/apache/unomi/api/services/SchemaRegistry.java
index b25a613f8..2afc78982 100644
--- a/api/src/main/java/org/apache/unomi/api/services/SchemaRegistry.java
+++ b/api/src/main/java/org/apache/unomi/api/services/SchemaRegistry.java
@@ -17,6 +17,7 @@
 
 package org.apache.unomi.api.services;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import org.apache.unomi.api.Metadata;
 import org.apache.unomi.api.PartialList;
 import org.apache.unomi.api.schema.json.JSONSchema;
@@ -51,7 +52,7 @@ public interface SchemaRegistry {
      * @param schemaId id of the schema used for the validation
      * @return true is the object is valid
      */
-    boolean isValid(Object object, String schemaId);
+    boolean isValid(JsonNode object, String schemaId);
 
 
     /**
diff --git a/persistence-spi/src/main/java/org/apache/unomi/persistence/spi/CustomObjectMapper.java b/persistence-spi/src/main/java/org/apache/unomi/persistence/spi/CustomObjectMapper.java
index f5f28448c..9e23a2399 100644
--- a/persistence-spi/src/main/java/org/apache/unomi/persistence/spi/CustomObjectMapper.java
+++ b/persistence-spi/src/main/java/org/apache/unomi/persistence/spi/CustomObjectMapper.java
@@ -20,6 +20,7 @@ package org.apache.unomi.persistence.spi;
 import com.fasterxml.jackson.core.Version;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.SerializationFeature;
+import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
 import com.fasterxml.jackson.databind.module.SimpleModule;
 import com.fasterxml.jackson.databind.util.ISO8601DateFormat;
 import com.fasterxml.jackson.module.jaxb.JaxbAnnotationModule;
@@ -50,6 +51,10 @@ public class CustomObjectMapper extends ObjectMapper {
     private ItemDeserializer itemDeserializer;
 
     public CustomObjectMapper() {
+        this(null);
+    }
+
+    public CustomObjectMapper(Map<Class, StdDeserializer<?>> deserializers) {
         super();
         super.registerModule(new JaxbAnnotationModule());
         configure(SerializationFeature.WRITE_DATES_AS_TIMESTAMPS, false);
@@ -67,6 +72,9 @@ public class CustomObjectMapper extends ObjectMapper {
         itemDeserializer = new ItemDeserializer();
         deserializerModule.addDeserializer(Item.class, itemDeserializer);
 
+        if (deserializers != null) {
+            deserializers.forEach(deserializerModule::addDeserializer);
+        }
 
         builtinItemTypeClasses = new HashMap<>();
         builtinItemTypeClasses.put(Campaign.ITEM_TYPE, Campaign.class);
diff --git a/rest/src/main/java/org/apache/unomi/rest/deserializers/ContextRequestDeserializer.java b/rest/src/main/java/org/apache/unomi/rest/deserializers/ContextRequestDeserializer.java
new file mode 100644
index 000000000..aa440a41e
--- /dev/null
+++ b/rest/src/main/java/org/apache/unomi/rest/deserializers/ContextRequestDeserializer.java
@@ -0,0 +1,145 @@
+/*
+ * 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.unomi.rest.deserializers;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import org.apache.unomi.api.ContextRequest;
+import org.apache.unomi.api.Event;
+import org.apache.unomi.api.Profile;
+import org.apache.unomi.api.services.PersonalizationService;
+import org.apache.unomi.api.services.SchemaRegistry;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Custom deserializer for ContextRequest that do validate the object using JSon Schema
+ */
+public class ContextRequestDeserializer extends StdDeserializer<ContextRequest> {
+
+    private final SchemaRegistry schemaRegistry;
+
+    public ContextRequestDeserializer(SchemaRegistry schemaRegistry) {
+        this(null,  schemaRegistry);
+    }
+
+    public ContextRequestDeserializer(Class<ContextRequest> vc, SchemaRegistry schemaRegistry) {
+        super(vc);
+        this.schemaRegistry = schemaRegistry;
+    }
+
+    @Override
+    public ContextRequest deserialize(JsonParser jsonParser, DeserializationContext context) throws IOException, JsonProcessingException {
+        JsonNode node = jsonParser.getCodec().readTree(jsonParser);
+        // Validate schema on it
+        if (schemaRegistry.isValid(node, "https://unomi.apache.org/schemas/json/contextrequest/1-0-0")) {
+            ContextRequest cr = new ContextRequest();
+            if (node.get("requiredSessionProperties") != null) {
+                cr.setRequireSegments(node.get("requiredSessionProperties").booleanValue());
+            }
+            if (node.get("requiredProfileProperties") != null) {
+                List<String> profileProperties = new ArrayList<>();
+                node.get("requiredProfileProperties").elements().forEachRemaining(el -> profileProperties.add(el.textValue()));
+                cr.setRequiredProfileProperties(profileProperties);
+            }
+            if (node.get("requiredSessionProperties") != null) {
+                List<String> requiredSessionProperties = new ArrayList<>();
+                node.get("requiredSessionProperties").elements().forEachRemaining(el -> requiredSessionProperties.add(el.textValue()));
+                cr.setRequiredProfileProperties(requiredSessionProperties);
+                cr.setRequireScores(node.get("requireScores").booleanValue());
+            }
+            if (node.get("events") != null) {
+                ArrayNode events = (ArrayNode) node.get("events");
+                List<Event> filteredEvents = new ArrayList<>();
+                for (JsonNode event : events) {
+                    if (schemaRegistry.isValid(event, "https://unomi.apache.org/schemas/json/events/" + event.get("eventType").textValue() + "/1-0-0")) {
+                        filteredEvents.add(jsonParser.getCodec().treeToValue(event, Event.class));
+                    }
+                }
+                cr.setEvents(filteredEvents);
+            }
+            if (node.get("filters") != null) {
+                ArrayNode filters = (ArrayNode) node.get("filters");
+                List<PersonalizationService.PersonalizedContent> f = new ArrayList<>();
+                filters.elements().forEachRemaining(el -> {
+                    try {
+                        jsonParser.getCodec().treeToValue(el, PersonalizationService.PersonalizedContent.class);
+                    } catch (JsonProcessingException e) {
+                        // Unable to deserialize, ignore the entry
+                    }
+                });
+                cr.setFilters(f);
+            }
+            if (node.get("personalizations") != null) {
+                ArrayNode personalizations = (ArrayNode) node.get("personalizations");
+                List<PersonalizationService.PersonalizationRequest> p = new ArrayList<>();
+                personalizations.elements().forEachRemaining(el -> {
+                    try {
+                        jsonParser.getCodec().treeToValue(el, PersonalizationService.PersonalizationRequest.class);
+                    } catch (JsonProcessingException e) {
+                        // Unable to deserialize, ignore the entry
+                    }
+                });
+                cr.setPersonalizations(p);
+            }
+            if (node.get("profileOverrides") != null) {
+                cr.setProfileOverrides(jsonParser.getCodec().treeToValue(node.get("profileOverrides"), Profile.class));
+            }
+            if (node.get("sessionPropertiesOverrides") != null) {
+                jsonParser.getCodec().treeToValue(node.get("sessionPropertiesOverrides"), Map.class);
+            }
+            if (node.get("sessionId") != null) {
+                cr.setSessionId(node.get("sessionId").textValue());
+            }
+            if (node.get("profileId") != null) {
+                cr.setProfileId(node.get("profileId").textValue());
+            }
+            if (node.get("clientId") != null) {
+                cr.setClientId(node.get("clientId").textValue());
+            }
+            return cr;
+        }
+        throw new IOException("Unable to deserialize provided context request");
+    }
+}
+/**
+ * private Item source;
+ *     private boolean requireSegments;
+ *     private List<String> requiredProfileProperties;
+ *     private List<String> requiredSessionProperties;
+ *     private boolean requireScores;
+ *     private List<Event> events;
+ *     private List<PersonalizationService.PersonalizedContent> filters;
+ *     private List<PersonalizationService.PersonalizationRequest> personalizations;
+ *     private Profile profileOverrides;
+ *     private Map<String, Object> sessionPropertiesOverrides;
+ *
+ *     @Pattern(regexp = ValidationPattern.TEXT_VALID_CHARACTERS_PATTERN)
+ *     private String sessionId;
+ *
+ *     @Pattern(regexp = ValidationPattern.TEXT_VALID_CHARACTERS_PATTERN)
+ *     private String profileId;
+ *
+ *     private String clientId;
+ */
\ No newline at end of file
diff --git a/rest/src/main/java/org/apache/unomi/rest/deserializers/EventCollectorRequestDeserializer.java b/rest/src/main/java/org/apache/unomi/rest/deserializers/EventCollectorRequestDeserializer.java
new file mode 100644
index 000000000..06ab2ab43
--- /dev/null
+++ b/rest/src/main/java/org/apache/unomi/rest/deserializers/EventCollectorRequestDeserializer.java
@@ -0,0 +1,65 @@
+/*
+ * 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.unomi.rest.deserializers;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import org.apache.unomi.api.Event;
+import org.apache.unomi.api.EventsCollectorRequest;
+import org.apache.unomi.api.services.SchemaRegistry;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Custom deserializer for EventCollectorRequest that do validate the events using JSon Schema
+ */
+public class EventCollectorRequestDeserializer extends StdDeserializer<EventsCollectorRequest> {
+
+    private final SchemaRegistry schemaRegistry;
+
+    public EventCollectorRequestDeserializer(SchemaRegistry schemaRegistry) {
+        this(null, schemaRegistry);
+    }
+
+    public EventCollectorRequestDeserializer(Class<EventsCollectorRequest> vc, SchemaRegistry schemaRegistry) {
+        super(vc);
+        this.schemaRegistry = schemaRegistry;
+    }
+
+    @Override
+    public EventsCollectorRequest deserialize(JsonParser jsonParser, DeserializationContext context) throws IOException, JsonProcessingException {
+        JsonNode node = jsonParser.getCodec().readTree(jsonParser);
+        // Validate schema on each event
+        ArrayNode events = (ArrayNode) node.get("events");
+        List<Event> filteredEvents = new ArrayList<>();
+        for (JsonNode event : events) {
+            if (schemaRegistry.isValid(event, "https://unomi.apache.org/schemas/json/events/" + event.get("eventType").textValue() + "/1-0-0")) {
+                filteredEvents.add(jsonParser.getCodec().treeToValue(event, Event.class));
+            }
+        }
+        EventsCollectorRequest eventsCollectorRequest = new EventsCollectorRequest();
+        eventsCollectorRequest.setSessionId(node.get("sessionId").textValue());
+        eventsCollectorRequest.setEvents(filteredEvents);
+        return eventsCollectorRequest;
+    }
+}
diff --git a/rest/src/main/java/org/apache/unomi/rest/server/RestServer.java b/rest/src/main/java/org/apache/unomi/rest/server/RestServer.java
index c260ffee0..82a32378e 100644
--- a/rest/src/main/java/org/apache/unomi/rest/server/RestServer.java
+++ b/rest/src/main/java/org/apache/unomi/rest/server/RestServer.java
@@ -17,6 +17,7 @@
 package org.apache.unomi.rest.server;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
 import com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider;
 import org.apache.cxf.Bus;
 import org.apache.cxf.endpoint.Server;
@@ -28,10 +29,15 @@ import org.apache.cxf.jaxrs.security.SimpleAuthorizingFilter;
 import org.apache.cxf.jaxrs.validation.JAXRSBeanValidationInInterceptor;
 import org.apache.cxf.jaxrs.validation.JAXRSBeanValidationOutInterceptor;
 import org.apache.cxf.message.Message;
+import org.apache.unomi.api.ContextRequest;
+import org.apache.unomi.api.EventsCollectorRequest;
 import org.apache.unomi.api.services.ConfigSharingService;
+import org.apache.unomi.api.services.SchemaRegistry;
 import org.apache.unomi.rest.authentication.AuthenticationFilter;
 import org.apache.unomi.rest.authentication.AuthorizingInterceptor;
 import org.apache.unomi.rest.authentication.RestAuthenticationConfig;
+import org.apache.unomi.rest.deserializers.ContextRequestDeserializer;
+import org.apache.unomi.rest.deserializers.EventCollectorRequestDeserializer;
 import org.apache.unomi.rest.server.provider.RetroCompatibilityParamConverterProvider;
 import org.apache.unomi.rest.validation.JAXRSBeanValidationInInterceptorOverride;
 import org.apache.unomi.rest.validation.BeanValidationService;
@@ -52,11 +58,7 @@ import org.slf4j.LoggerFactory;
 
 import javax.ws.rs.ext.ExceptionMapper;
 import javax.xml.namespace.QName;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.List;
-import java.util.Timer;
-import java.util.TimerTask;
+import java.util.*;
 import java.util.concurrent.CopyOnWriteArrayList;
 
 @Component
@@ -75,6 +77,7 @@ public class RestServer {
     private List<ExceptionMapper> exceptionMappers = new ArrayList<>();
     private BeanValidationService beanValidationService;
     private ConfigSharingService configSharingService;
+    private SchemaRegistry schemaRegistry;
 
     // refresh
     private long timeOfLastUpdate = System.currentTimeMillis();
@@ -83,6 +86,11 @@ public class RestServer {
 
     private static final QName UNOMI_REST_SERVER_END_POINT_NAME = new QName("http://rest.unomi.apache.org/", "UnomiRestServerEndPoint");
 
+    @Reference(cardinality = ReferenceCardinality.MANDATORY)
+    public void setSchemaRegistry(SchemaRegistry schemaRegistry) {
+        this.schemaRegistry = schemaRegistry;
+    }
+
     @Reference(cardinality = ReferenceCardinality.MANDATORY)
     public void setServerBus(Bus serverBus) {
         this.serverBus = serverBus;
@@ -203,8 +211,13 @@ public class RestServer {
         List<Interceptor<? extends Message>> inInterceptors = new ArrayList<>();
         List<Interceptor<? extends Message>> outInterceptors = new ArrayList<>();
 
+        Map<Class, StdDeserializer<?>> desers = new HashMap<>();
+        desers.put(ContextRequest.class, new ContextRequestDeserializer(schemaRegistry));
+        desers.put(EventsCollectorRequest.class, new EventCollectorRequestDeserializer(schemaRegistry));
+
+
         // Build the server
-        ObjectMapper objectMapper = new org.apache.unomi.persistence.spi.CustomObjectMapper();
+        ObjectMapper objectMapper = new org.apache.unomi.persistence.spi.CustomObjectMapper(desers);
         JAXRSServerFactoryBean jaxrsServerFactoryBean = new JAXRSServerFactoryBean();
         jaxrsServerFactoryBean.setAddress("/");
         jaxrsServerFactoryBean.setBus(serverBus);
diff --git a/rest/src/main/java/org/apache/unomi/rest/service/impl/RestServiceUtilsImpl.java b/rest/src/main/java/org/apache/unomi/rest/service/impl/RestServiceUtilsImpl.java
index 35a3d92a0..9f05bd6be 100644
--- a/rest/src/main/java/org/apache/unomi/rest/service/impl/RestServiceUtilsImpl.java
+++ b/rest/src/main/java/org/apache/unomi/rest/service/impl/RestServiceUtilsImpl.java
@@ -87,11 +87,6 @@ public class RestServiceUtilsImpl implements RestServiceUtils {
             for (Event event : events) {
                 processedEventsCnt++;
                 if (event.getEventType() != null) {
-                    if (!eventService.isEventValid(event)) {
-                        logger.warn("Event is not valid : {}", event.getEventType());
-                        continue;
-                    }
-
                     Event eventToSend = new Event(event.getEventType(), session, profile, event.getSourceId(), event.getSource(),
                             event.getTarget(), event.getProperties(), timestamp, event.isPersistent());
                     if (!eventService.isEventAllowed(event, thirdPartyId)) {
diff --git a/services/src/main/java/org/apache/unomi/services/impl/events/EventServiceImpl.java b/services/src/main/java/org/apache/unomi/services/impl/events/EventServiceImpl.java
index ddaefeac5..a58546e28 100644
--- a/services/src/main/java/org/apache/unomi/services/impl/events/EventServiceImpl.java
+++ b/services/src/main/java/org/apache/unomi/services/impl/events/EventServiceImpl.java
@@ -17,6 +17,7 @@
 
 package org.apache.unomi.services.impl.events;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import inet.ipaddr.IPAddress;
 import inet.ipaddr.IPAddressString;
 import org.apache.commons.lang3.StringUtils;
@@ -31,6 +32,7 @@ import org.apache.unomi.api.actions.ActionPostExecutor;
 import org.apache.unomi.api.conditions.Condition;
 import org.apache.unomi.api.query.Query;
 import org.apache.unomi.api.services.*;
+import org.apache.unomi.persistence.spi.CustomObjectMapper;
 import org.apache.unomi.persistence.spi.PersistenceService;
 import org.apache.unomi.persistence.spi.aggregate.TermsAggregate;
 import org.apache.unomi.api.utils.ParserHelper;
@@ -139,7 +141,8 @@ public class EventServiceImpl implements EventService {
     }
 
     public boolean isEventValid(Event event) {
-        return schemaRegistry.isValid(event, "https://unomi.apache.org/schemas/json/events/" + event.getEventType() + "/1-0-0");
+
+        return schemaRegistry.isValid(CustomObjectMapper.getObjectMapper().convertValue(event, JsonNode.class), "https://unomi.apache.org/schemas/json/events/" + event.getEventType() + "/1-0-0");
     }
 
     public String authenticateThirdPartyServer(String key, String ip) {
diff --git a/services/src/main/java/org/apache/unomi/services/impl/schemas/SchemaRegistryImpl.java b/services/src/main/java/org/apache/unomi/services/impl/schemas/SchemaRegistryImpl.java
index ae4e3dbeb..d51e6cb1b 100644
--- a/services/src/main/java/org/apache/unomi/services/impl/schemas/SchemaRegistryImpl.java
+++ b/services/src/main/java/org/apache/unomi/services/impl/schemas/SchemaRegistryImpl.java
@@ -105,7 +105,7 @@ public class SchemaRegistryImpl implements SchemaRegistry {
     }
 
     @Override
-    public boolean isValid(Object object, String schemaId) {
+    public boolean isValid(JsonNode jsonNode, String schemaId) {
         String schemaAsString;
         JsonSchema jsonSchema = null;
         try {
@@ -121,7 +121,7 @@ public class SchemaRegistryImpl implements SchemaRegistry {
         }
 
         if (jsonSchema != null) {
-            JsonNode jsonNode = CustomObjectMapper.getObjectMapper().convertValue(object, JsonNode.class);
+
             Set<ValidationMessage> validationMessages = jsonSchema.validate(jsonNode);
             if (validationMessages == null || validationMessages.isEmpty()) {
                 return true;
diff --git a/services/src/main/resources/META-INF/cxs/schemas/contextrequest.json b/services/src/main/resources/META-INF/cxs/schemas/contextrequest.json
new file mode 100644
index 000000000..30ad5702b
--- /dev/null
+++ b/services/src/main/resources/META-INF/cxs/schemas/contextrequest.json
@@ -0,0 +1,64 @@
+{
+  "$id": "https://unomi.apache.org/schemas/json/contextrequest/1-0-0",
+  "$schema": "https://json-schema.org/draft/2019-09/schema",
+  "title": "ContextRequest",
+  "type": "object",
+  "properties": {
+    "source": {
+      "$ref": "https://unomi.apache.org/schemas/json/item/1-0-0"
+    },
+    "requireSegments": {
+      "type": "boolean"
+    },
+    "requiredProfileProperties": {
+      "type": ["array"],
+      "items": {
+        "type": "string"
+      }
+    },
+    "requiredSessionProperties": {
+      "type": ["array"],
+      "items": {
+        "type": "string"
+      }
+    },
+    "requireScores": {
+      "type": "boolean"
+    },
+    "events": {
+      "type": "array",
+      "items": {
+        "$ref": "https://unomi.apache.org/schemas/json/event/1-0-0"
+      }
+    },
+    "filters": {
+      "type": ["array"],
+      "items": {
+        "$ref": "https://unomi.apache.org/schemas/json/personalization/personalizedcontent/1-0-0"
+      }
+    },
+    "personalizations": {
+      "type": ["array"],
+      "items": {
+        "$ref": "https://unomi.apache.org/schemas/json/event/1-0-0"
+      }
+    },
+    "profileOverrides": {
+      "$ref": "https://unomi.apache.org/schemas/json/profile/1-0-0"
+    },
+    "sessionPropertiesOverrides": {
+      "type": ["object"]
+    },
+    "sessionId": {
+      "type": ["string"],
+      "pattern" : "^(\\w|[-_@\\.]){0,60}$"
+    },
+    "profileId": {
+      "type": ["string"],
+      "pattern" : "^(\\w|[-_@\\.]){0,60}$"
+    },
+    "clientId": {
+      "type": ["string"]
+    }
+  }
+}
diff --git a/services/src/main/resources/META-INF/cxs/schemas/event.json b/services/src/main/resources/META-INF/cxs/schemas/event.json
index 66cb8ad85..7c3b5bfd1 100644
--- a/services/src/main/resources/META-INF/cxs/schemas/event.json
+++ b/services/src/main/resources/META-INF/cxs/schemas/event.json
@@ -10,19 +10,19 @@
       "pattern" : "^(\\w|[-_@\\.]){0,60}$"
     },
     "profileId" : {
-      "type" : [ "null", "string"],
+      "type" : [ "string"],
       "pattern" : "^(\\w|[-_@\\.]){0,60}$"
     },
     "sessionId" : {
-      "type" : [ "null", "string"],
+      "type" : [ "string"],
       "pattern" : "^(\\w|[-_@\\.]){0,60}$"
     },
     "scope" : {
-      "type" : [ "null", "string"],
+      "type" : [ "string"],
       "pattern" : "^(\\w|[-_@\\.]){0,60}$"
     },
     "sourceId" : {
-      "type" : [ "null", "string"],
+      "type" : [ "string"],
       "pattern" : "^(\\w|[-_@\\.]){0,60}$"
     }
   }
diff --git a/services/src/main/resources/META-INF/cxs/schemas/item.json b/services/src/main/resources/META-INF/cxs/schemas/item.json
index ed9dff56c..484ce11ed 100644
--- a/services/src/main/resources/META-INF/cxs/schemas/item.json
+++ b/services/src/main/resources/META-INF/cxs/schemas/item.json
@@ -5,7 +5,7 @@
   "type": "object",
   "properties" : {
     "itemId" : {
-      "type" : ["null","string"],
+      "type" : ["string"],
       "pattern" : "^(\\w|[-_@\\.]){0,60}$",
       "description" : "The identifier for the item"
     },
@@ -14,16 +14,16 @@
       "description" : "The type for the item"
     },
     "scope" : {
-      "type" : ["null","string"],
+      "type" : ["string"],
       "description" : "The item's scope"
     },
     "version" : {
-      "type" : ["null","integer"],
+      "type" : ["integer"],
       "minimum" : 0,
       "description" : "The item's version number"
     },
     "systemMetadata" : {
-      "type" : ["null","object"],
+      "type" : ["object"],
       "description" : "Any system metadata for the item",
       "maxProperties" : 50
     }
diff --git a/services/src/main/resources/META-INF/cxs/schemas/personalization/filter.json b/services/src/main/resources/META-INF/cxs/schemas/personalization/filter.json
new file mode 100644
index 000000000..112be929e
--- /dev/null
+++ b/services/src/main/resources/META-INF/cxs/schemas/personalization/filter.json
@@ -0,0 +1,21 @@
+{
+  "$id": "https://unomi.apache.org/schemas/json/personalization/filter/1-0-0",
+  "$schema": "https://json-schema.org/draft/2019-09/schema",
+  "title": "Filter",
+  "type": "object",
+  "properties": {
+    "appliesOn": {
+      "type": "array",
+      "items": {
+        "$ref": "https://unomi.apache.org/schemas/json/personalization/target/1-0-0"
+      }
+    },
+    "condition": {
+      "$ref": "https://unomi.apache.org/schemas/json/condition/1-0-0"
+    },
+    "properties": {
+      "type": "object",
+      "maxProperties": 50
+    }
+  }
+}
\ No newline at end of file
diff --git a/services/src/main/resources/META-INF/cxs/schemas/personalization/personalizationrequest.json b/services/src/main/resources/META-INF/cxs/schemas/personalization/personalizationrequest.json
new file mode 100644
index 000000000..37e99f71c
--- /dev/null
+++ b/services/src/main/resources/META-INF/cxs/schemas/personalization/personalizationrequest.json
@@ -0,0 +1,25 @@
+{
+  "$id": "https://unomi.apache.org/schemas/json/personalization/personalizedrequest/1-0-0",
+  "$schema": "https://json-schema.org/draft/2019-09/schema",
+  "title": "PersonalizationRequest",
+  "type": "object",
+  "properties": {
+    "id" : {
+      "type" : "string"
+    },
+    "strategy" : {
+      "type" : "string"
+    },
+    "strategyOptions" : {
+      "type": "object",
+      "maxProperties": 50
+    },
+    "contents" : {
+      "type" : "array",
+      "items": {
+        "$ref": "https://unomi.apache.org/schemas/json/personalization/personalizedcontent/1-0-0"
+      },
+      "maxProperties": 50
+    }
+  }
+}
\ No newline at end of file
diff --git a/services/src/main/resources/META-INF/cxs/schemas/personalization/personalizedcontent.json b/services/src/main/resources/META-INF/cxs/schemas/personalization/personalizedcontent.json
new file mode 100644
index 000000000..b8a29ef73
--- /dev/null
+++ b/services/src/main/resources/META-INF/cxs/schemas/personalization/personalizedcontent.json
@@ -0,0 +1,22 @@
+{
+  "$id": "https://unomi.apache.org/schemas/json/personalization/personalizedcontent/1-0-0",
+  "$schema": "https://json-schema.org/draft/2019-09/schema",
+  "title": "PersonalizedContent",
+  "type": "object",
+  "properties": {
+    "id": {
+      "type": "string"
+    },
+    "filters": {
+      "type": "array",
+      "items": {
+        "$ref": "https://unomi.apache.org/schemas/json/personalization/filter/1-0-0"
+      },
+      "maxProperties": 50
+    },
+    "properties": {
+      "type": "object",
+      "maxProperties": 50
+    }
+  }
+}
\ No newline at end of file
diff --git a/services/src/main/resources/META-INF/cxs/schemas/personalization/target.json b/services/src/main/resources/META-INF/cxs/schemas/personalization/target.json
new file mode 100644
index 000000000..4de017a47
--- /dev/null
+++ b/services/src/main/resources/META-INF/cxs/schemas/personalization/target.json
@@ -0,0 +1,17 @@
+{
+  "$id": "https://unomi.apache.org/schemas/json/personalization/target/1-0-0",
+  "$schema": "https://json-schema.org/draft/2019-09/schema",
+  "title": "Target",
+  "type": "object",
+  "properties": {
+    "target" : {
+      "type" : "string"
+    },
+    "values" : {
+      "type" : "array",
+      "items": {
+        "type": "string"
+      }
+    }
+  }
+}
\ No newline at end of file
diff --git a/services/src/main/resources/META-INF/cxs/schemas/timestampeditem.json b/services/src/main/resources/META-INF/cxs/schemas/timestampeditem.json
index 1d7c6ad3a..b4e1528d3 100644
--- a/services/src/main/resources/META-INF/cxs/schemas/timestampeditem.json
+++ b/services/src/main/resources/META-INF/cxs/schemas/timestampeditem.json
@@ -6,7 +6,7 @@
   "allOf": [{ "$ref": "https://unomi.apache.org/schemas/json/item/1-0-0" }],
   "properties" : {
     "timeStamp" : {
-      "type" : ["null","string"],
+      "type" : ["string"],
       "format" : "date-time"
     }
   }