|
| 1 | +/* Copyright (c) 2022 vesoft inc. All rights reserved. |
| 2 | + * |
| 3 | + * This source code is licensed under Apache 2.0 License. |
| 4 | + */ |
| 5 | + |
| 6 | +package com.vesoft.nebula.algorithm |
| 7 | + |
| 8 | +import com.vesoft.nebula.connector.connector.NebulaDataFrameReader |
| 9 | +import com.facebook.thrift.protocol.TCompactProtocol |
| 10 | +import com.vesoft.nebula.connector.{NebulaConnectionConfig, ReadNebulaConfig} |
| 11 | +import org.apache.log4j.Logger |
| 12 | +import org.apache.spark.SparkConf |
| 13 | +import org.apache.spark.graphx.{Edge, EdgeDirection, EdgeTriplet, Graph, Pregel, VertexId} |
| 14 | +import org.apache.spark.rdd.RDD |
| 15 | +import org.apache.spark.sql.{DataFrame, Encoder, SparkSession} |
| 16 | + |
| 17 | +import scala.collection.mutable |
| 18 | + |
| 19 | +object DeepQueryTest { |
| 20 | + private val LOGGER = Logger.getLogger(this.getClass) |
| 21 | + |
| 22 | + def main(args: Array[String]): Unit = { |
| 23 | + val sparkConf = new SparkConf() |
| 24 | + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") |
| 25 | + val spark = SparkSession |
| 26 | + .builder() |
| 27 | + .config(sparkConf) |
| 28 | + .getOrCreate() |
| 29 | + val iter = args(0).toInt |
| 30 | + val id = args(1).toInt |
| 31 | + |
| 32 | + query(spark, iter, id) |
| 33 | + } |
| 34 | + |
| 35 | + def readNebulaData(spark: SparkSession): DataFrame = { |
| 36 | + |
| 37 | + val config = |
| 38 | + NebulaConnectionConfig |
| 39 | + .builder() |
| 40 | + .withMetaAddress("192.168.15.5:9559") |
| 41 | + .withTimeout(6000) |
| 42 | + .withConenctionRetry(2) |
| 43 | + .build() |
| 44 | + val nebulaReadEdgeConfig: ReadNebulaConfig = ReadNebulaConfig |
| 45 | + .builder() |
| 46 | + .withSpace("twitter") |
| 47 | + .withLabel("FOLLOW") |
| 48 | + .withNoColumn(true) |
| 49 | + .withLimit(20000) |
| 50 | + .withPartitionNum(120) |
| 51 | + .build() |
| 52 | + val df: DataFrame = |
| 53 | + spark.read.nebula(config, nebulaReadEdgeConfig).loadEdgesToDF() |
| 54 | + df |
| 55 | + } |
| 56 | + |
| 57 | + def deepQuery(df: DataFrame, |
| 58 | + maxIterations: Int, |
| 59 | + startId: Int): Graph[mutable.HashSet[Int], Double] = { |
| 60 | + implicit val encoder: Encoder[Edge[Double]] = org.apache.spark.sql.Encoders.kryo[Edge[Double]] |
| 61 | + val edges: RDD[Edge[Double]] = df |
| 62 | + .map(row => { |
| 63 | + Edge(row.get(0).toString.toLong, row.get(1).toString.toLong, 1.0) |
| 64 | + })(encoder) |
| 65 | + .rdd |
| 66 | + |
| 67 | + val graph = Graph.fromEdges(edges, None) |
| 68 | + |
| 69 | + val queryGraph = graph.mapVertices { (vid, _) => |
| 70 | + mutable.HashSet[Int](vid.toInt) |
| 71 | + } |
| 72 | + queryGraph.cache() |
| 73 | + queryGraph.numVertices |
| 74 | + queryGraph.numEdges |
| 75 | + df.unpersist() |
| 76 | + |
| 77 | + def sendMessage(edge: EdgeTriplet[mutable.HashSet[Int], Double]) |
| 78 | + : Iterator[(VertexId, mutable.HashSet[Int])] = { |
| 79 | + val (smallSet, largeSet) = if (edge.srcAttr.size < edge.dstAttr.size) { |
| 80 | + (edge.srcAttr, edge.dstAttr) |
| 81 | + } else { |
| 82 | + (edge.dstAttr, edge.srcAttr) |
| 83 | + } |
| 84 | + |
| 85 | + if (smallSet.size == maxIterations) { |
| 86 | + Iterator.empty |
| 87 | + } else { |
| 88 | + val newNeighbors = |
| 89 | + (for (id <- smallSet; neighbor <- largeSet if neighbor != id) yield neighbor) |
| 90 | + Iterator((edge.dstId, newNeighbors)) |
| 91 | + } |
| 92 | + } |
| 93 | + |
| 94 | + val initialMessage = mutable.HashSet[Int]() |
| 95 | + |
| 96 | + val pregelGraph = Pregel(queryGraph, initialMessage, maxIterations, EdgeDirection.Both)( |
| 97 | + vprog = (id, attr, msg) => attr ++ msg, |
| 98 | + sendMsg = sendMessage, |
| 99 | + mergeMsg = (a, b) => { |
| 100 | + val setResult = a ++ b |
| 101 | + setResult |
| 102 | + } |
| 103 | + ) |
| 104 | + pregelGraph.cache() |
| 105 | + pregelGraph.numVertices |
| 106 | + pregelGraph.numEdges |
| 107 | + queryGraph.unpersist() |
| 108 | + pregelGraph |
| 109 | + } |
| 110 | + |
| 111 | + def query(spark: SparkSession, maxIter: Int, startId: Int): Unit = { |
| 112 | + val start = System.currentTimeMillis() |
| 113 | + val df = readNebulaData(spark) |
| 114 | + df.cache() |
| 115 | + df.count() |
| 116 | + println(s"read data cost time ${(System.currentTimeMillis() - start)}") |
| 117 | + |
| 118 | + val startQuery = System.currentTimeMillis() |
| 119 | + val graph = deepQuery(df, maxIter, startId) |
| 120 | + |
| 121 | + val endQuery = System.currentTimeMillis() |
| 122 | + val num = graph.vertices.filter(row => row._2.contains(startId)).count() |
| 123 | + val end = System.currentTimeMillis() |
| 124 | + println(s"query cost: ${endQuery - startQuery}") |
| 125 | + println(s"count: ${num}, cost: ${end - endQuery}") |
| 126 | + } |
| 127 | +} |
0 commit comments