1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
package mobvista.dmp.datasource.id_mapping
import com.alibaba.fastjson.JSONObject
import mobvista.dmp.common.MobvistaConstant.{sdf1, sdf2}
import mobvista.dmp.common.{CommonSparkJob, MobvistaConstant}
import mobvista.dmp.datasource.id_mapping.Constant._
import mobvista.dmp.util.DateUtil
import mobvista.dmp.utils.common.MD5Util.hashMD5
import org.apache.commons.cli.{BasicParser, Options}
import org.apache.commons.lang3.StringUtils
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.io.compress.GzipCodec
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{Row, SaveMode, SparkSession}
import org.apache.spark.storage.StorageLevel
import java.net.URI
import java.text.SimpleDateFormat
import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer
/**
* @package: mobvista.dmp.datasource.id_mapping
* @author: wangjf
* @date: 2021/12/7
* @time: 2:39 下午
* @email: jinfeng.wang@mobvista.com
*/
class IDMappingGraphx extends CommonSparkJob with Serializable {
def commandOptions(): Options = {
val options = new Options()
options.addOption("country", true, "country")
options.addOption("platform", true, "platform")
options.addOption("date", true, "date")
options.addOption("output", true, "output")
options.addOption("coalesce", true, "coalesce")
options.addOption("result_output", true, "result_output")
options
}
override protected def run(args: Array[String]): Int = {
val parser = new BasicParser()
val options = commandOptions()
val commandLine = parser.parse(options, args)
val country = commandLine.getOptionValue("country")
val platform = commandLine.getOptionValue("platform")
val date = commandLine.getOptionValue("date")
val output = commandLine.getOptionValue("output")
val result_output = commandLine.getOptionValue("result_output")
val coalesce = Integer.parseInt(commandLine.getOptionValue("coalesce"))
val spark = MobvistaConstant.createSparkSession(s"IDMappingGraphx.$date.$country.$platform")
try {
oldAndTodayIdMapping(country.toUpperCase, platform, date, spark, output, result_output, coalesce)
} finally {
if (spark != null) {
spark.stop()
}
}
0
}
def oldAndTodayIdMapping(country: String, platform: String, date: String, spark: SparkSession, outPutPath: String,
resultOutPutPath: String, coalesce: Int) = {
var dailySQL = ""
var schame: StructType = null
var idSet: Array[String] = null
var idMainSet: Set[String] = null
var scoreMap: Map[String, Double] = null
// 1.今日数据加载
platform match {
case "ios" =>
schame = iosVertSchema
idSet = iosIDSet
idMainSet = iosMainIDSet
scoreMap = iosIDScoreMap
country match {
case "CN" =>
dailySQL = Constant.ios_id_mapping_sql_v2.replace("@date", date).replace("@filter_country", s"AND country = '${country}'")
case _ =>
dailySQL = Constant.ios_id_mapping_sql_v2.replace("@date", date).replace("@filter_country", s"")
}
case "android" => {
scoreMap = androidIDScoreMap
country match {
case "CN" =>
idMainSet = androidCNMainIDSet
schame = adrCNVertSchema
idSet = androidCNIDSet
dailySQL = Constant.android_id_mapping_sql_v2.replace("@date", date).replace("@filter_country", s"AND country = '${country}'")
case _ =>
idMainSet = androidMainIDSet
schame = adrVertSchema
idSet = androidIDSet
dailySQL = Constant.android_id_mapping_sql_v2.replace("@date", date).replace("@filter_country", s"")
}
}
case _ =>
""
}
val df = spark.sql(dailySQL)
/*
FileSystem.get(new URI(s"s3://mob-emr-test"), spark.sparkContext.hadoopConfiguration).delete(new Path(outPutPath), true)
df.persist(StorageLevel.MEMORY_AND_DISK_SER)
df.repartition(coalesce)
.write.mode(SaveMode.Overwrite)
.option("orc.compress", "zlib")
.orc(frequencyOutPutPath)
val fre_table = platform match {
case "ios" =>
"dws_device_id_ios_frequency"
case _ =>
"dws_device_id_android_frequency"
}
spark.sql(
s"""
|ALTER TABLE dws.$fre_table ADD IF NOT EXISTS PARTITION (dt='$date',source='${country.toLowerCase}')
| LOCATION '$frequencyOutPutPath'
|""".stripMargin)
*/
val todayDF = spark.createDataFrame(df.rdd.map(row => {
processData(row, platform)
}), schema = schame)
val schedule_date = sdf1.format(sdf2.parse(date))
val vertex = todayDF.rdd.map(row => {
processVertex(schedule_date, row, idSet, idMainSet)
}).flatMap(l => l)
vertex.persist(StorageLevel.MEMORY_AND_DISK_SER)
val maxGraphFilter = vertex.map(l => {
(l._1, 1)
}).groupByKey().map(l => {
(l._1, l._2.size)
}).filter(l => {
l._2 > 1000
})
maxGraphFilter.cache()
val maxGraph = vertex.leftOuterJoin(maxGraphFilter)
.mapPartitions(kvs => {
kvs.map(kv => {
val key = kv._1
val value = kv._2
if (value._2.isEmpty) {
(key, value._1)
} else {
null
}
})
}).filter(line => {
line != null
}).combineByKey(
(v: (String, String, Long)) => Iterable(v),
(c: Iterable[(String, String, Long)], v: (String, String, Long)) => c ++ Seq(v),
(c1: Iterable[(String, String, Long)], c2: Iterable[(String, String, Long)]) => c1 ++ c2
)
// 非主ID生成OneID
val multiOneIDRDD = maxGraph.filter(kv => {
kv._2.size > 1
}).mapPartitions(rs => new CustomInterator(schedule_date, rs, idSet, idMainSet))
.flatMap(l => l)
// 主ID生成OneID
val singleOneIDRDD = maxGraph.filter(kv => {
kv._2.size == 1
}).map(kv => {
val oneID = new JSONObject()
kv._2.foreach(t => {
val json = new JSONObject()
json.put("one_type", t._2)
json.put("one_date", schedule_date)
json.put("one_cnt", t._3)
oneID.put(t._1, json)
})
(kv._1, (oneID.toJSONString, schedule_date))
})
val yesDate = DateUtil.getDayByString(date, "yyyyMMdd", -1)
val updateDate = sdf1.format(sdf2.parse(DateUtil.getDayByString(date, "yyyyMMdd", -7)))
val oldMidMergeOneIDRDD = spark.sql(
s"""
|SELECT device_id, device_type, one_id, update_date
| FROM ads.ads_device_id_mapping WHERE dt = '$yesDate' AND source = '${country.toLowerCase}' AND platform = '$platform' AND `type` = 'mid'
|""".stripMargin)
.rdd
.map(row => {
((row.getAs[String]("device_id"), row.getAs[String]("device_type")), (row.getAs[String]("one_id"), row.getAs[String]("update_date")))
}).filter(rs => {
filterAction(rs._1._2, idMainSet) || (!filterAction(rs._1._2, idMainSet) && rs._2._2.compareTo(updateDate) >= 0)
})
val midMergeOneIDRDD = spark.sparkContext.union(Seq(singleOneIDRDD, multiOneIDRDD, oldMidMergeOneIDRDD))
.combineByKey(
(v: (String, String)) => Iterable(v),
(c: Iterable[(String, String)], v: (String, String)) => c ++ Seq(v),
(c1: Iterable[(String, String)], c2: Iterable[(String, String)]) => c1 ++ c2
).map(kv => {
val srcId = kv._1._1
val srcType = kv._1._2
var update_date = ""
val oneIDJSON = new JSONObject()
kv._2.foreach(ou => {
val json = MobvistaConstant.String2JSONObject(ou._1)
val keys = json.keySet().asScala
keys.foreach(key => {
if (oneIDJSON.containsKey(key) && oneIDJSON.getJSONObject(key).getString("one_date")
.compareTo(json.getJSONObject(key).getString("one_date")) < 0
|| !oneIDJSON.containsKey(key)) {
oneIDJSON.put(key, json.getJSONObject(key))
}
})
if (StringUtils.isBlank(update_date) || update_date.compareTo(ou._2) < 0) {
update_date = ou._2
}
})
Result(srcId, srcType, oneIDJSON.toJSONString, update_date)
})
import spark.implicits._
FileSystem.get(new URI(s"s3://mob-emr-test"), spark.sparkContext.hadoopConfiguration).delete(new Path(outPutPath), true)
midMergeOneIDRDD.toDF
.coalesce(coalesce)
.write.mode(SaveMode.Overwrite)
.option("orc.compress", "zlib")
.orc(outPutPath)
spark.sql(
s"""
|ALTER TABLE ads.ads_device_id_mapping ADD IF NOT EXISTS PARTITION (dt='$date',source='${country.toLowerCase}',platform='$platform',`type`='mid')
| LOCATION '$outPutPath'
|""".stripMargin)
val resultOneID = midMergeOneIDRDD.mapPartitions(rs => {
rs.map(r => {
val device_id = r.device_id
val device_type = r.device_type
val one_id = MobvistaConstant.String2JSONObject(r.one_id)
val update_date = r.update_date
val keys = one_id.keySet().asScala
var oneIDScore: OneIDScore = OneIDScore("", "", 0, "")
keys.foreach(key => {
val sdf = new SimpleDateFormat("yyyy-MM-dd")
val json = one_id.getJSONObject(key)
val id_type = json.getString("one_type")
val id_type_score = scoreMap(id_type)
val active_date = json.getString("one_date")
val cnt = json.getLongValue("one_cnt")
val days = (sdf.parse(schedule_date).getTime - sdf.parse(active_date).getTime) / 1000 / 3600 / 24 + 1
val score = id_type_score * 30 / days + 0.1 * cnt
if (idSet.indexOf(id_type) < idSet.indexOf(oneIDScore.one_type) || idSet.indexOf(oneIDScore.one_type) == -1
|| (idSet.indexOf(id_type) == idSet.indexOf(oneIDScore.one_type) && score >= oneIDScore.one_score)) {
oneIDScore = OneIDScore(key, id_type, score, active_date)
}
})
val json = new JSONObject()
json.put("one_id", oneIDScore.one_id)
json.put("type", oneIDScore.one_type)
json.put("score", oneIDScore.one_score)
json.put("version", oneIDScore.one_version)
Result(device_id, device_type, json.toJSONString, update_date)
})
})
/*
FileSystem.get(new URI(s"s3://mob-emr-test"), spark.sparkContext.hadoopConfiguration).delete(new Path(resultOutPutPath), true)
resultOneID
.toDF
.repartition(coalesce)
.write.mode(SaveMode.Overwrite)
.option("orc.compress", "zlib")
.orc(resultOutPutPath)
spark.sql(
s"""
|ALTER TABLE ads.ads_device_id_mapping ADD IF NOT EXISTS PARTITION (dt='$date',source='${country.toLowerCase}',platform='$platform',`type`='result')
| LOCATION '$resultOutPutPath'
|""".stripMargin)
*/
}
def filterAction(device_type: String, mainIDSet: Set[String]): Boolean = {
mainIDSet.contains(device_type)
}
def processData(row: Row, platform: String): Row = {
platform match {
case "ios" =>
var idfa = row.getAs[String]("idfa")
idfa = if (StringUtils.isNotBlank(idfa) && (idfa.matches(didPtn) && !idfa.matches(allZero) || idfa.matches(md5Ptn))) {
idfa
} else {
""
}
var idfv = row.getAs[String]("idfv")
idfv = if (StringUtils.isNotBlank(idfv) && (idfv.matches(didPtn) && !idfv.matches(allZero) || idfv.matches(md5Ptn))) {
idfv
} else {
""
}
val pkg_name = row.getAs[String]("pkg_name")
var sysid = row.getAs[String]("sysid")
sysid = if (StringUtils.isNotBlank(sysid) && (sysid.matches(didPtn) && !sysid.matches(allZero) || sysid.matches(md5Ptn))) {
sysid
} else {
""
}
var bkupid = row.getAs[String]("bkupid")
bkupid = if (StringUtils.isNotBlank(bkupid) && (bkupid.matches(didPtn) && !bkupid.matches(allZero) || bkupid.matches(md5Ptn))) {
bkupid
} else {
""
}
val xwho = row.getAs[String]("xwho")
val user_id = row.getAs[String]("user_id")
// val country = row.getAs[String]("country")
val ip = row.getAs[String]("ip")
val ua = row.getAs[String]("ua")
val brand = row.getAs[String]("brand")
val model = row.getAs[String]("model")
val os_version = row.getAs[String]("os_version")
val osv_upt = row.getAs[String]("osv_upt")
val upt = row.getAs[String]("upt")
val cnt = row.getAs[Long]("cnt")
val idfv_bundle = if (StringUtils.isNotBlank(idfv)) {
hashMD5(idfv + pkg_name)
} else {
""
}
val bmosv_osv_upt = if (StringUtils.isNotBlank(osv_upt)) {
hashMD5(brand + model + os_version + osv_upt)
} else {
""
}
val bmosv_upt = if (StringUtils.isNotBlank(upt)) {
hashMD5(brand + model + os_version + upt)
} else {
""
}
val bmosv_ipua_bundle = if (StringUtils.isNotBlank(ip)) {
hashMD5(brand + model + os_version + ip + ua + pkg_name)
} else {
""
}
// IosVert(idfa, sysid, idfv_bundle, bmosv_osv_upt, bmosv_upt, bmosv_ipua_bundle, xwho, user_id, bkupid, cnt)
Row(idfa, sysid, idfv_bundle, bmosv_osv_upt, bmosv_upt, bmosv_ipua_bundle, xwho, user_id, bkupid, cnt)
case "android" =>
var imei = row.getAs[String]("imei")
imei = if (StringUtils.isNotBlank(imei) && (imei.matches(imeiPtn) && !imei.matches(imeiPtnAll) || imei.matches(md5Ptn))) {
imei
} else {
""
}
var android_id = row.getAs[String]("android_id")
android_id = if (StringUtils.isNotBlank(android_id) && (android_id.matches(andriodIdPtn) && !android_id.matches(andriodIdAll)
|| android_id.matches(md5Ptn))) {
android_id
} else {
""
}
val pkg_name = row.getAs[String]("pkg_name")
var oaid = row.getAs[String]("oaid")
oaid = if (StringUtils.isNotBlank(oaid) && (oaid.length >= 16 && oaid.length <= 64 && !oaid.matches(allZero) || oaid.matches(md5Ptn))) {
oaid
} else {
""
}
var gaid = row.getAs[String]("gaid")
gaid = if (StringUtils.isNotBlank(gaid) && (gaid.matches(didPtn) && !gaid.matches(allZero) || gaid.matches(md5Ptn))) {
gaid
} else {
""
}
var sysid = row.getAs[String]("sysid")
sysid = if (StringUtils.isNotBlank(sysid) && (sysid.matches(didPtn) && !sysid.matches(allZero) || sysid.matches(md5Ptn))) {
sysid
} else {
""
}
var bkupid = row.getAs[String]("bkupid")
bkupid = if (StringUtils.isNotBlank(bkupid) && (bkupid.matches(didPtn) && !bkupid.matches(allZero) || bkupid.matches(md5Ptn))) {
bkupid
} else {
""
}
val xwho = row.getAs[String]("xwho")
val user_id = row.getAs[String]("user_id")
val country = row.getAs[String]("country")
val ip = row.getAs[String]("ip")
val ua = row.getAs[String]("ua")
val brand = row.getAs[String]("brand")
val model = row.getAs[String]("model")
val os_version = row.getAs[String]("os_version")
val upt = row.getAs[String]("upt")
val cnt = row.getAs[Long]("cnt")
val android_pkg = if (StringUtils.isNotBlank(android_id)) {
hashMD5(android_id + pkg_name)
} else {
""
}
val bmosv_upt = if (StringUtils.isNotBlank(upt)) {
hashMD5(brand + model + os_version + upt)
} else {
""
}
val bmosv_ipua_pkg = if (StringUtils.isNotBlank(ip)) {
hashMD5(brand + model + os_version + ip + ua + pkg_name)
} else {
""
}
// AdrVert(imei, gaid, oaid, sysid, android_pkg, bmosv_upt, bmosv_ipua_pkg, xwho, user_id, bkupid, cnt)
if ("CN".equalsIgnoreCase(country)) {
Row(imei, oaid, gaid, sysid, android_pkg, bmosv_upt, bmosv_ipua_pkg, xwho, user_id, bkupid, cnt)
} else {
Row(gaid, imei, oaid, sysid, android_pkg, bmosv_upt, bmosv_ipua_pkg, xwho, user_id, bkupid, cnt)
}
case _ =>
Row("")
}
}
/**
*
* @param date
* @param row
* @param ids
* @param mainIDSet
* @return
* (srcID,oneID,oneIDJSON,srcType)
* (oneID,srcID,oneIDJSON,srcType)
*/
def processVertex(date: String, row: Row, ids: Array[String], mainIDSet: Set[String]): ArrayBuffer[((String, String), (String, String, Long))] = {
val array = new ArrayBuffer[((String, String), (String, String, Long))]()
// val json = JSON.parseObject(Serialization.write(row))
// 事件频次
val cnt = row.getAs[Long]("cnt")
// date 活跃日期,用于计算权重
var flag = true
for (i <- ids.indices) {
if (StringUtils.isNotBlank(row.getAs[String](String.valueOf(ids(i)))) && flag) {
val oneIDType = ids(i)
val oneID = if (row.getAs[String](String.valueOf(ids(i))).matches(md5Ptn)) {
row.getAs[String](String.valueOf(ids(i)))
} else {
hashMD5(row.getAs[String](String.valueOf(ids(i))))
}
array += (((oneID, oneIDType), (oneID, oneIDType, cnt)))
for (j <- i + 1 until ids.length) {
if (StringUtils.isNotBlank(row.getAs[String](String.valueOf(ids(j))))) {
val srcType = ids(j)
val srcOrg = if (row.getAs[String](srcType).matches(md5Ptn)) {
row.getAs[String](srcType)
} else {
hashMD5(row.getAs[String](srcType))
}
if (mainIDSet.contains(oneIDType)) {
array += (((srcOrg, srcType), (oneID, oneIDType, cnt)))
} else {
array += (((oneID, oneIDType), (srcOrg, srcType, cnt)))
}
}
}
flag = false
}
}
array
}
/**
*
* @param kv
* @param mainIDSet
* @return
* ((srcID,srcType),oneID)
*/
def updateOneID(active_date: String, kv: ((String, String), Set[(String, String, Long)]), idArray: Array[String], mainIDSet: Set[String]): ArrayBuffer[((String, String), String)] = {
val array = new ArrayBuffer[((String, String), String)]()
val tmpOneId = kv._1._1
val tmpOneIdType = kv._1._2
val iters = kv._2
val oneID = new JSONObject()
var minTypeIndex = idArray.indexOf(tmpOneIdType)
iters.foreach(t => {
if (idArray.indexOf(t._2) < minTypeIndex) {
minTypeIndex = idArray.indexOf(t._2)
}
if (tmpOneId.equals(t._1) || mainIDSet.contains(t._2)) {
val json = new JSONObject()
json.put("one_type", t._2)
json.put("one_date", active_date)
json.put("one_cnt", t._3)
oneID.put(t._1, json)
}
})
array += (((tmpOneId, tmpOneIdType), oneID.toJSONString))
if (idArray.indexOf(tmpOneIdType) > minTypeIndex) {
iters.map(itr => {
var oneJSON = new JSONObject()
if (oneID.containsKey(itr._1)) {
oneJSON.put(itr._1, oneID.getJSONObject(itr._1))
} else {
oneJSON = oneID
}
array += (((itr._1, itr._2), oneJSON.toJSONString))
})
}
array
}
}
object IDMappingGraphx {
def main(args: Array[String]): Unit = {
new IDMappingGraphx().run(args)
}
}