├── .gitignore ├── .travis.yml ├── LICENSE ├── README.md ├── build.sbt ├── codecov.yml ├── docs ├── NETFLOW_V5.md └── NETFLOW_V7.md ├── project ├── build.properties └── plugins.sbt ├── scalastyle-config.xml ├── src ├── main │ ├── java │ │ └── com │ │ │ └── github │ │ │ └── sadikovi │ │ │ └── netflowlib │ │ │ ├── Buffers.java │ │ │ ├── CorruptNetFlowHeader.java │ │ │ ├── NetFlowHeader.java │ │ │ ├── NetFlowReader.java │ │ │ ├── ScanPlanner.java │ │ │ ├── Strategies.java │ │ │ ├── fields │ │ │ ├── InterfaceAlias.java │ │ │ └── InterfaceName.java │ │ │ ├── predicate │ │ │ ├── Columns.java │ │ │ ├── FilterApi.java │ │ │ ├── Inspectors.java │ │ │ ├── Operators.java │ │ │ ├── PredicateTransform.java │ │ │ └── Visitor.java │ │ │ ├── record │ │ │ ├── PredicateRecordMaterializer.java │ │ │ ├── RecordMaterializer.java │ │ │ └── ScanRecordMaterializer.java │ │ │ ├── statistics │ │ │ ├── Statistics.java │ │ │ └── StatisticsTypes.java │ │ │ ├── util │ │ │ ├── FilterIterator.java │ │ │ ├── ReadAheadInputStream.java │ │ │ ├── SafeIterator.java │ │ │ └── WrappedByteBuf.java │ │ │ └── version │ │ │ ├── NetFlow.java │ │ │ ├── NetFlowV5.java │ │ │ └── NetFlowV7.java │ └── scala │ │ └── com │ │ └── github │ │ └── sadikovi │ │ └── spark │ │ ├── benchmark │ │ ├── Benchmark.scala │ │ └── NetFlowReadBenchmark.scala │ │ ├── netflow │ │ ├── DefaultSource.scala │ │ ├── NetFlowFilters.scala │ │ ├── NetFlowOptions.scala │ │ ├── package.scala │ │ ├── sources │ │ │ ├── NetFlowRegistry.scala │ │ │ ├── ResolvedInterface.scala │ │ │ └── conversions.scala │ │ ├── version5 │ │ │ └── DefaultProvider.scala │ │ └── version7 │ │ │ └── DefaultProvider.scala │ │ └── util │ │ ├── CloseableIterator.scala │ │ ├── SerializableConfiguration.scala │ │ └── Utils.scala └── test │ ├── java │ └── com │ │ └── github │ │ └── sadikovi │ │ └── netflowlib │ │ ├── NetFlowHeaderSuite.java │ │ ├── NetFlowReaderSuite.java │ │ ├── ScanPlannerSuite.java │ │ ├── UtilSuite.java │ │ └── predicate │ │ ├── JavaColumnSuite.java │ │ └── OperatorSuite.java │ ├── resources │ ├── anomaly │ │ ├── ftv5.2016-03-15.compress2.bigend.empty │ │ ├── ftv5.2016-03-15.compress9.bigend.empty │ │ ├── ftv5.2016-03-15.compress9.bigend.records1 │ │ ├── ftv5.2016-03-15.nocompress.bigend.empty │ │ └── ftv5.2016-04-09.compress9.large-byte-start │ ├── correct │ │ ├── ftv5.2016-01-13.compress.9.sample │ │ ├── ftv5.2016-01-13.nocompress.bigend.sample │ │ ├── ftv7.2016-02-14.compress.9.bigend.sample │ │ ├── ftv7.2016-02-14.compress.9.litend.sample │ │ └── ftv7.2016-02-14.nocompress.bigend.sample │ ├── corrupt │ │ ├── ftv5.2016-01-13.compress.9.sample-00 │ │ └── ftv5.2016-01-13.compress.9.sample-01 │ ├── log4j.properties │ ├── path-resolver │ └── unsupport │ │ └── ftv8.2016-01-17.compress.7.bigend.sample │ └── scala │ └── com │ └── github │ └── sadikovi │ ├── spark │ ├── netflow │ │ ├── NetFlowFiltersSuite.scala │ │ ├── NetFlowOptionsSuite.scala │ │ ├── NetFlowSuite.scala │ │ └── sources │ │ │ ├── ConvertFunctionSuite.scala │ │ │ ├── NetFlowRegistrySuite.scala │ │ │ ├── VersionSuite.scala │ │ │ └── testproviders.scala │ └── util │ │ └── UtilsSuite.scala │ └── testutil │ ├── SparkBase.scala │ ├── SparkLocal.scala │ ├── TestBase.scala │ └── package.scala └── version.sbt /.gitignore: -------------------------------------------------------------------------------- 1 | *.class 2 | *.log 3 | 4 | # sbt specific 5 | .cache 6 | .history 7 | .lib/ 8 | dist/* 9 | target/ 10 | lib_managed/ 11 | src_managed/ 12 | project/boot/ 13 | project/plugins/project/ 14 | 15 | # Scala-IDE specific 16 | .scala_dependencies 17 | .worksheet 18 | 19 | metastore_db/ 20 | spark-warehouse/ 21 | 22 | temp/ 23 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: scala 2 | sudo: false 3 | cache: 4 | directories: 5 | - $HOME/.ivy2 6 | matrix: 7 | include: 8 | - jdk: openjdk8 9 | scala: 2.12.12 10 | env: TEST_SPARK_VERSION="3.0.0" 11 | - jdk: openjdk8 12 | scala: 2.12.12 13 | env: TEST_SPARK_VERSION="3.0.1" 14 | script: 15 | - sbt ++$TRAVIS_SCALA_VERSION scalastyle 16 | - sbt ++$TRAVIS_SCALA_VERSION "test:scalastyle" 17 | - sbt -Dspark.testVersion=$TEST_SPARK_VERSION ++$TRAVIS_SCALA_VERSION coverage test 18 | - sbt ++$TRAVIS_SCALA_VERSION assembly 19 | after_success: 20 | - bash <(curl -s https://codecov.io/bash) 21 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # spark-netflow 2 | A library for reading NetFlow files from [Spark SQL](http://spark.apache.org/docs/latest/sql-programming-guide.html). 3 | 4 | [![Build Status](https://travis-ci.org/sadikovi/spark-netflow.svg?branch=master)](https://travis-ci.org/sadikovi/spark-netflow) 5 | [![codecov](https://codecov.io/gh/sadikovi/spark-netflow/branch/master/graph/badge.svg)](https://codecov.io/gh/sadikovi/spark-netflow) 6 | 7 | ## Requirements 8 | | Spark version | spark-netflow latest version | 9 | |---------------|------------------------------| 10 | | 1.4.x | [1.3.1](http://spark-packages.org/package/sadikovi/spark-netflow) | 11 | | 1.5.x | [1.3.1](http://spark-packages.org/package/sadikovi/spark-netflow) | 12 | | 1.6.x | [1.3.1](http://spark-packages.org/package/sadikovi/spark-netflow) | 13 | | 2.0.x | [2.0.4](http://spark-packages.org/package/sadikovi/spark-netflow) | 14 | | 2.1.x | [2.0.4](http://spark-packages.org/package/sadikovi/spark-netflow) | 15 | | 3.0.x | [2.1.0](http://spark-packages.org/package/sadikovi/spark-netflow) | 16 | 17 | > Documentation reflects changes in master branch, for documentation on a specific version, please 18 | > select corresponding version tag or branch. 19 | 20 | ## Linking 21 | The spark-netflow library can be added to Spark by using the `--packages` command line option. For 22 | example, run this to include it when starting the spark shell: 23 | ```shell 24 | $SPARK_HOME/bin/spark-shell --packages com.github.sadikovi:spark-netflow_2.12:2.1.0 25 | ``` 26 | See other available versions at http://spark-packages.org/package/sadikovi/spark-netflow. 27 | 28 | ## Features 29 | - Column pruning 30 | - Predicate pushdown to the NetFlow file 31 | - Auto statistics based on file header information 32 | - Fields conversion (IP addresses, protocol, etc.) 33 | - NetFlow version 5 support ([list of columns](./docs/NETFLOW_V5.md)) 34 | - NetFlow version 7 support ([list of columns](./docs/NETFLOW_V7.md)) 35 | - Reading files from local file system and HDFS 36 | 37 | ### Options 38 | Currently supported options: 39 | 40 | | Name | Example | Description | 41 | |------|:-------:|-------------| 42 | | `version` | _5, 7_ | Version to use when parsing NetFlow files. This setting is optional, by default the package will resolve the version from provided files 43 | | `buffer` | _1024, 32Kb, 3Mb, etc_ | Buffer size for NetFlow compressed stream (default `1Mb`) 44 | | `stringify` | _true, false_ | Enables conversion of certain supported fields (e.g. IP, protocol) into human-readable format. If performance is essential, consider disabling the feature (default `true`) 45 | | `predicate-pushdown` | _true, false_ | Enables predicate pushdown at NetFlow library level (default `true`) 46 | 47 | ### Dealing with corrupt files 48 | Package supports Spark option `spark.sql.files.ignoreCorruptFiles`. When set to `true`, corrupt files 49 | are ignored (corrupt header, wrong format) or partially read (corrupt data block in a middle of a 50 | file). By default, option is set to `false`, meaning exception will be raised when such file is 51 | encountered, this behaviour is similar to Spark. 52 | 53 | ### Other NetFlow formats 54 | If you would like to have the package support NetFlow files for other formats, e.g. NetFlow 9, feel free to open an issue or a pull request. 55 | 56 | ## Example 57 | 58 | ### Scala API 59 | ```scala 60 | // You can provide only format, package will infer version from provided files, 61 | // or you can enforce version of the files with `version` option. 62 | val df = spark.read.format("com.github.sadikovi.spark.netflow").load("...") 63 | 64 | // You can read files from local file system or HDFS. 65 | val df = spark.read.format("com.github.sadikovi.spark.netflow") 66 | .option("version", "5") 67 | .load("file:/...") 68 | .select("srcip", "dstip", "packets") 69 | 70 | // You can also specify buffer size when reading compressed NetFlow files. 71 | val df = spark.read.format("com.github.sadikovi.spark.netflow") 72 | .option("version", "5") 73 | .option("buffer", "2Mb") 74 | .load("hdfs://sandbox:8020/tmp/...") 75 | ``` 76 | 77 | Alternatively you can use shortcuts for NetFlow files 78 | ```scala 79 | import com.github.sadikovi.spark.netflow._ 80 | 81 | // This will read version 5 with default buffer size. 82 | val df = spark.read.netflow5("hdfs:/...") 83 | 84 | // This will read version 7 without fields conversion. 85 | val df = spark.read.option("stringify", "false").netflow7("file:/...") 86 | ``` 87 | 88 | ### Python API 89 | ```python 90 | df = spark.read.format("com.github.sadikovi.spark.netflow") \ 91 | .option("version", "5") \ 92 | .load("file:/...") \ 93 | .select("srcip", "srcport") 94 | 95 | res = df.where("srcip > 10") 96 | ``` 97 | 98 | ### SQL API 99 | ```sql 100 | CREATE TEMPORARY TABLE ips 101 | USING com.github.sadikovi.spark.netflow 102 | OPTIONS (path "file:/...", version "5"); 103 | 104 | SELECT srcip, dstip, srcport, dstport FROM ips LIMIT 10; 105 | ``` 106 | 107 | ## Building From Source 108 | This library is built using `sbt`, to build a JAR file simply run `sbt package` from project root. 109 | 110 | ## Testing 111 | Run `sbt test` from project root. 112 | 113 | ## Running benchmark 114 | Run `sbt package` to package project, next run `spark-submit` with following options: 115 | ```shell 116 | $ spark-submit --class com.github.sadikovi.spark.benchmark.NetFlowReadBenchmark \ 117 | target/scala-2.12/spark-netflow_2.12-2.1.0.jar \ 118 | --iterations 5 \ 119 | --files 'file:/Users/sadikovi/developer/spark-netflow/temp/ftn/0[1,2,3]/ft*' \ 120 | --version 5 121 | ``` 122 | 123 | Latest benchmarks: 124 | ``` 125 | - Iterations: 5 126 | - Files: file:/tmp/spark-netflow/files/0[1,2,3]/ft* 127 | - Version: 5 128 | 129 | Java HotSpot(TM) 64-Bit Server VM 1.7.0_80-b15 on Mac OS X 10.12.4 130 | Intel(R) Core(TM) i5-4258U CPU @ 2.40GHz 131 | NetFlow full scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative 132 | ------------------------------------------------------------------------------------------------ 133 | Scan, stringify = F 567 / 633 0.0 56726.7 1.0X 134 | Scan, stringify = T 968 / 1049 0.0 96824.6 0.6X 135 | 136 | Java HotSpot(TM) 64-Bit Server VM 1.7.0_80-b15 on Mac OS X 10.12.4 137 | Intel(R) Core(TM) i5-4258U CPU @ 2.40GHz 138 | NetFlow predicate scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative 139 | ------------------------------------------------------------------------------------------------ 140 | Predicate pushdown = F, high 1148 / 1200 0.0 114845.4 1.0X 141 | Predicate pushdown = T, high 1208 / 1257 0.0 120818.0 1.0X 142 | Predicate pushdown = F, low 706 / 732 0.0 70559.3 1.6X 143 | Predicate pushdown = T, low 226 / 243 0.0 22575.0 5.1X 144 | 145 | Java HotSpot(TM) 64-Bit Server VM 1.7.0_80-b15 on Mac OS X 10.12.4 146 | Intel(R) Core(TM) i5-4258U CPU @ 2.40GHz 147 | NetFlow aggregated report: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative 148 | ------------------------------------------------------------------------------------------------ 149 | Aggregated report 2171 / 2270 0.0 217089.9 1.0X 150 | ``` 151 | 152 | ## Using `netflowlib` library separately 153 | You can use `netflowlib` without using `spark-netflow` package. Here some basic concepts and 154 | examples: 155 | - `com.github.sadikovi.netflowlib.predicate.Columns.*` all available column types in the library, 156 | check out `com.github.sadikovi.netflowlib.version.*` classes to see what columns are already defined 157 | for a specific NetFlow format. 158 | - `com.github.sadikovi.netflowlib.predicate.FilterApi` utility class to create predicates for 159 | NetFlow file 160 | - `com.github.sadikovi.netflowlib.statistics.StatisticsTypes` statistics that you can use to reduce 161 | boundaries of filter or allow filter to be evaluated before scanning the file. For example, library 162 | creates statistics on time, so time filter can be resolved upfront 163 | - `com.github.sadikovi.netflowlib.NetFlowReader` main entry to work with NetFlow file, gives access 164 | to file header and iterator of rows, allows to pass additional predicate and statistics 165 | - `com.github.sadikovi.netflowlib.NetFlowHeader` header information can be accessed using this 166 | class from `NetFlowReader.getHeader()`, see class for more information on flags available 167 | 168 | Here is the general usage pattern: 169 | ```scala 170 | import com.github.sadikovi.netflowlib.NetFlowReader 171 | import com.github.sadikovi.netflowlib.version.NetFlowV5 172 | 173 | // Create input stream by opening NetFlow file, e.g. `fs.open(hadoopFile)` 174 | val stm: DataInputStream = ... 175 | // Prepare reader based on input stream and buffer size, you can use 176 | // overloaded alternative with default buffer size 177 | val reader = NetFlowReader.prepareReader(stm, 10000) 178 | // Check out header, optional 179 | val header = reader.getHeader() 180 | // Actual NetFlow version of the file 181 | val actualVersion = header.getFlowVersion() 182 | // Whether or not file is compressed 183 | val isCompressed = header.isCompressed() 184 | 185 | // This is list of fields that will be returned in iterator as values in 186 | // array (same order) 187 | val fields = Array( 188 | NetFlowV5.FIELD_UNIX_SECS, 189 | NetFlowV5.FIELD_SRCADDR, 190 | NetFlowV5.FIELD_DSTADDR, 191 | NetFlowV5.FIELD_SRCPORT, 192 | NetFlowV5.FIELD_DSTPORT 193 | ) 194 | 195 | // Build record buffer and iterator that you can use to get values. 196 | // Note that you can also use set of filters, if you want to get 197 | // particular records 198 | val recordBuffer = reader.prepareRecordBuffer(fields) 199 | val iter = recordBuffer.iterator() 200 | 201 | while (iter.hasNext) { 202 | // print every row with values 203 | println(iter.next) 204 | } 205 | ``` 206 | 207 | Here is an example of using predicate to keep certain records: 208 | ```scala 209 | import com.github.sadikovi.netflowlib.predicate.FilterApi 210 | val predicate = FilterApi.and( 211 | FilterApi.eq(NetFlowV5.FIELD_SRCPORT, 123), 212 | FilterApi.eq(NetFlowV5.FIELD_DSTPORT, 456) 213 | ) 214 | 215 | ... 216 | val recordBuffer = reader.prepareRecordBuffer(fields, predicate) 217 | ``` 218 | -------------------------------------------------------------------------------- /build.sbt: -------------------------------------------------------------------------------- 1 | name := "spark-netflow" 2 | 3 | organization := "com.github.sadikovi" 4 | 5 | scalaVersion := "2.12.12" 6 | 7 | spName := "sadikovi/spark-netflow" 8 | 9 | val defaultSparkVersion = "3.0.1" 10 | 11 | sparkVersion := sys.props.getOrElse("spark.testVersion", defaultSparkVersion) 12 | 13 | val defaultHadoopVersion = "2.6.0" 14 | 15 | val hadoopVersion = settingKey[String]("The version of Hadoop to test against.") 16 | 17 | hadoopVersion := sys.props.getOrElse("hadoop.testVersion", defaultHadoopVersion) 18 | 19 | spAppendScalaVersion := true 20 | 21 | spIncludeMaven := true 22 | 23 | spIgnoreProvided := true 24 | 25 | sparkComponents := Seq("sql") 26 | 27 | libraryDependencies ++= Seq( 28 | "org.scalatest" %% "scalatest" % "3.2.0" % "test", 29 | "com.novocode" % "junit-interface" % "0.11" % "test" 30 | ) 31 | 32 | libraryDependencies ++= Seq( 33 | "org.apache.hadoop" % "hadoop-client" % hadoopVersion.value % "test" exclude("javax.servlet", "servlet-api") force(), 34 | "org.apache.spark" %% "spark-core" % sparkVersion.value % "test" exclude("org.apache.hadoop", "hadoop-client"), 35 | "org.apache.spark" %% "spark-sql" % sparkVersion.value % "test" exclude("org.apache.hadoop", "hadoop-client") 36 | ) 37 | 38 | // check deprecation and unchecked without manual restart 39 | javacOptions in ThisBuild ++= Seq("-Xlint:unchecked") 40 | scalacOptions in ThisBuild ++= Seq("-unchecked", "-deprecation", "-feature") 41 | 42 | // Display full-length stacktraces from ScalaTest: 43 | testOptions in Test += Tests.Argument("-oF") 44 | testOptions in Test += Tests.Argument(TestFrameworks.JUnit, "-a", "-v", "+q") 45 | 46 | parallelExecution in Test := false 47 | 48 | // Skip tests during assembly 49 | test in assembly := {} 50 | 51 | coverageMinimum := 80 52 | coverageHighlighting := true 53 | coverageFailOnMinimum := true 54 | coverageExcludedPackages := ".*.benchmark" 55 | 56 | EclipseKeys.eclipseOutput := Some("target/eclipse") 57 | 58 | // tasks dependencies 59 | lazy val compileScalastyle = taskKey[Unit]("compileScalastyle") 60 | compileScalastyle := org.scalastyle.sbt.ScalastylePlugin.scalastyle.in(Compile).toTask("").value 61 | (compile in Compile) <<= (compile in Compile).dependsOn(compileScalastyle) 62 | 63 | // Create a default Scala style task to run with tests 64 | lazy val testScalastyle = taskKey[Unit]("testScalastyle") 65 | testScalastyle := org.scalastyle.sbt.ScalastylePlugin.scalastyle.in(Test).toTask("").value 66 | (test in Test) <<= (test in Test).dependsOn(testScalastyle) 67 | 68 | /******************** 69 | * Release settings * 70 | ********************/ 71 | 72 | publishTo := { 73 | val nexus = "https://oss.sonatype.org/" 74 | if (version.value.endsWith("SNAPSHOT")) 75 | Some("snapshots" at nexus + "content/repositories/snapshots") 76 | else 77 | Some("releases" at nexus + "service/local/staging/deploy/maven2") 78 | } 79 | 80 | publishMavenStyle := true 81 | 82 | publishArtifact in Test := false 83 | 84 | pomIncludeRepository := { _ => false } 85 | 86 | releaseCrossBuild := true 87 | 88 | licenses += ("Apache-2.0", url("http://www.apache.org/licenses/LICENSE-2.0")) 89 | 90 | releasePublishArtifactsAction := PgpKeys.publishSigned.value 91 | 92 | pomExtra := ( 93 | https://github.com/sadikovi/spark-netflow 94 | 95 | git@github.com:sadikovi/spark-netflow.git 96 | scm:git:git@github.com:sadikovi/spark-netflow.git 97 | 98 | 99 | 100 | sadikovi 101 | Ivan Sadikov 102 | https://github.com/sadikovi 103 | 104 | 105 | ) 106 | 107 | bintrayReleaseOnPublish in ThisBuild := false 108 | 109 | import ReleaseTransformations._ 110 | 111 | // Add publishing to spark packages as another step. 112 | releaseProcess := Seq[ReleaseStep]( 113 | checkSnapshotDependencies, 114 | inquireVersions, 115 | runTest, 116 | setReleaseVersion, 117 | commitReleaseVersion, 118 | tagRelease, 119 | publishArtifacts, 120 | setNextVersion, 121 | commitNextVersion, 122 | pushChanges, 123 | releaseStepTask(spPublish) 124 | ) 125 | 126 | // Credentials for sbt-spark-package 127 | credentials += Credentials(Path.userHome / ".ivy2" / ".sbtcredentials") 128 | // Credentials for publishing to sonatype 129 | credentials += Credentials(Path.userHome / ".ivy2" / ".sonatype.sbt") 130 | -------------------------------------------------------------------------------- /codecov.yml: -------------------------------------------------------------------------------- 1 | comment: 2 | layout: header, changes, diff 3 | coverage: 4 | status: 5 | project: 6 | enabled: true 7 | patch: 8 | enabled: true 9 | changes: 10 | enabled: false 11 | -------------------------------------------------------------------------------- /docs/NETFLOW_V5.md: -------------------------------------------------------------------------------- 1 | ## NetFlow version 5 2 | 3 | This is a list of columns that are supported in **spark-netflow**. Note that some fields have two 4 | different types associated with them, this means that those fields have `String` representation of 5 | their values (handled with `stringify` option). 6 | 7 | | # | Column name | Column type (Spark SQL) | Description | 8 | |---|-------------|-------------------------|-------------| 9 | | 1 | `unix_secs` | `LongType` | Current seconds since 0000 UTC 1970 | 10 | | 2 | `unix_nsecs` | `LongType` | Residual nanoseconds since 0000 UTC 1970 | 11 | | 3 | `sysuptime` | `LongType` | Current time in milliseconds since router booted | 12 | | 4 | `exaddr` | `LongType` / `StringType` | Exporter IP address | 13 | | 5 | `srcip` | `LongType` / `StringType` | Source IP address | 14 | | 6 | `dstip` | `LongType` / `StringType` | Destination IP address | 15 | | 7 | `nexthop` | `LongType` / `StringType` | Next hop router's IP address | 16 | | 8 | `input` | `IntegerType` | Input interface index (known as Sif) | 17 | | 9 | `output` | `IntegerType` | Output interface index (known as Dif) | 18 | | 10 | `packets` | `LongType` | Packets sent in duration | 19 | | 11 | `octets` | `LongType` | Octets sent in duration | 20 | | 12 | `first_flow` | `LongType` | System uptime at start of flow | 21 | | 13 | `last_flow` | `LongType` | System uptime of last packet of flow | 22 | | 14 | `srcport` | `IntegerType` | TCP/UDP source port number or equivalent | 23 | | 15 | `dstport` | `IntegerType` | TCP/UDP destination port number or equivalent | 24 | | 16 | `protocol` | `ShortType` / `StringType` | IP protocol, e.g. 6 = TCP, 17 = UDP, etc. | 25 | | 17 | `tos` | `ShortType` | IP Type-of-Service | 26 | | 18 | `tcp_flags` | `ShortType` | OR of TCP header bits | 27 | | 19 | `engine_type` | `ShortType` | Type of flow switching engine (RP, VIP, etc.) | 28 | | 20 | `engine_id` | `ShortType` | Slot number of the flow switching engine | 29 | | 21 | `src_mask` | `ShortType` | Mask length of source address | 30 | | 22 | `dst_mask` | `ShortType` | Mask length of destination address | 31 | | 23 | `src_as` | `IntegerType` | AS of source address | 32 | | 24 | `dst_as` | `IntegerType` | AS of destination address | 33 | -------------------------------------------------------------------------------- /docs/NETFLOW_V7.md: -------------------------------------------------------------------------------- 1 | ## NetFlow version 7 2 | 3 | This is a list of columns that are supported in **spark-netflow**. Note that some fields have two 4 | different types associated with them, this means that those fields have `String` representation of 5 | their values (handled with `stringify` option). 6 | 7 | | # | Column name | Column type (Spark SQL) | Description | 8 | |---|-------------|-------------------------|-------------| 9 | | 1 | `unix_secs` | `LongType` | Current seconds since 0000 UTC 1970 | 10 | | 2 | `unix_nsecs` | `LongType` | Residual nanoseconds since 0000 UTC 1970 | 11 | | 3 | `sysuptime` | `LongType` | Current time in milliseconds since router booted | 12 | | 4 | `exaddr` | `LongType` / `StringType` | Exporter IP address | 13 | | 5 | `srcip` | `LongType` / `StringType` | Source IP address | 14 | | 6 | `dstip` | `LongType` / `StringType` | Destination IP address | 15 | | 7 | `nexthop` | `LongType` / `StringType` | Next hop router's IP address | 16 | | 8 | `input` | `IntegerType` | Input interface index (known as Sif) | 17 | | 9 | `output` | `IntegerType` | Output interface index (known as Dif) | 18 | | 10 | `packets` | `LongType` | Packets sent in duration | 19 | | 11 | `octets` | `LongType` | Octets sent in duration | 20 | | 12 | `first_flow` | `LongType` | System uptime at start of flow | 21 | | 13 | `last_flow` | `LongType` | System uptime of last packet of flow | 22 | | 14 | `srcport` | `IntegerType` | TCP/UDP source port number or equivalent | 23 | | 15 | `dstport` | `IntegerType` | TCP/UDP destination port number or equivalent | 24 | | 16 | `protocol` | `ShortType` / `StringType` | IP protocol, e.g. 6 = TCP, 17 = UDP, etc. | 25 | | 17 | `tos` | `ShortType` | IP Type-of-Service | 26 | | 18 | `tcp_flags` | `ShortType` | OR of TCP header bits | 27 | | 19 | `flags` | `ShortType` | Reason flow discarded | 28 | | 20 | `engine_type` | `ShortType` | Type of flow switching engine (RP, VIP, etc.) | 29 | | 21 | `engine_id` | `ShortType` | Slot number of the flow switching engine | 30 | | 22 | `src_mask` | `ShortType` | Mask length of source address | 31 | | 23 | `dst_mask` | `ShortType` | Mask length of destination address | 32 | | 24 | `src_as` | `IntegerType` | AS of source address | 33 | | 25 | `dst_as` | `IntegerType` | AS of destination address | 34 | | 26 | `router_sc` | `LongType` / `StringType` | ID of router shortcut by switch | 35 | -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version=0.13.8 2 | -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | resolvers += Resolver.url("artifactory", url("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases"))(Resolver.ivyStylePatterns) 2 | 3 | resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/" 4 | 5 | resolvers += "sonatype-releases" at "https://oss.sonatype.org/content/repositories/releases/" 6 | 7 | resolvers += "Spark Package Main Repo" at "https://repos.spark-packages.org/" 8 | 9 | addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "4.0.0") 10 | 11 | addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.6.0") 12 | 13 | addSbtPlugin("com.jsuereth" % "sbt-pgp" % "1.0.0") 14 | 15 | addSbtPlugin("org.spark-packages" % "sbt-spark-package" % "0.2.6") 16 | 17 | addSbtPlugin("org.scoverage" % "sbt-scoverage" % "1.5.1") 18 | 19 | addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.6.0") 20 | 21 | addSbtPlugin("me.lessis" % "bintray-sbt" % "0.3.0") 22 | 23 | addSbtPlugin("com.github.gseitz" % "sbt-release" % "1.0.0") 24 | -------------------------------------------------------------------------------- /scalastyle-config.xml: -------------------------------------------------------------------------------- 1 | 17 | 19 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | Scalastyle standard configuration 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 | 50 | 51 | 52 | 53 | 54 | 55 | 56 | 57 | 58 | 59 | true 60 | 61 | 62 | 63 | 64 | 65 | 66 | 67 | 68 | 69 | 70 | 71 | 72 | 73 | 74 | 75 | 76 | 77 | 78 | 79 | 80 | 81 | 82 | 83 | 84 | 85 | 86 | 87 | 88 | 89 | 90 | 91 | 92 | 93 | 94 | 95 | 96 | 97 | 98 | 99 | 100 | 101 | 102 | 103 | 104 | 105 | 106 | 107 | 108 | 109 | 110 | 111 | 112 | 113 | 114 | 115 | 116 | 117 | 118 | 119 | 120 | 121 | 122 | 123 | 124 | 125 | 126 | 127 | 128 | 129 | 130 | 131 | 132 | 133 | 134 | 135 | 136 | 137 | 138 | 139 | 140 | 141 | 142 | 143 | 144 | 145 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/Buffers.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib; 18 | 19 | import java.io.BufferedInputStream; 20 | import java.io.DataInputStream; 21 | import java.io.FilterInputStream; 22 | import java.io.IOException; 23 | import java.nio.ByteOrder; 24 | import java.util.Iterator; 25 | import java.util.NoSuchElementException; 26 | import java.util.zip.Inflater; 27 | import java.util.zip.InflaterInputStream; 28 | 29 | import com.github.sadikovi.netflowlib.record.RecordMaterializer; 30 | import com.github.sadikovi.netflowlib.util.FilterIterator; 31 | import com.github.sadikovi.netflowlib.util.ReadAheadInputStream; 32 | import com.github.sadikovi.netflowlib.util.SafeIterator; 33 | import com.github.sadikovi.netflowlib.util.WrappedByteBuf; 34 | 35 | /** 36 | * All buffers supported in NetFlow reader. 37 | * 38 | */ 39 | public final class Buffers { 40 | private Buffers() { } 41 | 42 | public static abstract class RecordBuffer implements Iterable { 43 | // min length of the buffer in bytes, usually 32768 44 | public static final int MIN_BUFFER_LENGTH = 32768; 45 | // length of buffer in bytes ~3Mb (option 1) 46 | public static final int BUFFER_LENGTH_1 = 3698688; 47 | // length of buffer in bytes ~1Mb (option 2) 48 | public static final int BUFFER_LENGTH_2 = 1048576; 49 | 50 | public abstract Iterator iterator(); 51 | 52 | @Override 53 | public String toString() { 54 | return "Record buffer: " + getClass().getCanonicalName(); 55 | } 56 | } 57 | 58 | /** 59 | * [[EmptyRecordBuffer]] is introduced for [[SkipScan]] strategy when entire file needs to be 60 | * skipped. Essentially returns empty iterator of records. 61 | */ 62 | public static final class EmptyRecordBuffer extends RecordBuffer { 63 | public EmptyRecordBuffer() { } 64 | 65 | @Override 66 | public Iterator iterator() { 67 | return new Iterator() { 68 | @Override 69 | public boolean hasNext() { 70 | return false; 71 | } 72 | 73 | @Override 74 | public Object[] next() { 75 | throw new NoSuchElementException("Empty iterator"); 76 | } 77 | 78 | @Override 79 | public void remove() { 80 | throw new UnsupportedOperationException("Remove operation is not supported"); 81 | } 82 | }; 83 | } 84 | } 85 | 86 | /** 87 | * [[ScanRecordBuffer]] is used when full scan is required, since there is no filtering on a 88 | * result from [[RecordMaterializer]], though can be used in [[FilterScan]] too. 89 | */ 90 | public static class ScanRecordBuffer extends RecordBuffer { 91 | public ScanRecordBuffer( 92 | DataInputStream in, 93 | RecordMaterializer recordMaterializer, 94 | int recordSize, 95 | ByteOrder byteOrder, 96 | boolean isCompressed, 97 | int bufferLength, 98 | boolean ignoreCorrupt) { 99 | if (isCompressed) { 100 | inflater = new Inflater(); 101 | // InflaterInputStream is replaced with ReadAheadInputStream to allow to resolve EOF before 102 | // actual record reading, we also wrap read ahead stream into buffered input stream 103 | stream = new BufferedInputStream( 104 | new ReadAheadInputStream(in, inflater, bufferLength), bufferLength); 105 | compression = true; 106 | } else { 107 | inflater = null; 108 | stream = new BufferedInputStream(in); 109 | compression = false; 110 | } 111 | 112 | this.recordMaterializer = recordMaterializer; 113 | this.recordSize = recordSize; 114 | this.ignoreCorrupt = ignoreCorrupt; 115 | recordBytes = new byte[recordSize]; 116 | buffer = WrappedByteBuf.init(recordBytes, byteOrder); 117 | numBytesRead = 0; 118 | } 119 | 120 | @Override 121 | public Iterator iterator() { 122 | Iterator iter = new Iterator() { 123 | @Override 124 | public boolean hasNext() { 125 | // `stream.available()` returns either [0, 1] in case of compressed stream and 126 | // number of bytes left in case of uncompressed stream. When it fails then we 127 | // reach EOF. 128 | boolean hasNext = true; 129 | try { 130 | // `ReadAheadInputStream` allows to check compressed stream availability correctly, 131 | // even for empty stream. 132 | hasNext = stream.available() > 0; 133 | } catch (IOException io) { 134 | hasNext = false; 135 | } finally { 136 | if (!hasNext) { 137 | try { 138 | stream.close(); 139 | } catch (IOException io) { 140 | stream = null; 141 | } 142 | buffer = null; 143 | } 144 | } 145 | 146 | return hasNext; 147 | } 148 | 149 | @Override 150 | public Object[] next() { 151 | try { 152 | numBytesRead = stream.read(recordBytes, 0, recordSize); 153 | if (numBytesRead < 0) { 154 | throw new IOException("EOF, " + numBytesRead + " bytes read"); 155 | } else if (numBytesRead < recordSize) { 156 | // We have to read entire record when there is no compression, anything else is 157 | // considered failure. When stream is compressed we can read less, but then we need 158 | // buffer up remaning data. 159 | if (!compression) { 160 | throw new IllegalArgumentException( 161 | "Failed to read record: " + numBytesRead + " < " + recordSize); 162 | } else { 163 | int remaining = recordSize - numBytesRead; 164 | int addBytes = stream.read(recordBytes, numBytesRead, remaining); 165 | if (addBytes != remaining) { 166 | throw new IllegalArgumentException( 167 | "Failed to read record: " + addBytes + " != " + remaining); 168 | } 169 | } 170 | } 171 | } catch (IOException io) { 172 | throw new IllegalArgumentException("Unexpected EOF", io); 173 | } 174 | 175 | return recordMaterializer.processRecord(buffer); 176 | } 177 | 178 | @Override 179 | public void remove() { 180 | throw new UnsupportedOperationException("Remove operation is not supported"); 181 | } 182 | }; 183 | 184 | // when ignoring corrupt records, wrap it into iterator with safe termination on failures 185 | return ignoreCorrupt ? new SafeIterator(iter) : iter; 186 | } 187 | 188 | @Override 189 | public String toString() { 190 | return "Record buffer: " + getClass().getCanonicalName() + "[compression: " + compression + 191 | ", record size: " + recordSize + ", ignoreCorrupt: " + ignoreCorrupt + "]"; 192 | } 193 | 194 | // Whether or not input stream is compressed 195 | private final boolean compression; 196 | // Reference to inflater to find out EOF mainly 197 | private final Inflater inflater; 198 | // Stream to read either standard DataInputStream or InflaterInputStream 199 | private BufferedInputStream stream; 200 | // Array of bytes for a record, updated partially when compression buffer needs to be refilled 201 | private final byte[] recordBytes; 202 | // Buffer for the record 203 | private WrappedByteBuf buffer; 204 | // Number of bytes currently have been read 205 | private int numBytesRead; 206 | // Size of record, depends on NetFlow format 207 | private final int recordSize; 208 | // Record materializer to process individual record 209 | private final RecordMaterializer recordMaterializer; 210 | // Ignore corrupt records and terminate iterator once encountered 211 | private final boolean ignoreCorrupt; 212 | } 213 | 214 | /** 215 | * [[FilterRecordBuffer]] is used when filtering is required on result of [[RecordMaterializer]], 216 | * in this case all "null" records would be skipped, e.g. records that failed predicate 217 | * requirement. 218 | */ 219 | public static final class FilterRecordBuffer extends ScanRecordBuffer { 220 | public FilterRecordBuffer( 221 | DataInputStream in, 222 | RecordMaterializer recordMaterializer, 223 | int recordSize, 224 | ByteOrder byteOrder, 225 | boolean isCompressed, 226 | int bufferLength, 227 | boolean ignoreCorrupt) { 228 | super(in, recordMaterializer, recordSize, byteOrder, isCompressed, bufferLength, 229 | ignoreCorrupt); 230 | } 231 | 232 | @Override 233 | public Iterator iterator() { 234 | return new FilterIterator(super.iterator()); 235 | } 236 | } 237 | } 238 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/CorruptNetFlowHeader.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib; 18 | 19 | import java.nio.ByteOrder; 20 | 21 | /** 22 | * Class to indicate that header is corrupt. Overwrites parent method `isValid()` to hint on 23 | * incorrectness of header. 24 | */ 25 | public class CorruptNetFlowHeader extends NetFlowHeader { 26 | public CorruptNetFlowHeader() { } 27 | 28 | //////////////////////////////////////////////////////////// 29 | // Setters API 30 | //////////////////////////////////////////////////////////// 31 | @Override 32 | public void setFlowVersion(short version) { 33 | throw new UnsupportedOperationException("Header is corrupt"); 34 | } 35 | 36 | @Override 37 | public void setStartCapture(long value) { 38 | throw new UnsupportedOperationException("Header is corrupt"); 39 | } 40 | 41 | @Override 42 | public void setEndCapture(long value) { 43 | throw new UnsupportedOperationException("Header is corrupt"); 44 | } 45 | 46 | @Override 47 | public void setHeaderFlags(long flags) { 48 | throw new UnsupportedOperationException("Header is corrupt"); 49 | } 50 | 51 | @Override 52 | public void setRotation(long value) { 53 | throw new UnsupportedOperationException("Header is corrupt"); 54 | } 55 | 56 | @Override 57 | public void setNumFlows(long value) { 58 | throw new UnsupportedOperationException("Header is corrupt"); 59 | } 60 | 61 | @Override 62 | public void setNumDropped(long value) { 63 | throw new UnsupportedOperationException("Header is corrupt"); 64 | } 65 | 66 | @Override 67 | public void setNumMisordered(long value) { 68 | throw new UnsupportedOperationException("Header is corrupt"); 69 | } 70 | 71 | @Override 72 | public void setHostname(String hostname) { 73 | throw new UnsupportedOperationException("Header is corrupt"); 74 | } 75 | 76 | @Override 77 | public void setComments(String comments) { 78 | throw new UnsupportedOperationException("Header is corrupt"); 79 | } 80 | 81 | @Override 82 | public void setVendor(int value) { 83 | throw new UnsupportedOperationException("Header is corrupt"); 84 | } 85 | 86 | @Override 87 | public void setAggVersion(short value) { 88 | throw new UnsupportedOperationException("Header is corrupt"); 89 | } 90 | 91 | @Override 92 | public void setAggMethod(short value) { 93 | throw new UnsupportedOperationException("Header is corrupt"); 94 | } 95 | 96 | @Override 97 | public void setExporterIP(long value) { 98 | throw new UnsupportedOperationException("Header is corrupt"); 99 | } 100 | 101 | @Override 102 | public void setNumCorrupt(long value) { 103 | throw new UnsupportedOperationException("Header is corrupt"); 104 | } 105 | 106 | @Override 107 | public void setSeqReset(long value) { 108 | throw new UnsupportedOperationException("Header is corrupt"); 109 | } 110 | 111 | @Override 112 | public void setInterfaceName(long ip, int ifIndex, String name) { 113 | throw new UnsupportedOperationException("Header is corrupt"); 114 | } 115 | 116 | @Override 117 | public void setInterfaceAlias(long ip, int ifIndexCount, int ifIndex, String alias) { 118 | throw new UnsupportedOperationException("Header is corrupt"); 119 | } 120 | 121 | //////////////////////////////////////////////////////////// 122 | // Getters API 123 | //////////////////////////////////////////////////////////// 124 | @Override 125 | public short getStreamVersion() { 126 | throw new UnsupportedOperationException("Header is corrupt"); 127 | } 128 | 129 | @Override 130 | public ByteOrder getByteOrder() { 131 | throw new UnsupportedOperationException("Header is corrupt"); 132 | } 133 | 134 | @Override 135 | public int getHeaderSize() { 136 | throw new UnsupportedOperationException("Header is corrupt"); 137 | } 138 | 139 | @Override 140 | public short getFlowVersion() { 141 | throw new UnsupportedOperationException("Header is corrupt"); 142 | } 143 | 144 | @Override 145 | public long getStartCapture() { 146 | throw new UnsupportedOperationException("Header is corrupt"); 147 | } 148 | 149 | @Override 150 | public long getEndCapture() { 151 | throw new UnsupportedOperationException("Header is corrupt"); 152 | } 153 | 154 | @Override 155 | public long getHeaderFlags() { 156 | throw new UnsupportedOperationException("Header is corrupt"); 157 | } 158 | 159 | @Override 160 | public boolean isCompressed() { 161 | throw new UnsupportedOperationException("Header is corrupt"); 162 | } 163 | 164 | @Override 165 | public String getHostname() { 166 | throw new UnsupportedOperationException("Header is corrupt"); 167 | } 168 | 169 | @Override 170 | public String getComments() { 171 | throw new UnsupportedOperationException("Header is corrupt"); 172 | } 173 | 174 | @Override 175 | public short getAggMethod() { 176 | throw new UnsupportedOperationException("Header is corrupt"); 177 | } 178 | 179 | @Override 180 | public short getAggVersion() { 181 | throw new UnsupportedOperationException("Header is corrupt"); 182 | } 183 | 184 | @Override 185 | public long getFields() { 186 | throw new UnsupportedOperationException("Header is corrupt"); 187 | } 188 | 189 | @Override 190 | public boolean isValid() { 191 | return false; 192 | } 193 | } 194 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/Strategies.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib; 18 | 19 | import java.util.ArrayList; 20 | import java.util.HashMap; 21 | 22 | import com.github.sadikovi.netflowlib.predicate.Columns.Column; 23 | import com.github.sadikovi.netflowlib.predicate.Inspectors.Inspector; 24 | import com.github.sadikovi.netflowlib.predicate.Inspectors.ValueInspector; 25 | import com.github.sadikovi.netflowlib.record.RecordMaterializer; 26 | import com.github.sadikovi.netflowlib.record.ScanRecordMaterializer; 27 | import com.github.sadikovi.netflowlib.record.PredicateRecordMaterializer; 28 | 29 | /** 30 | * All possible strategies to scan, e.g. skipping entire file, full scan of records, or different 31 | * filtering scans for a record. Every strategy must be instantiated with set of pruned columns 32 | * and/or set of filters to apply. Make sure that predicate is optimized on previous step. 33 | */ 34 | public final class Strategies { 35 | private Strategies() { } 36 | 37 | public static abstract class ScanStrategy { 38 | public abstract boolean skipScan(); 39 | 40 | public abstract boolean fullScan(); 41 | 42 | public abstract RecordMaterializer getRecordMaterializer(); 43 | 44 | @Override 45 | public String toString() { 46 | return "Strategy [" + getClass().getSimpleName() + "]"; 47 | } 48 | } 49 | 50 | ////////////////////////////////////////////////////////////// 51 | // Strategies 52 | ////////////////////////////////////////////////////////////// 53 | 54 | /** Skip scan fully without reading file */ 55 | public static final class SkipScan extends ScanStrategy { 56 | public SkipScan() { } 57 | 58 | @Override 59 | public boolean skipScan() { 60 | return true; 61 | } 62 | 63 | @Override 64 | public boolean fullScan() { 65 | return false; 66 | } 67 | 68 | @Override 69 | public RecordMaterializer getRecordMaterializer() { 70 | throw new UnsupportedOperationException("RecordMaterializer is not supported for " + 71 | getClass().getSimpleName()); 72 | } 73 | } 74 | 75 | /** Full scan of the file without any predicate */ 76 | public static final class FullScan extends ScanStrategy { 77 | public FullScan(Column[] columns) { 78 | rm = new ScanRecordMaterializer(columns); 79 | } 80 | 81 | @Override 82 | public boolean skipScan() { 83 | return false; 84 | } 85 | 86 | @Override 87 | public boolean fullScan() { 88 | return true; 89 | } 90 | 91 | @Override 92 | public RecordMaterializer getRecordMaterializer() { 93 | return rm; 94 | } 95 | 96 | private final RecordMaterializer rm; 97 | } 98 | 99 | /** Scan with filtering (fairly expensive, so we try resolving either to skip or full scan) */ 100 | public static class FilterScan extends ScanStrategy { 101 | public FilterScan( 102 | Column[] columns, 103 | Inspector tree, 104 | HashMap> inspectors) { 105 | rm = new PredicateRecordMaterializer(columns, tree, inspectors); 106 | } 107 | 108 | @Override 109 | public boolean skipScan() { 110 | return false; 111 | } 112 | 113 | @Override 114 | public boolean fullScan() { 115 | return false; 116 | } 117 | 118 | @Override 119 | public RecordMaterializer getRecordMaterializer() { 120 | return rm; 121 | } 122 | 123 | private final RecordMaterializer rm; 124 | } 125 | } 126 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/fields/InterfaceAlias.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.fields; 18 | 19 | import java.nio.ByteOrder; 20 | 21 | /** Interface alias as one of the TLV */ 22 | public class InterfaceAlias { 23 | public InterfaceAlias(long ip, int ifIndexCount, int ifIndex, String aliasName) { 24 | this.ip = ip; 25 | this.ifIndexCount = ifIndexCount; 26 | this.ifIndex = ifIndex; 27 | this.aliasName = aliasName; 28 | } 29 | 30 | /** Get IP of device */ 31 | public long getIP() { 32 | return this.ip; 33 | } 34 | 35 | /** Get ifIndex count */ 36 | public int getIfIndexCount() { 37 | return this.ifIndexCount; 38 | } 39 | 40 | /** Get ifIndex of interface */ 41 | public int getIfIndex() { 42 | return this.ifIndex; 43 | } 44 | 45 | /** Get alias name */ 46 | public String getAliasName() { 47 | return this.aliasName; 48 | } 49 | 50 | private long ip = 0; 51 | private int ifIndexCount = 0; 52 | private int ifIndex = 0; 53 | private String aliasName = null; 54 | } 55 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/fields/InterfaceName.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.fields; 18 | 19 | import java.nio.ByteOrder; 20 | 21 | /** Interface name as one of the TLV */ 22 | public class InterfaceName { 23 | public InterfaceName(long ip, int ifIndex, String ifName) { 24 | this.ip = ip; 25 | this.ifIndex = ifIndex; 26 | this.ifName = ifName; 27 | } 28 | 29 | /** Get interface IP */ 30 | public long getIP() { 31 | return this.ip; 32 | } 33 | 34 | /** Get interface IfIndex */ 35 | public int getIfIndex() { 36 | return this.ifIndex; 37 | } 38 | 39 | /** Get interface name */ 40 | public String getInterfaceName() { 41 | return this.ifName; 42 | } 43 | 44 | private long ip = 0; 45 | private int ifIndex = 0; 46 | private String ifName = null; 47 | } 48 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/predicate/Columns.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.predicate; 18 | 19 | import java.io.Serializable; 20 | 21 | import com.github.sadikovi.netflowlib.predicate.Inspectors.ValueInspector; 22 | import com.github.sadikovi.netflowlib.statistics.Statistics; 23 | import com.github.sadikovi.netflowlib.util.WrappedByteBuf; 24 | 25 | /** 26 | * [[Column]] is a base class for all typed columns for all NetFlow versions. They contain basic 27 | * information about name, type, and offset in the record, so it allows to use standard interface of 28 | * [[RecordMaterializer]] for parsing a row. 29 | */ 30 | public final class Columns { 31 | private Columns() { } 32 | 33 | public static abstract class Column implements Serializable, Statistics { 34 | Column() { } 35 | 36 | Column(String name, Class type, int offset) { 37 | if (offset < 0) { 38 | throw new IllegalArgumentException("Wrong offset " + offset); 39 | } 40 | 41 | columnName = name; 42 | columnType = type; 43 | columnOffset = offset; 44 | } 45 | 46 | /** Get column name */ 47 | public String getColumnName() { 48 | return columnName; 49 | } 50 | 51 | /** Get column type */ 52 | public Class getColumnType() { 53 | return columnType; 54 | } 55 | 56 | /** Get column offset in a record */ 57 | public int getColumnOffset() { 58 | return columnOffset; 59 | } 60 | 61 | /** Read field from byte buffer and return value that confirms to column class */ 62 | public abstract Object readField(WrappedByteBuf buffer); 63 | 64 | /** Update value inspector with value from buffer */ 65 | public abstract void updateValueInspector(WrappedByteBuf buffer, ValueInspector vi); 66 | 67 | @Override 68 | public String toString() { 69 | return getClass().getSimpleName() + "(name=" + columnName + ", offset=" + columnOffset + ")"; 70 | } 71 | 72 | @Override 73 | public boolean equals(Object obj) { 74 | if (this == obj) return true; 75 | if (obj == null || getClass() != obj.getClass()) return false; 76 | 77 | Column that = (Column) obj; 78 | 79 | if (!columnType.equals(that.columnType)) return false; 80 | if (!columnName.equals(that.columnName)) return false; 81 | if (columnOffset != that.columnOffset) return false; 82 | 83 | return true; 84 | } 85 | 86 | @Override 87 | public int hashCode() { 88 | int result = columnName.hashCode(); 89 | result = 31 * result + columnType.hashCode(); 90 | result = 31 * result + columnOffset; 91 | return result; 92 | } 93 | 94 | private String columnName = null; 95 | private Class columnType = null; 96 | private int columnOffset = -1; 97 | } 98 | 99 | /** Column for byte values */ 100 | public static class ByteColumn extends Column { 101 | ByteColumn() { } 102 | 103 | ByteColumn(String name, int offset, byte min, byte max) { 104 | super(name, Byte.class, offset); 105 | minValue = min; 106 | maxValue = max; 107 | } 108 | 109 | public ByteColumn(String name, int offset) { 110 | this(name, offset, (byte) 0, Byte.MAX_VALUE); 111 | } 112 | 113 | @Override 114 | public Object readField(WrappedByteBuf buffer) { 115 | return buffer.getByte(getColumnOffset()); 116 | } 117 | 118 | @Override 119 | public void updateValueInspector(WrappedByteBuf buffer, ValueInspector vi) { 120 | vi.update(buffer.getByte(getColumnOffset())); 121 | } 122 | 123 | @Override 124 | public Object getMin() { 125 | return minValue; 126 | } 127 | 128 | @Override 129 | public Object getMax() { 130 | return maxValue; 131 | } 132 | 133 | private byte minValue; 134 | private byte maxValue; 135 | } 136 | 137 | /** Column for short values */ 138 | public static class ShortColumn extends Column { 139 | ShortColumn() { } 140 | 141 | ShortColumn(String name, int offset, short min, short max) { 142 | super(name, Short.class, offset); 143 | minValue = min; 144 | maxValue = max; 145 | } 146 | 147 | public ShortColumn(String name, int offset) { 148 | this(name, offset, (short) 0, Short.MAX_VALUE); 149 | } 150 | 151 | @Override 152 | public Object readField(WrappedByteBuf buffer) { 153 | return buffer.getUnsignedByte(getColumnOffset()); 154 | } 155 | 156 | @Override 157 | public void updateValueInspector(WrappedByteBuf buffer, ValueInspector vi) { 158 | vi.update(buffer.getUnsignedByte(getColumnOffset())); 159 | } 160 | 161 | @Override 162 | public Object getMin() { 163 | return minValue; 164 | } 165 | 166 | @Override 167 | public Object getMax() { 168 | return maxValue; 169 | } 170 | 171 | private short minValue; 172 | private short maxValue; 173 | } 174 | 175 | /** Column for integer values */ 176 | public static class IntColumn extends Column { 177 | IntColumn() { } 178 | 179 | IntColumn(String name, int offset, int min, int max) { 180 | super(name, Integer.class, offset); 181 | minValue = min; 182 | maxValue = max; 183 | } 184 | 185 | public IntColumn(String name, int offset) { 186 | this(name, offset, (int) 0, Integer.MAX_VALUE); 187 | } 188 | 189 | @Override 190 | public Object readField(WrappedByteBuf buffer) { 191 | return buffer.getUnsignedShort(getColumnOffset()); 192 | } 193 | 194 | @Override 195 | public void updateValueInspector(WrappedByteBuf buffer, ValueInspector vi) { 196 | vi.update(buffer.getUnsignedShort(getColumnOffset())); 197 | } 198 | 199 | @Override 200 | public Object getMin() { 201 | return minValue; 202 | } 203 | 204 | @Override 205 | public Object getMax() { 206 | return maxValue; 207 | } 208 | 209 | private int minValue; 210 | private int maxValue; 211 | } 212 | 213 | /** Column for long values */ 214 | public static class LongColumn extends Column { 215 | LongColumn() { } 216 | 217 | LongColumn(String name, int offset, long min, long max) { 218 | super(name, Long.class, offset); 219 | minValue = min; 220 | maxValue = max; 221 | } 222 | 223 | public LongColumn(String name, int offset) { 224 | this(name, offset, (long) 0, Long.MAX_VALUE); 225 | } 226 | 227 | @Override 228 | public Object readField(WrappedByteBuf buffer) { 229 | return buffer.getUnsignedInt(getColumnOffset()); 230 | } 231 | 232 | @Override 233 | public void updateValueInspector(WrappedByteBuf buffer, ValueInspector vi) { 234 | vi.update(buffer.getUnsignedInt(getColumnOffset())); 235 | } 236 | 237 | @Override 238 | public Object getMin() { 239 | return minValue; 240 | } 241 | 242 | @Override 243 | public Object getMax() { 244 | return maxValue; 245 | } 246 | 247 | private long minValue; 248 | private long maxValue; 249 | } 250 | } 251 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/predicate/FilterApi.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.predicate; 18 | 19 | import java.util.HashSet; 20 | 21 | import com.github.sadikovi.netflowlib.predicate.Columns.Column; 22 | import com.github.sadikovi.netflowlib.predicate.Operators.FilterPredicate; 23 | import com.github.sadikovi.netflowlib.predicate.Operators.Eq; 24 | import com.github.sadikovi.netflowlib.predicate.Operators.Gt; 25 | import com.github.sadikovi.netflowlib.predicate.Operators.Ge; 26 | import com.github.sadikovi.netflowlib.predicate.Operators.Lt; 27 | import com.github.sadikovi.netflowlib.predicate.Operators.Le; 28 | import com.github.sadikovi.netflowlib.predicate.Operators.In; 29 | import com.github.sadikovi.netflowlib.predicate.Operators.And; 30 | import com.github.sadikovi.netflowlib.predicate.Operators.Or; 31 | import com.github.sadikovi.netflowlib.predicate.Operators.Not; 32 | import com.github.sadikovi.netflowlib.predicate.Operators.TrivialPredicate; 33 | 34 | /** 35 | * Filter API to create predicate tree. Usage 36 | * {{{ 37 | * import com.github.sadikovi.netflowlib.predicate.Columns.IntColumn; 38 | * import com.github.sadikovi.netflowlib.predicate.Columns.ShortColumn; 39 | * import com.github.sadikovi.netflowlib.predicate.Operators.FilterPredicate; 40 | * 41 | * ShortColumn col1 = new ShortColumn(1, 0); 42 | * IntColumn col2 = new IntColumn(1, 2); 43 | * FilterPredicate predicate = and(eq(col1, 2), gt(col2, 100)); 44 | * }}} 45 | * 46 | */ 47 | public final class FilterApi { 48 | private FilterApi() { } 49 | 50 | /** Test equality of column and value */ 51 | public static Eq eq(Column column, Object value) { 52 | return new Eq(column, value); 53 | } 54 | 55 | /** Test if column is greater than value */ 56 | public static Gt gt(Column column, Object value) { 57 | return new Gt(column, value); 58 | } 59 | 60 | /** Test if column is greater than or equal to value */ 61 | public static Ge ge(Column column, Object value) { 62 | return new Ge(column, value); 63 | } 64 | 65 | /** Test if column is less than value */ 66 | public static Lt lt(Column column, Object value) { 67 | return new Lt(column, value); 68 | } 69 | 70 | /** Test if column is less than or equal to value */ 71 | public static Le le(Column column, Object value) { 72 | return new Le(column, value); 73 | } 74 | 75 | /** Test if column is in the set of requested values */ 76 | public static In in(Column column, HashSet values) { 77 | return new In(column, values); 78 | } 79 | 80 | /** Test "and" logical operator for left and right predicates */ 81 | public static And and(FilterPredicate left, FilterPredicate right) { 82 | return new And(left, right); 83 | } 84 | 85 | /** Test "or" logical operator for left and right predicates */ 86 | public static Or or(FilterPredicate left, FilterPredicate right) { 87 | return new Or(left, right); 88 | } 89 | 90 | /** Test inversed operator for child predicate */ 91 | public static Not not(FilterPredicate child) { 92 | return new Not(child); 93 | } 94 | 95 | /** 96 | * Trivial predicate, should not be used directly, but only for building predicate tree from 97 | * external system. 98 | */ 99 | public static TrivialPredicate trivial(boolean result) { 100 | return new TrivialPredicate(result); 101 | } 102 | } 103 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/predicate/Inspectors.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.predicate; 18 | 19 | import java.io.Serializable; 20 | 21 | /** 22 | * [[Inspector]] interface is designed to provide methods of resolving predicate values and give 23 | * an answer on when to skip record. [[ValueInspector]] should be implemented for all basic 24 | * filters (leaf nodes). Note that [[TrivialPredicate]] does not support value inspector, so 25 | * `update()` should be run beforehand to optimizie predicate and remove them. [[BinaryLogical]] 26 | * inspectors are supported by corresponding predicates `And` and `Or`, and [[UnaryLogical]] 27 | * inspector is backed by `Not` filter. 28 | * `Inspector` can be used to resolve predicate incrementally, though it depends on design of a 29 | * caller. 30 | */ 31 | public final class Inspectors { 32 | private Inspectors() { } 33 | 34 | public static interface Inspector { 35 | 36 | boolean accept(Visitor visitor); 37 | } 38 | 39 | /** Inspector for leaf nodes, e.g. Eq, Ge, Gt, Le, Lt, In */ 40 | public static class ValueInspector implements Inspector, Serializable { 41 | public ValueInspector() { } 42 | 43 | public void update(boolean value) { throw new UnsupportedOperationException(); } 44 | public void update(byte value) { throw new UnsupportedOperationException(); } 45 | public void update(short value) { throw new UnsupportedOperationException(); } 46 | public void update(int value) { throw new UnsupportedOperationException(); } 47 | public void update(long value) { throw new UnsupportedOperationException(); } 48 | 49 | public final void reset() { 50 | known = false; 51 | result = false; 52 | } 53 | 54 | public final void setResult(boolean expression) { 55 | if (isKnown()) { 56 | throw new IllegalStateException("Inspector is already known, cannot set result"); 57 | } 58 | 59 | result = expression; 60 | known = true; 61 | } 62 | 63 | public final boolean getResult() { 64 | if (!isKnown()) { 65 | throw new IllegalStateException("Inspector is not known, cannot return result"); 66 | } 67 | 68 | return result; 69 | } 70 | 71 | public final boolean isKnown() { 72 | return known; 73 | } 74 | 75 | @Override 76 | public boolean accept(Visitor visitor) { 77 | return visitor.visit(this); 78 | } 79 | 80 | private boolean known = false; 81 | private boolean result = false; 82 | } 83 | 84 | /** Inspector for binary logical operators, e.g. And, Or */ 85 | static abstract class BinaryLogical implements Inspector, Serializable { 86 | BinaryLogical() { } 87 | 88 | BinaryLogical(Inspector left, Inspector right) { 89 | this.left = left; 90 | this.right = right; 91 | } 92 | 93 | public final Inspector getLeft() { 94 | return left; 95 | } 96 | 97 | public final Inspector getRight() { 98 | return right; 99 | } 100 | 101 | private Inspector left; 102 | private Inspector right; 103 | } 104 | 105 | public static final class AndInspector extends BinaryLogical { 106 | public AndInspector(Inspector left, Inspector right) { 107 | super(left, right); 108 | } 109 | 110 | @Override 111 | public boolean accept(Visitor visitor) { 112 | return visitor.visit(this); 113 | } 114 | } 115 | 116 | public static final class OrInspector extends BinaryLogical { 117 | public OrInspector(Inspector left, Inspector right) { 118 | super(left, right); 119 | } 120 | 121 | @Override 122 | public boolean accept(Visitor visitor) { 123 | return visitor.visit(this); 124 | } 125 | } 126 | 127 | /** Inspector for unary logical operators, e.g. Not */ 128 | static abstract class UnaryLogical implements Inspector, Serializable { 129 | UnaryLogical(Inspector child) { 130 | this.child = child; 131 | } 132 | 133 | public final Inspector getChild() { 134 | return child; 135 | } 136 | 137 | private final Inspector child; 138 | } 139 | 140 | public static final class NotInspector extends UnaryLogical { 141 | public NotInspector(Inspector child) { 142 | super(child); 143 | } 144 | 145 | @Override 146 | public boolean accept(Visitor visitor) { 147 | return visitor.visit(this); 148 | } 149 | } 150 | } 151 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/predicate/PredicateTransform.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.predicate; 18 | 19 | import java.util.HashMap; 20 | 21 | import com.github.sadikovi.netflowlib.predicate.Operators.FilterPredicate; 22 | import com.github.sadikovi.netflowlib.predicate.Operators.Eq; 23 | import com.github.sadikovi.netflowlib.predicate.Operators.Gt; 24 | import com.github.sadikovi.netflowlib.predicate.Operators.Ge; 25 | import com.github.sadikovi.netflowlib.predicate.Operators.Lt; 26 | import com.github.sadikovi.netflowlib.predicate.Operators.Le; 27 | import com.github.sadikovi.netflowlib.predicate.Operators.In; 28 | import com.github.sadikovi.netflowlib.predicate.Operators.And; 29 | import com.github.sadikovi.netflowlib.predicate.Operators.Or; 30 | import com.github.sadikovi.netflowlib.predicate.Operators.Not; 31 | import com.github.sadikovi.netflowlib.predicate.Operators.TrivialPredicate; 32 | import com.github.sadikovi.netflowlib.statistics.Statistics; 33 | 34 | /** 35 | * Abstract [[PredicateTransform]] interface allows to modify predicate tree. By default predicate 36 | * tree is immutable, it is recommended to always return a new node, if modified. Also some 37 | * optimizations might apply in concrete implementations. 38 | */ 39 | public abstract interface PredicateTransform { 40 | FilterPredicate transform(Eq predicate, HashMap stats); 41 | 42 | FilterPredicate transform(Gt predicate, HashMap stats); 43 | 44 | FilterPredicate transform(Ge predicate, HashMap stats); 45 | 46 | FilterPredicate transform(Lt predicate, HashMap stats); 47 | 48 | FilterPredicate transform(Le predicate, HashMap stats); 49 | 50 | FilterPredicate transform(In predicate, HashMap stats); 51 | 52 | FilterPredicate transform(And predicate); 53 | 54 | FilterPredicate transform(Or predicate); 55 | 56 | FilterPredicate transform(Not predicate); 57 | } 58 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/predicate/Visitor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.predicate; 18 | 19 | import com.github.sadikovi.netflowlib.predicate.Inspectors.AndInspector; 20 | import com.github.sadikovi.netflowlib.predicate.Inspectors.NotInspector; 21 | import com.github.sadikovi.netflowlib.predicate.Inspectors.OrInspector; 22 | import com.github.sadikovi.netflowlib.predicate.Inspectors.ValueInspector; 23 | 24 | /** 25 | * [[Visitor]] interface to traverse [[Inspector]] instances and resolve boolean expressions for 26 | * each of them. Usually implemented by [[RecordMaterializer]] subclasses to resolve predicate 27 | * for each record. 28 | */ 29 | public abstract interface Visitor { 30 | boolean visit(ValueInspector inspector); 31 | 32 | boolean visit(AndInspector inspector); 33 | 34 | boolean visit(OrInspector inspector); 35 | 36 | boolean visit(NotInspector inspector); 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/record/PredicateRecordMaterializer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.record; 18 | 19 | import java.util.ArrayList; 20 | import java.util.HashMap; 21 | 22 | import com.github.sadikovi.netflowlib.predicate.Columns.Column; 23 | import com.github.sadikovi.netflowlib.predicate.Inspectors.Inspector; 24 | import com.github.sadikovi.netflowlib.predicate.Inspectors.ValueInspector; 25 | import com.github.sadikovi.netflowlib.predicate.Inspectors.AndInspector; 26 | import com.github.sadikovi.netflowlib.predicate.Inspectors.OrInspector; 27 | import com.github.sadikovi.netflowlib.predicate.Inspectors.NotInspector; 28 | import com.github.sadikovi.netflowlib.predicate.Visitor; 29 | import com.github.sadikovi.netflowlib.util.WrappedByteBuf; 30 | 31 | public final class PredicateRecordMaterializer implements RecordMaterializer, Visitor { 32 | public PredicateRecordMaterializer( 33 | Column[] columns, 34 | Inspector tree, 35 | HashMap> columnInspectors) { 36 | this.tree = tree; 37 | this.columns = columns; 38 | this.numColumns = this.columns.length; 39 | this.filterColumns = columnInspectors.keySet().toArray(new Column[columnInspectors.size()]); 40 | this.numFilterColumns = this.filterColumns.length; 41 | this.inspectors = new HashMap>(); 42 | for (Column col: filterColumns) { 43 | inspectors.put(col.getColumnName(), columnInspectors.get(col)); 44 | } 45 | } 46 | 47 | @Override 48 | public Object[] processRecord(WrappedByteBuf buffer) { 49 | // Process filter columns, evaluate predicate upfront 50 | for (int i = 0; i < numFilterColumns; i++) { 51 | updateValueInspectors(filterColumns[i], buffer); 52 | } 53 | 54 | boolean result = tree.accept(this); 55 | // Reset value inspectors 56 | for (int i=0; i ins = inspectors.get(column.getColumnName()); 71 | for (int i = 0; i < ins.size(); i++) { 72 | column.updateValueInspector(buffer, ins.get(i)); 73 | } 74 | } 75 | 76 | private void resetValueInspectors(Column column) { 77 | ArrayList ins = inspectors.get(column.getColumnName()); 78 | for (int i = 0; i < ins.size(); i++) { 79 | ins.get(i).reset(); 80 | } 81 | } 82 | 83 | @Override 84 | public boolean visit(ValueInspector inspector) { 85 | return inspector.getResult(); 86 | } 87 | 88 | @Override 89 | public boolean visit(AndInspector inspector) { 90 | return inspector.getLeft().accept(this) && inspector.getRight().accept(this); 91 | } 92 | 93 | @Override 94 | public boolean visit(OrInspector inspector) { 95 | return inspector.getLeft().accept(this) || inspector.getRight().accept(this); 96 | } 97 | 98 | @Override 99 | public boolean visit(NotInspector inspector) { 100 | return !inspector.getChild().accept(this); 101 | } 102 | 103 | private final Inspector tree; 104 | private final Column[] columns; 105 | private final int numColumns; 106 | private final Column[] filterColumns; 107 | private final int numFilterColumns; 108 | private final HashMap> inspectors; 109 | } 110 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/record/RecordMaterializer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.record; 18 | 19 | import com.github.sadikovi.netflowlib.util.WrappedByteBuf; 20 | 21 | /** 22 | * [[RecordMaterializer]] interface provides all necessary methods to parse single record and return 23 | * either array of values that match list and order of pruned columns, or null, if record does not 24 | * pass predicate. Predicate check is optional and should depend on implementation. 25 | */ 26 | public interface RecordMaterializer { 27 | 28 | /** 29 | * Process record using wrapped byte buffer, array holds value for each column in projection. 30 | * Returned values should have the same type as corresponding columns. 31 | * 32 | * Can return null array. 33 | * 34 | * @param buffer wrapped byte buffer 35 | * @return list of values 36 | */ 37 | Object[] processRecord(WrappedByteBuf buffer); 38 | } 39 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/record/ScanRecordMaterializer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.record; 18 | 19 | import com.github.sadikovi.netflowlib.predicate.Columns.Column; 20 | import com.github.sadikovi.netflowlib.util.WrappedByteBuf; 21 | 22 | /** 23 | * [[ScanRecordMaterializer]] is part of [[FullScan]] strategy where we only scan pruned columns. 24 | * Note that columns might not be unique, e.g. [IntColumn("a"), ShortColumn("b"), IntColumn("a"), 25 | * LongColumn("c")]. 26 | */ 27 | public final class ScanRecordMaterializer implements RecordMaterializer { 28 | public ScanRecordMaterializer(Column[] columns) { 29 | this.columns = columns; 30 | numColumns = columns.length; 31 | } 32 | 33 | @Override 34 | public Object[] processRecord(WrappedByteBuf buffer) { 35 | Object[] newRecord = new Object[numColumns]; 36 | for (int i = 0; i < numColumns; i++) { 37 | newRecord[i] = columns[i].readField(buffer); 38 | } 39 | return newRecord; 40 | } 41 | 42 | private final Column[] columns; 43 | private final int numColumns; 44 | } 45 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/statistics/Statistics.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.statistics; 18 | 19 | import java.io.Serializable; 20 | 21 | public abstract interface Statistics { 22 | public Object getMin(); 23 | 24 | public Object getMax(); 25 | } 26 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/statistics/StatisticsTypes.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.statistics; 18 | 19 | import java.io.Serializable; 20 | 21 | public final class StatisticsTypes { 22 | private StatisticsTypes() { } 23 | 24 | /** 25 | * [[GenericStatistics]] interface to keep track of minimum and maximum values. 26 | */ 27 | public static abstract class GenericStatistics> 28 | implements Statistics, Serializable { 29 | GenericStatistics(T min, T max) { 30 | if (min.compareTo(max) > 0) { 31 | throw new IllegalArgumentException("Min " + min + " is greater than max " + max); 32 | } 33 | 34 | this.min = min; 35 | this.max = max; 36 | } 37 | 38 | @Override 39 | public T getMin() { 40 | return min; 41 | } 42 | 43 | @Override 44 | public T getMax() { 45 | return max; 46 | } 47 | 48 | private final T max; 49 | private final T min; 50 | } 51 | 52 | public static final class ByteStatistics extends GenericStatistics { 53 | public ByteStatistics(byte min, byte max) { 54 | super(min, max); 55 | } 56 | } 57 | 58 | public static final class ShortStatistics extends GenericStatistics { 59 | public ShortStatistics(short min, short max) { 60 | super(min, max); 61 | } 62 | } 63 | 64 | public static final class IntStatistics extends GenericStatistics { 65 | public IntStatistics(int min, int max) { 66 | super(min, max); 67 | } 68 | } 69 | 70 | public static final class LongStatistics extends GenericStatistics { 71 | public LongStatistics(long min, long max) { 72 | super(min, max); 73 | } 74 | } 75 | } 76 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/util/FilterIterator.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.util; 18 | 19 | import java.util.Iterator; 20 | import java.util.NoSuchElementException; 21 | 22 | /** 23 | * [[FilterIterator]] is a wrapper on standard iterator interface, allows to filter out "null" 24 | * values. Instead of implementing `hasNext()` and `next()` methods, caller needs to implement 25 | * internal methods to provide original stream of values. When iterator is called, standard methods 26 | * will be called with filter already. 27 | */ 28 | public class FilterIterator implements Iterator { 29 | public FilterIterator(Iterator iterator) { 30 | this.iterator = iterator; 31 | } 32 | 33 | @Override 34 | public final boolean hasNext() { 35 | while (!found && iterator.hasNext()) { 36 | foundItem = iterator.next(); 37 | if (foundItem != null) { 38 | found = true; 39 | } 40 | } 41 | return found; 42 | } 43 | 44 | @Override 45 | public final E next() { 46 | if (!found) { 47 | throw new NoSuchElementException("Either iterator is empty, or iterator state has not " + 48 | "been updated"); 49 | } 50 | 51 | if (foundItem == null) { 52 | throw new IllegalStateException("Potential out of sync error in " + 53 | getClass().getName()); 54 | } 55 | 56 | found = false; 57 | return foundItem; 58 | } 59 | 60 | @Override 61 | public void remove() { 62 | throw new UnsupportedOperationException("Operation 'remove' is not supported"); 63 | } 64 | 65 | private E foundItem = null; 66 | private boolean found = false; 67 | private final Iterator iterator; 68 | } 69 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/util/ReadAheadInputStream.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.util; 18 | 19 | import java.io.IOException; 20 | import java.io.InputStream; 21 | import java.util.zip.Inflater; 22 | import java.util.zip.InflaterInputStream; 23 | 24 | /** 25 | * [[ReadAheadInputStream]] interface is a simple wrapper around InflaterInputStream. Provides 26 | * convinient methods to check end of stream through inflater availability, even when compressed 27 | * stream is empty. Standard methods `available()` method can be used to check if stream is 28 | * finished. 29 | * See: https://docs.oracle.com/javase/7/docs/api/java/util/zip/InflaterInputStream.html 30 | */ 31 | public class ReadAheadInputStream extends InflaterInputStream { 32 | public ReadAheadInputStream(InputStream in, Inflater inf, int size) { 33 | super(in, inf, size); 34 | int maybeFirstByte; 35 | try { 36 | maybeFirstByte = super.read(); 37 | } catch (IOException ioe) { 38 | throw new UnsupportedOperationException( 39 | "Unexpected EOF when reading first bytes, might indicate corrupt input", ioe); 40 | } 41 | 42 | // We always use first byte offset for the second read 43 | useFirstByteOffset = true; 44 | // If `maybeFirstByte` returns -1, we have reached EOF, after this point `super.available()` 45 | // will report stream status correctly 46 | isEOF = maybeFirstByte == -1; 47 | // First byte is considered to be unsigned, but we can still assign 255 as -1 byte, or -1 48 | // directly, since it will be EOF in latter case anyway. 49 | firstByte = (byte) maybeFirstByte; 50 | } 51 | 52 | /** Check if stream is still open */ 53 | private void ensureOpen() throws IOException { 54 | if (closed) { 55 | throw new IOException("Stream closed"); 56 | } 57 | } 58 | 59 | @Override 60 | public int read(byte[] b, int off, int len) throws IOException { 61 | ensureOpen(); 62 | if (b == null) { 63 | throw new NullPointerException(); 64 | } else if (off < 0 || len < 0 || len > b.length - off) { 65 | throw new IndexOutOfBoundsException(); 66 | } else if (len == 0) { 67 | return 0; 68 | } 69 | // If offset is not used yet, we correct read buffer by overwriting first byte and fetching 70 | // less data from the stream. Note that we have to return correct number of bytes read including 71 | // first byte. 72 | if (useFirstByteOffset) { 73 | useFirstByteOffset = false; 74 | b[off] = firstByte; 75 | return super.read(b, off + 1, len - 1) + 1; 76 | } else { 77 | return super.read(b, off, len); 78 | } 79 | } 80 | 81 | @Override 82 | public int available() throws IOException { 83 | // `available()` returns 0, if EOF has been reached, but in case of empty compressed stream, it 84 | // is unclear, and `available()` will return 1. 85 | // for `ReadAheadInputStream` availability is determined either by first byte offset, whether 86 | // or read was successful or based on `finished()` method of Inflater instance. 87 | // Note that `available()` does not reset `useFirstByteOffset`. 88 | if (useFirstByteOffset) { 89 | return isEOF ? 0 : 1; 90 | } else { 91 | return inf.finished() ? 0 : 1; 92 | } 93 | } 94 | 95 | @Override 96 | public long skip(long n) throws IOException { 97 | if (n < 0) { 98 | throw new IllegalArgumentException("negative skip length"); 99 | } 100 | 101 | ensureOpen(); 102 | if (n == 0) { 103 | return 0; 104 | } 105 | // Similar to `read()` we have to make a correction to the fact that we have used first byte. 106 | // Though bytes skipped are less than provided, actual returned number of bytes is corrected by 107 | // the offset. 108 | if (useFirstByteOffset) { 109 | useFirstByteOffset = false; 110 | return super.skip(n - 1) + 1; 111 | } else { 112 | return super.skip(n); 113 | } 114 | } 115 | 116 | @Override 117 | public void close() throws IOException { 118 | if (!closed) { 119 | inf.end(); 120 | in.close(); 121 | closed = true; 122 | } 123 | } 124 | 125 | // Flag to indicate whether or not it is end of stream, syncronized with parent 126 | private boolean isEOF = false; 127 | // Flag to show whether or not underlying stream is closed 128 | private boolean closed = false; 129 | // Flag to indicate if we need to take offset into account 130 | private boolean useFirstByteOffset; 131 | // When `useFirstByteOffset` we have to include first byte read 132 | private byte firstByte; 133 | } 134 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/util/SafeIterator.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.util; 18 | 19 | import java.util.Iterator; 20 | import java.util.NoSuchElementException; 21 | 22 | /** 23 | * Iterator implementation that terminates delegate iterator when exception occurs during value 24 | * extraction. 25 | */ 26 | public class SafeIterator implements Iterator { 27 | private boolean gotNext = false; 28 | private E nextValue = null; 29 | protected boolean finished = false; 30 | private Iterator delegate; 31 | 32 | public SafeIterator(Iterator delegate) { 33 | this.delegate = delegate; 34 | } 35 | 36 | /** 37 | * If no next element is available, `finished` is set to `true` and may return any value 38 | * (it will be ignored). This convention is required because `null` may be a valid value. 39 | * @return E instance, or set 'finished' to `true` when done 40 | */ 41 | private E getNext() { 42 | try { 43 | if (delegate.hasNext()) { 44 | return delegate.next(); 45 | } else { 46 | finished = true; 47 | return null; 48 | } 49 | } catch (Exception err) { 50 | finished = true; 51 | return null; 52 | } 53 | } 54 | 55 | @Override 56 | public boolean hasNext() { 57 | if (!finished) { 58 | if (!gotNext) { 59 | nextValue = getNext(); 60 | gotNext = true; 61 | } 62 | } 63 | return !finished; 64 | } 65 | 66 | @Override 67 | public E next() { 68 | if (!hasNext()) { 69 | throw new NoSuchElementException("End of stream"); 70 | } 71 | gotNext = false; 72 | return nextValue; 73 | } 74 | 75 | @Override 76 | public void remove() { 77 | throw new UnsupportedOperationException("Operation 'remove' is not supported"); 78 | } 79 | } 80 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/util/WrappedByteBuf.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.util; 18 | 19 | import java.nio.ByteOrder; 20 | 21 | /** 22 | * Simple alternative to java.nio.ByteBuffer with methods to read unsigned values similar to 23 | * netty ByteBuf. 24 | */ 25 | public abstract class WrappedByteBuf { 26 | protected final byte[] data; 27 | 28 | /** 29 | * Create wrapped byte buffer for provided byte array and required endianness of bytes in array. 30 | * @param data byte array 31 | * @param order endianness of the data in array 32 | * @return wrapped big/little endian byte buffer 33 | */ 34 | public static WrappedByteBuf init(byte[] data, ByteOrder order) { 35 | return (order == ByteOrder.BIG_ENDIAN) ? new WrappedByteBufB(data) : new WrappedByteBufL(data); 36 | } 37 | 38 | // only keep reference to the data 39 | protected WrappedByteBuf(byte[] data) { 40 | this.data = data; 41 | } 42 | 43 | /** 44 | * Gets a byte at the specified absolute index in this buffer. 45 | * @param ordinal start position in buffer 46 | * @return byte value 47 | */ 48 | public abstract byte getByte(int ordinal); 49 | 50 | /** 51 | * Gets an unsigned byte at the specified absolute index in this buffer. 52 | * @param ordinal start position in buffer 53 | * @return short value that represents unsigned byte 54 | */ 55 | public abstract short getUnsignedByte(int ordinal); 56 | 57 | /** 58 | * Get short at the specified absolute index in this buffer. 59 | * @param ordinal start position in buffer 60 | * @return short value 61 | */ 62 | public abstract short getShort(int ordinal); 63 | 64 | /** 65 | * Gets an unsigned 16-bit short integer at the specified absolute index in this buffer. 66 | * @param ordinal start position in buffer 67 | * @return int value that represents unsigned short 68 | */ 69 | public abstract int getUnsignedShort(int ordinal); 70 | 71 | /** 72 | * Get int at the specified absolute index in this buffer. 73 | * @param ordinal start position in buffer 74 | * @return int value 75 | */ 76 | public abstract int getInt(int ordinal); 77 | 78 | /** 79 | * Gets an unsigned 32-bit integer at the specified absolute index in this buffer. 80 | * @param ordinal start position in buffer 81 | * @return long value that represents unsigned int 82 | */ 83 | public abstract long getUnsignedInt(int ordinal); 84 | 85 | /** 86 | * Transfers this buffer's data to the specified destination starting at the specified 87 | * absolute index. 88 | * @param index start ordinal of this buffer 89 | * @param dst destination byte array 90 | * @param dstIndex start index of dst array 91 | * @param length number of bytes to transfer 92 | */ 93 | public abstract void getBytes(int index, byte[] dst, int dstIndex, int length); 94 | 95 | /** 96 | * Return reference to the underlying array. 97 | * @return backed byte array 98 | */ 99 | public byte[] array() { 100 | return data; 101 | } 102 | 103 | /** Wrapped byte buffer for little endianness */ 104 | static class WrappedByteBufL extends WrappedByteBuf { 105 | protected WrappedByteBufL(byte[] data) { 106 | super(data); 107 | } 108 | 109 | @Override 110 | public byte getByte(int ordinal) { 111 | return data[ordinal]; 112 | } 113 | 114 | @Override 115 | public short getUnsignedByte(int ordinal) { 116 | return (short) (data[ordinal] & 0xff); 117 | } 118 | 119 | @Override 120 | public short getShort(int ordinal) { 121 | return (short) (data[ordinal + 1] << 8 | data[ordinal] & 0xff); 122 | } 123 | 124 | @Override 125 | public int getUnsignedShort(int ordinal) { 126 | return ((data[ordinal + 1] & 0xff) << 8) | (data[ordinal] & 0xff); 127 | } 128 | 129 | @Override 130 | public int getInt(int ordinal) { 131 | return ((data[ordinal + 3] & 0xff) << 24) | 132 | ((data[ordinal + 2] & 0xff) << 16) | 133 | ((data[ordinal + 1] & 0xff) << 8) | 134 | (data[ordinal + 0] & 0xff); 135 | } 136 | 137 | @Override 138 | public long getUnsignedInt(int ordinal) { 139 | return getInt(ordinal) & 0xffffffffL; 140 | } 141 | 142 | @Override 143 | public void getBytes(int index, byte[] dst, int dstIndex, int length) { 144 | System.arraycopy(data, index, dst, dstIndex, length); 145 | } 146 | } 147 | 148 | /** Wrapped byte buffer for big endianness */ 149 | static class WrappedByteBufB extends WrappedByteBuf { 150 | protected WrappedByteBufB(byte[] data) { 151 | super(data); 152 | } 153 | 154 | @Override 155 | public byte getByte(int ordinal) { 156 | return data[ordinal]; 157 | } 158 | 159 | @Override 160 | public short getUnsignedByte(int ordinal) { 161 | return (short) (data[ordinal] & 0xff); 162 | } 163 | 164 | @Override 165 | public short getShort(int ordinal) { 166 | return (short) (data[ordinal] << 8 | data[ordinal + 1] & 0xff); 167 | } 168 | 169 | @Override 170 | public int getUnsignedShort(int ordinal) { 171 | return ((data[ordinal] & 0xff) << 8) | (data[ordinal + 1] & 0xff); 172 | } 173 | 174 | @Override 175 | public int getInt(int ordinal) { 176 | return ((data[ordinal] & 0xff) << 24) | 177 | ((data[ordinal + 1] & 0xff) << 16) | 178 | ((data[ordinal + 2] & 0xff) << 8) | 179 | (data[ordinal + 3] & 0xff); 180 | } 181 | 182 | @Override 183 | public long getUnsignedInt(int ordinal) { 184 | return getInt(ordinal) & 0xffffffffL; 185 | } 186 | 187 | @Override 188 | public void getBytes(int index, byte[] dst, int dstIndex, int length) { 189 | System.arraycopy(data, index, dst, dstIndex, length); 190 | } 191 | } 192 | } 193 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/version/NetFlow.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.version; 18 | 19 | /** 20 | * Updated [[NetFlow]] implementation, defines essential methods for subclasses of different NetFlow 21 | * versions, also provides default data checks. 22 | */ 23 | public abstract class NetFlow { 24 | 25 | /** Record size in bytes for a particular version */ 26 | public abstract int recordSize(); 27 | } 28 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/version/NetFlowV5.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.version; 18 | 19 | import com.github.sadikovi.netflowlib.predicate.Columns.IntColumn; 20 | import com.github.sadikovi.netflowlib.predicate.Columns.LongColumn; 21 | import com.github.sadikovi.netflowlib.predicate.Columns.ShortColumn; 22 | 23 | public class NetFlowV5 extends NetFlow { 24 | // list of supported columns and size in bytes 25 | // Current seconds since 0000 UTC 1970, size: 4 26 | public static final LongColumn FIELD_UNIX_SECS = new LongColumn("unix_secs", 0); 27 | // Residual nanoseconds since 0000 UTC 1970, size: 4 28 | public static final LongColumn FIELD_UNIX_NSECS = new LongColumn("unix_nsecs", 4); 29 | // Current time in millisecs since router booted, size: 4 30 | public static final LongColumn FIELD_SYSUPTIME = new LongColumn("sys_uptime", 8); 31 | // Exporter IP address, size: 4 32 | public static final LongColumn FIELD_EXADDR = new LongColumn("export_ip", 12); 33 | // Source IP Address, size: 4 34 | public static final LongColumn FIELD_SRCADDR = new LongColumn("srcip", 16); 35 | // Destination IP Address, size: 4 36 | public static final LongColumn FIELD_DSTADDR = new LongColumn("dstip", 20); 37 | // Next hop router's IP Address, size: 4 38 | public static final LongColumn FIELD_NEXTHOP = new LongColumn("nexthop", 24); 39 | // Input interface index (known as Sif), size: 2 40 | public static final IntColumn FIELD_INPUT = new IntColumn("input", 28); 41 | // Output interface index (known as Dif), size: 2 42 | public static final IntColumn FIELD_OUTPUT = new IntColumn("output", 30); 43 | // Packets sent in Duration, size: 4 44 | public static final LongColumn FIELD_DPKTS = new LongColumn("packets", 32); 45 | // Octets sent in Duration, size: 4 46 | public static final LongColumn FIELD_DOCTETS = new LongColumn("octets", 36); 47 | // SysUptime at start of flow, size: 4 48 | public static final LongColumn FIELD_FIRST = new LongColumn("first", 40); 49 | // and of last packet of flow, size: 4 50 | public static final LongColumn FIELD_LAST = new LongColumn("last", 44); 51 | // TCP/UDP source port number or equivalent, size: 2 52 | public static final IntColumn FIELD_SRCPORT = new IntColumn("srcport", 48); 53 | // TCP/UDP destination port number or equiv, size: 2 54 | public static final IntColumn FIELD_DSTPORT = new IntColumn("dstport", 50); 55 | // IP protocol, e.g., 6=TCP, 17=UDP, ..., size: 1 56 | public static final ShortColumn FIELD_PROT = new ShortColumn("protocol", 52); 57 | // IP Type-of-Service, size: 1 58 | public static final ShortColumn FIELD_TOS = new ShortColumn("tos", 53); 59 | // OR of TCP header bits, size: 1 60 | public static final ShortColumn FIELD_TCP_FLAGS = new ShortColumn("tcp_flags", 54); 61 | // Type of flow switching engine (RP, VIP, etc.), size: 1 62 | // There is field "pad" which is unused byte in record, we skip it 63 | public static final ShortColumn FIELD_ENGINE_TYPE = new ShortColumn("engine_type", 56); 64 | // Slot number of the flow switching engine, size: 1 65 | public static final ShortColumn FIELD_ENGINE_ID = new ShortColumn("engine_id", 57); 66 | // mask length of source address, size: 1 67 | public static final ShortColumn FIELD_SRC_MASK = new ShortColumn("src_mask", 58); 68 | // mask length of destination address, size: 1 69 | public static final ShortColumn FIELD_DST_MASK = new ShortColumn("dst_mask", 59); 70 | // AS of source address, size: 2 71 | public static final IntColumn FIELD_SRC_AS = new IntColumn("src_as", 60); 72 | // AS of destination address, size: 2 73 | public static final IntColumn FIELD_DST_AS = new IntColumn("dst_as", 62); 74 | 75 | public NetFlowV5() { } 76 | 77 | @Override 78 | public int recordSize() { 79 | return 64; 80 | } 81 | } 82 | -------------------------------------------------------------------------------- /src/main/java/com/github/sadikovi/netflowlib/version/NetFlowV7.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.version; 18 | 19 | import com.github.sadikovi.netflowlib.predicate.Columns.IntColumn; 20 | import com.github.sadikovi.netflowlib.predicate.Columns.LongColumn; 21 | import com.github.sadikovi.netflowlib.predicate.Columns.ShortColumn; 22 | 23 | public class NetFlowV7 extends NetFlow { 24 | // list of supported columns and size in bytes 25 | // Current seconds since 0000 UTC 1970, size: 4 26 | public static final LongColumn FIELD_UNIX_SECS = new LongColumn("unix_secs", 0); 27 | // Residual nanoseconds since 0000 UTC 1970, size: 4 28 | public static final LongColumn FIELD_UNIX_NSECS = new LongColumn("unix_nsecs", 4); 29 | // Current time in millisecs since router booted, size: 4 30 | public static final LongColumn FIELD_SYSUPTIME = new LongColumn("sys_uptime", 8); 31 | // Exporter IP address, size: 4 32 | public static final LongColumn FIELD_EXADDR = new LongColumn("export_ip", 12); 33 | // Source IP Address, size: 4 34 | public static final LongColumn FIELD_SRCADDR = new LongColumn("srcip", 16); 35 | // Destination IP Address, size: 4 36 | public static final LongColumn FIELD_DSTADDR = new LongColumn("dstip", 20); 37 | // Next hop router's IP Address, size: 4 38 | public static final LongColumn FIELD_NEXTHOP = new LongColumn("nexthop", 24); 39 | // Input interface index, size: 2 40 | public static final IntColumn FIELD_INPUT = new IntColumn("input", 28); 41 | // Output interface index, size: 2 42 | public static final IntColumn FIELD_OUTPUT = new IntColumn("output", 30); 43 | // Packets sent in Duration, size: 4 44 | public static final LongColumn FIELD_DPKTS = new LongColumn("packets", 32); 45 | // Octets sent in Duration, size: 4 46 | public static final LongColumn FIELD_DOCTETS = new LongColumn("octets", 36); 47 | // SysUptime at start of flow, size: 4 48 | public static final LongColumn FIELD_FIRST = new LongColumn("first", 40); 49 | // and of last packet of flow, size: 4 50 | public static final LongColumn FIELD_LAST = new LongColumn("last", 44); 51 | // TCP/UDP source port number or equivalent, size: 2 52 | public static final IntColumn FIELD_SRCPORT = new IntColumn("srcport", 48); 53 | // TCP/UDP destination port number or equiv, size: 2 54 | public static final IntColumn FIELD_DSTPORT = new IntColumn("dstport", 50); 55 | // IP protocol, e.g., 6=TCP, 17=UDP, ..., size: 1 56 | public static final ShortColumn FIELD_PROT = new ShortColumn("protocol", 52); 57 | // IP Type-of-Service, size: 1 58 | public static final ShortColumn FIELD_TOS = new ShortColumn("tos", 53); 59 | // OR of TCP header bits, size: 1 60 | public static final ShortColumn FIELD_TCP_FLAGS = new ShortColumn("tcp_flags", 54); 61 | // Reason flow discarded, etc, size: 1 62 | public static final ShortColumn FIELD_FLAGS = new ShortColumn("flags", 55); 63 | // Type of flow switching engine (RP, VIP, etc.), size: 1 64 | public static final ShortColumn FIELD_ENGINE_TYPE = new ShortColumn("engine_type", 56); 65 | // Slot number of the flow switching engine, size: 1 66 | public static final ShortColumn FIELD_ENGINE_ID = new ShortColumn("engine_id", 57); 67 | // mask length of source address, size: 1 68 | public static final ShortColumn FIELD_SRC_MASK = new ShortColumn("src_mask", 58); 69 | // mask length of destination address, size: 1 70 | public static final ShortColumn FIELD_DST_MASK = new ShortColumn("dst_mask", 59); 71 | // AS of source address, size: 2 72 | public static final IntColumn FIELD_SRC_AS = new IntColumn("src_as", 60); 73 | // AS of destination address, size: 2 74 | public static final IntColumn FIELD_DST_AS = new IntColumn("dst_as", 62); 75 | // ID of router shortcut by switch, size: 4 76 | public static final LongColumn FIELD_ROUTER_SC = new LongColumn("router_sc", 64); 77 | 78 | public NetFlowV7() { } 79 | 80 | @Override 81 | public int recordSize() { 82 | return 68; 83 | } 84 | } 85 | -------------------------------------------------------------------------------- /src/main/scala/com/github/sadikovi/spark/benchmark/Benchmark.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.spark.benchmark 18 | 19 | import java.io.{OutputStream, PrintStream} 20 | 21 | import scala.collection.mutable 22 | import scala.concurrent.duration._ 23 | import scala.sys.process.Process 24 | import scala.util.Try 25 | 26 | import org.apache.commons.io.output.TeeOutputStream 27 | import org.apache.commons.lang3.SystemUtils 28 | 29 | /** 30 | * Copy from "apache/spark/master/core/src/main/scala/org/apache/spark/util/Benchmark.scala" with 31 | * some minor changes, because class is private to spark package. 32 | * 33 | * Utility class to benchmark components. An example of how to use this is: 34 | * val benchmark = new Benchmark("My Benchmark", valuesPerIteration) 35 | * benchmark.addCase("V1")() 36 | * benchmark.addCase("V2")() 37 | * benchmark.run 38 | * This will output the average time to run each function and the rate of each function. 39 | */ 40 | private[sadikovi] class Benchmark( 41 | name: String, 42 | valuesPerIteration: Long, 43 | minNumIters: Int = 5, 44 | warmupTime: FiniteDuration = 2.seconds, 45 | minTime: FiniteDuration = 2.seconds, 46 | outputPerIteration: Boolean = false, 47 | output: Option[OutputStream] = None) { 48 | import Benchmark._ 49 | val benchmarks = mutable.ArrayBuffer.empty[Benchmark.Case] 50 | 51 | val out = if (output.isDefined) { 52 | new PrintStream(new TeeOutputStream(System.out, output.get)) 53 | } else { 54 | System.out 55 | } 56 | 57 | /** 58 | * Adds a case to run when run() is called. The given function will be run for several 59 | * iterations to collect timing statistics. 60 | * 61 | * @param name of the benchmark case 62 | * @param numIters if non-zero, forces exactly this many iterations to be run 63 | */ 64 | def addCase(name: String, numIters: Int = 0)(f: Int => Unit): Unit = { 65 | addTimerCase(name, numIters) { timer => 66 | timer.startTiming() 67 | f(timer.iteration) 68 | timer.stopTiming() 69 | } 70 | } 71 | 72 | /** 73 | * Adds a case with manual timing control. When the function is run, timing does not start 74 | * until timer.startTiming() is called within the given function. The corresponding 75 | * timer.stopTiming() method must be called before the function returns. 76 | * 77 | * @param name of the benchmark case 78 | * @param numIters if non-zero, forces exactly this many iterations to be run 79 | */ 80 | def addTimerCase(name: String, numIters: Int = 0)(f: Benchmark.Timer => Unit): Unit = { 81 | benchmarks += Benchmark.Case(name, f, numIters) 82 | } 83 | 84 | /** 85 | * Runs the benchmark and outputs the results to stdout. This should be copied and added as 86 | * a comment with the benchmark. Although the results vary from machine to machine, it should 87 | * provide some baseline. 88 | */ 89 | def run(): Unit = { 90 | require(benchmarks.nonEmpty) 91 | // scalastyle:off 92 | println("Running benchmark: " + name) 93 | 94 | val results = benchmarks.map { c => 95 | println(" Running case: " + c.name) 96 | measure(valuesPerIteration, c.numIters)(c.fn) 97 | } 98 | println 99 | 100 | val firstBest = results.head.bestMs 101 | // The results are going to be processor specific so it is useful to include that. 102 | out.println(Benchmark.getJVMOSInfo()) 103 | out.println(Benchmark.getProcessorName()) 104 | out.printf("%-40s %16s %12s %13s %10s\n", name + ":", "Best/Avg Time(ms)", "Rate(M/s)", 105 | "Per Row(ns)", "Relative") 106 | out.println("-" * 96) 107 | results.zip(benchmarks).foreach { case (result, benchmark) => 108 | out.printf("%-40s %16s %12s %13s %10s\n", 109 | benchmark.name, 110 | "%5.0f / %4.0f" format (result.bestMs, result.avgMs), 111 | "%10.1f" format result.bestRate, 112 | "%6.1f" format (1000 / result.bestRate), 113 | "%3.1fX" format (firstBest / result.bestMs)) 114 | } 115 | out.println 116 | // scalastyle:on 117 | } 118 | 119 | /** 120 | * Runs a single function `f` for iters, returning the average time the function took and 121 | * the rate of the function. 122 | */ 123 | def measure(num: Long, overrideNumIters: Int)(f: Timer => Unit): Result = { 124 | System.gc() // ensures garbage from previous cases don't impact this one 125 | val warmupDeadline = warmupTime.fromNow 126 | while (!warmupDeadline.isOverdue) { 127 | f(new Benchmark.Timer(-1)) 128 | } 129 | val minIters = if (overrideNumIters != 0) overrideNumIters else minNumIters 130 | val minDuration = if (overrideNumIters != 0) 0 else minTime.toNanos 131 | val runTimes = mutable.ArrayBuffer[Long]() 132 | var i = 0 133 | while (i < minIters || runTimes.sum < minDuration) { 134 | val timer = new Benchmark.Timer(i) 135 | f(timer) 136 | val runTime = timer.totalTime() 137 | runTimes += runTime 138 | 139 | if (outputPerIteration) { 140 | // scalastyle:off 141 | println(s"Iteration $i took ${runTime / 1000} microseconds") 142 | // scalastyle:on 143 | } 144 | i += 1 145 | } 146 | // scalastyle:off 147 | println(s" Stopped after $i iterations, ${runTimes.sum / 1000000} ms") 148 | // scalastyle:on 149 | val best = runTimes.min 150 | val avg = runTimes.sum / runTimes.size 151 | Result(avg / 1000000.0, num / (best / 1000.0), best / 1000000.0) 152 | } 153 | } 154 | 155 | private[sadikovi] object Benchmark { 156 | 157 | /** 158 | * Object available to benchmark code to control timing e.g. to exclude set-up time. 159 | * 160 | * @param iteration specifies this is the nth iteration of running the benchmark case 161 | */ 162 | class Timer(val iteration: Int) { 163 | private var accumulatedTime: Long = 0L 164 | private var timeStart: Long = 0L 165 | 166 | def startTiming(): Unit = { 167 | assert(timeStart == 0L, "Already started timing.") 168 | timeStart = System.nanoTime 169 | } 170 | 171 | def stopTiming(): Unit = { 172 | assert(timeStart != 0L, "Have not started timing.") 173 | accumulatedTime += System.nanoTime - timeStart 174 | timeStart = 0L 175 | } 176 | 177 | def totalTime(): Long = { 178 | assert(timeStart == 0L, "Have not stopped timing.") 179 | accumulatedTime 180 | } 181 | } 182 | 183 | case class Case(name: String, fn: Timer => Unit, numIters: Int) 184 | case class Result(avgMs: Double, bestRate: Double, bestMs: Double) 185 | 186 | /** Execute command as sequence, block until get result and return output */ 187 | private def executeAndGetOutput(command: Seq[String]): String = { 188 | Process(command).!!.trim() 189 | } 190 | 191 | /** 192 | * This should return a user helpful processor information. Getting at this depends on the OS. 193 | * This should return something like "Intel(R) Core(TM) i7-4870HQ CPU @ 2.50GHz" 194 | */ 195 | def getProcessorName(): String = { 196 | val cpu = if (SystemUtils.IS_OS_MAC_OSX) { 197 | executeAndGetOutput(Seq("/usr/sbin/sysctl", "-n", "machdep.cpu.brand_string")) 198 | } else if (SystemUtils.IS_OS_LINUX) { 199 | Try { 200 | val grepPath = executeAndGetOutput(Seq("which", "grep")).stripLineEnd 201 | executeAndGetOutput(Seq(grepPath, "-m", "1", "model name", "/proc/cpuinfo")) 202 | .stripLineEnd.replaceFirst("model name[\\s*]:[\\s*]", "") 203 | }.getOrElse("Unknown processor") 204 | } else { 205 | System.getenv("PROCESSOR_IDENTIFIER") 206 | } 207 | cpu 208 | } 209 | 210 | /** 211 | * This should return a user helpful JVM & OS information. 212 | * This should return something like 213 | * "OpenJDK 64-Bit Server VM 1.8.0_65-b17 on Linux 4.1.13-100.fc21.x86_64" 214 | */ 215 | def getJVMOSInfo(): String = { 216 | val vmName = System.getProperty("java.vm.name") 217 | val runtimeVersion = System.getProperty("java.runtime.version") 218 | val osName = System.getProperty("os.name") 219 | val osVersion = System.getProperty("os.version") 220 | s"${vmName} ${runtimeVersion} on ${osName} ${osVersion}" 221 | } 222 | } 223 | -------------------------------------------------------------------------------- /src/main/scala/com/github/sadikovi/spark/benchmark/NetFlowReadBenchmark.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.spark.benchmark 18 | 19 | import java.util.{HashMap => JHashMap} 20 | 21 | import org.apache.spark.SparkConf 22 | import org.apache.spark.sql.SparkSession 23 | import org.apache.spark.sql.functions.col 24 | 25 | /** Configuration option for cli */ 26 | private case class ConfOption(name: String) 27 | 28 | /** Configuration map for cli */ 29 | private case class Conf() { 30 | private val conf: JHashMap[ConfOption, String] = new JHashMap[ConfOption, String]() 31 | 32 | def addOption(opt: ConfOption, value: String): Unit = conf.put(opt, value) 33 | 34 | def get(opt: ConfOption): Option[String] = Option(conf.get(opt)) 35 | } 36 | 37 | /** 38 | * NetFlow benchmarks divided into several categories: 39 | * - full scan without any predicate with field conversion and without 40 | * (result is cached and counted) 41 | * - predicate scan with predicate pushdown and without (result is aggregated by protocol) 42 | * - aggregated scan with predicate pushdown trying to replicate report 43 | * (result is cached and counted) 44 | */ 45 | object NetFlowReadBenchmark { 46 | // Required options 47 | private val ITERATIONS = ConfOption("--iterations") 48 | private val FILES = ConfOption("--files") 49 | private val VERSION = ConfOption("--version") 50 | 51 | // Initialize Spark context 52 | val sparkConf = new SparkConf(). 53 | setMaster("local[1]"). 54 | setAppName("spark-netflow-benchmark") 55 | val spark = SparkSession.builder().config(sparkConf).getOrCreate() 56 | 57 | def main(args: Array[String]): Unit = { 58 | val conf = process(args.toList, Conf()) 59 | 60 | // Extract options 61 | val iterations = conf.get(ITERATIONS).getOrElse( 62 | sys.error("Number of iterations must be specified, e.g. '--iterations 3'")).toInt 63 | val files = conf.get(FILES).getOrElse( 64 | sys.error("Files / directory must be specified, e.g. '--files /tmp/files'")) 65 | val version = conf.get(VERSION).getOrElse( 66 | sys.error("NetFlow version must be specified, e.g. '--version 5'")) 67 | 68 | // scalastyle:off 69 | println(s"- Iterations: $iterations") 70 | println(s"- Files: $files") 71 | println(s"- Version: $version") 72 | // scalastyle:on 73 | 74 | // Defined benchmarks 75 | fullScanBenchmark(iterations, version, files) 76 | predicateScanBenchmark(iterations, version, files) 77 | aggregatedScanBenchmark(iterations, version, files) 78 | } 79 | 80 | private def process(args: List[String], conf: Conf): Conf = args match { 81 | case ITERATIONS.name :: value :: tail => 82 | conf.addOption(ITERATIONS, value) 83 | process(tail, conf) 84 | case FILES.name :: value :: tail => 85 | conf.addOption(FILES, value) 86 | process(tail, conf) 87 | case VERSION.name :: value :: tail => 88 | conf.addOption(VERSION, value) 89 | process(tail, conf) 90 | case other :: tail => process(tail, conf) 91 | case Nil => conf 92 | } 93 | 94 | /** Test full read of files provided with or without `stringify` option */ 95 | def fullScanBenchmark(iters: Int, version: String, files: String): Unit = { 96 | val sqlBenchmark = new Benchmark("NetFlow full scan", 10000, iters) 97 | 98 | sqlBenchmark.addCase("Scan, stringify = F") { iter => 99 | val df = spark.read.format("com.github.sadikovi.spark.netflow"). 100 | option("version", version).option("stringify", "false").load(files) 101 | df.foreach(_ => ()) 102 | } 103 | 104 | sqlBenchmark.addCase("Scan, stringify = T") { iter => 105 | val df = spark.read.format("com.github.sadikovi.spark.netflow"). 106 | option("version", version).option("stringify", "true").load(files) 107 | df.foreach(_ => ()) 108 | } 109 | 110 | sqlBenchmark.run() 111 | } 112 | 113 | /** Predicate scan benchmark, test high and low selectivity */ 114 | def predicateScanBenchmark(iters: Int, version: String, files: String): Unit = { 115 | val sqlBenchmark = new Benchmark("NetFlow predicate scan", 10000, iters) 116 | 117 | sqlBenchmark.addCase("Predicate pushdown = F, high") { iter => 118 | val df = spark.read.format("com.github.sadikovi.spark.netflow"). 119 | option("version", version).option("predicate-pushdown", "false").load(files). 120 | filter(col("srcport") =!= 10) 121 | df.foreach(_ => ()) 122 | } 123 | 124 | sqlBenchmark.addCase("Predicate pushdown = T, high") { iter => 125 | val df = spark.read.format("com.github.sadikovi.spark.netflow"). 126 | option("version", version).option("predicate-pushdown", "true").load(files). 127 | filter(col("srcport") =!= 10) 128 | df.foreach(_ => ()) 129 | } 130 | 131 | sqlBenchmark.addCase("Predicate pushdown = F, low") { iter => 132 | val df = spark.read.format("com.github.sadikovi.spark.netflow"). 133 | option("version", version).option("predicate-pushdown", "false").load(files). 134 | filter(col("srcip") === "127.0.0.1") 135 | df.foreach(_ => ()) 136 | } 137 | 138 | sqlBenchmark.addCase("Predicate pushdown = T, low") { iter => 139 | val df = spark.read.format("com.github.sadikovi.spark.netflow"). 140 | option("version", version).option("predicate-pushdown", "true").load(files). 141 | filter(col("srcip") === "127.0.0.1") 142 | df.foreach(_ => ()) 143 | } 144 | 145 | sqlBenchmark.run() 146 | } 147 | 148 | /** Run simple aggregation based with filtering */ 149 | def aggregatedScanBenchmark(iters: Int, version: String, files: String): Unit = { 150 | val sqlBenchmark = new Benchmark("NetFlow aggregated report", 10000, iters) 151 | 152 | sqlBenchmark.addCase("Aggregated report") { iter => 153 | val df = spark.read.format("com.github.sadikovi.spark.netflow"). 154 | option("version", version).load(files). 155 | filter(col("srcport") > 10). 156 | select("srcip", "dstip", "srcport", "dstport", "packets", "octets") 157 | 158 | val agg = df.groupBy(col("srcip"), col("dstip"), col("srcport"), col("dstport")).count() 159 | agg.foreach(_ => ()) 160 | } 161 | 162 | sqlBenchmark.run() 163 | } 164 | } 165 | -------------------------------------------------------------------------------- /src/main/scala/com/github/sadikovi/spark/netflow/NetFlowOptions.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.spark.netflow 18 | 19 | import com.github.sadikovi.netflowlib.Buffers.RecordBuffer 20 | import com.github.sadikovi.spark.util.Utils 21 | 22 | /** 23 | * [[NetFlowOptions]] consolidates all options available for NetFlow file format and can be 24 | * set as datasource options for `DataFrameReader`. 25 | */ 26 | private[netflow] class NetFlowOptions(options: Map[String, String]) extends Serializable { 27 | // Conversion of some supported numeric fields into string, such as IP, enabled by default 28 | val applyConversion = options.get("stringify") match { 29 | case Some("true") => true 30 | case Some("false") => false 31 | case _ => true 32 | } 33 | 34 | // Buffer size in bytes, by default use standard record buffer size ~1Mb 35 | val bufferSize = options.get("buffer") match { 36 | case Some(str) => 37 | val bytes = Utils.byteStringAsBytes(str) 38 | if (bytes > Integer.MAX_VALUE) { 39 | sys.error(s"Buffer size (${bytes}) bytes > maximum buffer size " + 40 | s"(${Integer.MAX_VALUE} bytes)") 41 | } else if (bytes < RecordBuffer.MIN_BUFFER_LENGTH) { 42 | sys.error(s"Buffer size (${bytes} bytes) < minimum buffer size " + 43 | s"(${RecordBuffer.MIN_BUFFER_LENGTH} bytes)") 44 | } else { 45 | bytes.toInt 46 | } 47 | case None => 48 | RecordBuffer.BUFFER_LENGTH_2 49 | } 50 | 51 | // Whether or not to use predicate pushdown at the NetFlow library level, enabled by default 52 | val usePredicatePushdown = options.get("predicate-pushdown") match { 53 | case Some("true") => true 54 | case Some("false") => false 55 | case _ => true 56 | } 57 | 58 | override def toString(): String = { 59 | s"${getClass.getSimpleName}(" + 60 | s"applyConversion=$applyConversion, " + 61 | s"bufferSize=$bufferSize, " + 62 | s"usePredicatePushdown=$usePredicatePushdown)" 63 | } 64 | } 65 | -------------------------------------------------------------------------------- /src/main/scala/com/github/sadikovi/spark/netflow/package.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.spark 18 | 19 | import org.apache.spark.sql.{SQLContext, DataFrameReader, DataFrame} 20 | 21 | package object netflow { 22 | 23 | /** 24 | * Adds a shortcut method `netflow()` to DataFrameReader that allows to omit specifying format 25 | * and specifies version by default. 26 | */ 27 | implicit class NetFlowDataFrameReader(reader: DataFrameReader) { 28 | def netflow5: String => DataFrame = { 29 | reader.format("com.github.sadikovi.spark.netflow"). 30 | option("version", DefaultSource.VERSION_5).load 31 | } 32 | 33 | def netflow7: String => DataFrame = { 34 | reader.format("com.github.sadikovi.spark.netflow"). 35 | option("version", DefaultSource.VERSION_7).load 36 | } 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /src/main/scala/com/github/sadikovi/spark/netflow/sources/NetFlowRegistry.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.spark.netflow.sources 18 | 19 | import scala.util.{Failure, Success, Try} 20 | 21 | import com.github.sadikovi.spark.util.Utils 22 | 23 | /** NetFlow registry to resolve provider for a specific version. */ 24 | object NetFlowRegistry { 25 | /** 26 | * Look up NetFlow interface, also resolve package to full class name, similar to Spark 27 | * datasource API. 28 | */ 29 | private[sources] def lookupInterface(provider: String): Class[_] = { 30 | val extendedProvider = s"$provider.DefaultProvider" 31 | val loader = Utils.getContextClassLoader() 32 | 33 | Try(loader.loadClass(provider)).orElse(Try(loader.loadClass(extendedProvider))) match { 34 | case Success(someInterface) => 35 | someInterface 36 | case Failure(error) => 37 | throw new ClassNotFoundException(s"Failed to find NetFlow interface for $provider", 38 | error) 39 | } 40 | } 41 | 42 | /** 43 | * Create interface from specified provider. 44 | * @param providerName name of the [[NetFlowProvider]] to load 45 | * @return resolved interface for specified provider 46 | */ 47 | def createInterface(providerName: String): ResolvedInterface = { 48 | val provider = lookupInterface(providerName).newInstance() match { 49 | case versionProvider: NetFlowProvider => versionProvider 50 | case _ => throw new UnsupportedOperationException( 51 | s"Provider $providerName does not support NetFlowProvider interface") 52 | } 53 | 54 | // Create and check consistency of the interface 55 | val createdInterface = provider.createInterface() 56 | createdInterface.ensureColumnConsistency() 57 | 58 | createdInterface 59 | } 60 | } 61 | 62 | trait NetFlowProvider { 63 | /** Return new `ResolvedInterface` instance. */ 64 | def createInterface(): ResolvedInterface 65 | } 66 | -------------------------------------------------------------------------------- /src/main/scala/com/github/sadikovi/spark/netflow/sources/ResolvedInterface.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.spark.netflow.sources 18 | 19 | import org.apache.spark.sql.types._ 20 | 21 | import com.github.sadikovi.netflowlib.predicate.Columns.Column 22 | 23 | /** 24 | * Representation of columns of Spark SQL into NetFlow fields. 25 | * @param columnName SQL name of the column (in most cases derived from internal column) 26 | * @param internalColumn internal NetFlow column 27 | * @param collectStatistics whether or not to collect statistics on column 28 | * @param convertFunction possible conversion (usually to human-readable format) 29 | */ 30 | private[spark] case class MappedColumn( 31 | columnName: String, 32 | internalColumn: Column, 33 | collectStatistics: Boolean, 34 | convertFunction: Option[ConvertFunction] 35 | ) 36 | 37 | /** Internal types, used to resolve SQL type for a column */ 38 | private[spark] object InternalType { 39 | val BYTE = classOf[java.lang.Byte] 40 | val SHORT = classOf[java.lang.Short] 41 | val INT = classOf[java.lang.Integer] 42 | val LONG = classOf[java.lang.Long] 43 | } 44 | 45 | /** Abstract interface for NetFlow version */ 46 | abstract class ResolvedInterface { 47 | 48 | /** Interface columns, sequence has to contain at least one column */ 49 | protected val columns: Seq[MappedColumn] 50 | 51 | /** Interface version. */ 52 | def version(): Short 53 | 54 | /** 55 | * Get Spark SQL schema for columns. If `applyConversion` then we specify `StringType` as column 56 | * type, since conversion is always "Any -> String". All fields are non-nullable. 57 | * @param applyConversion change type to `StringType` if true 58 | * @return Spark SQL schema 59 | */ 60 | def getSQLSchema(applyConversion: Boolean): StructType = { 61 | val sqlColumns = columns.map(column => { 62 | if (applyConversion && column.convertFunction.isDefined) { 63 | StructField(column.columnName, StringType, false) 64 | } else { 65 | StructField(column.columnName, javaToSQLType(column.internalColumn.getColumnType()), false) 66 | } 67 | }) 68 | StructType(sqlColumns) 69 | } 70 | 71 | /** Get all `MappedColumn` instances. */ 72 | def getColumns(): Seq[MappedColumn] = columns 73 | 74 | /** Get first `MappedColumn` (mostly used for count to avoid reading entire record). */ 75 | def getFirstColumn(): MappedColumn = columns.head 76 | 77 | /** Get first `MappedColumn` as `Option`. */ 78 | def getFirstColumnOption(): Option[MappedColumn] = columns.headOption 79 | 80 | /** Get columns with enabled statistics */ 81 | def getStatisticsColumns(): Seq[MappedColumn] = columns.filter { _.collectStatistics } 82 | 83 | /** Get `MappedColumn` for a specified column name. Fail, if column name is not present. */ 84 | def getColumn(columnName: String): MappedColumn = { 85 | columnsMap.getOrElse(columnName, 86 | sys.error(s"Interface does not have information about column $columnName")) 87 | } 88 | 89 | /** Get size in bytes for a particular SQL data type. */ 90 | private[sources] def sizeInBytes(dtype: DataType): Short = dtype match { 91 | case byte: ByteType => 1 92 | case short: ShortType => 2 93 | case int: IntegerType => 4 94 | case long: LongType => 8 95 | case other => throw new UnsupportedOperationException(s"Cannot get size for $other type") 96 | } 97 | 98 | /** Resolve internal type into SQL type */ 99 | private[sources] def javaToSQLType(clazz: Class[_]): DataType = clazz match { 100 | case InternalType.BYTE => ByteType 101 | case InternalType.SHORT => ShortType 102 | case InternalType.INT => IntegerType 103 | case InternalType.LONG => LongType 104 | case otherType => throw new UnsupportedOperationException(s"Unsupported type $otherType") 105 | } 106 | 107 | private[sources] def ensureColumnConsistency(): Unit = { 108 | if (columns.isEmpty) { 109 | throw new IllegalArgumentException(s"Columns are empty for ${toString()}") 110 | } 111 | // check that columns (SQL and internal) are not duplicated 112 | val columnNames = columns.map(_.columnName) 113 | assert(columnNames.length == columnNames.distinct.length, 114 | s"Found duplicate column names in ${toString()}") 115 | 116 | val internalColumnNames = columns.map(_.internalColumn) 117 | assert(internalColumnNames.length == internalColumnNames.distinct.length, 118 | s"Found duplicate internal column names in ${toString()}") 119 | } 120 | 121 | override def toString(): String = { 122 | s"Interface: ${getClass.getCanonicalName} for version ${version()}" 123 | } 124 | 125 | private lazy val columnsMap: Map[String, MappedColumn] = columns.map(mappedColumn => 126 | (mappedColumn.columnName, mappedColumn)).toMap 127 | } 128 | -------------------------------------------------------------------------------- /src/main/scala/com/github/sadikovi/spark/netflow/sources/conversions.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.spark.netflow.sources 18 | 19 | import scala.collection.mutable.HashMap 20 | import scala.util.Try 21 | 22 | import org.apache.spark.unsafe.types.UTF8String 23 | 24 | /** 25 | * [[ConvertFunction]] interface to provide direct `Any -> String` and reversed `String -> Any` 26 | * conversions. 27 | */ 28 | abstract class ConvertFunction { 29 | /** Direct conversion rule */ 30 | def direct(value: Any): String 31 | 32 | /** Direct conversion for Catalyst internal type */ 33 | def directCatalyst(value: Any): UTF8String = { 34 | UTF8String.fromString(direct(value)) 35 | } 36 | 37 | /** Reversed conversion rule */ 38 | def reversed(value: String): Any 39 | } 40 | 41 | /** Conversion function for IPv4 values. */ 42 | case class IPv4ConvertFunction() extends ConvertFunction { 43 | override def direct(value: Any): String = { 44 | val num = value.asInstanceOf[Long] 45 | val sb = new StringBuilder() 46 | sb.append((num & 4278190080L) >> 24) 47 | sb.append('.') 48 | sb.append((num & 16711680) >> 16) 49 | sb.append('.') 50 | sb.append((num & 65280) >> 8) 51 | sb.append('.') 52 | sb.append(num & 255) 53 | sb.toString 54 | } 55 | 56 | private def store(bytes: Array[Byte], i: Int, value: Long): Int = { 57 | var j = i 58 | if (value >= 100) { 59 | bytes(j) = ('0' + value / 100).toByte 60 | j += 1 61 | } 62 | if (value >= 10) { 63 | bytes(j) = ('0' + (value % 100) / 10).toByte 64 | j += 1 65 | } 66 | bytes(j) = ('0' + (value % 10)).toByte 67 | j + 1 68 | } 69 | 70 | override def directCatalyst(value: Any): UTF8String = { 71 | val num = value.asInstanceOf[Long] 72 | val bytes = new Array[Byte](15) 73 | var i = store(bytes, 0, (num & 4278190080L) >> 24) 74 | bytes(i) = '.' 75 | i = store(bytes, i + 1, (num & 16711680) >> 16) 76 | bytes(i) = '.' 77 | i = store(bytes, i + 1, (num & 65280) >> 8) 78 | bytes(i) = '.' 79 | i = store(bytes, i + 1, num & 255) 80 | UTF8String.fromBytes(bytes, 0, i) 81 | } 82 | 83 | override def reversed(value: String): Any = { 84 | val arr = value.split('.').map(_.toLong) 85 | require(arr.length == 4, s"Invalid IPv4: ${value}") 86 | arr(0) << 24 | arr(1) << 16 | arr(2) << 8 | arr(3) 87 | } 88 | } 89 | 90 | /** Conversion function for protocol (most common services) */ 91 | case class ProtocolConvertFunction() extends ConvertFunction { 92 | private[sources] val protocolMap: HashMap[Short, String] = HashMap( 93 | 1.toShort -> "ICMP", // Internet Control Message Protocol 94 | 3.toShort -> "GGP", // Gateway-Gateway Protocol 95 | 6.toShort -> "TCP", // Transmission Control Protocol 96 | 8.toShort -> "EGP", // Exterior Gateway Protocol 97 | 12.toShort -> "PUP", // PARC Universal Packet Protocol 98 | 17.toShort -> "UDP", // User Datagram Protocol 99 | 20.toShort -> "HMP", // Host Monitoring Protocol 100 | 27.toShort -> "RDP", // Reliable Datagram Protocol 101 | 46.toShort -> "RSVP", // Reservation Protocol QoS 102 | 47.toShort -> "GRE", // General Routing Encapsulation 103 | 50.toShort -> "ESP", // Encapsulation Security Payload IPSec 104 | 51.toShort -> "AH", // Authentication Header IPSec 105 | 66.toShort -> "RVD", // MIT Remote Virtual Disk 106 | 88.toShort -> "IGMP", // Internet Group Management Protocol 107 | 89.toShort -> "OSPF" // Open Shortest Path First 108 | ) 109 | 110 | private[sources] lazy val reversedProtocolMap = protocolMap.map { case (key, value) => 111 | (value, key) }.toMap 112 | 113 | override def direct(value: Any): String = { 114 | protocolMap.getOrElse(value.asInstanceOf[Short], value.toString()) 115 | } 116 | 117 | override def reversed(value: String): Any = { 118 | reversedProtocolMap.getOrElse(value, Try(value.toShort).getOrElse( 119 | sys.error(s"Failed to convert $value for ${getClass().getSimpleName()}"))) 120 | } 121 | } 122 | -------------------------------------------------------------------------------- /src/main/scala/com/github/sadikovi/spark/netflow/version5/DefaultProvider.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.spark.netflow.version5 18 | 19 | import com.github.sadikovi.netflowlib.version.NetFlowV5 20 | import com.github.sadikovi.spark.netflow.sources._ 21 | 22 | class DefaultProvider extends NetFlowProvider { 23 | def createInterface(): ResolvedInterface = { 24 | new InterfaceV5() 25 | } 26 | } 27 | 28 | /** NetFlow interface for version 5. */ 29 | private class InterfaceV5 extends ResolvedInterface { 30 | override protected val columns: Seq[MappedColumn] = Seq( 31 | MappedColumn("unix_secs", NetFlowV5.FIELD_UNIX_SECS, true, None), 32 | MappedColumn("unix_nsecs", NetFlowV5.FIELD_UNIX_NSECS, false, None), 33 | MappedColumn("sysuptime", NetFlowV5.FIELD_SYSUPTIME, false, None), 34 | MappedColumn("exaddr", NetFlowV5.FIELD_EXADDR, false, Some(IPv4ConvertFunction())), 35 | MappedColumn("srcip", NetFlowV5.FIELD_SRCADDR, true, Some(IPv4ConvertFunction())), 36 | MappedColumn("dstip", NetFlowV5.FIELD_DSTADDR, true, Some(IPv4ConvertFunction())), 37 | MappedColumn("nexthop", NetFlowV5.FIELD_NEXTHOP, false, Some(IPv4ConvertFunction())), 38 | MappedColumn("input", NetFlowV5.FIELD_INPUT, false, None), 39 | MappedColumn("output", NetFlowV5.FIELD_OUTPUT, false, None), 40 | MappedColumn("packets", NetFlowV5.FIELD_DPKTS, false, None), 41 | MappedColumn("octets", NetFlowV5.FIELD_DOCTETS, false, None), 42 | MappedColumn("first_flow", NetFlowV5.FIELD_FIRST, false, None), 43 | MappedColumn("last_flow", NetFlowV5.FIELD_LAST, false, None), 44 | MappedColumn("srcport", NetFlowV5.FIELD_SRCPORT, true, None), 45 | MappedColumn("dstport", NetFlowV5.FIELD_DSTPORT, true, None), 46 | MappedColumn("protocol", NetFlowV5.FIELD_PROT, true, Some(ProtocolConvertFunction())), 47 | MappedColumn("tos", NetFlowV5.FIELD_TOS, false, None), 48 | MappedColumn("tcp_flags", NetFlowV5.FIELD_TCP_FLAGS, false, None), 49 | MappedColumn("engine_type", NetFlowV5.FIELD_ENGINE_TYPE, false, None), 50 | MappedColumn("engine_id", NetFlowV5.FIELD_ENGINE_ID, false, None), 51 | MappedColumn("src_mask", NetFlowV5.FIELD_SRC_MASK, false, None), 52 | MappedColumn("dst_mask", NetFlowV5.FIELD_DST_MASK, false, None), 53 | MappedColumn("src_as", NetFlowV5.FIELD_SRC_AS, false, None), 54 | MappedColumn("dst_as", NetFlowV5.FIELD_DST_AS, false, None) 55 | ) 56 | 57 | override def version(): Short = 5 58 | } 59 | -------------------------------------------------------------------------------- /src/main/scala/com/github/sadikovi/spark/netflow/version7/DefaultProvider.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.spark.netflow.version7 18 | 19 | import com.github.sadikovi.netflowlib.version.NetFlowV7 20 | import com.github.sadikovi.spark.netflow.sources._ 21 | 22 | class DefaultProvider extends NetFlowProvider { 23 | def createInterface(): ResolvedInterface = { 24 | new InterfaceV7() 25 | } 26 | } 27 | 28 | /** NetFlow interface for version 7. */ 29 | private class InterfaceV7 extends ResolvedInterface { 30 | override protected val columns: Seq[MappedColumn] = Seq( 31 | MappedColumn("unix_secs", NetFlowV7.FIELD_UNIX_SECS, true, None), 32 | MappedColumn("unix_nsecs", NetFlowV7.FIELD_UNIX_NSECS, false, None), 33 | MappedColumn("sysuptime", NetFlowV7.FIELD_SYSUPTIME, false, None), 34 | MappedColumn("exaddr", NetFlowV7.FIELD_EXADDR, false, Some(IPv4ConvertFunction())), 35 | MappedColumn("srcip", NetFlowV7.FIELD_SRCADDR, true, Some(IPv4ConvertFunction())), 36 | MappedColumn("dstip", NetFlowV7.FIELD_DSTADDR, true, Some(IPv4ConvertFunction())), 37 | MappedColumn("nexthop", NetFlowV7.FIELD_NEXTHOP, false, Some(IPv4ConvertFunction())), 38 | MappedColumn("input", NetFlowV7.FIELD_INPUT, false, None), 39 | MappedColumn("output", NetFlowV7.FIELD_OUTPUT, false, None), 40 | MappedColumn("packets", NetFlowV7.FIELD_DPKTS, false, None), 41 | MappedColumn("octets", NetFlowV7.FIELD_DOCTETS, false, None), 42 | MappedColumn("first_flow", NetFlowV7.FIELD_FIRST, false, None), 43 | MappedColumn("last_flow", NetFlowV7.FIELD_LAST, false, None), 44 | MappedColumn("srcport", NetFlowV7.FIELD_SRCPORT, true, None), 45 | MappedColumn("dstport", NetFlowV7.FIELD_DSTPORT, true, None), 46 | MappedColumn("protocol", NetFlowV7.FIELD_PROT, true, Some(ProtocolConvertFunction())), 47 | MappedColumn("tos", NetFlowV7.FIELD_TOS, false, None), 48 | MappedColumn("tcp_flags", NetFlowV7.FIELD_TCP_FLAGS, false, None), 49 | MappedColumn("flags", NetFlowV7.FIELD_FLAGS, false, None), 50 | MappedColumn("engine_type", NetFlowV7.FIELD_ENGINE_TYPE, false, None), 51 | MappedColumn("engine_id", NetFlowV7.FIELD_ENGINE_ID, false, None), 52 | MappedColumn("src_mask", NetFlowV7.FIELD_SRC_MASK, false, None), 53 | MappedColumn("dst_mask", NetFlowV7.FIELD_DST_MASK, false, None), 54 | MappedColumn("src_as", NetFlowV7.FIELD_SRC_AS, false, None), 55 | MappedColumn("dst_as", NetFlowV7.FIELD_DST_AS, false, None), 56 | MappedColumn("router_sc", NetFlowV7.FIELD_ROUTER_SC, false, Some(IPv4ConvertFunction())) 57 | ) 58 | 59 | override def version(): Short = 7 60 | } 61 | -------------------------------------------------------------------------------- /src/main/scala/com/github/sadikovi/spark/util/CloseableIterator.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.spark.util 18 | 19 | /** 20 | * [[CloseableIterator]] provides ability to close associated resources once iteration is finished. 21 | * Copied from apache/spark and slightly modified version of `NextIterator`. 22 | */ 23 | private[spark] abstract class CloseableIterator[U] extends Iterator[U] { 24 | private var gotNext = false 25 | private var nextValue: U = _ 26 | private var closed = false 27 | protected var finished = false 28 | 29 | /** 30 | * Method for subclasses to implement to provide the next element. 31 | * 32 | * If no next element is available, the subclass should set `finished` 33 | * to `true` and may return any value (it will be ignored). 34 | * 35 | * This convention is required because `null` may be a valid value, 36 | * and using `Option` seems like it might create unnecessary Some/None 37 | * instances, given some iterators might be called in a tight loop. 38 | * 39 | * @return U, or set 'finished' when done 40 | */ 41 | protected def getNext(): U 42 | 43 | /** 44 | * Method for subclasses to implement when all elements have been successfully 45 | * iterated, and the iteration is done. 46 | * 47 | * Ideally you should have another try/catch that ensures any resources are closed should 48 | * iteration fail. 49 | */ 50 | protected def close() 51 | 52 | /** 53 | * Calls the subclass-defined close method, but only once. 54 | * 55 | * Usually calling `close` multiple times should be fine, but historically 56 | * there have been issues with some InputFormats throwing exceptions. 57 | */ 58 | def closeIfNeeded() { 59 | if (!closed) { 60 | // Note: it's important that we set closed = true before calling close(), since setting it 61 | // afterwards would permit us to call close() multiple times if close() threw an exception. 62 | closed = true 63 | close() 64 | } 65 | } 66 | 67 | override def hasNext: Boolean = { 68 | if (!finished) { 69 | if (!gotNext) { 70 | nextValue = getNext() 71 | if (finished) { 72 | closeIfNeeded() 73 | } 74 | gotNext = true 75 | } 76 | } 77 | !finished 78 | } 79 | 80 | override def next(): U = { 81 | if (!hasNext) { 82 | throw new NoSuchElementException("End of stream") 83 | } 84 | gotNext = false 85 | nextValue 86 | } 87 | } 88 | -------------------------------------------------------------------------------- /src/main/scala/com/github/sadikovi/spark/util/SerializableConfiguration.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.spark.util 18 | 19 | import java.io.{ObjectInputStream, ObjectOutputStream} 20 | 21 | import org.apache.hadoop.conf.Configuration 22 | 23 | /** 24 | * Serializable hadoop configuration. Clone of `org.apache.spark.util.SerializableConfiguration`, 25 | * since it cannot be reused outside `spark` package. 26 | */ 27 | private[spark] class SerializableConfiguration( 28 | @transient var value: Configuration) extends Serializable { 29 | 30 | private def writeObject(out: ObjectOutputStream): Unit = { 31 | out.defaultWriteObject() 32 | value.write(out) 33 | } 34 | 35 | private def readObject(in: ObjectInputStream): Unit = { 36 | value = new Configuration(false) 37 | value.readFields(in) 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /src/main/scala/com/github/sadikovi/spark/util/Utils.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.spark.util 18 | 19 | import java.util.UUID 20 | 21 | import org.apache.hadoop.conf.{Configuration => HadoopConf} 22 | import org.apache.hadoop.fs.{Path => HadoopPath} 23 | 24 | import org.apache.spark.network.util.JavaUtils 25 | 26 | private[spark] object Utils { 27 | /** 28 | * Convert string of bytes (1024, 10Mb, 5Kb, etc.) into number of bytes. 29 | * Copied from Apache Spark `Utils.scala`. 30 | * @param str string to parse 31 | * @return number of bytes for corresponding string 32 | */ 33 | def byteStringAsBytes(str: String): Long = { 34 | JavaUtils.byteStringAsBytes(str) 35 | } 36 | 37 | /** 38 | * Create UUID for a string as 128-bit value string. 39 | * @param str string to create uuid for 40 | * @return generated UUID as string 41 | */ 42 | def uuidForString(str: String): String = { 43 | UUID.nameUUIDFromBytes(str.getBytes()).toString() 44 | } 45 | 46 | /** 47 | * Get context class laoder on this thread or, if not present, default class loader for this 48 | * class. 49 | */ 50 | def getContextClassLoader(): ClassLoader = { 51 | Option(Thread.currentThread().getContextClassLoader).getOrElse(getClass.getClassLoader) 52 | } 53 | 54 | /** Return updated path with suffix appended */ 55 | def withSuffix(path: HadoopPath, suffix: String*): HadoopPath = { 56 | path.suffix(s"${HadoopPath.SEPARATOR}${suffix.mkString(HadoopPath.SEPARATOR)}") 57 | } 58 | 59 | /** 60 | * Compute histogram mode, default step is chosen as square root of data points in array, but can 61 | * be provided custom step function. Instead of computing average of the largest bucket (one has 62 | * more elements), we just return the largest element in the bucket, since bucket range can be 63 | * very significant. `stepFunc` takes min element, max element and number of data points and 64 | * expects number of buckets as a return value. 65 | */ 66 | def histogramMode(arr: Array[Long], stepFunc: Option[(Long, Long, Int) => Int] = None): Long = { 67 | require(arr.nonEmpty, "Expected non-empty array to compute histogram mode") 68 | if (arr.length == 1) { 69 | arr.head 70 | } else { 71 | // Sort in increasing order 72 | val srt = arr.sortWith(_ < _) 73 | val min = srt.head 74 | val max = srt.last 75 | 76 | // Compute number of buckets based on step function 77 | val numBuckets = stepFunc match { 78 | case Some(func) => func(min, max, arr.length) 79 | case None => Math.ceil(Math.pow(arr.length, 0.5)).toInt 80 | } 81 | 82 | val buckets = for (bucket <- 0 until numBuckets) yield { 83 | val start = Math.ceil((max - min) * bucket.toDouble / numBuckets).toLong + min 84 | val end = Math.floor((max - min) * (bucket + 1).toDouble / numBuckets).toLong + min 85 | (start, end) 86 | } 87 | 88 | val bucketStats = buckets.map { case (start, end) => { 89 | var maxElem: Long = 0 90 | var cnt: Int = 0 91 | 92 | for (elem <- srt) { 93 | if (elem >= start && elem <= end) { 94 | cnt += 1 95 | maxElem = Math.max(elem, maxElem) 96 | } 97 | } 98 | (maxElem, cnt) 99 | } } 100 | 101 | // Extract largest bucket, and return maximum value from the bucket 102 | val largestBucket = bucketStats.sortWith(_._2 > _._2).head 103 | largestBucket._1 104 | } 105 | } 106 | 107 | /** 108 | * Compute truncated mean of the dataset based on sample. `1 - sample` of the dataset is 109 | * discarded, mean is computed on the rest and flatten across all data points. Currently only 110 | * tail is truncated, making assumption that skewness is positive only. 111 | */ 112 | def truncatedMean(arr: Array[Long], sample: Double): Long = { 113 | val n = arr.length 114 | require(n > 0, "Expected non-empty array to compute mean") 115 | require(sample > 0, s"Expected positive sample, got ${sample}") 116 | 117 | if (n == 1) { 118 | arr.head 119 | } else { 120 | val srt = arr.sortWith(_ < _) 121 | 122 | var sum: Long = 0 123 | for (i <- 0 until (sample * n).toInt) { 124 | sum = sum + srt(i) 125 | } 126 | 127 | Math.ceil(sum / n / sample).toLong 128 | } 129 | } 130 | 131 | /** Create temporary directory on local file system */ 132 | def createTempDir( 133 | root: String = System.getProperty("java.io.tmpdir"), 134 | namePrefix: String = "netflow"): HadoopPath = { 135 | val dir = Utils.withSuffix(new HadoopPath(root), namePrefix, UUID.randomUUID().toString) 136 | val fs = dir.getFileSystem(new HadoopConf(false)) 137 | fs.mkdirs(dir) 138 | dir 139 | } 140 | 141 | /** Execute block of code with temporary hadoop path */ 142 | private def withTempHadoopPath(path: HadoopPath)(func: HadoopPath => Unit): Unit = { 143 | try { 144 | func(path) 145 | } finally { 146 | val fs = path.getFileSystem(new HadoopConf(false)) 147 | fs.delete(path, true) 148 | } 149 | } 150 | 151 | /** Execute code block with created temporary directory */ 152 | def withTempDir(func: HadoopPath => Unit): Unit = { 153 | withTempHadoopPath(Utils.createTempDir())(func) 154 | } 155 | 156 | /** Execute code block with created temporary file */ 157 | def withTempFile(func: HadoopPath => Unit): Unit = { 158 | val file = Utils.withSuffix(Utils.createTempDir(), UUID.randomUUID().toString) 159 | withTempHadoopPath(file)(func) 160 | } 161 | } 162 | -------------------------------------------------------------------------------- /src/test/java/com/github/sadikovi/netflowlib/NetFlowHeaderSuite.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib; 18 | 19 | import java.nio.ByteOrder; 20 | import java.util.ArrayList; 21 | import java.util.Iterator; 22 | import java.util.List; 23 | import java.util.NoSuchElementException; 24 | 25 | import org.junit.Test; 26 | import org.junit.Ignore; 27 | import static org.junit.Assert.assertSame; 28 | import static org.junit.Assert.assertEquals; 29 | 30 | import com.github.sadikovi.netflowlib.CorruptNetFlowHeader; 31 | import com.github.sadikovi.netflowlib.NetFlowHeader; 32 | 33 | public class NetFlowHeaderSuite { 34 | @Test 35 | public void testInitNetFlowHeader() { 36 | NetFlowHeader header = new NetFlowHeader((short) 5, ByteOrder.BIG_ENDIAN); 37 | assertEquals(header.getStreamVersion(), 5); 38 | assertEquals(header.getByteOrder(), ByteOrder.BIG_ENDIAN); 39 | assertEquals(header.isValid(), true); 40 | } 41 | 42 | @Test 43 | public void testInitCorruptNetFlowHeader() { 44 | NetFlowHeader header = new CorruptNetFlowHeader(); 45 | assertEquals(header.isValid(), false); 46 | } 47 | 48 | @Test(expected = UnsupportedOperationException.class) 49 | public void testFailOnCorruptHeaderMethod1() { 50 | NetFlowHeader header = new CorruptNetFlowHeader(); 51 | header.getStreamVersion(); 52 | } 53 | 54 | @Test(expected = UnsupportedOperationException.class) 55 | public void testFailOnCorruptHeaderMethod2() { 56 | NetFlowHeader header = new CorruptNetFlowHeader(); 57 | header.getByteOrder(); 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /src/test/java/com/github/sadikovi/netflowlib/UtilSuite.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib; 18 | 19 | import java.nio.ByteBuffer; 20 | import java.nio.ByteOrder; 21 | import java.util.ArrayList; 22 | import java.util.Iterator; 23 | import java.util.List; 24 | import java.util.NoSuchElementException; 25 | import java.util.Random; 26 | 27 | import org.junit.Test; 28 | import org.junit.Ignore; 29 | import static org.junit.Assert.assertArrayEquals; 30 | import static org.junit.Assert.assertSame; 31 | import static org.junit.Assert.assertEquals; 32 | 33 | import com.github.sadikovi.netflowlib.util.FilterIterator; 34 | import com.github.sadikovi.netflowlib.util.SafeIterator; 35 | import com.github.sadikovi.netflowlib.util.WrappedByteBuf; 36 | 37 | public class UtilSuite { 38 | @Test(expected = NoSuchElementException.class) 39 | public void testFilterIteratorFailIfEmpty() { 40 | List values = new ArrayList(); 41 | FilterIterator iter = new FilterIterator(values.iterator()); 42 | iter.next(); 43 | } 44 | 45 | @Test 46 | public void testFilterIteratorReturnAll() { 47 | List values = new ArrayList(); 48 | values.add("a"); 49 | values.add("a"); 50 | values.add("a"); 51 | FilterIterator iter = new FilterIterator(values.iterator()); 52 | 53 | int count = 0; 54 | while (iter.hasNext()) { 55 | assertSame(iter.next(), "a"); 56 | count++; 57 | } 58 | 59 | assertEquals(count, values.size()); 60 | } 61 | 62 | @Test 63 | public void testFilterIteratorReturnSome() { 64 | List values = new ArrayList(); 65 | values.add("a"); 66 | values.add(null); 67 | values.add("a"); 68 | FilterIterator iter = new FilterIterator(values.iterator()); 69 | 70 | int count = 0; 71 | while (iter.hasNext()) { 72 | assertSame(iter.next(), "a"); 73 | count++; 74 | } 75 | 76 | assertEquals(count, 2); 77 | } 78 | 79 | @Test 80 | public void testFilterIteratorReturnNone() { 81 | List values = new ArrayList(); 82 | values.add(null); 83 | values.add(null); 84 | values.add(null); 85 | FilterIterator iter = new FilterIterator(values.iterator()); 86 | 87 | int count = 0; 88 | while (iter.hasNext()) { 89 | count++; 90 | } 91 | 92 | assertEquals(count, 0); 93 | } 94 | 95 | @Test(expected = NoSuchElementException.class) 96 | public void testSafeIteratorFailIfEmpty() { 97 | List values = new ArrayList(); 98 | SafeIterator iter = new SafeIterator(values.iterator()); 99 | iter.next(); 100 | } 101 | 102 | @Test 103 | public void testSafeIteratorReturnAll() { 104 | List values = new ArrayList(); 105 | values.add("a"); 106 | values.add("a"); 107 | values.add("a"); 108 | SafeIterator iter = new SafeIterator(values.iterator()); 109 | 110 | int count = 0; 111 | while (iter.hasNext()) { 112 | assertSame(iter.next(), "a"); 113 | count++; 114 | } 115 | 116 | assertEquals(count, values.size()); 117 | } 118 | 119 | @Test 120 | public void testSafeIteratorTerminateOnErrorInHasNext() { 121 | final List values = new ArrayList(); 122 | values.add("a"); 123 | values.add(null); 124 | values.add("a"); 125 | Iterator delegate = new Iterator() { 126 | private Iterator parent = values.iterator(); 127 | private String current; 128 | 129 | @Override 130 | public boolean hasNext() { 131 | current = parent.next(); 132 | if (current == null) { 133 | throw new IllegalStateException("Test"); 134 | } 135 | return parent.hasNext(); 136 | } 137 | 138 | @Override 139 | public String next() { 140 | return current; 141 | } 142 | 143 | @Override 144 | public void remove() { } 145 | }; 146 | SafeIterator iter = new SafeIterator(delegate); 147 | 148 | int count = 0; 149 | while (iter.hasNext()) { 150 | assertSame(iter.next(), "a"); 151 | count++; 152 | } 153 | 154 | // Expect one record only, since second record fails with state exception 155 | assertEquals(count, 1); 156 | } 157 | 158 | @Test 159 | public void testSafeIteratorTerminateOnErrorInNext() { 160 | final List values = new ArrayList(); 161 | values.add("a"); 162 | values.add(null); 163 | values.add("a"); 164 | Iterator delegate = new Iterator() { 165 | private Iterator parent = values.iterator(); 166 | 167 | @Override 168 | public boolean hasNext() { 169 | return parent.hasNext(); 170 | } 171 | 172 | @Override 173 | public String next() { 174 | return parent.next().toString(); 175 | } 176 | 177 | @Override 178 | public void remove() { } 179 | }; 180 | SafeIterator iter = new SafeIterator(delegate); 181 | 182 | int count = 0; 183 | while (iter.hasNext()) { 184 | assertSame(iter.next(), "a"); 185 | count++; 186 | } 187 | 188 | // Expect one record only, since second record fails with null pointer exception 189 | assertEquals(count, 1); 190 | } 191 | 192 | @Test 193 | public void testWrappedByteBufGetValue() { 194 | byte[] bytes = new byte[1024]; 195 | Random rand = new Random(); 196 | // check big endian 197 | for (int i = 0; i <= bytes.length - 4; i++) { 198 | rand.nextBytes(bytes); 199 | WrappedByteBuf buf = WrappedByteBuf.init(bytes, ByteOrder.BIG_ENDIAN); 200 | ByteBuffer javaBuf = ByteBuffer.wrap(bytes).order(ByteOrder.BIG_ENDIAN); 201 | 202 | assertSame(buf.array(), bytes); 203 | assertEquals(buf.getByte(i), javaBuf.get(i)); 204 | assertEquals(buf.getUnsignedByte(i), (short) (javaBuf.get(i) & 0xff)); 205 | assertEquals(buf.getShort(i), javaBuf.getShort(i)); 206 | assertEquals(buf.getUnsignedShort(i), javaBuf.getShort(i) & 0xffff); 207 | assertEquals(buf.getInt(i), javaBuf.getInt(i)); 208 | assertEquals(buf.getUnsignedInt(i), javaBuf.getInt(i) & 0xffffffffL); 209 | } 210 | 211 | // check little endian 212 | for (int i = 0; i <= bytes.length - 4; i++) { 213 | rand.nextBytes(bytes); 214 | WrappedByteBuf buf = WrappedByteBuf.init(bytes, ByteOrder.LITTLE_ENDIAN); 215 | ByteBuffer javaBuf = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); 216 | 217 | assertSame(buf.array(), bytes); 218 | assertEquals(buf.getByte(i), javaBuf.get(i)); 219 | assertEquals(buf.getUnsignedByte(i), (short) (javaBuf.get(i) & 0xff)); 220 | assertEquals(buf.getShort(i), javaBuf.getShort(i)); 221 | assertEquals(buf.getUnsignedShort(i), javaBuf.getShort(i) & 0xffff); 222 | assertEquals(buf.getInt(i), javaBuf.getInt(i)); 223 | assertEquals(buf.getUnsignedInt(i), javaBuf.getInt(i) & 0xffffffffL); 224 | } 225 | } 226 | 227 | @Test 228 | public void testWrappedByteBufGetBytesBigEndian() { 229 | byte[] bytes = new byte[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 0}; 230 | WrappedByteBuf buf = WrappedByteBuf.init(bytes, ByteOrder.BIG_ENDIAN); 231 | ByteBuffer javaBuf = ByteBuffer.wrap(bytes).order(ByteOrder.BIG_ENDIAN); 232 | 233 | byte[] dst1 = new byte[10]; 234 | byte[] dst2 = new byte[10]; 235 | 236 | buf.getBytes(0, dst1, 0, dst1.length); 237 | javaBuf.position(0); 238 | javaBuf.get(dst2, 0, dst2.length); 239 | assertArrayEquals(dst1, dst2); 240 | 241 | buf.getBytes(4, dst1, 1, 2); 242 | javaBuf.position(4); 243 | javaBuf.get(dst2, 1, 2); 244 | assertArrayEquals(dst1, dst2); 245 | 246 | buf.getBytes(8, dst1, 8, 1); 247 | javaBuf.position(8); 248 | javaBuf.get(dst2, 8, 1); 249 | assertArrayEquals(dst1, dst2); 250 | 251 | // array should not be modified 252 | assertArrayEquals(buf.array(), new byte[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 0}); 253 | } 254 | 255 | @Test 256 | public void testWrappedByteBufGetBytesLittleEndian() { 257 | byte[] bytes = new byte[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 0}; 258 | WrappedByteBuf buf = WrappedByteBuf.init(bytes, ByteOrder.LITTLE_ENDIAN); 259 | ByteBuffer javaBuf = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); 260 | 261 | byte[] dst1 = new byte[10]; 262 | byte[] dst2 = new byte[10]; 263 | 264 | buf.getBytes(0, dst1, 0, dst1.length); 265 | javaBuf.position(0); 266 | javaBuf.get(dst2, 0, dst2.length); 267 | assertArrayEquals(dst1, dst2); 268 | 269 | buf.getBytes(4, dst1, 1, 2); 270 | javaBuf.position(4); 271 | javaBuf.get(dst2, 1, 2); 272 | assertArrayEquals(dst1, dst2); 273 | 274 | buf.getBytes(8, dst1, 8, 1); 275 | javaBuf.position(8); 276 | javaBuf.get(dst2, 8, 1); 277 | assertArrayEquals(dst1, dst2); 278 | 279 | // array should not be modified 280 | assertArrayEquals(buf.array(), new byte[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 0}); 281 | } 282 | } 283 | -------------------------------------------------------------------------------- /src/test/java/com/github/sadikovi/netflowlib/predicate/JavaColumnSuite.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.netflowlib.predicate; 18 | 19 | import static org.hamcrest.CoreMatchers.containsString; 20 | 21 | import org.junit.Test; 22 | import static org.junit.Assert.assertEquals; 23 | import static org.junit.Assert.assertFalse; 24 | import static org.junit.Assert.assertSame; 25 | import static org.junit.Assert.assertThat; 26 | import static org.junit.Assert.assertTrue; 27 | 28 | import com.github.sadikovi.netflowlib.predicate.Columns.ByteColumn; 29 | import com.github.sadikovi.netflowlib.predicate.Columns.IntColumn; 30 | import com.github.sadikovi.netflowlib.predicate.Columns.LongColumn; 31 | import com.github.sadikovi.netflowlib.predicate.Columns.ShortColumn; 32 | 33 | public class JavaColumnSuite { 34 | @Test 35 | public void testColumnInit() { 36 | // initialize byte column and check min, max 37 | ByteColumn col1 = new ByteColumn("name", 1); 38 | assertEquals(col1.getColumnName(), "name"); 39 | assertEquals(col1.getColumnOffset(), 1); 40 | assertEquals(col1.getColumnType(), Byte.class); 41 | assertSame(Byte.class.cast(col1.getMin()), (byte) 0); 42 | assertSame(Byte.class.cast(col1.getMax()) - Byte.MAX_VALUE, 0); 43 | 44 | ByteColumn col2 = new ByteColumn("name", 1, (byte) 1, (byte) 2); 45 | assertEquals(col2.getColumnName(), "name"); 46 | assertEquals(col2.getColumnOffset(), 1); 47 | assertEquals(col2.getColumnType(), Byte.class); 48 | assertSame(Byte.class.cast(col2.getMin()), (byte) 1); 49 | assertSame(Byte.class.cast(col2.getMax()), (byte) 2); 50 | 51 | // initialize short column and check min, max 52 | ShortColumn col3 = new ShortColumn("name", 2); 53 | assertEquals(col3.getColumnName(), "name"); 54 | assertEquals(col3.getColumnOffset(), 2); 55 | assertEquals(col3.getColumnType(), Short.class); 56 | assertSame(Short.class.cast(col3.getMin()), (short) 0); 57 | assertSame(Short.class.cast(col3.getMax()) - Short.MAX_VALUE, 0); 58 | 59 | ShortColumn col4 = new ShortColumn("name", 2, (short) 10, (short) 100); 60 | assertEquals(col4.getColumnName(), "name"); 61 | assertEquals(col4.getColumnOffset(), 2); 62 | assertEquals(col4.getColumnType(), Short.class); 63 | assertSame(Short.class.cast(col4.getMin()), (short) 10); 64 | assertSame(Short.class.cast(col4.getMax()), (short) 100); 65 | 66 | IntColumn col5 = new IntColumn("name", 3); 67 | assertEquals(col5.getColumnName(), "name"); 68 | assertEquals(col5.getColumnOffset(), 3); 69 | assertEquals(col5.getColumnType(), Integer.class); 70 | assertSame(Integer.class.cast(col5.getMin()), 0); 71 | assertSame(Integer.class.cast(col5.getMax()) - Integer.MAX_VALUE, 0); 72 | 73 | IntColumn col6 = new IntColumn("name", 3, 10, 100); 74 | assertEquals(col6.getColumnName(), "name"); 75 | assertEquals(col6.getColumnOffset(), 3); 76 | assertEquals(col6.getColumnType(), Integer.class); 77 | assertSame(Integer.class.cast(col6.getMin()), 10); 78 | assertSame(Integer.class.cast(col6.getMax()), 100); 79 | 80 | LongColumn col7 = new LongColumn("name", 4); 81 | assertEquals(col7.getColumnName(), "name"); 82 | assertEquals(col7.getColumnOffset(), 4); 83 | assertEquals(col7.getColumnType(), Long.class); 84 | assertSame(Long.class.cast(col7.getMin()), (long) 0); 85 | assertSame(Long.class.cast(col7.getMax()) - Long.MAX_VALUE, (long) 0); 86 | 87 | LongColumn col8 = new LongColumn("name", 4, 10, 100); 88 | assertEquals(col8.getColumnName(), "name"); 89 | assertEquals(col8.getColumnOffset(), 4); 90 | assertEquals(col8.getColumnType(), Long.class); 91 | assertSame(Long.class.cast(col8.getMin()), (long) 10); 92 | assertSame(Long.class.cast(col8.getMax()), (long) 100); 93 | } 94 | 95 | @Test 96 | public void testWrongOffset() { 97 | try { 98 | new ByteColumn("name", -1); 99 | } catch (IllegalArgumentException iae) { 100 | assertThat(iae.getMessage(), containsString("Wrong offset")); 101 | } 102 | 103 | try { 104 | new ShortColumn("name", -1); 105 | } catch (IllegalArgumentException iae) { 106 | assertThat(iae.getMessage(), containsString("Wrong offset")); 107 | } 108 | 109 | try { 110 | new IntColumn("name", -1); 111 | } catch (IllegalArgumentException iae) { 112 | assertThat(iae.getMessage(), containsString("Wrong offset")); 113 | } 114 | 115 | try { 116 | new LongColumn("name", -1); 117 | } catch (IllegalArgumentException iae) { 118 | assertThat(iae.getMessage(), containsString("Wrong offset")); 119 | } 120 | } 121 | 122 | @Test 123 | public void testWrongMinMax() { 124 | try { 125 | new ByteColumn("name", 1, (byte) 2, (byte) 1); 126 | } catch (IllegalArgumentException iae) { 127 | assertThat(iae.getMessage(), containsString("is greater than")); 128 | } 129 | 130 | try { 131 | new ShortColumn("name", 1, (byte) 2, (byte) 1); 132 | } catch (IllegalArgumentException iae) { 133 | assertThat(iae.getMessage(), containsString("is greater than")); 134 | } 135 | 136 | try { 137 | new IntColumn("name", 1, (byte) 2, (byte) 1); 138 | } catch (IllegalArgumentException iae) { 139 | assertThat(iae.getMessage(), containsString("is greater than")); 140 | } 141 | 142 | try { 143 | new LongColumn("name", 1, (byte) 2, (byte) 1); 144 | } catch (IllegalArgumentException iae) { 145 | assertThat(iae.getMessage(), containsString("is greater than")); 146 | } 147 | } 148 | 149 | @Test 150 | public void testEquality() { 151 | IntColumn col1 = new IntColumn("col1", 0); 152 | 153 | IntColumn col2 = new IntColumn("col1", 0); 154 | assertTrue(col1.equals(col2)); 155 | 156 | IntColumn col3 = new IntColumn("col2", 0); 157 | assertFalse(col1.equals(col3)); 158 | 159 | IntColumn col4 = new IntColumn("col1", 1); 160 | assertFalse(col1.equals(col4)); 161 | 162 | ShortColumn col5 = new ShortColumn("col1", 0); 163 | assertFalse(col1.equals(col5)); 164 | 165 | IntColumn col6 = new IntColumn("col1", 0, 10, 100); 166 | assertTrue(col1.equals(col6)); 167 | } 168 | } 169 | -------------------------------------------------------------------------------- /src/test/resources/anomaly/ftv5.2016-03-15.compress2.bigend.empty: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/sadikovi/spark-netflow/695bfd55c9e731914bf4b3f416757e8db76bd137/src/test/resources/anomaly/ftv5.2016-03-15.compress2.bigend.empty -------------------------------------------------------------------------------- /src/test/resources/anomaly/ftv5.2016-03-15.compress9.bigend.empty: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/sadikovi/spark-netflow/695bfd55c9e731914bf4b3f416757e8db76bd137/src/test/resources/anomaly/ftv5.2016-03-15.compress9.bigend.empty -------------------------------------------------------------------------------- /src/test/resources/anomaly/ftv5.2016-03-15.compress9.bigend.records1: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/sadikovi/spark-netflow/695bfd55c9e731914bf4b3f416757e8db76bd137/src/test/resources/anomaly/ftv5.2016-03-15.compress9.bigend.records1 -------------------------------------------------------------------------------- /src/test/resources/anomaly/ftv5.2016-03-15.nocompress.bigend.empty: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/sadikovi/spark-netflow/695bfd55c9e731914bf4b3f416757e8db76bd137/src/test/resources/anomaly/ftv5.2016-03-15.nocompress.bigend.empty -------------------------------------------------------------------------------- /src/test/resources/anomaly/ftv5.2016-04-09.compress9.large-byte-start: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/sadikovi/spark-netflow/695bfd55c9e731914bf4b3f416757e8db76bd137/src/test/resources/anomaly/ftv5.2016-04-09.compress9.large-byte-start -------------------------------------------------------------------------------- /src/test/resources/correct/ftv5.2016-01-13.compress.9.sample: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/sadikovi/spark-netflow/695bfd55c9e731914bf4b3f416757e8db76bd137/src/test/resources/correct/ftv5.2016-01-13.compress.9.sample -------------------------------------------------------------------------------- /src/test/resources/correct/ftv5.2016-01-13.nocompress.bigend.sample: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/sadikovi/spark-netflow/695bfd55c9e731914bf4b3f416757e8db76bd137/src/test/resources/correct/ftv5.2016-01-13.nocompress.bigend.sample -------------------------------------------------------------------------------- /src/test/resources/correct/ftv7.2016-02-14.compress.9.bigend.sample: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/sadikovi/spark-netflow/695bfd55c9e731914bf4b3f416757e8db76bd137/src/test/resources/correct/ftv7.2016-02-14.compress.9.bigend.sample -------------------------------------------------------------------------------- /src/test/resources/correct/ftv7.2016-02-14.compress.9.litend.sample: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/sadikovi/spark-netflow/695bfd55c9e731914bf4b3f416757e8db76bd137/src/test/resources/correct/ftv7.2016-02-14.compress.9.litend.sample -------------------------------------------------------------------------------- /src/test/resources/correct/ftv7.2016-02-14.nocompress.bigend.sample: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/sadikovi/spark-netflow/695bfd55c9e731914bf4b3f416757e8db76bd137/src/test/resources/correct/ftv7.2016-02-14.nocompress.bigend.sample -------------------------------------------------------------------------------- /src/test/resources/corrupt/ftv5.2016-01-13.compress.9.sample-00: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/sadikovi/spark-netflow/695bfd55c9e731914bf4b3f416757e8db76bd137/src/test/resources/corrupt/ftv5.2016-01-13.compress.9.sample-00 -------------------------------------------------------------------------------- /src/test/resources/corrupt/ftv5.2016-01-13.compress.9.sample-01: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/sadikovi/spark-netflow/695bfd55c9e731914bf4b3f416757e8db76bd137/src/test/resources/corrupt/ftv5.2016-01-13.compress.9.sample-01 -------------------------------------------------------------------------------- /src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Set root logger level to DEBUG and its only appender to console. 2 | log4j.logger.com.github.sadikovi=INFO, netflowconsole 3 | log4j.additivity.com.github.sadikovi=false 4 | 5 | log4j.appender.netflowconsole=org.apache.log4j.ConsoleAppender 6 | 7 | log4j.appender.netflowconsole.target=System.err 8 | log4j.appender.netflowconsole.layout=org.apache.log4j.PatternLayout 9 | log4j.appender.netflowconsole.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n 10 | -------------------------------------------------------------------------------- /src/test/resources/path-resolver: -------------------------------------------------------------------------------- 1 | # PATH-RESOLVER: DO NOT DELETE THIS FILE 2 | -------------------------------------------------------------------------------- /src/test/resources/unsupport/ftv8.2016-01-17.compress.7.bigend.sample: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/sadikovi/spark-netflow/695bfd55c9e731914bf4b3f416757e8db76bd137/src/test/resources/unsupport/ftv8.2016-01-17.compress.7.bigend.sample -------------------------------------------------------------------------------- /src/test/scala/com/github/sadikovi/spark/netflow/NetFlowOptionsSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.spark.netflow 18 | 19 | import com.github.sadikovi.netflowlib.Buffers.RecordBuffer 20 | import com.github.sadikovi.testutil.UnitTestSuite 21 | 22 | class NetFlowOptionsSuite extends UnitTestSuite { 23 | test("NetFlowOptions - predicate pushdown is enabled when no option is provided") { 24 | val opts = new NetFlowOptions(Map.empty) 25 | opts.usePredicatePushdown should be (true) 26 | } 27 | 28 | test("NetFlowOptions - predicate pushdown is disabled when 'false' is provided") { 29 | val opts = new NetFlowOptions(Map("predicate-pushdown" -> "false")) 30 | opts.usePredicatePushdown should be (false) 31 | } 32 | 33 | test("NetFlowOptions - predicate pushdown is enabled when 'true' is provided") { 34 | val opts = new NetFlowOptions(Map("predicate-pushdown" -> "true")) 35 | opts.usePredicatePushdown should be (true) 36 | } 37 | 38 | test("NetFlowOptions - test buffer size") { 39 | // check that buffer size is default 40 | var opts = new NetFlowOptions(Map.empty) 41 | opts.bufferSize should be (RecordBuffer.BUFFER_LENGTH_2) 42 | 43 | // set buffer size to be 64Kb 44 | opts = new NetFlowOptions(Map("buffer" -> "64Kb")) 45 | opts.bufferSize should be (64 * 1024) 46 | } 47 | 48 | test("NetFlowOptions - invalid buffer size") { 49 | // buffer size >> Integer.MAX_VALUE 50 | var err = intercept[RuntimeException] { 51 | new NetFlowOptions(Map("buffer" -> "10Gb")) 52 | } 53 | assert(err.getMessage.contains("> maximum buffer size")) 54 | 55 | // negative buffer size 56 | intercept[NumberFormatException] { 57 | new NetFlowOptions(Map("buffer" -> "-1")) 58 | } 59 | 60 | // buffer size < min buffer size 61 | err = intercept[RuntimeException] { 62 | new NetFlowOptions(Map("buffer" -> "10")) 63 | } 64 | assert(err.getMessage.contains("< minimum buffer size")) 65 | 66 | // just for completeness, test on wrong buffer value 67 | intercept[NumberFormatException] { 68 | new NetFlowOptions(Map("buffer" -> "wrong")) 69 | } 70 | } 71 | 72 | test("NetFlowOptions - stringify is enabled by default") { 73 | val opts = new NetFlowOptions(Map.empty) 74 | opts.applyConversion should be (true) 75 | } 76 | 77 | test("NetFlowOptions - stringify is disabled, if false") { 78 | val opts = new NetFlowOptions(Map("stringify" -> "false")) 79 | opts.applyConversion should be (false) 80 | } 81 | 82 | test("NetFlowOptions - stringify is enabled, if true") { 83 | val opts = new NetFlowOptions(Map("stringify" -> "true")) 84 | opts.applyConversion should be (true) 85 | } 86 | 87 | test("NetFlowOptions - toString 1") { 88 | val opts = new NetFlowOptions(Map.empty) 89 | opts.toString should be ("NetFlowOptions(applyConversion=true, " + 90 | s"bufferSize=${RecordBuffer.BUFFER_LENGTH_2}, usePredicatePushdown=true)") 91 | } 92 | 93 | test("NetFlowOptions - toString 2") { 94 | val opts = new NetFlowOptions(Map("stringify" -> "false", "buffer" -> "32768", 95 | "predicate-pushdown" -> "false")) 96 | opts.toString should be ("NetFlowOptions(applyConversion=false, " + 97 | s"bufferSize=32768, usePredicatePushdown=false)") 98 | } 99 | } 100 | -------------------------------------------------------------------------------- /src/test/scala/com/github/sadikovi/spark/netflow/sources/ConvertFunctionSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.spark.netflow.sources 18 | 19 | import org.apache.spark.unsafe.types.UTF8String 20 | 21 | import com.github.sadikovi.testutil.UnitTestSuite 22 | 23 | class ConvertFunctionSuite extends UnitTestSuite { 24 | test("ip conversion") { 25 | val dataset = Seq( 26 | ("127.0.0.1", 2130706433L), 27 | ("172.71.4.54", 2890335286L), 28 | ("147.10.8.41", 2466908201L), 29 | ("10.208.97.205", 181428685L), 30 | ("144.136.17.61", 2424836413L), 31 | ("139.168.155.28", 2343082780L), 32 | ("172.49.10.53", 2888895029L), 33 | ("139.168.51.129", 2343056257L), 34 | ("10.152.185.135", 177781127L), 35 | ("144.131.33.125", 2424512893L), 36 | ("138.217.81.41", 2329497897L), 37 | ("147.10.7.77", 2466907981L), 38 | ("10.164.0.185", 178520249L), 39 | ("144.136.28.121", 2424839289L), 40 | ("172.117.8.117", 2893351029L), 41 | ("139.168.164.113", 2343085169L), 42 | ("147.132.87.29", 2474923805L), 43 | ("10.111.3.73", 175047497L), 44 | ("255.255.255.255", (2L<<31) - 1) 45 | ) 46 | 47 | val convertFunction = IPv4ConvertFunction() 48 | 49 | // test direct conversion 50 | for (elem <- dataset) { 51 | val (ip, num) = elem 52 | convertFunction.direct(num) should equal (ip) 53 | convertFunction.directCatalyst(num) should equal (UTF8String.fromString(ip)) 54 | } 55 | 56 | // test reversed conversion 57 | for (elem <- dataset) { 58 | val (ip, num) = elem 59 | convertFunction.reversed(ip) should equal (num) 60 | } 61 | } 62 | 63 | test("fail ip conversion for invalid input") { 64 | val convertFunction = IPv4ConvertFunction() 65 | var err = intercept[IllegalArgumentException] { 66 | convertFunction.reversed("123") 67 | } 68 | assert(err.getMessage.contains("Invalid IPv4: 123")) 69 | 70 | err = intercept[IllegalArgumentException] { 71 | convertFunction.reversed("1.2.3") 72 | } 73 | assert(err.getMessage.contains("Invalid IPv4: 1.2.3")) 74 | } 75 | 76 | test("protocol conversion") { 77 | val protocols: Array[Short] = (0 until 256).map(_.toShort).toArray 78 | 79 | val convertFunction = ProtocolConvertFunction() 80 | 81 | // test direct conversion 82 | for (num <- protocols) { 83 | val protocol = convertFunction.direct(num) 84 | if (!convertFunction.reversedProtocolMap.contains(protocol)) { 85 | protocol should be (num.toString()) 86 | } 87 | } 88 | 89 | // test direct conversion for all indices of protocol 90 | convertFunction.direct(1.toShort) should be ("ICMP") 91 | convertFunction.directCatalyst(1.toShort) should be (UTF8String.fromString("ICMP")) 92 | 93 | convertFunction.direct(3.toShort) should be ("GGP") 94 | convertFunction.directCatalyst(3.toShort) should be (UTF8String.fromString("GGP")) 95 | 96 | convertFunction.direct(6.toShort) should be ("TCP") 97 | convertFunction.directCatalyst(6.toShort) should be (UTF8String.fromString("TCP")) 98 | 99 | convertFunction.direct(8.toShort) should be ("EGP") 100 | convertFunction.directCatalyst(8.toShort) should be (UTF8String.fromString("EGP")) 101 | 102 | convertFunction.direct(12.toShort) should be ("PUP") 103 | convertFunction.directCatalyst(12.toShort) should be (UTF8String.fromString("PUP")) 104 | 105 | convertFunction.direct(17.toShort) should be ("UDP") 106 | convertFunction.directCatalyst(17.toShort) should be (UTF8String.fromString("UDP")) 107 | 108 | convertFunction.direct(20.toShort) should be ("HMP") 109 | convertFunction.directCatalyst(20.toShort) should be (UTF8String.fromString("HMP")) 110 | 111 | convertFunction.direct(27.toShort) should be ("RDP") 112 | convertFunction.directCatalyst(27.toShort) should be (UTF8String.fromString("RDP")) 113 | 114 | convertFunction.direct(46.toShort) should be ("RSVP") 115 | convertFunction.directCatalyst(46.toShort) should be (UTF8String.fromString("RSVP")) 116 | 117 | convertFunction.direct(47.toShort) should be ("GRE") 118 | convertFunction.directCatalyst(47.toShort) should be (UTF8String.fromString("GRE")) 119 | 120 | convertFunction.direct(50.toShort) should be ("ESP") 121 | convertFunction.directCatalyst(50.toShort) should be (UTF8String.fromString("ESP")) 122 | 123 | convertFunction.direct(51.toShort) should be ("AH") 124 | convertFunction.directCatalyst(51.toShort) should be (UTF8String.fromString("AH")) 125 | 126 | convertFunction.direct(66.toShort) should be ("RVD") 127 | convertFunction.directCatalyst(66.toShort) should be (UTF8String.fromString("RVD")) 128 | 129 | convertFunction.direct(88.toShort) should be ("IGMP") 130 | convertFunction.directCatalyst(88.toShort) should be (UTF8String.fromString("IGMP")) 131 | 132 | convertFunction.direct(89.toShort) should be ("OSPF") 133 | convertFunction.directCatalyst(89.toShort) should be (UTF8String.fromString("OSPF")) 134 | 135 | // test reversed conversion 136 | convertFunction.reversed("ICMP") should be (1) 137 | convertFunction.reversed("TCP") should be (6) 138 | convertFunction.reversed("UDP") should be (17) 139 | convertFunction.reversed("255") should be (255) 140 | } 141 | 142 | test("fail protocol conversion, if value is invalid") { 143 | val convertFunction = ProtocolConvertFunction() 144 | intercept[RuntimeException] { 145 | convertFunction.reversed("udp") 146 | } 147 | } 148 | } 149 | -------------------------------------------------------------------------------- /src/test/scala/com/github/sadikovi/spark/netflow/sources/NetFlowRegistrySuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.spark.netflow.sources 18 | 19 | import org.apache.spark.sql.types.{ByteType, IntegerType, LongType, ShortType, StringType} 20 | 21 | import com.github.sadikovi.testutil.UnitTestSuite 22 | 23 | class NetFlowRegistrySuite extends UnitTestSuite { 24 | test("resolve test provider") { 25 | // resolve full class name for provider 26 | var provider = NetFlowRegistry.lookupInterface( 27 | "com.github.sadikovi.spark.netflow.sources.DefaultProvider") 28 | Option(provider).nonEmpty should be (true) 29 | 30 | // resolve package name for provider 31 | provider = NetFlowRegistry.lookupInterface( 32 | "com.github.sadikovi.spark.netflow.sources") 33 | Option(provider).nonEmpty should be (true) 34 | 35 | intercept[ClassNotFoundException] { 36 | NetFlowRegistry.lookupInterface("wrong.package") 37 | } 38 | } 39 | 40 | test("resolve test interface") { 41 | // resolve full class name for provider 42 | var interface: ResolvedInterface = null 43 | 44 | try { 45 | interface = NetFlowRegistry.createInterface( 46 | "com.github.sadikovi.spark.netflow.sources.TestEmptyDefaultProvider") 47 | } catch { 48 | case iae: IllegalArgumentException => 49 | assert(iae.getMessage() == "Columns are empty for Interface: " + 50 | "com.github.sadikovi.spark.netflow.sources.TestEmptyInterface for version -1") 51 | case other: Throwable => throw other 52 | } 53 | 54 | intercept[NullPointerException] { 55 | interface = NetFlowRegistry.createInterface( 56 | "com.github.sadikovi.spark.netflow.sources.DefaultProvider") 57 | } 58 | 59 | intercept[UnsupportedOperationException] { 60 | NetFlowRegistry.createInterface( 61 | "com.github.sadikovi.spark.netflow.sources.WrongProvider") 62 | } 63 | 64 | interface = NetFlowRegistry.createInterface( 65 | "com.github.sadikovi.spark.netflow.sources.TestFullDefaultProvider") 66 | Option(interface).isEmpty should be (false) 67 | } 68 | 69 | test("check column consistency") { 70 | var interface: ResolvedInterface = null 71 | 72 | intercept[IllegalArgumentException] { 73 | NetFlowRegistry.createInterface( 74 | "com.github.sadikovi.spark.netflow.sources.TestEmptyDefaultProvider") 75 | } 76 | 77 | interface = NetFlowRegistry.createInterface( 78 | "com.github.sadikovi.spark.netflow.sources.TestFullDefaultProvider") 79 | // should not raise any errors 80 | interface.ensureColumnConsistency() 81 | 82 | // should fail with assertion error on duplicate columns 83 | intercept[AssertionError] { 84 | interface = NetFlowRegistry.createInterface( 85 | "com.github.sadikovi.spark.netflow.sources.Test1FullDefaultProvider") 86 | } 87 | 88 | // should fail with assertion error on duplicate internal column names 89 | intercept[AssertionError] { 90 | interface = NetFlowRegistry.createInterface( 91 | "com.github.sadikovi.spark.netflow.sources.Test2FullDefaultProvider") 92 | } 93 | } 94 | 95 | test("size in bytes for different types") { 96 | val interface = NetFlowRegistry.createInterface( 97 | "com.github.sadikovi.spark.netflow.sources.TestFullDefaultProvider") 98 | 99 | interface.sizeInBytes(ByteType) should be (1) 100 | interface.sizeInBytes(ShortType) should be (2) 101 | interface.sizeInBytes(IntegerType) should be (4) 102 | interface.sizeInBytes(LongType) should be (8) 103 | 104 | intercept[UnsupportedOperationException] { 105 | interface.sizeInBytes(StringType) 106 | } 107 | } 108 | } 109 | -------------------------------------------------------------------------------- /src/test/scala/com/github/sadikovi/spark/netflow/sources/VersionSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.spark.netflow.sources 18 | 19 | import org.apache.spark.sql.types.StringType 20 | 21 | import com.github.sadikovi.netflowlib.version.{NetFlowV5, NetFlowV7} 22 | import com.github.sadikovi.testutil.UnitTestSuite 23 | 24 | class VersionSuite extends UnitTestSuite { 25 | // interface for version 5 26 | val interface5 = NetFlowRegistry.createInterface("com.github.sadikovi.spark.netflow.version5") 27 | val interface7 = NetFlowRegistry.createInterface("com.github.sadikovi.spark.netflow.version7") 28 | 29 | test("get SQL schema v5") { 30 | var schema = interface5.getSQLSchema(false) 31 | 32 | schema.length should be (interface5.getColumns().length) 33 | schema.forall(p => p.dataType != StringType) 34 | 35 | schema = interface5.getSQLSchema(true) 36 | schema.length should be (interface5.getColumns().length) 37 | schema.exists(p => p.dataType == StringType) 38 | } 39 | 40 | test("get SQL schema v7") { 41 | var schema = interface7.getSQLSchema(false) 42 | 43 | schema.length should be (interface7.getColumns().length) 44 | schema.forall(p => p.dataType != StringType) 45 | 46 | schema = interface7.getSQLSchema(true) 47 | schema.length should be (interface7.getColumns().length) 48 | schema.exists(p => p.dataType == StringType) 49 | } 50 | 51 | test("get mapped columns") { 52 | interface5.getColumns().head should be (interface5.getFirstColumn()) 53 | } 54 | 55 | test("get mapped column as option") { 56 | interface5.getFirstColumnOption() should be (Some(interface5.getFirstColumn())) 57 | } 58 | 59 | test("get mapped column for name v5") { 60 | intercept[RuntimeException] { 61 | interface5.getColumn("wrong-column") 62 | } 63 | 64 | interface5.getColumn("unix_secs").internalColumn should be (NetFlowV5.FIELD_UNIX_SECS) 65 | interface5.getColumn("srcip").internalColumn should be (NetFlowV5.FIELD_SRCADDR) 66 | } 67 | 68 | test("get mapped column for name v7") { 69 | intercept[RuntimeException] { 70 | interface7.getColumn("wrong-column") 71 | } 72 | 73 | interface7.getColumn("unix_secs").internalColumn should be (NetFlowV7.FIELD_UNIX_SECS) 74 | interface7.getColumn("srcip").internalColumn should be (NetFlowV7.FIELD_SRCADDR) 75 | interface7.getColumn("router_sc").internalColumn should be (NetFlowV7.FIELD_ROUTER_SC) 76 | } 77 | 78 | test("get conversions for fields") { 79 | var convertFunction = interface5.getColumn("srcip").convertFunction 80 | convertFunction.isEmpty should be (false) 81 | convertFunction.get.isInstanceOf[IPv4ConvertFunction] should be (true) 82 | 83 | convertFunction = interface5.getColumn("unix_secs").convertFunction 84 | convertFunction.isEmpty should be (true) 85 | 86 | // get conversions for "router_sc", similar to "srcip" 87 | convertFunction = interface7.getColumn("router_sc").convertFunction 88 | convertFunction.isEmpty should be (false) 89 | } 90 | } 91 | -------------------------------------------------------------------------------- /src/test/scala/com/github/sadikovi/spark/netflow/sources/testproviders.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.spark.netflow.sources 18 | 19 | import com.github.sadikovi.netflowlib.predicate.Columns.{ByteColumn, ShortColumn, IntColumn, LongColumn} 20 | 21 | /** Test providers */ 22 | private class DefaultProvider extends NetFlowProvider { 23 | def createInterface(): ResolvedInterface = null 24 | } 25 | 26 | private class WrongProvider 27 | 28 | private class TestEmptyDefaultProvider extends NetFlowProvider { 29 | def createInterface(): ResolvedInterface = new TestEmptyInterface() 30 | } 31 | 32 | private class TestFullDefaultProvider extends NetFlowProvider { 33 | def createInterface(): ResolvedInterface = new TestFullInterface() 34 | } 35 | 36 | private class Test1FullDefaultProvider extends NetFlowProvider { 37 | def createInterface(): ResolvedInterface = new Test1FullInterface() 38 | } 39 | 40 | private class Test2FullDefaultProvider extends NetFlowProvider { 41 | def createInterface(): ResolvedInterface = new Test2FullInterface() 42 | } 43 | 44 | /** Test interfaces */ 45 | private class TestEmptyInterface extends ResolvedInterface { 46 | override protected val columns: Seq[MappedColumn] = Seq.empty 47 | 48 | override def version(): Short = -1 49 | } 50 | 51 | private class TestFullInterface extends ResolvedInterface { 52 | override protected val columns: Seq[MappedColumn] = Seq( 53 | MappedColumn("test", new LongColumn("test", 0), false, None)) 54 | 55 | override def version(): Short = -2 56 | } 57 | 58 | private class Test1FullInterface extends ResolvedInterface { 59 | override protected val columns: Seq[MappedColumn] = Seq( 60 | MappedColumn("duplicate", new LongColumn("duplicate", 0), false, None), 61 | MappedColumn("duplicate", new LongColumn("duplicate", 0), false, None)) 62 | 63 | override def version(): Short = -3 64 | } 65 | 66 | private class Test2FullInterface extends ResolvedInterface { 67 | override protected val columns: Seq[MappedColumn] = Seq( 68 | MappedColumn("test", new LongColumn("test1", 0), false, None), 69 | MappedColumn("test", new LongColumn("test2", 1), false, None)) 70 | 71 | override def version(): Short = -3 72 | } 73 | 74 | //////////////////////////////////////////////////////////////// 75 | // Fake provider and interface for NetFlowFilters suite 76 | //////////////////////////////////////////////////////////////// 77 | 78 | private class FakeDefaultProvider extends NetFlowProvider { 79 | def createInterface(): ResolvedInterface = new FakeInterface() 80 | } 81 | 82 | private class FakeInterface extends ResolvedInterface { 83 | override protected val columns: Seq[MappedColumn] = Seq( 84 | MappedColumn("col1", new ByteColumn("col1", 0), false, None), 85 | MappedColumn("col2", new ShortColumn("col2", 0), false, None), 86 | MappedColumn("col3", new IntColumn("col3", 0), false, None), 87 | MappedColumn("col4", new LongColumn("col4", 0), false, None)) 88 | 89 | override def version(): Short = -1 90 | } 91 | -------------------------------------------------------------------------------- /src/test/scala/com/github/sadikovi/spark/util/UtilsSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.spark.util 18 | 19 | import com.github.sadikovi.testutil.UnitTestSuite 20 | import com.github.sadikovi.testutil.implicits._ 21 | 22 | class UtilsSuite extends UnitTestSuite { 23 | 24 | test("parse buffer size") { 25 | val sizes = Seq("1024", "10Kb", "10K", "10Mb", "10M", "10Gb", "10G") 26 | val expected = Seq(1024L, 10240L, 10240L, 10485760L, 10485760L, 10737418240L, 10737418240L) 27 | val result = sizes.map(str => Utils.byteStringAsBytes(str)) 28 | result should be (expected) 29 | } 30 | 31 | test("fail to parse correctly") { 32 | val sizes = Seq("Kb", "corrupt", ".0", "-1Kb", "-1") 33 | sizes.foreach(str => { 34 | intercept[NumberFormatException] { 35 | Utils.byteStringAsBytes(str) 36 | } 37 | }) 38 | 39 | intercept[NullPointerException] { 40 | Utils.byteStringAsBytes(null) 41 | } 42 | } 43 | 44 | test("create UUID for string") { 45 | val str = baseDirectory() / "_metadata" 46 | val uuids = for (i <- 0 until 10) yield Utils.uuidForString(str) 47 | // check that uuids are the same 48 | uuids.distinct.length should be (1) 49 | uuids.distinct.head should be (uuids.head) 50 | } 51 | 52 | test("truncate mean - empty array") { 53 | intercept[IllegalArgumentException] { 54 | Utils.truncatedMean(Array.empty, 0.5) 55 | } 56 | } 57 | 58 | test("truncate mean - negative sample") { 59 | intercept[IllegalArgumentException] { 60 | Utils.truncatedMean(Array(1L, 2L), -0.5) 61 | } 62 | } 63 | 64 | test("truncate mean - simple array") { 65 | Utils.truncatedMean(Array(1L, 2L, 3L, 4L, 5L), 1.0) should be (3) 66 | Utils.truncatedMean(Array(1L, 2L, 3L, 4L, 5L), 0.8) should be (3) 67 | Utils.truncatedMean(Array(1L, 2L, 3L, 4L, 5L), 0.6) should be (2) 68 | } 69 | 70 | test("histogram mode - empty array") { 71 | intercept[IllegalArgumentException] { 72 | Utils.histogramMode(Array.empty, None) 73 | } 74 | } 75 | 76 | test("histogram mode - single element") { 77 | val mean = Utils.histogramMode(Array(1L), None) 78 | mean should be (1L) 79 | } 80 | 81 | test("histogram mode - simple array") { 82 | val arr = Array(1L, 1L, 1L, 1L, 2L, 2L, 3L, 4L, 5L) 83 | val mean = Utils.histogramMode(arr, None) 84 | mean should be (2L) 85 | } 86 | 87 | test("histogram mode - extreme array") { 88 | val arr = Array(1L, 2L) 89 | val mean = Utils.histogramMode(arr, None) 90 | mean should be (1L) 91 | } 92 | } 93 | -------------------------------------------------------------------------------- /src/test/scala/com/github/sadikovi/testutil/SparkBase.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.testutil 18 | 19 | import scala.util.Try 20 | 21 | import org.apache.log4j.{Level, Logger} 22 | import org.apache.spark.sql.SparkSession 23 | 24 | /** General Spark base */ 25 | private[testutil] trait SparkBase { 26 | @transient private[testutil] var _spark: SparkSession = null 27 | 28 | def createSparkSession(): SparkSession 29 | 30 | /** Start (or create) Spark session */ 31 | def startSparkSession(): Unit = { 32 | stopSparkSession() 33 | setLoggingLevel(Level.ERROR) 34 | _spark = createSparkSession() 35 | } 36 | 37 | /** Stop Spark session */ 38 | def stopSparkSession(): Unit = { 39 | if (_spark != null) { 40 | _spark.stop() 41 | } 42 | _spark = null 43 | } 44 | 45 | /** 46 | * Set logging level globally for all. 47 | * Supported log levels: 48 | * Level.OFF 49 | * Level.ERROR 50 | * Level.WARN 51 | * Level.INFO 52 | * @param level logging level 53 | */ 54 | def setLoggingLevel(level: Level) { 55 | Logger.getLogger("org").setLevel(level) 56 | Logger.getLogger("akka").setLevel(level) 57 | Logger.getRootLogger().setLevel(level) 58 | } 59 | 60 | /** Returns Spark session */ 61 | def spark: SparkSession = _spark 62 | 63 | /** Allow tests to set custom SQL configuration for duration of the closure */ 64 | def withSQLConf(pairs: (String, String)*)(func: => Unit): Unit = { 65 | val (keys, values) = pairs.unzip 66 | val currentValues = keys.map(key => Try(spark.conf.get(key)).toOption) 67 | (keys, values).zipped.foreach(spark.conf.set) 68 | try func finally { 69 | keys.zip(currentValues).foreach { 70 | case (key, Some(value)) => spark.conf.set(key, value) 71 | case (key, None) => spark.conf.unset(key) 72 | } 73 | } 74 | } 75 | 76 | /** Allow tests to set custom SQL configuration for duration of closure using map of options */ 77 | def withSQLConf(options: Map[String, String])(func: => Unit): Unit = { 78 | withSQLConf(options.toSeq: _*)(func) 79 | } 80 | } 81 | -------------------------------------------------------------------------------- /src/test/scala/com/github/sadikovi/testutil/SparkLocal.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.testutil 18 | 19 | import org.apache.log4j.Level 20 | import org.apache.spark.SparkConf 21 | import org.apache.spark.sql.SparkSession 22 | 23 | /** Spark context with master "local[4]" */ 24 | trait SparkLocal extends SparkBase { 25 | /** Loading Spark configuration for local mode */ 26 | private def localConf: SparkConf = { 27 | new SparkConf(). 28 | setMaster("local[4]"). 29 | setAppName("spark-local-test"). 30 | set("spark.driver.memory", "1g"). 31 | set("spark.executor.memory", "2g") 32 | } 33 | 34 | override def createSparkSession(): SparkSession = { 35 | SparkSession.builder().config(localConf).getOrCreate() 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /src/test/scala/com/github/sadikovi/testutil/TestBase.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.testutil 18 | 19 | import java.io.{InputStream, OutputStream} 20 | import java.util.UUID 21 | 22 | import org.apache.hadoop.conf.{Configuration => HadoopConf} 23 | import org.apache.hadoop.fs.{FileSystem, Path => HadoopPath} 24 | import org.apache.hadoop.fs.permission.FsPermission 25 | 26 | import org.apache.spark.sql.{DataFrame, Row} 27 | 28 | import com.github.sadikovi.testutil.implicits._ 29 | 30 | trait TestBase { 31 | val RESOLVER = "path-resolver" 32 | 33 | var path: String = "" 34 | 35 | // local file system for tests 36 | val fs = FileSystem.get(new HadoopConf(false)) 37 | 38 | /** returns raw path of the folder where it finds resolver */ 39 | private def getRawPath(): String = { 40 | if (path.isEmpty) { 41 | path = getClass.getResource("/" + RESOLVER).getPath() 42 | } 43 | path 44 | } 45 | 46 | /** base directory of the project */ 47 | final protected def baseDirectory(): String = { 48 | val original = getRawPath().split("/") 49 | require(original.length > 4, s"Path length is too short (<= 4): ${original.length}") 50 | val base = original.dropRight(4) 51 | var dir = "" 52 | for (suffix <- base) { 53 | if (suffix.nonEmpty) { 54 | dir = dir / suffix 55 | } 56 | } 57 | dir 58 | } 59 | 60 | /** main directory of the project (./src/main) */ 61 | final protected def mainDirectory(): String = { 62 | baseDirectory() / "src" / "main" 63 | } 64 | 65 | /** test directory of the project (./src/test) */ 66 | final protected def testDirectory(): String = { 67 | baseDirectory() / "src" / "test" 68 | } 69 | 70 | /** target directory of the project (./target) */ 71 | final protected def targetDirectory(): String = { 72 | baseDirectory() / "target" 73 | } 74 | 75 | /** Create directories for path recursively */ 76 | final protected def mkdirs(path: String): Boolean = { 77 | mkdirs(new HadoopPath(path)) 78 | } 79 | 80 | final protected def mkdirs(path: HadoopPath): Boolean = { 81 | fs.mkdirs(path) 82 | } 83 | 84 | /** Create empty file, similar to "touch" shell command, but creates intermediate directories */ 85 | final protected def touch(path: String): Boolean = { 86 | touch(new HadoopPath(path)) 87 | } 88 | 89 | final protected def touch(path: HadoopPath): Boolean = { 90 | fs.mkdirs(path.getParent) 91 | fs.createNewFile(path) 92 | } 93 | 94 | /** Delete directory / file with path. Recursive must be true for directory */ 95 | final protected def rm(path: String, recursive: Boolean): Boolean = { 96 | rm(new HadoopPath(path), recursive) 97 | } 98 | 99 | /** Delete directory / file with path. Recursive must be true for directory */ 100 | final protected def rm(path: HadoopPath, recursive: Boolean): Boolean = { 101 | fs.delete(path, recursive) 102 | } 103 | 104 | /** Open file for a path */ 105 | final protected def open(path: String): InputStream = { 106 | open(new HadoopPath(path)) 107 | } 108 | 109 | final protected def open(path: HadoopPath): InputStream = { 110 | fs.open(path) 111 | } 112 | 113 | /** Create file with a path and return output stream */ 114 | final protected def create(path: String): OutputStream = { 115 | create(new HadoopPath(path)) 116 | } 117 | 118 | final protected def create(path: HadoopPath): OutputStream = { 119 | fs.create(path) 120 | } 121 | 122 | /** Compare two DataFrame objects */ 123 | final protected def checkAnswer(df: DataFrame, expected: DataFrame): Unit = { 124 | val got = df.collect.map(_.toString).sortWith(_ < _) 125 | val exp = expected.collect.map(_.toString).sortWith(_ < _) 126 | assert(got.sameElements(exp), s"Failed to compare DataFrame ${got.mkString("[", ", ", "]")} " + 127 | s"with expected input ${exp.mkString("[", ", ", "]")}") 128 | } 129 | 130 | final protected def checkAnswer(df: DataFrame, expected: Seq[Row]): Unit = { 131 | val sc = df.sqlContext.sparkContext 132 | checkAnswer(df, df.sqlContext.createDataFrame(sc.parallelize(expected), df.schema)) 133 | } 134 | 135 | /** Create temporary directory on local file system */ 136 | def createTempDir( 137 | root: String = System.getProperty("java.io.tmpdir"), 138 | namePrefix: String = "lightcopy"): HadoopPath = { 139 | val dir = new HadoopPath(root / namePrefix / UUID.randomUUID().toString) 140 | fs.mkdirs(dir) 141 | dir 142 | } 143 | 144 | /** Execute block of code with temporary hadoop path and path permission */ 145 | private def withTempHadoopPath(path: HadoopPath, permission: Option[FsPermission]) 146 | (func: HadoopPath => Unit): Unit = { 147 | try { 148 | if (permission.isDefined) { 149 | fs.setPermission(path, permission.get) 150 | } 151 | func(path) 152 | } finally { 153 | fs.delete(path, true) 154 | } 155 | } 156 | 157 | /** Execute code block with created temporary directory with provided permission */ 158 | def withTempDir(permission: FsPermission)(func: HadoopPath => Unit): Unit = { 159 | withTempHadoopPath(createTempDir(), Some(permission))(func) 160 | } 161 | 162 | /** Execute code block with created temporary directory */ 163 | def withTempDir(func: HadoopPath => Unit): Unit = { 164 | withTempHadoopPath(createTempDir(), None)(func) 165 | } 166 | } 167 | -------------------------------------------------------------------------------- /src/test/scala/com/github/sadikovi/testutil/package.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 sadikovi 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.github.sadikovi.testutil 18 | 19 | import java.io.File 20 | 21 | import org.apache.hadoop.fs.{Path => HadoopPath} 22 | 23 | import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} 24 | import org.scalatest.funsuite.AnyFunSuite 25 | import org.scalatest.matchers.should.Matchers 26 | 27 | package object implicits { 28 | implicit class PathStringBuilder(path: String) { 29 | def /(suffix: String): String = path + File.separator + suffix 30 | 31 | def `:`(append: String): String = path + File.pathSeparator + append 32 | } 33 | 34 | implicit class PathBuilder(path: HadoopPath) { 35 | def /(suffix: String): HadoopPath = path.suffix(s"${HadoopPath.SEPARATOR}$suffix") 36 | } 37 | } 38 | 39 | /** abstract general testing class */ 40 | abstract class UnitTestSuite extends AnyFunSuite with Matchers 41 | with TestBase with BeforeAndAfterAll with BeforeAndAfter 42 | -------------------------------------------------------------------------------- /version.sbt: -------------------------------------------------------------------------------- 1 | version in ThisBuild := "2.2.0-SNAPSHOT" 2 | --------------------------------------------------------------------------------