@@ -3,52 +3,90 @@ import java.time.LocalDateTime
3
3
import java .time .format .DateTimeParseException
4
4
import java .time .temporal .ChronoUnit
5
5
import org .apache .spark ._
6
+ import org .apache .spark .rdd .RDD
6
7
import org .apache .spark .streaming ._
7
8
import org .apache .spark .streaming .kafka010 ._
8
9
import org .apache .kafka .common .serialization .StringDeserializer
10
+ import org .apache .kafka .common .TopicPartition
11
+ import org .apache .kafka .clients .consumer .ConsumerRecord
9
12
import scalikejdbc ._
10
13
11
14
12
15
object ExactlyOnce {
13
16
def main (args : Array [String ]): Unit = {
17
+ val brokers = " localhost:9092"
18
+ val topic = " alog"
19
+
14
20
val kafkaParams = Map [String , Object ](
15
- " bootstrap.servers" -> " localhost:9092 " ,
21
+ " bootstrap.servers" -> brokers ,
16
22
" key.deserializer" -> classOf [StringDeserializer ],
17
23
" value.deserializer" -> classOf [StringDeserializer ],
18
24
" group.id" -> " exactly-once" ,
19
- " auto.offset.reset" -> " latest" )
25
+ " enable.auto.commit" -> (false : java.lang.Boolean ),
26
+ " auto.offset.reset" -> " none" )
27
+
28
+ ConnectionPool .singleton(" jdbc:mysql://localhost:3306/spark" , " root" , " " )
20
29
21
30
val conf = new SparkConf ().setAppName(" ExactlyOnce" ).setIfMissing(" spark.master" , " local[2]" )
22
31
val ssc = new StreamingContext (conf, Seconds (5 ))
23
32
33
+ val fromOffsets = DB .readOnly { implicit session =>
34
+ sql """
35
+ select `partition`, offset from kafka_offset
36
+ where topic = ${topic}
37
+ """ .map { rs =>
38
+ new TopicPartition (topic, rs.int(" partition" )) -> rs.long(" offset" )
39
+ }.list.apply().toMap
40
+ }
41
+
24
42
val messages = KafkaUtils .createDirectStream[String , String ](ssc,
25
43
LocationStrategies .PreferConsistent ,
26
- ConsumerStrategies .Subscribe [String , String ](Seq ( " alog " ) , kafkaParams))
44
+ ConsumerStrategies .Assign [String , String ](fromOffsets.keys , kafkaParams, fromOffsets ))
27
45
28
- messages.map(_.value)
29
- .flatMap(parseLog)
30
- .filter(_.level == " ERROR" )
31
- .map(log => log.time.truncatedTo(ChronoUnit .MINUTES ) -> 1 )
32
- .reduceByKey(_ + _)
33
- .foreachRDD { (rdd, time) =>
34
- rdd.foreachPartition { iter =>
35
- ConnectionPool .singleton(" jdbc:mysql://localhost:3306/spark" , " root" , " " )
36
- iter.foreach { case (time, count) =>
37
- DB .autoCommit { implicit session =>
38
- sql """
39
- insert into error_log (log_time, log_count)
40
- value ( ${time}, ${count})
41
- on duplicate key update log_count = log_count + values(log_count)
42
- """ .update.apply()
43
- }
46
+ messages.foreachRDD { rdd =>
47
+ val offsetRanges = rdd.asInstanceOf [HasOffsetRanges ].offsetRanges
48
+ val result = processLogs(rdd).collect()
49
+
50
+ DB .localTx { implicit session =>
51
+ result.foreach { case (time, count) =>
52
+ sql """
53
+ insert into error_log (log_time, log_count)
54
+ value ( ${time}, ${count})
55
+ on duplicate key update log_count = log_count + values(log_count)
56
+ """ .update.apply()
57
+ }
58
+
59
+ offsetRanges.foreach { offsetRange =>
60
+ sql """
61
+ insert ignore into kafka_offset (topic, `partition`, offset)
62
+ value ( ${topic}, ${offsetRange.partition}, ${offsetRange.fromOffset})
63
+ """ .update.apply()
64
+
65
+ val affectedRows = sql """
66
+ update kafka_offset set offset = ${offsetRange.untilOffset}
67
+ where topic = ${topic} and `partition` = ${offsetRange.partition}
68
+ and offset = ${offsetRange.fromOffset}
69
+ """ .update.apply()
70
+
71
+ if (affectedRows != 1 ) {
72
+ throw new Exception (" fail to update offset" )
44
73
}
45
74
}
46
75
}
76
+ }
47
77
48
78
ssc.start()
49
79
ssc.awaitTermination()
50
80
}
51
81
82
+ def processLogs (messages : RDD [ConsumerRecord [String , String ]]): RDD [(LocalDateTime , Int )] = {
83
+ messages.map(_.value)
84
+ .flatMap(parseLog)
85
+ .filter(_.level == " ERROR" )
86
+ .map(log => log.time.truncatedTo(ChronoUnit .MINUTES ) -> 1 )
87
+ .reduceByKey(_ + _)
88
+ }
89
+
52
90
case class Log (time : LocalDateTime , level : String )
53
91
54
92
val logPattern = " ^(.{19}) ([A-Z]+).*" .r
0 commit comments