├── .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 | [](https://travis-ci.org/sadikovi/spark-netflow)
5 | [](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 |
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