You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by ot...@apache.org on 2017/12/11 21:42:27 UTC

metron git commit: METRON-1341 Projection FieldTransformation (simonellistonball via ottobackwards) closes apache/metron#861

Repository: metron
Updated Branches:
  refs/heads/master 2e78df67c -> 610540ef0


METRON-1341 Projection FieldTransformation (simonellistonball via ottobackwards) closes apache/metron#861


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

Branch: refs/heads/master
Commit: 610540ef034b7bd2555acb347f5c9be20a485954
Parents: 2e78df6
Author: simonellistonball <si...@simonellistonball.com>
Authored: Mon Dec 11 16:09:40 2017 -0500
Committer: otto <ot...@apache.org>
Committed: Mon Dec 11 16:09:40 2017 -0500

----------------------------------------------------------------------
 .../transformation/FieldTransformations.java    |  1 +
 .../transformation/SelectTransformation.java    | 57 ++++++++++++
 .../SelectTransformationTest.java               | 98 ++++++++++++++++++++
 metron-platform/metron-parsers/README.md        | 17 ++++
 4 files changed, 173 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/metron/blob/610540ef/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/FieldTransformations.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/FieldTransformations.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/FieldTransformations.java
index 3565609..a905123 100644
--- a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/FieldTransformations.java
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/FieldTransformations.java
@@ -24,6 +24,7 @@ public enum FieldTransformations {
   IP_PROTOCOL(new IPProtocolTransformation())
   ,REMOVE(new RemoveTransformation())
   ,STELLAR(new StellarTransformation())
+  ,SELECT(new SelectTransformation())
   ;
   FieldTransformation mapping;
   FieldTransformations(FieldTransformation mapping) {

http://git-wip-us.apache.org/repos/asf/metron/blob/610540ef/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/SelectTransformation.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/SelectTransformation.java b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/SelectTransformation.java
new file mode 100644
index 0000000..6f0defd
--- /dev/null
+++ b/metron-platform/metron-common/src/main/java/org/apache/metron/common/field/transformation/SelectTransformation.java
@@ -0,0 +1,57 @@
+/**
+ * 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.transformation;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.metron.stellar.dsl.Context;
+
+public class SelectTransformation implements FieldTransformation {
+
+	private static final List<String> systemFields = Arrays.asList("timestamp", "original_string", "source.type");
+
+	@Override
+	public Map<String, Object> map(Map<String, Object> input, List<String> outputField,
+			LinkedHashMap<String, Object> fieldMappingConfig, Context context, Map<String, Object>... sensorConfig) {
+		// note that we have to set the value to null for any field not in the output
+		// list, since FieldTransformer will otherwise put them back again from the
+		// original output.
+
+		// note, this cannot be implemented with streams because HashMap merge guards
+		// against null values
+
+		HashMap<String, Object> output = new HashMap<String, Object>();
+		for (Entry<String, Object> e : input.entrySet()) {
+			if (outputField.contains(e.getKey())) {
+				output.put(e.getKey(), e.getValue());
+			} else {
+				if (!systemFields.contains(e.getKey())) {
+					output.put(e.getKey(), null);
+				}
+			}
+		}
+		return output;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/610540ef/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/transformation/SelectTransformationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/transformation/SelectTransformationTest.java b/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/transformation/SelectTransformationTest.java
new file mode 100644
index 0000000..9219c6b
--- /dev/null
+++ b/metron-platform/metron-common/src/test/java/org/apache/metron/common/field/transformation/SelectTransformationTest.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.transformation;
+
+import java.util.HashMap;
+
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.metron.common.configuration.FieldTransformer;
+import org.apache.metron.common.configuration.SensorParserConfig;
+import org.apache.metron.stellar.dsl.Context;
+import org.json.simple.JSONObject;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.collect.Iterables;
+
+public class SelectTransformationTest {
+	
+	public static String selectSingleFieldConfig = "{ \"fieldTransformations\" : [{\"output\": [\"field1\"] , \"transformation\": \"SELECT\" } ] }";
+
+	public static String selectMultiFieldConfig = "{ \"fieldTransformations\" : [{\"output\": [\"field1\", \"field2\"] , \"transformation\": \"SELECT\" } ] }";
+
+	@Test
+	public void testSingleFieldReturned() throws Exception {
+		SensorParserConfig sensorConfig = SensorParserConfig.fromBytes(Bytes.toBytes(selectSingleFieldConfig));
+		FieldTransformer handler = Iterables.getFirst(sensorConfig.getFieldTransformations(), null);
+		JSONObject input = new JSONObject(new HashMap<String, Object>() {
+			{
+				put("field1", "foo");
+				put("field2", "bar");
+			}
+		});
+		handler.transformAndUpdate(input, Context.EMPTY_CONTEXT());
+
+		Assert.assertTrue(input.containsKey("field1"));
+		Assert.assertFalse(input.containsKey("field2"));
+		Assert.assertEquals(1, input.size());
+	}
+
+	@Test
+	public void testMulitpleFieldReturned() throws Exception {
+		SensorParserConfig sensorConfig = SensorParserConfig.fromBytes(Bytes.toBytes(selectMultiFieldConfig));
+		FieldTransformer handler = Iterables.getFirst(sensorConfig.getFieldTransformations(), null);
+		JSONObject input = new JSONObject(new HashMap<String, Object>() {
+			{
+				put("field1", "foo");
+				put("field2", "bar");
+				put("field3", "bar2");
+			}
+		});
+		handler.transformAndUpdate(input, Context.EMPTY_CONTEXT());
+
+		Assert.assertTrue(input.containsKey("field1"));
+		Assert.assertTrue(input.containsKey("field2"));
+		Assert.assertFalse(input.containsKey("field3"));
+		Assert.assertEquals(2, input.size());
+	}
+	
+	@Test
+	public void testPreserveSystemFields() throws Exception { 
+		SensorParserConfig sensorConfig = SensorParserConfig.fromBytes(Bytes.toBytes(selectSingleFieldConfig));
+		FieldTransformer handler = Iterables.getFirst(sensorConfig.getFieldTransformations(), null);
+		JSONObject input = new JSONObject(new HashMap<String, Object>() {
+			{
+				put("timestamp", 12345);
+				put("original_string", "foo,bar");
+				put("source.type", "test");
+				put("field1", "foo");
+				put("field2", "bar");
+			}
+		});
+		handler.transformAndUpdate(input, Context.EMPTY_CONTEXT());
+		
+		Assert.assertTrue(input.containsKey("timestamp"));
+		Assert.assertTrue(input.containsKey("original_string"));
+		Assert.assertTrue(input.containsKey("source.type"));
+		Assert.assertTrue(input.containsKey("field1"));
+		Assert.assertFalse(input.containsKey("field2"));
+		Assert.assertEquals(4, input.size());
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/metron/blob/610540ef/metron-platform/metron-parsers/README.md
----------------------------------------------------------------------
diff --git a/metron-platform/metron-parsers/README.md b/metron-platform/metron-parsers/README.md
index 54dbef2..88dde44 100644
--- a/metron-platform/metron-parsers/README.md
+++ b/metron-platform/metron-parsers/README.md
@@ -216,6 +216,23 @@ whenever `field2` exists and whose corresponding equal to 'foo':
 }
 ```
 
+* `SELECT`: This transformation filters the fields in the message to include only the configured output fields, and drops any not explicitly included. 
+
+For example: 
+```
+{
+...
+    "fieldTransformations" : [
+          {
+            "output" : ["field1", "field2" ] 
+          , "transformation" : "SELECT"
+          }
+                      ]
+}
+```
+
+when applied to a message containing keys field1, field2 and field3, will only output the first two. It is also worth noting that two standard fields - timestamp and original_source - will always be passed along whether they are listed in output or not, since they are considered core required fields.
+
 * `IP_PROTOCOL` : This transformation maps IANA protocol numbers to consistent string representations.
 
 Consider the following sensor parser config to map the `protocol` field