You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by ce...@apache.org on 2016/06/02 00:10:35 UTC

[2/2] incubator-metron git commit: METRON-189: Add the ability to do global validations on messages passing through the parser. This closes apache/incubator-metron#138

METRON-189: Add the ability to do global validations on messages passing through the parser. This closes apache/incubator-metron#138


Project: http://git-wip-us.apache.org/repos/asf/incubator-metron/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-metron/commit/025f64c4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-metron/tree/025f64c4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-metron/diff/025f64c4

Branch: refs/heads/master
Commit: 025f64c4fa235379da9bfdccdc4d097190fc8958
Parents: d3efe3f
Author: cstella <ce...@gmail.com>
Authored: Wed Jun 1 20:10:15 2016 -0400
Committer: cstella <ce...@gmail.com>
Committed: Wed Jun 1 20:10:15 2016 -0400

----------------------------------------------------------------------
 metron-platform/metron-common/README.md         |  41 ++++-
 metron-platform/metron-common/pom.xml           |   5 +
 .../org/apache/metron/common/Constants.java     |   1 +
 .../common/configuration/Configurations.java    |  26 +++-
 .../common/configuration/FieldValidator.java    | 151 +++++++++++++++++++
 .../field/validation/FieldValidation.java       |  29 ++++
 .../field/validation/FieldValidations.java      |  54 +++++++
 .../field/validation/QueryValidation.java       |  72 +++++++++
 .../field/validation/SimpleValidation.java      |  73 +++++++++
 .../validation/network/DomainValidation.java    |  34 +++++
 .../validation/network/EmailValidation.java     |  32 ++++
 .../field/validation/network/IPValidation.java  | 116 ++++++++++++++
 .../field/validation/network/URLValidation.java |  31 ++++
 .../validation/primitive/DateValidation.java    | 117 ++++++++++++++
 .../validation/primitive/IntegerValidation.java |  32 ++++
 .../primitive/NotEmptyValidation.java           |  35 +++++
 .../validation/primitive/RegexValidation.java   |  68 +++++++++
 .../metron/common/query/LogicalFunctions.java   |  24 ++-
 .../metron/common/query/PredicateProcessor.java |  13 +-
 .../metron/common/query/QueryCompiler.java      |   5 +-
 .../field/validation/BaseValidationTest.java    |  48 ++++++
 .../field/validation/QueryValidationTest.java   |  98 ++++++++++++
 .../common/field/validation/ValidationTest.java |  89 +++++++++++
 .../network/DomainValidationTest.java           |  89 +++++++++++
 .../validation/network/EmailValidationTest.java |  91 +++++++++++
 .../validation/network/IPValidationTest.java    |  85 +++++++++++
 .../validation/network/URLValidationTest.java   |  91 +++++++++++
 .../primitive/DateValidationTest.java           |  95 ++++++++++++
 .../primitive/IntegerValidationTest.java        |  90 +++++++++++
 .../primitive/NotEmptyValidationTest.java       |  83 ++++++++++
 .../primitive/RegexValidationTest.java          |  87 +++++++++++
 .../metron/common/query/QueryParserTest.java    |   6 +-
 .../src/main/config/zookeeper/global.json       |   8 +-
 .../components/FluxTopologyComponent.java       |   4 +-
 .../apache/metron/parsers/bolt/ParserBolt.java  |  19 ++-
 35 files changed, 1922 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/README.md
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/README.md b/metron-platform/metron-common/README.md
index 678c6a2..1799287 100644
--- a/metron-platform/metron-common/README.md
+++ b/metron-platform/metron-common/README.md
@@ -17,6 +17,13 @@ The query language supports the following:
     * `TO_LOWER`
     * `TO_UPPER`
     * `TRIM`
+    * `IS_IP` : Validates that the input fields are an IP address.  By default, if no second arg is set, it assumes `IPV4`, but you can specify the type by passing in either `IPV6` or `IPV4` to the second argument.
+   * `IS_DOMAIN` 
+   * `IS_EMAIL`
+   * `IS_URL`
+   * `IS_DATE`
+   * `IS_INTEGER`
+
 
 Example query:
 
@@ -47,10 +54,42 @@ This configuration is stored in zookeeper, but looks something like
   "es.clustername": "metron",
   "es.ip": "node1",
   "es.port": "9300",
