Skip to content

Commit

Permalink
reconstruct the data process (#45)
Browse files Browse the repository at this point in the history
* reconstruct the data process

* extract data process

* extract data process

* add test

* update test
  • Loading branch information
Nicole00 authored Dec 28, 2021
1 parent d229bee commit 657b83b
Show file tree
Hide file tree
Showing 14 changed files with 1,198 additions and 393 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,8 @@ package object exchange {
def propertyValues: String = values.mkString(", ")

override def toString: String = {
s"Edge: ${source}->${destination}@${ranking} values: ${propertyValues}"
val rank = if (ranking.isEmpty) 0 else ranking.get
s"Edge: ${source}->${destination}@${rank} values: ${propertyValues}"
}
}

Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,21 @@

package com.vesoft.nebula.exchange.processor

import com.vesoft.nebula.{Coordinate, Date, DateTime, Geography, LineString, NullType, Point, Polygon, PropertyType, Time, Value}
import com.vesoft.nebula.exchange.Vertex
import com.vesoft.nebula.exchange.config.{EdgeConfigEntry, TagConfigEntry}
import com.vesoft.nebula.{
Coordinate,
Date,
DateTime,
Geography,
LineString,
NullType,
Point,
Polygon,
PropertyType,
Time,
Value
}
import com.vesoft.nebula.exchange.utils.NebulaUtils.DEFAULT_EMPTY_VALUE
import com.vesoft.nebula.exchange.utils.{HDFSUtils, NebulaUtils}
import org.apache.log4j.Logger
Expand Down Expand Up @@ -174,15 +188,6 @@ trait Processor extends Serializable {
HDFSUtils.getContent(path).toLong
}

def getLong(row: Row, field: String): Long = {
val index = row.schema.fieldIndex(field)
row.schema.fields(index).dataType match {
case LongType => row.getLong(index)
case IntegerType => row.getInt(index).toLong
case StringType => row.getString(index).toLong
}
}

def convertJTSGeometryToGeography(jtsGeom: org.locationtech.jts.geom.Geometry): Geography = {
jtsGeom.getGeometryType match {
case "Point" => {
Expand Down Expand Up @@ -224,7 +229,8 @@ trait Processor extends Serializable {
}

def printChoice(streamFlag: Boolean, context: String): Unit = {
if (streamFlag) LOG.info(context)
if (streamFlag) LOG.warn(context)
else assert(assertion = false, context)
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ package com.vesoft.nebula.exchange.processor
import java.nio.file.{Files, Paths}
import java.nio.{ByteBuffer, ByteOrder}

import com.google.common.net.HostAndPort
import com.vesoft.nebula.client.graph.data.HostAddress
import com.vesoft.nebula.encoder.NebulaCodecImpl
import com.vesoft.nebula.exchange.{
Expand All @@ -29,12 +30,14 @@ import com.vesoft.nebula.exchange.config.{
import com.vesoft.nebula.exchange.utils.NebulaUtils.DEFAULT_EMPTY_VALUE
import com.vesoft.nebula.exchange.utils.{HDFSUtils, NebulaUtils}
import com.vesoft.nebula.exchange.writer.{NebulaGraphClientWriter, NebulaSSTWriter}
import com.vesoft.nebula.meta.TagItem
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}
import org.apache.spark.util.LongAccumulator
import org.rocksdb.SstFileWriter

import scala.collection.JavaConverters._
import scala.collection.mutable.{ArrayBuffer, ListBuffer}
Expand Down Expand Up @@ -125,152 +128,36 @@ class VerticesProcessor(data: DataFrame,
.dropDuplicates(tagConfig.vertexField)
.mapPartitions { iter =>
iter.map { row =>
val index: Int = row.schema.fieldIndex(tagConfig.vertexField)
assert(index >= 0 && !row.isNullAt(index),
s"vertexId must exist and cannot be null, your row data is $row")
var vertexId: String = row.get(index).toString
if (vertexId.equals(DEFAULT_EMPTY_VALUE)) {
vertexId = ""
}
if (tagConfig.vertexPolicy.isDefined) {
tagConfig.vertexPolicy.get match {
case KeyPolicy.HASH =>
vertexId = MurmurHash2
.hash64(vertexId.getBytes(), vertexId.getBytes().length, 0xc70f6907)
.toString
case KeyPolicy.UUID =>
throw new UnsupportedOperationException("do not support uuid yet")
case _ =>
throw new IllegalArgumentException(
s"policy ${tagConfig.vertexPolicy.get} is invalidate")
}
}

val hostAddrs: ListBuffer[HostAddress] = new ListBuffer[HostAddress]
for (addr <- address) {
hostAddrs.append(new HostAddress(addr.getHostText, addr.getPort))
}

val partitionId = NebulaUtils.getPartitionId(vertexId, partitionNum, vidType)

val codec = new NebulaCodecImpl()

import java.nio.ByteBuffer
val vidBytes = if (vidType == VidType.INT) {
ByteBuffer
.allocate(8)
.order(ByteOrder.nativeOrder)
.putLong(vertexId.toLong)
.array
} else {
vertexId.getBytes()
}

val vertexKey = codec.vertexKey(spaceVidLen, partitionId, vidBytes, tagItem.getTag_id)
val values = for {
property <- fieldKeys if property.trim.length != 0
} yield
extraValueForSST(row, property, fieldTypeMap)
.asInstanceOf[AnyRef]
val vertexValue = codec.encodeTag(tagItem, nebulaKeys.asJava, values.asJava)
(vertexKey, vertexValue)
encodeVertex(row, partitionNum, vidType, spaceVidLen, tagItem, fieldTypeMap)
}
}(Encoders.tuple(Encoders.BINARY, Encoders.BINARY))
.toDF("key", "value")
.sortWithinPartitions("key")
.foreachPartition { iterator: Iterator[Row] =>
val taskID = TaskContext.get().taskAttemptId()
var writer: NebulaSSTWriter = null
var currentPart = -1
val localPath = fileBaseConfig.localPath
val remotePath = fileBaseConfig.remotePath
try {
iterator.foreach { vertex =>
val key = vertex.getAs[Array[Byte]](0)
val value = vertex.getAs[Array[Byte]](1)
var part = ByteBuffer
.wrap(key, 0, 4)
.order(ByteOrder.nativeOrder)
.getInt >> 8
if (part <= 0) {
part = part + partitionNum
}

if (part != currentPart) {
if (writer != null) {
writer.close()
val localFile = s"$localPath/$currentPart-$taskID.sst"
HDFSUtils.upload(localFile,
s"$remotePath/${currentPart}/$currentPart-$taskID.sst",
namenode)
Files.delete(Paths.get(localFile))
}
currentPart = part
val tmp = s"$localPath/$currentPart-$taskID.sst"
writer = new NebulaSSTWriter(tmp)
writer.prepare()
}
writer.write(key, value)
}
} catch {
case e: Throwable => {
LOG.error(e)
batchFailure.add(1)
}
} finally {
if (writer != null) {
writer.close()
val localFile = s"$localPath/$currentPart-$taskID.sst"
HDFSUtils.upload(localFile,
s"$remotePath/${currentPart}/$currentPart-$taskID.sst",
namenode)
Files.delete(Paths.get(localFile))
}
}
val sstFileWriter = new NebulaSSTWriter
sstFileWriter.writeSstFiles(iterator,
fileBaseConfig,
partitionNum,
namenode,
batchFailure)
}
} else {
val streamFlag = data.isStreaming
val vertices = data
.filter { row => //filter and check row data,if streaming only print log
val index = row.schema.fieldIndex(tagConfig.vertexField)
if (index < 0 || row.isNullAt(index)) {
printChoice(streamFlag, s"vertexId must exist and cannot be null, your row data is $row")
false
} else {
val vertexId = row.get(index).toString
// process int type vid
if (tagConfig.vertexPolicy.isEmpty && !isVidStringType && !NebulaUtils.isNumic(vertexId)) {
printChoice(streamFlag, s"space vidType is int, but your vertex id $vertexId is not numeric.your row data is $row")
false
} else if (tagConfig.vertexPolicy.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
}
.filter { row =>
isVertexValid(row, tagConfig, streamFlag, isVidStringType)
}
.map { row =>
val index = row.schema.fieldIndex(tagConfig.vertexField)
var vertexId = row.get(index).toString
if (vertexId.equals(DEFAULT_EMPTY_VALUE)) {
vertexId = ""
}

if (tagConfig.vertexPolicy.isEmpty && isVidStringType){
vertexId = NebulaUtils.escapeUtil(vertexId).mkString("\"", "", "\"")
}

val values = for {
property <- fieldKeys if property.trim.length != 0
} yield extraValueForClient(row, property, fieldTypeMap)
Vertex(vertexId, values)
convertToVertex(row, tagConfig, isVidStringType, fieldKeys, fieldTypeMap)
}(Encoders.kryo[Vertex])

// streaming write
if (streamFlag) {
val streamingDataSourceConfig =
tagConfig.dataSourceConfigEntry.asInstanceOf[StreamingDataSourceConfigEntry]
val wStream = vertices.writeStream
if (tagConfig.checkPointPath.isDefined) wStream.option("checkpointLocation", tagConfig.checkPointPath.get)
if (tagConfig.checkPointPath.isDefined)
wStream.option("checkpointLocation", tagConfig.checkPointPath.get)

wStream
.foreachBatch((vertexSet, batchId) => {
Expand All @@ -284,4 +171,113 @@ class VerticesProcessor(data: DataFrame,
vertices.foreachPartition(processEachPartition _)
}
}

/**
* filter and check row data for vertex, if streaming only print log
* for not streaming datasource, if the vertex data is invalid, throw AssertException.
*/
def isVertexValid(row: Row,
tagConfig: TagConfigEntry,
streamFlag: Boolean,
isVidStringType: Boolean): Boolean = {
val index = row.schema.fieldIndex(tagConfig.vertexField)
if (index < 0 || row.isNullAt(index)) {
printChoice(streamFlag, s"vertexId must exist and cannot be null, your row data is $row")
return false
}

val vertexId = row.get(index).toString
// process int type vid
if (tagConfig.vertexPolicy.isEmpty && !isVidStringType && !NebulaUtils.isNumic(vertexId)) {
printChoice(
streamFlag,
s"space vidType is int, but your vertex id $vertexId is not numeric.your row data is $row")
return false
}
// process string type vid
if (tagConfig.vertexPolicy.isDefined && isVidStringType) {
printChoice(
streamFlag,
s"only int vidType can use policy, but your vidType is FIXED_STRING.your row data is $row")
return false
}
true
}

/**
* Convert row data to {@link Vertex}
*/
def convertToVertex(row: Row,
tagConfig: TagConfigEntry,
isVidStringType: Boolean,
fieldKeys: List[String],
fieldTypeMap: Map[String, Int]): Vertex = {
val index = row.schema.fieldIndex(tagConfig.vertexField)
var vertexId = row.get(index).toString
if (vertexId.equals(DEFAULT_EMPTY_VALUE)) {
vertexId = ""
}

if (tagConfig.vertexPolicy.isEmpty && isVidStringType) {
vertexId = NebulaUtils.escapeUtil(vertexId).mkString("\"", "", "\"")
}

val values = for {
property <- fieldKeys if property.trim.length != 0
} yield extraValueForClient(row, property, fieldTypeMap)
Vertex(vertexId, values)
}

/**
* encode vertex
*/
def encodeVertex(row: Row,
partitionNum: Int,
vidType: VidType.Value,
spaceVidLen: Int,
tagItem: TagItem,
fieldTypeMap: Map[String, Int]): (Array[Byte], Array[Byte]) = {
// check if vertex id is valid, if not, throw AssertException
isVertexValid(row, tagConfig, false, vidType == VidType.STRING)

val index: Int = row.schema.fieldIndex(tagConfig.vertexField)
var vertexId: String = row.get(index).toString
if (vertexId.equals(DEFAULT_EMPTY_VALUE)) {
vertexId = ""
}
if (tagConfig.vertexPolicy.isDefined) {
tagConfig.vertexPolicy.get match {
case KeyPolicy.HASH =>
vertexId = MurmurHash2
.hash64(vertexId.getBytes(), vertexId.getBytes().length, 0xc70f6907)
.toString
case KeyPolicy.UUID =>
throw new UnsupportedOperationException("do not support uuid yet")
case _ =>
throw new IllegalArgumentException(s"policy ${tagConfig.vertexPolicy.get} is invalidate")
}
}

val partitionId = NebulaUtils.getPartitionId(vertexId, partitionNum, vidType)

import java.nio.ByteBuffer
val vidBytes = if (vidType == VidType.INT) {
ByteBuffer
.allocate(8)
.order(ByteOrder.nativeOrder)
.putLong(vertexId.toLong)
.array
} else {
vertexId.getBytes()
}
val codec = new NebulaCodecImpl()
val vertexKey = codec.vertexKey(spaceVidLen, partitionId, vidBytes, tagItem.getTag_id)
val values = for {
property <- fieldKeys if property.trim.length != 0
} yield
extraValueForSST(row, property, fieldTypeMap)
.asInstanceOf[AnyRef]
val vertexValue = codec.encodeTag(tagItem, nebulaKeys.asJava, values.asJava)
(vertexKey, vertexValue)
}
}
Loading

0 comments on commit 657b83b

Please sign in to comment.