Skip to content
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 @@ -25,16 +25,20 @@ import org.apache.sedona.sql.datasources.geopackage.model.{GeoPackageOptions, Ge
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{BoundReference, JoinedRow}
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory}
import org.apache.spark.sql.execution.datasources.FilePartition
import org.apache.spark.sql.types.StructType
import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.util.SerializableConfiguration

case class GeoPackagePartitionReaderFactory(
sparkSession: SparkSession,
broadcastedConf: Broadcast[SerializableConfiguration],
loadOptions: GeoPackageOptions,
dataSchema: StructType)
dataSchema: StructType,
metadataSchema: StructType)
extends PartitionReaderFactory {

override def createReader(partition: InputPartition): PartitionReader[InternalRow] = {
Expand Down Expand Up @@ -72,7 +76,7 @@ case class GeoPackagePartitionReaderFactory(
case _ => None
}

GeoPackagePartitionReader(
val baseReader = GeoPackagePartitionReader(
rs = rs,
options = GeoPackageReadOptions(
tableName = loadOptions.tableName,
Expand All @@ -84,5 +88,52 @@ case class GeoPackagePartitionReaderFactory(
broadcastedConf = broadcastedConf,
currentTempFile = tempFile,
copying = copied)

if (metadataSchema.nonEmpty) {
val gpkgFile = partitionFiles.head
val filePath = gpkgFile.filePath.toString
val fileName = new Path(filePath).getName

val allMetadataValues: Map[String, Any] = Map(
"file_path" -> UTF8String.fromString(filePath),
"file_name" -> UTF8String.fromString(fileName),
"file_size" -> gpkgFile.fileSize,
"file_block_start" -> gpkgFile.start,
"file_block_length" -> gpkgFile.length,
"file_modification_time" -> (gpkgFile.modificationTime * 1000L))

val innerStructType = metadataSchema.fields.head.dataType.asInstanceOf[StructType]
val prunedValues = innerStructType.fields.map(f => allMetadataValues(f.name))
val metadataStruct = InternalRow.fromSeq(prunedValues.toSeq)
val metadataRow = InternalRow.fromSeq(Seq(metadataStruct))

new PartitionReaderWithMetadata(baseReader, dataSchema, metadataSchema, metadataRow)
} else {
baseReader
}
}
}

private[geopackage] class PartitionReaderWithMetadata(
reader: PartitionReader[InternalRow],
baseSchema: StructType,
metadataSchema: StructType,
metadataValues: InternalRow)
extends PartitionReader[InternalRow] {

private val joinedRow = new JoinedRow()
private val unsafeProjection =
GenerateUnsafeProjection.generate(baseSchema.fields.zipWithIndex.map { case (f, i) =>
BoundReference(i, f.dataType, f.nullable)
} ++ metadataSchema.fields.zipWithIndex.map { case (f, i) =>
BoundReference(baseSchema.length + i, f.dataType, f.nullable)
})

override def next(): Boolean = reader.next()

override def get(): InternalRow = {
unsafeProjection(joinedRow(reader.get(), metadataValues))
}

override def close(): Unit = reader.close()
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,10 +36,14 @@ case class GeoPackageScan(
fileIndex: PartitioningAwareFileIndex,
readDataSchema: StructType,
readPartitionSchema: StructType,
metadataSchema: StructType,
options: CaseInsensitiveStringMap,
loadOptions: GeoPackageOptions)
extends FileScan {

override def readSchema(): StructType =
StructType(readDataSchema.fields ++ readPartitionSchema.fields ++ metadataSchema.fields)

override def partitionFilters: Seq[Expression] = {
Seq.empty
}
Expand All @@ -54,6 +58,11 @@ case class GeoPackageScan(
val broadcastedConf =
sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))

GeoPackagePartitionReaderFactory(sparkSession, broadcastedConf, loadOptions, dataSchema)
GeoPackagePartitionReaderFactory(
sparkSession,
broadcastedConf,
loadOptions,
dataSchema,
metadataSchema)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,18 @@ class GeoPackageScanBuilder(
userDefinedSchema: Option[StructType] = None)
extends FileScanBuilder(sparkSession, fileIndex, dataSchema) {

private var _requiredMetadataSchema: StructType = StructType(Seq.empty)

override def pruneColumns(requiredSchema: StructType): Unit = {
val resolver = sparkSession.sessionState.conf.resolver
val metaFields = requiredSchema.fields.filter { field =>
!dataSchema.fields.exists(df => resolver(df.name, field.name)) &&
!fileIndex.partitionSchema.fields.exists(pf => resolver(pf.name, field.name))
}
_requiredMetadataSchema = StructType(metaFields)
super.pruneColumns(requiredSchema)
}

override def build(): Scan = {
val paths = fileIndex.allFiles().map(_.getPath.toString)

Expand All @@ -54,6 +66,7 @@ class GeoPackageScanBuilder(
fileIndexAdjusted,
dataSchema,
readPartitionSchema(),
_requiredMetadataSchema,
options,
loadOptions)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,12 @@ import org.apache.hadoop.fs.FileStatus
import org.apache.sedona.sql.datasources.geopackage.connection.{FileSystemUtils, GeoPackageConnectionManager}
import org.apache.sedona.sql.datasources.geopackage.model.{GeoPackageOptions, MetadataSchema, TableType}
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.connector.catalog.{MetadataColumn, SupportsMetadataColumns}
import org.apache.spark.sql.connector.read.ScanBuilder
import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder}
import org.apache.spark.sql.execution.datasources.FileFormat
import org.apache.spark.sql.execution.datasources.v2.FileTable
import org.apache.spark.sql.types.{DoubleType, IntegerType, StringType, StructField, StructType, TimestampType}
import org.apache.spark.sql.types.{DataType, DoubleType, IntegerType, LongType, StringType, StructField, StructType, TimestampType}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
import org.apache.spark.util.SerializableConfiguration

Expand All @@ -40,7 +41,8 @@ case class GeoPackageTable(
userSpecifiedSchema: Option[StructType],
fallbackFileFormat: Class[_ <: FileFormat],
loadOptions: GeoPackageOptions)
extends FileTable(sparkSession, options, paths, userSpecifiedSchema) {
extends FileTable(sparkSession, options, paths, userSpecifiedSchema)
with SupportsMetadataColumns {

override def inferSchema(files: Seq[FileStatus]): Option[StructType] = {
if (loadOptions.showMetadata) {
Expand Down Expand Up @@ -74,6 +76,8 @@ case class GeoPackageTable(
"GeoPackage"
}

override def metadataColumns(): Array[MetadataColumn] = GeoPackageTable.fileMetadataColumns

override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
new GeoPackageScanBuilder(
sparkSession,
Expand All @@ -88,3 +92,21 @@ case class GeoPackageTable(
null
}
}

object GeoPackageTable {

private val FILE_METADATA_STRUCT_TYPE: StructType = StructType(
Seq(
StructField("file_path", StringType, nullable = false),
StructField("file_name", StringType, nullable = false),
StructField("file_size", LongType, nullable = false),
StructField("file_block_start", LongType, nullable = false),
StructField("file_block_length", LongType, nullable = false),
StructField("file_modification_time", TimestampType, nullable = false)))

private[geopackage] val fileMetadataColumns: Array[MetadataColumn] = Array(new MetadataColumn {
override def name: String = "_metadata"
override def dataType: DataType = FILE_METADATA_STRUCT_TYPE
override def isNullable: Boolean = false
})
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@
package org.apache.sedona.sql

import io.minio.{MakeBucketArgs, MinioClient}
import org.apache.spark.sql.DataFrame
import org.apache.spark.sql.{DataFrame, Row}
Copy link

Copilot AI Feb 15, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The Row import is unused and should be removed. The tests do not create or use Row objects directly.

Suggested change
import org.apache.spark.sql.{DataFrame, Row}
import org.apache.spark.sql.DataFrame

Copilot uses AI. Check for mistakes.
import org.apache.spark.sql.functions.expr
import org.apache.spark.sql.sedona_sql.UDT.GeometryUDT
import org.apache.spark.sql.types._
Expand Down Expand Up @@ -321,6 +321,78 @@ class GeoPackageReaderTest extends TestBaseScala with Matchers {
}
}

describe("_metadata hidden column support") {
it("should expose _metadata struct with all expected fields") {
val df = readFeatureData("point1")
val metaDf = df.select("_metadata")
val metaSchema = metaDf.schema.fields.head.dataType.asInstanceOf[StructType]
val fieldNames = metaSchema.fieldNames.toSet
fieldNames should contain("file_path")
fieldNames should contain("file_name")
fieldNames should contain("file_size")
fieldNames should contain("file_block_start")
fieldNames should contain("file_block_length")
fieldNames should contain("file_modification_time")
}

it("should not include _metadata in select(*)") {
val df = readFeatureData("point1")
val starCols = df.select("*").columns.toSet
starCols should not contain "_metadata"
}

it("should return correct file_path and file_name in _metadata") {
val df = readFeatureData("point1")
val row = df.select("_metadata.file_path", "_metadata.file_name").head()
val filePath = row.getString(0)
val fileName = row.getString(1)
filePath should endWith("example.gpkg")
fileName shouldEqual "example.gpkg"
}

it("should return actual file_size matching the .gpkg file on disk") {
val df = readFeatureData("point1")
val metaFileSize = df.select("_metadata.file_size").head().getLong(0)
val actualFile = new java.io.File(path)
metaFileSize shouldEqual actualFile.length()
}

it("should return file_block_start=0 and file_block_length=file_size") {
val df = readFeatureData("point1")
val row = df
.select(
"_metadata.file_block_start",
"_metadata.file_block_length",
"_metadata.file_size")
.head()
row.getLong(0) shouldEqual 0L
row.getLong(1) shouldEqual row.getLong(2)
}

it("should return file_modification_time matching the .gpkg file on disk") {
val df = readFeatureData("point1")
val metaModTime = df.select("_metadata.file_modification_time").head().getTimestamp(0)
val actualFile = new java.io.File(path)
val expectedModTime = new java.sql.Timestamp(actualFile.lastModified())
metaModTime shouldEqual expectedModTime
}

it("should allow filtering on _metadata fields") {
val df = readFeatureData("point1")
val filtered = df.filter(df("_metadata.file_name") === "example.gpkg")
filtered.count() shouldEqual df.count()
val empty = df.filter(df("_metadata.file_name") === "nonexistent.gpkg")
empty.count() shouldEqual 0
}

it("should select _metadata along with data columns") {
val df = readFeatureData("point1")
val result = df.select("id", "_metadata.file_name").head()
result.getInt(0) shouldEqual 1
result.getString(1) shouldEqual "example.gpkg"
}
}

private def readFeatureData(tableName: String): DataFrame = {
sparkSession.read
.format("geopackage")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,16 +25,20 @@ import org.apache.sedona.sql.datasources.geopackage.model.{GeoPackageOptions, Ge
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{BoundReference, JoinedRow}
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory}
import org.apache.spark.sql.execution.datasources.FilePartition
import org.apache.spark.sql.types.StructType
import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.util.SerializableConfiguration

case class GeoPackagePartitionReaderFactory(
sparkSession: SparkSession,
broadcastedConf: Broadcast[SerializableConfiguration],
loadOptions: GeoPackageOptions,
dataSchema: StructType)
dataSchema: StructType,
metadataSchema: StructType)
extends PartitionReaderFactory {

override def createReader(partition: InputPartition): PartitionReader[InternalRow] = {
Expand Down Expand Up @@ -72,7 +76,7 @@ case class GeoPackagePartitionReaderFactory(
case _ => None
}

GeoPackagePartitionReader(
val baseReader = GeoPackagePartitionReader(
rs = rs,
options = GeoPackageReadOptions(
tableName = loadOptions.tableName,
Expand All @@ -84,5 +88,52 @@ case class GeoPackagePartitionReaderFactory(
broadcastedConf = broadcastedConf,
currentTempFile = tempFile,
copying = copied)

if (metadataSchema.nonEmpty) {
val gpkgFile = partitionFiles.head
val filePath = gpkgFile.filePath.toString
val fileName = new Path(filePath).getName

val allMetadataValues: Map[String, Any] = Map(
"file_path" -> UTF8String.fromString(filePath),
"file_name" -> UTF8String.fromString(fileName),
"file_size" -> gpkgFile.fileSize,
"file_block_start" -> gpkgFile.start,
"file_block_length" -> gpkgFile.length,
"file_modification_time" -> (gpkgFile.modificationTime * 1000L))

val innerStructType = metadataSchema.fields.head.dataType.asInstanceOf[StructType]
val prunedValues = innerStructType.fields.map(f => allMetadataValues(f.name))
val metadataStruct = InternalRow.fromSeq(prunedValues.toSeq)
val metadataRow = InternalRow.fromSeq(Seq(metadataStruct))

new PartitionReaderWithMetadata(baseReader, dataSchema, metadataSchema, metadataRow)
} else {
baseReader
}
}
}

private[geopackage] class PartitionReaderWithMetadata(
reader: PartitionReader[InternalRow],
baseSchema: StructType,
metadataSchema: StructType,
metadataValues: InternalRow)
extends PartitionReader[InternalRow] {

private val joinedRow = new JoinedRow()
private val unsafeProjection =
GenerateUnsafeProjection.generate(baseSchema.fields.zipWithIndex.map { case (f, i) =>
BoundReference(i, f.dataType, f.nullable)
} ++ metadataSchema.fields.zipWithIndex.map { case (f, i) =>
BoundReference(baseSchema.length + i, f.dataType, f.nullable)
})

override def next(): Boolean = reader.next()

override def get(): InternalRow = {
unsafeProjection(joinedRow(reader.get(), metadataValues))
}

override def close(): Unit = reader.close()
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,10 +36,14 @@ case class GeoPackageScan(
fileIndex: PartitioningAwareFileIndex,
readDataSchema: StructType,
readPartitionSchema: StructType,
metadataSchema: StructType,
options: CaseInsensitiveStringMap,
loadOptions: GeoPackageOptions)
extends FileScan {

override def readSchema(): StructType =
StructType(readDataSchema.fields ++ readPartitionSchema.fields ++ metadataSchema.fields)

override def partitionFilters: Seq[Expression] = {
Seq.empty
}
Expand All @@ -54,6 +58,11 @@ case class GeoPackageScan(
val broadcastedConf =
sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))

GeoPackagePartitionReaderFactory(sparkSession, broadcastedConf, loadOptions, dataSchema)
GeoPackagePartitionReaderFactory(
sparkSession,
broadcastedConf,
loadOptions,
dataSchema,
metadataSchema)
}
}
Loading
Loading