├── README.md
├── config
├── c3p0.properties
├── conf.properties
└── log4j.properties
├── doc
└── scripts.txt
├── pom.xml
└── src
├── main
└── scala
│ ├── META-INF
│ └── MANIFEST.MF
│ ├── com
│ └── hb
│ │ ├── analysis
│ │ └── NginxFlowAnalysis.scala
│ │ ├── falcon
│ │ ├── NginxFalcon.scala
│ │ ├── Pack.scala
│ │ └── Sender.scala
│ │ ├── model
│ │ ├── IPMapping.scala
│ │ ├── IPRecord.scala
│ │ ├── IpToLong.scala
│ │ └── LocationInfo.scala
│ │ ├── pool
│ │ └── ConnectionPool.scala
│ │ └── utils
│ │ └── Num.scala
│ ├── consumer
│ └── kafka
│ │ ├── Broker.java
│ │ ├── BrokerHosts.java
│ │ ├── Config.java
│ │ ├── DynamicBrokersReader.java
│ │ ├── DynamicPartitionConnections.java
│ │ ├── FailedFetchException.java
│ │ ├── GlobalPartitionInformation.java
│ │ ├── IBrokerReader.java
│ │ ├── IdentityMessageHandler.java
│ │ ├── KafkaConfig.java
│ │ ├── KafkaConsumer.java
│ │ ├── KafkaError.java
│ │ ├── KafkaMessageHandler.java
│ │ ├── KafkaUtils.java
│ │ ├── MessageAndMetadata.java
│ │ ├── PIDController.java
│ │ ├── Partition.java
│ │ ├── PartitionCoordinator.java
│ │ ├── PartitionManager.java
│ │ ├── PartitionOffsetPair.java
│ │ ├── ProcessedOffsetManager.java
│ │ ├── ReceiverLauncher.java
│ │ ├── ReceiverStreamListener.java
│ │ ├── ScalaUtil.java
│ │ ├── Utils.java
│ │ ├── ZkBrokerReader.java
│ │ ├── ZkCoordinator.java
│ │ ├── ZkState.java
│ │ └── client
│ │ ├── KafkaRangeReceiver.java
│ │ ├── KafkaReceiver.java
│ │ └── SampleConsumer.java
│ └── resources
│ └── ipCity.properties
└── test
└── scala
└── com
└── hb
└── producer
└── KafkaProducerSimulator.scala
/README.md:
--------------------------------------------------------------------------------
1 | # spark real-time analysis
2 | Spark version v2.0.0
3 | Kafka version v0.9.0.1
4 |
5 | # Introduction
6 | flume采集日志传输到kafka,再通过spark-streaming进行实时分析,得到pv,uv,用户分布图等指标,结果保存两份。一份通过http接口发送给open-falcon做监控和告警,一部分存到数据库。
7 | # Authors
8 | |Email Address | Name |
9 | |simonwindwf@gmail.com| Simon |
10 |
11 | # Contents
12 |
13 | ## Initializing StreamingContext
14 |
15 | ```
16 | ssc = new StreamingContext(conf, Seconds(60))
17 | val messages = ReceiverLauncher.launch(ssc, props, numberOfReceivers, StorageLevel.MEMORY_ONLY)
18 | val partitonOffset_stream = ProcessedOffsetManager.getPartitionOffset(messages, props)
19 | ProcessedOffsetManager.persists(partitonOffset_stream, props)
20 | ```
21 |
22 | ## Filter and Split
23 |
24 | 日志分隔符为"^^A"
25 |
26 | ```
27 | val filterMessages = messages.map { x => new String(x.getPayload) }
28 | .filter(s => s.contains("GET") || s.contains("POST"))
29 | .map(line => line.split("\\^\\^A"))
30 | .map(line => Array(line(column1), line(column2).split(" ")(1), line(column3), line(column4), line(column5)))
31 | ```
32 | ## Jobs
33 | 分多个JOB计算指标,详情见代码注释。结果通过foreachRdd和foreachPartiion算子发送给外部接口和数据库
34 |
35 | ## Algorithms
36 | ### 1:百分位的计算
37 |
38 | 目前采用的是快排再求相应百分位的值,数据量很大的时候可能会有瓶颈。可以考虑采用分治法求第K大个数,做粗略估计
39 |
40 | ```
41 | /**
42 | * @param arr 输入数组
43 | * @return 快速排序后的数组
44 | */
45 | def quickSort(arr: Array[Double]): Array[Double] = {
46 | if (arr.length <= 1)
47 | arr
48 | else {
49 | val index = arr(arr.length / 2)
50 | Array.concat(
51 | quickSort(arr filter (index >)),
52 | arr filter (_ == index),
53 | quickSort(arr filter (index <))
54 | )
55 | }
56 | }
57 |
58 | /**
59 | * @param arr 输入数组
60 | * @return p 百分位
61 | */
62 | def percentile(arr: Array[Double], p: Double) = {
63 | if (p > 1 || p < 0) throw new IllegalArgumentException("p must be in [0,1]")
64 | val sorted = quickSort(arr)
65 | val f = (sorted.length + 1) * p
66 | val i = f.toInt
67 | if (i == 0) sorted.head
68 | else if (i >= sorted.length) sorted.last
69 | else {
70 | sorted(i - 1) + (f - i) * (sorted(i) - sorted(i - 1))
71 | }
72 | }
73 | ```
74 |
75 | ### 2:每天用户数
76 |
77 | 采用updateStateByKey算子保存HLL对象,并且在每天0点的时候重新计数
78 |
79 |
80 | ```
81 |
82 | val updateCardinal = (values: Seq[String], state: Option[HyperLogLogPlus]) => {
83 | val calendar = Calendar.getInstance()
84 | val hh = calendar.get(Calendar.HOUR_OF_DAY)
85 | val mm = calendar.get(Calendar.MINUTE)
86 | if (hh == 0 && mm == 0 ) {
87 | val hll = new HyperLogLogPlus(14)
88 | for (value <- values) { hll.offer(value) }
89 | Option(hll)
90 | }
91 | else {
92 | if (state.nonEmpty) {
93 | val hll = state.get
94 | for (value <- values) { hll.offer(value) }
95 | Option(hll)
96 | } else {
97 | val hll = new HyperLogLogPlus(14)
98 | for (value <- values) { hll.offer(value) }
99 | Option(hll)
100 | }
101 | }
102 | }
103 | //计算结果实时UV只有一条数据可不用批量提交
104 | filterMessages.map(x => (null, x(3))).updateStateByKey(updateCardinal)
105 | .map(x => x._2.cardinality).foreachRDD(rdd => {
106 | ....(省略)
107 | }
108 | )
109 | ```
110 |
111 | ### 3:各省用户数
112 |
113 | 计算为笛卡尔乘积,待改进
114 |
115 |
116 | ```
117 | if (aggregateProvinceFlag) {
118 | ipRecords.map(x => (IpToLong.IPv4ToLong(x._1.trim),x._2))
119 | .map(x => (LocationInfo.findLocation(ipMapBroadCast.value,x._1),x._2))
120 | .reduceByKey(_+_) foreachRDD( rdd => {
121 | rdd.foreachPartition { data =>
122 | if (data != null) {
123 | val conn = ConnectionPool.getConnectionPool(propC3p0BroadCast.value).getConnection
124 | conn.setAutoCommit(false)
125 |
126 | val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm")
127 | val currentTimestamp = sdf.format(new Date())
128 |
129 | try {
130 | val sql = "insert into uv_province(time,province,uv) values (?,?,?)"
131 | val preparedStatement = conn.prepareStatement(sql)
132 | data.foreach(r => {
133 | preparedStatement.setString(1, currentTimestamp)
134 | preparedStatement.setString(2, r._1.toString)
135 | preparedStatement.setInt(3, r._2)
136 | preparedStatement.addBatch()
137 | })
138 |
139 | preparedStatement.executeBatch()
140 | conn.commit()
141 | preparedStatement.clearBatch()
142 | } catch {
143 | case e:SQLException => conn.rollback()
144 | case e:Exception => e.printStackTrace()
145 | } finally {
146 | conn.close()
147 | }
148 | }
149 | }
150 | }
151 | )
152 | }
153 | ```
154 |
155 |
156 | # Output
157 |
158 | 具体指标为:
159 |
160 | 1:每分钟请求数
161 |
162 | 2:每分钟错误请求数
163 |
164 | 3:每分钟各错误码的数量
165 |
166 | 4:每分钟用户数(精确统计,distinct去重)
167 |
168 | 5:每天实时用户数(HyperLogLog基数估计)
169 |
170 | 6: 99th,95th,75,50th的百分位时延
171 |
172 | 7: 各省用户分布
173 |
174 | 8:访问频次异常IP
175 |
176 | # Defect
177 |
178 | 1: 多个job每个job每分钟都会跟falcon的接口建立HTTP连接,链接过于频繁。
179 |
180 | 2: 每60秒生成一个RDD,计算一次指标。不是按访问时间计算的指标,统计有误差。改进方法:从日志里面获取访问时间作为key再groupByKey进行统计,存储到hbase,延迟日志的补算可采用hbase的counter。如果用mysql做补算,要先查询,再update,性能会很差,不推荐。
181 |
182 | 3: 时延相关指标涉及到对整个时延的集合进行排序和求百分位,目前的方法可能统计会有误差。但是一般情况下延迟到达的日志记录很少,误差也可以忽略。
183 |
--------------------------------------------------------------------------------
/config/c3p0.properties:
--------------------------------------------------------------------------------
1 | #jdbc基本信息
2 | driverClass=com.mysql.jdbc.Driver
3 | jdbcUrl=jdbc:mysql://192.168.2.245:3306/nginxlog?useUnicode=true&characterEncoding=UTF8
4 | user=root
5 | password=
6 |
7 | #c3p0连接池信息
8 | c3p0.minPoolSize=4
9 | c3p0.maxPoolSize=10
10 |
11 | #当连接池中的连接耗尽的时候c3p0一次同时获取的连接数
12 | c3p0.acquireIncrement=3
13 | #定义在从数据库获取新连接失败后重复尝试的次数
14 | c3p0.acquireRetryAttempts=60
15 | #两次连接中间隔时间,单位毫秒
16 | c3p0.acquireRetryDelay=1000
17 | #连接关闭时默认将所有未提交的操作回滚
18 | c3p0.autoCommitOnClose=false
19 | #当连接池用完时客户端调用getConnection()后等待获取新连接的时间,超时后将抛出SQLException,如设为0则无限期等待。单位毫秒
20 | c3p0.checkoutTimeout=3000
21 | #每120秒检查所有连接池中的空闲连接。Default: 0
22 | c3p0.idleConnectionTestPeriod=120
23 | #最大空闲时间,60秒内未使用则连接被丢弃。若为0则永不丢弃。Default: 0
24 | c3p0.maxIdleTime=60
25 | #如果设为true那么在取得连接的同时将校验连接的有效性。Default: false
26 | c3p0.testConnectionOnCheckin=true
27 | #c3p0将建一张名为c3p0TestTable的空表,并使用其自带的查询语句进行测试。
28 | jdbc.automaticTestTable = c3p0TestTable
29 |
30 |
--------------------------------------------------------------------------------
/config/conf.properties:
--------------------------------------------------------------------------------
1 | zkAddress = 192.168.2.245:2181,192.168.2.246:2181,192.168.2.247:2181
2 | group = nginxconsumer
3 | topics = nginx
4 | numberOfReceivers = 3
5 | falconUrl = http://127.0.0.1:1988/v1/push
6 | splitColumns = 0,2,4,7,12
7 | percentileNums = 0.99,0.95,0.75,0.50
8 | abnormalVisitThreshold = 50
9 | aggregateProvinceFlag = false
10 |
--------------------------------------------------------------------------------
/config/log4j.properties:
--------------------------------------------------------------------------------
https://raw.githubusercontent.com/simon-wind/realtime-analysis/2030c5ecc467158a75f8a76c7f5266dc626623d4/config/log4j.properties
--------------------------------------------------------------------------------
/doc/scripts.txt:
--------------------------------------------------------------------------------
1 | //每省用户数
2 | create table uv_province (
3 | time DATETIME,
4 | province varchar(50),
5 | uv int,
6 | id INT(20) not null AUTO_INCREMENT,
7 | primary key (id)
8 | ) charset=utf8 auto_increment=1;
9 |
10 | //访问频次过高IP
11 | create table abnormal_ip (
12 | time DATETIME,
13 | ip varchar(20),
14 | frequency int,
15 | id INT(20) not null AUTO_INCREMENT,
16 | primary key (id)
17 | ) charset=utf8 auto_increment=1;
18 |
19 | //请求响应相关指标
20 | create table requests_minute (
21 | time DATETIME,
22 | pv_minute int default 0 not null,
23 | errs_minute int default 0 not null,
24 | errs_400 int default 0 not null,
25 | errs_404 int default 0 not null,
26 | errs_405 int default 0 not null,
27 | errs_408 int default 0 not null,
28 | errs_499 int default 0 not null,
29 | errs_502 int default 0 not null,
30 | errs_503 int default 0 not null,
31 | uv_minute int default 0 not null
32 | ) charset=utf8 auto_increment=1;
33 |
34 | //时延指标
35 | create table latency (
36 | time DATETIME,
37 | pen99th double,
38 | pen95th double,
39 | pen75th double,
40 | pen50th double
41 | ) charset=utf8 auto_increment=1;
42 |
43 | //每天用户数
44 | create table uv_day (
45 | time DATETIME,
46 | uv int
47 | ) charset=utf8 auto_increment=1;
--------------------------------------------------------------------------------
/pom.xml:
--------------------------------------------------------------------------------
1 |
2 | 4.0.0
3 | scala
4 | log-analysis
5 | 1.0-SNAPSHOT
6 | 2008
7 |
8 | 2.11.8
9 |
10 |
11 |
12 |
13 | scala-tools.org
14 | Scala-Tools Maven2 Repository
15 | http://scala-tools.org/repo-releases
16 |
17 |
18 |
19 |
20 |
21 | scala-tools.org
22 | Scala-Tools Maven2 Repository
23 | http://scala-tools.org/repo-releases
24 |
25 |
26 |
27 |
28 |
29 |
30 | org.scala-lang
31 | scala-library
32 | ${scala.version}
33 |
34 |
35 | junit
36 | junit
37 | 4.4
38 | test
39 |
40 |
41 | org.specs
42 | specs
43 | 1.2.5
44 | test
45 |
46 |
47 | org.apache.spark
48 | spark-streaming_2.11
49 | 2.0.0
50 |
51 |
52 | org.apache.spark
53 | spark-core_2.11
54 | 2.0.0
55 |
56 |
57 | org.apache.spark
58 | spark-sql_2.11
59 | 2.0.0
60 |
61 |
62 | mysql
63 | mysql-connector-java
64 | 5.1.25
65 |
66 |
67 | org.apache.spark
68 | spark-streaming-kafka-0-8_2.11
69 | 2.0.0
70 |
71 |
72 | org.apache.httpcomponents
73 | httpclient
74 | 4.5.2
75 |
76 |
77 | com.google.code.gson
78 | gson
79 | 2.7
80 |
81 |
82 | org.apache.hive
83 | hive-serde
84 | 2.1.1
85 |
86 |
87 | com.googlecode.json-simple
88 | json-simple
89 | 1.1
90 |
91 |
92 | c3p0
93 | c3p0
94 | 0.9.1.2
95 |
96 |
97 |
98 |
99 |
100 | src/main/scala
101 | src/test/scala
102 |
103 |
104 | org.scala-tools
105 | maven-scala-plugin
106 |
107 |
108 |
109 | compile
110 | testCompile
111 |
112 |
113 |
114 |
115 |
116 | ${scala.version}
117 |
118 | -target:jvm-1.5
119 |
120 |
121 |
122 |
123 | org.apache.maven.plugins
124 | maven-eclipse-plugin
125 |
126 | true
127 |
128 | ch.epfl.lamp.sdt.core.scalabuilder
129 |
130 |
131 | ch.epfl.lamp.sdt.core.scalanature
132 |
133 |
134 | org.eclipse.jdt.launching.JRE_CONTAINER
135 | ch.epfl.lamp.sdt.launching.SCALA_CONTAINER
136 |
137 |
138 |
139 |
140 |
141 |
142 | src/main/scala/
143 |
144 | META-INF/*.DSA
145 |
146 |
147 |
148 |
149 |
150 |
151 |
152 |
153 | org.scala-tools
154 | maven-scala-plugin
155 |
156 | ${scala.version}
157 |
158 |
159 |
160 |
161 |
162 |
163 |
164 | jdk-1.8
165 |
166 | true
167 | 1.8
168 |
169 |
170 | 1.8
171 | 1.8
172 | 1.8
173 |
174 |
175 |
176 |
177 |
--------------------------------------------------------------------------------
/src/main/scala/META-INF/MANIFEST.MF:
--------------------------------------------------------------------------------
1 | Manifest-Version: 1.0
2 | Main-Class: com.hb.analysis.NginxFlowAnalysis
3 |
4 |
--------------------------------------------------------------------------------
/src/main/scala/com/hb/analysis/NginxFlowAnalysis.scala:
--------------------------------------------------------------------------------
1 | package com.hb.analysis
2 |
3 | import java.io.{File, FileInputStream}
4 | import java.sql.SQLException
5 | import java.util.{ArrayList, Calendar, Date, Properties}
6 | import java.text.SimpleDateFormat
7 |
8 | import org.apache.log4j.Logger
9 | import org.apache.log4j.PropertyConfigurator
10 | import org.apache.spark.SparkConf
11 | import org.apache.spark.storage.StorageLevel
12 | import org.apache.spark.streaming.Seconds
13 | import org.apache.spark.streaming.StreamingContext
14 | import consumer.kafka.ReceiverLauncher
15 | import consumer.kafka.ProcessedOffsetManager
16 | import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus
17 | import com.hb.falcon.{Pack, Sender}
18 | import com.hb.model.{IPMapping, IpToLong, LocationInfo}
19 | import com.hb.pool.ConnectionPool
20 | import com.hb.utils.Num
21 |
22 | /**
23 | * Created by Simon on 2017/2/23.
24 | */
25 |
26 | object NginxFlowAnalysis {
27 | //接口参数
28 | private val endpoint = "ngxmaster"
29 | private val step = 60
30 | private val counterType = "GAUGE"
31 | private val tags = "_Minute"
32 | private val ip_file = "/resources/ipCity.properties"
33 |
34 | //计算指标
35 | private val metric1 = "pv_min"
36 | private val metric2 = "errcounts"
37 | private val metric3 = "errs"
38 | private val metric4 = "uv_Min"
39 | private val metric5 = "pen99th"
40 | private val metric6 = "pen95th"
41 | private val metric7 = "pen75th"
42 | private val metric8 = "pen50th"
43 | private val metric9 = "uvTotal"
44 | private val metric10= "abnormalIPCounts"
45 |
46 |
47 | val logger = Logger.getLogger(NginxFlowAnalysis.getClass.getName)
48 |
49 | /**
50 | * 更新HyperLogLogPlus对象
51 | */
52 | val updateCardinal = (values: Seq[String], state: Option[HyperLogLogPlus]) => {
53 | val calendar = Calendar.getInstance()
54 | val hh = calendar.get(Calendar.HOUR_OF_DAY)
55 | val mm = calendar.get(Calendar.MINUTE)
56 | if (hh == 0 && mm == 0 ) {
57 | val hll = new HyperLogLogPlus(14)
58 | for (value <- values) { hll.offer(value) }
59 | Option(hll)
60 | }
61 | else {
62 | if (state.nonEmpty) {
63 | val hll = state.get
64 | for (value <- values) { hll.offer(value) }
65 | Option(hll)
66 | } else {
67 | val hll = new HyperLogLogPlus(14)
68 | for (value <- values) { hll.offer(value) }
69 | Option(hll)
70 | }
71 | }
72 | }
73 |
74 |
75 | def main(args: Array[String]): Unit = {
76 | if (args.length != 3) {
77 | println("Usage: spark-2.0.0/bin/spark-submit --class com.hb.analysis.NginxFlowAnalysis --master yarn --num-executors 4 --executor-memory 8G --executor-cores 4 --driver-memory 1000M log-analysis.jar conf/log4j.properties conf/conf.properties conf/c3p0.properties" )
78 | System.exit(1)
79 | }
80 |
81 | val Array(logProperties,configProperties,dbProperties) = args
82 | val checkpointDirectory = "analysisCheckpoint"
83 |
84 | def createContext(logConfig : String,applicationConfig : String, dbConfig : String) = {
85 | PropertyConfigurator.configure(logConfig)
86 |
87 | //获取应用相关配置
88 | val in= new FileInputStream(new File(applicationConfig))
89 | val properties = new Properties
90 | properties.load(in)
91 |
92 | val master = properties.getProperty("master")
93 | logger.info("master address is : " + master)
94 | val zkHosts = properties.getProperty("zkAddress").split(",").map(line => line.split(":")(0)).mkString(",")
95 | logger.info("zkHosts is : " + zkHosts)
96 | val zkPort = properties.getProperty("zkAddress").split(",")(0).split(":")(1)
97 | logger.info("zkPort is : " + zkPort)
98 | val zkAddress = properties.getProperty("zkAddress")
99 | logger.info("zkAddress is : " + zkAddress)
100 | val group = properties.getProperty("group")
101 | logger.info("consumer group id is : " + group)
102 | val url = properties.getProperty("falconUrl")
103 | logger.info("falcon http interface is : " + url)
104 | val topic = properties.getProperty("topics")
105 | logger.info("consumer topic is : " + topic)
106 |
107 | val numberOfReceivers = properties.getProperty("numberOfReceivers").toInt
108 | val abnormalVisitThreshold = properties.getProperty("abnormalVisitThreshold").toInt
109 | val aggregateProvinceFlag = properties.getProperty("aggregateProvinceFlag").toBoolean
110 | val splitColumns = properties.getProperty("splitColumns")
111 | val percentileNums = properties.getProperty("percentileNums")
112 |
113 | //split提取ip,请求api,状态码,设备id,时延五个维度的数据
114 | val column1 = splitColumns.split(",")(0).toInt
115 | val column2 = splitColumns.split(",")(1).toInt
116 | val column3 = splitColumns.split(",")(2).toInt
117 | val column4 = splitColumns.split(",")(3).toInt
118 | val column5 = splitColumns.split(",")(4).toInt
119 |
120 | //各百分位指标
121 | val percentile1 = percentileNums.split(",")(0).toFloat
122 | val percentile2 = percentileNums.split(",")(1).toFloat
123 | val percentile3 = percentileNums.split(",")(2).toFloat
124 | val percentile4 = percentileNums.split(",")(3).toFloat
125 |
126 | val kafkaProperties: Map[String, String] =
127 | Map("zookeeper.hosts" -> zkHosts,
128 | "zookeeper.port" -> zkPort,
129 | "kafka.topic" -> topic,
130 | "zookeeper.consumer.connection" -> zkAddress,
131 | "kafka.consumer.id" -> group,
132 | "consumer.forcefromstart" -> "true",
133 | "consumer.backpressure.enabled" -> "true"
134 | )
135 |
136 | val props = new java.util.Properties()
137 | kafkaProperties foreach { case (key, value) => props.put(key, value) }
138 |
139 | val conf = new SparkConf().setAppName("NginxPerformanceMonitor")
140 | .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
141 | logger.info("initializing spark config")
142 |
143 | val ssc = new StreamingContext(conf, Seconds(60))
144 | logger.info("starting spark streaming job")
145 |
146 | val inC3p0 = new FileInputStream(new File(dbConfig))
147 | val propC3p0 = new Properties()
148 | propC3p0.load(inC3p0)
149 | val propC3p0BroadCast = ssc.sparkContext.broadcast(propC3p0)
150 |
151 | //广播IP段和city的对应关系
152 | val ipMap = IPMapping.getIpMapping(ip_file)
153 | val ipMapBroadCast = ssc.sparkContext.broadcast(ipMap)
154 |
155 | ssc.checkpoint(checkpointDirectory)
156 |
157 | val messages = ReceiverLauncher.launch(ssc, props, numberOfReceivers, StorageLevel.MEMORY_ONLY)
158 | val partitonOffset_stream = ProcessedOffsetManager.getPartitionOffset(messages, props)
159 | logger.info("fetching current offset from zookeeper cluster")
160 |
161 | /**
162 | * 分多个JOB计算指标。
163 | */
164 | val filterMessages = messages.map { x => new String(x.getPayload) }
165 | .filter(s => s.contains("GET") || s.contains("POST"))
166 | .map(line => line.split("\\^\\^A"))
167 | .map(line => Array(line(column1), line(column2).split(" ")(1), line(column3), line(column4), line(column5)))
168 |
169 | import org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK
170 | filterMessages.persist(MEMORY_AND_DISK)
171 |
172 | filterMessages.foreachRDD(rdd => {
173 | val ls = new ArrayList[Any]
174 | /**
175 | * 计算每分钟请求数
176 | */
177 | val counts = rdd.count()
178 | val countsJson = Pack.pack(endpoint, metric1, step, counts, counterType,tags)
179 | ls.add(countsJson)
180 |
181 | /**
182 | * 计算每分钟错误请求数
183 | */
184 | val errRecords = rdd.filter(_(2).trim().toInt >= 400).cache()
185 | val errCounts = errRecords.count()
186 | val errCountsJson = Pack.pack(endpoint, metric2, step, errCounts, counterType,tags)
187 | ls.add(errCountsJson)
188 |
189 | /**
190 | * 计算每分钟不同错误请求数
191 | */
192 | val diffErrors = errRecords.map(x => (x(2).trim.toInt, 1)).reduceByKey(_+_).collect()
193 | diffErrors.foreach{ x =>
194 | ls.add(Pack.pack(endpoint, metric3 + x._1.toString, step, x._2.toDouble, counterType,tags))
195 | }
196 |
197 | /**
198 | * 每分钟用户数
199 | */
200 | val uniqueVisitor = rdd.map(x => (x(3),1)).reduceByKey(_ + _).count()
201 | val uniqueVisitorJson = Pack.pack(endpoint, metric4, step, uniqueVisitor, counterType,tags)
202 | ls.add(uniqueVisitorJson)
203 |
204 | //输出给open-falcon agent
205 | Sender.sender(ls,url)
206 |
207 | //保存到数据库
208 | val conn = ConnectionPool.getConnectionPool(propC3p0BroadCast.value).getConnection
209 | conn.setAutoCommit(false)
210 |
211 | val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm")
212 | val currentTimestamp = sdf.format(new Date())
213 |
214 | try {
215 | val sql = "insert into requests_minute(time,pv_minute,errs_minute,errs_400,errs_404,errs_405,errs_408,errs_499,errs_502,errs_503,uv_minute) values (?,?,?,?,?,?,?,?,?,?,?)"
216 | val preparedStatement = conn.prepareStatement(sql)
217 | preparedStatement.setString(1, currentTimestamp)
218 | preparedStatement.setLong(2, counts)
219 | preparedStatement.setLong(3, errCounts)
220 |
221 | diffErrors.foreach{ errs => {
222 | errs._1.toInt match {
223 | case 400 => preparedStatement.setLong(4, errs._2)
224 | case 404 => preparedStatement.setLong(5, errs._2)
225 | case 405 => preparedStatement.setLong(6, errs._2)
226 | case 408 => preparedStatement.setLong(7, errs._2)
227 | case 499 => preparedStatement.setLong(8, errs._2)
228 | case 502 => preparedStatement.setLong(9, errs._2)
229 | case 503 => preparedStatement.setLong(10, errs._2)
230 | case _ =>
231 | }
232 | }
233 | }
234 |
235 | val errColumnMap : Map[Int,Int] = Map (400 -> 4, 404 -> 5, 405 -> 6, 408 -> 7, 499 -> 8, 502 -> 9, 503 -> 10)
236 | val errAllSet : Set[Int]= Set(400,404,405,408,499,502,503)
237 | val errGotSet = diffErrors.map(x => x._1.toInt).toSet
238 | val errLostSet = errAllSet -- errGotSet
239 | //如果记录里面没有相关错误码,error次数置0
240 | for (key <- errLostSet) {preparedStatement.setLong(errColumnMap.get(key).get,0)}
241 |
242 | preparedStatement.setLong(11, uniqueVisitor)
243 |
244 | preparedStatement.addBatch()
245 | preparedStatement.executeBatch()
246 | conn.commit()
247 | preparedStatement.clearBatch()
248 | } catch {
249 | case e:SQLException => conn.rollback()
250 | case e:Exception => e.printStackTrace()
251 | } finally {
252 | conn.close()
253 | }
254 | }
255 | )
256 |
257 | filterMessages.foreachRDD(rdd => {
258 | /**
259 | * 各百分位时延迟,99,95,75,50百分位
260 | * 每分钟的数据量不大的时候,为简化逻辑,用repartition函数进行partition合并,在一个worker进行计算,数据量大了应分布式计算再合并
261 | */
262 |
263 | rdd.map(x => x(4).toDouble).repartition(1).foreachPartition { partitionRecords =>
264 |
265 | val arrRecords = partitionRecords.toArray
266 | if (arrRecords.length > 0) {
267 | val ls = new ArrayList[Any]()
268 | val pen99th = Num.percentile(arrRecords, percentile1)
269 | val pen95th = Num.percentile(arrRecords, percentile2)
270 | val pen75th = Num.percentile(arrRecords, percentile3)
271 | val pen50th = Num.percentile(arrRecords, percentile4)
272 |
273 | val pen99thJson = Pack.pack(endpoint, metric5, step, pen99th, counterType,tags)
274 | val pen95thJson = Pack.pack(endpoint, metric6, step, pen95th, counterType,tags)
275 | val pen75thJson = Pack.pack(endpoint, metric7, step, pen75th, counterType,tags)
276 | val pen50thJson = Pack.pack(endpoint, metric8, step, pen50th, counterType,tags)
277 |
278 | ls.add(pen99thJson)
279 | ls.add(pen95thJson)
280 | ls.add(pen75thJson)
281 | ls.add(pen50thJson)
282 |
283 | //发送给open-falcon agent
284 | Sender.sender(ls,url)
285 | //保存到数据库
286 | val conn = ConnectionPool.getConnectionPool(propC3p0BroadCast.value).getConnection
287 | conn.setAutoCommit(false)
288 |
289 | val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm")
290 | val currentTimestamp = sdf.format(new Date())
291 | try {
292 | val sql = "insert into latency(time,pen99th,pen95th,pen75th,pen50th) values (?,?,?,?,?)"
293 | val preparedStatement = conn.prepareStatement(sql)
294 | preparedStatement.setString(1, currentTimestamp)
295 | preparedStatement.setDouble(2, pen99th)
296 | preparedStatement.setDouble(3, pen95th)
297 | preparedStatement.setDouble(4, pen75th)
298 | preparedStatement.setDouble(5, pen50th)
299 |
300 | preparedStatement.addBatch()
301 | preparedStatement.executeBatch()
302 | conn.commit()
303 | preparedStatement.clearBatch()
304 | } catch {
305 | case e:SQLException => conn.rollback()
306 | case e:Exception => e.printStackTrace()
307 | } finally {
308 | conn.close()
309 | }
310 | }
311 | }
312 | }
313 | )
314 |
315 | /**
316 | * 总用户数UV,采用基数估计
317 | */
318 | filterMessages.map(x => (null, x(3))).updateStateByKey(updateCardinal)
319 | .map(x => x._2.cardinality).foreachRDD(rdd => {
320 |
321 | rdd.foreach { x =>
322 | val ls = new ArrayList[Any]
323 | val uvTotalJson = Pack.pack(endpoint, metric9, step, x, counterType, tags)
324 | ls.add(uvTotalJson)
325 | //发送给open-falcon agent
326 | Sender.sender(ls, url)
327 |
328 | // 保存数据库
329 | val conn = ConnectionPool.getConnectionPool(propC3p0BroadCast.value).getConnection
330 | conn.setAutoCommit(false)
331 | val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm");
332 | val currentTimestamp = sdf.format(new Date())
333 |
334 | try {
335 | val sql = "insert into uv_day(time,uv) values (?,?)"
336 | val preparedStatement = conn.prepareStatement(sql)
337 | preparedStatement.setString(1, currentTimestamp)
338 | preparedStatement.setLong(2, x.toLong)
339 | preparedStatement.addBatch()
340 | preparedStatement.executeBatch()
341 | conn.commit()
342 | preparedStatement.clearBatch()
343 | } catch {
344 | case e:SQLException => conn.rollback()
345 | case e:Exception => e.printStackTrace()
346 | } finally {
347 | conn.close()
348 | }
349 | }
350 | }
351 | )
352 |
353 |
354 | val ipRecords = filterMessages.map(x => (x(0),1)).reduceByKey(_+_)
355 | ipRecords.persist(MEMORY_AND_DISK)
356 |
357 |
358 | /**
359 | * 异常访问IP次数和记录 记录详情保存数据库,异常次数输出给open-falcon做前端展示
360 | */
361 | ipRecords.filter(_._2 > abnormalVisitThreshold).foreachRDD( rdd => {
362 | //异常次数输出给open-falcon做前端展示
363 | val abnormalIPCounts = rdd.count()
364 | val ls = new ArrayList[Any]()
365 | val abnormalIPCountsJson = Pack.pack(endpoint, metric10, step, abnormalIPCounts, counterType,tags)
366 | ls.add(abnormalIPCountsJson)
367 | Sender.sender(ls, url)
368 |
369 | //ip详单保存到数据库
370 | rdd.foreachPartition{
371 | data => {
372 | val conn = ConnectionPool.getConnectionPool(propC3p0BroadCast.value).getConnection
373 | conn.setAutoCommit(false)
374 |
375 | val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm")
376 | val currentTimestamp = sdf.format(new Date())
377 |
378 | try {
379 | val sql = "insert into abnormal_ip(time,ip,frequency) values (?,?,?)"
380 | val preparedStatement = conn.prepareStatement(sql)
381 | data.foreach(r => {
382 | preparedStatement.setString(1, currentTimestamp)
383 | preparedStatement.setString(2, r._1.toString)
384 | preparedStatement.setInt(3, r._2)
385 | preparedStatement.addBatch()
386 | })
387 |
388 | preparedStatement.executeBatch()
389 | conn.commit()
390 | preparedStatement.clearBatch()
391 | } catch {
392 | case e:Exception => {
393 | case e:SQLException => conn.rollback()
394 | case e:Exception => e.printStackTrace()
395 | }
396 | } finally {
397 | conn.close()
398 | }
399 | }
400 | }
401 | })
402 |
403 | /**
404 | * 各省用户数(笛卡尔乘积数据库大的时候比较耗cpu,暂时没想到好方法)
405 | * 记录保存到数据库
406 | */
407 | if (aggregateProvinceFlag) {
408 | ipRecords.map(x => (IpToLong.IPv4ToLong(x._1.trim),x._2))
409 | .map(x => (LocationInfo.findLocation(ipMapBroadCast.value,x._1),x._2))
410 | .reduceByKey(_+_) foreachRDD( rdd => {
411 | rdd.foreachPartition { data =>
412 | if (data != null) {
413 | val conn = ConnectionPool.getConnectionPool(propC3p0BroadCast.value).getConnection
414 | conn.setAutoCommit(false)
415 |
416 | val sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm")
417 | val currentTimestamp = sdf.format(new Date())
418 |
419 | try {
420 | val sql = "insert into uv_province(time,province,uv) values (?,?,?)"
421 | val preparedStatement = conn.prepareStatement(sql)
422 | data.foreach(r => {
423 | preparedStatement.setString(1, currentTimestamp)
424 | preparedStatement.setString(2, r._1.toString)
425 | preparedStatement.setInt(3, r._2)
426 | preparedStatement.addBatch()
427 | })
428 |
429 | preparedStatement.executeBatch()
430 | conn.commit()
431 | preparedStatement.clearBatch()
432 | } catch {
433 | case e:SQLException => conn.rollback()
434 | case e:Exception => e.printStackTrace()
435 | } finally {
436 | conn.close()
437 | }
438 | }
439 | }
440 | }
441 | )
442 | }
443 | ProcessedOffsetManager.persists(partitonOffset_stream, props)
444 | logger.info("persist current offset in zookeeper cluster")
445 |
446 | ssc
447 | }
448 | //broadcast value无法从checkpoint恢复,只能自己序列化保存updateStateByKey的状态,用于重启恢复
449 | //val ssc = StreamingContext.getOrCreate(checkpointDirectory,() => createContext(logProperties,configProperties,dbProperties))
450 | val ssc = createContext(logProperties,configProperties,dbProperties)
451 |
452 | ssc.start()
453 | ssc.awaitTermination()
454 | }
455 | }
456 |
457 |
458 |
459 |
460 |
461 |
--------------------------------------------------------------------------------
/src/main/scala/com/hb/falcon/NginxFalcon.scala:
--------------------------------------------------------------------------------
1 | package com.hb.falcon
2 |
3 | /**
4 | * Created by Simon on 2017/2/24.
5 | */
6 | case class NginxFalcon(endpoint: String,
7 | metric: String,
8 | timestamp: Int,
9 | step: Int,
10 | value: Double,
11 | counterType: String ,
12 | tags: String )
13 |
14 |
--------------------------------------------------------------------------------
/src/main/scala/com/hb/falcon/Pack.scala:
--------------------------------------------------------------------------------
1 | package com.hb.falcon
2 |
3 |
4 | import com.google.gson.Gson
5 |
6 | /**
7 | * Created by Simon on 2017/3/27.
8 | */
9 | object Pack {
10 | /**
11 | * case class转换成json格式字符串
12 | * @param endpoint 终端名称
13 | * @param metric 指标名称
14 | * @param step 时间间隔
15 | * @param value 指标值
16 | * @param counterType Counter or Gauge
17 | * @param tags tags
18 | * @return json格式字符串
19 | */
20 | def pack(endpoint: String,
21 | metric: String,
22 | step: Int,
23 | value: Double,
24 | counterType: String ,
25 | tags: String) = {
26 | val timestamp = (System.currentTimeMillis()/1000).toInt
27 | val nginxFalcon = new NginxFalcon(endpoint, metric, timestamp , step, value, counterType, tags)
28 | new Gson().toJson(nginxFalcon)
29 | }
30 | }
31 |
--------------------------------------------------------------------------------
/src/main/scala/com/hb/falcon/Sender.scala:
--------------------------------------------------------------------------------
1 | package com.hb.falcon
2 |
3 | import java.io.IOException
4 | import java.util
5 | import javax.xml.ws.http.HTTPException
6 |
7 | import org.apache.http.client.methods.HttpPost
8 | import org.apache.http.entity.StringEntity
9 | import org.apache.http.impl.client.DefaultHttpClient
10 |
11 | /**
12 | * Created by Simon on 2017/3/1.
13 | */
14 |
15 | object Sender {
16 | /**
17 | * 数据发送到监控平台
18 | * @param dataList json字符串组成的ArrayList
19 | * @param url open-falcon agent接口
20 | */
21 | def sender (dataList:util.ArrayList[Any], url: String) = {
22 | val post = new HttpPost(url)
23 | post.setEntity(new StringEntity(dataList.toString))
24 | println("dataList is " + dataList.toString)
25 | post.setHeader("Content-Type","application/json")
26 | val client = new DefaultHttpClient
27 |
28 | try {
29 | client.execute(post)
30 | } catch {
31 | case e:HTTPException => e.printStackTrace()
32 | case e:IOException => e.printStackTrace()
33 | } finally {
34 | post.releaseConnection()
35 | }
36 | }
37 |
38 | }
39 |
40 |
--------------------------------------------------------------------------------
/src/main/scala/com/hb/model/IPMapping.scala:
--------------------------------------------------------------------------------
1 | package com.hb.model
2 |
3 | import java.io.{BufferedReader, IOException, InputStreamReader}
4 |
5 | import org.apache.commons.lang.StringUtils
6 |
7 | import scala.collection.mutable.ArrayBuffer
8 |
9 | /**
10 | * Created by Simon on 2017/3/30.
11 | */
12 | object IPMapping {
13 | /**
14 | * 生成IP段和城市的映射关系
15 | * @param fileName ip2city 文件,每行对应一个IPRecord
16 | * @return IPrecord列表
17 | */
18 | def getIpMapping(fileName:String) = {
19 | val IPArray = new ArrayBuffer[IPRecord]()
20 | val inputStream = IPMapping.getClass.getResourceAsStream(fileName)
21 | val bufferReader = new BufferedReader(new InputStreamReader(inputStream,"UTF-8"))
22 | var line : String = null
23 | try {
24 | line = bufferReader.readLine()
25 | } catch {
26 | case e: IOException => e.printStackTrace()
27 | }
28 | while (line != null) {
29 | line = StringUtils.trimToEmpty(line)
30 | if (!StringUtils.isEmpty(line)) {
31 | val record = new IPRecord()
32 | if (record.updateIPRecord(line)) {
33 | IPArray += record
34 | }
35 |
36 | try {
37 | line = bufferReader.readLine()
38 | } catch {
39 | case e:IOException => e.printStackTrace()
40 | }
41 | }
42 | }
43 |
44 | try {
45 | bufferReader.close()
46 | } catch {
47 | case e:IOException => e.printStackTrace()
48 | }
49 | IPArray
50 | }
51 | }
52 |
--------------------------------------------------------------------------------
/src/main/scala/com/hb/model/IPRecord.scala:
--------------------------------------------------------------------------------
1 | package com.hb.model
2 |
3 | /**
4 | * Created by Simon on 2017/3/30.
5 | */
6 | class IPRecord extends java.io.Serializable{
7 | private val IP_SEP = " "
8 | private val China = "中国"
9 | private var ipBegin : Long = 0
10 | private var ipEnd : Long = 0
11 | private var country : String = ""
12 | private var locProc : String = ""
13 | private var isForeign : Boolean = false
14 |
15 | /**
16 | * 初始化IP段和归属城市的对应关系
17 | * @param lineText Ip Record
18 | * @return Boolean
19 | */
20 | def updateIPRecord(lineText:String) = {
21 |
22 | val lineSplitArray = lineText.split(IP_SEP)
23 | if (lineSplitArray.length <= 4) {
24 | false
25 | } else {
26 | this.ipBegin = lineSplitArray(0).toLong
27 | this.ipEnd = lineSplitArray(1).toLong
28 | val tempCountry = lineSplitArray(2)
29 | this.country = if (tempCountry.equals("*")) "unknown_country" else tempCountry
30 | val tempProc = lineSplitArray(4)
31 | val proc = if (tempProc.equals("*")) "unknown_proc" else tempProc
32 |
33 | if (country.equals("中国")) {
34 | this.locProc = proc
35 | } else {
36 | this.locProc = country
37 | this.isForeign = true
38 | }
39 | true
40 | }
41 | }
42 |
43 | /**
44 | * 判断Ip是否属于该IPRecord IP段
45 | * @param ip 整数型Ip
46 | * @return Boolean
47 | */
48 | def isIPMatch(ip:Long) = {
49 | if (ip >= ipBegin && ip <= ipEnd ) true else false
50 | }
51 |
52 | override def toString: String = {
53 | ipBegin + " " + ipEnd + " " + locProc + " " + isForeign
54 | }
55 |
56 | def getLocation = {
57 | locProc
58 | }
59 |
60 | }
61 |
--------------------------------------------------------------------------------
/src/main/scala/com/hb/model/IpToLong.scala:
--------------------------------------------------------------------------------
1 | package com.hb.model
2 |
3 | import java.net.InetAddress
4 |
5 | /**
6 | * Created by Simon on 2017/4/1.
7 | */
8 | object IpToLong {
9 | /**
10 | * @param ipAddress ip 字符串
11 | * @return ip long
12 | */
13 | def IPv4ToLong(ipAddress: String): Long = {
14 | val addrArray: Array[String] = ipAddress.split("\\.")
15 | var num: Long = 0
16 | var i: Int = 0
17 | while (i < addrArray.length) {
18 | val power: Int = 3 - i
19 | num = num + ((addrArray(i).toInt % 256) * Math.pow(256, power)).toLong
20 | i += 1
21 | }
22 | num
23 | }
24 |
25 | /**
26 | * @param ip 整数ip地址
27 | * @return ip 点分十进制ip地址
28 | */
29 |
30 | def LongToIPv4 (ip : Long) : String = {
31 | val bytes: Array[Byte] = new Array[Byte](4)
32 | bytes(0) = ((ip & 0xff000000) >> 24).toByte
33 | bytes(1) = ((ip & 0x00ff0000) >> 16).toByte
34 | bytes(2) = ((ip & 0x0000ff00) >> 8).toByte
35 | bytes(3) = (ip & 0x000000ff).toByte
36 | InetAddress.getByAddress(bytes).getHostAddress()
37 | }
38 |
39 | }
40 |
--------------------------------------------------------------------------------
/src/main/scala/com/hb/model/LocationInfo.scala:
--------------------------------------------------------------------------------
1 | package com.hb.model
2 |
3 | import scala.collection.mutable.ArrayBuffer
4 |
5 | /**
6 | * Created by Simon on 2017/4/1.
7 | */
8 | object LocationInfo {
9 | /**
10 | * 通过IP获取所在省市
11 | * @param arrBuffer IPRecord数组
12 | * @param ipInt 点分十进制IP
13 | * @return
14 | */
15 | def findLocation(arrBuffer: ArrayBuffer[IPRecord],ipInt : Long) = {
16 | var loc : String = null
17 | var foundLocation : Boolean = false
18 | for (elems <- arrBuffer if !foundLocation ){
19 | if (elems.isIPMatch(ipInt)){
20 | loc = elems.getLocation
21 | foundLocation = true
22 | }
23 | }
24 | loc
25 | }
26 | }
27 |
--------------------------------------------------------------------------------
/src/main/scala/com/hb/pool/ConnectionPool.scala:
--------------------------------------------------------------------------------
1 | package com.hb.pool
2 |
3 | import java.util.Properties
4 |
5 | import com.mchange.v2.c3p0.ComboPooledDataSource
6 |
7 | /**
8 | * Created by Simon on 2017/4/5.
9 | */
10 | class ConnectionPool private(prop : Properties) extends Serializable{
11 | private val cpds:ComboPooledDataSource = new ComboPooledDataSource(true)
12 | try {
13 | cpds.setJdbcUrl(prop.getProperty("jdbcUrl"))
14 | cpds.setDriverClass(prop.getProperty("driverClass"))
15 | cpds.setUser(prop.getProperty("user"))
16 | cpds.setPassword(prop.getProperty("password"))
17 | cpds.setMaxPoolSize(prop.getProperty("maxPoolSize").toInt)
18 | cpds.setMinPoolSize(prop.getProperty("minPoolSize").toInt)
19 | cpds.setAcquireIncrement(prop.getProperty("acquireIncrement").toInt)
20 | cpds.setInitialPoolSize(prop.getProperty("initialPoolSize").toInt)
21 | cpds.setMaxIdleTime(prop.getProperty("maxIdleTime").toInt)
22 | } catch {
23 | case ex:Exception => ex.printStackTrace()
24 | }
25 |
26 | /**
27 | *从c3p0连接池里面获取一个连接
28 | */
29 | def getConnection = {
30 | try {
31 | cpds.getConnection
32 | } catch {
33 | case ex : Exception => ex.printStackTrace
34 | null
35 | }
36 | }
37 | }
38 |
39 | object ConnectionPool {
40 | var connectionPool : ConnectionPool = null
41 |
42 | /**
43 | * 单例模式获取连接池对象
44 | * @return ConnectionPool 连接池对象
45 | */
46 | def getConnectionPool (prop : Properties) : ConnectionPool = {
47 | synchronized{
48 | if (connectionPool == null) {
49 | connectionPool = new ConnectionPool(prop)
50 | }
51 | }
52 | connectionPool
53 | }
54 |
55 | }
56 |
--------------------------------------------------------------------------------
/src/main/scala/com/hb/utils/Num.scala:
--------------------------------------------------------------------------------
1 | package com.hb.utils
2 |
3 |
4 | /**
5 | * Created by Simon on 2017/2/27.
6 | */
7 |
8 | object Num {
9 |
10 | /**
11 | * @param arr 输入数组
12 | * @return 快速排序后的数组
13 | */
14 | def quickSort(arr: Array[Double]): Array[Double] = {
15 | if (arr.length <= 1)
16 | arr
17 | else {
18 | val index = arr(arr.length / 2)
19 | Array.concat(
20 | quickSort(arr filter (index >)),
21 | arr filter (_ == index),
22 | quickSort(arr filter (index <))
23 | )
24 | }
25 | }
26 |
27 | /**
28 | * @param arr 输入数组
29 | * @return p 百分位
30 | */
31 | def percentile(arr: Array[Double], p: Double) = {
32 | if (p > 1 || p < 0) throw new IllegalArgumentException("p must be in [0,1]")
33 | val sorted = quickSort(arr)
34 | val f = (sorted.length + 1) * p
35 | val i = f.toInt
36 | if (i == 0) sorted.head
37 | else if (i >= sorted.length) sorted.last
38 | else {
39 | sorted(i - 1) + (f - i) * (sorted(i) - sorted(i - 1))
40 | }
41 | }
42 | }
43 |
44 |
--------------------------------------------------------------------------------
/src/main/scala/consumer/kafka/Broker.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | /*
20 | * This file is based on the source code of the Kafka spout of the Apache Storm project.
21 | * (https://github.com/apache/storm/tree/master/external/storm-kafka)
22 | * This file has been modified to work with Spark Streaming.
23 | */
24 |
25 | package consumer.kafka;
26 |
27 | import com.google.common.base.Objects;
28 |
29 | import java.io.Serializable;
30 |
31 | @SuppressWarnings("serial")
32 | public class Broker implements Serializable, Comparable {
33 |
34 | public final String host;
35 | public final int port;
36 |
37 | public Broker(String host, int port) {
38 | this.host = host;
39 | this.port = port;
40 | }
41 |
42 | public Broker(String host) {
43 | this(host, 9092);
44 | }
45 |
46 | @Override
47 | public int hashCode() {
48 | return Objects.hashCode(host, port);
49 | }
50 |
51 | @Override
52 | public boolean equals(Object obj) {
53 | if (this == obj) {
54 | return true;
55 | }
56 | if (obj == null || getClass() != obj.getClass()) {
57 | return false;
58 | }
59 | final Broker other = (Broker) obj;
60 | return Objects.equal(this.host, other.host)
61 | && Objects.equal(this.port, other.port);
62 | }
63 |
64 | @Override
65 | public String toString() {
66 | return host + ":" + port;
67 | }
68 |
69 | public static Broker fromString(String host) {
70 | Broker hp;
71 | String[] spec = host.split(":");
72 | if (spec.length == 1) {
73 | hp = new Broker(spec[0]);
74 | } else if (spec.length == 2) {
75 | hp = new Broker(spec[0], Integer.parseInt(spec[1]));
76 | } else {
77 | throw new IllegalArgumentException("Invalid host specification: " + host);
78 | }
79 | return hp;
80 | }
81 |
82 | public int compareTo(Broker o) {
83 | if (this.host.equals(o.host)) {
84 | return this.port - o.port;
85 | } else {
86 | return this.host.compareTo(o.host);
87 | }
88 | }
89 | }
90 |
--------------------------------------------------------------------------------
/src/main/scala/consumer/kafka/BrokerHosts.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | /*
20 | * This file is based on the source code of the Kafka spout of the Apache Storm project.
21 | * (https://github.com/apache/storm/tree/master/external/storm-kafka)
22 | * This file has been modified to work with Spark Streaming.
23 | */
24 |
25 | package consumer.kafka;
26 |
27 | import java.io.Serializable;
28 |
29 | public interface BrokerHosts extends Serializable {
30 |
31 | }
--------------------------------------------------------------------------------
/src/main/scala/consumer/kafka/Config.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | /*
20 | * This file is based on the source code of the Kafka spout of the Apache Storm project.
21 | * (https://github.com/apache/storm/tree/master/external/storm-kafka)
22 | * This file has been modified to work with Spark Streaming.
23 | */
24 |
25 | package consumer.kafka;
26 |
27 | import java.io.Serializable;
28 | import java.util.HashMap;
29 |
30 | @SuppressWarnings("serial")
31 | public class Config extends HashMap implements Serializable {
32 |
33 | /**
34 | * Kafka related configurations
35 | */
36 | public static final String ZOOKEEPER_HOSTS =
37 | "zookeeper.hosts";
38 | public static final String ZOOKEEPER_PORT =
39 | "zookeeper.port";
40 | public static final String KAFKA_TOPIC =
41 | "kafka.topic";
42 | public static final String ZOOKEEPER_BROKER_PATH =
43 | "zookeeper.broker.path";
44 |
45 | /**
46 | * Consumer related configurations
47 | */
48 | public static final String ZOOKEEPER_CONSUMER_PATH =
49 | "zookeeper.consumer.path";
50 | public static final String ZOOKEEPER_CONSUMER_CONNECTION =
51 | "zookeeper.consumer.connection";
52 | public static final String KAFKA_CONSUMER_ID = "kafka.consumer.id";
53 |
54 | /**
55 | * Optional Configurations
56 | */
57 | public static final String KAFKA_PARTITIONS_NUMBER =
58 | "kafka.partitions.number";
59 | public static final String CONSUMER_FORCE_FROM_START =
60 | "consumer.forcefromstart";
61 | public static final String CONSUMER_FETCH_SIZE_BYTES =
62 | "consumer.fetchsizebytes";
63 | public static final String CONSUMER_FILL_FREQ_MS =
64 | "consumer.fillfreqms";
65 | public static final String CONSUMER_STOP_GRACEFULLY =
66 | "consumer.stopgracefully";
67 |
68 | /**
69 | * Configuration Related to Back Pressure
70 | */
71 |
72 | public static final String CONSUMER_BACKPRESSURE_ENABLED =
73 | "consumer.backpressure.enabled";
74 | public static final String CONSUMER_BACKPRESSURE_PROPORTIONAL =
75 | "consumer.backpressure.proportional";
76 | public static final String CONSUMER_BACKPRESSURE_INTEGRAL =
77 | "consumer.backpressure.integral";
78 | public static final String CONSUMER_BACKPRESSURE_DERIVATIVE =
79 | "consumer.backpressure.derivative";
80 |
81 | public static final String KAFKA_RECEIVER_NUMBER =
82 | "kafka.receiver.number";
83 | public static final String SPARK_RESOURCE_MANAGER_CLASS =
84 | "spark.resource.manager.class";
85 |
86 | }
87 |
--------------------------------------------------------------------------------
/src/main/scala/consumer/kafka/DynamicBrokersReader.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Licensed to the Apache Software Foundation (ASF) under one
3 | * or more contributor license agreements. See the NOTICE file
4 | * distributed with this work for additional information
5 | * regarding copyright ownership. The ASF licenses this file
6 | * to you under the Apache License, Version 2.0 (the
7 | * "License"); you may not use this file except in compliance
8 | * with the License. You may obtain a copy of the License at
9 | *
10 | * http://www.apache.org/licenses/LICENSE-2.0
11 | *
12 | * Unless required by applicable law or agreed to in writing, software
13 | * distributed under the License is distributed on an "AS IS" BASIS,
14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 | * See the License for the specific language governing permissions and
16 | * limitations under the License.
17 | */
18 |
19 | /*
20 | * This file is based on the source code of the Kafka spout of the Apache Storm project.
21 | * (https://github.com/apache/storm/tree/master/external/storm-kafka)
22 | * This file has been modified to work with Spark Streaming.
23 | */
24 |
25 | package consumer.kafka;
26 |
27 | import org.apache.curator.framework.CuratorFramework;
28 | import org.json.simple.JSONValue;
29 | import org.slf4j.Logger;
30 | import org.slf4j.LoggerFactory;
31 |
32 | import java.io.Serializable;
33 | import java.io.UnsupportedEncodingException;
34 | import java.util.List;
35 | import java.util.Map;
36 |
37 | @SuppressWarnings("serial")
38 | public class DynamicBrokersReader implements Serializable {
39 |
40 | public static final Logger LOG = LoggerFactory
41 | .getLogger(DynamicBrokersReader.class);
42 |
43 | transient private CuratorFramework _curator;
44 | private String _zkPath;
45 | private String _topic;
46 |
47 | public DynamicBrokersReader(KafkaConfig config, ZkState zkState) {
48 | _zkPath = (String) config._stateConf.get(Config.ZOOKEEPER_BROKER_PATH);
49 | _topic = (String) config._stateConf.get(Config.KAFKA_TOPIC);
50 | _curator = zkState.getCurator();
51 | }
52 |
53 | /**
54 | * Get all partitions with their current leaders
55 | */
56 | public GlobalPartitionInformation getBrokerInfo() {
57 | GlobalPartitionInformation globalPartitionInformation =
58 | new GlobalPartitionInformation();
59 | try {
60 | int numPartitionsForTopic = getNumPartitions();
61 | String brokerInfoPath = brokerPath();
62 | for (int partition = 0; partition < numPartitionsForTopic; partition++) {
63 | int leader = getLeaderFor(partition);
64 | String path = brokerInfoPath + "/" + leader;
65 | try {
66 | byte[] brokerData = _curator.getData().forPath(path);
67 | Broker hp = getBrokerHost(brokerData);
68 | globalPartitionInformation.addPartition(partition, hp);
69 | } catch (org.apache.zookeeper.KeeperException.NoNodeException e) {
70 | LOG.error("Node {} does not exist ", path);
71 | }
72 | }
73 | } catch (Exception e) {
74 | throw new RuntimeException(e);
75 | }
76 | LOG.debug("Read partition info from zookeeper: {}",globalPartitionInformation );
77 | return globalPartitionInformation;
78 | }
79 |
80 | public int getNumPartitions() {
81 | try {
82 | String topicBrokersPath = partitionPath();
83 | List children = _curator.getChildren().forPath(topicBrokersPath);
84 | return children.size();
85 | } catch (Exception e) {
86 | throw new RuntimeException(e);
87 | }
88 | }
89 |
90 | public String partitionPath() {
91 | return _zkPath + "/topics/" + _topic + "/partitions";
92 | }
93 |
94 | public String brokerPath() {
95 | return _zkPath + "/ids";
96 | }
97 |
98 | /**
99 | * get /brokers/topics/distributedTopic/partitions/1/state {
100 | * "controller_epoch":4, "isr":[ 1, 0 ], "leader":1, "leader_epoch":1,
101 | * "version":1 }
102 | *
103 | * @param partition
104 | * @return
105 | */
106 | @SuppressWarnings("unchecked")
107 | private int getLeaderFor(long partition) {
108 | try {
109 | String topicBrokersPath = partitionPath();
110 | byte[] hostPortData =
111 | _curator.getData().forPath(
112 | topicBrokersPath + "/" + partition + "/state");
113 | Map