Skip to content
Draft
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
@@ -0,0 +1,175 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.gluten.backendsapi.velox

import org.apache.gluten.backendsapi.velox.VeloxIteratorApi.unescapePathName
import org.apache.gluten.sql.shims.SparkShimLoader

import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.delta.actions.DeletionVectorDescriptor
import org.apache.spark.sql.delta.deletionvectors.{RoaringBitmapArrayFormat, StoredBitmap}
import org.apache.spark.sql.delta.storage.dv.HadoopFileSystemDVStore
import org.apache.spark.sql.execution.datasources.PartitionedFile

import org.apache.hadoop.fs.Path

import java.util.{ArrayList => JArrayList, HashMap => JHashMap, List => JList, Map => JMap}

import scala.collection.JavaConverters._
import scala.util.Try
import scala.util.control.NonFatal

object VeloxDeltaMetadataUtils {
val DeltaDvCardinality = "delta_dv_cardinality"
val DeltaDvPayloadIndex = "delta_dv_payload_index"

private val RowIndexFilterIdEncoded = "row_index_filter_id_encoded"
private val RowIndexFilterType = "row_index_filter_type"
private val RowIndexFilterTypeIfContained = "IF_CONTAINED"

final class NormalizedSplitMetadata(
val otherMetadataColumns: JList[JMap[String, Object]],
val deletionVectorPayloads: Array[Array[Byte]])
extends Serializable

private def decodeDescriptor(
normalizedMetadata: JMap[String, Object]): Option[DeletionVectorDescriptor] = {
Option(normalizedMetadata.get(RowIndexFilterIdEncoded))
.map(_.toString)
.filter(_.nonEmpty)
.flatMap(parseDescriptor)
}

private def parseDescriptor(encodedDescriptor: String): Option[DeletionVectorDescriptor] = {
val methods = Seq("deserializeFromBase64", "fromJson")
methods.iterator
.map {
methodName =>
Try {
val method = DeletionVectorDescriptor.getClass.getMethod(methodName, classOf[String])
method
.invoke(DeletionVectorDescriptor, encodedDescriptor)
.asInstanceOf[DeletionVectorDescriptor]
}.toOption
}
.collectFirst { case Some(descriptor) => descriptor }
}

private def serializePayload(
dvStore: HadoopFileSystemDVStore,
tablePath: Path,
descriptor: DeletionVectorDescriptor): Option[Array[Byte]] = {
if (tablePath == null) {
return None
}
Some(
StoredBitmap
.create(descriptor, tablePath)
.load(dvStore)
.serializeAsByteArray(RoaringBitmapArrayFormat.Portable))
}

private def normalizeMetadataWithDescriptor(
metadata: JMap[String, Object],
descriptor: DeletionVectorDescriptor): JMap[String, Object] = {
val normalized = new JHashMap[String, Object]()
if (metadata != null) {
normalized.putAll(metadata)
}
normalized.put(DeltaDvCardinality, Long.box(descriptor.cardinality))
normalized.remove(RowIndexFilterIdEncoded)
if (!normalized.containsKey(RowIndexFilterType)) {
normalized.put(RowIndexFilterType, RowIndexFilterTypeIfContained)
}
normalized
}

def normalizeSplitMetadata(
partitionColumnCount: Int,
files: JList[PartitionedFile]): NormalizedSplitMetadata = {
val dvStore = new HadoopFileSystemDVStore(activeSpark.sessionState.newHadoopConf())
val normalizedMetadataColumns = new JArrayList[JMap[String, Object]](files.size())
val deletionVectorPayloads = scala.collection.mutable.ArrayBuffer.empty[Array[Byte]]

files.asScala.foreach {
file =>
val otherMetadata =
SparkShimLoader.getSparkShims.getOtherConstantMetadataColumnValues(file)
val metadataWithDecodedPayload = new JHashMap[String, Object]()
if (otherMetadata != null) {
metadataWithDecodedPayload.putAll(otherMetadata)
}

val descriptor = decodeDescriptor(metadataWithDecodedPayload)

descriptor match {
case Some(descriptor) =>
val normalized = normalizeMetadataWithDescriptor(metadataWithDecodedPayload, descriptor)
val payloadTablePath = resolveTablePath(partitionColumnCount, file)
val serializedPayload = serializePayload(dvStore, payloadTablePath, descriptor)
serializedPayload.foreach {
payload =>
normalized.put(DeltaDvPayloadIndex, Int.box(deletionVectorPayloads.length))
deletionVectorPayloads += payload
}
normalizedMetadataColumns.add(normalized)
case None =>
normalizedMetadataColumns.add(metadataWithDecodedPayload)
}
}

new NormalizedSplitMetadata(normalizedMetadataColumns, deletionVectorPayloads.toArray)
}

private def activeSpark: SparkSession = {
SparkSession.getActiveSession
.orElse(SparkSession.getDefaultSession)
.getOrElse {
throw new IllegalStateException(
"Active SparkSession is required to materialize Delta deletion vectors")
}
}

private def resolveTablePath(partitionColumnCount: Int, file: PartitionedFile): Path = {
val fileParent = new Path(unescapePathName(file.filePath.toString)).getParent
var tablePath = fileParent
for (_ <- 0 until partitionColumnCount) {
tablePath = tablePath.getParent
}
val spark = activeSpark
if (tablePath != null && isDeltaTablePath(spark, tablePath)) {
return tablePath
}

// Spark can report a partition column count that does not map 1:1 to path depth for
// prepared Delta scans. Find the nearest ancestor of the file path that has _delta_log.
var candidate = fileParent
while (candidate != null && !isDeltaTablePath(spark, candidate)) {
candidate = candidate.getParent
}
if (candidate != null) candidate else tablePath
}

private def isDeltaTablePath(spark: SparkSession, tablePath: Path): Boolean = {
val deltaLogPath = new Path(tablePath, "_delta_log")
try {
deltaLogPath.getFileSystem(spark.sessionState.newHadoopConf()).exists(deltaLogPath)
} catch {
case NonFatal(_) => false
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,160 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.gluten.backendsapi.velox

import org.apache.gluten.backendsapi.velox.VeloxIteratorApi.unescapePathName
import org.apache.gluten.sql.shims.SparkShimLoader

import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.delta.actions.DeletionVectorDescriptor
import org.apache.spark.sql.delta.deletionvectors.{RoaringBitmapArrayFormat, StoredBitmap}
import org.apache.spark.sql.delta.storage.dv.HadoopFileSystemDVStore
import org.apache.spark.sql.execution.datasources.PartitionedFile

import org.apache.hadoop.fs.Path

import java.util.{ArrayList => JArrayList, HashMap => JHashMap, List => JList, Map => JMap}

import scala.collection.JavaConverters._
import scala.util.Try
import scala.util.control.NonFatal

object VeloxDeltaMetadataUtils {
val DeltaDvCardinality = "delta_dv_cardinality"
val DeltaDvPayloadIndex = "delta_dv_payload_index"

private val RowIndexFilterIdEncoded = "row_index_filter_id_encoded"
private val RowIndexFilterType = "row_index_filter_type"
private val RowIndexFilterTypeIfContained = "IF_CONTAINED"

final class NormalizedSplitMetadata(
val otherMetadataColumns: JList[JMap[String, Object]],
val deletionVectorPayloads: Array[Array[Byte]])
extends Serializable

private def decodeDescriptor(
normalizedMetadata: JMap[String, Object]): Option[DeletionVectorDescriptor] = {
Option(normalizedMetadata.get(RowIndexFilterIdEncoded))
.map(_.toString)
.filter(_.nonEmpty)
.flatMap(encoded => Try(DeletionVectorDescriptor.deserializeFromBase64(encoded)).toOption)
}

private def serializePayload(
dvStore: HadoopFileSystemDVStore,
tablePath: Path,
descriptor: DeletionVectorDescriptor): Option[Array[Byte]] = {
if (tablePath == null) {
return None
}
Some(
StoredBitmap
.create(descriptor, tablePath)
.load(dvStore)
.serializeAsByteArray(RoaringBitmapArrayFormat.Portable))
}

private def normalizeMetadataWithDescriptor(
metadata: JMap[String, Object],
descriptor: DeletionVectorDescriptor): JMap[String, Object] = {
val normalized = new JHashMap[String, Object]()
if (metadata != null) {
normalized.putAll(metadata)
}
normalized.put(DeltaDvCardinality, Long.box(descriptor.cardinality))
normalized.remove(RowIndexFilterIdEncoded)
if (!normalized.containsKey(RowIndexFilterType)) {
normalized.put(RowIndexFilterType, RowIndexFilterTypeIfContained)
}
normalized
}

def normalizeSplitMetadata(
partitionColumnCount: Int,
files: JList[PartitionedFile]): NormalizedSplitMetadata = {
val dvStore = new HadoopFileSystemDVStore(activeSpark.sessionState.newHadoopConf())
val normalizedMetadataColumns = new JArrayList[JMap[String, Object]](files.size())
val deletionVectorPayloads = scala.collection.mutable.ArrayBuffer.empty[Array[Byte]]

files.asScala.foreach {
file =>
val otherMetadata =
SparkShimLoader.getSparkShims.getOtherConstantMetadataColumnValues(file)
val metadataWithDecodedPayload = new JHashMap[String, Object]()
if (otherMetadata != null) {
metadataWithDecodedPayload.putAll(otherMetadata)
}

val descriptor = decodeDescriptor(metadataWithDecodedPayload)

descriptor match {
case Some(descriptor) =>
val normalized = normalizeMetadataWithDescriptor(metadataWithDecodedPayload, descriptor)
val payloadTablePath = resolveTablePath(partitionColumnCount, file)
val serializedPayload = serializePayload(dvStore, payloadTablePath, descriptor)
serializedPayload.foreach {
payload =>
normalized.put(DeltaDvPayloadIndex, Int.box(deletionVectorPayloads.length))
deletionVectorPayloads += payload
}
normalizedMetadataColumns.add(normalized)
case None =>
normalizedMetadataColumns.add(metadataWithDecodedPayload)
}
}

new NormalizedSplitMetadata(normalizedMetadataColumns, deletionVectorPayloads.toArray)
}

private def activeSpark: SparkSession = {
SparkSession.getActiveSession
.orElse(SparkSession.getDefaultSession)
.getOrElse {
throw new IllegalStateException(
"Active SparkSession is required to materialize Delta deletion vectors")
}
}

private def resolveTablePath(partitionColumnCount: Int, file: PartitionedFile): Path = {
val fileParent = new Path(unescapePathName(file.filePath.toString)).getParent
var tablePath = fileParent
for (_ <- 0 until partitionColumnCount) {
tablePath = tablePath.getParent
}
val spark = activeSpark
if (tablePath != null && isDeltaTablePath(spark, tablePath)) {
return tablePath
}

// Spark can report a partition column count that does not map 1:1 to path depth for
// prepared Delta scans. Find the nearest ancestor of the file path that has _delta_log.
var candidate = fileParent
while (candidate != null && !isDeltaTablePath(spark, candidate)) {
candidate = candidate.getParent
}
if (candidate != null) candidate else tablePath
}

private def isDeltaTablePath(spark: SparkSession, tablePath: Path): Boolean = {
val deltaLogPath = new Path(tablePath, "_delta_log")
try {
deltaLogPath.getFileSystem(spark.sessionState.newHadoopConf()).exists(deltaLogPath)
} catch {
case NonFatal(_) => false
}
}
}
Loading
Loading