Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

add switch for tagless #100

Merged
merged 2 commits into from
Oct 19, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -267,6 +267,7 @@ object Configs {

val space = nebulaConfig.getString("space")
val databaseEntry = DataBaseConfigEntry(addresses, space, metaAddresses)
val enableTagless = getOrElse(nebulaConfig, "enableTagless", false)
LOG.info(s"DataBase Config ${databaseEntry}")

val user = nebulaConfig.getString("user")
Expand Down Expand Up @@ -409,7 +410,8 @@ object Configs {
batch,
partition,
checkPointPath,
repartitionWithNebula)
repartitionWithNebula,
enableTagless)
LOG.info(s"Tag Config: ${entry}")
tags += entry
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,8 @@ case class TagConfigEntry(override val name: String,
override val batch: Int,
override val partition: Int,
override val checkPointPath: Option[String],
repartitionWithNebula: Boolean = false)
repartitionWithNebula: Boolean = false,
enableTagless: Boolean = false)
extends SchemaConfigEntry {
require(name.trim.nonEmpty && vertexField.trim.nonEmpty && batch > 0)

Expand All @@ -71,7 +72,9 @@ case class TagConfigEntry(override val name: String,
s"vertex field: $vertexField, " +
s"vertex policy: $vertexPolicy, " +
s"batch: $batch, " +
s"partition: $partition."
s"partition: $partition, " +
s"repartitionWithNebula: $repartitionWithNebula, " +
s"enableTagless: $enableTagless."
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -121,13 +121,31 @@ class VerticesProcessor(spark: SparkSession,
val tagItem = metaProvider.getTagItem(space, tagName)
val emptyValue = ByteBuffer.allocate(0).array()

var sstKeyValueData = data
.dropDuplicates(tagConfig.vertexField)
.mapPartitions { iter =>
iter.map { row =>
encodeVertex(row, partitionNum, vidType, spaceVidLen, tagItem, fieldTypeMap)
}
}(Encoders.tuple(Encoders.BINARY, Encoders.BINARY))
var sstKeyValueData = if (tagConfig.enableTagless) {
data
.dropDuplicates(tagConfig.vertexField)
.mapPartitions { iter =>
iter.map { row =>
encodeVertexForTageless(row,
partitionNum,
vidType,
spaceVidLen,
tagItem,
fieldTypeMap)
}
}(Encoders.tuple(Encoders.BINARY, Encoders.BINARY, Encoders.BINARY))
.flatMap(line => {
List((line._1, emptyValue), (line._2, line._3))
})(Encoders.tuple(Encoders.BINARY, Encoders.BINARY))
} else {
data
.dropDuplicates(tagConfig.vertexField)
.mapPartitions { iter =>
iter.map { row =>
encodeVertex(row, partitionNum, vidType, spaceVidLen, tagItem, fieldTypeMap)
}
}(Encoders.tuple(Encoders.BINARY, Encoders.BINARY))
}

// repartition dataframe according to nebula part, to make sure sst files for one part has no overlap
if (tagConfig.repartitionWithNebula) {
Expand Down Expand Up @@ -224,6 +242,34 @@ class VerticesProcessor(spark: SparkSession,
spaceVidLen: Int,
tagItem: TagItem,
fieldTypeMap: Map[String, Int]): (Array[Byte], Array[Byte]) = {
val (orphanVertexKey, vertexKey, vertexValue) =
getVertexKeyValue(row, partitionNum, vidType, spaceVidLen, tagItem, fieldTypeMap)
(vertexKey, vertexValue)
}

/**
* encode vertex for tagless
*/
def encodeVertexForTageless(
row: Row,
partitionNum: Int,
vidType: VidType.Value,
spaceVidLen: Int,
tagItem: TagItem,
fieldTypeMap: Map[String, Int]): (Array[Byte], Array[Byte], Array[Byte]) = {
getVertexKeyValue(row, partitionNum, vidType, spaceVidLen, tagItem, fieldTypeMap)
}

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

Expand Down Expand Up @@ -259,12 +305,14 @@ class VerticesProcessor(spark: SparkSession,
}
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)
val vertexValue = codec.encodeTag(tagItem, nebulaKeys.asJava, values.asJava)
val orphanVertexKey = codec.orphanVertexKey(spaceVidLen, partitionId, vidBytes)
(orphanVertexKey, vertexKey, vertexValue)
}
}
5 changes: 5 additions & 0 deletions nebula-exchange_spark_2.4/src/main/resources/application.conf
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,11 @@
pswd: nebula
space: test

# enableTagless is used for sst mode.
# if false, just write vertex with tag to sst file
# if true, write vertex with tag and vertex without tag
enableTagless:false

# if com.vesoft.exchange.common.config graph ssl encrypted transmission
ssl:{
# if enable is false, other params of ssl are invalid.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -121,14 +121,28 @@ class VerticesProcessor(spark: SparkSession,

val spaceVidLen = metaProvider.getSpaceVidLen(space)
val tagItem = metaProvider.getTagItem(space, tagName)
val emptyValue = ByteBuffer.allocate(0).array()

var sstKeyValueData = data
.dropDuplicates(tagConfig.vertexField)
.mapPartitions { iter =>
iter.map { row =>
encodeVertex(row, partitionNum, vidType, spaceVidLen, tagItem, fieldTypeMap)
}
}(Encoders.tuple(Encoders.BINARY, Encoders.BINARY))
var sstKeyValueData = if (tagConfig.enableTagless) {
data
.dropDuplicates(tagConfig.vertexField)
.mapPartitions { iter =>
iter.map { row =>
encodeVertexForTagless(row, partitionNum, vidType, spaceVidLen, tagItem, fieldTypeMap)
}
}(Encoders.tuple(Encoders.BINARY, Encoders.BINARY, Encoders.BINARY))
.flatMap(line => {
List((line._1, emptyValue), (line._2, line._3))
})(Encoders.tuple(Encoders.BINARY, Encoders.BINARY))
} else {
data
.dropDuplicates(tagConfig.vertexField)
.mapPartitions { iter =>
iter.map { row =>
encodeVertex(row, partitionNum, vidType, spaceVidLen, tagItem, fieldTypeMap)
}
}(Encoders.tuple(Encoders.BINARY, Encoders.BINARY))
}

// repartition dataframe according to nebula part, to make sure sst files for one part has no overlap
if (tagConfig.repartitionWithNebula) {
Expand Down Expand Up @@ -242,6 +256,34 @@ class VerticesProcessor(spark: SparkSession,
spaceVidLen: Int,
tagItem: TagItem,
fieldTypeMap: Map[String, Int]): (Array[Byte], Array[Byte]) = {
val (orphanVertexKey, vertexKey, vertexValue) =
getVertexKeyValue(row, partitionNum, vidType, spaceVidLen, tagItem, fieldTypeMap)
(vertexKey, vertexValue)
}

/**
* encode vertex for tagless
*/
def encodeVertexForTagless(
row: Row,
partitionNum: Int,
vidType: VidType.Value,
spaceVidLen: Int,
tagItem: TagItem,
fieldTypeMap: Map[String, Int]): (Array[Byte], Array[Byte], Array[Byte]) = {
getVertexKeyValue(row, partitionNum, vidType, spaceVidLen, tagItem, fieldTypeMap)
}

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

Expand Down Expand Up @@ -277,12 +319,14 @@ class VerticesProcessor(spark: SparkSession,
}
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)
val vertexValue = codec.encodeTag(tagItem, nebulaKeys.asJava, values.asJava)
val orphanVertexKey = codec.orphanVertexKey(spaceVidLen, partitionId, vidBytes)
(orphanVertexKey, vertexKey, vertexValue)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -123,13 +123,31 @@ class VerticesProcessor(spark: SparkSession,
val tagItem = metaProvider.getTagItem(space, tagName)
val emptyValue = ByteBuffer.allocate(0).array()

var sstKeyValueData = data
.dropDuplicates(tagConfig.vertexField)
.mapPartitions { iter =>
iter.map { row =>
encodeVertex(row, partitionNum, vidType, spaceVidLen, tagItem, fieldTypeMap)
}
}(Encoders.tuple(Encoders.BINARY, Encoders.BINARY))
var sstKeyValueData = if (tagConfig.enableTagless) {
data
.dropDuplicates(tagConfig.vertexField)
.mapPartitions { iter =>
iter.map { row =>
encodeVertexForTageless(row,
partitionNum,
vidType,
spaceVidLen,
tagItem,
fieldTypeMap)
}
}(Encoders.tuple(Encoders.BINARY, Encoders.BINARY, Encoders.BINARY))
.flatMap(line => {
List((line._1, emptyValue), (line._2, line._3))
})(Encoders.tuple(Encoders.BINARY, Encoders.BINARY))
} else {
data
.dropDuplicates(tagConfig.vertexField)
.mapPartitions { iter =>
iter.map { row =>
encodeVertex(row, partitionNum, vidType, spaceVidLen, tagItem, fieldTypeMap)
}
}(Encoders.tuple(Encoders.BINARY, Encoders.BINARY))
}

// repartition dataframe according to nebula part, to make sure sst files for one part has no overlap
if (tagConfig.repartitionWithNebula) {
Expand Down Expand Up @@ -243,6 +261,34 @@ class VerticesProcessor(spark: SparkSession,
spaceVidLen: Int,
tagItem: TagItem,
fieldTypeMap: Map[String, Int]): (Array[Byte], Array[Byte]) = {
val (orphanVertexKey, vertexKey, vertexValue) =
getVertexKeyValue(row, partitionNum, vidType, spaceVidLen, tagItem, fieldTypeMap)
(vertexKey, vertexValue)
}

/**
* encode vertex for tagless
*/
def encodeVertexForTageless(
row: Row,
partitionNum: Int,
vidType: VidType.Value,
spaceVidLen: Int,
tagItem: TagItem,
fieldTypeMap: Map[String, Int]): (Array[Byte], Array[Byte], Array[Byte]) = {
getVertexKeyValue(row, partitionNum, vidType, spaceVidLen, tagItem, fieldTypeMap)
}

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

Expand Down Expand Up @@ -278,12 +324,14 @@ class VerticesProcessor(spark: SparkSession,
}
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)
val vertexValue = codec.encodeTag(tagItem, nebulaKeys.asJava, values.asJava)
val orphanVertexKey = codec.orphanVertexKey(spaceVidLen, partitionId, vidBytes)
(orphanVertexKey, vertexKey, vertexValue)
}
}