-  "es.date.format": "yyyy.MM.dd.HH"
+  "es.date.format": "yyyy.MM.dd.HH",
+  "fieldValidations" : [
+              {
+                "input" : [ "ip_src_addr", "ip_dst_addr" ],
+                "validation" : "IP",
+                "config" : {
+                    "type" : "IPV4"
+                           }
+              } 
+                       ]
 }
 ```
 
+###Validation Framework
+
+Inside of the global configuration, there is a validation framework in
+place that enables the validation that messages coming from all parsers
+are valid.  This is done in the form of validation plugins where
+assertions about fields or whole messages can be made. 
+
+The format for this is a `fieldValidations` field inside of global
+config.  This is associated with an array of field validation objects
+structured like so:
+* `input` : An array of input fields or a single field.  If this is omitted, then the whole messages is passed to the validator.
+* `config` : A String to Object map for validation configuration.  This is optional if the validation function requires no configuration.
+* `validation` : The validation function to be used.  This is one of
+   * `MQL` : Execute a Query Language statement.  Expects the query string in the `condition` field of the config.
+   * `IP` : Validates that the input fields are an IP address.  By default, if no configuration is set, it assumes `IPV4`, but you can specify the type by passing in the config by passing in `type` with either `IPV6` or `IPV4`.
+   * `DOMAIN` : Validates that the fields are all domains.
+   * `EMAIL` : Validates that the fields are all email addresses
+   * `URL` : Validates that the fields are all URLs
+   * `DATE` : Validates that the fields are a date.  Expects `format` in the config.
+   * `INTEGER` : Validates that the fields are an integer.  String representation of an integer is allowed.
+   * `REGEX_MATCH` : Validates that the fields match a regex.  Expects `pattern` in the config.
+   * `NOT_EMPTY` : Validates that the fields exist and are not empty (after trimming.)
+
 ##Sensor Enrichment Configuration
 
 The sensor specific configuration is intended to configure the

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/pom.xml
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/pom.xml b/metron-platform/metron-common/pom.xml
index 10c192c..9fa8daf 100644
--- a/metron-platform/metron-common/pom.xml
+++ b/metron-platform/metron-common/pom.xml
@@ -38,6 +38,11 @@
     </repositories>
     <dependencies>
         <dependency>
+            <groupId>commons-validator</groupId>
+            <artifactId>commons-validator</artifactId>
+            <version>1.5.1</version>
+        </dependency>
+        <dependency>
             <groupId>com.opencsv</groupId>
             <artifactId>opencsv</artifactId>
             <version>${global_opencsv_version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/main/java/org/apache/metron/common/Constants.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/Constants.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/Constants.java
index 7e791d5..1b0695f 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/Constants.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/Constants.java
@@ -28,6 +28,7 @@ public class Constants {
   public static final String SENSOR_TYPE = "source.type";
   public static final String ENRICHMENT_TOPIC = "enrichments";
   public static final String ERROR_STREAM = "error";
+  public static final String INVALID_STREAM = "invalid";
   public static final String SIMPLE_HBASE_ENRICHMENT = "hbaseEnrichment";
   public static final String SIMPLE_HBASE_THREAT_INTEL = "hbaseThreatIntel";
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configurations.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configurations.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configurations.java
index f33ebd7..8c39fab 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configurations.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/Configurations.java
@@ -19,12 +19,15 @@ package org.apache.metron.common.configuration;
 
 import com.fasterxml.jackson.core.type.TypeReference;
 import org.apache.log4j.Logger;
+import org.apache.metron.common.field.validation.FieldValidation;
 import org.apache.metron.common.utils.JSONUtils;
 
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -32,7 +35,7 @@ import java.util.concurrent.ConcurrentMap;
 public class Configurations implements Serializable {
 
   private static final Logger LOG = Logger.getLogger(Configurations.class);
-
+  private List<FieldValidator> validations = new ArrayList<>();
   protected ConcurrentMap<String, Object> configurations = new ConcurrentHashMap<>();
 
   @SuppressWarnings("unchecked")
@@ -40,6 +43,10 @@ public class Configurations implements Serializable {
     return (Map<String, Object>) configurations.get(ConfigurationType.GLOBAL.getName());
   }
 
+  public List<FieldValidator> getFieldValidations() {
+    return validations;
+  }
+
   public void updateGlobalConfig(byte[] data) throws IOException {
     if (data == null) throw new IllegalStateException("global config data cannot be null");
     updateGlobalConfig(new ByteArrayInputStream(data));
@@ -53,23 +60,34 @@ public class Configurations implements Serializable {
 
   public void updateGlobalConfig(Map<String, Object> globalConfig) {
     configurations.put(ConfigurationType.GLOBAL.getName(), globalConfig);
+    validations = FieldValidator.readValidations(getGlobalConfig());
   }
 
+
   @Override
   public boolean equals(Object o) {
     if (this == o) return true;
     if (o == null || getClass() != o.getClass()) return false;
+
     Configurations that = (Configurations) o;
-    return configurations.equals(that.configurations);
+
+    if (validations != null ? !validations.equals(that.validations) : that.validations != null) return false;
+    return configurations != null ? configurations.equals(that.configurations) : that.configurations == null;
+
   }
 
   @Override
   public int hashCode() {
-    return configurations.hashCode();
+    int result = validations != null ? validations.hashCode() : 0;
+    result = 31 * result + (configurations != null ? configurations.hashCode() : 0);
+    return result;
   }
 
   @Override
   public String toString() {
-    return configurations.toString();
+    return "Configurations{" +
+            "validations=" + validations +
+            ", configurations=" + configurations +
+            '}';
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/FieldValidator.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/FieldValidator.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/FieldValidator.java
new file mode 100644
index 0000000..46b4b74
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/configuration/FieldValidator.java
@@ -0,0 +1,151 @@
+/**
+ * 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.metron.common.configuration;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.metron.common.field.validation.FieldValidation;
+import org.apache.metron.common.field.validation.FieldValidations;
+import org.json.simple.JSONObject;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class FieldValidator {
+
+  public enum Config {
+     FIELD_VALIDATIONS("fieldValidations")
+    ,VALIDATION("validation")
+    ,INPUT("input")
+    ,CONFIG("config")
+    ;
+    String key;
+    Config(String key) {
+      this.key = key;
+    }
+    public <T> T get(Map<String, Object> config, Class<T> clazz) {
+      Object o = config.get(key);
+      if(o == null) {
+        return null;
+      }
+      return clazz.cast(o);
+    }
+  }
+  private FieldValidation validation;
+  private List<String> input;
+  private Map<String, Object> config;
+
+  public FieldValidator(Object o) {
+    if(o instanceof Map) {
+      Map<String, Object> validatorConfig = (Map<String, Object>) o;
+      Object inputObj = Config.INPUT.get(validatorConfig, Object.class);
+      if(inputObj instanceof String) {
+        input = ImmutableList.of(inputObj.toString());
+      }
+      else if(inputObj instanceof List) {
+        input = new ArrayList<>();
+        for(Object inputO : (List<Object>)inputObj) {
+          input.add(inputO.toString());
+        }
+      }
+      config = Config.CONFIG.get(validatorConfig, Map.class);
+      if(config == null) {
+        config = new HashMap<>();
+      }
+      String validator = Config.VALIDATION.get(validatorConfig, String.class);
+      if(validator == null) {
+        throw new IllegalStateException("Validation not set.");
+      }
+      validation= FieldValidations.get(validator);
+    }
+    else {
+      throw new IllegalStateException("Unable to configure field validations");
+    }
+  }
+
+  public FieldValidation getValidation() {
+    return validation;
+  }
+
+  public List<String> getInput() {
+    return input;
+  }
+
+  public Map<String, Object> getConfig() {
+    return config;
+  }
+
+  public boolean isValid(JSONObject inputData, Map<String, Object> globalConfig) {
+    Map<String, Object> in = inputData;
+    if(input != null && !input.isEmpty()) {
+      in = new HashMap<>();
+      for(String i : input) {
+        Object o = inputData.get(i);
+        in.put(i,o);
+      }
+    }
+    return validation.isValid(in, config, globalConfig);
+  }
+
+  public static List<FieldValidator> readValidations(Map<String, Object> globalConfig) {
+    List<FieldValidator> validators = new ArrayList<>();
+    List<Object> validations = (List<Object>) Config.FIELD_VALIDATIONS.get(globalConfig, List.class);
+    if(validations != null) {
+      for (Object o : validations) {
+        FieldValidator f = new FieldValidator(o);
+        f.getValidation().initialize(f.getConfig(), globalConfig);
+        validators.add(new FieldValidator(o));
+      }
+    }
+    return validators;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    FieldValidator that = (FieldValidator) o;
+
+    if (getValidation() != null ? !getValidation().equals(that.getValidation()) : that.getValidation() != null)
+      return false;
+    if (getInput() != null ? !getInput().equals(that.getInput()) : that.getInput() != null) return false;
+    return getConfig() != null ? getConfig().equals(that.getConfig()) : that.getConfig() == null;
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = getValidation() != null ? getValidation().hashCode() : 0;
+    result = 31 * result + (getInput() != null ? getInput().hashCode() : 0);
+    result = 31 * result + (getConfig() != null ? getConfig().hashCode() : 0);
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return "FieldValidator{" +
+            "validation=" + validation +
+            ", input=" + input +
+            ", config=" + config +
+            '}';
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/FieldValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/FieldValidation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/FieldValidation.java
new file mode 100644
index 0000000..442f2d6
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/FieldValidation.java
@@ -0,0 +1,29 @@
+/**
+ * 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.metron.common.field.validation;
+
+import java.util.Map;
+
+public interface FieldValidation {
+  boolean isValid( Map<String, Object> input
+                 , Map<String, Object> validationConfig
+                 , Map<String, Object> globalConfig
+                 );
+  void initialize(Map<String, Object> validationConfig, Map<String, Object> globalConfig);
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/FieldValidations.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/FieldValidations.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/FieldValidations.java
new file mode 100644
index 0000000..30a0f3a
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/FieldValidations.java
@@ -0,0 +1,54 @@
+/**
+ * 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.metron.common.field.validation;
+
+import org.apache.metron.common.field.validation.network.DomainValidation;
+import org.apache.metron.common.field.validation.network.EmailValidation;
+import org.apache.metron.common.field.validation.network.IPValidation;
+import org.apache.metron.common.field.validation.network.URLValidation;
+import org.apache.metron.common.field.validation.primitive.DateValidation;
+import org.apache.metron.common.field.validation.primitive.IntegerValidation;
+import org.apache.metron.common.field.validation.primitive.NotEmptyValidation;
+import org.apache.metron.common.field.validation.primitive.RegexValidation;
+import org.apache.metron.common.utils.ReflectionUtils;
+
+public enum FieldValidations {
+  MQL(new QueryValidation())
+  ,IP(new IPValidation())
+  ,DOMAIN(new DomainValidation())
+  ,EMAIL(new EmailValidation())
+  ,URL(new URLValidation())
+  ,DATE(new DateValidation())
+  ,INTEGER(new IntegerValidation())
+  ,REGEX_MATCH(new RegexValidation())
+  ,NOT_EMPTY(new NotEmptyValidation())
+  ;
+  private FieldValidation validation;
+  FieldValidations(FieldValidation validation) {
+    this.validation = validation;
+  }
+  public static FieldValidation get(String validation) {
+    try {
+      return FieldValidations.valueOf(validation).validation;
+    }
+    catch(Exception ex) {
+      return ReflectionUtils.createInstance(validation);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/QueryValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/QueryValidation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/QueryValidation.java
new file mode 100644
index 0000000..6c0ab86
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/QueryValidation.java
@@ -0,0 +1,72 @@
+/**
+ * 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.metron.common.field.validation;
+
+import org.apache.metron.common.query.MapVariableResolver;
+import org.apache.metron.common.query.PredicateProcessor;
+
+import java.util.Map;
+
+public class QueryValidation implements FieldValidation {
+
+  private enum Config {
+    CONDITION("condition")
+    ;
+    String key;
+    Config(String key) {
+      this.key = key;
+    }
+    public <T> T get(Map<String, Object> config, Class<T> clazz) {
+      Object o = config.get(key);
+      if(o == null) {
+        return null;
+      }
+      return clazz.cast(o);
+    }
+  }
+
+  @Override
+  public boolean isValid( Map<String, Object> input
+                        , Map<String, Object> validationConfig
+                        , Map<String, Object> globalConfig
+                        ) {
+    String condition = Config.CONDITION.get(validationConfig, String.class);
+    if(condition == null) {
+      return true;
+    }
+    else {
+      PredicateProcessor processor = new PredicateProcessor();
+      return processor.parse(condition, new MapVariableResolver(input));
+    }
+  }
+
+  @Override
+  public void initialize(Map<String, Object> validationConfig, Map<String, Object> globalConfig) {
+    String condition = Config.CONDITION.get(validationConfig, String.class);
+    if(condition == null) {
+      throw new IllegalStateException("You must specify a condition.");
+    }
+    try {
+      new PredicateProcessor().validate(condition);
+    }
+    catch(Exception e) {
+      throw new IllegalStateException("Invalid condition: " + condition, e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/SimpleValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/SimpleValidation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/SimpleValidation.java
new file mode 100644
index 0000000..c409253
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/SimpleValidation.java
@@ -0,0 +1,73 @@
+/**
+ * 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.metron.common.field.validation;
+
+import org.apache.metron.common.query.BooleanOp;
+
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.function.Predicate;
+
+public abstract class SimpleValidation implements FieldValidation, Predicate<List<String>> {
+  @Override
+  public boolean isValid( Map<String, Object> input
+                        , Map<String, Object> validationConfig
+                        , Map<String, Object> globalConfig
+                        )
+  {
+    Predicate<String> predicate = getPredicate();
+    if(isNonExistentOk()) {
+      for (Object o : input.values()) {
+        if (o != null && !predicate.test(o.toString())) {
+          return false;
+        }
+      }
+    }
+    else {
+      for (Object o : input.values()) {
+        if (o == null || !predicate.test(o.toString())) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public boolean test(List<String> input) {
+    Predicate<String> predicate = getPredicate();
+    for(String o : input) {
+      if(o == null || !predicate.test(o)){
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public void initialize(Map<String, Object> validationConfig, Map<String, Object> globalConfig) {
+
+  }
+
+  public abstract Predicate<String> getPredicate();
+  protected boolean isNonExistentOk() {
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/DomainValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/DomainValidation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/DomainValidation.java
new file mode 100644
index 0000000..5a9c121
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/DomainValidation.java
@@ -0,0 +1,34 @@
+/**
+ * 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.metron.common.field.validation.network;
+
+import org.apache.commons.validator.routines.DomainValidator;
+import org.apache.metron.common.field.validation.FieldValidation;
+import org.apache.metron.common.field.validation.SimpleValidation;
+
+import java.util.Map;
+import java.util.function.Predicate;
+
+public class DomainValidation extends SimpleValidation {
+
+  @Override
+  public Predicate<String> getPredicate() {
+    return domain -> DomainValidator.getInstance().isValid(domain);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/EmailValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/EmailValidation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/EmailValidation.java
new file mode 100644
index 0000000..f2a09bd
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/EmailValidation.java
@@ -0,0 +1,32 @@
+/**
+ * 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.metron.common.field.validation.network;
+
+import org.apache.commons.validator.routines.EmailValidator;
+import org.apache.metron.common.field.validation.SimpleValidation;
+
+import java.util.Map;
+import java.util.function.Predicate;
+
+public class EmailValidation extends SimpleValidation{
+  @Override
+  public Predicate<String> getPredicate() {
+    return email -> EmailValidator.getInstance().isValid(email);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/IPValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/IPValidation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/IPValidation.java
new file mode 100644
index 0000000..42b0aaf
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/IPValidation.java
@@ -0,0 +1,116 @@
+/**
+ * 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.metron.common.field.validation.network;
+
+import org.apache.commons.validator.routines.InetAddressValidator;
+import org.apache.metron.common.field.validation.FieldValidation;
+
+import java.util.List;
+import java.util.Map;
+import java.util.function.Predicate;
+
+public class IPValidation implements FieldValidation, Predicate<List<String>> {
+
+
+  private enum IPType {
+     IPV4(ip -> InetAddressValidator.getInstance().isValidInet4Address(ip))
+    ,IPV6(ip -> InetAddressValidator.getInstance().isValidInet6Address(ip))
+    , DEFAULT(ip -> InetAddressValidator.getInstance().isValid(ip));
+    Predicate<String> validationPredicate;
+    IPType(Predicate<String> validationPredicate) {
+      this.validationPredicate = validationPredicate;
+    }
+    public boolean isValid(String ip) {
+      return validationPredicate.test(ip);
+    }
+    public static IPType get(String type) {
+      if(type == null) {
+        return DEFAULT;
+      }
+      else {
+        try {
+          return IPType.valueOf(type);
+        }
+        catch(Exception e) {
+          return DEFAULT;
+        }
+      }
+    }
+  }
+  private enum Config {
+    TYPE("type")
+    ;
+    String key;
+    Config(String key) {
+      this.key = key;
+    }
+    public <T> T get(Map<String, Object> config, Class<T> clazz) {
+      Object o = config.get(key);
+      if(o == null) {
+        return null;
+      }
+      return clazz.cast(o);
+    }
+  }
+
+  /**
+   * Evaluates this predicate on the given argument.
+   *
+   * @param strings the input argument
+   * @return {@code true} if the input argument matches the predicate,
+   * otherwise {@code false}
+   */
+  @Override
+  public boolean test(List<String> strings) {
+    IPType type = IPType.DEFAULT;
+    if(strings.isEmpty()) {
+      return false;
+    }
+    String ip = strings.get(0);
+    if(ip == null) {
+      return false;
+    }
+    if(strings.size() >= 2) {
+      try {
+        type = IPType.get(strings.get(1));
+      }
+      catch(Exception e) {
+        type = IPType.DEFAULT;
+      }
+    }
+    return type.isValid(ip);
+  }
+  @Override
+  public boolean isValid( Map<String, Object> input
+                        , Map<String, Object> validationConfig
+                        , Map<String, Object> globalConfig
+                        ) {
+    IPType type = IPType.get(Config.TYPE.get(validationConfig, String.class));
+    for(Object o : input.values()) {
+      if(o != null && !type.isValid(o.toString())) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public void initialize(Map<String, Object> validationConfig, Map<String, Object> globalConfig) {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/URLValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/URLValidation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/URLValidation.java
new file mode 100644
index 0000000..4a044b0
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/network/URLValidation.java
@@ -0,0 +1,31 @@
+/**
+ * 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.metron.common.field.validation.network;
+
+import org.apache.commons.validator.routines.UrlValidator;
+import org.apache.metron.common.field.validation.SimpleValidation;
+
+import java.util.function.Predicate;
+
+public class URLValidation extends SimpleValidation {
+  @Override
+  public Predicate<String> getPredicate() {
+    return url -> UrlValidator.getInstance().isValid(url);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/DateValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/DateValidation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/DateValidation.java
new file mode 100644
index 0000000..3a28462
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/DateValidation.java
@@ -0,0 +1,117 @@
+/**
+ * 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.metron.common.field.validation.primitive;
+
+import org.apache.metron.common.field.validation.FieldValidation;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Predicate;
+
+public class DateValidation implements FieldValidation, Predicate<List<String>> {
+
+  /**
+   * Evaluates this predicate on the given argument.
+   *
+   * @param strings the input argument
+   * @return {@code true} if the input argument matches the predicate,
+   * otherwise {@code false}
+   */
+  @Override
+  public boolean test(List<String> strings) {
+    if(strings.isEmpty()) {
+      return false;
+    }
+    if(strings.size() >= 2) {
+      String date = strings.get(0);
+      String format = strings.get(1);
+      if(date == null || format == null) {
+        return false;
+      }
+      try {
+        SimpleDateFormat sdf = new SimpleDateFormat(format);
+        sdf.parse(date);
+        return true;
+      }
+      catch(ParseException pe) {
+        return false;
+      }
+    }
+    else {
+      return false;
+    }
+  }
+
+  private enum Config {
+    FORMAT("format")
+    ;
+    String key;
+    Config(String key) {
+      this.key = key;
+    }
+    public <T> T get(Map<String, Object> config, Class<T> clazz) {
+      Object o = config.get(key);
+      if(o == null) {
+        return null;
+      }
+      return clazz.cast(o);
+    }
+  }
+  @Override
+  public boolean isValid( Map<String, Object> input
+                        , Map<String, Object> validationConfig
+                        , Map<String, Object> globalConfig
+                        )
+  {
+    String format = Config.FORMAT.get(validationConfig, String.class);
+    if(format == null) {
+      return false;
+    }
+    SimpleDateFormat sdf = new SimpleDateFormat(format);
+    for(Object o : input.values()) {
+      if(o == null) {
+        return true;
+      }
+      try {
+        Date d = sdf.parse(o.toString());
+      } catch (ParseException e) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public void initialize(Map<String, Object> validationConfig, Map<String, Object> globalConfig) {
+    String format = Config.FORMAT.get(validationConfig, String.class);
+    if(format == null) {
+      throw new IllegalStateException("You must specify '" + Config.FORMAT.key + "' in the config");
+    }
+    SimpleDateFormat sdf = new SimpleDateFormat(format);
+    try {
+      sdf.format(new Date());
+    }
+    catch(Exception e) {
+      throw new IllegalStateException("Invalid date format: " + format, e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/IntegerValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/IntegerValidation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/IntegerValidation.java
new file mode 100644
index 0000000..320d56d
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/IntegerValidation.java
@@ -0,0 +1,32 @@
+/**
+ * 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.metron.common.field.validation.primitive;
+
+import org.apache.commons.validator.routines.DoubleValidator;
+import org.apache.commons.validator.routines.LongValidator;
+import org.apache.metron.common.field.validation.SimpleValidation;
+
+import java.util.function.Predicate;
+
+public class IntegerValidation extends SimpleValidation{
+  @Override
+  public Predicate<String> getPredicate() {
+    return x -> LongValidator.getInstance().isValid(x);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/NotEmptyValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/NotEmptyValidation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/NotEmptyValidation.java
new file mode 100644
index 0000000..98d2269
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/NotEmptyValidation.java
@@ -0,0 +1,35 @@
+/**
+ * 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.metron.common.field.validation.primitive;
+
+import org.apache.metron.common.field.validation.SimpleValidation;
+
+import java.util.function.Predicate;
+
+public class NotEmptyValidation extends SimpleValidation {
+  @Override
+  public Predicate<String> getPredicate() {
+    return s -> !(s == null || s.trim().isEmpty());
+  }
+
+  @Override
+  protected boolean isNonExistentOk() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/RegexValidation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/RegexValidation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/RegexValidation.java
new file mode 100644
index 0000000..248eaff
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/validation/primitive/RegexValidation.java
@@ -0,0 +1,68 @@
+/**
+ * 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.metron.common.field.validation.primitive;
+
+import org.apache.metron.common.field.validation.FieldValidation;
+
+import java.util.Map;
+
+public class RegexValidation implements FieldValidation {
+
+  private enum Config {
+    REGEX("pattern")
+    ;
+    String key;
+    Config(String key) {
+      this.key = key;
+    }
+    public <T> T get(Map<String, Object> config, Class<T> clazz) {
+      Object o = config.get(key);
+      if(o == null) {
+        return null;
+      }
+      return clazz.cast(o);
+    }
+  }
+
+  @Override
+  public boolean isValid( Map<String, Object> input
+                        , Map<String, Object> validationConfig
+                        , Map<String, Object> globalConfig
+                        ) {
+
+    String regex = Config.REGEX.get(validationConfig, String.class);
+    if(regex == null) {
+      return false;
+    }
+    for(Object o : input.values()) {
+      if(o != null && !o.toString().matches(regex)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public void initialize(Map<String, Object> validationConfig, Map<String, Object> globalConfig) {
+    String regex = Config.REGEX.get(validationConfig, String.class);
+    if(regex == null) {
+      throw new IllegalStateException("You must specify '" + Config.REGEX.key + "' in the config");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/LogicalFunctions.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/LogicalFunctions.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/LogicalFunctions.java
index 77aa173..bb35f04 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/LogicalFunctions.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/LogicalFunctions.java
@@ -18,13 +18,19 @@
 
 package org.apache.metron.common.query;
 
-import com.google.common.base.Function;
 import org.apache.commons.net.util.SubnetUtils;
+import org.apache.metron.common.field.validation.network.DomainValidation;
+import org.apache.metron.common.field.validation.network.EmailValidation;
+import org.apache.metron.common.field.validation.network.IPValidation;
+import org.apache.metron.common.field.validation.network.URLValidation;
+import org.apache.metron.common.field.validation.primitive.DateValidation;
+import org.apache.metron.common.field.validation.primitive.IntegerValidation;
 
 import javax.annotation.Nullable;
 import java.util.List;
+import java.util.function.Predicate;
 
-public enum LogicalFunctions implements Function<List<String>, Boolean> {
+public enum LogicalFunctions implements Predicate<List<String>> {
   IS_EMPTY ( list -> {
     if(list.size() == 0) {
       throw new IllegalStateException("IS_EMPTY expects one string arg");
@@ -86,14 +92,20 @@ public enum LogicalFunctions implements Function<List<String>, Boolean> {
     }
     return str.matches(pattern);
   })
+  , IS_IP(new IPValidation())
+  , IS_DOMAIN(new DomainValidation())
+  , IS_EMAIL(new EmailValidation())
+  , IS_URL(new URLValidation())
+  , IS_DATE(new DateValidation())
+  , IS_INTEGER(new IntegerValidation())
   ;
-  Function<List<String>, Boolean> func;
-  LogicalFunctions(Function<List<String>, Boolean> func) {
+  Predicate<List<String>> func;
+  LogicalFunctions(Predicate<List<String>> func) {
     this.func = func;
   }
   @Nullable
   @Override
-  public Boolean apply(@Nullable List<String> input) {
-    return func.apply(input);
+  public boolean test(@Nullable List<String> input) {
+    return func.test(input);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/PredicateProcessor.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/PredicateProcessor.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/PredicateProcessor.java
index 26e4da8..95b943a 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/PredicateProcessor.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/PredicateProcessor.java
@@ -18,16 +18,23 @@
 
 package org.apache.metron.common.query;
 
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
 import org.antlr.v4.runtime.ANTLRInputStream;
 import org.antlr.v4.runtime.CommonTokenStream;
 import org.antlr.v4.runtime.TokenStream;
 import org.apache.metron.common.query.generated.PredicateLexer;
 import org.apache.metron.common.query.generated.PredicateParser;
 
+import java.util.Map;
+
 import static org.apache.commons.lang3.StringUtils.isEmpty;
 
 
 public class PredicateProcessor {
+
   public boolean parse(String rule, VariableResolver resolver) {
     if (rule == null || isEmpty(rule.trim())) {
       return true;
@@ -38,11 +45,11 @@ public class PredicateProcessor {
     lexer.addErrorListener(new ErrorListener());
     TokenStream tokens = new CommonTokenStream(lexer);
     PredicateParser parser = new PredicateParser(tokens);
-
-    QueryCompiler treeBuilder = new QueryCompiler(resolver);
-    parser.addParseListener(treeBuilder);
     parser.removeErrorListeners();
     parser.addErrorListener(new ErrorListener());
+    QueryCompiler treeBuilder = new QueryCompiler(resolver);
+    parser.removeParseListeners();
+    parser.addParseListener(treeBuilder);
     parser.single_rule();
     return treeBuilder.getResult();
   }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/QueryCompiler.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/QueryCompiler.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/QueryCompiler.java
index 6351eed..3d85d9e 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/QueryCompiler.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/query/QueryCompiler.java
@@ -24,6 +24,7 @@ import org.apache.metron.common.query.generated.PredicateBaseListener;
 import org.apache.metron.common.query.generated.PredicateParser;
 
 import java.util.*;
+import java.util.function.Predicate;
 
 class QueryCompiler extends PredicateBaseListener {
   private VariableResolver resolver = null;
@@ -212,7 +213,7 @@ class QueryCompiler extends PredicateBaseListener {
   @Override
   public void exitLogicalFunc(PredicateParser.LogicalFuncContext ctx) {
     String funcName = ctx.getChild(0).getText();
-    Function<List<String>, Boolean> func;
+    Predicate<List<String>> func;
     try {
       func = LogicalFunctions.valueOf(funcName);
     }
@@ -229,7 +230,7 @@ class QueryCompiler extends PredicateBaseListener {
     else {
       throw new ParseException("Unable to process in clause because " + left.getValue() + " is not a set");
     }
-    Boolean result = func.apply(argList);
+    Boolean result = func.test(argList);
     tokenStack.push(new PredicateToken<>(result, Boolean.class));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/BaseValidationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/BaseValidationTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/BaseValidationTest.java
new file mode 100644
index 0000000..ab78c3c
--- /dev/null
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/BaseValidationTest.java
@@ -0,0 +1,48 @@
+/**
+ * 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.metron.common.field.validation;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.metron.common.configuration.Configurations;
+import org.apache.metron.common.configuration.FieldValidator;
+import org.json.simple.JSONObject;
+
+import java.io.IOException;
+import java.util.Map;
+
+public class BaseValidationTest {
+  public Configurations getConfiguration(String config) throws IOException {
+    Configurations configurations = new Configurations();
+    configurations.updateGlobalConfig(Bytes.toBytes(config));
+    return configurations;
+  }
+
+  public FieldValidator getValidator(Configurations configurations) throws IOException {
+    return configurations.getFieldValidations().get(0);
+  }
+
+  public boolean execute(String config, Map<String, Object> input) throws IOException {
+    Configurations configurations = getConfiguration(config);
+
+    FieldValidator validator = getValidator(configurations);
+    return validator.isValid(new JSONObject(input)
+                                       ,configurations.getGlobalConfig()
+                            );
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/QueryValidationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/QueryValidationTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/QueryValidationTest.java
new file mode 100644
index 0000000..e658fc6
--- /dev/null
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/QueryValidationTest.java
@@ -0,0 +1,98 @@
+/**
+ * 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.metron.common.field.validation;
+
+import com.google.common.collect.ImmutableMap;
+import org.adrianwalker.multilinestring.Multiline;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.metron.common.configuration.Configurations;
+import org.apache.metron.common.configuration.FieldValidator;
+import org.json.simple.JSONObject;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+
+public class QueryValidationTest extends BaseValidationTest{
+  /**
+   {
+    "fieldValidations" : [
+          {
+           "validation" : "MQL"
+          ,"config" : {
+                "condition" : "exists(field1)"
+                      }
+          }
+                         ]
+   }
+   */
+  @Multiline
+  public static String validQueryConfig;
+
+  /**
+   {
+    "fieldValidations" : [
+          {
+           "validation" : "MQL"
+          ,"config" : {
+                      }
+          }
+                         ]
+   }
+   */
+  @Multiline
+  public static String invalidQueryConfig1;
+
+  /**
+   {
+    "fieldValidations" : [
+          {
+           "validation" : "MQL"
+          ,"config" : {
+              "condition" : "exi"
+                      }
+          }
+                         ]
+   }
+   */
+  @Multiline
+  public static String invalidQueryConfig2;
+  @Test
+  public void testPositive() throws IOException {
+    Assert.assertTrue(execute(validQueryConfig, ImmutableMap.of("field1", "foo")));
+  }
+
+  @Test
+  public void testNegative() throws IOException {
+    Assert.assertFalse(execute(validQueryConfig, ImmutableMap.of("field2", "foo")));
+  }
+
+  @Test(expected=IllegalStateException.class)
+  public void testInvalidConfig_missingConfig() throws IOException {
+    getConfiguration(invalidQueryConfig1);
+  }
+
+  @Test(expected=IllegalStateException.class)
+  public void testInvalidConfig_invalidQuery() throws IOException {
+    getConfiguration(invalidQueryConfig2);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/ValidationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/ValidationTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/ValidationTest.java
new file mode 100644
index 0000000..da2e6aa
--- /dev/null
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/ValidationTest.java
@@ -0,0 +1,89 @@
+/**
+ * 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.metron.common.field.validation;
+
+import com.google.common.collect.ImmutableList;
+import org.adrianwalker.multilinestring.Multiline;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.metron.common.configuration.Configurations;
+import org.apache.metron.common.configuration.FieldValidator;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+public class ValidationTest extends BaseValidationTest{
+  /**
+   {
+    "fieldValidations" : [
+            {
+              "input" : "field1"
+             ,"validation" : "NOT_EMPTY"
+            }
+                         ]
+   }
+   */
+  @Multiline
+  public static String validValidationConfigWithStringInput;
+
+  /**
+   {
+    "fieldValidations" : [
+            {
+              "input" : [ "field1", "field2" ]
+             ,"validation" : "NOT_EMPTY"
+            }
+                         ]
+   }
+   */
+  @Multiline
+  public static String validValidationConfigWithListInput;
+  /**
+   {
+    "fieldValidations" : [
+            {
+              "input" : "field1"
+            }
+                         ]
+   }
+   */
+  @Multiline
+  public static String invalidValidationConfig;
+  @Test
+  public void testValidConfiguration() throws IOException {
+    {
+      Configurations configurations = getConfiguration(validValidationConfigWithStringInput);
+      Assert.assertNotNull(configurations.getFieldValidations());
+      Assert.assertEquals(1, configurations.getFieldValidations().size());
+      Assert.assertEquals(ImmutableList.of("field1"), configurations.getFieldValidations().get(0).getInput());
+    }
+    {
+      Configurations configurations = getConfiguration(validValidationConfigWithListInput);
+      Assert.assertNotNull(configurations.getFieldValidations());
+      Assert.assertEquals(1, configurations.getFieldValidations().size());
+      Assert.assertEquals(ImmutableList.of("field1", "field2"), configurations.getFieldValidations().get(0).getInput());
+    }
+  }
+
+  @Test(expected = IllegalStateException.class)
+  public void testInvalidConfiguration() throws IOException {
+    getConfiguration(invalidValidationConfig);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/network/DomainValidationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/network/DomainValidationTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/network/DomainValidationTest.java
new file mode 100644
index 0000000..de3cca5
--- /dev/null
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/network/DomainValidationTest.java
@@ -0,0 +1,89 @@
+/**
+ * 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.metron.common.field.validation.network;
+
+import com.google.common.collect.ImmutableMap;
+import org.adrianwalker.multilinestring.Multiline;
+import org.apache.metron.common.field.validation.BaseValidationTest;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.metron.common.query.QueryParserTest.run;
+
+public class DomainValidationTest extends BaseValidationTest{
+  /**
+   {
+    "fieldValidations" : [
+            {
+              "input" : "field1"
+             ,"validation" : "DOMAIN"
+            }
+                         ]
+   }
+   */
+  @Multiline
+  public static String validWithSingleField;
+  public static String validWithSingleField_MQL = "IS_DOMAIN(field1)";
+
+  /**
+   {
+    "fieldValidations" : [
+            {
+              "input" : [ "field1", "field2" ]
+             ,"validation" : "DOMAIN"
+            }
+                         ]
+   }
+   */
+  @Multiline
+  public static String validWithMultipleFields;
+  public static String validWithMultipleFields_MQL = "IS_DOMAIN(field1) and IS_DOMAIN(field2)";
+
+  @Test
+  public void positiveTest_single() throws IOException {
+    Assert.assertTrue(execute(validWithSingleField, ImmutableMap.of("field1", "caseystella.com")));
+    Assert.assertTrue(run(validWithSingleField_MQL, ImmutableMap.of("field1", "caseystella.com")));
+    Assert.assertTrue(execute(validWithSingleField, ImmutableMap.of("field1", "www.hotmail.co.uk")));
+    Assert.assertTrue(run(validWithSingleField_MQL, ImmutableMap.of("field1", "www.hotmail.co.uk")));
+  }
+  @Test
+  public void negativeTest_single() throws IOException {
+    Assert.assertFalse(execute(validWithSingleField, ImmutableMap.of("field1", "foo")));
+    Assert.assertFalse(run(validWithSingleField_MQL, ImmutableMap.of("field1", "foo")));
+    Assert.assertFalse(execute(validWithSingleField, ImmutableMap.of("field1", "caseystella.turtle")));
+    Assert.assertFalse(run(validWithSingleField_MQL, ImmutableMap.of("field1", "caseystella.turtle")));
+    Assert.assertFalse(execute(validWithSingleField, ImmutableMap.of("field1", 2.7f)));
+    Assert.assertFalse(run(validWithSingleField_MQL, ImmutableMap.of("field1", 2.7f)));
+  }
+  @Test
+  public void positiveTest_multiple() throws IOException {
+    Assert.assertTrue(execute(validWithMultipleFields, ImmutableMap.of("field1", "www.gmail.com", "field2", "www.hotmail.com")));
+    Assert.assertTrue(run(validWithMultipleFields_MQL, ImmutableMap.of("field1", "www.gmail.com", "field2", "www.hotmail.com")));
+  }
+
+  @Test
+  public void negativeTest_multiple() throws IOException {
+    Assert.assertTrue(execute(validWithMultipleFields, ImmutableMap.of("field2", "hotmail.edu")));
+    Assert.assertFalse(run(validWithMultipleFields_MQL, ImmutableMap.of("field2", "hotmail.edu")));
+    Assert.assertFalse(execute(validWithMultipleFields, ImmutableMap.of("field1", "", "field2", "gmail.com")));
+    Assert.assertFalse(run(validWithMultipleFields_MQL, ImmutableMap.of("field1", "", "field2", "gmail.com")));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/network/EmailValidationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/network/EmailValidationTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/network/EmailValidationTest.java
new file mode 100644
index 0000000..22bdde8
--- /dev/null
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/network/EmailValidationTest.java
@@ -0,0 +1,91 @@
+/**
+ * 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.metron.common.field.validation.network;
+
+import com.google.common.collect.ImmutableMap;
+import org.adrianwalker.multilinestring.Multiline;
+import org.apache.metron.common.field.validation.BaseValidationTest;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.metron.common.query.QueryParserTest.run;
+
+public class EmailValidationTest extends BaseValidationTest {
+  /**
+   {
+    "fieldValidations" : [
+            {
+              "input" : "field1"
+             ,"validation" : "EMAIL"
+            }
+                         ]
+   }
+   */
+  @Multiline
+  public static String validWithSingleField;
+  public static String validWithSingleField_MQL = "IS_EMAIL(field1)";
+
+  /**
+   {
+    "fieldValidations" : [
+            {
+              "input" : [ "field1", "field2" ]
+             ,"validation" : "EMAIL"
+            }
+                         ]
+   }
+   */
+  @Multiline
+  public static String validWithMultipleFields;
+  public static String validWithMultipleFields_MQL = "IS_EMAIL(field1) and IS_EMAIL(field2)";
+
+  @Test
+  public void positiveTest_single() throws IOException {
+    Assert.assertTrue(execute(validWithSingleField, ImmutableMap.of("field1", "me@caseystella.com")));
+    Assert.assertTrue(run(validWithSingleField_MQL, ImmutableMap.of("field1", "me@caseystella.com")));
+    Assert.assertTrue(execute(validWithSingleField, ImmutableMap.of("field1", "me@www.hotmail.co.uk")));
+    Assert.assertTrue(run(validWithSingleField_MQL, ImmutableMap.of("field1", "me@www.hotmail.co.uk")));
+  }
+  @Test
+  public void negativeTest_single() throws IOException {
+    Assert.assertFalse(execute(validWithSingleField, ImmutableMap.of("field1", "me@foo")));
+    Assert.assertFalse(run(validWithSingleField_MQL, ImmutableMap.of("field1", "me@foo")));
+    Assert.assertFalse(execute(validWithSingleField, ImmutableMap.of("field1", "caseystella.turtle")));
+    Assert.assertFalse(run(validWithSingleField_MQL, ImmutableMap.of("field1", "caseystella.turtle")));
+    Assert.assertFalse(execute(validWithSingleField, ImmutableMap.of("field1", "caseystella.com")));
+    Assert.assertFalse(run(validWithSingleField_MQL, ImmutableMap.of("field1", "caseystella.com")));
+    Assert.assertFalse(execute(validWithSingleField, ImmutableMap.of("field1", 2.7f)));
+    Assert.assertFalse(run(validWithSingleField_MQL, ImmutableMap.of("field1", 2.7f)));
+  }
+  @Test
+  public void positiveTest_multiple() throws IOException {
+    Assert.assertTrue(execute(validWithMultipleFields, ImmutableMap.of("field1", "me@www.gmail.com", "field2", "me@www.hotmail.com")));
+    Assert.assertTrue(run(validWithMultipleFields_MQL, ImmutableMap.of("field1", "me@www.gmail.com", "field2", "me@www.hotmail.com")));
+  }
+
+  @Test
+  public void negativeTest_multiple() throws IOException {
+    Assert.assertTrue(execute(validWithMultipleFields, ImmutableMap.of("field2", "me@hotmail.edu")));
+    Assert.assertFalse(run(validWithMultipleFields_MQL, ImmutableMap.of("field2", "me@hotmail.edu")));
+    Assert.assertFalse(execute(validWithMultipleFields, ImmutableMap.of("field1", "", "field2", "me@gmail.com")));
+    Assert.assertFalse(run(validWithMultipleFields_MQL, ImmutableMap.of("field1", "", "field2", "me@gmail.com")));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/network/IPValidationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/network/IPValidationTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/network/IPValidationTest.java
new file mode 100644
index 0000000..c0a043a
--- /dev/null
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/network/IPValidationTest.java
@@ -0,0 +1,85 @@
+/**
+ * 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.metron.common.field.validation.network;
+
+import com.google.common.collect.ImmutableMap;
+import org.adrianwalker.multilinestring.Multiline;
+import org.apache.metron.common.field.validation.BaseValidationTest;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.metron.common.query.QueryParserTest.run;
+
+public class IPValidationTest extends BaseValidationTest {
+  /**
+   {
+    "fieldValidations" : [
+            {
+              "input" : "field1"
+             ,"validation" : "IP"
+            }
+                         ]
+   }
+   */
+  @Multiline
+  public static String validWithSingleField;
+  public static String validWithSingleField_MQL = "IS_IP(field1)";
+  /**
+   {
+    "fieldValidations" : [
+            {
+              "input" : [ "field1", "field2" ]
+             ,"validation" : "IP"
+             ,"config" : {
+                  "type" : "IPV4"
+                         }
+            }
+                         ]
+   }
+   */
+  @Multiline
+  public static String validWithMultipleFields;
+  public static String validWithMultipleFields_MQL = "IS_IP(field1, 'IPV4') && IS_IP(field2, 'IPV4')";
+
+  @Test
+  public void positiveTest_single() throws IOException {
+    Assert.assertTrue(execute(validWithSingleField, ImmutableMap.of("field1", "127.0.0.1")));
+    Assert.assertTrue(run(validWithSingleField_MQL, ImmutableMap.of("field1", "127.0.0.1")));
+  }
+  @Test
+  public void negativeTest_single() throws IOException {
+    Assert.assertFalse(execute(validWithSingleField, ImmutableMap.of("field1", "2014/05/01")));
+    Assert.assertFalse(run(validWithSingleField_MQL, ImmutableMap.of("field1", "2014/05/01")));
+    Assert.assertFalse(execute(validWithSingleField, ImmutableMap.of("field1", 2.3f)));
+    Assert.assertFalse(run(validWithSingleField_MQL, ImmutableMap.of("field1", 2.3f)));
+  }
+  @Test
+  public void positiveTest_multiple() throws IOException {
+    Assert.assertTrue(execute(validWithMultipleFields, ImmutableMap.of("field1", "192.168.0.1", "field2", "127.0.0.2")));
+    Assert.assertTrue(run(validWithMultipleFields_MQL, ImmutableMap.of("field1", "192.168.0.1", "field2", "127.0.0.2")));
+  }
+
+  @Test
+  public void negativeTest_multiple() throws IOException {
+    Assert.assertFalse(execute(validWithMultipleFields, ImmutableMap.of("field1", 1, "field2", "192.168.1")));
+    Assert.assertFalse(run(validWithMultipleFields_MQL, ImmutableMap.of("field1", 1, "field2", "192.168.1")));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/network/URLValidationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/network/URLValidationTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/network/URLValidationTest.java
new file mode 100644
index 0000000..4eb5eee
--- /dev/null
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/network/URLValidationTest.java
@@ -0,0 +1,91 @@
+/**
+ * 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.metron.common.field.validation.network;
+
+import com.google.common.collect.ImmutableMap;
+import org.adrianwalker.multilinestring.Multiline;
+import org.apache.metron.common.field.validation.BaseValidationTest;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.metron.common.query.QueryParserTest.run;
+
+public class URLValidationTest  extends BaseValidationTest {
+  /**
+   {
+    "fieldValidations" : [
+            {
+              "input" : "field1"
+             ,"validation" : "URL"
+            }
+                         ]
+   }
+   */
+  @Multiline
+  public static String validWithSingleField;
+  public static String validWithSingleField_MQL = "IS_URL(field1)";
+
+  /**
+   {
+    "fieldValidations" : [
+            {
+              "input" : [ "field1", "field2" ]
+             ,"validation" : "URL"
+            }
+                         ]
+   }
+   */
+  @Multiline
+  public static String validWithMultipleFields;
+  public static String validWithMultipleFields_MQL = "IS_URL(field1) and IS_URL(field2)";
+
+  @Test
+  public void positiveTest_single() throws IOException {
+    Assert.assertTrue(execute(validWithSingleField, ImmutableMap.of("field1", "http://caseystella.com/foo")));
+    Assert.assertTrue(run(validWithSingleField_MQL, ImmutableMap.of("field1", "http://caseystella.com/foo")));
+    Assert.assertTrue(execute(validWithSingleField, ImmutableMap.of("field1", "https://www.hotmail.co.uk")));
+    Assert.assertTrue(run(validWithSingleField_MQL, ImmutableMap.of("field1", "https://www.hotmail.co.uk")));
+  }
+
+
+  @Test
+  public void negativeTest_single() throws IOException {
+    Assert.assertFalse(execute(validWithSingleField, ImmutableMap.of("field1", "foo")));
+    Assert.assertFalse(run(validWithSingleField_MQL, ImmutableMap.of("field1", "foo")));
+    Assert.assertFalse(execute(validWithSingleField, ImmutableMap.of("field1", "http://caseystella.turtle")));
+    Assert.assertFalse(run(validWithSingleField_MQL, ImmutableMap.of("field1", "http://caseystella.turtle")));
+    Assert.assertFalse(execute(validWithSingleField, ImmutableMap.of("field1", 2.7f)));
+    Assert.assertFalse(run(validWithSingleField_MQL, ImmutableMap.of("field1", 2.7f)));
+  }
+  @Test
+  public void positiveTest_multiple() throws IOException {
+    Assert.assertTrue(execute(validWithMultipleFields, ImmutableMap.of("field1", "ftp://www.gmail.com", "field2", "http://www.hotmail.com")));
+    Assert.assertTrue(run(validWithMultipleFields_MQL, ImmutableMap.of("field1", "ftp://www.gmail.com", "field2", "http://www.hotmail.com")));
+  }
+
+  @Test
+  public void negativeTest_multiple() throws IOException {
+    Assert.assertTrue(execute(validWithMultipleFields, ImmutableMap.of("field2", "http://hotmail.edu")));
+    Assert.assertFalse(run(validWithMultipleFields_MQL, ImmutableMap.of("field2", "http://hotmail.edu")));
+    Assert.assertFalse(execute(validWithMultipleFields, ImmutableMap.of("field1", "", "field2", "http://gmail.com")));
+    Assert.assertFalse(run(validWithMultipleFields_MQL, ImmutableMap.of("field1", "", "field2", "http://gmail.com")));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/025f64c4/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/primitive/DateValidationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/primitive/DateValidationTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/primitive/DateValidationTest.java
new file mode 100644
index 0000000..60808a3
--- /dev/null
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/validation/primitive/DateValidationTest.java
@@ -0,0 +1,95 @@
+/**
+ * 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.metron.common.field.validation.primitive;
+
+import com.google.common.collect.ImmutableMap;
+import org.adrianwalker.multilinestring.Multiline;
+import org.apache.metron.common.field.validation.BaseValidationTest;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.metron.common.query.QueryParserTest.run;
+
+public class DateValidationTest extends BaseValidationTest{
+  /**
+   {
+    "fieldValidations" : [
+            {
+              "input" : "field1"
+             ,"validation" : "DATE"
+             ,"config" : {
+                  "format" : "yyyy-MM-dd"
+                         }
+            }
+                         ]
+   }
+   */
+
+  @Multiline
+  public static String validWithSingleField;
+  public static String validWithSingleField_MQL = "IS_DATE(field1, 'yyyy-MM-dd')";
+
+  /**
+   {
+    "fieldValidations" : [
+            {
+              "input" : [ "field1", "field2" ]
+             ,"validation" : "DATE"
+             ,"config" : {
+                  "format" : "yyyy-MM-dd"
+                         }
+            }
+                         ]
+   }
+   */
+  @Multiline
+  public static String validWithMultipleFields;
+  public static String validWithMultipleFields_MQL = "IS_DATE(field1, 'yyyy-MM-dd') && IS_DATE(field2, 'yyyy-MM-dd')";
+
+  @Test
+  public void positiveTest_single() throws IOException {
+    Assert.assertTrue(execute(validWithSingleField, ImmutableMap.of("field1", "2014-05-01")));
+    Assert.assertTrue(run(validWithSingleField_MQL, ImmutableMap.of("field1", "2014-05-01")));
+  }
+  @Test
+  public void negativeTest_single() throws IOException {
+    Assert.assertFalse(execute(validWithSingleField, ImmutableMap.of("field1", "2014/05/01")));
+    Assert.assertFalse(run(validWithSingleField_MQL, ImmutableMap.of("field1", "2014/05/01")));
+    Assert.assertFalse(execute(validWithSingleField, ImmutableMap.of("field1", 2.3f)));
+    Assert.assertFalse(run(validWithSingleField_MQL, ImmutableMap.of("field1", 2.3f)));
+  }
+  @Test
+  public void positiveTest_multiple() throws IOException {
+    Assert.assertTrue(execute(validWithMultipleFields, ImmutableMap.of("field1", "2014-06-01", "field2", "2014-06-02")));
+    Assert.assertTrue(run(validWithMultipleFields_MQL, ImmutableMap.of("field1", "2014-06-01", "field2", "2014-06-02")));
+  }
+
+  @Test
+  public void negativeTest_multiple() throws IOException {
+
+    Assert.assertTrue(execute(validWithMultipleFields, ImmutableMap.of("field2", "2014-06-02")));
+    Assert.assertFalse(run(validWithMultipleFields_MQL, ImmutableMap.of("field2", "2014-06-02")));
+    Assert.assertFalse(execute(validWithMultipleFields, ImmutableMap.of("field1", 1, "field2", "2014-06-02")));
+    Assert.assertFalse(run(validWithMultipleFields_MQL, ImmutableMap.of("field1", 1, "field2", "2014-06-02")));
+    Assert.assertTrue(execute(validWithMultipleFields, ImmutableMap.of("field3", "2014-06-02")));
+    Assert.assertFalse(run(validWithMultipleFields_MQL, ImmutableMap.of("field3", "2014-06-02")));
+  }
+}