You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@metron.apache.org by mm...@apache.org on 2016/10/11 14:43:55 UTC

incubator-metron git commit: METRON-432: Fix pcap field resolver to return object instead of string value (mmiklavc) closes apache/incubator-metron#298

Repository: incubator-metron
Updated Branches:
  refs/heads/master 070473f5f -> 9ac36dec9


METRON-432: Fix pcap field resolver to return object instead of string value (mmiklavc) closes apache/incubator-metron#298


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

Branch: refs/heads/master
Commit: 9ac36dec91a215d53b855225f0c813b24ce85b9d
Parents: 070473f
Author: mmiklavc <mi...@gmail.com>
Authored: Tue Oct 11 10:43:16 2016 -0400
Committer: Michael Miklavcic <mi...@gmail.com>
Committed: Tue Oct 11 10:43:16 2016 -0400

----------------------------------------------------------------------
 .../apache/metron/pcap/QueryPcapFilterTest.java | 12 ++--
 .../PcapTopologyIntegrationTest.java            | 64 +++++++++++++++++++-
 .../src/test/resources/log4j.properties         | 24 ++++++++
 .../metron/pcap/filter/PcapFieldResolver.java   | 10 +--
 .../pcap/filter/fixed/FixedPcapFilter.java      | 18 +++---
 5 files changed, 104 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9ac36dec/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/QueryPcapFilterTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/QueryPcapFilterTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/QueryPcapFilterTest.java
