|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | +package org.apache.spark.sql.execution |
| 18 | + |
| 19 | +import org.apache.gluten.backendsapi.BackendsApiManager |
| 20 | +import org.apache.gluten.columnarbatch.ColumnarBatches |
| 21 | +import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators |
| 22 | + |
| 23 | +import org.apache.spark.internal.Logging |
| 24 | +import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage |
| 25 | +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow |
| 26 | +import org.apache.spark.sql.execution.datasources._ |
| 27 | +import org.apache.spark.sql.vectorized.ColumnarBatch |
| 28 | + |
| 29 | +import com.fasterxml.jackson.databind.ObjectMapper |
| 30 | +import com.fasterxml.jackson.module.scala.DefaultScalaModule |
| 31 | + |
| 32 | +import scala.collection.mutable |
| 33 | + |
| 34 | +// Velox write files metrics start |
| 35 | +// |
| 36 | +// Follows the code in velox `HiveDataSink::close()` |
| 37 | +// The json can be as following: |
| 38 | +// { |
| 39 | +// "inMemoryDataSizeInBytes":0, |
| 40 | +// "containsNumberedFileNames":true, |
| 41 | +// "onDiskDataSizeInBytes":307, |
| 42 | +// "fileWriteInfos":[ |
| 43 | +// { |
| 44 | +// "fileSize":307, |
| 45 | +// "writeFileName": |
| 46 | +// "Gluten_Stage_1_TID_2_0_2_d1db3b31-4f99-41cb-a4e7-3b8607506168.parquet", |
| 47 | +// "targetFileName": |
| 48 | +// "Gluten_Stage_1_TID_2_0_2_d1db3b31-4f99-41cb-a4e7-3b8607506168.parquet" |
| 49 | +// } |
| 50 | +// ], |
| 51 | +// "writePath":"file:/home/gluten/spark-warehouse/inserttable/part1=1/part2=1", |
| 52 | +// "rowCount":1, |
| 53 | +// "targetPath":"file:/home/gluten/spark-warehouse/inserttable/part1=1/part2=1", |
| 54 | +// "updateMode":"NEW", |
| 55 | +// "name":"part1=1/part2=1" |
| 56 | +// } |
| 57 | +case class VeloxWriteFilesInfo(writeFileName: String, targetFileName: String, fileSize: Long) |
| 58 | + |
| 59 | +case class VeloxWriteFilesMetrics( |
| 60 | + name: String, |
| 61 | + updateMode: String, |
| 62 | + writePath: String, |
| 63 | + targetPath: String, |
| 64 | + fileWriteInfos: Seq[VeloxWriteFilesInfo], |
| 65 | + rowCount: Long, |
| 66 | + inMemoryDataSizeInBytes: Long, |
| 67 | + onDiskDataSizeInBytes: Long, |
| 68 | + containsNumberedFileNames: Boolean) |
| 69 | + |
| 70 | +// Velox write files metrics end |
| 71 | + |
| 72 | +case class VeloxBackendWrite(description: WriteJobDescription) extends BackendWrite with Logging { |
| 73 | + |
| 74 | + override def injectTempPath(commitProtocol: SparkWriteFilesCommitProtocol): String = { |
| 75 | + val writePath = commitProtocol.newTaskAttemptTempPath() |
| 76 | + logDebug(s"Velox staging write path: $writePath") |
| 77 | + BackendsApiManager.getIteratorApiInstance.injectWriteFilesTempPath(writePath) |
| 78 | + writePath |
| 79 | + } |
| 80 | + |
| 81 | + override def collectNativeWriteFilesMetrics(cb: ColumnarBatch): Option[WriteTaskResult] = { |
| 82 | + // Currently, the cb contains three columns: row, fragments, and context. |
| 83 | + // The first row in the row column contains the number of written numRows. |
| 84 | + // The fragments column contains detailed information about the file writes. |
| 85 | + val loadedCb = ColumnarBatches.ensureLoaded(ArrowBufferAllocators.contextInstance, cb) |
| 86 | + assert(loadedCb.numCols() == 3) |
| 87 | + val numWrittenRows = loadedCb.column(0).getLong(0) |
| 88 | + |
| 89 | + var updatedPartitions = Set.empty[String] |
| 90 | + val addedAbsPathFiles: mutable.Map[String, String] = mutable.Map[String, String]() |
| 91 | + var numBytes = 0L |
| 92 | + val objectMapper = new ObjectMapper() |
| 93 | + objectMapper.registerModule(DefaultScalaModule) |
| 94 | + for (i <- 0 until loadedCb.numRows() - 1) { |
| 95 | + val fragments = loadedCb.column(1).getUTF8String(i + 1) |
| 96 | + val metrics = objectMapper |
| 97 | + .readValue(fragments.toString.getBytes("UTF-8"), classOf[VeloxWriteFilesMetrics]) |
| 98 | + logDebug(s"Velox write files metrics: $metrics") |
| 99 | + |
| 100 | + val fileWriteInfos = metrics.fileWriteInfos |
| 101 | + assert(fileWriteInfos.length == 1) |
| 102 | + val fileWriteInfo = fileWriteInfos.head |
| 103 | + numBytes += fileWriteInfo.fileSize |
| 104 | + val targetFileName = fileWriteInfo.targetFileName |
| 105 | + val outputPath = description.path |
| 106 | + |
| 107 | + // part1=1/part2=1 |
| 108 | + val partitionFragment = metrics.name |
| 109 | + // Write a partitioned table |
| 110 | + if (partitionFragment != "") { |
| 111 | + updatedPartitions += partitionFragment |
| 112 | + val tmpOutputPath = outputPath + "/" + partitionFragment + "/" + targetFileName |
| 113 | + val customOutputPath = description.customPartitionLocations.get( |
| 114 | + PartitioningUtils.parsePathFragment(partitionFragment)) |
| 115 | + if (customOutputPath.isDefined) { |
| 116 | + addedAbsPathFiles(tmpOutputPath) = customOutputPath.get + "/" + targetFileName |
| 117 | + } |
| 118 | + } |
| 119 | + } |
| 120 | + |
| 121 | + val numFiles = loadedCb.numRows() - 1 |
| 122 | + val partitionsInternalRows = updatedPartitions.map { |
| 123 | + part => |
| 124 | + val parts = new Array[Any](1) |
| 125 | + parts(0) = part |
| 126 | + new GenericInternalRow(parts) |
| 127 | + }.toSeq |
| 128 | + val stats = BasicWriteTaskStats( |
| 129 | + partitions = partitionsInternalRows, |
| 130 | + numFiles = numFiles, |
| 131 | + numBytes = numBytes, |
| 132 | + numRows = numWrittenRows) |
| 133 | + val summary = |
| 134 | + ExecutedWriteSummary(updatedPartitions = updatedPartitions, stats = Seq(stats)) |
| 135 | + |
| 136 | + // Write an empty iterator |
| 137 | + if (numFiles == 0) { |
| 138 | + None |
| 139 | + } else { |
| 140 | + Some( |
| 141 | + WriteTaskResult( |
| 142 | + new TaskCommitMessage(addedAbsPathFiles.toMap -> updatedPartitions), |
| 143 | + summary)) |
| 144 | + } |
| 145 | + } |
| 146 | +} |
0 commit comments