@@ -21,7 +21,7 @@ import com.alibaba.graphar.{GeneralParams, EdgeInfo, FileType, AdjListType, Prop
21
21
import org .apache .spark .sql .SparkSession
22
22
import org .apache .spark .sql .Row
23
23
import org .apache .spark .sql .DataFrame
24
- import org .apache .spark .sql .types .{LongType , StructField }
24
+ import org .apache .spark .sql .types .{IntegerType , LongType , StructType , StructField }
25
25
import org .apache .spark .util .Utils
26
26
import org .apache .spark .rdd .RDD
27
27
import org .apache .spark .sql .functions ._
@@ -148,19 +148,36 @@ class EdgeWriter(prefix: String, edgeInfo: EdgeInfo, adjListType: AdjListType.V
148
148
}
149
149
}
150
150
151
- // generate the Offset chunks files from edge dataframe for this edge type
151
+ // generate the offset chunks files from edge dataframe for this edge type
152
152
private def writeOffset (): Unit = {
153
+ val spark = edgeDf.sparkSession
153
154
val file_type = edgeInfo.getAdjListFileType(adjListType)
154
- var chunk_index : Long = 0
155
+ var chunk_index : Int = 0
156
+ val offset_schema = StructType (Seq (StructField (GeneralParams .offsetCol, LongType )))
157
+ val vertex_chunk_size = if (adjListType == AdjListType .ordered_by_source) edgeInfo.getSrc_chunk_size() else edgeInfo.getDst_chunk_size()
158
+ val index_column = if (adjListType == AdjListType .ordered_by_source) GeneralParams .srcIndexCol else GeneralParams .dstIndexCol
159
+ val output_prefix = prefix + edgeInfo.getAdjListOffsetDirPath(adjListType)
155
160
for (chunk <- chunks) {
156
- val output_prefix = prefix + edgeInfo.getAdjListOffsetDirPath(adjListType) + " part" + chunk_index.toString + " /"
157
- if (adjListType == AdjListType .ordered_by_source) {
158
- val offset_chunk = chunk.select(GeneralParams .srcIndexCol).groupBy(GeneralParams .srcIndexCol).count().coalesce(1 ).orderBy(GeneralParams .srcIndexCol).select(" count" )
159
- FileSystem .writeDataFrame(offset_chunk, FileType .FileTypeToString (file_type), output_prefix)
160
- } else {
161
- val offset_chunk = chunk.select(GeneralParams .dstIndexCol).groupBy(GeneralParams .dstIndexCol).count().coalesce(1 ).orderBy(GeneralParams .dstIndexCol).select(" count" )
162
- FileSystem .writeDataFrame(offset_chunk, FileType .FileTypeToString (file_type), output_prefix)
163
- }
161
+ val edge_count_df = chunk.select(index_column).groupBy(index_column).count()
162
+ // init a edge count dataframe of vertex range [begin, end] to include isloated vertex
163
+ val begin_index : Long = chunk_index * vertex_chunk_size;
164
+ val end_index : Long = (chunk_index + 1 ) * vertex_chunk_size
165
+ val init_count_rdd = spark.sparkContext.parallelize(begin_index to end_index).map(key => Row (key, 0L ))
166
+ val init_count_df = spark.createDataFrame(init_count_rdd, edge_count_df.schema)
167
+ // union edge count dataframe and initialized count dataframe
168
+ val union_count_chunk = edge_count_df.unionByName(init_count_df).groupBy(index_column).agg(sum(" count" )).coalesce(1 ).orderBy(index_column).select(" sum(count)" )
169
+ // calculate offset rdd from count chunk
170
+ val offset_rdd = union_count_chunk.rdd.mapPartitionsWithIndex((i, ps) => {
171
+ var sum = 0L
172
+ var pre_sum = 0L
173
+ for (row <- ps ) yield {
174
+ pre_sum = sum
175
+ sum = sum + row.getLong(0 )
176
+ Row (pre_sum)
177
+ }
178
+ })
179
+ val offset_df = spark.createDataFrame(offset_rdd, offset_schema)
180
+ FileSystem .writeDataFrame(offset_df, FileType .FileTypeToString (file_type), output_prefix, chunk_index)
164
181
chunk_index = chunk_index + 1
165
182
}
166
183
}
0 commit comments