index f07dc48..f5e61fe 100644
--- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/QueryPcapFilterTest.java
+++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/QueryPcapFilterTest.java
@@ -54,7 +54,7 @@ public class QueryPcapFilterTest {
   @Test
   public void testTrivialEquality() throws Exception {
     Configuration config = new Configuration();
-    String query = "ip_src_addr == 'src_ip' and ip_src_port == '0' and ip_dst_addr == 'dst_ip' and ip_dst_port == '1'";
+    String query = "ip_src_addr == 'src_ip' and ip_src_port == 0 and ip_dst_addr == 'dst_ip' and ip_dst_port == 1";
     new QueryPcapFilter.Configurator().addToConfig(query, config);
     {
       PcapFilter filter = new QueryPcapFilter() {
@@ -76,7 +76,7 @@ public class QueryPcapFilterTest {
   @Test
   public void testMissingDstAddr() throws Exception {
     Configuration config = new Configuration();
-    String query = "ip_src_addr == 'src_ip' and ip_src_port == '0' and ip_dst_port == '1'";
+    String query = "ip_src_addr == 'src_ip' and ip_src_port == 0 and ip_dst_port == 1";
     new QueryPcapFilter.Configurator().addToConfig(query, config);
     {
       QueryPcapFilter filter = new QueryPcapFilter() {
@@ -99,7 +99,7 @@ public class QueryPcapFilterTest {
         @Override
         protected EnumMap<Constants.Fields, Object> packetToFields(PacketInfo pi) {
           return new EnumMap<Constants.Fields, Object>(Constants.Fields.class) {{
-            put(Constants.Fields.SRC_ADDR, "src_ip1");
+            put(Constants.Fields.SRC_ADDR, "src_ip_no_match");
             put(Constants.Fields.SRC_PORT, 0);
             put(Constants.Fields.DST_ADDR, "dst_ip");
             put(Constants.Fields.DST_PORT, 1);
@@ -114,7 +114,7 @@ public class QueryPcapFilterTest {
   @Test
   public void testMissingDstPort() throws Exception {
     Configuration config = new Configuration();
-    String query = "ip_src_addr == 'src_ip' and ip_src_port == '0' and ip_dst_addr == 'dst_ip'";
+    String query = "ip_src_addr == 'src_ip' and ip_src_port == 0 and ip_dst_addr == 'dst_ip'";
     new QueryPcapFilter.Configurator().addToConfig(query, config);
     {
       QueryPcapFilter filter = new QueryPcapFilter() {
@@ -168,7 +168,7 @@ public class QueryPcapFilterTest {
   @Test
   public void testMissingSrcAddr() throws Exception {
     Configuration config = new Configuration();
-    String query = "ip_src_port == '0' and ip_dst_addr == 'dst_ip' and ip_dst_port == '1'";
+    String query = "ip_src_port == 0 and ip_dst_addr == 'dst_ip' and ip_dst_port == 1";
     new QueryPcapFilter.Configurator().addToConfig(query, config);
     {
       QueryPcapFilter filter = new QueryPcapFilter() {
@@ -190,7 +190,7 @@ public class QueryPcapFilterTest {
   @Test
   public void testMissingSrcPort() throws Exception {
     Configuration config = new Configuration();
-    String query = "ip_src_addr == 'src_ip' and ip_dst_addr == 'dst_ip' and ip_dst_port == '1'";
+    String query = "ip_src_addr == 'src_ip' and ip_dst_addr == 'dst_ip' and ip_dst_port == 1";
     new QueryPcapFilter.Configurator().addToConfig(query, config);
     {
       QueryPcapFilter filter = new QueryPcapFilter() {

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9ac36dec/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java
index 0dd07aa..fcdb108 100644
--- a/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java
+++ b/metron-platform/metron-pcap-backend/src/test/java/org/apache/metron/pcap/integration/PcapTopologyIntegrationTest.java
@@ -443,7 +443,67 @@ public class PcapTopologyIntegrationTest {
                         , getTimestamp(0, pcapEntries)
                         , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1
                         , 10
-                        , "ip_dst_port == '22'"
+                        , "ip_dst_port == 22"
+                        , new Configuration()
+                        , FileSystem.get(new Configuration())
+                        , new QueryPcapFilter.Configurator()
+                );
+        assertInOrder(results);
+        Assert.assertTrue(Iterables.size(results) > 0);
+        Assert.assertEquals(Iterables.size(results)
+                , Iterables.size(filterPcaps(pcapEntries, new Predicate<JSONObject>() {
+                          @Override
+                          public boolean apply(@Nullable JSONObject input) {
+                            Object prt = input.get(Constants.Fields.DST_PORT.getName());
+                            return prt != null && (Long) prt == 22;
+                          }
+                        }, withHeaders)
+                )
+        );
+        assertInOrder(results);
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        PcapMerger.merge(baos, Iterables.partition(results, 1).iterator().next());
+        Assert.assertTrue(baos.toByteArray().length > 0);
+      }
+      {
+        //test with query filter
+        Iterable<byte[]> results =
+                job.query(new Path(outDir.getAbsolutePath())
+                        , new Path(queryDir.getAbsolutePath())
+                        , getTimestamp(0, pcapEntries)
+                        , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1
+                        , 10
+                        , "ip_dst_port > 20 and ip_dst_port < 55792"
+                        , new Configuration()
+                        , FileSystem.get(new Configuration())
+                        , new QueryPcapFilter.Configurator()
+                );
+        assertInOrder(results);
+        Assert.assertTrue(Iterables.size(results) > 0);
+        Assert.assertEquals(Iterables.size(results)
+                , Iterables.size(filterPcaps(pcapEntries, new Predicate<JSONObject>() {
+                          @Override
+                          public boolean apply(@Nullable JSONObject input) {
+                            Object prt = input.get(Constants.Fields.DST_PORT.getName());
+                            return prt != null && ((Long) prt > 20 && (Long) prt < 55792);
+                          }
+                        }, withHeaders)
+                )
+        );
+        assertInOrder(results);
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        PcapMerger.merge(baos, Iterables.partition(results, 1).iterator().next());
+        Assert.assertTrue(baos.toByteArray().length > 0);
+      }
+      {
+        //test with query filter
+        Iterable<byte[]> results =
+                job.query(new Path(outDir.getAbsolutePath())
+                        , new Path(queryDir.getAbsolutePath())
+                        , getTimestamp(0, pcapEntries)
+                        , getTimestamp(pcapEntries.size()-1, pcapEntries) + 1
+                        , 10
+                        , "ip_dst_port > 55790"
                         , new Configuration()
                         , FileSystem.get(new Configuration())
                         , new QueryPcapFilter.Configurator()
@@ -455,7 +515,7 @@ public class PcapTopologyIntegrationTest {
                   @Override
                   public boolean apply(@Nullable JSONObject input) {
                     Object prt = input.get(Constants.Fields.DST_PORT.getName());
-                    return prt != null && prt.toString().equals("22");
+                    return prt != null && (Long) prt > 55790;
                   }
                 }, withHeaders)
                 )

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9ac36dec/metron-platform/metron-pcap-backend/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/metron-platform/metron-pcap-backend/src/test/resources/log4j.properties b/metron-platform/metron-pcap-backend/src/test/resources/log4j.properties
new file mode 100644
index 0000000..0d50388
--- /dev/null
+++ b/metron-platform/metron-pcap-backend/src/test/resources/log4j.properties
@@ -0,0 +1,24 @@
+#  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.
+
+# Root logger option
+log4j.rootLogger=ERROR, stdout
+
+# Direct log messages to stdout
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.Target=System.out
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9ac36dec/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/filter/PcapFieldResolver.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/filter/PcapFieldResolver.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/filter/PcapFieldResolver.java
index 0facfa8..4aeec6c 100644
--- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/filter/PcapFieldResolver.java
+++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/filter/PcapFieldResolver.java
@@ -31,12 +31,8 @@ public class PcapFieldResolver implements VariableResolver {
   }
 
   @Override
-  public String resolve(String variable) {
-    Object obj = fieldsMap.get(Constants.Fields.fromString(variable));
-    if (obj != null) {
-      return obj.toString();
-    } else {
-      return null;
-    }
+  public Object resolve(String variable) {
+    return fieldsMap.get(Constants.Fields.fromString(variable));
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-metron/blob/9ac36dec/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/filter/fixed/FixedPcapFilter.java
----------------------------------------------------------------------
diff --git a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/filter/fixed/FixedPcapFilter.java b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/filter/fixed/FixedPcapFilter.java
index 022fca2..a7751b4 100644
--- a/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/filter/fixed/FixedPcapFilter.java
+++ b/metron-platform/metron-pcap/src/main/java/org/apache/metron/pcap/filter/fixed/FixedPcapFilter.java
@@ -86,10 +86,10 @@ public class FixedPcapFilter implements PcapFilter {
   public boolean test(PacketInfo pi) {
     VariableResolver resolver = new PcapFieldResolver(packetToFields(pi));
     String srcAddrIn = (String) resolver.resolve(Constants.Fields.SRC_ADDR.getName());
-    String srcPortIn = (String) resolver.resolve(Constants.Fields.SRC_PORT.getName());
+    Integer srcPortIn = (Integer) resolver.resolve(Constants.Fields.SRC_PORT.getName());
     String dstAddrIn = (String) resolver.resolve(Constants.Fields.DST_ADDR.getName());
-    String dstPortIn = (String) resolver.resolve(Constants.Fields.DST_PORT.getName());
-    String protocolIn = (String) resolver.resolve(Constants.Fields.PROTOCOL.getName());
+    Integer dstPortIn = (Integer) resolver.resolve(Constants.Fields.DST_PORT.getName());
+    String protocolIn = "" + resolver.resolve(Constants.Fields.PROTOCOL.getName());
 
     if (areMatch(protocol, protocolIn)) {
       if (matchesSourceAndDestination(srcAddrIn, srcPortIn, dstAddrIn, dstPortIn)) {
@@ -101,8 +101,8 @@ public class FixedPcapFilter implements PcapFilter {
     return false;
   }
 
-  private boolean areMatch(Integer filter, String input) {
-    return filter == null || areMatch(filter.toString(), input);
+  private boolean areMatch(Integer filter, Integer input) {
+    return filter == null || areMatch(Integer.toUnsignedString(filter), Integer.toUnsignedString(input));
   }
 
   private boolean areMatch(String filter, String input) {
@@ -118,9 +118,9 @@ public class FixedPcapFilter implements PcapFilter {
   }
 
   private boolean matchesSourceAndDestination(String srcAddrComp,
-                                              String srcPortComp,
+                                              Integer srcPortComp,
                                               String dstAddrComp,
-                                              String dstPortComp) {
+                                              Integer dstPortComp) {
     boolean isMatch = true;
     isMatch &= areMatch(this.srcAddr, srcAddrComp);
     isMatch &= areMatch(this.srcPort, srcPortComp);
@@ -130,9 +130,9 @@ public class FixedPcapFilter implements PcapFilter {
   }
 
   private boolean matchesReverseSourceAndDestination(String srcAddr,
-                                                     String srcPort,
+                                                     Integer srcPort,
                                                      String dstAddr,
-                                                     String dstPort) {
+                                                     Integer dstPort) {
     return matchesSourceAndDestination(dstAddr, dstPort, srcAddr, srcPort);
   }