├── 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 value = 114 | (Map) JSONValue.parse(new String( 115 | hostPortData, 116 | "UTF-8")); 117 | Integer leader = ((Number) value.get("leader")).intValue(); 118 | return leader; 119 | } catch (Exception e) { 120 | throw new RuntimeException(e); 121 | } 122 | } 123 | 124 | public void close() { 125 | _curator.close(); 126 | } 127 | 128 | /** 129 | * [zk: localhost:2181(CONNECTED) 56] get /brokers/ids/0 { "host":"localhost", 130 | * "jmx_port":9999, "port":9092, "version":1 } 131 | * 132 | * @param contents 133 | * @return 134 | */ 135 | @SuppressWarnings("unchecked") 136 | private Broker getBrokerHost(byte[] contents) { 137 | try { 138 | Map value = 139 | (Map) JSONValue.parse(new String(contents, "UTF-8")); 140 | String host = (String) value.get("host"); 141 | Integer port = ((Long) value.get("port")).intValue(); 142 | return new Broker(host, port); 143 | } catch (UnsupportedEncodingException e) { 144 | throw new RuntimeException(e); 145 | } 146 | } 147 | 148 | } 149 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/DynamicPartitionConnections.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 kafka.javaapi.consumer.SimpleConsumer; 28 | import org.slf4j.Logger; 29 | import org.slf4j.LoggerFactory; 30 | 31 | import java.io.Serializable; 32 | import java.util.HashMap; 33 | import java.util.HashSet; 34 | import java.util.Map; 35 | import java.util.Set; 36 | 37 | @SuppressWarnings("serial") 38 | public class DynamicPartitionConnections implements Serializable { 39 | 40 | public static final Logger LOG = LoggerFactory 41 | .getLogger(DynamicPartitionConnections.class); 42 | 43 | class ConnectionInfo implements Serializable { 44 | 45 | SimpleConsumer consumer; 46 | Set partitions = new HashSet(); 47 | 48 | public ConnectionInfo(SimpleConsumer consumer) { 49 | this.consumer = consumer; 50 | } 51 | } 52 | 53 | Map _connections = new HashMap(); 54 | KafkaConfig _config; 55 | IBrokerReader _reader; 56 | 57 | public DynamicPartitionConnections( 58 | KafkaConfig config, 59 | IBrokerReader brokerReader) { 60 | _config = config; 61 | _reader = brokerReader; 62 | } 63 | 64 | public SimpleConsumer register(Partition partition) { 65 | Broker broker = _reader.getCurrentBrokers().getBrokerFor(partition.partition); 66 | return register(broker, partition.partition); 67 | } 68 | 69 | public SimpleConsumer register(Broker host, int partition) { 70 | if (!_connections.containsKey(host)) { 71 | _connections.put(host, new ConnectionInfo( 72 | new SimpleConsumer( 73 | host.host, 74 | host.port, 75 | _config._socketTimeoutMs, 76 | _config._bufferSizeBytes, 77 | (String) _config._stateConf.get(Config.KAFKA_CONSUMER_ID)) 78 | )); 79 | } 80 | ConnectionInfo info = _connections.get(host); 81 | info.partitions.add(partition); 82 | return info.consumer; 83 | } 84 | 85 | public SimpleConsumer getConnection(Partition partition) { 86 | ConnectionInfo info = _connections.get(partition.host); 87 | if (info != null) { 88 | return info.consumer; 89 | } 90 | return null; 91 | } 92 | 93 | public void unregister(Broker port, int partition) { 94 | ConnectionInfo info = _connections.get(port); 95 | info.partitions.remove(partition); 96 | if (info.partitions.isEmpty()) { 97 | info.consumer.close(); 98 | _connections.remove(port); 99 | } 100 | } 101 | 102 | public void unregister(Partition partition) { 103 | unregister(partition.host, partition.partition); 104 | } 105 | 106 | public void clear() { 107 | for (ConnectionInfo info : _connections.values()) { 108 | info.consumer.close(); 109 | } 110 | _connections.clear(); 111 | } 112 | } 113 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/FailedFetchException.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 | @SuppressWarnings("serial") 28 | public class FailedFetchException extends RuntimeException { 29 | 30 | public FailedFetchException(String message) { 31 | super(message); 32 | } 33 | 34 | public FailedFetchException(Exception e) { 35 | super(e); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/GlobalPartitionInformation.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 | import java.util.*; 31 | 32 | @SuppressWarnings("serial") 33 | public class GlobalPartitionInformation implements Iterable,Serializable { 34 | 35 | private Map partitionMap; 36 | 37 | public GlobalPartitionInformation() { 38 | partitionMap = new TreeMap(); 39 | } 40 | 41 | public void addPartition(int partitionId, Broker broker) { 42 | partitionMap.put(partitionId, broker); 43 | } 44 | 45 | @Override 46 | public String toString() { 47 | return "GlobalPartitionInformation{" + "partitionMap=" + partitionMap + '}'; 48 | } 49 | 50 | public Broker getBrokerFor(Integer partitionId) { 51 | return partitionMap.get(partitionId); 52 | } 53 | 54 | public List getOrderedPartitions() { 55 | List partitions = new LinkedList(); 56 | for (Map.Entry partition : partitionMap.entrySet()) { 57 | partitions.add(new Partition(partition.getValue(), partition.getKey())); 58 | } 59 | return partitions; 60 | } 61 | 62 | public Iterator iterator() { 63 | final Iterator> iterator = partitionMap.entrySet().iterator(); 64 | return new Iterator() { 65 | 66 | public boolean hasNext() { 67 | return iterator.hasNext(); 68 | } 69 | 70 | public Partition next() { 71 | Map.Entry next = iterator.next(); 72 | return new Partition(next.getValue(), next.getKey()); 73 | } 74 | 75 | public void remove() { 76 | iterator.remove(); 77 | } 78 | }; 79 | } 80 | 81 | @Override 82 | public int hashCode() { 83 | return Objects.hashCode(partitionMap); 84 | } 85 | 86 | @Override 87 | public boolean equals(Object obj) { 88 | if (this == obj) { 89 | return true; 90 | } 91 | if (obj == null || getClass() != obj.getClass()) { 92 | return false; 93 | } 94 | final GlobalPartitionInformation other = (GlobalPartitionInformation) obj; 95 | return Objects.equal(this.partitionMap, other.partitionMap); 96 | } 97 | } 98 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/IBrokerReader.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 | public interface IBrokerReader { 28 | public GlobalPartitionInformation getCurrentBrokers(); 29 | public void close(); 30 | } 31 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/IdentityMessageHandler.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 | package consumer.kafka; 20 | 21 | @SuppressWarnings("serial") 22 | public class IdentityMessageHandler extends KafkaMessageHandler { 23 | 24 | @Override 25 | protected byte[] process(byte[] payload) { 26 | return payload; 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/KafkaConfig.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 | package consumer.kafka; 20 | 21 | import java.io.Serializable; 22 | import java.util.HashMap; 23 | import java.util.Map; 24 | import java.util.Properties; 25 | 26 | @SuppressWarnings("serial") 27 | public class KafkaConfig implements Serializable { 28 | 29 | //Default fetch size . 1 MB 30 | public int _fetchSizeBytes = 1048576; 31 | //Default fill frequency 1 Seconds 32 | public int _fillFreqMs = 1000; 33 | //Default minimum fetch size 512 KB 34 | public int _minFetchSizeBytes = 524288; 35 | //Max allowable fetch size 36 | public int _maxFetchSizeBytes = _fetchSizeBytes * 2; 37 | public int _bufferSizeBytes = 1048576; 38 | 39 | //Automatic refresh of ZK Coordinator to check for Leader Re-balance 40 | public int _refreshFreqSecs = 300; 41 | public int _socketTimeoutMs = 10000; 42 | //If set to true, it will start from Earliest Offset. 43 | //Note this is only for first time start of the consumer. 44 | //During next successive restart it will either consumes 45 | //from Consumed or Processed offset whichever is applicable 46 | public boolean _forceFromStart = false; 47 | //PID Controller based back-pressure mechanism to rate control 48 | public boolean _backpressureEnabled = true; 49 | public int _maxRestartAttempts = -1; 50 | public long _startOffsetTime = kafka.api.OffsetRequest.EarliestTime(); 51 | public long _stateUpdateIntervalMs = 2000; 52 | public Map _stateConf; 53 | //Number of fetch consumer will buffer before writing to Spark Block Manager 54 | public int _numFetchToBuffer = 1; 55 | //Consumer will throttle to Zero rate if Queued batches reach this value 56 | //This is to avoid memory pressure 57 | public int _batchQueueToThrottle = 1; 58 | 59 | //Default PID values for Controller 60 | public double _proportional = 1.0; 61 | public double _integral = 0.0; 62 | public double _derivative = 0; 63 | 64 | //Parameters for Controllers 65 | 66 | //percent of Batch duration taking into rate calculation. 100 % default 67 | public double _safeBatchPercent = 1.0; 68 | //Max allowable rate change possible. 20 % default 69 | public double _maxRateChangePercent = 0.2; 70 | 71 | public String brokerZkPath = "/brokers"; 72 | public String consumerZkPath = "/consumers"; 73 | 74 | public KafkaConfig(Properties props) { 75 | 76 | //ZK Host and Port for Kafka Cluster 77 | String zkHost = props.getProperty("zookeeper.hosts"); 78 | String zkPort = props.getProperty("zookeeper.port"); 79 | //Kafka Topic 80 | String kafkaTopic = props.getProperty("kafka.topic"); 81 | //ZK host:port details for Offset writing 82 | String consumerConnection = props.getProperty("zookeeper.consumer.connection"); 83 | String consumerId = props.getProperty("kafka.consumer.id"); 84 | 85 | if (props.getProperty("zookeeper.broker.path") != null) { 86 | brokerZkPath = props.getProperty("zookeeper.broker.path"); 87 | } 88 | 89 | if (props.getProperty("zookeeper.consumer.path") != null) { 90 | consumerZkPath = props.getProperty("zookeeper.consumer.path"); 91 | } 92 | 93 | if (props.getProperty("consumer.forcefromstart") != null) { 94 | _forceFromStart = Boolean.parseBoolean(props.getProperty("consumer.forcefromstart")); 95 | } 96 | 97 | if (props.getProperty("consumer.num_fetch_to_buffer") != null) { 98 | _numFetchToBuffer = Integer.parseInt(props.getProperty("consumer.num_fetch_to_buffer")); 99 | } 100 | 101 | if (props.getProperty("consumer.fetchsizebytes") != null) { 102 | _fetchSizeBytes = Integer.parseInt(props.getProperty("consumer.fetchsizebytes")); 103 | } 104 | 105 | if (props.getProperty("consumer.min.fetchsizebytes") != null) { 106 | _minFetchSizeBytes = Integer.parseInt(props.getProperty("consumer.min.fetchsizebytes")); 107 | } 108 | 109 | if (props.getProperty("consumer.max.fetchsizebytes") != null) { 110 | _maxFetchSizeBytes = Integer.parseInt(props.getProperty("consumer.max.fetchsizebytes")); 111 | } 112 | 113 | if (props.getProperty("consumer.fillfreqms") != null) { 114 | _fillFreqMs = Integer.parseInt(props.getProperty("consumer.fillfreqms")); 115 | } 116 | 117 | if (props.getProperty("consumer.refresh_freq_sec") != null){ 118 | _refreshFreqSecs = Integer.parseInt(props.getProperty("consumer.refresh_freq_sec")); 119 | } 120 | 121 | if (props.getProperty("consumer.backpressure.enabled") != null) { 122 | _backpressureEnabled = Boolean.parseBoolean(props.getProperty("consumer.backpressure.enabled")); 123 | } 124 | 125 | if (props.getProperty("consumer.backpressure.proportional") != null) { 126 | _proportional = Double.parseDouble(props.getProperty("consumer.backpressure.proportional")); 127 | } 128 | 129 | if (props.getProperty("consumer.backpressure.integral") != null) { 130 | _integral = Double.parseDouble(props.getProperty("consumer.backpressure.integral")); 131 | } 132 | 133 | if (props.getProperty("consumer.backpressure.derivative") != null) { 134 | _derivative = Double.parseDouble(props.getProperty("consumer.backpressure.derivative")); 135 | } 136 | 137 | if (props.getProperty("kafka.consumer.restart.attempt") != null) { 138 | _maxRestartAttempts = Integer.parseInt(props.getProperty("kafka.consumer.restart.attempt")); 139 | } 140 | 141 | if (props.getProperty("consumer.queue.to.throttle") != null){ 142 | _batchQueueToThrottle = Integer.parseInt(props.getProperty("consumer.queue.to.throttle")); 143 | } 144 | 145 | if (props.getProperty("consumer.safe.batch.percent") != null) { 146 | _safeBatchPercent = Double.parseDouble(props.getProperty("consumer.safe.batch.percent")); 147 | } 148 | 149 | if (props.getProperty("consumer.max.rate.change.percent") != null) { 150 | _maxRateChangePercent = Double.parseDouble(props.getProperty("consumer.max.rate.change.percent")); 151 | } 152 | 153 | _stateConf = new HashMap(); 154 | _stateConf.put(Config.ZOOKEEPER_HOSTS, zkHost); 155 | _stateConf.put(Config.ZOOKEEPER_PORT, zkPort); 156 | _stateConf.put(Config.KAFKA_TOPIC, kafkaTopic); 157 | _stateConf.put(Config.ZOOKEEPER_BROKER_PATH, brokerZkPath); 158 | 159 | _stateConf.put(Config.ZOOKEEPER_CONSUMER_PATH, consumerZkPath); 160 | _stateConf.put(Config.ZOOKEEPER_CONSUMER_CONNECTION, consumerConnection); 161 | _stateConf.put(Config.KAFKA_CONSUMER_ID, consumerId); 162 | } 163 | } 164 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/KafkaConsumer.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.spark.streaming.receiver.Receiver; 28 | import org.slf4j.Logger; 29 | import org.slf4j.LoggerFactory; 30 | 31 | import java.io.Serializable; 32 | import java.util.List; 33 | 34 | @SuppressWarnings("serial") 35 | public class KafkaConsumer implements Runnable, Serializable, AutoCloseable { 36 | 37 | private static final Logger LOG = LoggerFactory.getLogger(KafkaConsumer.class); 38 | 39 | private KafkaConfig _kafkaconfig; 40 | private PartitionCoordinator _coordinator; 41 | private DynamicPartitionConnections _connections; 42 | private ZkState _state; 43 | private long _lastConsumeTime = 0L; 44 | private int _currPartitionIndex = 0; 45 | private Receiver> _receiver; 46 | private KafkaMessageHandler _messageHandler; 47 | 48 | public KafkaConsumer( 49 | KafkaConfig config, 50 | ZkState zkState, 51 | Receiver> receiver, 52 | KafkaMessageHandler messageHandler) { 53 | _kafkaconfig = config; 54 | _state = zkState; 55 | _receiver = receiver; 56 | _messageHandler = messageHandler; 57 | } 58 | 59 | public void open(int partitionId) { 60 | _currPartitionIndex = partitionId; 61 | _connections = 62 | new DynamicPartitionConnections( 63 | _kafkaconfig, 64 | new ZkBrokerReader( 65 | _kafkaconfig, 66 | _state)); 67 | 68 | _coordinator = 69 | new ZkCoordinator( 70 | _connections, 71 | _kafkaconfig, 72 | _state, 73 | partitionId, 74 | _receiver, 75 | true, 76 | _messageHandler); 77 | } 78 | 79 | @Override 80 | public void close() { 81 | if (_state != null) { 82 | _state.close(); 83 | } 84 | if (_connections != null) { 85 | _connections.clear(); 86 | } 87 | } 88 | 89 | public void createStream() throws Exception { 90 | try { 91 | List managers = _coordinator.getMyManagedPartitions(); 92 | if (managers == null || managers.size() == 0) { 93 | LOG.warn("Some issue getting Partition details.. Refreshing Corodinator.."); 94 | _coordinator.refresh(); 95 | } else { 96 | managers.get(0).next(); 97 | } 98 | } catch (FailedFetchException fe) { 99 | fe.printStackTrace(); 100 | LOG.warn("Fetch failed. Refresing Coordinator..", fe); 101 | _coordinator.refresh(); 102 | } catch (Exception ex) { 103 | LOG.error("Partition " 104 | + _currPartitionIndex + " encountered error during createStream : " 105 | + ex.getMessage()); 106 | ex.printStackTrace(); 107 | throw ex; 108 | } 109 | 110 | } 111 | 112 | @Override 113 | public void run() { 114 | try { 115 | while (!_receiver.isStopped()) { 116 | long timeSinceLastPull = System.currentTimeMillis() - _lastConsumeTime; 117 | if (timeSinceLastPull >= _kafkaconfig._fillFreqMs) { 118 | _lastConsumeTime = System.currentTimeMillis(); 119 | this.createStream(); 120 | } else { 121 | long waitTime = _kafkaconfig._fillFreqMs - timeSinceLastPull; 122 | if (waitTime > 0) 123 | Thread.sleep(waitTime); 124 | } 125 | } 126 | } catch (Exception ex) { 127 | try { 128 | this.close(); 129 | throw ex; 130 | } catch (Exception e) { 131 | e.printStackTrace(); 132 | } 133 | } 134 | } 135 | } 136 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/KafkaError.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 | public enum KafkaError { 28 | NO_ERROR, 29 | OFFSET_OUT_OF_RANGE, 30 | INVALID_MESSAGE, 31 | UNKNOWN_TOPIC_OR_PARTITION, 32 | INVALID_FETCH_SIZE, 33 | LEADER_NOT_AVAILABLE, 34 | NOT_LEADER_FOR_PARTITION, 35 | REQUEST_TIMED_OUT, 36 | BROKER_NOT_AVAILABLE, 37 | REPLICA_NOT_AVAILABLE, 38 | MESSAGE_SIZE_TOO_LARGE, 39 | STALE_CONTROLLER_EPOCH, 40 | OFFSET_METADATA_TOO_LARGE, 41 | UNKNOWN; 42 | 43 | public static KafkaError getError(int errorCode) { 44 | if (errorCode < 0 || errorCode >= UNKNOWN.ordinal()) { 45 | return UNKNOWN; 46 | } else { 47 | return values()[errorCode]; 48 | } 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/KafkaMessageHandler.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 | package consumer.kafka; 20 | 21 | import java.io.Serializable; 22 | 23 | @SuppressWarnings("serial") 24 | public abstract class KafkaMessageHandler implements Cloneable, Serializable { 25 | 26 | public void init() { 27 | // do nothing for default implementation 28 | } 29 | 30 | public MessageAndMetadata handle(long offset, Partition partition, String topic, String consumer, byte[] payload) throws Exception { 31 | E msg = process(payload); 32 | MessageAndMetadata m = new MessageAndMetadata<>(); 33 | m.setConsumer(consumer); 34 | m.setOffset(offset); 35 | m.setPartition(partition); 36 | m.setPayload(msg); 37 | m.setTopic(topic); 38 | return m; 39 | } 40 | 41 | protected abstract E process(byte[] payload); 42 | 43 | @Override 44 | public Object clone() throws CloneNotSupportedException { 45 | return (KafkaMessageHandler) super.clone(); 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/KafkaUtils.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 kafka.api.FetchRequest; 28 | import kafka.api.FetchRequestBuilder; 29 | import kafka.api.PartitionOffsetRequestInfo; 30 | import kafka.common.TopicAndPartition; 31 | import kafka.javaapi.FetchResponse; 32 | import kafka.javaapi.OffsetRequest; 33 | import kafka.javaapi.consumer.SimpleConsumer; 34 | import org.slf4j.Logger; 35 | import org.slf4j.LoggerFactory; 36 | 37 | import java.io.IOException; 38 | import java.net.ConnectException; 39 | import java.net.SocketTimeoutException; 40 | import java.nio.channels.UnresolvedAddressException; 41 | import java.util.HashMap; 42 | import java.util.Map; 43 | 44 | public class KafkaUtils { 45 | 46 | public static final Logger LOG = LoggerFactory.getLogger(KafkaUtils.class); 47 | public static final int NO_OFFSET = -5; 48 | 49 | public static long getOffset( 50 | SimpleConsumer consumer, String topic, int partition, KafkaConfig config) { 51 | long startOffsetTime = kafka.api.OffsetRequest.LatestTime(); 52 | if (config._forceFromStart) { 53 | startOffsetTime = config._startOffsetTime; 54 | } 55 | return getOffset(consumer, topic, partition, startOffsetTime); 56 | } 57 | 58 | public static long getOffset( 59 | SimpleConsumer consumer, String topic, int partition, long startOffsetTime) { 60 | TopicAndPartition topicAndPartition = 61 | new TopicAndPartition(topic, partition); 62 | Map requestInfo = 63 | new HashMap(); 64 | requestInfo.put(topicAndPartition, new PartitionOffsetRequestInfo( 65 | startOffsetTime, 1)); 66 | OffsetRequest request = 67 | new OffsetRequest( 68 | requestInfo, kafka.api.OffsetRequest.CurrentVersion(), 69 | consumer.clientId()); 70 | 71 | long[] offsets = 72 | consumer.getOffsetsBefore(request).offsets(topic, partition); 73 | if (offsets.length > 0) { 74 | return offsets[0]; 75 | } else { 76 | return NO_OFFSET; 77 | } 78 | } 79 | 80 | public static FetchResponse fetchMessages( 81 | KafkaConfig config, SimpleConsumer consumer, Partition partition, 82 | long offset, int fetchSize) { 83 | String topic = (String) config._stateConf.get(Config.KAFKA_TOPIC); 84 | int partitionId = partition.partition; 85 | LOG.debug("Fetching from Kafka for partition {} for fetchSize {} and bufferSize {}", 86 | partition.partition, fetchSize, consumer.bufferSize()); 87 | FetchRequestBuilder builder = new FetchRequestBuilder(); 88 | FetchRequest fetchRequest = 89 | builder.addFetch(topic, partitionId, offset, fetchSize) 90 | .clientId((String) config._stateConf.get(Config.KAFKA_CONSUMER_ID)) 91 | .build(); 92 | FetchResponse fetchResponse; 93 | try { 94 | fetchResponse = consumer.fetch(fetchRequest); 95 | } catch (Exception e) { 96 | if (e instanceof ConnectException 97 | || e instanceof SocketTimeoutException || e instanceof IOException 98 | || e instanceof UnresolvedAddressException) { 99 | 100 | LOG.warn("Network error when fetching messages:", e); 101 | throw new FailedFetchException(e); 102 | } else { 103 | throw new RuntimeException(e); 104 | } 105 | } 106 | 107 | return fetchResponse; 108 | } 109 | } 110 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/MessageAndMetadata.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 | package consumer.kafka; 20 | 21 | import java.io.Serializable; 22 | 23 | @SuppressWarnings("serial") 24 | public class MessageAndMetadata implements Serializable { 25 | 26 | private E payload; 27 | private byte[] key; 28 | private long offset; 29 | private Partition partition; 30 | private String topic; 31 | private String consumer; 32 | 33 | public byte[] getKey() { 34 | return key; 35 | } 36 | 37 | public void setKey(byte[] key) { 38 | this.key = key; 39 | } 40 | 41 | public Partition getPartition() { 42 | return partition; 43 | } 44 | 45 | public void setPartition(Partition partition) { 46 | this.partition = partition; 47 | } 48 | 49 | public String getConsumer() { 50 | return consumer; 51 | } 52 | 53 | public void setConsumer(String consumer) { 54 | this.consumer = consumer; 55 | } 56 | 57 | public E getPayload() { 58 | return payload; 59 | } 60 | 61 | public void setPayload(E msg) { 62 | this.payload = msg; 63 | } 64 | 65 | public long getOffset() { 66 | return offset; 67 | } 68 | 69 | public void setOffset(long offset) { 70 | this.offset = offset; 71 | } 72 | 73 | public String getTopic() { 74 | return topic; 75 | } 76 | 77 | public void setTopic(String topic) { 78 | this.topic = topic; 79 | } 80 | } 81 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/PIDController.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 | package consumer.kafka; 20 | 21 | import org.slf4j.Logger; 22 | import org.slf4j.LoggerFactory; 23 | 24 | public class PIDController { 25 | 26 | private double proportional; 27 | private double integral; 28 | private double derivative; 29 | private long latestTime; 30 | private double latestError = -1; 31 | private int lastBatchFetchPercentage = 0; 32 | public long lastBatchRecords = 1; 33 | public boolean controllerStarted; 34 | 35 | public static final Logger LOG = LoggerFactory.getLogger(PIDController.class); 36 | 37 | public PIDController(double proportional, double integral, double derivative) { 38 | 39 | this.proportional = proportional; 40 | this.integral = integral; 41 | this.derivative = derivative; 42 | this.latestTime = System.currentTimeMillis(); 43 | this.controllerStarted = false; 44 | } 45 | 46 | public int calculateRate( 47 | KafkaConfig config, long batchDurationMs, int partitionCount, int fetchSizeBytes, 48 | int fillFreqMs, long schedulingDelayMs, long processingDelayMs, long numRecords) { 49 | 50 | long time = System.currentTimeMillis(); 51 | double delaySinceUpdate = (time - latestTime); 52 | 53 | //Safe guard 90% of Batch Duration 54 | batchDurationMs = (long) ((long)batchDurationMs * config._safeBatchPercent); 55 | 56 | //Every Fill create one Spark Block 57 | double blocksPerSecond = 1000 / fillFreqMs; 58 | 59 | //Incoming fetchSizeBytes set to 1KB due to Queue size increase. 60 | //Let not consider that to calculate next rate 61 | if(fetchSizeBytes == 1024) { 62 | fetchSizeBytes = config._fetchSizeBytes; 63 | } 64 | 65 | //Pull Rate bytes / seconds 66 | double fixedRate = (partitionCount * fetchSizeBytes) * blocksPerSecond; 67 | 68 | double processingDecay = 1.0; 69 | 70 | //Decay in processing 71 | if(processingDelayMs > 0) { 72 | processingDecay = batchDurationMs / (double) processingDelayMs; 73 | } 74 | 75 | // Processing Rate in bytes / seconds 76 | double processingRate = partitionCount * fetchSizeBytes * blocksPerSecond * processingDecay; 77 | 78 | //Proportional Error = Fixed Rate - Processing Rate in bytes / seconds 79 | double proportationalError = (double) (fixedRate - processingRate); 80 | 81 | //Historical Error = Scheduling Delay * Processing Rate / Batch Duration (in bytes /second) 82 | double historicalError = (schedulingDelayMs * processingRate) / batchDurationMs; 83 | 84 | //Differential Error. Error Rate is changing (in bytes /second) 85 | double differentialError = 0.0; 86 | 87 | if(delaySinceUpdate > 0) { 88 | differentialError = (proportationalError - latestError) / delaySinceUpdate; 89 | } 90 | 91 | double revisedRate = (fixedRate 92 | - proportional * proportationalError 93 | - integral * historicalError 94 | - derivative * differentialError); 95 | 96 | //Predicted next batch fetch rate 97 | double revisedFetchSize = (revisedRate / partitionCount) / blocksPerSecond; 98 | //Predicted next batch fetch percentage 99 | int nextBatchFetchPercentage = (int)(((double)(((revisedFetchSize - fetchSizeBytes) / fetchSizeBytes)) * 100)); 100 | //Max allowable change is 20 %. 101 | //Cap the max change to avoid overshoot 102 | if(Math.abs(nextBatchFetchPercentage) > (int)(config._maxRateChangePercent * 100)) { 103 | if(nextBatchFetchPercentage > 0) { 104 | revisedFetchSize = fetchSizeBytes + fetchSizeBytes * config._maxRateChangePercent; 105 | nextBatchFetchPercentage = (int)(config._maxRateChangePercent * 100); 106 | } else { 107 | revisedFetchSize = fetchSizeBytes - fetchSizeBytes * config._maxRateChangePercent; 108 | nextBatchFetchPercentage = - (int)(config._maxRateChangePercent * 100); 109 | } 110 | } 111 | 112 | LOG.info("======== Rate Revision Starts ========"); 113 | LOG.info("Current Fetch Size : " + fetchSizeBytes); 114 | LOG.info("Fill Freq : " + fillFreqMs); 115 | LOG.info("Batch Duration : " + batchDurationMs); 116 | LOG.info("Partition count : " + partitionCount); 117 | LOG.info("Scheduling Delay : " + schedulingDelayMs); 118 | LOG.info("Processing Delay : " + processingDelayMs); 119 | LOG.info("Fixed Rate : " + (int)fixedRate); 120 | LOG.info("Processing rate : " + (int)processingRate); 121 | LOG.info("Proportional Error : " + (int)proportationalError); 122 | LOG.info("HistoricalError : " + (int)historicalError); 123 | LOG.info("DifferentialError : " + (int)differentialError); 124 | LOG.info("Reviced Rate : " + (int)revisedRate); 125 | LOG.info("Proposed FetchPercent : " + (int)nextBatchFetchPercentage); 126 | 127 | if(!controllerStarted) { 128 | //warm up controller for first iteration 129 | controllerStarted = true; 130 | lastBatchRecords = numRecords; 131 | lastBatchFetchPercentage = nextBatchFetchPercentage; 132 | revisedFetchSize = config._fetchSizeBytes; 133 | } else { 134 | // Check if predicted rate can be applied based on historical changes 135 | // If predicted percentage > 0, check if number of records fetched in equal proportion 136 | if(nextBatchFetchPercentage > 0) { 137 | int currentBatchRecordPercentage = (int)(((((numRecords - lastBatchRecords) / (double)lastBatchRecords)) * 100)); 138 | LOG.info("Last FetchPercent : " + (int)lastBatchFetchPercentage); 139 | LOG.info("Current RecordPercent : " + (int)currentBatchRecordPercentage); 140 | //Consumed records in this batch is higher than earlier batch 141 | if(currentBatchRecordPercentage > 0 && currentBatchRecordPercentage < nextBatchFetchPercentage ) { 142 | revisedFetchSize = fetchSizeBytes + fetchSizeBytes * currentBatchRecordPercentage / 100; 143 | nextBatchFetchPercentage = currentBatchRecordPercentage; 144 | //Lower number of records fetched from earlier batch. Let lower the rate 145 | } else if (currentBatchRecordPercentage <= 0) { 146 | revisedFetchSize = fetchSizeBytes + fetchSizeBytes * currentBatchRecordPercentage / 100; 147 | //If this goes below configured _fetchSizeBytes , floor it 148 | if(revisedFetchSize < config._fetchSizeBytes) { 149 | revisedFetchSize = config._fetchSizeBytes; 150 | nextBatchFetchPercentage = 0; 151 | } else { 152 | nextBatchFetchPercentage = currentBatchRecordPercentage; 153 | } 154 | } 155 | } 156 | 157 | LOG.info("Last FetchRecords : " + (int)lastBatchRecords); 158 | LOG.info("Current FetchRecords : " + (int)numRecords); 159 | //These will be used for next batch 160 | lastBatchRecords = (numRecords > 0) ? numRecords : 1; 161 | lastBatchFetchPercentage = nextBatchFetchPercentage; 162 | } 163 | LOG.info("Reviced FetchSize : " + (int)revisedFetchSize); 164 | LOG.info("Reviced PercentChange : " + nextBatchFetchPercentage ) ; 165 | LOG.info("======== Rate Revision Ends ========"); 166 | latestError = proportationalError; 167 | latestTime = time; 168 | return new Double(revisedFetchSize).intValue(); 169 | } 170 | } 171 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/Partition.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 Partition implements Serializable { 33 | 34 | public final Broker host; 35 | public final int partition; 36 | 37 | public Partition(Broker host, int partition) { 38 | this.host = host; 39 | this.partition = partition; 40 | } 41 | 42 | @Override 43 | public int hashCode() { 44 | return Objects.hashCode(host, partition); 45 | } 46 | 47 | @Override 48 | public boolean equals(Object obj) { 49 | if (this == obj) { 50 | return true; 51 | } 52 | if (obj == null || getClass() != obj.getClass()) { 53 | return false; 54 | } 55 | final Partition other = (Partition) obj; 56 | return Objects.equal(this.host, other.host) 57 | && Objects.equal(this.partition, other.partition); 58 | } 59 | 60 | @Override 61 | public String toString() { 62 | return "Partition{" + "host=" + host + ", partition=" + partition + '}'; 63 | } 64 | 65 | public String getId() { 66 | return Integer.toString(partition); 67 | } 68 | 69 | } 70 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/PartitionCoordinator.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.List; 29 | 30 | public interface PartitionCoordinator extends Serializable { 31 | public List getMyManagedPartitions(); 32 | public PartitionManager getManager(Partition partition); 33 | public void refresh(); 34 | } 35 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/PartitionManager.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.collect.ImmutableMap; 28 | import kafka.api.OffsetRequest; 29 | import kafka.javaapi.FetchResponse; 30 | import kafka.javaapi.consumer.SimpleConsumer; 31 | import kafka.javaapi.message.ByteBufferMessageSet; 32 | import kafka.message.Message; 33 | import kafka.message.MessageAndOffset; 34 | import org.apache.spark.streaming.receiver.Receiver; 35 | import org.slf4j.Logger; 36 | import org.slf4j.LoggerFactory; 37 | 38 | import java.io.Serializable; 39 | import java.util.LinkedList; 40 | import java.util.Map; 41 | 42 | @SuppressWarnings("serial") 43 | public class PartitionManager implements Serializable { 44 | private static final Logger LOG = LoggerFactory 45 | .getLogger(PartitionManager.class); 46 | private Long _emittedToOffset; 47 | private Long _lastComittedOffset; 48 | private long _lastEnquedOffset; 49 | private LinkedList _arrayBuffer = 50 | new LinkedList(); 51 | private Partition _partition; 52 | private KafkaConfig _kafkaconfig; 53 | private String _consumerId; 54 | private transient SimpleConsumer _consumer; 55 | private DynamicPartitionConnections _connections; 56 | private ZkState _state; 57 | private String _topic; 58 | private Map _stateConf; 59 | private Receiver _receiver; 60 | boolean _restart; 61 | private KafkaMessageHandler _handler; 62 | private int _numFetchBuffered = 1; 63 | 64 | public PartitionManager( 65 | DynamicPartitionConnections connections, 66 | ZkState state, 67 | KafkaConfig kafkaconfig, 68 | Partition partitionId, 69 | Receiver receiver, 70 | boolean restart, 71 | KafkaMessageHandler messageHandler) { 72 | _partition = partitionId; 73 | _connections = connections; 74 | _kafkaconfig = kafkaconfig; 75 | _stateConf = _kafkaconfig._stateConf; 76 | _consumerId = (String) _stateConf.get(Config.KAFKA_CONSUMER_ID); 77 | _consumer = connections.register(partitionId.host, partitionId.partition); 78 | _state = state; 79 | _topic = (String) _stateConf.get(Config.KAFKA_TOPIC); 80 | _receiver = receiver; 81 | _restart = restart; 82 | _handler = messageHandler; 83 | 84 | Long processOffset = null; 85 | Long consumedOffset = null; 86 | String processPath = zkPath("processed"); 87 | String consumedPath = zkPath("offsets"); 88 | 89 | try { 90 | byte[] pOffset = _state.readBytes(processPath); 91 | LOG.info("Read processed information from: {}", processPath); 92 | if (pOffset != null) { 93 | processOffset = Long.valueOf(new String(pOffset)); 94 | LOG.info("Processed offset for Partition : {} is {}",_partition.partition, processOffset); 95 | } 96 | byte[] conffset = _state.readBytes(consumedPath); 97 | LOG.info("Read consumed information from: {}", consumedPath); 98 | if (conffset != null) { 99 | consumedOffset = Long.valueOf(new String(conffset)); 100 | LOG.info("Consumed offset for Partition : {} is {}",_partition.partition, consumedOffset); 101 | } 102 | } catch (Throwable e) { 103 | LOG.warn("Error reading and/or parsing at ZkNode", e); 104 | throw e; 105 | } 106 | // failed to parse JSON? 107 | if (consumedOffset == null) { 108 | _lastComittedOffset = 109 | KafkaUtils.getOffset( 110 | _consumer, _topic, _partition.partition, kafkaconfig); 111 | LOG.info("No partition information found, using configuration to determine offset"); 112 | } else { 113 | if (_restart && processOffset != null) { 114 | _lastComittedOffset = processOffset + 1; 115 | } else { 116 | _lastComittedOffset = consumedOffset; 117 | } 118 | } 119 | 120 | LOG.info("Starting Receiver {} : {} from offset {}", _consumer.host(), _partition.partition, _lastComittedOffset); 121 | _emittedToOffset = _lastComittedOffset; 122 | _lastEnquedOffset = _lastComittedOffset; 123 | setZkCoordinator(); 124 | } 125 | 126 | //Used for Consumer offset Lag 127 | private void setZkCoordinator() { 128 | try{ 129 | String cordinate = String.format("Receiver-%s", _receiver.streamId()); 130 | _state.writeBytes(zkPath("owners"), (cordinate + "-0").getBytes()); 131 | Map data = 132 | (Map) ImmutableMap 133 | .builder() 134 | .put("version", 1) 135 | .put("subscription", ImmutableMap.of(_topic, 1)) 136 | .put("pattern", "static") 137 | .put("timestamp", Long.toString(System.currentTimeMillis())) 138 | .build(); 139 | _state.writeJSON(zkIdsPath("ids") + cordinate , data); 140 | } catch(Exception ne) { 141 | LOG.error("Node already exists" , ne); 142 | } 143 | } 144 | 145 | //Called every Fill Frequency 146 | public void next() throws Exception { 147 | fill(); 148 | //If consumer.num_fetch_to_buffer is default (1) , let commit consumed offset after every fill 149 | //Otherwise consumed offset will be written after buffer is filled during triggerBlockManagerWrite 150 | if ((_kafkaconfig._numFetchToBuffer == 1) && (_lastEnquedOffset >= _lastComittedOffset)) { 151 | try { 152 | _lastComittedOffset = _emittedToOffset; 153 | _state.writeBytes(zkPath("offsets"), Long.toString(_lastComittedOffset).getBytes()); 154 | LOG.info("Consumed offset {} for Partition {} written to ZK", _lastComittedOffset, _partition.partition); 155 | } catch (Exception ex) { 156 | _receiver.reportError("Retry ZK Commit for Partition " + _partition, ex); 157 | } 158 | } 159 | } 160 | 161 | private long getKafkaOffset() { 162 | long kafkaOffset = KafkaUtils.getOffset(_consumer, _topic, _partition.partition, -1); 163 | if (kafkaOffset == KafkaUtils.NO_OFFSET) { 164 | LOG.warn("kafka latest offset not found for partition {}", _partition.partition); 165 | } 166 | return kafkaOffset; 167 | } 168 | 169 | private void reportOffsetLag() { 170 | try { 171 | long offsetLag = calculateOffsetLag(); 172 | LOG.info("Offset Lag for Parittion {} is {} " , _partition.partition, offsetLag); 173 | } catch (Exception e) { 174 | LOG.error("failed to report offset lag to graphite", e); 175 | } 176 | } 177 | 178 | private long calculateOffsetLag() { 179 | long offsetLag = 0; 180 | long kafkaOffset = getKafkaOffset(); 181 | if (kafkaOffset != KafkaUtils.NO_OFFSET) { 182 | offsetLag = kafkaOffset - _lastComittedOffset; 183 | } 184 | return offsetLag; 185 | } 186 | 187 | 188 | //This is called when consumer.num_fetch_to_buffer is set and when buffer is filled and 189 | //written to Spark Block Manager during fill 190 | private void triggerBlockManagerWrite() { 191 | if ((_lastEnquedOffset >= _lastComittedOffset)) { 192 | try { 193 | synchronized (_receiver) { 194 | if (!_arrayBuffer.isEmpty() && !_receiver.isStopped()) { 195 | _receiver.store(_arrayBuffer.iterator()); 196 | _arrayBuffer.clear(); 197 | } 198 | _numFetchBuffered = 1; 199 | _lastComittedOffset = _emittedToOffset; 200 | //Write consumed offset to ZK 201 | _state.writeBytes(zkPath("offsets"), Long.toString(_lastComittedOffset).getBytes()); 202 | LOG.info("Consumed offset {} for Partition {} written to ZK", _lastComittedOffset, _partition.partition); 203 | } 204 | } catch (Exception ex) { 205 | _arrayBuffer.clear(); 206 | _receiver.reportError("Retry Store for Partition " + _partition, ex); 207 | } 208 | } 209 | } 210 | 211 | //Read from Kafka and write to Spark BlockManager 212 | private void fill() { 213 | String topic = _kafkaconfig._stateConf.get(Config.KAFKA_TOPIC); 214 | ByteBufferMessageSet msgs; 215 | //Get the present fetchSize from ZK set by PID Controller 216 | int fetchSize = getFetchSize(); 217 | //Fetch messages from Kafka 218 | msgs = fetchMessages(fetchSize, topic); 219 | for (MessageAndOffset msgAndOffset : msgs) { 220 | if (msgAndOffset.message() != null) { 221 | long key = msgAndOffset.offset(); 222 | Message msg = msgAndOffset.message(); 223 | _emittedToOffset = msgAndOffset.nextOffset(); 224 | _lastEnquedOffset = key; 225 | //Process only when fetched messages are having higher offset than last committed offset 226 | if (_lastEnquedOffset >= _lastComittedOffset) { 227 | if (msg.payload() != null) { 228 | byte[] payload = new byte[msg.payload().remaining()]; 229 | msg.payload().get(payload); 230 | MessageAndMetadata mm = null; 231 | try { 232 | //Perform Message Handling if configured. 233 | mm = _handler.handle(_lastEnquedOffset, _partition, _topic, _consumerId, payload); 234 | if (msg.hasKey()) { 235 | byte[] msgKey = new byte[msg.key().remaining()]; 236 | msg.key().get(msgKey); 237 | mm.setKey(msgKey); 238 | } 239 | } catch (Exception e) { 240 | LOG.error("Process Failed for offset {} partition {} topic {}", key, _partition, _topic, e); 241 | } 242 | if (_kafkaconfig._numFetchToBuffer > 1) { 243 | // Add to buffer 244 | if(mm != null ) { 245 | _arrayBuffer.add(mm); 246 | _numFetchBuffered = _numFetchBuffered + 1; 247 | } 248 | //Trigger write when buffer reach the limit 249 | LOG.debug("number of fetch buffered for partition {} is {}", _partition.partition, _numFetchBuffered); 250 | if (_numFetchBuffered > _kafkaconfig._numFetchToBuffer) { 251 | triggerBlockManagerWrite(); 252 | LOG.info("Trigger BM write till offset {} for Partition {}", _lastEnquedOffset, _partition.partition); 253 | } 254 | } else { 255 | //nothing to buffer. Just add to Spark Block Manager 256 | try { 257 | synchronized (_receiver) { 258 | if(mm != null) { 259 | _receiver.store(mm); 260 | LOG.debug("PartitionManager sucessfully written offset {} for partition {} to BM", _lastEnquedOffset, _partition.partition); 261 | } 262 | } 263 | } catch (Exception ex) { 264 | _receiver.reportError("Retry Store for Partition " + _partition, ex); 265 | } 266 | } 267 | } 268 | } 269 | } 270 | } 271 | } 272 | 273 | //Invoke Kafka API to fetch messages 274 | private ByteBufferMessageSet fetchMessages(int fetchSize, String topic) { 275 | FetchResponse fetchResponse = KafkaUtils.fetchMessages( 276 | _kafkaconfig, _consumer, _partition, _emittedToOffset, fetchSize); 277 | if (fetchResponse.hasError()) { 278 | KafkaError error = KafkaError.getError(fetchResponse.errorCode( 279 | topic, _partition.partition)); 280 | if (error.equals(KafkaError.OFFSET_OUT_OF_RANGE)) { 281 | long earliestTime = kafka.api.OffsetRequest.EarliestTime(); 282 | long latestTime = kafka.api.OffsetRequest.LatestTime(); 283 | long earliestOffset = KafkaUtils.getOffset(_consumer, topic, _partition.partition, earliestTime); 284 | long latestOffset = KafkaUtils.getOffset(_consumer, topic, _partition.partition, latestTime); 285 | LOG.warn("Got fetch request with offset out of range: {} for Topic {} partition {}" , _emittedToOffset, topic, _partition.partition); 286 | 287 | //If OFFSET_OUT_OF_RANGE , check if present _emittedToOffset is greater than Partition's latest offset 288 | //This can happen if new Leader is behind the previous leader when elected during a Kafka broker failure. 289 | if(_emittedToOffset >= latestOffset) { 290 | _emittedToOffset = latestOffset; 291 | LOG.warn("Offset reset to LatestTime {} for Topic {} partition {}" , _emittedToOffset, topic, _partition.partition); 292 | } else if (_emittedToOffset <= earliestOffset) { 293 | //This can happen if messages are deleted from Kafka due to Kafka's log retention period and 294 | //probably there is huge lag in Consumer. Or consumer is stopped for long time. 295 | _emittedToOffset = earliestOffset; 296 | _lastComittedOffset = earliestOffset; 297 | LOG.warn("Offset reset to EarliestTime {} for Topic {} partition {}" , _emittedToOffset, topic, _partition.partition); 298 | } 299 | fetchResponse = KafkaUtils.fetchMessages( 300 | _kafkaconfig, _consumer, _partition, _emittedToOffset, fetchSize); 301 | } else { 302 | String message = "Error fetching data from [" 303 | + _partition.partition + "] for topic [" + topic + "]: [" + error + "]"; 304 | LOG.error(message); 305 | throw new FailedFetchException(message); 306 | } 307 | } 308 | return fetchResponse.messageSet(topic, _partition.partition); 309 | } 310 | 311 | //Get fetchSize from ZK 312 | private int getFetchSize() { 313 | int newFetchSize = 0; 314 | try { 315 | byte[] rate = _state.readBytes(ratePath()); 316 | if (rate != null) { 317 | newFetchSize = Integer.valueOf(new String(rate)); 318 | LOG.debug("Current Fetch Rate for topic {} is {}", 319 | _kafkaconfig._stateConf.get(Config.KAFKA_TOPIC), newFetchSize); 320 | } else { 321 | newFetchSize = _kafkaconfig._fetchSizeBytes; 322 | } 323 | } catch (Throwable e) { 324 | newFetchSize = _kafkaconfig._fetchSizeBytes; 325 | } 326 | return newFetchSize; 327 | } 328 | 329 | public String ratePath() { 330 | return _kafkaconfig._stateConf.get(Config.ZOOKEEPER_CONSUMER_PATH) 331 | + "/" + _kafkaconfig._stateConf.get(Config.KAFKA_CONSUMER_ID) + "/newrate"; 332 | } 333 | 334 | private String zkPath(String type) { 335 | return _stateConf.get(Config.ZOOKEEPER_CONSUMER_PATH) 336 | + "/" + _stateConf.get(Config.KAFKA_CONSUMER_ID) + "/" + type+ "/" 337 | + _stateConf.get(Config.KAFKA_TOPIC) + "/" + _partition.getId(); 338 | } 339 | 340 | private String zkIdsPath(String type) { 341 | return _stateConf.get(Config.ZOOKEEPER_CONSUMER_PATH) 342 | + "/" + _stateConf.get(Config.KAFKA_CONSUMER_ID) + "/" + type+ "/" 343 | + _stateConf.get(Config.KAFKA_TOPIC) + "/"; 344 | } 345 | 346 | public long queryPartitionOffsetLatestTime() { 347 | return KafkaUtils.getOffset( 348 | _consumer, _topic, _partition.partition, OffsetRequest.LatestTime()); 349 | } 350 | 351 | public long lastCommittedOffset() { 352 | return _lastComittedOffset; 353 | } 354 | 355 | public Partition getPartition() { 356 | return _partition; 357 | } 358 | 359 | public void close() { 360 | try { 361 | LOG.info("Flush BlockManager Write for Partition {}", _partition.partition); 362 | _numFetchBuffered = _kafkaconfig._numFetchToBuffer; 363 | triggerBlockManagerWrite(); 364 | _connections.unregister(_partition.host, _partition.partition); 365 | _connections.clear(); 366 | LOG.info("Closed connection for {}", _partition); 367 | } catch (Exception ex) { 368 | ex.printStackTrace(); 369 | LOG.error("Error closing connection" + " for " + _partition); 370 | } 371 | } 372 | } 373 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/PartitionOffsetPair.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 | package consumer.kafka; 20 | 21 | import org.apache.spark.api.java.function.PairFlatMapFunction; 22 | import org.slf4j.Logger; 23 | import org.slf4j.LoggerFactory; 24 | import scala.Tuple2; 25 | 26 | import java.util.ArrayList; 27 | import java.util.Iterator; 28 | import java.util.List; 29 | 30 | 31 | /** 32 | * Extracts the kafka-paritition-number and largest-offset-read-for-that-partition from the kafka-receiver output 33 | */ 34 | public class PartitionOffsetPair implements PairFlatMapFunction>, Integer, Long> { 35 | private static final Logger LOG = LoggerFactory.getLogger(PartitionOffsetPair.class); 36 | 37 | @Override 38 | public Iterator> call(Iterator> it) throws Exception { 39 | MessageAndMetadata mmeta = null; 40 | while (it.hasNext()) { 41 | mmeta = it.next(); 42 | LOG.debug("Consumed partition = {}, offset = {}", mmeta.getPartition(), mmeta.getOffset()); 43 | } 44 | // Return the kafka-partition-number and the largest offset read 45 | List> kafkaPartitionToOffsetList = new ArrayList<>(1); 46 | if (mmeta != null) { 47 | LOG.debug("selected largest offset {} for partition {}", mmeta.getOffset(), mmeta.getPartition()); 48 | kafkaPartitionToOffsetList.add(new Tuple2<>(mmeta.getPartition().partition, mmeta.getOffset())); 49 | } 50 | return kafkaPartitionToOffsetList.iterator(); 51 | } 52 | } 53 | 54 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/ProcessedOffsetManager.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 | package consumer.kafka; 20 | 21 | import org.apache.commons.logging.Log; 22 | import org.apache.commons.logging.LogFactory; 23 | import org.apache.spark.api.java.JavaPairRDD; 24 | import org.apache.spark.api.java.JavaRDD; 25 | import org.apache.spark.api.java.function.VoidFunction; 26 | import org.apache.spark.streaming.api.java.JavaDStream; 27 | import org.apache.spark.streaming.api.java.JavaPairDStream; 28 | import org.apache.spark.streaming.dstream.DStream; 29 | import scala.Tuple2; 30 | import scala.reflect.ClassTag; 31 | 32 | import java.util.HashMap; 33 | import java.util.List; 34 | import java.util.Map; 35 | import java.util.Properties; 36 | 37 | public class ProcessedOffsetManager { 38 | 39 | private static final Log LOG = LogFactory.getLog(ProcessedOffsetManager.class); 40 | 41 | public static JavaPairDStream> getPartitionOffset( 42 | JavaDStream> unionStreams, Properties props) { 43 | JavaPairDStream partitonOffsetStream = unionStreams.mapPartitionsToPair(new PartitionOffsetPair<>()); 44 | JavaPairDStream> partitonOffset = partitonOffsetStream.groupByKey(1); 45 | return partitonOffset; 46 | } 47 | 48 | @SuppressWarnings("deprecation") 49 | public static void persists(JavaPairDStream> partitonOffset, Properties props) { 50 | partitonOffset.foreachRDD(new VoidFunction>>() { 51 | @Override 52 | public void call(JavaPairRDD> po) throws Exception { 53 | List>> poList = po.collect(); 54 | doPersists(poList, props); 55 | } 56 | }); 57 | } 58 | 59 | public static DStream>> getPartitionOffset( 60 | DStream> unionStreams, Properties props) { 61 | ClassTag> messageMetaClassTag = 62 | ScalaUtil.getMessageAndMetadataClassTag(); 63 | JavaDStream> javaDStream = 64 | new JavaDStream>(unionStreams, messageMetaClassTag); 65 | JavaPairDStream> partitonOffset = getPartitionOffset(javaDStream, props); 66 | return partitonOffset.dstream(); 67 | } 68 | 69 | @SuppressWarnings("deprecation") 70 | public static void persists(DStream>> partitonOffset, Properties props) { 71 | ClassTag>> tuple2ClassTag = 72 | ScalaUtil.>getTuple2ClassTag(); 73 | JavaDStream>> jpartitonOffset = 74 | new JavaDStream>>(partitonOffset, tuple2ClassTag); 75 | jpartitonOffset.foreachRDD(new VoidFunction>>>() { 76 | @Override 77 | public void call(JavaRDD>> po) throws Exception { 78 | List>> poList = po.collect(); 79 | doPersists(poList, props); 80 | } 81 | }); 82 | } 83 | 84 | private static void doPersists(List>> poList, Properties props) { 85 | Map partitionOffsetMap = new HashMap(); 86 | for(Tuple2> tuple : poList) { 87 | int partition = tuple._1(); 88 | Long offset = getMaximum(tuple._2()); 89 | partitionOffsetMap.put(partition, offset); 90 | } 91 | persistProcessedOffsets(props, partitionOffsetMap); 92 | } 93 | 94 | private static > T getMaximum(Iterable values) { 95 | T max = null; 96 | for (T value : values) { 97 | if (max == null || max.compareTo(value) < 0) { 98 | max = value; 99 | } 100 | } 101 | return max; 102 | } 103 | 104 | private static void persistProcessedOffsets(Properties props, Map partitionOffsetMap) { 105 | ZkState state = new ZkState(props.getProperty(Config.ZOOKEEPER_CONSUMER_CONNECTION)); 106 | for(Map.Entry po : partitionOffsetMap.entrySet()) { 107 | String path = processedPath(po.getKey(), props); 108 | try{ 109 | state.writeBytes(path, po.getValue().toString().getBytes()); 110 | LOG.info("Wrote processed offset " + po.getValue() + " for Parittion " + po.getKey()); 111 | }catch (Exception ex) { 112 | LOG.error("Error while comitting processed offset " + po.getValue() + " for Parittion " + po.getKey(), ex); 113 | state.close(); 114 | throw ex; 115 | } 116 | } 117 | state.close(); 118 | } 119 | 120 | public static String processedPath(int partition, Properties props) { 121 | String consumerZkPath = "/consumers"; 122 | if (props.getProperty("zookeeper.consumer.path") != null) { 123 | consumerZkPath = props.getProperty("zookeeper.consumer.path"); 124 | } 125 | return consumerZkPath + "/" 126 | + props.getProperty(Config.KAFKA_CONSUMER_ID) 127 | + "/processed/" 128 | + props.getProperty(Config.KAFKA_TOPIC) + "/" 129 | + partition; 130 | } 131 | } 132 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/ReceiverLauncher.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 | package consumer.kafka; 20 | 21 | import consumer.kafka.client.KafkaRangeReceiver; 22 | import consumer.kafka.client.KafkaReceiver; 23 | import org.apache.spark.storage.StorageLevel; 24 | import org.apache.spark.streaming.StreamingContext; 25 | import org.apache.spark.streaming.api.java.JavaDStream; 26 | import org.apache.spark.streaming.api.java.JavaStreamingContext; 27 | import org.apache.spark.streaming.dstream.DStream; 28 | import org.slf4j.Logger; 29 | import org.slf4j.LoggerFactory; 30 | 31 | import java.io.Serializable; 32 | import java.util.*; 33 | import java.util.concurrent.atomic.AtomicBoolean; 34 | 35 | @SuppressWarnings("serial") 36 | public class ReceiverLauncher implements Serializable { 37 | public static final Logger LOG = LoggerFactory.getLogger(ReceiverLauncher.class); 38 | 39 | private static String _zkPath; 40 | 41 | public static DStream> launch( 42 | StreamingContext ssc, Properties pros, int numberOfReceivers, 43 | StorageLevel storageLevel, KafkaMessageHandler messageHandler) { 44 | JavaStreamingContext jsc = new JavaStreamingContext(ssc); 45 | return createStream(jsc, pros, numberOfReceivers, storageLevel, messageHandler).dstream(); 46 | } 47 | 48 | public static DStream> launch( 49 | StreamingContext ssc, Properties pros, int numberOfReceivers, StorageLevel storageLevel) { 50 | JavaStreamingContext jsc = new JavaStreamingContext(ssc); 51 | return createStream(jsc, pros, numberOfReceivers, storageLevel, new IdentityMessageHandler()).dstream(); 52 | } 53 | 54 | public static JavaDStream> launch( 55 | JavaStreamingContext jsc, Properties pros, int numberOfReceivers, 56 | StorageLevel storageLevel, KafkaMessageHandler messageHandler) { 57 | return createStream(jsc, pros, numberOfReceivers, storageLevel, messageHandler); 58 | } 59 | 60 | public static JavaDStream> launch( 61 | JavaStreamingContext jsc, Properties pros, int numberOfReceivers, StorageLevel storageLevel) { 62 | return createStream(jsc, pros, numberOfReceivers, storageLevel, new IdentityMessageHandler()); 63 | } 64 | 65 | private static JavaDStream> createStream( 66 | JavaStreamingContext jsc, Properties pros, int numberOfReceivers, StorageLevel storageLevel, 67 | KafkaMessageHandler messageHandler) { 68 | 69 | AtomicBoolean terminateOnFailure = new AtomicBoolean(false); 70 | List>> streamsList = 71 | new ArrayList<>(); 72 | JavaDStream> unionStreams; 73 | int numberOfPartition; 74 | KafkaConfig kafkaConfig = new KafkaConfig(pros); 75 | ZkState zkState = new ZkState(kafkaConfig); 76 | String numberOfPartitionStr = 77 | (String) pros.getProperty(Config.KAFKA_PARTITIONS_NUMBER); 78 | if (numberOfPartitionStr != null) { 79 | numberOfPartition = Integer.parseInt(numberOfPartitionStr); 80 | } else { 81 | _zkPath = (String) kafkaConfig._stateConf.get(Config.ZOOKEEPER_BROKER_PATH); 82 | String _topic = (String) kafkaConfig._stateConf.get(Config.KAFKA_TOPIC); 83 | numberOfPartition = getNumPartitions(zkState, _topic); 84 | } 85 | 86 | // Create as many Receiver as Partition 87 | if (numberOfReceivers >= numberOfPartition) { 88 | for (int i = 0; i < numberOfPartition; i++) { 89 | streamsList.add(jsc.receiverStream(new KafkaReceiver( 90 | pros, i, storageLevel, messageHandler))); 91 | } 92 | } else { 93 | // create Range Receivers.. 94 | Map> rMap = new HashMap>(); 95 | 96 | for (int i = 0; i < numberOfPartition; i++) { 97 | int j = i % numberOfReceivers; 98 | Set pSet = rMap.get(j); 99 | if (pSet == null) { 100 | pSet = new HashSet(); 101 | pSet.add(i); 102 | } else { 103 | pSet.add(i); 104 | } 105 | rMap.put(j, pSet); 106 | } 107 | for (int i = 0; i < numberOfReceivers; i++) { 108 | streamsList.add(jsc.receiverStream(new KafkaRangeReceiver(pros, rMap 109 | .get(i), storageLevel, messageHandler))); 110 | } 111 | } 112 | 113 | // Union all the streams if there is more than 1 stream 114 | if (streamsList.size() > 1) { 115 | unionStreams = 116 | jsc.union( 117 | streamsList.get(0), streamsList.subList(1, streamsList.size())); 118 | } else { 119 | // Otherwise, just use the 1 stream 120 | unionStreams = streamsList.get(0); 121 | } 122 | 123 | final long batchDuration = jsc.ssc().graph().batchDuration().milliseconds(); 124 | ReceiverStreamListener listener = new ReceiverStreamListener(kafkaConfig, 125 | batchDuration, numberOfPartition, terminateOnFailure); 126 | 127 | jsc.addStreamingListener(listener); 128 | //Reset the fetch size 129 | Utils.setFetchRate(kafkaConfig, kafkaConfig._fetchSizeBytes); 130 | return unionStreams; 131 | } 132 | 133 | private static int getNumPartitions(ZkState zkState, String topic) { 134 | try { 135 | String topicBrokersPath = partitionPath(topic); 136 | List children = 137 | zkState.getCurator().getChildren().forPath(topicBrokersPath); 138 | return children.size(); 139 | } catch (Exception e) { 140 | throw new RuntimeException(e); 141 | } 142 | } 143 | 144 | private static String partitionPath(String topic) { 145 | return _zkPath + "/topics/" + topic + "/partitions"; 146 | } 147 | } 148 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/ReceiverStreamListener.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 | package consumer.kafka; 20 | 21 | import org.apache.spark.streaming.scheduler.*; 22 | import org.slf4j.Logger; 23 | import org.slf4j.LoggerFactory; 24 | import scala.Option; 25 | 26 | import java.util.concurrent.atomic.AtomicBoolean; 27 | 28 | public class ReceiverStreamListener implements StreamingListener { 29 | private static Logger LOG = LoggerFactory.getLogger(ReceiverStreamListener.class); 30 | 31 | private int batchSubmittedCount = 0; 32 | private int batchCompletedCount = 0; 33 | private KafkaConfig config; 34 | private int MAX_RATE; 35 | private int THROTTLE_QUEUE; 36 | private int MIN_RATE; 37 | private int fillFreqMs; 38 | private AtomicBoolean terminateOnFailure; 39 | private long batchDuration; 40 | private int partitionCount; 41 | private PIDController controller; 42 | 43 | public ReceiverStreamListener(KafkaConfig config, 44 | long batchDuration, int partitionCount, AtomicBoolean terminateOnFailure) { 45 | this.config = config; 46 | MAX_RATE = config._maxFetchSizeBytes; 47 | THROTTLE_QUEUE = config._batchQueueToThrottle; 48 | MIN_RATE = config._minFetchSizeBytes; 49 | fillFreqMs = config._fillFreqMs; 50 | this.terminateOnFailure = terminateOnFailure; 51 | this.batchDuration = batchDuration; 52 | this.partitionCount = partitionCount; 53 | 54 | controller = new PIDController( 55 | config._proportional, 56 | config._integral, 57 | config._derivative); 58 | } 59 | 60 | @Override 61 | public void onReceiverStopped(StreamingListenerReceiverStopped receiverStopped) { 62 | LOG.error("Receiver stopped with error {}", receiverStopped.receiverInfo().lastErrorMessage()); 63 | } 64 | 65 | @Override 66 | public void onReceiverStarted(StreamingListenerReceiverStarted arg0) { 67 | } 68 | 69 | @Override 70 | public void onReceiverError(StreamingListenerReceiverError error) { 71 | } 72 | 73 | @Override 74 | public void onOutputOperationStarted(StreamingListenerOutputOperationStarted outPutOpsStart) { 75 | } 76 | 77 | @Override 78 | public void onOutputOperationCompleted(StreamingListenerOutputOperationCompleted outPutOpsComplete) { 79 | Option reason = outPutOpsComplete.outputOperationInfo().failureReason(); 80 | if (!reason.isEmpty()) { 81 | String failure = reason.get(); 82 | if (failure != null) { 83 | LOG.error("Output Operation failed due to {}", failure); 84 | } 85 | } 86 | } 87 | 88 | @Override 89 | public void onBatchSubmitted(StreamingListenerBatchSubmitted arg0) { 90 | batchSubmittedCount++; 91 | int queueSize = batchSubmittedCount - batchCompletedCount; 92 | if (queueSize > THROTTLE_QUEUE) { 93 | LOG.warn("stop consumer as pending queue {} greater than configured limit {}",queueSize, THROTTLE_QUEUE); 94 | //Set fetch size to 1 KB to throttle the consumer 95 | Utils.setFetchRate(config,1024); 96 | } else if(!config._backpressureEnabled) { 97 | Utils.setFetchRate(config, config._fetchSizeBytes); 98 | } 99 | } 100 | 101 | @Override 102 | public void onBatchStarted(StreamingListenerBatchStarted arg0) { 103 | } 104 | 105 | @Override 106 | public void onBatchCompleted( 107 | StreamingListenerBatchCompleted batchCompleted) { 108 | batchCompletedCount++; 109 | boolean backPressureEnabled = (boolean) config._backpressureEnabled; 110 | if (backPressureEnabled) { 111 | long processingDelay = 112 | (Long) batchCompleted.batchInfo().processingDelay().get(); 113 | long schedulingDelay = 114 | (Long) batchCompleted.batchInfo().schedulingDelay().get(); 115 | long numrecords = batchCompleted.batchInfo().numRecords(); 116 | //Skip first batch as it may pull very less records which can have wrong rate for next batch 117 | if(batchCompletedCount > 1) { 118 | // Get last batch fetch size 119 | int batchFetchSize = getFetchSize(); 120 | LOG.info("Current Rate in ZooKeeper : " + batchFetchSize); 121 | // Revise rate on last rate 122 | int newRate = controller.calculateRate(config, batchDuration, partitionCount, 123 | batchFetchSize, fillFreqMs, schedulingDelay, processingDelay, numrecords); 124 | LOG.info("Modified Rate by Controller : " + newRate); 125 | // Setting to Min Rate 126 | if (newRate < MIN_RATE) { 127 | newRate = MIN_RATE; 128 | } 129 | // Setting to Max Rate 130 | if (newRate > MAX_RATE) { 131 | newRate = MAX_RATE; 132 | } 133 | Utils.setFetchRate(config, newRate); 134 | } 135 | } 136 | } 137 | 138 | private int getFetchSize() { 139 | int newFetchSize = 0; 140 | ZkState state = new ZkState((String) config._stateConf 141 | .get(Config.ZOOKEEPER_CONSUMER_CONNECTION)); 142 | try { 143 | byte[] rate = state.readBytes(ratePath()); 144 | if (rate != null) { 145 | newFetchSize = Integer.valueOf(new String(rate)); 146 | LOG.info("Current Fetch Rate for topic {} is {}", 147 | config._stateConf.get("kafka.input.topic"), newFetchSize); 148 | } else { 149 | newFetchSize = config._fetchSizeBytes; 150 | } 151 | } catch (Throwable e) { 152 | newFetchSize = config._fetchSizeBytes; 153 | }finally { 154 | state.close(); 155 | } 156 | return newFetchSize; 157 | } 158 | 159 | public String ratePath() { 160 | return config._stateConf.get(Config.ZOOKEEPER_CONSUMER_PATH) 161 | + "/" + config._stateConf.get(Config.KAFKA_CONSUMER_ID) + "/newrate"; 162 | } 163 | } 164 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/ScalaUtil.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 | package consumer.kafka; 20 | 21 | import scala.Tuple2; 22 | import scala.collection.JavaConversions; 23 | import scala.collection.Seq; 24 | import scala.reflect.ClassTag; 25 | import scala.reflect.ClassTag$; 26 | 27 | import java.util.List; 28 | 29 | 30 | public class ScalaUtil { 31 | 32 | /** 33 | * Scala 2.10 use ClassTag to replace ClassManifest 34 | */ 35 | public static ClassTag getClassTag(Class clazz) { 36 | return ClassTag$.MODULE$.apply(clazz); 37 | } 38 | 39 | @SuppressWarnings("unchecked") 40 | public static ClassTag> getMessageAndMetadataClassTag() { 41 | return (ClassTag>)(Object) getClassTag(MessageAndMetadata.class); 42 | } 43 | 44 | @SuppressWarnings("unchecked") 45 | public static ClassTag> getTuple2ClassTag() { 46 | return (ClassTag>)(Object) getClassTag(Tuple2.class); 47 | } 48 | 49 | public static Seq toScalaSeq(List list) { 50 | return JavaConversions.asScalaBuffer(list); 51 | } 52 | } -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/Utils.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.nio.ByteBuffer; 28 | 29 | public class Utils { 30 | 31 | public static Integer getInt(Object o) { 32 | if (o instanceof Long) { 33 | return ((Long) o).intValue(); 34 | } else if (o instanceof Integer) { 35 | return (Integer) o; 36 | } else if (o instanceof Short) { 37 | return ((Short) o).intValue(); 38 | } else { 39 | throw new IllegalArgumentException("Don't know how to convert " 40 | + o + " + to int"); 41 | } 42 | } 43 | 44 | public static byte[] toByteArray(ByteBuffer buffer) { 45 | byte[] ret = new byte[buffer.remaining()]; 46 | buffer.get(ret, 0, ret.length); 47 | return ret; 48 | } 49 | 50 | public static void setFetchRate(KafkaConfig config, Integer rate) { 51 | String path = config._stateConf.get(Config.ZOOKEEPER_CONSUMER_PATH) 52 | + "/" + config._stateConf.get(Config.KAFKA_CONSUMER_ID) + "/newrate"; 53 | ZkState state = new ZkState((String) config._stateConf 54 | .get(Config.ZOOKEEPER_CONSUMER_CONNECTION)); 55 | state.writeBytes(path, rate.toString().getBytes()); 56 | state.close(); 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/ZkBrokerReader.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.slf4j.Logger; 28 | import org.slf4j.LoggerFactory; 29 | 30 | import java.io.Serializable; 31 | 32 | @SuppressWarnings("serial") 33 | public class ZkBrokerReader implements IBrokerReader, Serializable { 34 | 35 | public static final Logger LOG = LoggerFactory 36 | .getLogger(ZkBrokerReader.class); 37 | 38 | private GlobalPartitionInformation _cachedBrokers; 39 | private DynamicBrokersReader _reader; 40 | private long _lastRefreshTimeMs; 41 | private long refreshMillis; 42 | 43 | public ZkBrokerReader(KafkaConfig config, ZkState zkState) { 44 | _reader = new DynamicBrokersReader(config, zkState); 45 | _cachedBrokers = _reader.getBrokerInfo(); 46 | _lastRefreshTimeMs = System.currentTimeMillis(); 47 | refreshMillis = config._refreshFreqSecs * 1000L; 48 | 49 | } 50 | 51 | @Override 52 | public GlobalPartitionInformation getCurrentBrokers() { 53 | long currTime = System.currentTimeMillis(); 54 | if (currTime > _lastRefreshTimeMs + refreshMillis) { 55 | LOG.info("brokers need refreshing because {} ms have expired", refreshMillis); 56 | _cachedBrokers = _reader.getBrokerInfo(); 57 | _lastRefreshTimeMs = currTime; 58 | } 59 | return _cachedBrokers; 60 | } 61 | 62 | @Override 63 | public void close() { 64 | _reader.close(); 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/ZkCoordinator.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.spark.streaming.receiver.Receiver; 28 | import org.slf4j.Logger; 29 | import org.slf4j.LoggerFactory; 30 | 31 | import java.io.Serializable; 32 | import java.util.*; 33 | 34 | @SuppressWarnings("serial") 35 | public class ZkCoordinator implements PartitionCoordinator, Serializable { 36 | private static final Logger LOG = LoggerFactory.getLogger(ZkCoordinator.class); 37 | 38 | private KafkaConfig _kafkaconfig; 39 | private int _partitionOwner; 40 | private Map _managers = 41 | new HashMap(); 42 | private List _cachedList; 43 | private long _lastRefreshTime = 0L; 44 | private int _refreshFreqMs; 45 | private DynamicPartitionConnections _connections; 46 | private DynamicBrokersReader _reader; 47 | private GlobalPartitionInformation _brokerInfo; 48 | private KafkaConfig _config; 49 | private Receiver _receiver; 50 | private boolean _restart; 51 | private KafkaMessageHandler _messageHandler; 52 | 53 | public ZkCoordinator( 54 | DynamicPartitionConnections connections, 55 | KafkaConfig config, 56 | ZkState state, 57 | int partitionId, 58 | Receiver receiver, 59 | boolean restart, 60 | KafkaMessageHandler messageHandler) { 61 | _kafkaconfig = config; 62 | _connections = connections; 63 | _partitionOwner = partitionId; 64 | _refreshFreqMs = config._refreshFreqSecs * 1000; 65 | _reader = new DynamicBrokersReader(_kafkaconfig, state); 66 | _brokerInfo = _reader.getBrokerInfo(); 67 | _config = config; 68 | _receiver = receiver; 69 | _restart = restart; 70 | _messageHandler = messageHandler; 71 | } 72 | 73 | @Override 74 | public List getMyManagedPartitions() { 75 | if ((System.currentTimeMillis() - _lastRefreshTime) > _refreshFreqMs) { 76 | refresh(); 77 | _lastRefreshTime = System.currentTimeMillis(); 78 | } 79 | _restart = false; 80 | return _cachedList; 81 | } 82 | 83 | @Override 84 | public void refresh() { 85 | try { 86 | LOG.info("Refreshing partition manager connections"); 87 | _brokerInfo = _reader.getBrokerInfo(); 88 | Set mine = new HashSet(); 89 | for (Partition partition : _brokerInfo) { 90 | if (partition.partition == _partitionOwner) { 91 | mine.add(partition); 92 | LOG.debug("Added partition index " 93 | + _partitionOwner + " for coordinator"); 94 | } 95 | } 96 | 97 | if (mine.size() == 0) { 98 | LOG.warn("Some issue getting Partition details.. Patrition Manager size Zero"); 99 | _managers.clear(); 100 | if (_cachedList != null) { 101 | _cachedList.clear(); 102 | } 103 | return; 104 | } else { 105 | Set curr = _managers.keySet(); 106 | Set newPartitions = new HashSet(mine); 107 | newPartitions.removeAll(curr); 108 | Set deletedPartitions = new HashSet(curr); 109 | deletedPartitions.removeAll(mine); 110 | LOG.info("Deleted partition managers: " + deletedPartitions.toString()); 111 | for (Partition id : deletedPartitions) { 112 | PartitionManager man = _managers.remove(id); 113 | man.close(); 114 | } 115 | LOG.info("New partition managers: " + newPartitions.toString()); 116 | for (Partition id : newPartitions) { 117 | PartitionManager man = 118 | new PartitionManager( 119 | _connections, new ZkState( 120 | (String) _config._stateConf 121 | .get(Config.ZOOKEEPER_CONSUMER_CONNECTION)), _kafkaconfig, 122 | id, _receiver, _restart, _messageHandler); 123 | _managers.put(id, man); 124 | } 125 | 126 | _cachedList = new ArrayList(_managers.values()); 127 | LOG.info("Finished refreshing"); 128 | } 129 | 130 | } catch (Exception e) { 131 | throw new FailedFetchException(e); 132 | } 133 | } 134 | 135 | @Override 136 | public PartitionManager getManager(Partition partition) { 137 | return _managers.get(partition); 138 | } 139 | } 140 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/ZkState.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.apache.curator.framework.CuratorFrameworkFactory; 29 | import org.apache.curator.retry.RetryNTimes; 30 | import org.apache.zookeeper.CreateMode; 31 | import org.json.simple.JSONValue; 32 | import org.slf4j.Logger; 33 | import org.slf4j.LoggerFactory; 34 | 35 | import java.io.Serializable; 36 | import java.nio.charset.Charset; 37 | import java.util.ArrayList; 38 | import java.util.Arrays; 39 | import java.util.List; 40 | import java.util.Map; 41 | 42 | @SuppressWarnings("serial") 43 | public class ZkState implements Serializable { 44 | public static final Logger LOG = LoggerFactory.getLogger(ZkState.class); 45 | private transient CuratorFramework _curator; 46 | 47 | private CuratorFramework newCurator(Map stateConf) throws Exception { 48 | Integer port = Integer.parseInt((String) stateConf.get(Config.ZOOKEEPER_PORT)); 49 | String serverPorts = ""; 50 | List zkServers = new ArrayList( 51 | Arrays.asList(((String) stateConf.get(Config.ZOOKEEPER_HOSTS)).split(","))); 52 | for (String server : zkServers) { 53 | serverPorts = serverPorts + server + ":" + port + ","; 54 | } 55 | return CuratorFrameworkFactory.newClient( 56 | serverPorts, 57 | 120000, 58 | 120000, 59 | new RetryNTimes(5, 1000)); 60 | } 61 | 62 | public CuratorFramework getCurator() { 63 | assert _curator != null; 64 | return _curator; 65 | } 66 | 67 | public ZkState(KafkaConfig config) { 68 | try { 69 | _curator = newCurator(config._stateConf); 70 | LOG.info("Starting curator service"); 71 | _curator.start(); 72 | } catch (Exception e) { 73 | LOG.error("Curator service not started"); 74 | throw new RuntimeException(e); 75 | } 76 | } 77 | 78 | public ZkState(String connectionStr) { 79 | try { 80 | _curator = CuratorFrameworkFactory.newClient( 81 | connectionStr, 82 | 120000, 83 | 120000, 84 | new RetryNTimes(5, 1000)); 85 | LOG.info("Starting curator service"); 86 | _curator.start(); 87 | } catch (Exception e) { 88 | LOG.error("Curator service not started"); 89 | throw new RuntimeException(e); 90 | } 91 | } 92 | 93 | public synchronized void writeJSON(String path, Map data) { 94 | LOG.debug("Writing to path : {} json: {}", path, data.toString()); 95 | writeBytes(path, JSONValue.toJSONString(data).getBytes(Charset.forName("UTF-8"))); 96 | } 97 | 98 | public void writeBytes(String path, byte[] bytes) { 99 | try { 100 | if (_curator.checkExists().forPath(path) == null) { 101 | _curator.create().creatingParentsIfNeeded() 102 | .withMode(CreateMode.PERSISTENT).forPath(path, bytes); 103 | } else { 104 | _curator.setData().forPath(path, bytes); 105 | } 106 | } catch (Exception e) { 107 | throw new RuntimeException(e); 108 | } 109 | } 110 | 111 | @SuppressWarnings("unchecked") 112 | public Map readJSON(String path) { 113 | try { 114 | byte[] b = readBytes(path); 115 | if (b == null) { 116 | return null; 117 | } 118 | return (Map) JSONValue.parse(new String(b, "UTF-8")); 119 | } catch (Exception e) { 120 | throw new RuntimeException(e); 121 | } 122 | } 123 | 124 | public byte[] readBytes(String path) { 125 | try { 126 | if (_curator.checkExists().forPath(path) != null) { 127 | return _curator.getData().forPath(path); 128 | } else { 129 | return null; 130 | } 131 | } catch (Exception e) { 132 | throw new RuntimeException(e); 133 | } 134 | } 135 | 136 | public void close() { 137 | _curator.close(); 138 | _curator = null; 139 | } 140 | } 141 | -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/client/KafkaRangeReceiver.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 | package consumer.kafka.client; 20 | 21 | import consumer.kafka.*; 22 | import org.apache.spark.storage.StorageLevel; 23 | import org.apache.spark.streaming.receiver.Receiver; 24 | import org.slf4j.Logger; 25 | import org.slf4j.LoggerFactory; 26 | 27 | import java.io.Serializable; 28 | import java.util.ArrayList; 29 | import java.util.List; 30 | import java.util.Properties; 31 | import java.util.Set; 32 | 33 | @SuppressWarnings("serial") 34 | public class KafkaRangeReceiver extends Receiver> { 35 | 36 | private static final Logger LOG = LoggerFactory.getLogger(KafkaRangeReceiver.class); 37 | private final Properties _props; 38 | private Set _partitionSet; 39 | private KafkaConsumer _kConsumer; 40 | private transient Thread _consumerThread; 41 | private List _threadList = new ArrayList(); 42 | private int maxRestartAttempts; 43 | private int restartAttempts; 44 | private KafkaMessageHandler _messageHandler; 45 | 46 | public KafkaRangeReceiver(Properties props, 47 | Set partitionSet, 48 | KafkaMessageHandler messageHandler) { 49 | this(props, partitionSet, StorageLevel.MEMORY_ONLY(), messageHandler); 50 | } 51 | 52 | public KafkaRangeReceiver( 53 | Properties props, 54 | Set partitionSet, 55 | StorageLevel storageLevel, 56 | KafkaMessageHandler messageHandler) { 57 | super(storageLevel); 58 | this._props = props; 59 | _partitionSet = partitionSet; 60 | _messageHandler = messageHandler; 61 | } 62 | 63 | @Override 64 | public void onStart() { 65 | try { 66 | start(); 67 | } catch (CloneNotSupportedException e) { 68 | LOG.error("Error while starting Receiver", e); 69 | } 70 | } 71 | 72 | public void start() throws CloneNotSupportedException { 73 | // Start the thread that receives data over a connection 74 | _threadList.clear(); 75 | KafkaConfig kafkaConfig = new KafkaConfig(_props); 76 | _messageHandler.init(); 77 | maxRestartAttempts = kafkaConfig._maxRestartAttempts; 78 | restartAttempts = restartAttempts + 1; 79 | for (Integer partitionId : _partitionSet) { 80 | 81 | ZkState zkState = new ZkState(kafkaConfig); 82 | _kConsumer = new KafkaConsumer(kafkaConfig, zkState, this, (KafkaMessageHandler) _messageHandler.clone()); 83 | _kConsumer.open(partitionId); 84 | 85 | Thread.UncaughtExceptionHandler eh = 86 | new Thread.UncaughtExceptionHandler() { 87 | public void uncaughtException(Thread th, Throwable ex) { 88 | LOG.error("Receiver got Uncaught Exception " + ex.getMessage() 89 | + " for Partition " + partitionId); 90 | if (ex instanceof InterruptedException) { 91 | LOG.error("Stopping Receiver for partition " + partitionId); 92 | th.interrupt(); 93 | stop(" Stopping Receiver due to " + ex); 94 | } else { 95 | if (maxRestartAttempts < 0 || restartAttempts < maxRestartAttempts) { 96 | LOG.error("Restarting Receiver in 5 Sec for Partition " + 97 | partitionId + " . restart attempt " + restartAttempts); 98 | restart("Restarting Receiver for Partition " + partitionId, ex, 5000); 99 | } else { 100 | LOG.error("tried maximum configured restart attemps " + 101 | maxRestartAttempts + " shutting down receiver"); 102 | stop(" Stopping Receiver for partition " 103 | + partitionId + ". Max restart attempt exhausted"); 104 | } 105 | } 106 | } 107 | }; 108 | 109 | _consumerThread = new Thread(_kConsumer); 110 | _consumerThread.setUncaughtExceptionHandler(eh); 111 | _threadList.add(_consumerThread); 112 | _consumerThread.start(); 113 | } 114 | } 115 | 116 | @Override 117 | public void onStop() { 118 | for (Thread t : _threadList) { 119 | if (t.isAlive()) 120 | t.interrupt(); 121 | } 122 | } 123 | } -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/client/KafkaReceiver.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 | package consumer.kafka.client; 20 | 21 | import consumer.kafka.*; 22 | import org.apache.spark.storage.StorageLevel; 23 | import org.apache.spark.streaming.receiver.Receiver; 24 | import org.slf4j.Logger; 25 | import org.slf4j.LoggerFactory; 26 | 27 | import java.io.Serializable; 28 | import java.util.Properties; 29 | 30 | 31 | @SuppressWarnings("serial") 32 | public class KafkaReceiver extends Receiver> { 33 | 34 | private static final Logger LOG = LoggerFactory.getLogger(KafkaReceiver.class); 35 | private final Properties _props; 36 | private int _partitionId; 37 | private KafkaConsumer _kConsumer; 38 | private transient Thread _consumerThread; 39 | private int maxRestartAttempts; 40 | private int restartAttempts; 41 | private KafkaMessageHandler _messageHandler; 42 | 43 | public KafkaReceiver(Properties props, 44 | int partitionId, 45 | KafkaMessageHandler messageHandler) { 46 | this(props, partitionId, StorageLevel.MEMORY_ONLY(), messageHandler); 47 | } 48 | 49 | public KafkaReceiver( 50 | Properties props, 51 | int partitionId, 52 | StorageLevel storageLevel, 53 | KafkaMessageHandler messageHandler) { 54 | super(storageLevel); 55 | this._props = props; 56 | this._partitionId = partitionId; 57 | this._messageHandler = messageHandler; 58 | } 59 | 60 | @Override 61 | public void onStart() { 62 | start(); 63 | } 64 | 65 | public void start() { 66 | // Start the thread that receives data over a connection 67 | KafkaConfig kafkaConfig = new KafkaConfig(_props); 68 | _messageHandler.init(); 69 | maxRestartAttempts = kafkaConfig._maxRestartAttempts; 70 | restartAttempts = restartAttempts + 1; 71 | ZkState zkState = new ZkState(kafkaConfig); 72 | _kConsumer = new KafkaConsumer(kafkaConfig, zkState, this, _messageHandler); 73 | _kConsumer.open(_partitionId); 74 | 75 | Thread.UncaughtExceptionHandler eh = new Thread.UncaughtExceptionHandler() { 76 | 77 | public void uncaughtException(Thread th, Throwable ex) { 78 | 79 | LOG.error("Receiver got Uncaught Exception " + ex.getMessage() 80 | + " for Partition " + _partitionId); 81 | if (ex instanceof InterruptedException) { 82 | LOG.error("Stopping Receiver for partition " + _partitionId); 83 | th.interrupt(); 84 | stop(" Stopping Receiver for partition " 85 | + _partitionId + " due to " + ex); 86 | 87 | } else { 88 | if (maxRestartAttempts < 0 || restartAttempts < maxRestartAttempts) { 89 | LOG.error("Restarting Receiver in 5 Sec for Partition " + 90 | _partitionId + " . restart attempt " + restartAttempts); 91 | restart("Restarting Receiver for Partition " + _partitionId, ex, 5000); 92 | } else { 93 | LOG.error("tried maximum configured restart attemps " + 94 | maxRestartAttempts + " shutting down receiver"); 95 | stop(" Stopping Receiver for partition " 96 | + _partitionId + ". Max restart attempt exhausted"); 97 | } 98 | } 99 | } 100 | }; 101 | _consumerThread = new Thread(_kConsumer); 102 | _consumerThread.setUncaughtExceptionHandler(eh); 103 | _consumerThread.start(); 104 | } 105 | 106 | @Override 107 | public void onStop() { 108 | if (_consumerThread.isAlive()) { 109 | _consumerThread.interrupt(); 110 | } 111 | } 112 | } -------------------------------------------------------------------------------- /src/main/scala/consumer/kafka/client/SampleConsumer.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 | package consumer.kafka.client; 20 | 21 | import consumer.kafka.MessageAndMetadata; 22 | import consumer.kafka.ProcessedOffsetManager; 23 | import consumer.kafka.ReceiverLauncher; 24 | import org.apache.spark.SparkConf; 25 | import org.apache.spark.api.java.JavaRDD; 26 | import org.apache.spark.api.java.function.VoidFunction; 27 | import org.apache.spark.storage.StorageLevel; 28 | import org.apache.spark.streaming.Durations; 29 | import org.apache.spark.streaming.api.java.JavaDStream; 30 | import org.apache.spark.streaming.api.java.JavaPairDStream; 31 | import org.apache.spark.streaming.api.java.JavaStreamingContext; 32 | 33 | import java.io.Serializable; 34 | import java.util.List; 35 | import java.util.Properties; 36 | 37 | @SuppressWarnings("serial") 38 | public class SampleConsumer implements Serializable { 39 | 40 | public void start() throws InstantiationException, IllegalAccessException, 41 | ClassNotFoundException { 42 | run(); 43 | } 44 | 45 | @SuppressWarnings("deprecation") 46 | private void run() { 47 | 48 | Properties props = new Properties(); 49 | props.put("zookeeper.hosts", "localhost"); 50 | props.put("zookeeper.port", "2181"); 51 | props.put("kafka.topic", "mytopic"); 52 | props.put("kafka.consumer.id", "kafka-consumer"); 53 | props.put("zookeeper.consumer.connection", "localhost:2181"); 54 | // Optional Properties 55 | props.put("consumer.forcefromstart", "false"); 56 | props.put("consumer.fetchsizebytes", "1048576"); 57 | props.put("consumer.fillfreqms", "1000"); 58 | props.put("consumer.backpressure.enabled", "true"); 59 | props.put("consumer.num_fetch_to_buffer", "1"); 60 | 61 | SparkConf _sparkConf = new SparkConf(); 62 | JavaStreamingContext jsc = new JavaStreamingContext(_sparkConf, Durations.seconds(30)); 63 | // Specify number of Receivers you need. 64 | int numberOfReceivers = 1; 65 | 66 | JavaDStream> unionStreams = ReceiverLauncher.launch( 67 | jsc, props, numberOfReceivers, StorageLevel.MEMORY_ONLY()); 68 | 69 | //Get the Max offset from each RDD Partitions. Each RDD Partition belongs to One Kafka Partition 70 | JavaPairDStream> partitonOffset = ProcessedOffsetManager 71 | .getPartitionOffset(unionStreams, props); 72 | 73 | unionStreams.foreachRDD(new VoidFunction>>() { 74 | @Override 75 | public void call(JavaRDD> rdd) throws Exception { 76 | List> rddList = rdd.collect(); 77 | System.out.println(" Number of records in this batch " + rddList.size()); 78 | } 79 | }); 80 | //End Application Logic 81 | 82 | //Persists the Max Offset of given Kafka Partition to ZK 83 | ProcessedOffsetManager.persists(partitonOffset, props); 84 | 85 | try { 86 | jsc.start(); 87 | jsc.awaitTermination(); 88 | }catch (Exception ex ) { 89 | jsc.ssc().sc().cancelAllJobs(); 90 | jsc.stop(true, false); 91 | System.exit(-1); 92 | } 93 | } 94 | 95 | public static void main(String[] args) throws Exception { 96 | SampleConsumer consumer = new SampleConsumer(); 97 | consumer.start(); 98 | } 99 | } 100 | -------------------------------------------------------------------------------- /src/test/scala/com/hb/producer/KafkaProducerSimulator.scala: -------------------------------------------------------------------------------- 1 | package com.hb.producer 2 | 3 | /** 4 | * Created by Simon on 2017/2/24. 5 | */ 6 | import java.util.HashMap 7 | 8 | import org.apache.kafka.clients.producer.KafkaProducer 9 | import org.apache.kafka.clients.producer.ProducerConfig 10 | import org.apache.kafka.clients.producer.ProducerRecord 11 | 12 | import scala.util.Random 13 | 14 | 15 | /** 16 | * 模拟flume将日志传输到kafka,测试使用 17 | */ 18 | 19 | 20 | object KafkaProducerSimulator { 21 | 22 | def getRandomString(length : Int){ 23 | val str="abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789" 24 | val random=new Random() 25 | val sb=new StringBuffer() 26 | for (i <- (0 until length)){ 27 | val number = random.nextInt(62) 28 | sb.append(str.charAt(number)) 29 | } 30 | sb.toString() 31 | } 32 | 33 | def main(args: Array[String]) { 34 | 35 | val Array(brokers, topic, messagesPerSec) = Array("192.168.2.245:9092", "nginx", "100") 36 | 37 | // Zookeeper connection properties 38 | val props = new HashMap[String, Object]() 39 | props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokers) 40 | props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, 41 | "org.apache.kafka.common.serialization.StringSerializer") 42 | props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, 43 | "org.apache.kafka.common.serialization.StringSerializer") 44 | 45 | val producer = new KafkaProducer[String, String](props) 46 | 47 | // Send some messages 48 | while (true) { 49 | (1 to messagesPerSec.toInt).foreach { messageNum => 50 | val appendString = getRandomString(3) 51 | val random = Math.random() 52 | val str = "183.0.227.212 ^^A 05/Apr/2017:17:12:59 +0800 ^^A GET /api/Message/GetUserMessageNotify HTTP/1.1 ^^A - ^^A 404 ^^A 166 ^^A - ^^A 7D9A692FA70A158C84E4FF61BC8A2490 ^^A iPhone6plus ^^A - ^^A deviceid/d41d8cd98f00b204e9800998ecf8427e4e8fe389 os/iOS manufacturer/Apple appversion/3.9.7 ip/192.168.9.102 systemversion/9.3.5 idfa/F6F2FBD8-17AA-4FCF-BD15-4AF2C8B5336E signalType/4 beta/0 ^^A - ^^A 0.163 ^^A 0.163" 53 | val message = new ProducerRecord[String, String](topic, null, str) 54 | producer.send(message) 55 | } 56 | Thread.sleep(1000) 57 | } 58 | 59 | } 60 | } --------------------------------------------------------------------------------