forked from vesoft-inc/nebula-exchange
-
Notifications
You must be signed in to change notification settings - Fork 0
/
EdgeProcessor.scala
392 lines (353 loc) · 14.8 KB
/
EdgeProcessor.scala
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
/* Copyright (c) 2020 vesoft inc. All rights reserved.
*
* This source code is licensed under Apache 2.0 License.
*/
package com.vesoft.nebula.exchange.processor
import java.nio.ByteOrder
import com.google.common.geometry.{S2CellId, S2LatLng}
import com.vesoft.exchange.common.{ErrorHandler, GraphProvider, MetaProvider, VidType}
import com.vesoft.exchange.common.{Edge, Edges, KeyPolicy}
import com.vesoft.exchange.common.config.{
Configs,
EdgeConfigEntry,
FileBaseSinkConfigEntry,
SinkCategory
}
import com.vesoft.exchange.common.processor.Processor
import com.vesoft.exchange.common.utils.NebulaUtils
import com.vesoft.exchange.common.utils.NebulaUtils.DEFAULT_EMPTY_VALUE
import com.vesoft.exchange.common.writer.{GenerateSstFile, NebulaGraphClientWriter, NebulaSSTWriter}
import com.vesoft.exchange.common.VidType
import com.vesoft.nebula.encoder.NebulaCodecImpl
import com.vesoft.nebula.meta.EdgeItem
import org.apache.commons.codec.digest.MurmurHash2
import org.apache.log4j.Logger
import org.apache.spark.TaskContext
import org.apache.spark.sql.streaming.Trigger
import org.apache.spark.sql.{DataFrame, Encoders, Row, SparkSession}
import org.apache.spark.util.LongAccumulator
import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer
class EdgeProcessor(spark: SparkSession,
data: DataFrame,
edgeConfig: EdgeConfigEntry,
fieldKeys: List[String],
nebulaKeys: List[String],
config: Configs,
batchSuccess: LongAccumulator,
batchFailure: LongAccumulator)
extends Processor {
@transient
private[this] lazy val LOG = Logger.getLogger(this.getClass)
private[this] val DEFAULT_MIN_CELL_LEVEL = 10
private[this] val DEFAULT_MAX_CELL_LEVEL = 18
private def processEachPartition(iterator: Iterator[Edge]): Unit = {
val graphProvider =
new GraphProvider(config.databaseConfig.getGraphAddress,
config.connectionConfig.timeout,
config.sslConfig)
val writer = new NebulaGraphClientWriter(config.databaseConfig,
config.userConfig,
config.rateConfig,
edgeConfig,
graphProvider)
val errorBuffer = ArrayBuffer[String]()
writer.prepare()
// batch write tags
val startTime = System.currentTimeMillis
iterator.grouped(edgeConfig.batch).foreach { edge =>
val edges = Edges(nebulaKeys, edge.toList, edgeConfig.sourcePolicy, edgeConfig.targetPolicy)
val failStatement = writer.writeEdges(edges)
if (failStatement == null) {
batchSuccess.add(1)
} else {
errorBuffer.append(failStatement)
batchFailure.add(1)
}
}
if (errorBuffer.nonEmpty) {
ErrorHandler.save(
errorBuffer,
s"${config.errorConfig.errorPath}/${edgeConfig.name}.${TaskContext.getPartitionId}")
errorBuffer.clear()
}
LOG.info(s"edge ${edgeConfig.name} import in spark partition ${TaskContext
.getPartitionId()} cost ${System.currentTimeMillis() - startTime}ms")
writer.close()
graphProvider.close()
}
override def process(): Unit = {
val address = config.databaseConfig.getMetaAddress
val space = config.databaseConfig.space
val timeout = config.connectionConfig.timeout
val retry = config.connectionConfig.retry
val metaProvider = new MetaProvider(address, timeout, retry, config.sslConfig)
val fieldTypeMap = NebulaUtils.getDataSourceFieldType(edgeConfig, space, metaProvider)
val isVidStringType = metaProvider.getVidType(space) == VidType.STRING
val partitionNum = metaProvider.getPartNumber(space)
if (edgeConfig.dataSinkConfigEntry.category == SinkCategory.SST) {
val fileBaseConfig = edgeConfig.dataSinkConfigEntry.asInstanceOf[FileBaseSinkConfigEntry]
val namenode = fileBaseConfig.fsName.orNull
val edgeName = edgeConfig.name
val vidType = metaProvider.getVidType(space)
val spaceVidLen = metaProvider.getSpaceVidLen(space)
val edgeItem = metaProvider.getEdgeItem(space, edgeName)
val distintData = if (edgeConfig.rankingField.isDefined) {
data.dropDuplicates(edgeConfig.sourceField,
edgeConfig.targetField,
edgeConfig.rankingField.get)
} else {
data.dropDuplicates(edgeConfig.sourceField, edgeConfig.targetField)
}
var sstKeyValueData = distintData
.mapPartitions { iter =>
iter.map { row =>
encodeEdge(row, partitionNum, vidType, spaceVidLen, edgeItem, fieldTypeMap)
}
}(Encoders.tuple(Encoders.BINARY, Encoders.BINARY, Encoders.BINARY))
.flatMap(line => {
List((line._1, line._3), (line._2, line._3))
})(Encoders.tuple(Encoders.BINARY, Encoders.BINARY))
// repartition dataframe according to nebula part, to make sure sst files for one part has no overlap
if (edgeConfig.repartitionWithNebula) {
sstKeyValueData = customRepartition(spark, sstKeyValueData, partitionNum)
}
sstKeyValueData
.toDF("key", "value")
.sortWithinPartitions("key")
.foreachPartition { iterator: Iterator[Row] =>
val generateSstFile = new GenerateSstFile
generateSstFile.writeSstFiles(iterator,
fileBaseConfig,
partitionNum,
namenode,
batchFailure)
}
} else {
val streamFlag = data.isStreaming
val edgeFrame = data
.filter { row =>
isEdgeValid(row, edgeConfig, streamFlag, isVidStringType)
}
.map { row =>
convertToEdge(row, edgeConfig, isVidStringType, fieldKeys, fieldTypeMap)
}(Encoders.kryo[Edge])
edgeFrame.foreachPartition(processEachPartition _)
}
}
private[this] def indexCells(lat: Double, lng: Double): IndexedSeq[Long] = {
val coordinate = S2LatLng.fromDegrees(lat, lng)
val s2CellId = S2CellId.fromLatLng(coordinate)
for (index <- DEFAULT_MIN_CELL_LEVEL to DEFAULT_MAX_CELL_LEVEL)
yield s2CellId.parent(index).id()
}
/**
* filter and check row data for edge, if streaming only print log
*/
def isEdgeValid(row: Row,
edgeConfig: EdgeConfigEntry,
streamFlag: Boolean,
isVidStringType: Boolean): Boolean = {
val sourceFlag = checkField(edgeConfig.sourceField,
"source_field",
row,
edgeConfig.sourcePolicy,
streamFlag,
isVidStringType)
val targetFlag = checkField(edgeConfig.targetField,
"target_field",
row,
edgeConfig.targetPolicy,
streamFlag,
isVidStringType)
val edgeRankFlag = if (edgeConfig.rankingField.isDefined) {
val index = row.schema.fieldIndex(edgeConfig.rankingField.get)
if (index < 0 || row.isNullAt(index)) {
printChoice(streamFlag, s"rank must exist and cannot be null, your row data is $row")
}
val ranking = row.get(index).toString
if (!NebulaUtils.isNumic(ranking)) {
printChoice(streamFlag,
s"Not support non-Numeric type for ranking field.your row data is $row")
false
} else true
} else true
sourceFlag && targetFlag && edgeRankFlag
}
/**
* check if edge source id and target id valid
*/
def checkField(field: String,
fieldType: String,
row: Row,
policy: Option[KeyPolicy.Value],
streamFlag: Boolean,
isVidStringType: Boolean): Boolean = {
val fieldValue = if (edgeConfig.isGeo && "source_field".equals(fieldType)) {
val lat = row.getDouble(row.schema.fieldIndex(edgeConfig.latitude.get))
val lng = row.getDouble(row.schema.fieldIndex(edgeConfig.longitude.get))
Some(indexCells(lat, lng).mkString(","))
} else {
val index = row.schema.fieldIndex(field)
if (index < 0 || row.isNullAt(index)) {
printChoice(streamFlag, s"$fieldType must exist and cannot be null, your row data is $row")
None
} else Some(row.get(index).toString)
}
val idFlag = fieldValue.isDefined
val policyFlag =
if (idFlag && policy.isEmpty && !isVidStringType
&& !NebulaUtils.isNumic(fieldValue.get)) {
printChoice(
streamFlag,
s"space vidType is int, but your $fieldType $fieldValue is not numeric.your row data is $row")
false
} else if (idFlag && policy.isDefined && isVidStringType) {
printChoice(
streamFlag,
s"only int vidType can use policy, but your vidType is FIXED_STRING.your row data is $row")
false
} else true
idFlag && policyFlag
}
/**
* convert row data to {@link Edge}
*/
def convertToEdge(row: Row,
edgeConfig: EdgeConfigEntry,
isVidStringType: Boolean,
fieldKeys: List[String],
fieldTypeMap: Map[String, Int]): Edge = {
val sourceField = processField(edgeConfig.sourceField,
"source_field",
row,
edgeConfig.sourcePolicy,
isVidStringType)
val targetField = processField(edgeConfig.targetField,
"target_field",
row,
edgeConfig.targetPolicy,
isVidStringType)
val values = for {
property <- fieldKeys if property.trim.length != 0
} yield extraValueForClient(row, property, fieldTypeMap)
if (edgeConfig.rankingField.isDefined) {
val index = row.schema.fieldIndex(edgeConfig.rankingField.get)
val ranking = row.get(index).toString
Edge(sourceField, targetField, Some(ranking.toLong), values)
} else {
Edge(sourceField, targetField, None, values)
}
}
/**
* process edge source and target field
*/
def processField(field: String,
fieldType: String,
row: Row,
policy: Option[KeyPolicy.Value],
isVidStringType: Boolean): String = {
var fieldValue = if (edgeConfig.isGeo && "source_field".equals(fieldType)) {
val lat = row.getDouble(row.schema.fieldIndex(edgeConfig.latitude.get))
val lng = row.getDouble(row.schema.fieldIndex(edgeConfig.longitude.get))
indexCells(lat, lng).mkString(",")
} else {
val index = row.schema.fieldIndex(field)
val value = row.get(index).toString
if (value.equals(DEFAULT_EMPTY_VALUE)) "" else value
}
// process string type vid
if (policy.isEmpty && isVidStringType) {
fieldValue = NebulaUtils.escapeUtil(fieldValue).mkString("\"", "", "\"")
}
fieldValue
}
/**
* encode edge
*/
def encodeEdge(row: Row,
partitionNum: Int,
vidType: VidType.Value,
spaceVidLen: Int,
edgeItem: EdgeItem,
fieldTypeMap: Map[String, Int]): (Array[Byte], Array[Byte], Array[Byte]) = {
isEdgeValid(row, edgeConfig, false, vidType == VidType.STRING)
val srcIndex: Int = row.schema.fieldIndex(edgeConfig.sourceField)
var srcId: String = row.get(srcIndex).toString
if (srcId.equals(DEFAULT_EMPTY_VALUE)) { srcId = "" }
val dstIndex: Int = row.schema.fieldIndex(edgeConfig.targetField)
var dstId: String = row.get(dstIndex).toString
if (dstId.equals(DEFAULT_EMPTY_VALUE)) { dstId = "" }
if (edgeConfig.sourcePolicy.isDefined) {
edgeConfig.sourcePolicy.get match {
case KeyPolicy.HASH =>
srcId = MurmurHash2
.hash64(srcId.getBytes(), srcId.getBytes().length, 0xc70f6907)
.toString
case KeyPolicy.UUID =>
throw new UnsupportedOperationException("do not support uuid yet")
case _ =>
throw new IllegalArgumentException(s"policy ${edgeConfig.sourcePolicy.get} is invalidate")
}
}
if (edgeConfig.targetPolicy.isDefined) {
edgeConfig.targetPolicy.get match {
case KeyPolicy.HASH =>
dstId = MurmurHash2
.hash64(dstId.getBytes(), dstId.getBytes().length, 0xc70f6907)
.toString
case KeyPolicy.UUID =>
throw new UnsupportedOperationException("do not support uuid yet")
case _ =>
throw new IllegalArgumentException(s"policy ${edgeConfig.targetPolicy.get} is invalidate")
}
}
val ranking: Long = if (edgeConfig.rankingField.isDefined) {
val rankIndex = row.schema.fieldIndex(edgeConfig.rankingField.get)
row.get(rankIndex).toString.toLong
} else {
0
}
val srcPartitionId = NebulaUtils.getPartitionId(srcId, partitionNum, vidType)
val dstPartitionId = NebulaUtils.getPartitionId(dstId, partitionNum, vidType)
val codec = new NebulaCodecImpl()
import java.nio.ByteBuffer
val srcBytes = if (vidType == VidType.INT) {
ByteBuffer
.allocate(8)
.order(ByteOrder.nativeOrder)
.putLong(srcId.toLong)
.array
} else {
srcId.getBytes()
}
val dstBytes = if (vidType == VidType.INT) {
ByteBuffer
.allocate(8)
.order(ByteOrder.nativeOrder)
.putLong(dstId.toLong)
.array
} else {
dstId.getBytes()
}
val positiveEdgeKey = codec.edgeKeyByDefaultVer(spaceVidLen,
srcPartitionId,
srcBytes,
edgeItem.getEdge_type,
ranking,
dstBytes)
val reverseEdgeKey = codec.edgeKeyByDefaultVer(spaceVidLen,
dstPartitionId,
dstBytes,
-edgeItem.getEdge_type,
ranking,
srcBytes)
val values = for {
property <- fieldKeys if property.trim.length != 0
} yield
extraValueForSST(row, property, fieldTypeMap)
.asInstanceOf[AnyRef]
val edgeValue = codec.encodeEdge(edgeItem, nebulaKeys.asJava, values.asJava)
(positiveEdgeKey, reverseEdgeKey, edgeValue)
}
}