From c7f9e2f7306365b2ad2e1a12c2a057bc9098415b Mon Sep 17 00:00:00 2001 From: jackylee-ch Date: Tue, 2 Jun 2026 15:28:03 +0800 Subject: [PATCH] [VL][TableCache] Add default lazy per-column deserialization Write V3 per-column cache bytes by default for Velox table cache. Partition stats now only controls the optional stats/pruning payload: stats off writes a no-stats V3 frame, stats on writes V3 with stats, and older native libraries still fall back to V2 stats or legacy bytes. Add the V3 no-stats JNI/native serializer, JVM parsing for statsLen=0, cross-language golden coverage, and GitHub Actions benchmark execution without committing local benchmark results. Change-Id: I2a8582f901fafd436cac1a1d16e0367e9330b336 --- .../ColumnarCachedBatchSerializer.scala | 371 +++++++++++++-- ...mnarCachedBatchBuildFilterPruneSuite.scala | 50 ++ .../ColumnarCachedBatchE2ESuite.scala | 61 ++- .../ColumnarCachedBatchFramedBytesSuite.scala | 281 +++++++++++ .../ColumnarCachedBatchLazySerdeTest.scala | 287 +++++++++++ ...mnarCachedBatchSerializerHelperSuite.scala | 143 +++++- ...ColumnarTableCacheLazyDeserBenchmark.scala | 366 +++++++++++++++ cpp/core/jni/JniWrapper.cc | 83 +++- .../serializer/ColumnarBatchSerializer.h | 31 +- cpp/core/utils/tac/ffor.hpp | 3 +- .../VeloxColumnarBatchSerializer.cc | 444 +++++++++++++++++- .../serializer/VeloxColumnarBatchSerializer.h | 26 +- .../tests/VeloxColumnarBatchSerializerTest.cc | 381 ++++++++++++++- docs/Configuration.md | 222 ++++----- .../ColumnarBatchSerializerJniWrapper.java | 18 +- .../apache/gluten/config/GlutenConfig.scala | 8 +- .../spark/sql/GlutenCachedTableSuite.scala | 2 +- .../spark/sql/GlutenCachedTableSuite.scala | 2 +- .../spark/sql/GlutenCachedTableSuite.scala | 2 +- .../spark/sql/GlutenCachedTableSuite.scala | 2 +- .../spark/sql/GlutenCachedTableSuite.scala | 2 +- 21 files changed, 2546 insertions(+), 239 deletions(-) create mode 100644 backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchLazySerdeTest.scala create mode 100644 backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnarTableCacheLazyDeserBenchmark.scala diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala index 85c9d2854c7..093477cb728 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala @@ -246,6 +246,52 @@ object CachedColumnarBatchKryoSerializer { val STATS_FRAMED_MAGIC: Array[Byte] = Array[Byte](0xfe.toByte, 0xca.toByte, 0x53.toByte, 0x02.toByte) + // V3 magic: same as V2 but last byte = 0x03. + val STATS_FRAMED_MAGIC_V3: Array[Byte] = + Array[Byte](0xfe.toByte, 0xca.toByte, 0x53.toByte, 0x03.toByte) + + private case class V3ParsedFrame(stats: InternalRow, bytes: Array[Byte], numRows: Int) + + private def magicHex(bytes: Array[Byte]): String = { + if (bytes == null || bytes.length < 4) { + "" + } else { + f"0x${bytes(0) & 0xff}%02X${bytes(1) & 0xff}%02X" + + f"${bytes(2) & 0xff}%02X${bytes(3) & 0xff}%02X" + } + } + + private[execution] def hasFrameMagic(bytes: Array[Byte], magic: Array[Byte]): Boolean = { + bytes != null && bytes.length >= magic.length && { + var i = 0 + while (i < magic.length) { + if (bytes(i) != magic(i)) { + return false + } + i += 1 + } + true + } + } + + private def requireFrameMagic(bytes: Array[Byte], magic: Array[Byte], version: String): Unit = { + require( + hasFrameMagic(bytes, magic), + s"$version framed bytes magic mismatch: expected ${magicHex(magic)}, got ${magicHex(bytes)}") + } + + private def framedMagicVersion(framed: Array[Byte]): Int = { + if (hasFrameMagic(framed, STATS_FRAMED_MAGIC)) { + 0x02 + } else if (hasFrameMagic(framed, STATS_FRAMED_MAGIC_V3)) { + 0x03 + } else { + throw new IllegalArgumentException( + s"framed bytes magic mismatch: expected ${magicHex(STATS_FRAMED_MAGIC)}(V2) or " + + s"${magicHex(STATS_FRAMED_MAGIC_V3)}(V3), got ${magicHex(framed)}") + } + } + // Per-column statsBlob layout (LE throughout, matches the cpp emitter in // VeloxColumnarBatchSerializer.cc): // @@ -617,45 +663,109 @@ object CachedColumnarBatchKryoSerializer { } /** - * Parse the JNI `serializeWithStats` framed return into (stats InternalRow, bytesBlob). - * - * Framed layout (matches cpp VeloxColumnarBatchSerializer.cc): `[ STATS_FRAMED_MAGIC: 4B ] [ - * statsLen: u32 LE ] [ statsBlob ] [ bytesLen: u32 LE ] [ bytesBlob ]`. + * Parse the JNI `serializeWithStats` framed return into (stats InternalRow, bytesBlob). Routes on + * the full 4-byte magic: V2 -> 0xFECA5302, V3 -> 0xFECA5303. * - * Eager guards catch corrupt magic / truncated framing before they propagate. + * V2 layout: `[ magic: 4B ] [ statsLen: u32 LE ] [ statsBlob ] [ bytesLen: u32 LE ] [ bytesBlob + * ]` V3 layout: `[ magic: 4B ] [ statsLen: u32 LE ] [ statsBlob ] [ numRows: u32 LE ] [ numCols: + * u32 LE ] [ per-col ]` */ private[execution] def parseFramedBytes( framed: Array[Byte], schema: StructType): (InternalRow, Array[Byte]) = { + // V2 minimum = 4+4+4=12B; V3 minimum = 4+4+4+4=16B; use 12 for dispatcher guard. require( - framed != null && framed.length >= 4 + 4 + 4, + framed != null && framed.length >= 12, s"framed bytes too short: len=${if (framed == null) -1 else framed.length}") - require( - framed(0) == STATS_FRAMED_MAGIC(0) && framed(1) == STATS_FRAMED_MAGIC(1) && - framed(2) == STATS_FRAMED_MAGIC(2) && framed(3) == STATS_FRAMED_MAGIC(3), - f"framed bytes magic mismatch: expected " + - f"0x${STATS_FRAMED_MAGIC(0) & 0xff}%02X${STATS_FRAMED_MAGIC(1) & 0xff}%02X" + - f"${STATS_FRAMED_MAGIC(2) & 0xff}%02X${STATS_FRAMED_MAGIC(3) & 0xff}%02X, got " + - f"0x${framed(0) & 0xff}%02X${framed(1) & 0xff}%02X" + - f"${framed(2) & 0xff}%02X${framed(3) & 0xff}%02X" - ) + framedMagicVersion(framed) match { + case 0x02 => parseV2Frame(framed, schema) + case 0x03 => parseV3Frame(framed, schema) + } + } + + /** V2 parse: extract stats + pure Presto bytesBlob. */ + private def parseV2Frame(framed: Array[Byte], schema: StructType): (InternalRow, Array[Byte]) = { + requireFrameMagic(framed, STATS_FRAMED_MAGIC, "V2") val buf = ByteBuffer.wrap(framed).order(ByteOrder.LITTLE_ENDIAN) buf.position(4) // skip magic val statsLen = buf.getInt require( statsLen >= 0 && statsLen <= buf.remaining() - 4, - s"framed bytes statsLen=$statsLen exceeds remaining buffer ${buf.remaining() - 4}") + s"V2 framed bytes statsLen=$statsLen exceeds remaining buffer ${buf.remaining() - 4}") val statsBlob = new Array[Byte](statsLen) buf.get(statsBlob) val stats = deserializeStats(statsBlob, schema) val bytesLen = buf.getInt require( bytesLen >= 0 && bytesLen == buf.remaining(), - s"framed bytes bytesLen=$bytesLen != remaining ${buf.remaining()} (truncated or trailing)") + s"V2 framed bytes bytesLen=$bytesLen != remaining ${buf.remaining()} (truncated or trailing)") val bytesBlob = new Array[Byte](bytesLen) buf.get(bytesBlob) (stats, bytesBlob) } + + /** + * V3 parse: extract stats; bytes = the full V3 framed array (C++ deserializeV3 starts at magic). + * Invariant: returned bytes[0..3] == V3 magic; C++ deserializeV3 re-validates the schema-level + * contract, while the JVM parser fails fast on top-level frame bounds. + */ + private def parseV3Frame(framed: Array[Byte], schema: StructType): (InternalRow, Array[Byte]) = { + val parsed = parseV3FrameInternal(framed, schema, decodeStats = true) + (parsed.stats, parsed.bytes) + } + + private[execution] def requireV3FrameNumRows( + framed: Array[Byte], + expectedNumRows: Int, + context: String): Unit = { + val frameNumRows = parseV3FrameInternal(framed, null, decodeStats = false).numRows + require( + frameNumRows == expectedNumRows, + s"$context: V3 frame numRows=$frameNumRows != CachedBatch numRows=$expectedNumRows") + } + + private def parseV3FrameInternal( + framed: Array[Byte], + schema: StructType, + decodeStats: Boolean): V3ParsedFrame = { + require(framed.length >= 16, s"V3 framed bytes too short (min 16B): len=${framed.length}") + requireFrameMagic(framed, STATS_FRAMED_MAGIC_V3, "V3") + val buf = ByteBuffer.wrap(framed).order(ByteOrder.LITTLE_ENDIAN) + buf.position(4) // skip magic + val statsLen = buf.getInt + require( + statsLen >= 0 && statsLen <= buf.remaining() - 8, // 8 = numRows(4)+numCols(4) + s"V3 framed bytes statsLen=$statsLen invalid") + val statsBlob = new Array[Byte](statsLen) + buf.get(statsBlob) + val stats = + if (!decodeStats || statsLen == 0) null else deserializeStats(statsBlob, schema) + val numRows = buf.getInt + require(numRows >= 0, s"V3 framed bytes numRows=$numRows invalid") + val numCols = buf.getInt + require(numCols >= 0, s"V3 framed bytes numCols=$numCols invalid") + var col = 0 + while (col < numCols) { + require( + buf.remaining() >= 4, + s"V3 framed bytes truncated before colLen col=$col") + val colLen = buf.getInt + require( + colLen >= 0, + s"V3 framed bytes colLen=$colLen invalid at col=$col") + require( + colLen <= buf.remaining(), + s"V3 framed bytes colBytes truncated at col=$col: colLen=$colLen " + + s"remaining=${buf.remaining()}") + buf.position(buf.position() + colLen) + col += 1 + } + require( + buf.remaining() == 0, + s"V3 framed bytes has trailing bytes after column payloads: trailing=${buf.remaining()}") + // Return full framed bytes; C++ deserializeV3 will skip magic+stats and per-col. + V3ParsedFrame(stats, framed, numRows) + } } /** @@ -762,6 +872,7 @@ class ColumnarCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer val structSchema = StructType( schema.map(a => StructField(a.name, a.dataType, a.nullable))) val backendName = BackendsApiManager.getBackendName + // Hoist partition-level configs: GlutenConfig.get allocates a fresh object on each call. val partitionStatsEnabled = GlutenConfig.get.getConf(GlutenConfig.COLUMNAR_TABLE_CACHE_PARTITION_STATS_ENABLED) val jni = ColumnarBatchSerializerJniWrapper.create( @@ -784,13 +895,31 @@ class ColumnarCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer stats = null, schema = null) } - // Route through serializeWithStats when the partition-stats conf is enabled and the - // JNI extension is linked in libgluten.so. Capability is detected lazily at the - // call site: a new Gluten jar paired with an older native library will throw - // UnsatisfiedLinkError on the first invocation; we catch it once, cache the - // result, and fall back to the legacy serialize() path emitting stats=null. The - // buildFilter wrapper directs such batches through without pruning. - if (partitionStatsEnabled && ColumnarCachedBatchSerializer.statsExtAvailable) { + def statsOrLegacySerializeInline(): CachedBatch = { + if (partitionStatsEnabled && ColumnarCachedBatchSerializer.statsExtAvailable) { + ColumnarCachedBatchSerializer.serializeOneBatchWithStats( + jni, + handle, + batch.numRows(), + structSchema, + () => legacySerializeInline()) + } else { + legacySerializeInline() + } + } + // V3 is the default cache format for Velox table cache: it stores each column + // independently so reads can materialize only requested columns. Partition stats are + // an optional V3 payload used for pruning, not a prerequisite for lazy reads. + if (ColumnarCachedBatchSerializer.statsExtV3Available) { + ColumnarCachedBatchSerializer.serializeOneBatchV3( + jni, + handle, + batch.numRows(), + structSchema, + includeStats = partitionStatsEnabled, + fallbackToV2OrLegacy = () => statsOrLegacySerializeInline()) + } else if (partitionStatsEnabled && ColumnarCachedBatchSerializer.statsExtAvailable) { + // V2 stats path. ColumnarCachedBatchSerializer.serializeOneBatchWithStats( jni, handle, @@ -820,8 +949,17 @@ class ColumnarCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer conf) } else { // Find the ordinals and data types of the requested columns. + val cacheExprIds = cacheAttributes.map(_.exprId) + val cachedAttrNames = + cacheAttributes.map(a => s"${a.name}#${a.exprId.id}").mkString("[", ",", "]") val requestedColumnIndices = selectedAttributes.map { - a => cacheAttributes.map(_.exprId).indexOf(a.exprId) + a => + val idx = cacheExprIds.indexOf(a.exprId) + require( + idx >= 0, + s"selected cache attribute ${a.name}#${a.exprId.id} is not present in cached " + + s"attributes $cachedAttrNames") + idx } val shouldSelectAttributes = cacheAttributes != selectedAttributes val localSchema = toStructType(cacheAttributes) @@ -847,21 +985,41 @@ class ColumnarCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer override def next(): ColumnarBatch = { val cachedBatch = it.next().asInstanceOf[CachedColumnarBatch] - val batchHandle = - jniWrapper - .deserialize(deserializerHandle, cachedBatch.bytes) - val batch = ColumnarBatches.create(batchHandle) - if (shouldSelectAttributes) { - try { - ColumnarBatches.select( - BackendsApiManager.getBackendName, - batch, - requestedColumnIndices.toArray) - } finally { - batch.close() - } + // V3 bytes are ALWAYS routed to deserializeWithProjection. + // V3 framed bytes must NOT go to jni.deserialize() (expects Presto format). + if (isV3Format(cachedBatch.bytes)) { + CachedColumnarBatchKryoSerializer.requireV3FrameNumRows( + cachedBatch.bytes, + cachedBatch.numRows, + "deserialize V3 cached batch") + // C++ returns the requested M-column batch; LazyVector loads those columns + // on first access instead of eagerly decoding the full cached schema. + val reqIndices: Array[Int] = + if (cacheAttributes == selectedAttributes) null // all cols: C++ loadAll + else if (requestedColumnIndices.isEmpty) Array.empty[Int] // count(*): 0 cols + else requestedColumnIndices.toArray // projection: M cols + val batchHandle = jniWrapper.deserializeWithProjection( + deserializerHandle, + cachedBatch.bytes, + reqIndices) + ColumnarBatches.create(batchHandle) + // No ColumnarBatches.select(): C++ returns M-column batch. } else { - batch + // V2 path (original logic). + val batchHandle = jniWrapper.deserialize(deserializerHandle, cachedBatch.bytes) + val batch = ColumnarBatches.create(batchHandle) + if (shouldSelectAttributes) { + try { + ColumnarBatches.select( + BackendsApiManager.getBackendName, + batch, + requestedColumnIndices.toArray) + } finally { + batch.close() + } + } else { + batch + } } } }) @@ -910,6 +1068,12 @@ class ColumnarCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer } } + /** True iff bytes starts with V3 magic (0xFE 0xCA 0x53 0x03). */ + private def isV3Format(bytes: Array[Byte]): Boolean = + CachedColumnarBatchKryoSerializer.hasFrameMagic( + bytes, + CachedColumnarBatchKryoSerializer.STATS_FRAMED_MAGIC_V3) + override def buildFilter( predicates: Seq[Expression], cachedAttributes: Seq[Attribute]) @@ -920,9 +1084,25 @@ class ColumnarCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer // predicate vector; empty filteredPredicates degrade gracefully because // super reduces partitionFilters with .reduceOption(And).getOrElse(Literal(true)) // -- verified against spark-sql_2.13-4.0.1-sources CachedBatchSerializer.scala. - val parent = super.buildFilter( - stripUnsupportedConjuncts(predicates, cachedAttributes), - cachedAttributes) + val strippedPredicates = stripUnsupportedConjuncts(predicates, cachedAttributes) + val parent = super.buildFilter(strippedPredicates, cachedAttributes) + // Cached-column ordinals (each maps to a 5-slot group in the stats InternalRow) referenced by + // the surviving predicates. A batch whose stats row carries a null min/max bound for any of + // these columns must NOT be pruned by the vanilla parent: vanilla turns `col = l` / `col <= l` + // into bound comparisons that evaluate to null on null bounds -> coerced false -> `!eval` -> + // the batch is silently dropped (data loss). Such per-batch null bounds come from + // data-dependent writer demotions that the schema-level stripUnsupportedConjuncts cannot see: + // e.g. a binary-collation VARCHAR whose 256B upper-bound prefix is all 0xFF (carry overflow, + // VeloxColumnarBatchSerializer.cc), or a dictionary-encoded numeric on the V2 fallback path. + // Those batches are routed through unchanged, exactly like stats==null batches. Float/double + // NaN columns no longer demote (the writer skips NaN), so they keep finite bounds and prune + // normally. + val referencedOrdinals: Set[Int] = { + val refIds = strippedPredicates.flatMap(_.references.map(_.exprId)).toSet + cachedAttributes.zipWithIndex.collect { + case (a, i) if refIds.contains(a.exprId) => i + }.toSet + } (index, cachedBatchIterator) => new Iterator[CachedBatch] { private val peekable = cachedBatchIterator.buffered @@ -932,16 +1112,16 @@ class ColumnarCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer // safe to call from both hasNext and next. private def advance(): Unit = { while (!staged.hasNext && peekable.hasNext) { - val stats = statsOf(peekable.head) - if (stats == null) { - // Pass through: do NOT feed to parent, which would NPE on null stats. + if (bypassPruning(peekable.head)) { + // Pass through: do NOT feed to parent, which would NPE on null stats or wrongly + // prune a batch on a null per-column bound. staged = Iterator.single(peekable.next()) } else { // Feed parent a self-terminating sub-iterator covering the contiguous run of - // stats!=null batches; loop afterwards in case parent prunes everything in the run. + // prunable batches; loop afterwards in case parent prunes everything in the run. val runIt = new Iterator[CachedBatch] { override def hasNext: Boolean = - peekable.hasNext && statsOf(peekable.head) != null + peekable.hasNext && !bypassPruning(peekable.head) override def next(): CachedBatch = peekable.next() } staged = parent(index, runIt) @@ -955,6 +1135,22 @@ class ColumnarCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer case _ => null } + // A batch skips vanilla pruning when it has no stats at all, or when any + // predicate-referenced column has a null lower/upper bound in this batch's stats row + // (vanilla would otherwise prune it to null -> drop; see referencedOrdinals). + private def bypassPruning(batch: CachedBatch): Boolean = { + val stats = statsOf(batch) + if (stats == null) { + true + } else { + referencedOrdinals.exists { + ci => + val base = ci * 5 + base + 1 < stats.numFields && (stats.isNullAt(base) || stats.isNullAt(base + 1)) + } + } + } + override def hasNext: Boolean = { advance() staged.hasNext @@ -1041,4 +1237,85 @@ object ColumnarCachedBatchSerializer extends Logging { ) } } + + // Visible for testing: reset the capability flag so a unit test can re-exercise the + // probe-once semantics. + private[execution] def resetStatsExtAvailableForTesting(): Unit = { + statsExtAvailableFlag = true + } + + // V3 lazy deserialization support + + // Separate capability latch for the V3 JNI symbols + // (framedSerializeV3 / framedSerializeWithStatsV3). + @volatile private var statsExtV3AvailableFlag: Boolean = true + + def statsExtV3Available: Boolean = statsExtV3AvailableFlag + + // Benchmark-only hook used by ColumnarTableCacheLazyDeserBenchmark to materialize the old + // eager/raw cache bytes as a baseline. This is intentionally package-private and not a user + // configuration: V3 lazy deserialization remains the production default. + private[execution] def withStatsExtV3AvailabilityForBenchmark[T](available: Boolean)( + f: => T): T = synchronized { + val previous = statsExtV3AvailableFlag + statsExtV3AvailableFlag = available + try { + f + } finally { + statsExtV3AvailableFlag = previous + } + } + + def markStatsExtV3Unavailable(cause: Throwable): Unit = { + if (statsExtV3AvailableFlag) { + statsExtV3AvailableFlag = false + logWarning( + "V3 table cache serialization JNI path is unavailable; " + + "disabling V3 per-column lazy deserialization for this JVM. " + + "This typically indicates a Gluten jar / native library version mismatch.", + cause + ) + } + } + + // V3 per-batch serialization: identical two-arm catch structure to serializeOneBatchWithStats. + // null return from JNI = non-Velox backend; treated as one-shot latch, not corrupt frame. + private[execution] def serializeOneBatchV3( + jni: ColumnarBatchSerializerJniWrapper, + handle: Long, + numRows: Int, + structSchema: StructType, + includeStats: Boolean, + fallbackToV2OrLegacy: () => CachedBatch): CachedBatch = { + try { + val framed = + if (includeStats) jni.serializeWithStatsV3(handle) + else jni.serializeV3(handle) + if (framed == null) { + // Non-Velox backend returns null; set latch and fall back. + markStatsExtV3Unavailable( + new RuntimeException("framedSerializeV3 returned null (backend not supported)")) + return fallbackToV2OrLegacy() + } + CachedColumnarBatchKryoSerializer.requireV3FrameNumRows( + framed, + numRows, + "serialize V3 cached batch") + val (stats, _) = CachedColumnarBatchKryoSerializer.parseFramedBytes(framed, structSchema) + // bytes = full V3 frame (C++ deserializeV3 parses from byte 0 including magic). + CachedColumnarBatch( + numRows, + framed.length, + framed, + stats, + schema = if (stats == null) null else structSchema) + } catch { + case e: UnsatisfiedLinkError => + markStatsExtV3Unavailable(e) + fallbackToV2OrLegacy() + case NonFatal(e) => + warnCorruptStatsFrame(e) // count against shared corrupt-frame cap + fallbackToV2OrLegacy() + } + } } diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchBuildFilterPruneSuite.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchBuildFilterPruneSuite.scala index 151cd400150..35a9edb14bf 100644 --- a/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchBuildFilterPruneSuite.scala +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchBuildFilterPruneSuite.scala @@ -99,6 +99,56 @@ class ColumnarCachedBatchBuildFilterPruneSuite extends AnyFunSuite { "order: stats=null pass-through first, then stats-covers-literal kept") } + // Regression: a batch whose stats row is non-null but carries a NULL lower/upper bound for a + // predicate-referenced column must NOT be pruned. Such per-batch null bounds arise from + // data-dependent writer demotions invisible to the schema-level stripUnsupportedConjuncts: + // a binary-collation VARCHAR whose 256B upper-bound prefix is all 0xFF (carry overflow), or a + // dictionary-encoded numeric on the V2 fallback path. Without the per-batch bypass, vanilla + // buildFilter evaluates `null <= 999 && 999 <= null` -> null -> coerced false -> the batch is + // silently dropped (data loss). + test("null lower/upper bound on referenced column bypasses pruning (batch kept)") { + val serializer = new ColumnarCachedBatchSerializer + val attr = AttributeReference("id", LongType, nullable = false)() + val predicate = EqualTo(attr, Literal(999L)) + val filter = serializer.buildFilter(Seq(predicate), Seq(attr)) + + val nullBoundStats = new GenericInternalRow(Array[Any](null, null, 0, 10, 80L)) + val batch = CachedColumnarBatch( + numRows = 10, + sizeInBytes = 80L, + bytes = Array.fill[Byte](40)(0), + stats = nullBoundStats) + + val result = filter(0, Iterator[CachedBatch](batch)).toList + assert( + result.length === 1, + "null-bound referenced column must bypass pruning -> batch kept (no silent data loss)") + assert(result.head.numRows === 10) + } + + // The bypass must split a contiguous run correctly: a finite-bound batch that the predicate + // excludes is still pruned, a null-bound batch in the middle is passed through, and a finite + // covering batch after it is still kept -- no batch double-emitted or skipped. + test("null-bound bypass splits a contiguous run: pruned dropped, null-bound + covering kept") { + val serializer = new ColumnarCachedBatchSerializer + val attr = AttributeReference("id", LongType, nullable = false)() + val predicate = EqualTo(attr, Literal(999L)) + val filter = serializer.buildFilter(Seq(predicate), Seq(attr)) + + val prunable = batchWithStats(5, 0L, 100L) // finite, excludes 999 -> pruned + val nullBound = CachedColumnarBatch( + numRows = 7, + sizeInBytes = 56L, + bytes = Array.fill[Byte](28)(0), + stats = new GenericInternalRow(Array[Any](null, null, 0, 7, 56L))) + val covering = batchWithStats(9, 900L, 1000L) // finite, covers 999 -> kept + + val result = filter(0, Iterator[CachedBatch](prunable, nullBound, covering)).toList + assert( + result.map(_.numRows) === Seq(7, 9), + "prunable dropped; null-bound bypassed (kept); covering kept -- run split correctly") + } + // --------------------------------------------------------------------------- // W1-W8 -- non-binary collation StringType wrapper behavior. // The wrapper strips AND-conjuncts referencing non-binary collation StringType diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchE2ESuite.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchE2ESuite.scala index fee8eccb651..a8e510301c0 100644 --- a/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchE2ESuite.scala +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchE2ESuite.scala @@ -176,25 +176,27 @@ class ColumnarCachedBatchE2ESuite } } - test("Float NaN partition: filter on non-NaN not silently pruned") { + test("Float NaN same batch: filter on non-NaN not silently pruned") { + // coalesce(1) forces the NaN row (id=7) and the queried finite row (id=42) into the SAME + // cached batch, deterministically reproducing the regression: previously a NaN poisoned the + // whole column to unsupported -> null min/max bounds -> vanilla buildFilter pruned the batch + // -> the finite k=42.0 row was silently dropped. NaN must instead be skipped so the finite + // bounds [0, 999] are emitted and the matching row is returned (parity with vanilla Spark). val df = spark .range(N) .select( when(col("id") === 7L, lit(Float.NaN)) .otherwise(col("id").cast("float")) .as("k")) - .repartition(P) + .coalesce(1) .cache() try { df.count() - // pivot=42 is a non-NaN value that exists somewhere; the partition that - // contains it may also contain the NaN row at id=7 (collision possible - // depending on hash partitioning). Either way, equality must find it. val result = df.filter(col("k") === 42.0f).count() assert( result == 1L, s"expected 1 row with k=42.0, got $result " + - s"(NaN may have poisoned partition stats)") + s"(NaN must not poison partition stats / prune the finite match)") } finally { df.unpersist() } @@ -384,20 +386,20 @@ class ColumnarCachedBatchE2ESuite } } - // Config-gate negative test: with partition stats disabled (the production default), - // serializeWithStats must NOT be invoked -- the legacy serialize() path is taken and stats - // are emitted as null. A bug in the gate could silently activate stats for all users, or - // break correctness on the legacy stats=null read path. + // Partition-stats negative test: with partition stats disabled (the production default), + // V3 lazy no-stats bytes are still written, but stats are emitted as null. A bug in the + // gate could silently activate stats for all users, or break correctness on the + // stats=null buildFilter pass-through path. // // Asserts correctness only, not numOutputRows: the Gluten native scan reports row counts // on a separate metrics path, so InMemoryTableScanExec.numOutputRows can legitimately be 0 // in either gated branch (see "numOutputRows reflects post-filter row count" above). - test("partitionStats.enabled=false: legacy serialize() path correctness preserved") { + test("partitionStats.enabled=false: V3 lazy no-stats path correctness preserved") { withSQLConf( GlutenConfig.COLUMNAR_TABLE_CACHE_PARTITION_STATS_ENABLED.key -> "false") { val cached = cacheRange() try { - cached.count() // materialize cache via legacy serialize() path (stats emitted as null) + cached.count() // materialize cache via V3 no-stats path (stats emitted as null) val result = cached.filter(col("k") === pivot).count() assert(result == 1L, s"expected exactly one row matching k=$pivot, got $result") } finally { @@ -431,8 +433,8 @@ class ColumnarCachedBatchE2ESuite } } - // Reverse: legacy v1 payload at build (stats=null), reader cannot fabricate - // stats. Distinct from the same-config legacy test: this forces cross-config. + // Reverse: V3 no-stats payload at build (stats=null), reader cannot fabricate stats. + // Distinct from the same-config no-stats test: this forces cross-config. test("cross-config: build with stats disabled, read with stats enabled") { var cached: DataFrame = null var filtered: DataFrame = null @@ -509,4 +511,35 @@ class ColumnarCachedBatchE2ESuite } } } + + // V3 lazy deserialization smoke tests + + test("V3 default: cache + equality filter produces correct result") { + val cached = cacheRange() + try { + cached.count() + val result = cached.filter(col("k") === pivot).count() + assert(result == 1L, s"V3: expected 1 row matching k=$pivot, got $result") + } finally { + cached.unpersist() + } + } + + test("V3 default: multi-column cache, partial projection, no crash") { + val cached = spark + .range(N) + .selectExpr( + "cast(id as bigint) as a", + "cast(id*2 as bigint) as b", + "cast(id+1 as bigint) as c") + .repartitionByRange(P, col("a")) + .cache() + try { + cached.count() + val result = cached.filter(col("a") === pivot).select("a", "c").count() + assert(result == 1L, s"V3 projection: expected 1 row, got $result") + } finally { + cached.unpersist() + } + } } diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchFramedBytesSuite.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchFramedBytesSuite.scala index 497b90ee9de..54f17d13f41 100644 --- a/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchFramedBytesSuite.scala +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchFramedBytesSuite.scala @@ -469,4 +469,285 @@ class ColumnarCachedBatchFramedBytesSuite extends AnyFunSuite { assert(row.numFields == 5) } + // V3 framing tests + + /** Build a minimal V3 framed byte array with one empty column. */ + private def craftV3Framed( + statsBlob: Array[Byte], + numRows: Int, + numCols: Int, + colBytesLists: List[Array[Byte]]): Array[Byte] = { + val out = new java.io.ByteArrayOutputStream() + // V3 magic + out.write(Array[Byte](0xfe.toByte, 0xca.toByte, 0x53.toByte, 0x03.toByte)) + writeU32LE(out, statsBlob.length) + out.write(statsBlob) + writeU32LE(out, numRows) + writeU32LE(out, numCols) + colBytesLists.foreach { + cb => + writeU32LE(out, cb.length) + out.write(cb) + } + out.toByteArray + } + + test("V3: parseFramedBytes routes magic 0x03 to parseV3Frame") { + val stats: InternalRow = new GenericInternalRow(Array[Any](1L, 10L, 0, 5, 100L)) + val statsBlob = CachedColumnarBatchKryoSerializer.serializeStats(stats, null) + val colBytes = Array[Byte](0xab.toByte, 0xcd.toByte) // dummy column bytes + val framed = craftV3Framed(statsBlob, 5, 1, List(colBytes)) + + val (parsedStats, returnedBytes) = + CachedColumnarBatchKryoSerializer.parseFramedBytes(framed, null) + // stats should be extracted correctly + assert(parsedStats != null) + // bytes = full V3 frame (C++ will parse from magic) + assert( + java.util.Arrays.equals(returnedBytes, framed), + "V3: returned bytes must equal full frame") + } + + test("V3: statsLen=0 means lazy frame without partition stats") { + val colBytes = Array[Byte](0xab.toByte, 0xcd.toByte) + val framed = craftV3Framed(Array.emptyByteArray, 5, 1, List(colBytes)) + + val (parsedStats, returnedBytes) = + CachedColumnarBatchKryoSerializer.parseFramedBytes(framed, StructType(Nil)) + + assert(parsedStats === null, "V3 no-stats frame should expose null stats to buildFilter") + assert( + java.util.Arrays.equals(returnedBytes, framed), + "V3 no-stats parser should return the full frame for native projected read") + } + + test("V3: wrong magic version throws with clear message") { + val badMagic = Array[Byte](0xfe.toByte, 0xca.toByte, 0x53.toByte, 0x05.toByte) // unknown 0x05 + // Need at least 12 bytes to pass the length guard. + val padded = badMagic ++ Array.fill(12)(0.toByte) + val ex = intercept[IllegalArgumentException] { + CachedColumnarBatchKryoSerializer.parseFramedBytes(padded, null) + } + assert( + ex.getMessage.contains("0x05") || ex.getMessage.toLowerCase(Locale.ROOT).contains("magic"), + s"expected version/magic info in message, got: ${ex.getMessage}" + ) + } + + test("V3: malformed magic prefix is rejected before stats parsing") { + val statsBlob = CachedColumnarBatchKryoSerializer.serializeStats( + new GenericInternalRow(Array.empty[Any]), + StructType(Nil)) + val framed = { + val out = new java.io.ByteArrayOutputStream() + out.write(Array[Byte](0x00.toByte, 0xca.toByte, 0x53.toByte, 0x03.toByte)) + writeU32LE(out, statsBlob.length) + out.write(statsBlob) + writeU32LE(out, 0) + writeU32LE(out, 0) + out.toByteArray + } + + val ex = intercept[IllegalArgumentException] { + CachedColumnarBatchKryoSerializer.parseFramedBytes(framed, StructType(Nil)) + } + assert( + ex.getMessage.toLowerCase(Locale.ROOT).contains("magic"), + s"expected full magic validation, got: ${ex.getMessage}") + } + + // Cross-language V3 golden paired with the cpp golden in + // cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc + // TEST_F(VeloxColumnarBatchSerializerTest, framedSerializeWithStatsV3EmptyGolden) + // Empty 0-row / 0-col input has no Velox per-column payload, so the byte literal + // pins the V3 magic + statsLen + statsBlob + numRows + numCols frame contract without + // PrestoSerde payload drift. + private val kGoldenFrameV3Empty: Array[Byte] = Array[Int]( + 0xfe, 0xca, 0x53, 0x03, + 0x04, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00 + ).map(_.toByte) + + test("V3: cpp golden empty frame parses on JVM") { + val (parsedStats, returnedBytes) = + CachedColumnarBatchKryoSerializer.parseFramedBytes(kGoldenFrameV3Empty, StructType(Nil)) + assert(parsedStats !== null) + assert(parsedStats.numFields === 0) + assert(java.util.Arrays.equals(returnedBytes, kGoldenFrameV3Empty)) + } + + // Cross-language V3 no-stats golden paired with the cpp golden: + // TEST_F(VeloxColumnarBatchSerializerTest, framedSerializeV3NoStatsEmptyGolden) + private val kGoldenFrameV3NoStatsEmpty: Array[Byte] = Array[Int]( + 0xfe, 0xca, 0x53, 0x03, + 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00 + ).map(_.toByte) + + test("V3: cpp no-stats empty frame parses on JVM") { + val (parsedStats, returnedBytes) = + CachedColumnarBatchKryoSerializer.parseFramedBytes( + kGoldenFrameV3NoStatsEmpty, + StructType(Nil)) + assert(parsedStats === null) + assert(java.util.Arrays.equals(returnedBytes, kGoldenFrameV3NoStatsEmpty)) + } + + // Cross-language V3 non-empty no-stats frame fixture paired with the cpp literal in + // TEST_F(VeloxColumnarBatchSerializerTest, + // deserializeV3ZeroProjectionNonEmptyNoStatsFrameFixture) + // + // The column payload bytes are intentionally tiny dummy payloads. The purpose of this + // fixture is to pin the V3 frame-level contract for non-empty batches: + // [magic][statsLen=0][numRows=5][numCols=3][colLen+colBytes]* + private val kFrameFixtureV3NoStatsNonEmpty: Array[Byte] = Array[Int]( + 0xfe, 0xca, 0x53, 0x03, + 0x00, 0x00, 0x00, 0x00, + 0x05, 0x00, 0x00, 0x00, + 0x03, 0x00, 0x00, 0x00, + 0x01, 0x00, 0x00, 0x00, + 0xaa, + 0x02, 0x00, 0x00, 0x00, + 0xbb, 0xcc, + 0x03, 0x00, 0x00, 0x00, + 0xdd, 0xee, 0xff + ).map(_.toByte) + + test("V3: cpp no-stats non-empty frame fixture parses on JVM") { + val schema = StructType( + Seq( + StructField("a", IntegerType), + StructField("b", LongType), + StructField("c", StringType))) + val (parsedStats, returnedBytes) = + CachedColumnarBatchKryoSerializer.parseFramedBytes( + kFrameFixtureV3NoStatsNonEmpty, + schema) + assert(parsedStats === null) + assert(java.util.Arrays.equals(returnedBytes, kFrameFixtureV3NoStatsNonEmpty)) + } + + // Cross-language V3 non-empty with-stats frame fixture paired with the cpp literal in + // TEST_F(VeloxColumnarBatchSerializerTest, + // deserializeV3ZeroProjectionNonEmptyWithStatsFrameFixture) + // + // statsBlob = [numCols=3] + 3 unsupported column records. This keeps the stats parser + // deterministic while still pinning non-empty V3 statsLen + per-column layout. + private val kFrameFixtureV3WithStatsNonEmpty: Array[Byte] = Array[Int]( + 0xfe, 0xca, 0x53, 0x03, + 0x37, 0x00, 0x00, 0x00, + 0x03, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, + 0x05, 0x00, 0x00, 0x00, + 0x0b, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x01, 0x00, 0x00, 0x00, + 0x05, 0x00, 0x00, 0x00, + 0x16, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x02, 0x00, 0x00, 0x00, + 0x05, 0x00, 0x00, 0x00, + 0x21, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x05, 0x00, 0x00, 0x00, + 0x03, 0x00, 0x00, 0x00, + 0x01, 0x00, 0x00, 0x00, + 0xaa, + 0x02, 0x00, 0x00, 0x00, + 0xbb, 0xcc, + 0x03, 0x00, 0x00, 0x00, + 0xdd, 0xee, 0xff + ).map(_.toByte) + + test("V3: cpp with-stats non-empty frame fixture parses on JVM") { + val schema = StructType( + Seq( + StructField("a", IntegerType), + StructField("b", LongType), + StructField("c", StringType))) + val (parsedStats, returnedBytes) = + CachedColumnarBatchKryoSerializer.parseFramedBytes( + kFrameFixtureV3WithStatsNonEmpty, + schema) + assert(parsedStats !== null) + assert(parsedStats.numFields === 15) + assert(parsedStats.isNullAt(0)) + assert(parsedStats.isNullAt(1)) + assert(parsedStats.getInt(2) === 0) + assert(parsedStats.getInt(3) === 5) + assert(parsedStats.getLong(4) === 11L) + assert(parsedStats.getInt(7) === 1) + assert(parsedStats.getLong(9) === 22L) + assert(parsedStats.getInt(12) === 2) + assert(parsedStats.getLong(14) === 33L) + assert(java.util.Arrays.equals(returnedBytes, kFrameFixtureV3WithStatsNonEmpty)) + } + + test("V3: too-short frame (< 12 bytes) rejected by dispatcher") { + val shortV3 = Array[Byte](0xfe.toByte, 0xca.toByte, 0x53.toByte, 0x03.toByte, 0, 0) + intercept[IllegalArgumentException] { + CachedColumnarBatchKryoSerializer.parseFramedBytes(shortV3, null) + } + } + + test("V3: frame with truncated colLen claim is rejected at JVM layer") { + val stats: InternalRow = new GenericInternalRow(Array[Any](1L, 10L, 0, 5, 100L)) + val statsBlob = CachedColumnarBatchKryoSerializer.serializeStats(stats, null) + val out = new java.io.ByteArrayOutputStream() + out.write(Array[Byte](0xfe.toByte, 0xca.toByte, 0x53.toByte, 0x03.toByte)) + writeU32LE(out, statsBlob.length) + out.write(statsBlob) + writeU32LE(out, 5) + writeU32LE(out, 1) + writeU32LE(out, 3) + out.write(Array[Byte](0xab.toByte, 0xcd.toByte)) + + val ex = intercept[IllegalArgumentException] { + CachedColumnarBatchKryoSerializer.parseFramedBytes(out.toByteArray, null) + } + assert( + ex.getMessage.contains("colBytes truncated"), + s"expected truncated-column message, got: ${ex.getMessage}") + } + + test("V3: trailing bytes after column payloads are rejected at JVM layer") { + val framed = + craftV3Framed(Array.emptyByteArray, 5, 1, List(Array[Byte](0xab.toByte))) :+ 0x42.toByte + + val ex = intercept[IllegalArgumentException] { + CachedColumnarBatchKryoSerializer.parseFramedBytes(framed, StructType(Nil)) + } + assert( + ex.getMessage.contains("trailing"), + s"expected trailing-bytes message, got: ${ex.getMessage}") + } + + test("V3: frame numRows must match outer CachedBatch numRows") { + val framed = craftV3Framed(Array.emptyByteArray, 5, 1, List(Array[Byte](0xab.toByte))) + + CachedColumnarBatchKryoSerializer.requireV3FrameNumRows( + framed, + expectedNumRows = 5, + context = "test") + + val ex = intercept[IllegalArgumentException] { + CachedColumnarBatchKryoSerializer.requireV3FrameNumRows( + framed, + expectedNumRows = 4, + context = "test") + } + assert( + ex.getMessage.contains("V3 frame numRows=5 != CachedBatch numRows=4"), + s"expected row-count mismatch message, got: ${ex.getMessage}") + } + + test("V3 + V2: V2 frames still parsed correctly after V3 magic added") { + val stats: InternalRow = new GenericInternalRow(Array[Any](5L, 50L, 0, 10, 200L)) + val payload = Array[Byte](10, 20, 30) + val v2Framed = craftFramed(stats, payload) // V2 magic 0x02 + val (parsedStats, bytesBlob) = + CachedColumnarBatchKryoSerializer.parseFramedBytes(v2Framed, null) + assert(parsedStats != null) + assert(java.util.Arrays.equals(bytesBlob, payload), "V2 bytesBlob must be pure Presto bytes") + } } diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchLazySerdeTest.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchLazySerdeTest.scala new file mode 100644 index 00000000000..f571252e5f2 --- /dev/null +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchLazySerdeTest.scala @@ -0,0 +1,287 @@ +/* + * 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.spark.sql.execution + +import org.apache.gluten.config.GlutenConfig +import org.apache.gluten.execution.VeloxWholeStageTransformerSuite + +import org.apache.spark.SparkConf +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal} +import org.apache.spark.sql.classic.ClassicDataset +import org.apache.spark.sql.columnar.CachedBatch +import org.apache.spark.sql.execution.columnar.{InMemoryRelation, SparkCacheUtil} +import org.apache.spark.sql.functions.col + +/** + * End-to-end tests for Gluten table cache lazy deserialization (V3 wire format). + * + * Validates: V3 format write, per-column lazy read, correct results, and V2/V3 read compatibility. + */ +class ColumnarCachedBatchLazySerdeTest extends VeloxWholeStageTransformerSuite { + override protected val resourcePath: String = "/tpch-data-parquet" + override protected val fileFormat: String = "parquet" + + private val N = 500L + private val P = 4 + + override def beforeAll(): Unit = { + super.beforeAll() + SparkCacheUtil.clearCacheSerializer() + } + + override protected def afterAll(): Unit = { + SparkCacheUtil.clearCacheSerializer() + super.afterAll() + } + + override protected def sparkConf: SparkConf = { + super.sparkConf + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.sql.shuffle.partitions", "4") + .set(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED.key, "true") + .set(GlutenConfig.COLUMNAR_TABLE_CACHE_PARTITION_STATS_ENABLED.key, "true") + } + + private def cachedRelation(df: DataFrame): InMemoryRelation = { + val classicDf = ClassicDataset.ofRows(spark, df.queryExecution.logical) + val cached = spark.sharedState.cacheManager.lookupCachedData(classicDf).getOrElse { + fail("expected DataFrame to be registered in Spark cache manager") + } + cached.cachedRepresentation match { + case relation: InMemoryRelation => relation + case other => fail(s"expected InMemoryRelation, got ${other.getClass.getName}") + } + } + + private def cachedBatches(df: DataFrame): Array[CachedColumnarBatch] = { + cachedRelation(df).cacheBuilder.cachedColumnBuffers.collect().map { + case batch: CachedColumnarBatch => batch + case other => fail(s"expected CachedColumnarBatch, got ${other.getClass.getName}") + } + } + + private def firstCachedBatch(df: DataFrame): CachedColumnarBatch = { + cachedBatches(df).head + } + + private def isV3Frame(bytes: Array[Byte]): Boolean = { + val magic = CachedColumnarBatchKryoSerializer.STATS_FRAMED_MAGIC_V3 + bytes != null && bytes.length >= magic.length && bytes.take(magic.length).sameElements(magic) + } + + test("partitionStats enabled writes V3 lazy format by default") { + val cached = spark.range(N).select(col("id").cast("bigint").as("k")).cache() + try { + val count = cached.count() + assert(count == N, s"Expected $N rows, got $count") + val batch = firstCachedBatch(cached) + assert( + isV3Frame(batch.bytes), + "partitionStats-enabled cache should write V3 lazy bytes by default") + val filtered = cached.filter(col("k") === 100L).count() + assert(filtered == 1L, s"Expected 1 row matching k=100, got $filtered") + } finally { + cached.unpersist() + } + } + + test("partitionStats disabled still writes V3 lazy format by default") { + withSQLConf(GlutenConfig.COLUMNAR_TABLE_CACHE_PARTITION_STATS_ENABLED.key -> "false") { + val cached = spark + .range(N) + .selectExpr("cast(id as bigint) as a", "cast(id * 2 as bigint) as b", "id + 100 as c") + .cache() + try { + val count = cached.count() + assert(count == N, s"Expected $N rows, got $count") + val batch = firstCachedBatch(cached) + assert( + isV3Frame(batch.bytes), + "V3 lazy bytes should be the default even when partition stats are disabled") + assert(batch.stats == null, "partitionStats=false should not attach pruning stats") + + val projected = cached.select("a", "c").collect() + assert(projected.length == N.toInt) + projected.foreach { + row => + val a = row.getLong(0) + val c = row.getLong(1) + assert(c == a + 100L, s"expected c=a+100 for a=$a, got c=$c") + } + } finally { + cached.unpersist() + } + } + } + + test("V3: projected read gives correct results (M < N columns)") { + val cached = spark + .range(N) + .selectExpr("cast(id as bigint) as a", "cast(id*2 as bigint) as b", "id + 100 as c") + .cache() + try { + cached.count() + val result = cached.select("a", "c").collect() + assert(result.length == N.toInt, s"Expected $N rows, got ${result.length}") + // Verify a and c columns are correct. + val expected = (0L until N).map(i => (i, i + 100L)).toMap + result.foreach { + row => + val a = row.getLong(0) + val c = row.getLong(1) + assert(expected.get(a).contains(c), s"a=$a expected c=${expected(a)}, got $c") + } + } finally { + cached.unpersist() + } + } + + test("V3: count(*) does not crash (zero-column path)") { + val df = spark + .range(N) + .selectExpr("cast(id as int) as k", "cast(id as string) as s", "id as d") + .cache() + try { + df.count() + val cnt = df.count() + assert(cnt == N, s"Expected $N, got $cnt") + } finally { + df.unpersist() + } + } + + test("V3 + stats pruning: equality filter gives correct result + pruning active") { + val pivot = 250L + val cached = spark + .range(N) + .select(col("id").cast("bigint").as("k")) + .repartitionByRange(P, col("k")) + .cache() + try { + cached.count() + val relation = cachedRelation(cached) + val batches = cachedBatches(cached) + assert( + batches.length > 1, + s"test setup should create multiple cached batches, got ${batches.length}") + assert(batches.forall(_.stats != null), "partitionStats=true should attach pruning stats") + + val cachedAttr = relation.output.find(_.name == "k").getOrElse { + fail(s"expected cached attribute k in ${relation.output.mkString(",")}") + } + val filter = new ColumnarCachedBatchSerializer() + .buildFilter(Seq(EqualTo(cachedAttr, Literal(pivot))), Seq(cachedAttr)) + val batchIterator: Iterator[CachedBatch] = batches.iterator + val kept = filter(0, batchIterator).toArray + assert(kept.nonEmpty, "stats pruning should keep the batch whose range contains the pivot") + assert( + kept.length < batches.length, + s"stats pruning should skip at least one cached batch, " + + s"kept=${kept.length}, total=${batches.length}") + + val result = cached.filter(col("k") === pivot).count() + assert(result == 1L, s"Expected 1 row matching k=$pivot, got $result") + } finally { + cached.unpersist() + } + } + + test("V3: all types roundtrip correctly") { + val df = spark + .range(100) + .selectExpr( + "cast(id as int) as int_col", + "cast(id as bigint) as long_col", + "cast(id as double) as double_col", + "cast(id/100.0 as decimal(10,2)) as dec_col", + "date_add(date('2020-01-01'), cast(id as int)) as date_col", + "cast(id as string) as str_col", + "cast(id % 2 = 0 as boolean) as bool_col" + ) + .cache() + try { + df.count() + // Only read a subset of columns - validates lazy deserialization of specific types. + val result = df.select("int_col", "date_col", "bool_col").collect() + assert(result.length == 100) + assert(result(0).getInt(0) == 0) + assert(!result(0).isNullAt(1)) + assert(result(0).getBoolean(2) == true) // 0 % 2 == 0 + } finally { + df.unpersist() + } + } + + test("V3: all-null projected column roundtrips correctly") { + val cached = spark + .range(N) + .selectExpr("cast(id as bigint) as k", "cast(null as bigint) as all_null") + .repartitionByRange(P, col("k")) + .cache() + try { + cached.count() + val result = cached.select("all_null").collect() + assert(result.length == N.toInt) + assert(result.forall(_.isNullAt(0)), "all_null should remain null after V3 projection") + val filtered = cached.where("all_null is null").count() + assert(filtered == N) + } finally { + cached.unpersist() + } + } + + test("V3 cached bytes remain readable after partitionStats is disabled") { + val cached = spark.range(N).select(col("id").cast("bigint").as("k")).cache() + cached.count() + assert( + isV3Frame(firstCachedBatch(cached).bytes), + "test setup should materialize V3 bytes") + try { + withSQLConf(GlutenConfig.COLUMNAR_TABLE_CACHE_PARTITION_STATS_ENABLED.key -> "false") { + val result = cached.filter(col("k") === 100L).count() + assert(result == 1L, s"V3 bytes must remain readable, got $result") + } + } finally { + cached.unpersist() + } + } + + test("V3: wide table - only requested columns decoded (3-col cache, 1-col query)") { + val N2 = 200L + val cached = spark + .range(N2) + .selectExpr( + "cast(id as int) as a", + "cast(id * 10 as bigint) as b", // not requested + "cast(id + 1000 as bigint) as c") + .repartitionByRange(P, col("a")) + .cache() + try { + cached.count() + // Only read column 'a'. + val results = cached.select("a").collect() + assert(results.length == N2.toInt) + results.zipWithIndex.foreach { + case (row, _) => + assert(!row.isNullAt(0), "column 'a' should not be null") + } + } finally { + cached.unpersist() + } + } +} diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializerHelperSuite.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializerHelperSuite.scala index badc4324b19..200c37c4ae2 100644 --- a/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializerHelperSuite.scala +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializerHelperSuite.scala @@ -26,9 +26,9 @@ import org.mockito.Mockito.{mock, when} import org.scalatest.funsuite.AnyFunSuite /** - * Unit tests for `ColumnarCachedBatchSerializer.serializeOneBatchWithStats`. Exercises the - * fast-path / fallback catch arms with a Mockito-stubbed JNI wrapper, without requiring a Velox - * runtime or native libraries. + * Unit tests for `ColumnarCachedBatchSerializer` serialization helpers. Exercises the fast-path / + * fallback catch arms with a Mockito-stubbed JNI wrapper, without requiring a Velox runtime or + * native libraries. */ class ColumnarCachedBatchSerializerHelperSuite extends AnyFunSuite { @@ -47,6 +47,30 @@ class ColumnarCachedBatchSerializerHelperSuite extends AnyFunSuite { (closure, () => called) } + private def writeU32LE(out: java.io.ByteArrayOutputStream, v: Int): Unit = { + out.write(v & 0xff) + out.write((v >>> 8) & 0xff) + out.write((v >>> 16) & 0xff) + out.write((v >>> 24) & 0xff) + } + + private def craftV3NoStatsFrame( + numRows: Int, + numCols: Int, + colBytes: Array[Byte]*): Array[Byte] = { + val out = new java.io.ByteArrayOutputStream() + out.write(Array[Byte](0xfe.toByte, 0xca.toByte, 0x53.toByte, 0x03.toByte)) + writeU32LE(out, 0) + writeU32LE(out, numRows) + writeU32LE(out, numCols) + colBytes.foreach { + bytes => + writeU32LE(out, bytes.length) + out.write(bytes) + } + out.toByteArray + } + test("corrupt magic frame is absorbed into legacy fallback (stats=null)") { // 12 bytes: 4 bogus magic + 4-byte statsLen=0 + 4-byte bytesLen=0 val corruptFramed: Array[Byte] = Array[Byte](0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0) @@ -151,4 +175,117 @@ class ColumnarCachedBatchSerializerHelperSuite extends AnyFunSuite { // Restore for subsequent tests / suites in the same JVM. flagField.setBoolean(ColumnarCachedBatchSerializer, true) } + + test("V3 null JNI return trips V3 capability latch and falls back") { + ColumnarCachedBatchSerializer.withStatsExtV3AvailabilityForBenchmark(true) { + val jni = mock(classOf[ColumnarBatchSerializerJniWrapper]) + when(jni.serializeV3(anyLong())).thenReturn(null) + val (fallback, wasCalled) = newFallbackProbe() + + val cb = ColumnarCachedBatchSerializer.serializeOneBatchV3( + jni, + 0L, + 1, + structSchema, + includeStats = false, + fallback) + + assert(wasCalled(), "fallback closure should be invoked when V3 returns null") + assert(cb.asInstanceOf[CachedColumnarBatch].stats == null) + assert( + !ColumnarCachedBatchSerializer.statsExtV3Available, + "null V3 JNI return should trip the V3 JVM-lifetime capability latch") + } + } + + test("V3 UnsatisfiedLinkError trips V3 capability latch and falls back") { + ColumnarCachedBatchSerializer.withStatsExtV3AvailabilityForBenchmark(true) { + val jni = mock(classOf[ColumnarBatchSerializerJniWrapper]) + when(jni.serializeWithStatsV3(anyLong())) + .thenThrow(new UnsatisfiedLinkError("serializeWithStatsV3 (test injection)")) + val (fallback, wasCalled) = newFallbackProbe() + + val cb = ColumnarCachedBatchSerializer.serializeOneBatchV3( + jni, + 0L, + 1, + structSchema, + includeStats = true, + fallback) + + assert(wasCalled(), "fallback closure should be invoked on V3 ULE") + assert(cb.asInstanceOf[CachedColumnarBatch].stats == null) + assert( + !ColumnarCachedBatchSerializer.statsExtV3Available, + "V3 ULE should trip the V3 JVM-lifetime capability latch") + } + } + + test("V3 corrupt frame falls back without disabling V3 capability") { + ColumnarCachedBatchSerializer.withStatsExtV3AvailabilityForBenchmark(true) { + val corruptV3Frame: Array[Byte] = Array[Byte]( + 0xfe.toByte, + 0xca.toByte, + 0x53.toByte, + 0x03.toByte, + 0, + 0, + 0, + 0, + 1, + 0, + 0, + 0, + 1, + 0, + 0, + 0, + 4, + 0, + 0, + 0, + 0x11.toByte + ) + val jni = mock(classOf[ColumnarBatchSerializerJniWrapper]) + when(jni.serializeV3(anyLong())).thenReturn(corruptV3Frame) + val (fallback, wasCalled) = newFallbackProbe() + + val cb = ColumnarCachedBatchSerializer.serializeOneBatchV3( + jni, + 0L, + 1, + structSchema, + includeStats = false, + fallback) + + assert(wasCalled(), "fallback closure should be invoked on corrupt V3 frame") + assert(cb.asInstanceOf[CachedColumnarBatch].stats == null) + assert( + ColumnarCachedBatchSerializer.statsExtV3Available, + "corrupt data should not disable the V3 JNI capability latch") + } + } + + test("V3 frame row-count mismatch falls back without disabling V3 capability") { + ColumnarCachedBatchSerializer.withStatsExtV3AvailabilityForBenchmark(true) { + val jni = mock(classOf[ColumnarBatchSerializerJniWrapper]) + when(jni.serializeV3(anyLong())) + .thenReturn(craftV3NoStatsFrame(numRows = 2, numCols = 1, Array[Byte](0x11.toByte))) + val (fallback, wasCalled) = newFallbackProbe() + + val cb = ColumnarCachedBatchSerializer.serializeOneBatchV3( + jni, + 0L, + numRows = 1, + structSchema, + includeStats = false, + fallback) + + assert(wasCalled(), "fallback closure should be invoked on V3 row-count mismatch") + assert(cb.asInstanceOf[CachedColumnarBatch].stats == null) + assert( + ColumnarCachedBatchSerializer.statsExtV3Available, + "row-count mismatch is corrupt data and should not disable the V3 capability latch") + } + } } diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnarTableCacheLazyDeserBenchmark.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnarTableCacheLazyDeserBenchmark.scala new file mode 100644 index 00000000000..6ca36519579 --- /dev/null +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnarTableCacheLazyDeserBenchmark.scala @@ -0,0 +1,366 @@ +/* + * 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.spark.sql.execution.benchmark + +import org.apache.gluten.config.GlutenConfig + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.execution.ColumnarCachedBatchSerializer +import org.apache.spark.storage.StorageLevel + +import java.nio.charset.StandardCharsets.UTF_8 +import java.util.Locale + +/** + * Benchmark to measure write/read overhead, cache footprint and column-skip benefit of the columnar + * table cache across the two serialization generations and the optional partition-stats payload. + * + * Compares the FOUR cache environments (eager "V2" vs lazy "V3", each with and without stats): + * - V2 without stats (eager): the legacy raw Presto payload. There is no V2 stats-less frame -- + * stats-off + V3-off writes the legacy bytes -- so this is the eager/no-pruning baseline. + * - V2 with stats (eager): `framedSerializeWithStats`. Eager full-batch decode, partition-stats + * pruning, but NO per-column skip. + * - V3 without stats (lazy): default per-column payload with lazy projected deserialization. + * - V3 with stats (lazy): same lazy per-column payload plus partition stats for pruning. + * + * The two axes are driven by two independent toggles: + * - lazy V3 vs eager V2: `ColumnarCachedBatchSerializer.withStatsExtV3AvailabilityForBenchmark` + * (when V3 is unavailable the write path falls back to V2-with-stats / legacy, exactly the + * paths an older native library would take). + * - stats on/off: `spark.gluten.sql.columnar.tableCache.partitionStats.enabled`. + * + * Set `spark.gluten.benchmark.includeLegacyBaseline=false` to drop the two eager V2 modes and + * benchmark only the V3 lazy modes (halves the cache memory needed for the read phases). + * + * To run this benchmark: + * {{{ + * 1. without sbt: + * bin/spark-submit --class --jars + * }}} + */ +object ColumnarTableCacheLazyDeserBenchmark extends SqlBasedBenchmark { + private val requestedRows = + spark.sparkContext.conf.getLong("spark.gluten.benchmark.rows", 100L * 1000 * 1000) + private val numParts = + spark.sparkContext.conf.getInt("spark.gluten.benchmark.partitions", 32) + private val benchmarkIters = + spark.sparkContext.conf.getInt("spark.gluten.benchmark.iterations", 3) + private val benchmarkPhases = spark.sparkContext.conf + .get("spark.gluten.benchmark.phases", "build,read1,read4,readAll,filter") + .split(",") + .map(_.trim.toLowerCase(Locale.ROOT)) + .filter(_.nonEmpty) + .toSet + // Backward-compatible knob (the original 3-mode benchmark called the eager path the "legacy + // baseline"). When true, the two eager V2 modes (no-stats + with-stats) are included alongside + // the two V3 lazy modes for the full 4-way comparison; when false only the V3 modes run. + private val includeV2Modes = + spark.sparkContext.conf.getBoolean("spark.gluten.benchmark.includeLegacyBaseline", true) + // Wide schema: 16 columns; queries project only a small subset to demonstrate skip benefit. + private val numCols = 16 + private val cacheMarkerCol = "__cache_benchmark_mode" + private val statsConfKey = GlutenConfig.COLUMNAR_TABLE_CACHE_PARTITION_STATS_ENABLED.key + + /** + * One of the four cache environments. `marker` is an arbitrary per-mode literal embedded in the + * cached frame so each mode produces a distinct logical plan and therefore a separate Spark cache + * entry (no accidental reuse between modes). + */ + private case class Mode(label: String, statsOn: Boolean, v3Available: Boolean, marker: Int) + + private val modes: Seq[Mode] = { + val v2 = Seq( + Mode("V2 without stats (eager)", statsOn = false, v3Available = false, marker = -2), + Mode("V2 with stats (eager)", statsOn = true, v3Available = false, marker = -1) + ) + val v3 = Seq( + Mode("V3 without stats (lazy)", statsOn = false, v3Available = true, marker = 0), + Mode("V3 with stats (lazy)", statsOn = true, v3Available = true, marker = 1) + ) + (if (includeV2Modes) v2 else Seq.empty) ++ v3 + } + + // Both the build phase and the read phases keep only ONE mode's cache materialized at a time + // (build -> measure -> unpersist), so peak heap is ~a single cache (the cached payloads are JVM + // byte[] blocks; ~110 B/row for this 16-col schema, measured ~104 B at 5M). Auto-scale the row + // count down to a fraction of the actual JVM heap ceiling (Runtime.maxMemory, which reflects + // -Xmx / -XX:MaxRAMPercentage and any container cgroup limit) so the single cache never evicts + // or OOMs; on a large heap the full request runs unchanged. This is what lets a 100M-row run + // complete on a modest runner where 4 simultaneous caches could not. The effective row count is + // logged in the benchmark header so results are honest about the scale actually exercised. + private val bytesPerRowPerMode = + spark.sparkContext.conf.getLong("spark.gluten.benchmark.bytesPerRowPerMode", 110L) + // Fraction of the JVM heap budgeted for the single live cache; stays inside Spark's ~0.6 unified + // storage region with margin for query execution. + private val memBudgetFraction = + spark.sparkContext.conf.getDouble("spark.gluten.benchmark.memBudgetFraction", 0.5) + + private val heapMaxBytes = Runtime.getRuntime.maxMemory + private val numRows: Long = { + val perRowOneCache = math.max(1L, bytesPerRowPerMode) + val budget = (heapMaxBytes * memBudgetFraction).toLong + val maxRows = math.max(1L, budget / perRowOneCache) + math.min(requestedRows, maxRows) + } + private val rowsAutoScaled = numRows < requestedRows + + private val filterValue = + spark.sparkContext.conf.getLong("spark.gluten.benchmark.filterValue", Math.max(0L, numRows / 2)) + + private case class CacheFootprint( + storageMemoryBytes: Long, + storageDiskBytes: Long, + numPartitions: Int, + numCachedPartitions: Int, + rddCount: Int) + + private case class CachedTable( + mode: Mode, + df: DataFrame, + cachedDf: DataFrame, + footprint: CacheFootprint) + + private def buildCache(mode: Mode): DataFrame = { + import org.apache.spark.sql.functions.col + val prevStats = spark.conf.getOption(statsConfKey) + spark.conf.set(statsConfKey, mode.statsOn.toString) + ColumnarCachedBatchSerializer.withStatsExtV3AvailabilityForBenchmark(mode.v3Available) { + try { + // Wide table: c0 (key), c1-c15 (payload columns) + val exprs = Seq( + "cast(id as int) as c0", + "id as c1", + "cast(id as string) as c2", + "concat('payload-a-', cast(id % 100000 as string)) as c3", + "cast(id % 100 as int) as c4", + "cast(id * 2 as long) as c5", + "cast(id as double) as c6", + "cast(id % 10 as int) as c7", + "concat('payload-b-', cast(id % 100000 as string)) as c8", + "cast(id + 1 as long) as c9", + "cast(id as string) as c10", + "cast(id % 50 as int) as c11", + "cast(id * 3 as long) as c12", + "concat('payload-c-', cast(id % 100000 as string)) as c13", + "cast(id % 200 as int) as c14", + "cast(id as double) as c15" + ) + val cached = spark + .range(numRows) + .selectExpr((exprs :+ s"${mode.marker} as $cacheMarkerCol"): _*) + .repartitionByRange(numParts, col("c0")) + .persist(StorageLevel.MEMORY_ONLY) + cached.count() // materialize + cached + } finally { + prevStats match { + case Some(v) => spark.conf.set(statsConfKey, v) + case None => spark.conf.unset(statsConfKey) + } + } + } + } + + private def buildCacheWithFootprint(mode: Mode): CachedTable = { + val beforeIds = spark.sparkContext.getRDDStorageInfo.map(_.id).toSet + val cached = buildCache(mode) + val rdds = spark.sparkContext.getRDDStorageInfo + .filterNot(info => beforeIds.contains(info.id)) + val footprint = CacheFootprint( + rdds.map(_.memSize).sum, + rdds.map(_.diskSize).sum, + rdds.map(_.numPartitions).sum, + rdds.map(_.numCachedPartitions).sum, + rdds.length) + CachedTable(mode, cached.drop(cacheMarkerCol), cached, footprint) + } + + private def printCacheFootprint(rows: Seq[(String, CacheFootprint)]): Unit = { + outputLine() + outputLine("Cache footprint after materialization") + outputLine("-------------------------------------") + outputLine( + f"${"mode"}%-32s ${"rdds"}%6s ${"cachedPartitions"}%18s ${"partitions"}%12s " + + f"${"storageMemoryBytes"}%20s ${"storageMemoryMiB"}%18s ${"storageDiskBytes"}%18s") + rows.foreach { + case (mode, footprint) => + outputLine( + f"$mode%-32s ${footprint.rddCount}%6d ${footprint.numCachedPartitions}%18d " + + f"${footprint.numPartitions}%12d ${footprint.storageMemoryBytes}%20d " + + f"${footprint.storageMemoryBytes.toDouble / 1024.0 / 1024.0}%18.2f " + + f"${footprint.storageDiskBytes}%18d") + } + outputLine() + } + + private def phaseEnabled(name: String): Boolean = + benchmarkPhases.contains(name.toLowerCase(Locale.ROOT)) + + private def outputLine(line: String = ""): Unit = { + val out = output.getOrElse(System.out) + out.write((line + System.lineSeparator()).getBytes(UTF_8)) + out.flush() + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val heapMiB = heapMaxBytes / (1024 * 1024) + val scaleNote = + if (rowsAutoScaled) { + s" [AUTO-SCALED DOWN from requested $requestedRows to fit jvmHeapMax=${heapMiB}MiB x " + + s"budgetFraction=$memBudgetFraction for a single live cache @ " + + s"~${bytesPerRowPerMode}B/row]" + } else { + "" + } + outputLine( + s"Benchmark config: rows=$numRows (requested=$requestedRows)$scaleNote, " + + s"partitions=$numParts, iterations=$benchmarkIters, " + + s"phases=${benchmarkPhases.toSeq.sorted.mkString(",")}, " + + s"includeV2Modes=$includeV2Modes, modes=${modes.map(_.label).mkString(" | ")}") + + // === Benchmark 1: write-path overhead (cache build) === + // Measures the eager-vs-lazy serialize cost and the optional partition-stats overhead. + if (phaseEnabled("build")) { + val buildBench = + new Benchmark( + s"table cache build ($numCols cols, $numParts parts)", + numRows, + output = output) + modes.foreach { + mode => + buildBench.addCase(mode.label, benchmarkIters) { + _ => + spark.catalog.clearCache() + buildCache(mode).unpersist() + } + } + buildBench.run() + spark.catalog.clearCache() + } + + // Read benchmarks: build ONE mode's cache at a time, measure all read phases against it, then + // unpersist before moving to the next mode. Holding a single cache (instead of all `modes` at + // once) caps peak heap to ~one cache, so large datasets (e.g. 100M rows) fit where 4 + // simultaneous caches would not. Results are collated into a combined per-phase table after. + import org.apache.spark.sql.functions._ + def aggAll(df: DataFrame): Unit = + df.agg( + sum("c0"), + sum("c1"), + sum(length(col("c2"))), + sum(length(col("c3"))), + sum("c4"), + sum("c5"), + sum("c6"), + sum("c7"), + sum(length(col("c8"))), + sum("c9"), + sum(length(col("c10"))), + sum("c11"), + sum("c12"), + sum(length(col("c13"))), + sum("c14"), + sum("c15") + ).noop() + val readPhases: Seq[(String, String, DataFrame => Unit)] = Seq( + ( + "read1", + s"table cache read: 1/$numCols columns, sum(c0)", + (df: DataFrame) => df.agg(sum("c0")).noop()), + ( + "read4", + s"table cache read: 4/$numCols columns, group+agg", + (df: DataFrame) => + df.groupBy((col("c0") % 1000).as("g")).agg( + sum("c1"), + sum(length(col("c2"))), + avg("c4")).noop()), + ( + "readall", + s"table cache read: all $numCols columns (no skip benefit)", + (df: DataFrame) => aggAll(df)), + ( + "filter", + s"table cache read: filter + 2/$numCols columns (batch skip + lazy)", + (df: DataFrame) => df.where(s"c0 = $filterValue").select("c0", "c1").noop()) + ).filter { case (key, _, _) => phaseEnabled(key) } + + if (readPhases.nonEmpty) { + val footprints = scala.collection.mutable.ArrayBuffer.empty[(String, CacheFootprint)] + val results = scala.collection.mutable.LinkedHashMap + .empty[String, scala.collection.mutable.ArrayBuffer[(String, Double, Double)]] + readPhases.foreach { + case (_, title, _) => results(title) = scala.collection.mutable.ArrayBuffer.empty + } + modes.foreach { + mode => + val ct = buildCacheWithFootprint(mode) + footprints += mode.label -> ct.footprint + try { + readPhases.foreach { + case (_, title, op) => + val (best, avg) = timePhase(ct.df, op) + results(title) += ((mode.label, best, avg)) + } + } finally { + ct.cachedDf.unpersist(blocking = true) + } + } + printCacheFootprint(footprints.toSeq) + readPhases.foreach { case (_, title, _) => printResultTable(title, results(title).toSeq) } + } + + spark.catalog.clearCache() + } + + // Manual timing (one cache alive at a time, so the Spark Benchmark group harness is not usable). + // First two untimed runs warm up JIT and trigger the lazy cache load; then `benchmarkIters` timed + // runs report best + average wall time in ms. + private def timePhase(df: DataFrame, op: DataFrame => Unit): (Double, Double) = { + op(df) + op(df) + var best = Double.MaxValue + var total = 0.0 + var i = 0 + while (i < benchmarkIters) { + val t0 = System.nanoTime() + op(df) + val ms = (System.nanoTime() - t0) / 1e6 + if (ms < best) best = ms + total += ms + i += 1 + } + (best, total / benchmarkIters) + } + + // Relative is baseline(first mode) avg / this avg, so a faster mode shows > 1.0X (matches the + // convention of org.apache.spark.benchmark.Benchmark). + private def printResultTable(title: String, rows: Seq[(String, Double, Double)]): Unit = { + outputLine() + outputLine(title) + outputLine("-" * math.min(title.length, 80)) + outputLine(f"${"mode"}%-28s ${"best(ms)"}%12s ${"avg(ms)"}%12s ${"relative"}%10s") + val baselineAvg = rows.headOption.map(_._3).getOrElse(1.0) + rows.foreach { + case (label, best, avg) => + val rel = if (avg > 0) baselineAvg / avg else 0.0 + outputLine(f"$label%-28s $best%12.1f $avg%12.1f $rel%9.2fX") + } + outputLine() + } +} diff --git a/cpp/core/jni/JniWrapper.cc b/cpp/core/jni/JniWrapper.cc index b1580782c01..430db893283 100644 --- a/cpp/core/jni/JniWrapper.cc +++ b/cpp/core/jni/JniWrapper.cc @@ -72,6 +72,17 @@ jmethodID shuffleReaderMetricsSetDeserializeTime; jclass shuffleStreamReaderClass; jmethodID shuffleStreamReaderNextStream; +jbyteArray toJByteArray(JNIEnv* env, const std::vector& bytes, const std::string& context) { + GLUTEN_CHECK( + bytes.size() <= static_cast(std::numeric_limits::max()), + context + " size exceeds Java byte[] limit: " + std::to_string(bytes.size())); + jbyteArray out = env->NewByteArray(static_cast(bytes.size())); + if (!bytes.empty()) { + env->SetByteArrayRegion(out, 0, static_cast(bytes.size()), reinterpret_cast(bytes.data())); + } + return out; +} + class JavaInputStreamAdaptor final : public arrow::io::InputStream { public: JavaInputStreamAdaptor(JNIEnv* env, arrow::MemoryPool* pool, jobject jniIn) : pool_(pool) { @@ -1317,19 +1328,42 @@ Java_org_apache_gluten_vectorized_ColumnarBatchSerializerJniWrapper_serializeWit auto serializer = ctx->createColumnarBatchSerializer(nullptr); std::vector framed = serializer->framedSerializeWithStats(batch); - // Outer-layer size defense (inner layer = bytesLen <= UINT32_MAX in - // framedSerializeWithStats). jsize is signed int32; > INT32_MAX wraps - // negative -> NewByteArray would throw NegativeArraySizeException. - // Fail fast here so JNI_METHOD_END surfaces it as GlutenException. - GLUTEN_CHECK( - framed.size() <= static_cast(std::numeric_limits::max()), - "serializeWithStats: framed payload (" + std::to_string(framed.size()) + " bytes) exceeds Java byte[] limit (" + - std::to_string(std::numeric_limits::max()) + ")"); - jbyteArray out = env->NewByteArray(static_cast(framed.size())); - if (!framed.empty()) { - env->SetByteArrayRegion(out, 0, static_cast(framed.size()), reinterpret_cast(framed.data())); + return toJByteArray(env, framed, "serializeWithStats framed payload"); + JNI_METHOD_END(nullptr) +} + +JNIEXPORT jbyteArray JNICALL Java_org_apache_gluten_vectorized_ColumnarBatchSerializerJniWrapper_serializeV3( // NOLINT + JNIEnv* env, + jobject wrapper, + jlong handle) { + JNI_METHOD_START + auto ctx = getRuntime(env, wrapper); + auto batch = ObjectStore::retrieve(handle); + GLUTEN_DCHECK(batch != nullptr, "Cannot find the ColumnarBatch with handle " + std::to_string(handle)); + auto serializer = ctx->createColumnarBatchSerializer(nullptr); + std::vector framed = serializer->framedSerializeV3(batch); + if (framed.empty()) { + return nullptr; // Non-Velox backend; caller treats null as "V3 not supported". } - return out; + return toJByteArray(env, framed, "serializeV3 framed payload"); + JNI_METHOD_END(nullptr) +} + +JNIEXPORT jbyteArray JNICALL +Java_org_apache_gluten_vectorized_ColumnarBatchSerializerJniWrapper_serializeWithStatsV3( // NOLINT + JNIEnv* env, + jobject wrapper, + jlong handle) { + JNI_METHOD_START + auto ctx = getRuntime(env, wrapper); + auto batch = ObjectStore::retrieve(handle); + GLUTEN_DCHECK(batch != nullptr, "Cannot find the ColumnarBatch with handle " + std::to_string(handle)); + auto serializer = ctx->createColumnarBatchSerializer(nullptr); + std::vector framed = serializer->framedSerializeWithStatsV3(batch); + if (framed.empty()) { + return nullptr; // Non-Velox backend; caller treats null as "V3 not supported". + } + return toJByteArray(env, framed, "serializeWithStatsV3 framed payload"); JNI_METHOD_END(nullptr) } @@ -1385,6 +1419,31 @@ JNIEXPORT void JNICALL Java_org_apache_gluten_vectorized_ColumnarBatchSerializer JNI_METHOD_END() } +JNIEXPORT jlong JNICALL +Java_org_apache_gluten_vectorized_ColumnarBatchSerializerJniWrapper_deserializeWithProjection( // NOLINT + JNIEnv* env, + jobject wrapper, + jlong serializerHandle, + jbyteArray data, + jintArray requestedCols) { + JNI_METHOD_START + auto ctx = getRuntime(env, wrapper); + auto serializer = ObjectStore::retrieve(serializerHandle); + GLUTEN_DCHECK(serializer != nullptr, "ColumnarBatchSerializer cannot be null"); + int32_t size = env->GetArrayLength(data); + auto safeData = getByteArrayElementsSafe(env, data); + // null requestedCols → all columns (nullopt); non-null (including int[0]) → selection. + std::optional> requestedOpt; + if (requestedCols != nullptr) { + jsize nCols = env->GetArrayLength(requestedCols); + auto safeCols = getIntArrayElementsSafe(env, requestedCols); + requestedOpt = std::vector(safeCols.elems(), safeCols.elems() + nCols); + } + auto batch = serializer->deserializeV3(safeData.elems(), size, requestedOpt); + return ctx->saveObject(batch); + JNI_METHOD_END(kInvalidObjectHandle) +} + #ifdef __cplusplus } #endif diff --git a/cpp/core/operators/serializer/ColumnarBatchSerializer.h b/cpp/core/operators/serializer/ColumnarBatchSerializer.h index b7bf6b41bea..23bc758e84d 100644 --- a/cpp/core/operators/serializer/ColumnarBatchSerializer.h +++ b/cpp/core/operators/serializer/ColumnarBatchSerializer.h @@ -18,9 +18,11 @@ #pragma once #include +#include #include #include "memory/ColumnarBatch.h" +#include "utils/Exception.h" namespace gluten { @@ -38,14 +40,35 @@ class ColumnarBatchSerializer { virtual std::shared_ptr deserialize(uint8_t* data, int32_t size) = 0; - // Backend-overridable framed serialization carrying per-column stats. - // Layout: [magic | statsLen | statsBlob | bytesLen | bytesBlob]. Default returns an empty - // vector to indicate the stats extension is not supported; callers detect that and fall back - // to the legacy serialize() path. The Velox backend overrides with the full implementation. + // V2: Backend-overridable framed serialization carrying per-column stats. + // Layout: [magic=0xFECA5302 | statsLen | statsBlob | bytesLen | bytesBlob]. + // Default returns empty vector (not supported); callers fall back to legacy serialize(). virtual std::vector framedSerializeWithStats(const std::shared_ptr& /*batch*/) { return {}; } + // V3: Per-column framed serialization without stats (lazy deserialization support). + // Layout: [magic=0xFECA5303 | statsLen=0 | numRows | numCols | per-col(colLen+colBytes)]. + // Default returns empty vector (not supported); callers detect and fall back. + virtual std::vector framedSerializeV3(const std::shared_ptr& /*batch*/) { + return {}; + } + + // V3: Per-column framed serialization + stats (lazy deserialization + pruning support). + // Layout: [magic=0xFECA5303 | statsLen | statsBlob | numRows | numCols | per-col(colLen+colBytes)]. + // Default returns empty vector (not supported); callers detect and fall back. + virtual std::vector framedSerializeWithStatsV3(const std::shared_ptr& /*batch*/) { + return {}; + } + + // V3: Deserialize with column projection; returns M-column RowVector (only requested columns). + // requestedColumns: nullopt=all columns, optional= zero columns, optional=M cols. + // Default throws GlutenException (not supported for non-Velox backends). + virtual std::shared_ptr + deserializeV3(uint8_t* /*data*/, int32_t /*size*/, const std::optional>& /*requestedColumns*/) { + throw GlutenException("deserializeV3 is not supported for this backend"); + } + protected: arrow::MemoryPool* arrowPool_; }; diff --git a/cpp/core/utils/tac/ffor.hpp b/cpp/core/utils/tac/ffor.hpp index 761a1ec2623..0d632efff5a 100644 --- a/cpp/core/utils/tac/ffor.hpp +++ b/cpp/core/utils/tac/ffor.hpp @@ -437,8 +437,7 @@ inline size_t decompress64Impl(const uint8_t* input, size_t inputSize, uint64_t* if (bw == kBwTailMarker) { if (count > 0) { // memcpy handles any alignment, no special case needed. - std::memcpy( - reinterpret_cast(output) + nDecoded * sizeof(uint64_t), inPtr, count * sizeof(uint64_t)); + std::memcpy(reinterpret_cast(output) + nDecoded * sizeof(uint64_t), inPtr, count * sizeof(uint64_t)); nDecoded += count; } break; diff --git a/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc b/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc index 6df1ab509c0..4dbda85a8bd 100644 --- a/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc +++ b/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc @@ -22,11 +22,14 @@ #include #include #include +#include +#include #include "memory/ArrowMemory.h" #include "memory/VeloxColumnarBatch.h" #include "velox/common/memory/Memory.h" #include "velox/vector/FlatVector.h" +#include "velox/vector/LazyVector.h" #include "velox/vector/arrow/Bridge.h" #include @@ -58,6 +61,9 @@ VeloxColumnarBatchSerializer::VeloxColumnarBatchSerializer( arena_ = std::make_unique(veloxPool_.get()); serde_ = std::make_unique(); options_.useLosslessTimestamp = true; + // Required by serializeSingleColumn / deserializeSingleColumn APIs (VELOX_USER_CHECK_EQ). + options_.compressionKind = common::CompressionKind::CompressionKind_NONE; + options_.nullsFirst = false; } void VeloxColumnarBatchSerializer::append(const std::shared_ptr& batch) { @@ -101,14 +107,25 @@ std::shared_ptr VeloxColumnarBatchSerializer::deserialize(uint8_t namespace { -// Per-type FlatVector min/max scan + NaN guard. Returns false when the column must be marked -// unsupported (any NaN observed for floating-point types -- Spark equality NaN != NaN means -// min/max-based pruning would silently drop matching rows). On NaN, scan still completes the -// loop to accrue real nullCnt -- framed stats serialize nullCount even when emitSupported=0, -// and Spark IsNull pruning reads `statsFor(a).nullCount > 0`; an under-counted nullCount on a -// `[NaN, null]` partition would let `col IS NULL` predicates incorrectly prune matching rows. +// Per-type FlatVector min/max scan. Returns true (column may carry min/max bounds); the caller +// gates on `seen` to decide whether any non-null value was actually observed. // -// Floating-point edge cases that DO NOT poison the column: +// Floating-point NaN handling -- mirror vanilla Spark Float/DoubleColumnStats.gatherValueStats: +// vanilla updates bounds with `if (value > upper)` / `if (value < lower)`, and since every NaN +// comparison is false, NaN never widens min/max -- vanilla silently SKIPS NaN yet still emits +// the finite min/max. We must do the SAME. Poisoning the whole column to unsupported (the prior +// behavior) instead emits NULL bounds; the vanilla SimpleMetricsCachedBatchSerializer.buildFilter +// then turns `col = l` into `lowerBound <= l && l <= upperBound`, which evaluates to null on null +// bounds -> coerced to false -> `!eval` -> the batch is PRUNED. That silently drops finite rows +// that genuinely match when a NaN happens to share the batch (data loss, and a regression vs +// vanilla). Skipping NaN keeps the finite bounds and matches vanilla exactly. NaN-literal +// predicates (`k = NaN`, `k > huge`) inherit vanilla's existing finite-bound behavior -- that is +// parity with Spark, not a new Gluten divergence. +// +// NaN still counts as a non-null value, so it is NOT added to nullCnt -- only true nulls are, +// keeping Spark IsNull/IsNotNull pruning (`nullCount > 0`, `count - nullCount > 0`) correct. +// +// Floating-point edge cases that produce normal finite/ordered bounds (never skipped): // - +/-Infinity: ordered (-Inf < x < +Inf for finite x); participate in min/max normally. // - +0 and -0: IEEE 754 declares them equal under <, ==; min/max bound is correct either way. // - subnormal (denormal) values: ordered like normal floats; no special handling needed. @@ -118,7 +135,6 @@ bool scanMinMax(const facebook::velox::FlatVector* flat, T& tLo, T& tHi, int6 const auto size = flat->size(); const uint64_t* nulls = flat->rawNulls(); const T* values = flat->rawValues(); - bool floatingUnsupported = false; for (vector_size_t i = 0; i < size; ++i) { if (nulls != nullptr && bits::isBitNull(nulls, i)) { ++nullCnt; @@ -127,15 +143,10 @@ bool scanMinMax(const facebook::velox::FlatVector* flat, T& tLo, T& tHi, int6 T v = values[i]; if constexpr (std::is_floating_point_v) { if (std::isnan(v)) { - floatingUnsupported = true; - // Continue scanning to accrue real nullCnt -- do NOT early-return. + // Skip NaN (matches vanilla); it neither widens min/max nor counts as null. continue; } } - if (floatingUnsupported) { - // NaN already poisoned min/max; skip bound updates but keep counting (nulls handled above). - continue; - } if (!seen) { tLo = v; tHi = v; @@ -147,7 +158,7 @@ bool scanMinMax(const facebook::velox::FlatVector* flat, T& tLo, T& tHi, int6 tHi = v; } } - return !floatingUnsupported; + return true; } // BOOLEAN-specific scan: FlatVector::rawValues() is unsupported in Velox (bit-packed @@ -502,7 +513,11 @@ std::vector VeloxColumnarBatchSerializer::framedSerializeWithStats( const auto& hiTs = s.upperBound.value(); int64_t loMicros = loTs.toMicros(); int64_t hiMicros = hiTs.toMicros(); - if (hiTs.getNanos() % 1000 != 0) { + // Ceil hi by +1us on sub-microsecond residue so the upper bound stays a superset. + // Guard int64 overflow at the max representable instant: a +1 at INT64_MAX would wrap + // negative and wrongly prune. INT64_MAX micros already bounds any Spark + // (micros-resolution) value, so no ceil is needed there. + if (hiTs.getNanos() % 1000 != 0 && hiMicros != std::numeric_limits::max()) { hiMicros += 1; } pushU32(8); @@ -566,4 +581,401 @@ std::vector VeloxColumnarBatchSerializer::framedSerializeWithStats( return framed; } +// ─── V3: per-column serialization ──────────────────────────────────────────── + +namespace { + +// Lazy column loader backed by a per-column Presto-format byte slice. +// Implements VectorLoader::loadInternal by calling deserializeSingleColumn. +// Lifecycle: colBytes_ is cleared after the first load to release memory (avoid double-buffer). +class CachedColumnLoader : public facebook::velox::VectorLoader { + public: + CachedColumnLoader( + std::vector colBytes, + facebook::velox::TypePtr type, + facebook::velox::vector_size_t expectedRows, + std::shared_ptr pool, + const facebook::velox::serializer::presto::PrestoVectorSerde::PrestoOptions& options) + : colBytes_(std::move(colBytes)), + type_(std::move(type)), + expectedRows_(expectedRows), + pool_(std::move(pool)), + options_(options) { + GLUTEN_CHECK(!colBytes_.empty(), "CachedColumnLoader: colBytes must not be empty"); + GLUTEN_CHECK( + colBytes_.size() <= static_cast(std::numeric_limits::max()), + "CachedColumnLoader: colBytes size exceeds int32 ByteRange limit"); + } + + protected: + void loadInternal( + facebook::velox::RowSet /* rows */, + facebook::velox::ValueHook* /* hook */, + facebook::velox::vector_size_t /* resultSize */, + facebook::velox::VectorPtr* result) override { + // Guard against double-invocation (colBytes_ cleared after first load). + GLUTEN_CHECK(!colBytes_.empty(), "CachedColumnLoader::loadInternal: called after bytes already consumed"); + std::vector ranges; + ranges.push_back({const_cast(colBytes_.data()), static_cast(colBytes_.size()), 0}); + auto stream = std::make_unique(ranges); + facebook::velox::serializer::presto::PrestoVectorSerde serde; + serde.deserializeSingleColumn(stream.get(), pool_.get(), type_, result, &options_); + GLUTEN_CHECK( + result != nullptr && *result != nullptr, + "CachedColumnLoader::loadInternal: deserializeSingleColumn returned null result"); + GLUTEN_CHECK( + (*result)->size() == expectedRows_, + "CachedColumnLoader::loadInternal: decoded column size=" + std::to_string((*result)->size()) + + " != V3 frame numRows=" + std::to_string(expectedRows_)); + // Free raw bytes after decode to avoid holding two copies simultaneously. + std::vector().swap(colBytes_); + } + + private: + std::vector colBytes_; + facebook::velox::TypePtr type_; + facebook::velox::vector_size_t expectedRows_; + std::shared_ptr pool_; + facebook::velox::serializer::presto::PrestoVectorSerde::PrestoOptions options_; +}; + +} // namespace + +std::vector VeloxColumnarBatchSerializer::buildStatsBlob( + const std::vector& perCol, + uint32_t numRows, + uint32_t numCols) { + std::vector statsBlob; + auto pushU8 = [&](uint8_t v) { statsBlob.push_back(v); }; + auto pushU32 = [&](uint32_t v) { + statsBlob.push_back(static_cast(v & 0xFF)); + statsBlob.push_back(static_cast((v >> 8) & 0xFF)); + statsBlob.push_back(static_cast((v >> 16) & 0xFF)); + statsBlob.push_back(static_cast((v >> 24) & 0xFF)); + }; + auto pushU64 = [&](uint64_t v) { + for (int i = 0; i < 8; ++i) { + statsBlob.push_back(static_cast((v >> (8 * i)) & 0xFF)); + } + }; + auto pushI64LE = [&](int64_t v) { pushU64(static_cast(v)); }; + auto pushU16LE = [&](uint16_t v) { + statsBlob.push_back(static_cast(v & 0xFF)); + statsBlob.push_back(static_cast((v >> 8) & 0xFF)); + }; + + pushU32(numCols); + for (const auto& s : perCol) { + auto kind = s.lowerBound.kind(); + bool emitSupported = s.hasLowerBound && s.hasUpperBound && s.lowerBound.kind() == s.upperBound.kind() && + (kind == facebook::velox::TypeKind::BIGINT || kind == facebook::velox::TypeKind::INTEGER || + kind == facebook::velox::TypeKind::SMALLINT || kind == facebook::velox::TypeKind::TINYINT || + kind == facebook::velox::TypeKind::HUGEINT || kind == facebook::velox::TypeKind::REAL || + kind == facebook::velox::TypeKind::DOUBLE || kind == facebook::velox::TypeKind::BOOLEAN || + kind == facebook::velox::TypeKind::TIMESTAMP || kind == facebook::velox::TypeKind::VARCHAR); + pushU8(emitSupported ? 1 : 0); + pushU32(static_cast(s.nullCount)); + pushU32(numRows); + pushU64(0); // sizeInBytes placeholder + if (emitSupported) { + switch (kind) { + case facebook::velox::TypeKind::BIGINT: + pushU32(8); + pushI64LE(s.lowerBound.value()); + pushU32(8); + pushI64LE(s.upperBound.value()); + break; + case facebook::velox::TypeKind::INTEGER: + pushU32(4); + pushU32(static_cast(s.lowerBound.value())); + pushU32(4); + pushU32(static_cast(s.upperBound.value())); + break; + case facebook::velox::TypeKind::SMALLINT: + pushU32(2); + pushU16LE(static_cast(s.lowerBound.value())); + pushU32(2); + pushU16LE(static_cast(s.upperBound.value())); + break; + case facebook::velox::TypeKind::TINYINT: + pushU32(1); + pushU8(static_cast(s.lowerBound.value())); + pushU32(1); + pushU8(static_cast(s.upperBound.value())); + break; + case facebook::velox::TypeKind::HUGEINT: { + auto pushI128LE = [&](int128_t v) { + pushU64(static_cast(v)); + pushU64(static_cast(v >> 64)); + }; + pushU32(16); + pushI128LE(s.lowerBound.value()); + pushU32(16); + pushI128LE(s.upperBound.value()); + break; + } + case facebook::velox::TypeKind::REAL: { + uint32_t loBits, hiBits; + float lo = s.lowerBound.value(), hi = s.upperBound.value(); + std::memcpy(&loBits, &lo, sizeof(uint32_t)); + std::memcpy(&hiBits, &hi, sizeof(uint32_t)); + pushU32(4); + pushU32(loBits); + pushU32(4); + pushU32(hiBits); + break; + } + case facebook::velox::TypeKind::DOUBLE: { + uint64_t loBits, hiBits; + double lo = s.lowerBound.value(), hi = s.upperBound.value(); + std::memcpy(&loBits, &lo, sizeof(uint64_t)); + std::memcpy(&hiBits, &hi, sizeof(uint64_t)); + pushU32(8); + pushU64(loBits); + pushU32(8); + pushU64(hiBits); + break; + } + case facebook::velox::TypeKind::BOOLEAN: + pushU32(1); + pushU8(s.lowerBound.value() ? 1 : 0); + pushU32(1); + pushU8(s.upperBound.value() ? 1 : 0); + break; + case facebook::velox::TypeKind::TIMESTAMP: { + const auto& loTs = s.lowerBound.value(); + const auto& hiTs = s.upperBound.value(); + int64_t loMicros = loTs.toMicros(); + int64_t hiMicros = hiTs.toMicros(); + // See framedSerializeWithStats: guard the +1us ceil against int64 overflow at INT64_MAX. + if (hiTs.getNanos() % 1000 != 0 && hiMicros != std::numeric_limits::max()) + hiMicros += 1; + pushU32(8); + pushI64LE(loMicros); + pushU32(8); + pushI64LE(hiMicros); + break; + } + case facebook::velox::TypeKind::VARCHAR: { + const auto& loStr = s.lowerBound.value(); + const auto& hiStr = s.upperBound.value(); + pushU32(static_cast(loStr.size())); + for (auto c : loStr) + pushU8(static_cast(c)); + pushU32(static_cast(hiStr.size())); + for (auto c : hiStr) + pushU8(static_cast(c)); + break; + } + default: + break; + } + } + } + return statsBlob; +} + +std::vector VeloxColumnarBatchSerializer::framedSerializeV3(const std::shared_ptr& batch) { + return framedSerializeV3Impl(batch, false); +} + +std::vector VeloxColumnarBatchSerializer::framedSerializeWithStatsV3( + const std::shared_ptr& batch) { + return framedSerializeV3Impl(batch, true); +} + +std::vector VeloxColumnarBatchSerializer::framedSerializeV3Impl( + const std::shared_ptr& batch, + bool includeStats) { + // Use getFlattenedRowVector() to force-load lazy children and flatten + // DictionaryVector / ConstantVector encodings before serializeSingleColumn. + // This can increase V3 cache bytes for dictionary-heavy inputs, but keeps the + // per-column payload readable by PrestoVectorSerde. + auto vb = VeloxColumnarBatch::from(veloxPool_.get(), batch); + auto rowVector = vb->getFlattenedRowVector(); + GLUTEN_CHECK( + rowVector->size() >= 0 && static_cast(rowVector->size()) <= std::numeric_limits::max(), + "V3 row count exceeds u32 frame limit: " + std::to_string(rowVector->size())); + GLUTEN_CHECK( + rowVector->childrenSize() <= std::numeric_limits::max(), + "V3 column count exceeds u32 frame limit: " + std::to_string(rowVector->childrenSize())); + const uint32_t numRows = static_cast(rowVector->size()); + const uint32_t numCols = static_cast(rowVector->childrenSize()); + + // 1. Optionally compute stats. Lazy materialization itself is a base V3 + // capability; partition stats are an optional payload used only for pruning. + std::vector statsBlob; + if (includeStats) { + std::vector perCol = computeStats(rowVector); + statsBlob = buildStatsBlob(perCol, numRows, numCols); + GLUTEN_CHECK( + statsBlob.size() <= static_cast(std::numeric_limits::max()), + "V3 stats blob size exceeds Java byte array limit"); + } + + // 2. Serialize each column independently using serializeSingleColumn. + // options_ must have compressionKind=NONE and nullsFirst=false (checked by Velox). + facebook::velox::serializer::presto::PrestoVectorSerde localSerde; + auto pushU32LE = [](std::vector& buf, uint32_t v) { + buf.push_back(static_cast(v & 0xFF)); + buf.push_back(static_cast((v >> 8) & 0xFF)); + buf.push_back(static_cast((v >> 16) & 0xFF)); + buf.push_back(static_cast((v >> 24) & 0xFF)); + }; + + std::vector> colBytesList(numCols); + for (uint32_t col = 0; col < numCols; ++col) { + std::ostringstream colStream; + localSerde.serializeSingleColumn(rowVector->childAt(col), &options_, veloxPool_.get(), &colStream); + const std::string colStr = colStream.str(); + GLUTEN_CHECK( + colStr.size() <= static_cast(std::numeric_limits::max()), + "V3 column " + std::to_string(col) + " size exceeds u32 limit"); + GLUTEN_CHECK( + colStr.size() <= static_cast(std::numeric_limits::max()), + "V3 column " + std::to_string(col) + " size exceeds int32 ByteRange limit"); + colBytesList[col] = std::vector(colStr.begin(), colStr.end()); + } + + // 3. Assemble V3 framed format. + // [magic=0x03(4B)][statsLen(4B)][statsBlob][numRows(4B)][numCols(4B)][per-col: colLen+colBytes] + std::vector framed; + const uint32_t statsLen = static_cast(statsBlob.size()); + // Capacity hint only (does not affect emitted bytes). Compute in size_t so the + // per-column length-prefix term cannot wrap for pathological column counts. + framed.reserve(static_cast(4) + 4 + statsLen + 4 + 4 + static_cast(numCols) * 4); + // V3 magic + framed.push_back(0xFE); + framed.push_back(0xCA); + framed.push_back(0x53); + framed.push_back(0x03); + pushU32LE(framed, statsLen); + framed.insert(framed.end(), statsBlob.begin(), statsBlob.end()); + pushU32LE(framed, numRows); + pushU32LE(framed, numCols); + for (uint32_t col = 0; col < numCols; ++col) { + const auto& cb = colBytesList[col]; + pushU32LE(framed, static_cast(cb.size())); + framed.insert(framed.end(), cb.begin(), cb.end()); + } + GLUTEN_CHECK( + framed.size() <= static_cast(std::numeric_limits::max()), + "V3 framed payload size exceeds Java byte array limit"); + return framed; +} + +std::shared_ptr VeloxColumnarBatchSerializer::deserializeV3( + uint8_t* data, + int32_t size, + const std::optional>& requestedColumns) { + // Local helpers. + auto readU32LE = [](const uint8_t*& p) -> uint32_t { + uint32_t v = static_cast(p[0]) | (static_cast(p[1]) << 8) | + (static_cast(p[2]) << 16) | (static_cast(p[3]) << 24); + p += 4; + return v; + }; + + GLUTEN_CHECK(size >= 0, "V3 frame size must be non-negative"); + const uint8_t* p = data; + const uint8_t* end = data + size; + auto remaining = [&]() -> size_t { return static_cast(end - p); }; + auto requireRemaining = [&](size_t n, const std::string& message) { GLUTEN_CHECK(remaining() >= n, message); }; + + // 1. Validate V3 magic. + GLUTEN_CHECK(size >= 4, "V3 frame too short for magic"); + GLUTEN_CHECK( + p[0] == 0xFE && p[1] == 0xCA && p[2] == 0x53 && p[3] == 0x03, + "deserializeV3: magic mismatch (expected V3=0x03, got 0x" + std::to_string(p[3] & 0xFF) + ")"); + p += 4; + + // 2. Skip statsBlob (parsed by JVM side). + requireRemaining(4, "V3 frame truncated before statsLen"); + uint32_t statsLen = readU32LE(p); + requireRemaining(statsLen, "V3 frame statsBlob truncated"); + p += statsLen; + + // 3. Read numRows and numCols. + requireRemaining(8, "V3 frame truncated before numRows/numCols"); + uint32_t numRows = readU32LE(p); + uint32_t numCols = readU32LE(p); + GLUTEN_CHECK( + numRows <= static_cast(std::numeric_limits::max()), + "V3 frame numRows exceeds Velox vector_size_t max: " + std::to_string(numRows)); + const auto expectedRows = static_cast(numRows); + + GLUTEN_CHECK(rowType_ != nullptr, "deserializeV3: rowType_ not initialized"); + GLUTEN_CHECK( + rowType_->size() == numCols, + "V3 frame numCols=" + std::to_string(numCols) + " != schema size=" + std::to_string(rowType_->size())); + + // 4. Read per-column byte ranges; requested columns are copied before loaders are built. + struct ColRange { + const uint8_t* start; + uint32_t len; + }; + std::vector colRanges(numCols); + for (uint32_t col = 0; col < numCols; ++col) { + requireRemaining(4, "V3 frame truncated at colLen col=" + std::to_string(col)); + uint32_t colLen = readU32LE(p); + requireRemaining(colLen, "V3 frame colBytes truncated at col=" + std::to_string(col)); + colRanges[col] = {p, colLen}; + p += colLen; + } + GLUTEN_CHECK( + p == end, + "V3 frame has trailing bytes after column payloads: trailing=" + std::to_string(static_cast(end - p))); + + // 5. Determine requested columns. + const bool loadAll = !requestedColumns.has_value(); + // Use value (not reference) to avoid binding a const-ref to a temporary (C++ UB when loadAll=true). + const std::vector reqVec = loadAll ? std::vector{} : requestedColumns.value(); + const uint32_t M = loadAll ? numCols : static_cast(reqVec.size()); + + // 6. Build M-column subset RowVector with LazyVector children. + std::vector subNames(M); + std::vector subTypes(M); + std::vector subChildren(M); + + const uint32_t i_limit = M; + for (uint32_t i = 0; i < i_limit; ++i) { + const int32_t col = loadAll ? static_cast(i) : reqVec[i]; + GLUTEN_CHECK( + col >= 0 && static_cast(col) < numCols, + "deserializeV3: requestedColumn " + std::to_string(col) + " out of range [0," + std::to_string(numCols) + ")"); + + const auto& range = colRanges[col]; + auto colType = rowType_->childAt(col); + + if (range.len == 0 && numRows == 0) { + // Truly empty batch (0 rows): any column type is safely represented as null constant. + subChildren[i] = facebook::velox::BaseVector::createNullConstant(colType, 0, veloxPool_.get()); + } else if (range.len == 0) { + // numRows > 0 but colLen == 0: this is malformed — serializeSingleColumn always emits at + // least a column header. Treat as a serialization bug; surface clearly rather than silently + // substituting wrong-type null data. + GLUTEN_CHECK( + false, + "V3 deserializeV3: col=" + std::to_string(col) + " colLen=0 but numRows=" + std::to_string(numRows) + + " (malformed V3 frame; serializeSingleColumn never emits 0 bytes for non-empty batch)"); + } else { + // Copy bytes while JNI pin is still valid. + std::vector colBytes(range.start, range.start + range.len); + auto loader = + std::make_unique(std::move(colBytes), colType, expectedRows, veloxPool_, options_); + subChildren[i] = + std::make_shared(veloxPool_.get(), colType, expectedRows, std::move(loader)); + } + subTypes[i] = colType; + subNames[i] = rowType_->nameOf(col); + } + + // 7. Construct M-column RowVector. + auto subRowType = facebook::velox::ROW(std::move(subNames), std::move(subTypes)); + auto result = std::make_shared( + veloxPool_.get(), subRowType, facebook::velox::BufferPtr(nullptr), expectedRows, std::move(subChildren)); + + return std::make_shared(result); +} + } // namespace gluten diff --git a/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.h b/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.h index 860c3ec5361..33ff1301b6b 100644 --- a/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.h +++ b/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.h @@ -18,6 +18,7 @@ #pragma once #include +#include #include "memory/ColumnarBatch.h" #include "operators/serializer/ColumnarBatchSerializer.h" @@ -57,11 +58,30 @@ class VeloxColumnarBatchSerializer : public ColumnarBatchSerializer { // false so the JVM side falls back to pass-through in buildFilter. std::vector computeStats(facebook::velox::RowVectorPtr rowVector); - // Returns framed bytes: [STATS_FRAMED_MAGIC: 4B] [statsLen: u32 LE] [statsBlob] [bytesLen: u32 LE] - // [bytesBlob]. statsBlob layout matches the JVM-side reader (CachedColumnarBatchKryoSerializer - // .deserializeStats). Magic 0xFE 0xCA 0x53 0x02 aligns with the JVM Kryo STATS_FRAMED_MAGIC. + // V2: Returns framed bytes [STATS_FRAMED_MAGIC=0x02: 4B][statsLen: u32 LE][statsBlob] + // [bytesLen: u32 LE][bytesBlob]. statsBlob matches JVM CachedColumnarBatchKryoSerializer. std::vector framedSerializeWithStats(const std::shared_ptr& batch) override; + // V3: Per-column framed bytes [STATS_FRAMED_MAGIC_V3=0x03: 4B][statsLen=0: u32 LE] + // [numRows: u32 LE][numCols: u32 LE][per-col: colLen(u32 LE) + colBytes]. + // Each colBytes is produced by PrestoVectorSerde::serializeSingleColumn (self-contained). + std::vector framedSerializeV3(const std::shared_ptr& batch) override; + + // V3: Per-column framed bytes [STATS_FRAMED_MAGIC_V3=0x03: 4B][statsLen: u32 LE][statsBlob] + // [numRows: u32 LE][numCols: u32 LE][per-col: colLen(u32 LE) + colBytes]. + // Each colBytes is produced by PrestoVectorSerde::serializeSingleColumn (self-contained). + std::vector framedSerializeWithStatsV3(const std::shared_ptr& batch) override; + + // V3: Deserialize with column projection; returns M-column RowVector. + // requestedColumns: nullopt=all columns, optional=zero cols, optional=M cols. + std::shared_ptr + deserializeV3(uint8_t* data, int32_t size, const std::optional>& requestedColumns) override; + + private: + // Extract statsBlob from per-column stats (shared by V2 and V3 write paths). + std::vector buildStatsBlob(const std::vector& perCol, uint32_t numRows, uint32_t numCols); + std::vector framedSerializeV3Impl(const std::shared_ptr& batch, bool includeStats); + protected: std::shared_ptr veloxPool_; std::unique_ptr arena_; diff --git a/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc b/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc index d51a9304ffb..a29f98e8beb 100644 --- a/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc +++ b/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc @@ -23,6 +23,7 @@ #include "memory/VeloxColumnarBatch.h" #include "operators/serializer/VeloxColumnarBatchSerializer.h" +#include "velox/vector/LazyVector.h" #include "velox/vector/arrow/Bridge.h" #include "velox/vector/tests/utils/VectorTestBase.h" @@ -44,6 +45,17 @@ class VeloxColumnarBatchSerializerTest : public ::testing::Test, public test::Ve static void TearDownTestSuite() { VeloxBackend::get()->tearDown(); } + + std::shared_ptr makeV3Deserializer(arrow::MemoryPool* arrowPool) { + auto schemaVector = makeRowVector( + {"a", "b", "c"}, + {makeFlatVector({1, 2, 3, 4, 5}), + makeFlatVector({10, 20, 30, 40, 50}), + makeFlatVector({"a", "bb", "ccc", "dddd", "eeeee"})}); + ArrowSchema cSchema; + exportToArrow(schemaVector, cSchema, ArrowUtils::getBridgeOptions()); + return std::make_shared(arrowPool, pool_, &cSchema); + } }; TEST_F(VeloxColumnarBatchSerializerTest, serialize) { @@ -98,10 +110,13 @@ TEST_F(VeloxColumnarBatchSerializerTest, computeStatsBigintFlatVector) { EXPECT_EQ(stats[0].nullCount, 0); } -// REAL FlatVector: no-NaN partition becomes supported; any NaN row poisons the -// column to hasLowerBound=hasUpperBound=false (Spark NaN != NaN, would silently +// REAL FlatVector: a no-NaN partition is supported; a partition that mixes NaN with finite +// values stays supported and emits the FINITE min/max (NaN skipped) -- matching vanilla Spark +// Float/DoubleColumnStats, which ignores NaN and keeps finite bounds. Poisoning to +// hasLowerBound=hasUpperBound=false here would emit null bounds and let the vanilla buildFilter +// silently prune finite matching rows (data loss). (Spark NaN != NaN, would silently // drop matching rows under min/max pruning). -TEST_F(VeloxColumnarBatchSerializerTest, computeStatsNaNRealPartitionPoisons) { +TEST_F(VeloxColumnarBatchSerializerTest, computeStatsNaNRealSkippedFiniteBoundsEmitted) { auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); auto serializer = std::make_shared(arrowPool, pool_, nullptr); @@ -115,9 +130,13 @@ TEST_F(VeloxColumnarBatchSerializerTest, computeStatsNaNRealPartitionPoisons) { ASSERT_EQ(stats.size(), 1u); EXPECT_TRUE(stats[0].hasLowerBound) << "REAL FlatVector w/o NaN must be supported"; EXPECT_TRUE(stats[0].hasUpperBound); + EXPECT_EQ(stats[0].lowerBound.value(), 0.5f); + EXPECT_EQ(stats[0].upperBound.value(), 3.5f); } - // (b) REAL FlatVector WITH NaN -- must fall back to unsupported. + // (b) REAL FlatVector mixing NaN with finite values -- NaN is skipped (matches vanilla Spark), + // the column stays supported, and the FINITE [1.5, 3.5] bounds are emitted. Emitting null + // bounds here would let the vanilla buildFilter prune the batch and drop the finite rows. { const float nan = std::numeric_limits::quiet_NaN(); std::vector children = { @@ -126,7 +145,24 @@ TEST_F(VeloxColumnarBatchSerializerTest, computeStatsNaNRealPartitionPoisons) { auto vector = makeRowVector(children); auto stats = serializer->computeStats(vector); ASSERT_EQ(stats.size(), 1u); - EXPECT_FALSE(stats[0].hasLowerBound) << "NaN-poisoned REAL column must emit hasLowerBound=false (NB4)"; + EXPECT_TRUE(stats[0].hasLowerBound) << "NaN must be skipped, finite bounds still emitted (no poison)"; + EXPECT_TRUE(stats[0].hasUpperBound); + EXPECT_EQ(stats[0].lowerBound.value(), 1.5f); + EXPECT_EQ(stats[0].upperBound.value(), 3.5f); + } + + // (c) REAL FlatVector that is ALL NaN -- no finite value observed, so no bounds are emitted + // (hasLowerBound=false). With null bounds the JVM stats row stays unsupported for this column; + // there are no finite values that could be wrongly pruned. + { + const float nan = std::numeric_limits::quiet_NaN(); + std::vector children = { + makeFlatVector({nan, nan, nan}), + }; + auto vector = makeRowVector(children); + auto stats = serializer->computeStats(vector); + ASSERT_EQ(stats.size(), 1u); + EXPECT_FALSE(stats[0].hasLowerBound) << "all-NaN column observes no finite value -> no bounds"; EXPECT_FALSE(stats[0].hasUpperBound); } } @@ -683,26 +719,26 @@ TEST_F(VeloxColumnarBatchSerializerTest, computeStatsBooleanFlatVectorWithNulls) EXPECT_EQ(stats[0].nullCount, 2); } -// NaN-poisoned float column must STILL accrue real nullCount (not -// early-return). framed stats serialize nullCount even when -// emitSupported=0; under-counting on `[NaN, null]` would let -// `col IS NULL` predicates incorrectly prune matching rows under -// Spark IsNull pruning. +// A float column mixing NaN with nulls and finite values: NaN is skipped (matches vanilla), +// finite bounds are still emitted, and nullCount counts only true nulls (NaN is non-null). +// Under-counting nullCount would let `col IS NULL` pruning (`nullCount > 0`) wrongly drop rows. TEST_F(VeloxColumnarBatchSerializerTest, computeStatsNaNFloatStillCountsNulls) { auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); auto serializer = std::make_shared(arrowPool, pool_, nullptr); const float nan = std::numeric_limits::quiet_NaN(); std::vector children = { - // [1.0, null, NaN, null, 2.0] -- 2 nulls, 1 NaN poisons min/max. + // [1.0, null, NaN, null, 2.0] -- 2 nulls, 1 NaN (skipped), finite bounds [1.0, 2.0]. makeNullableFlatVector({1.0f, std::nullopt, nan, std::nullopt, 2.0f}), }; auto vector = makeRowVector(children); auto stats = serializer->computeStats(vector); ASSERT_EQ(stats.size(), 1u); - EXPECT_FALSE(stats[0].hasLowerBound) << "NaN poisons min/max -> unsupported"; - EXPECT_FALSE(stats[0].hasUpperBound); - EXPECT_EQ(stats[0].nullCount, 2) << "NaN scan must continue and count both nulls (IsNull prune correctness)"; + EXPECT_TRUE(stats[0].hasLowerBound) << "NaN skipped, finite bounds still emitted (matches vanilla)"; + EXPECT_TRUE(stats[0].hasUpperBound); + EXPECT_EQ(stats[0].lowerBound.value(), 1.0f); + EXPECT_EQ(stats[0].upperBound.value(), 2.0f); + EXPECT_EQ(stats[0].nullCount, 2) << "NaN is non-null; only true nulls counted (IsNull prune correctness)"; } // Non-flat encoding (Dictionary / Constant / Complex) must still @@ -1023,4 +1059,321 @@ TEST_F(VeloxColumnarBatchSerializerTest, framedSerializeWithStatsAllNullColNoBou // No lowerLen / upperLen / bounds bytes follow when supported=0. } +// Cross-language V3 golden-frame pin. +// +// Empty 0-row / 0-col input has no per-column PrestoSerde payload, so this +// byte-equal golden pins the V3 top-level frame contract: +// [ magic=0x03 ][ statsLen ][ statsBlob(numCols=0) ][ numRows=0 ][ numCols=0 ] +// Paired with ColumnarCachedBatchFramedBytesSuite's kGoldenFrameV3Empty parser +// round-trip on the JVM side. +TEST_F(VeloxColumnarBatchSerializerTest, framedSerializeWithStatsV3EmptyGolden) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + auto rowVector = makeRowVector(ROW({}, {}), 0); + auto batch = std::make_shared(rowVector); + + std::vector actual = serializer->framedSerializeWithStatsV3(batch); + + static const std::vector kGoldenFrame = {0xFE, 0xCA, 0x53, 0x03, 0x04, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00}; + EXPECT_EQ(actual, kGoldenFrame) << "framedSerializeWithStatsV3 empty frame diverged."; +} + +TEST_F(VeloxColumnarBatchSerializerTest, framedSerializeV3NoStatsEmptyGolden) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + auto rowVector = makeRowVector(ROW({}, {}), 0); + auto batch = std::make_shared(rowVector); + + std::vector actual = serializer->framedSerializeV3(batch); + + static const std::vector kGoldenFrame = { + 0xFE, 0xCA, 0x53, 0x03, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00}; + EXPECT_EQ(actual, kGoldenFrame) << "framedSerializeV3 no-stats empty frame diverged."; +} + +namespace { + +uint32_t readU32LE(const std::vector& bytes, size_t& offset) { + GLUTEN_CHECK(offset + 4 <= bytes.size(), "readU32LE offset out of range"); + uint32_t value = static_cast(bytes[offset]) | (static_cast(bytes[offset + 1]) << 8) | + (static_cast(bytes[offset + 2]) << 16) | (static_cast(bytes[offset + 3]) << 24); + offset += 4; + return value; +} + +void appendU32LE(std::vector& bytes, uint32_t value) { + bytes.push_back(static_cast(value & 0xFF)); + bytes.push_back(static_cast((value >> 8) & 0xFF)); + bytes.push_back(static_cast((value >> 16) & 0xFF)); + bytes.push_back(static_cast((value >> 24) & 0xFF)); +} + +void appendU64LE(std::vector& bytes, uint64_t value) { + for (int i = 0; i < 8; ++i) { + bytes.push_back(static_cast((value >> (8 * i)) & 0xFF)); + } +} + +void assertV3WriterFrameLayout( + const std::vector& framed, + uint32_t expectedRows, + uint32_t expectedColumns, + bool expectStats) { + ASSERT_GE(framed.size(), 16u); + ASSERT_EQ(framed[0], 0xFE); + ASSERT_EQ(framed[1], 0xCA); + ASSERT_EQ(framed[2], 0x53); + ASSERT_EQ(framed[3], 0x03); + + size_t offset = 4; + const uint32_t statsLen = readU32LE(framed, offset); + if (expectStats) { + ASSERT_GT(statsLen, 0u); + ASSERT_GE(framed.size() - offset, statsLen); + size_t statsOffset = offset; + ASSERT_EQ(readU32LE(framed, statsOffset), expectedColumns); + } else { + ASSERT_EQ(statsLen, 0u); + } + offset += statsLen; + + ASSERT_EQ(readU32LE(framed, offset), expectedRows); + ASSERT_EQ(readU32LE(framed, offset), expectedColumns); + for (uint32_t col = 0; col < expectedColumns; ++col) { + const uint32_t colLen = readU32LE(framed, offset); + ASSERT_GT(colLen, 0u) << "non-empty writer frame must include serialized bytes for col " << col; + ASSERT_GE(framed.size() - offset, colLen); + offset += colLen; + } + ASSERT_EQ(offset, framed.size()) << "writer frame must not contain trailing bytes"; +} + +std::vector v3NoStatsNonEmptyFrameFixture() { + return {0xFE, 0xCA, 0x53, 0x03, 0x00, 0x00, 0x00, 0x00, 0x05, 0x00, 0x00, 0x00, 0x03, 0x00, 0x00, 0x00, 0x01, + 0x00, 0x00, 0x00, 0xaa, 0x02, 0x00, 0x00, 0x00, 0xbb, 0xcc, 0x03, 0x00, 0x00, 0x00, 0xdd, 0xee, 0xff}; +} + +std::vector v3WithStatsNonEmptyFrameFixture() { + std::vector stats; + appendU32LE(stats, 3); + for (uint32_t col = 0; col < 3; ++col) { + stats.push_back(0); // unsupported: no lower/upper bound payload follows. + appendU32LE(stats, 0); // nullCount + appendU32LE(stats, 5); // count + appendU64LE(stats, 0); // sizeInBytes placeholder + } + + std::vector framed = {0xFE, 0xCA, 0x53, 0x03}; + appendU32LE(framed, static_cast(stats.size())); + framed.insert(framed.end(), stats.begin(), stats.end()); + appendU32LE(framed, 5); + appendU32LE(framed, 3); + framed.insert( + framed.end(), + {0x01, 0x00, 0x00, 0x00, 0xaa, 0x02, 0x00, 0x00, 0x00, 0xbb, 0xcc, 0x03, 0x00, 0x00, 0x00, 0xdd, 0xee, 0xff}); + return framed; +} + +} // namespace + +TEST_F(VeloxColumnarBatchSerializerTest, framedSerializeV3NonEmptyWriterLayout) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + std::vector children = { + makeNullableFlatVector({1, 2, std::nullopt, 4, 5}), + makeFlatVector({10, 20, 30, 40, 50}), + makeFlatVector({"a", "bb", "ccc", "dddd", "eeeee"}), + }; + auto vector = makeRowVector({"a", "b", "c"}, children); + auto batch = std::make_shared(vector); + + auto framed = serializer->framedSerializeV3(batch); + + assertV3WriterFrameLayout(framed, 5, 3, false); + + std::vector requestedColumns = {0, 2}; + auto deserializer = makeV3Deserializer(arrowPool); + auto projectedBatch = deserializer->deserializeV3( + framed.data(), static_cast(framed.size()), std::optional>(requestedColumns)); + auto projectedVector = std::dynamic_pointer_cast(projectedBatch)->getRowVector(); + auto expectedProjected = makeRowVector({"a", "c"}, {children[0], children[2]}); + test::assertEqualVectors(expectedProjected, projectedVector); +} + +TEST_F(VeloxColumnarBatchSerializerTest, framedSerializeWithStatsV3NonEmptyWriterLayout) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + std::vector children = { + makeNullableFlatVector({1, 2, std::nullopt, 4, 5}), + makeFlatVector({10, 20, 30, 40, 50}), + makeFlatVector({"a", "bb", "ccc", "dddd", "eeeee"}), + }; + auto vector = makeRowVector({"a", "b", "c"}, children); + auto batch = std::make_shared(vector); + + auto framed = serializer->framedSerializeWithStatsV3(batch); + + assertV3WriterFrameLayout(framed, 5, 3, true); + + std::vector requestedColumns = {0, 2}; + auto deserializer = makeV3Deserializer(arrowPool); + auto projectedBatch = deserializer->deserializeV3( + framed.data(), static_cast(framed.size()), std::optional>(requestedColumns)); + auto projectedVector = std::dynamic_pointer_cast(projectedBatch)->getRowVector(); + auto expectedProjected = makeRowVector({"a", "c"}, {children[0], children[2]}); + test::assertEqualVectors(expectedProjected, projectedVector); +} + +TEST_F(VeloxColumnarBatchSerializerTest, deserializeV3ZeroProjectionNonEmptyNoStatsFrameFixture) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto deserializer = makeV3Deserializer(arrowPool); + auto framed = v3NoStatsNonEmptyFrameFixture(); + std::vector requestedColumns; + + assertV3WriterFrameLayout(framed, 5, 3, false); + auto projectedBatch = deserializer->deserializeV3( + framed.data(), static_cast(framed.size()), std::optional>(requestedColumns)); + auto projectedVector = std::dynamic_pointer_cast(projectedBatch)->getRowVector(); + + ASSERT_EQ(projectedVector->size(), 5); + ASSERT_EQ(projectedVector->childrenSize(), 0u); +} + +TEST_F(VeloxColumnarBatchSerializerTest, deserializeV3ZeroProjectionNonEmptyWithStatsFrameFixture) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto deserializer = makeV3Deserializer(arrowPool); + auto framed = v3WithStatsNonEmptyFrameFixture(); + std::vector requestedColumns; + + assertV3WriterFrameLayout(framed, 5, 3, true); + auto projectedBatch = deserializer->deserializeV3( + framed.data(), static_cast(framed.size()), std::optional>(requestedColumns)); + auto projectedVector = std::dynamic_pointer_cast(projectedBatch)->getRowVector(); + + ASSERT_EQ(projectedVector->size(), 5); + ASSERT_EQ(projectedVector->childrenSize(), 0u); +} + +TEST_F(VeloxColumnarBatchSerializerTest, deserializeV3RejectsTrailingBytes) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto deserializer = makeV3Deserializer(arrowPool); + auto framed = v3NoStatsNonEmptyFrameFixture(); + framed.push_back(0x42); + std::vector requestedColumns; + + EXPECT_THROW( + (void)deserializer->deserializeV3( + framed.data(), static_cast(framed.size()), std::optional>(requestedColumns)), + GlutenException); +} + +TEST_F(VeloxColumnarBatchSerializerTest, deserializeV3RejectsTruncatedColumnBytes) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto deserializer = makeV3Deserializer(arrowPool); + auto framed = v3NoStatsNonEmptyFrameFixture(); + framed.pop_back(); + std::vector requestedColumns; + + EXPECT_THROW( + (void)deserializer->deserializeV3( + framed.data(), static_cast(framed.size()), std::optional>(requestedColumns)), + GlutenException); +} + +TEST_F(VeloxColumnarBatchSerializerTest, deserializeV3RejectsSchemaNumColsMismatch) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto deserializer = makeV3Deserializer(arrowPool); + auto framed = v3NoStatsNonEmptyFrameFixture(); + framed[12] = 0x04; + std::vector requestedColumns; + + EXPECT_THROW( + (void)deserializer->deserializeV3( + framed.data(), static_cast(framed.size()), std::optional>(requestedColumns)), + GlutenException); +} + +TEST_F(VeloxColumnarBatchSerializerTest, deserializeV3RejectsRequestedColumnOutOfRange) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto deserializer = makeV3Deserializer(arrowPool); + auto framed = v3NoStatsNonEmptyFrameFixture(); + std::vector requestedColumns = {3}; + + EXPECT_THROW( + (void)deserializer->deserializeV3( + framed.data(), static_cast(framed.size()), std::optional>(requestedColumns)), + GlutenException); +} + +TEST_F(VeloxColumnarBatchSerializerTest, deserializeV3RejectsDecodedColumnRowCountMismatch) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + + std::vector children = { + makeNullableFlatVector({1, 2, std::nullopt, 4, 5}), + makeFlatVector({10, 20, 30, 40, 50}), + makeFlatVector({"a", "bb", "ccc", "dddd", "eeeee"}), + }; + auto vector = makeRowVector({"a", "b", "c"}, children); + auto batch = std::make_shared(vector); + + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + std::vector framed = serializer->framedSerializeV3(batch); + + // No-stats V3 layout puts numRows at offset 8. Keep the 5-row column payloads intact, but + // declare 4 rows in the frame header so the lazy column loader must reject the mismatch. + ASSERT_GE(framed.size(), 12u); + framed[8] = 0x04; + framed[9] = 0x00; + framed[10] = 0x00; + framed[11] = 0x00; + + ArrowSchema cSchema; + exportToArrow(vector, cSchema, ArrowUtils::getBridgeOptions()); + auto deserializer = std::make_shared(arrowPool, pool_, &cSchema); + std::vector requestedColumns = {0}; + + auto projectedBatch = deserializer->deserializeV3( + framed.data(), static_cast(framed.size()), std::optional>(requestedColumns)); + auto projectedVector = std::dynamic_pointer_cast(projectedBatch)->getRowVector(); + + ASSERT_EQ(projectedVector->size(), 4); + ASSERT_TRUE(isLazyNotLoaded(*projectedVector->childAt(0))); + EXPECT_THROW((void)projectedVector->childAt(0)->loadedVector(), GlutenException); +} + +TEST_F(VeloxColumnarBatchSerializerTest, deserializeV3ProjectsNonEmptyFrameAsLazyColumns) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + + std::vector children = { + makeNullableFlatVector({1, 2, std::nullopt, 4, 5}), + makeFlatVector({10, 20, 30, 40, 50}), + makeFlatVector({"a", "bb", "ccc", "dddd", "eeeee"}), + }; + auto vector = makeRowVector({"a", "b", "c"}, children); + auto batch = std::make_shared(vector); + + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + std::vector framed = serializer->framedSerializeV3(batch); + + ArrowSchema cSchema; + exportToArrow(vector, cSchema, ArrowUtils::getBridgeOptions()); + auto deserializer = std::make_shared(arrowPool, pool_, &cSchema); + std::vector requestedColumns = {0, 2}; + + auto projectedBatch = deserializer->deserializeV3( + framed.data(), static_cast(framed.size()), std::optional>(requestedColumns)); + auto projectedVector = std::dynamic_pointer_cast(projectedBatch)->getRowVector(); + + ASSERT_EQ(projectedVector->size(), vector->size()); + ASSERT_EQ(projectedVector->childrenSize(), 2u); + ASSERT_TRUE(isLazyNotLoaded(*projectedVector->childAt(0))); + ASSERT_TRUE(isLazyNotLoaded(*projectedVector->childAt(1))); + + auto expected = makeRowVector({"a", "c"}, {children[0], children[2]}); + test::assertEqualVectors(expected, projectedVector); +} + } // namespace gluten diff --git a/docs/Configuration.md b/docs/Configuration.md index 748a200cfbf..f5dc569b0a0 100644 --- a/docs/Configuration.md +++ b/docs/Configuration.md @@ -20,133 +20,133 @@ nav_order: 15 ## Gluten configurations -| Key | Modifiability | Default | Description | -|---------------------------------------------------------------------|---------------|-------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| spark.gluten.costModel | 🔄 Dynamic | legacy | The class name of user-defined cost model that will be used by Gluten's transition planner. If not specified, a legacy built-in cost model will be used. The legacy cost model helps RAS planner exhaustively offload computations, and helps transition planner choose columnar-to-columnar transition over others. | -| spark.gluten.enabled | 🔄 Dynamic | true | Whether to enable gluten. Default value is true. Just an experimental property. Recommend to enable/disable Gluten through the setting for spark.plugins. | -| spark.gluten.execution.resource.expired.time | 🔄 Dynamic | 86400 | Expired time of execution with resource relation has cached. | -| spark.gluten.expression.blacklist | 🔄 Dynamic | <undefined> | A black list of expression to skip transform, multiple values separated by commas. | -| spark.gluten.loadLibFromJar | 🔄 Dynamic | false | Whether to load shared libraries from jars. | -| spark.gluten.loadLibOS | 🔄 Dynamic | <undefined> | The shared library loader's OS name. | -| spark.gluten.loadLibOSVersion | 🔄 Dynamic | <undefined> | The shared library loader's OS version. | -| spark.gluten.memory.isolation | 🔄 Dynamic | false | Enable isolated memory mode. If true, Gluten controls the maximum off-heap memory can be used by each task to X, X = executor memory / max task slots. It's recommended to set true if Gluten serves concurrent queries within a single session, since not all memory Gluten allocated is guaranteed to be spillable. In the case, the feature should be enabled to avoid OOM. | -| spark.gluten.memory.overAcquiredMemoryRatio | 🔄 Dynamic | 0.3 | If larger than 0, Velox backend will try over-acquire this ratio of the total allocated memory as backup to avoid OOM. | -| spark.gluten.memory.reservationBlockSize | 🔄 Dynamic | 8MB | Block size of native reservation listener reserve memory from Spark. | -| spark.gluten.numTaskSlotsPerExecutor | 🔄 Dynamic | -1 | Must provide default value since non-execution operations (e.g. org.apache.spark.sql.Dataset#summary) doesn't propagate configurations using org.apache.spark.sql.execution.SQLExecution#withSQLConfPropagated | +| Key | Modifiability | Default | Description | +|---------------------------------------------------------------------|---------------|-------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| spark.gluten.costModel | 🔄 Dynamic | legacy | The class name of user-defined cost model that will be used by Gluten's transition planner. If not specified, a legacy built-in cost model will be used. The legacy cost model helps RAS planner exhaustively offload computations, and helps transition planner choose columnar-to-columnar transition over others. | +| spark.gluten.enabled | 🔄 Dynamic | true | Whether to enable gluten. Default value is true. Just an experimental property. Recommend to enable/disable Gluten through the setting for spark.plugins. | +| spark.gluten.execution.resource.expired.time | 🔄 Dynamic | 86400 | Expired time of execution with resource relation has cached. | +| spark.gluten.expression.blacklist | 🔄 Dynamic | <undefined> | A black list of expression to skip transform, multiple values separated by commas. | +| spark.gluten.loadLibFromJar | 🔄 Dynamic | false | Whether to load shared libraries from jars. | +| spark.gluten.loadLibOS | 🔄 Dynamic | <undefined> | The shared library loader's OS name. | +| spark.gluten.loadLibOSVersion | 🔄 Dynamic | <undefined> | The shared library loader's OS version. | +| spark.gluten.memory.isolation | 🔄 Dynamic | false | Enable isolated memory mode. If true, Gluten controls the maximum off-heap memory can be used by each task to X, X = executor memory / max task slots. It's recommended to set true if Gluten serves concurrent queries within a single session, since not all memory Gluten allocated is guaranteed to be spillable. In the case, the feature should be enabled to avoid OOM. | +| spark.gluten.memory.overAcquiredMemoryRatio | 🔄 Dynamic | 0.3 | If larger than 0, Velox backend will try over-acquire this ratio of the total allocated memory as backup to avoid OOM. | +| spark.gluten.memory.reservationBlockSize | 🔄 Dynamic | 8MB | Block size of native reservation listener reserve memory from Spark. | +| spark.gluten.numTaskSlotsPerExecutor | 🔄 Dynamic | -1 | Must provide default value since non-execution operations (e.g. org.apache.spark.sql.Dataset#summary) doesn't propagate configurations using org.apache.spark.sql.execution.SQLExecution#withSQLConfPropagated | | spark.gluten.shuffleWriter.bufferSize | 🔄 Dynamic | <undefined> | -| spark.gluten.soft-affinity.duplicateReading.maxCacheItems | 🔄 Dynamic | 10000 | Enable Soft Affinity duplicate reading detection | -| spark.gluten.soft-affinity.duplicateReadingDetect.enabled | 🔄 Dynamic | false | If true, Enable Soft Affinity duplicate reading detection | -| spark.gluten.soft-affinity.enabled | 🔄 Dynamic | false | Whether to enable Soft Affinity scheduling. | -| spark.gluten.soft-affinity.min.target-hosts | 🔄 Dynamic | 1 | For on HDFS, if there are already target hosts, and then prefer to use the original target hosts to schedule | -| spark.gluten.soft-affinity.replications.num | 🔄 Dynamic | 2 | Calculate the number of the replications for scheduling to the target executors per file | -| spark.gluten.sql.adaptive.costEvaluator.enabled | ⚓ Static | true | If true, use org.apache.spark.sql.execution.adaptive.GlutenCostEvaluator as custom cost evaluator class, else follow the configuration spark.sql.adaptive.customCostEvaluatorClass. | -| spark.gluten.sql.ansiFallback.enabled | 🔄 Dynamic | true | When true (default), Gluten will fall back to Spark when ANSI mode is enabled. When false, Gluten will attempt to execute in ANSI mode. | -| spark.gluten.sql.broadcastNestedLoopJoinTransformerEnabled | 🔄 Dynamic | true | Config to enable BroadcastNestedLoopJoinExecTransformer. | -| spark.gluten.sql.cacheWholeStageTransformerContext | 🔄 Dynamic | false | When true, `WholeStageTransformer` will cache the `WholeStageTransformerContext` when executing. It is used to get substrait plan node and native plan string. | -| spark.gluten.sql.cartesianProductTransformerEnabled | 🔄 Dynamic | true | Config to enable CartesianProductExecTransformer. | -| spark.gluten.sql.collapseGetJsonObject.enabled | 🔄 Dynamic | false | Collapse nested get_json_object functions as one for optimization. | -| spark.gluten.sql.columnar.appendData | 🔄 Dynamic | true | Enable or disable columnar v2 command append data. | -| spark.gluten.sql.columnar.arrowUdf | 🔄 Dynamic | true | Enable or disable columnar arrow udf. | -| spark.gluten.sql.columnar.batchscan | 🔄 Dynamic | true | Enable or disable columnar batchscan. | -| spark.gluten.sql.columnar.broadcastExchange | 🔄 Dynamic | true | Enable or disable columnar broadcastExchange. | -| spark.gluten.sql.columnar.broadcastJoin | 🔄 Dynamic | true | Enable or disable columnar broadcastJoin. | +| spark.gluten.soft-affinity.duplicateReading.maxCacheItems | 🔄 Dynamic | 10000 | Enable Soft Affinity duplicate reading detection | +| spark.gluten.soft-affinity.duplicateReadingDetect.enabled | 🔄 Dynamic | false | If true, Enable Soft Affinity duplicate reading detection | +| spark.gluten.soft-affinity.enabled | 🔄 Dynamic | false | Whether to enable Soft Affinity scheduling. | +| spark.gluten.soft-affinity.min.target-hosts | 🔄 Dynamic | 1 | For on HDFS, if there are already target hosts, and then prefer to use the original target hosts to schedule | +| spark.gluten.soft-affinity.replications.num | 🔄 Dynamic | 2 | Calculate the number of the replications for scheduling to the target executors per file | +| spark.gluten.sql.adaptive.costEvaluator.enabled | ⚓ Static | true | If true, use org.apache.spark.sql.execution.adaptive.GlutenCostEvaluator as custom cost evaluator class, else follow the configuration spark.sql.adaptive.customCostEvaluatorClass. | +| spark.gluten.sql.ansiFallback.enabled | 🔄 Dynamic | true | When true (default), Gluten will fall back to Spark when ANSI mode is enabled. When false, Gluten will attempt to execute in ANSI mode. | +| spark.gluten.sql.broadcastNestedLoopJoinTransformerEnabled | 🔄 Dynamic | true | Config to enable BroadcastNestedLoopJoinExecTransformer. | +| spark.gluten.sql.cacheWholeStageTransformerContext | 🔄 Dynamic | false | When true, `WholeStageTransformer` will cache the `WholeStageTransformerContext` when executing. It is used to get substrait plan node and native plan string. | +| spark.gluten.sql.cartesianProductTransformerEnabled | 🔄 Dynamic | true | Config to enable CartesianProductExecTransformer. | +| spark.gluten.sql.collapseGetJsonObject.enabled | 🔄 Dynamic | false | Collapse nested get_json_object functions as one for optimization. | +| spark.gluten.sql.columnar.appendData | 🔄 Dynamic | true | Enable or disable columnar v2 command append data. | +| spark.gluten.sql.columnar.arrowUdf | 🔄 Dynamic | true | Enable or disable columnar arrow udf. | +| spark.gluten.sql.columnar.batchscan | 🔄 Dynamic | true | Enable or disable columnar batchscan. | +| spark.gluten.sql.columnar.broadcastExchange | 🔄 Dynamic | true | Enable or disable columnar broadcastExchange. | +| spark.gluten.sql.columnar.broadcastJoin | 🔄 Dynamic | true | Enable or disable columnar broadcastJoin. | | spark.gluten.sql.columnar.cast.avg | 🔄 Dynamic | true | -| spark.gluten.sql.columnar.coalesce | 🔄 Dynamic | true | Enable or disable columnar coalesce. | -| spark.gluten.sql.columnar.collectLimit | 🔄 Dynamic | true | Enable or disable columnar collectLimit. | -| spark.gluten.sql.columnar.collectTail | 🔄 Dynamic | true | Enable or disable columnar collectTail. | -| spark.gluten.sql.columnar.enableNestedColumnPruningInHiveTableScan | 🔄 Dynamic | true | Enable or disable nested column pruning in hivetablescan. | -| spark.gluten.sql.columnar.enableVanillaVectorizedReaders | ⚓ Static | true | Enable or disable vanilla vectorized scan. | -| spark.gluten.sql.columnar.executor.libpath | 🔄 Dynamic || The gluten executor library path. | -| spark.gluten.sql.columnar.expand | 🔄 Dynamic | true | Enable or disable columnar expand. | -| spark.gluten.sql.columnar.fallback.expressions.threshold | 🔄 Dynamic | 50 | Fall back filter/project if number of nested expressions reaches this threshold, considering Spark codegen can bring better performance for such case. | -| spark.gluten.sql.columnar.fallback.ignoreRowToColumnar | 🔄 Dynamic | true | When true, the fallback policy ignores the RowToColumnar when counting fallback number. | -| spark.gluten.sql.columnar.fallback.preferColumnar | 🔄 Dynamic | true | When true, the fallback policy prefers to use Gluten plan rather than vanilla Spark plan if the both of them contains ColumnarToRow and the vanilla Spark plan ColumnarToRow number is not smaller than Gluten plan. | -| spark.gluten.sql.columnar.filescan | 🔄 Dynamic | true | Enable or disable columnar filescan. | -| spark.gluten.sql.columnar.filter | 🔄 Dynamic | true | Enable or disable columnar filter. | -| spark.gluten.sql.columnar.force.hashagg | 🔄 Dynamic | true | Whether to force to use gluten's hash agg for replacing vanilla spark's sort agg. | +| spark.gluten.sql.columnar.coalesce | 🔄 Dynamic | true | Enable or disable columnar coalesce. | +| spark.gluten.sql.columnar.collectLimit | 🔄 Dynamic | true | Enable or disable columnar collectLimit. | +| spark.gluten.sql.columnar.collectTail | 🔄 Dynamic | true | Enable or disable columnar collectTail. | +| spark.gluten.sql.columnar.enableNestedColumnPruningInHiveTableScan | 🔄 Dynamic | true | Enable or disable nested column pruning in hivetablescan. | +| spark.gluten.sql.columnar.enableVanillaVectorizedReaders | ⚓ Static | true | Enable or disable vanilla vectorized scan. | +| spark.gluten.sql.columnar.executor.libpath | 🔄 Dynamic || The gluten executor library path. | +| spark.gluten.sql.columnar.expand | 🔄 Dynamic | true | Enable or disable columnar expand. | +| spark.gluten.sql.columnar.fallback.expressions.threshold | 🔄 Dynamic | 50 | Fall back filter/project if number of nested expressions reaches this threshold, considering Spark codegen can bring better performance for such case. | +| spark.gluten.sql.columnar.fallback.ignoreRowToColumnar | 🔄 Dynamic | true | When true, the fallback policy ignores the RowToColumnar when counting fallback number. | +| spark.gluten.sql.columnar.fallback.preferColumnar | 🔄 Dynamic | true | When true, the fallback policy prefers to use Gluten plan rather than vanilla Spark plan if the both of them contains ColumnarToRow and the vanilla Spark plan ColumnarToRow number is not smaller than Gluten plan. | +| spark.gluten.sql.columnar.filescan | 🔄 Dynamic | true | Enable or disable columnar filescan. | +| spark.gluten.sql.columnar.filter | 🔄 Dynamic | true | Enable or disable columnar filter. | +| spark.gluten.sql.columnar.force.hashagg | 🔄 Dynamic | true | Whether to force to use gluten's hash agg for replacing vanilla spark's sort agg. | | spark.gluten.sql.columnar.forceShuffledHashJoin | 🔄 Dynamic | true | | spark.gluten.sql.columnar.generate | 🔄 Dynamic | true | -| spark.gluten.sql.columnar.hashagg | 🔄 Dynamic | true | Enable or disable columnar hashagg. | -| spark.gluten.sql.columnar.hivetablescan | 🔄 Dynamic | true | Enable or disable columnar hivetablescan. | -| spark.gluten.sql.columnar.libname | 🔄 Dynamic | gluten | The gluten library name. | -| spark.gluten.sql.columnar.libpath | 🔄 Dynamic || The gluten library path. | +| spark.gluten.sql.columnar.hashagg | 🔄 Dynamic | true | Enable or disable columnar hashagg. | +| spark.gluten.sql.columnar.hivetablescan | 🔄 Dynamic | true | Enable or disable columnar hivetablescan. | +| spark.gluten.sql.columnar.libname | 🔄 Dynamic | gluten | The gluten library name. | +| spark.gluten.sql.columnar.libpath | 🔄 Dynamic || The gluten library path. | | spark.gluten.sql.columnar.limit | 🔄 Dynamic | true | | spark.gluten.sql.columnar.maxBatchSize | 🔄 Dynamic | 4096 | -| spark.gluten.sql.columnar.overwriteByExpression | 🔄 Dynamic | true | Enable or disable columnar v2 command overwrite by expression. | -| spark.gluten.sql.columnar.overwritePartitionsDynamic | 🔄 Dynamic | true | Enable or disable columnar v2 command overwrite partitions dynamic. | +| spark.gluten.sql.columnar.overwriteByExpression | 🔄 Dynamic | true | Enable or disable columnar v2 command overwrite by expression. | +| spark.gluten.sql.columnar.overwritePartitionsDynamic | 🔄 Dynamic | true | Enable or disable columnar v2 command overwrite partitions dynamic. | | spark.gluten.sql.columnar.parquet.write.blockSize | 🔄 Dynamic | 128MB | -| spark.gluten.sql.columnar.partial.generate | 🔄 Dynamic | true | Evaluates the non-offload-able HiveUDTF using vanilla Spark generator | -| spark.gluten.sql.columnar.partial.project | 🔄 Dynamic | true | Break up one project node into 2 phases when some of the expressions are non offload-able. Phase one is a regular offloaded project transformer that evaluates the offload-able expressions in native, phase two preserves the output from phase one and evaluates the remaining non-offload-able expressions using vanilla Spark projections | -| spark.gluten.sql.columnar.physicalJoinOptimizationLevel | 🔄 Dynamic | 12 | Fallback to row operators if there are several continuous joins. | -| spark.gluten.sql.columnar.physicalJoinOptimizationOutputSize | 🔄 Dynamic | 52 | Fallback to row operators if there are several continuous joins and matched output size. | -| spark.gluten.sql.columnar.physicalJoinOptimizeEnable | 🔄 Dynamic | false | Enable or disable columnar physicalJoinOptimize. | -| spark.gluten.sql.columnar.preferStreamingAggregate | 🔄 Dynamic | true | Velox backend supports `StreamingAggregate`. `StreamingAggregate` uses the less memory as it does not need to hold all groups in memory, so it could avoid spill. When true and the child output ordering satisfies the grouping key then Gluten will choose `StreamingAggregate` as the native operator. | -| spark.gluten.sql.columnar.project | 🔄 Dynamic | true | Enable or disable columnar project. | -| spark.gluten.sql.columnar.project.collapse | 🔄 Dynamic | true | Combines two columnar project operators into one and perform alias substitution | -| spark.gluten.sql.columnar.query.fallback.threshold | 🔄 Dynamic | -1 | The threshold for whether query will fall back by counting the number of ColumnarToRow & vanilla leaf node. | -| spark.gluten.sql.columnar.range | 🔄 Dynamic | true | Enable or disable columnar range. | -| spark.gluten.sql.columnar.replaceData | 🔄 Dynamic | true | Enable or disable columnar v2 command replace data. | -| spark.gluten.sql.columnar.scanOnly | 🔄 Dynamic | false | When enabled, only scan and the filter after scan will be offloaded to native. | -| spark.gluten.sql.columnar.shuffle | 🔄 Dynamic | true | Enable or disable columnar shuffle. | -| spark.gluten.sql.columnar.shuffle.celeborn.fallback.enabled | ⚓ Static | true | If enabled, fall back to ColumnarShuffleManager when celeborn service is unavailable.Otherwise, throw an exception. | -| spark.gluten.sql.columnar.shuffle.celeborn.useRssSort | 🔄 Dynamic | true | If true, use RSS sort implementation for Celeborn sort-based shuffle.If false, use Gluten's row-based sort implementation. Only valid when `spark.celeborn.client.spark.shuffle.writer` is set to `sort`. | -| spark.gluten.sql.columnar.shuffle.codec | 🔄 Dynamic | <undefined> | By default, the supported codecs are lz4 and zstd. When spark.gluten.sql.columnar.shuffle.codecBackend=qat,the supported codecs are gzip and zstd. | +| spark.gluten.sql.columnar.partial.generate | 🔄 Dynamic | true | Evaluates the non-offload-able HiveUDTF using vanilla Spark generator | +| spark.gluten.sql.columnar.partial.project | 🔄 Dynamic | true | Break up one project node into 2 phases when some of the expressions are non offload-able. Phase one is a regular offloaded project transformer that evaluates the offload-able expressions in native, phase two preserves the output from phase one and evaluates the remaining non-offload-able expressions using vanilla Spark projections | +| spark.gluten.sql.columnar.physicalJoinOptimizationLevel | 🔄 Dynamic | 12 | Fallback to row operators if there are several continuous joins. | +| spark.gluten.sql.columnar.physicalJoinOptimizationOutputSize | 🔄 Dynamic | 52 | Fallback to row operators if there are several continuous joins and matched output size. | +| spark.gluten.sql.columnar.physicalJoinOptimizeEnable | 🔄 Dynamic | false | Enable or disable columnar physicalJoinOptimize. | +| spark.gluten.sql.columnar.preferStreamingAggregate | 🔄 Dynamic | true | Velox backend supports `StreamingAggregate`. `StreamingAggregate` uses the less memory as it does not need to hold all groups in memory, so it could avoid spill. When true and the child output ordering satisfies the grouping key then Gluten will choose `StreamingAggregate` as the native operator. | +| spark.gluten.sql.columnar.project | 🔄 Dynamic | true | Enable or disable columnar project. | +| spark.gluten.sql.columnar.project.collapse | 🔄 Dynamic | true | Combines two columnar project operators into one and perform alias substitution | +| spark.gluten.sql.columnar.query.fallback.threshold | 🔄 Dynamic | -1 | The threshold for whether query will fall back by counting the number of ColumnarToRow & vanilla leaf node. | +| spark.gluten.sql.columnar.range | 🔄 Dynamic | true | Enable or disable columnar range. | +| spark.gluten.sql.columnar.replaceData | 🔄 Dynamic | true | Enable or disable columnar v2 command replace data. | +| spark.gluten.sql.columnar.scanOnly | 🔄 Dynamic | false | When enabled, only scan and the filter after scan will be offloaded to native. | +| spark.gluten.sql.columnar.shuffle | 🔄 Dynamic | true | Enable or disable columnar shuffle. | +| spark.gluten.sql.columnar.shuffle.celeborn.fallback.enabled | ⚓ Static | true | If enabled, fall back to ColumnarShuffleManager when celeborn service is unavailable.Otherwise, throw an exception. | +| spark.gluten.sql.columnar.shuffle.celeborn.useRssSort | 🔄 Dynamic | true | If true, use RSS sort implementation for Celeborn sort-based shuffle.If false, use Gluten's row-based sort implementation. Only valid when `spark.celeborn.client.spark.shuffle.writer` is set to `sort`. | +| spark.gluten.sql.columnar.shuffle.codec | 🔄 Dynamic | <undefined> | By default, the supported codecs are lz4 and zstd. When spark.gluten.sql.columnar.shuffle.codecBackend=qat,the supported codecs are gzip and zstd. | | spark.gluten.sql.columnar.shuffle.codecBackend | 🔄 Dynamic | <undefined> | -| spark.gluten.sql.columnar.shuffle.compression.threshold | 🔄 Dynamic | 100 | If number of rows in a batch falls below this threshold, will copy all buffers into one buffer to compress. | -| spark.gluten.sql.columnar.shuffle.dictionary.enabled | 🔄 Dynamic | false | Enable dictionary in hash-based shuffle. | +| spark.gluten.sql.columnar.shuffle.compression.threshold | 🔄 Dynamic | 100 | If number of rows in a batch falls below this threshold, will copy all buffers into one buffer to compress. | +| spark.gluten.sql.columnar.shuffle.dictionary.enabled | 🔄 Dynamic | false | Enable dictionary in hash-based shuffle. | | spark.gluten.sql.columnar.shuffle.merge.threshold | 🔄 Dynamic | 0.25 | -| spark.gluten.sql.columnar.shuffle.partitionBufferEvictThreshold | 🔄 Dynamic | -1 | For Velox hash shuffle writer, evict partition buffers larger than this threshold after splitting an input batch. Use non-positive value to disable this feature. | -| spark.gluten.sql.columnar.shuffle.readerBufferSize | 🔄 Dynamic | 1MB | Buffer size in bytes for shuffle reader reading input stream from local or remote. | +| spark.gluten.sql.columnar.shuffle.partitionBufferEvictThreshold | 🔄 Dynamic | -1 | For Velox hash shuffle writer, evict partition buffers larger than this threshold after splitting an input batch. Use non-positive value to disable this feature. | +| spark.gluten.sql.columnar.shuffle.readerBufferSize | 🔄 Dynamic | 1MB | Buffer size in bytes for shuffle reader reading input stream from local or remote. | | spark.gluten.sql.columnar.shuffle.realloc.threshold | 🔄 Dynamic | 0.25 | -| spark.gluten.sql.columnar.shuffle.sort.columns.threshold | 🔄 Dynamic | 100000 | The threshold to determine whether to use sort-based columnar shuffle. Sort-based shuffle will be used if the number of columns is greater than this threshold. | -| spark.gluten.sql.columnar.shuffle.sort.deserializerBufferSize | 🔄 Dynamic | 1MB | Buffer size in bytes for sort-based shuffle reader deserializing raw input to columnar batch. | -| spark.gluten.sql.columnar.shuffle.sort.partitions.threshold | 🔄 Dynamic | 4000 | The threshold to determine whether to use sort-based columnar shuffle. Sort-based shuffle will be used if the number of partitions is greater than this threshold. | -| spark.gluten.sql.columnar.shuffle.typeAwareCompress.enabled | 🔄 Dynamic | false | Enable type-aware compression (e.g. FFor for 64-bit integers) in shuffle. Not compatible with dictionary encoding; if both are enabled, type-aware compression is automatically disabled. | -| spark.gluten.sql.columnar.shuffledHashJoin | 🔄 Dynamic | true | Enable or disable columnar shuffledHashJoin. | -| spark.gluten.sql.columnar.shuffledHashJoin.optimizeBuildSide | 🔄 Dynamic | true | Whether to allow Gluten to choose an optimal build side for shuffled hash join. | -| spark.gluten.sql.columnar.smallFileThreshold | 🔄 Dynamic | 0.5 | The total size threshold of small files in table scan.To avoid small files being placed into the same partition, Gluten will try to distribute small files into different partitions when the total size of small files is below this threshold. | -| spark.gluten.sql.columnar.sort | 🔄 Dynamic | true | Enable or disable columnar sort. | -| spark.gluten.sql.columnar.sortMergeJoin | 🔄 Dynamic | true | Enable or disable columnar sortMergeJoin. This should be set with preferSortMergeJoin=false. | -| spark.gluten.sql.columnar.tableCache | ⚓ Static | true | Enable or disable columnar table cache. | -| spark.gluten.sql.columnar.tableCache.partitionStats.enabled | 🔄 Dynamic | false | When true, the Velox columnar cache serializer computes per-partition min/max/null/row-count stats and embeds them in the cached payload so that the Spark optimizer can prune whole partitions on equality / range predicates. When false (default) the serializer writes the legacy raw payload with no stats, and partition pruning is disabled. Default is off until cross-workload benchmarks confirm zero regression on non-pruning queries. | +| spark.gluten.sql.columnar.shuffle.sort.columns.threshold | 🔄 Dynamic | 100000 | The threshold to determine whether to use sort-based columnar shuffle. Sort-based shuffle will be used if the number of columns is greater than this threshold. | +| spark.gluten.sql.columnar.shuffle.sort.deserializerBufferSize | 🔄 Dynamic | 1MB | Buffer size in bytes for sort-based shuffle reader deserializing raw input to columnar batch. | +| spark.gluten.sql.columnar.shuffle.sort.partitions.threshold | 🔄 Dynamic | 4000 | The threshold to determine whether to use sort-based columnar shuffle. Sort-based shuffle will be used if the number of partitions is greater than this threshold. | +| spark.gluten.sql.columnar.shuffle.typeAwareCompress.enabled | 🔄 Dynamic | false | Enable type-aware compression (e.g. FFor for 64-bit integers) in shuffle. Not compatible with dictionary encoding; if both are enabled, type-aware compression is automatically disabled. | +| spark.gluten.sql.columnar.shuffledHashJoin | 🔄 Dynamic | true | Enable or disable columnar shuffledHashJoin. | +| spark.gluten.sql.columnar.shuffledHashJoin.optimizeBuildSide | 🔄 Dynamic | true | Whether to allow Gluten to choose an optimal build side for shuffled hash join. | +| spark.gluten.sql.columnar.smallFileThreshold | 🔄 Dynamic | 0.5 | The total size threshold of small files in table scan.To avoid small files being placed into the same partition, Gluten will try to distribute small files into different partitions when the total size of small files is below this threshold. | +| spark.gluten.sql.columnar.sort | 🔄 Dynamic | true | Enable or disable columnar sort. | +| spark.gluten.sql.columnar.sortMergeJoin | 🔄 Dynamic | true | Enable or disable columnar sortMergeJoin. This should be set with preferSortMergeJoin=false. | +| spark.gluten.sql.columnar.tableCache | ⚓ Static | true | Enable or disable columnar table cache. | +| spark.gluten.sql.columnar.tableCache.partitionStats.enabled | 🔄 Dynamic | false | When true, the Velox columnar cache serializer computes per-partition min/max/null/row-count stats and embeds them in the cached payload so that the Spark optimizer can prune whole partitions on equality / range predicates. When false (default), the serializer still writes the V3 per-column payload with empty stats so projected cache reads can lazily materialize only requested columns, while partition pruning is disabled. | | spark.gluten.sql.columnar.takeOrderedAndProject | 🔄 Dynamic | true | -| spark.gluten.sql.columnar.union | 🔄 Dynamic | true | Enable or disable columnar union. | -| spark.gluten.sql.columnar.wholeStage.fallback.threshold | 🔄 Dynamic | -1 | The threshold for whether whole stage will fall back in AQE supported case by counting the number of ColumnarToRow & vanilla leaf node. | -| spark.gluten.sql.columnar.window | 🔄 Dynamic | true | Enable or disable columnar window. | -| spark.gluten.sql.columnar.window.group.limit | 🔄 Dynamic | true | Enable or disable columnar window group limit. | -| spark.gluten.sql.columnar.writeToDataSourceV2 | 🔄 Dynamic | true | Enable or disable columnar v2 command write to data source v2. | -| spark.gluten.sql.columnarSampleEnabled | 🔄 Dynamic | false | Disable or enable columnar sample. | +| spark.gluten.sql.columnar.union | 🔄 Dynamic | true | Enable or disable columnar union. | +| spark.gluten.sql.columnar.wholeStage.fallback.threshold | 🔄 Dynamic | -1 | The threshold for whether whole stage will fall back in AQE supported case by counting the number of ColumnarToRow & vanilla leaf node. | +| spark.gluten.sql.columnar.window | 🔄 Dynamic | true | Enable or disable columnar window. | +| spark.gluten.sql.columnar.window.group.limit | 🔄 Dynamic | true | Enable or disable columnar window group limit. | +| spark.gluten.sql.columnar.writeToDataSourceV2 | 🔄 Dynamic | true | Enable or disable columnar v2 command write to data source v2. | +| spark.gluten.sql.columnarSampleEnabled | 🔄 Dynamic | false | Disable or enable columnar sample. | | spark.gluten.sql.columnarToRowMemoryThreshold | 🔄 Dynamic | 64MB | -| spark.gluten.sql.countDistinctWithoutExpand | 🔄 Dynamic | false | Convert Count Distinct to a UDAF called count_distinct to prevent SparkPlanner converting it to Expand+Count. WARNING: When enabled, count distinct queries will fail to fallback!!! | -| spark.gluten.sql.extendedColumnPruning.enabled | 🔄 Dynamic | true | Do extended nested column pruning for cases ignored by vanilla Spark. | -| spark.gluten.sql.fallbackRegexpExpressions | 🔄 Dynamic | false | If true, fall back all regexp expressions. There are a few incompatible cases between RE2 (used by native engine) and java.util.regex (used by Spark). User should enable this property if their incompatibility is intolerable. | -| spark.gluten.sql.fallbackUnexpectedMetadataParquet | 🔄 Dynamic | false | If enabled, Gluten will not offload scan when unexpected metadata is detected. | -| spark.gluten.sql.fallbackUnexpectedMetadataParquet.limit | 🔄 Dynamic | 10 | If supplied, metadata of `limit` number of Parquet files will be checked to determine whether to fall back to java scan. | -| spark.gluten.sql.fallbackUnexpectedMetadataParquet.samplePercentage | 🔄 Dynamic | 0.1 | The percentage of root paths to sample for metadata validation when the number of root paths is large. Value range is (0, 1.0]. 1.0 means check all paths (no sampling). A smaller value reduces validation cost for tables with many partitions. | -| spark.gluten.sql.injectNativePlanStringToExplain | 🔄 Dynamic | false | When true, Gluten will inject native plan tree to Spark's explain output. | -| spark.gluten.sql.mergeTwoPhasesAggregate.enabled | 🔄 Dynamic | true | Whether to merge two phases aggregate if there are no other operators between them. | +| spark.gluten.sql.countDistinctWithoutExpand | 🔄 Dynamic | false | Convert Count Distinct to a UDAF called count_distinct to prevent SparkPlanner converting it to Expand+Count. WARNING: When enabled, count distinct queries will fail to fallback!!! | +| spark.gluten.sql.extendedColumnPruning.enabled | 🔄 Dynamic | true | Do extended nested column pruning for cases ignored by vanilla Spark. | +| spark.gluten.sql.fallbackRegexpExpressions | 🔄 Dynamic | false | If true, fall back all regexp expressions. There are a few incompatible cases between RE2 (used by native engine) and java.util.regex (used by Spark). User should enable this property if their incompatibility is intolerable. | +| spark.gluten.sql.fallbackUnexpectedMetadataParquet | 🔄 Dynamic | false | If enabled, Gluten will not offload scan when unexpected metadata is detected. | +| spark.gluten.sql.fallbackUnexpectedMetadataParquet.limit | 🔄 Dynamic | 10 | If supplied, metadata of `limit` number of Parquet files will be checked to determine whether to fall back to java scan. | +| spark.gluten.sql.fallbackUnexpectedMetadataParquet.samplePercentage | 🔄 Dynamic | 0.1 | The percentage of root paths to sample for metadata validation when the number of root paths is large. Value range is (0, 1.0]. 1.0 means check all paths (no sampling). A smaller value reduces validation cost for tables with many partitions. | +| spark.gluten.sql.injectNativePlanStringToExplain | 🔄 Dynamic | false | When true, Gluten will inject native plan tree to Spark's explain output. | +| spark.gluten.sql.mergeTwoPhasesAggregate.enabled | 🔄 Dynamic | true | Whether to merge two phases aggregate if there are no other operators between them. | | spark.gluten.sql.native.bloomFilter | 🔄 Dynamic | true | -| spark.gluten.sql.native.hive.writer.enabled | 🔄 Dynamic | true | This is config to specify whether to enable the native columnar writer for HiveFileFormat. Currently only supports HiveFileFormat with Parquet as the output file type. | +| spark.gluten.sql.native.hive.writer.enabled | 🔄 Dynamic | true | This is config to specify whether to enable the native columnar writer for HiveFileFormat. Currently only supports HiveFileFormat with Parquet as the output file type. | | spark.gluten.sql.native.hyperLogLog.Aggregate | 🔄 Dynamic | true | | spark.gluten.sql.native.parquet.write.blockRows | 🔄 Dynamic | 100000000 | -| spark.gluten.sql.native.union | 🔄 Dynamic | false | Enable or disable native union where computation is completely offloaded to backend. | -| spark.gluten.sql.native.writeColumnMetadataExclusionList | 🔄 Dynamic | comment | Native write files does not support column metadata. Metadata in list would be removed to support native write files. Multiple values separated by commas. | -| spark.gluten.sql.native.writer.enabled | 🔄 Dynamic | <undefined> | This is config to specify whether to enable the native columnar parquet/orc writer | -| spark.gluten.sql.orc.charType.scan.fallback.enabled | 🔄 Dynamic | true | Force fallback for orc char type scan. | -| spark.gluten.sql.pushAggregateThroughJoin.enabled | 🔄 Dynamic | false | Enables the push-aggregate-through-join optimization in Gluten. When enabled, aggregate operators may be pushed below joins during logical optimization and corresponding physical plans may be rewritten to execute the aggregation earlier. | -| spark.gluten.sql.pushAggregateThroughJoin.maxDepth | 🔄 Dynamic | 2147483647 | Maximum join traversal depth when applying the push-aggregate-through-join optimization. A value of 1 allows pushing an aggregate through a single join; larger values allow the rule to traverse and push through multiple consecutive joins. | -| spark.gluten.sql.removeNativeWriteFilesSortAndProject | 🔄 Dynamic | true | When true, Gluten will remove the vanilla Spark V1Writes added sort and project for velox backend. | -| spark.gluten.sql.rewrite.dateTimestampComparison | 🔄 Dynamic | true | Rewrite the comparision between date and timestamp to timestamp comparison.For example `from_unixtime(ts) > date` will be rewritten to `ts > to_unixtime(date)` | -| spark.gluten.sql.scan.fileSchemeValidation.enabled | 🔄 Dynamic | true | When true, enable file path scheme validation for scan. Validation will fail if file scheme is not supported by registered file systems, which will cause scan operator fall back. | -| spark.gluten.sql.supported.flattenNestedFunctions | 🔄 Dynamic | and,or | Flatten nested functions as one for optimization. | -| spark.gluten.sql.text.input.empty.as.default | 🔄 Dynamic | false | treat empty fields in CSV input as default values. | -| spark.gluten.sql.text.input.max.block.size | 🔄 Dynamic | 8KB | the max block size for text input rows | +| spark.gluten.sql.native.union | 🔄 Dynamic | false | Enable or disable native union where computation is completely offloaded to backend. | +| spark.gluten.sql.native.writeColumnMetadataExclusionList | 🔄 Dynamic | comment | Native write files does not support column metadata. Metadata in list would be removed to support native write files. Multiple values separated by commas. | +| spark.gluten.sql.native.writer.enabled | 🔄 Dynamic | <undefined> | This is config to specify whether to enable the native columnar parquet/orc writer | +| spark.gluten.sql.orc.charType.scan.fallback.enabled | 🔄 Dynamic | true | Force fallback for orc char type scan. | +| spark.gluten.sql.pushAggregateThroughJoin.enabled | 🔄 Dynamic | false | Enables the push-aggregate-through-join optimization in Gluten. When enabled, aggregate operators may be pushed below joins during logical optimization and corresponding physical plans may be rewritten to execute the aggregation earlier. | +| spark.gluten.sql.pushAggregateThroughJoin.maxDepth | 🔄 Dynamic | 2147483647 | Maximum join traversal depth when applying the push-aggregate-through-join optimization. A value of 1 allows pushing an aggregate through a single join; larger values allow the rule to traverse and push through multiple consecutive joins. | +| spark.gluten.sql.removeNativeWriteFilesSortAndProject | 🔄 Dynamic | true | When true, Gluten will remove the vanilla Spark V1Writes added sort and project for velox backend. | +| spark.gluten.sql.rewrite.dateTimestampComparison | 🔄 Dynamic | true | Rewrite the comparision between date and timestamp to timestamp comparison.For example `from_unixtime(ts) > date` will be rewritten to `ts > to_unixtime(date)` | +| spark.gluten.sql.scan.fileSchemeValidation.enabled | 🔄 Dynamic | true | When true, enable file path scheme validation for scan. Validation will fail if file scheme is not supported by registered file systems, which will cause scan operator fall back. | +| spark.gluten.sql.supported.flattenNestedFunctions | 🔄 Dynamic | and,or | Flatten nested functions as one for optimization. | +| spark.gluten.sql.text.input.empty.as.default | 🔄 Dynamic | false | treat empty fields in CSV input as default values. | +| spark.gluten.sql.text.input.max.block.size | 🔄 Dynamic | 8KB | the max block size for text input rows | | spark.gluten.sql.validation.printStackOnFailure | 🔄 Dynamic | false | -| spark.gluten.storage.hdfsViewfs.enabled | ⚓ Static | false | If enabled, gluten will convert the viewfs path to hdfs path in scala side | -| spark.gluten.supported.hive.udfs | 🔄 Dynamic || Supported hive udf names. | -| spark.gluten.supported.python.udfs | 🔄 Dynamic || Supported python udf names. | -| spark.gluten.supported.scala.udfs | 🔄 Dynamic || Supported scala udf names. | -| spark.gluten.ui.enabled | ⚓ Static | true | Whether to enable the gluten web UI, If true, attach the gluten UI page to the Spark web UI. | +| spark.gluten.storage.hdfsViewfs.enabled | ⚓ Static | false | If enabled, gluten will convert the viewfs path to hdfs path in scala side | +| spark.gluten.supported.hive.udfs | 🔄 Dynamic || Supported hive udf names. | +| spark.gluten.supported.python.udfs | 🔄 Dynamic || Supported python udf names. | +| spark.gluten.supported.scala.udfs | 🔄 Dynamic || Supported scala udf names. | +| spark.gluten.ui.enabled | ⚓ Static | true | Whether to enable the gluten web UI, If true, attach the gluten UI page to the Spark web UI. | ## Gluten *experimental* configurations diff --git a/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ColumnarBatchSerializerJniWrapper.java b/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ColumnarBatchSerializerJniWrapper.java index 9a5247c823e..8d9bba1ab2c 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ColumnarBatchSerializerJniWrapper.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ColumnarBatchSerializerJniWrapper.java @@ -39,12 +39,17 @@ public long rtHandle() { public native JniUnsafeByteBuffer serialize(long handle); - // Framed [magic | statsLen | statsBlob | bytesLen | bytesBlob] payload produced by - // VeloxColumnarBatchSerializer::framedSerializeWithStats. Returns byte[] (not - // JniUnsafeByteBuffer) because the framed wire is small enough that the simpler return type - // avoids ByteBuffer lifetime concerns. + // Framed [magic=0x02 | statsLen | statsBlob | bytesLen | bytesBlob] payload (V2). public native byte[] serializeWithStats(long handle); + // V3 per-column framed payload [magic=0x03 | statsLen=0 | numRows | numCols | per-col]. + // Returns null when the backend does not support V3 (callers should fall back). + public native byte[] serializeV3(long handle); + + // V3 per-column framed payload [magic=0x03 | statsLen | statsBlob | numRows | numCols | per-col]. + // Returns null when the backend does not support V3 (callers should fall back to V2). + public native byte[] serializeWithStatsV3(long handle); + // Return the native ColumnarBatchSerializer handle public native long init(long cSchema); @@ -54,4 +59,9 @@ public long rtHandle() { public native long deserializeDirect(long serializerHandle, long offset, int len); public native void close(long serializerHandle); + + // V3 deserialize with column projection. Returns M-column native batch handle. + // requestedColumnIndices: null = all columns; int[0] = zero columns; int[m] = M specified cols. + public native long deserializeWithProjection( + long serializerHandle, byte[] data, int[] requestedColumnIndices); } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/config/GlutenConfig.scala b/gluten-substrait/src/main/scala/org/apache/gluten/config/GlutenConfig.scala index 821d7472287..5e3ecec4e10 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/config/GlutenConfig.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/config/GlutenConfig.scala @@ -1031,10 +1031,10 @@ object GlutenConfig extends ConfigRegistry { "When true, the Velox columnar cache serializer computes per-partition " + "min/max/null/row-count stats and embeds them in the cached payload so " + "that the Spark optimizer can prune whole partitions on equality / " + - "range predicates. When false (default) the serializer writes the " + - "legacy raw payload with no stats, and partition pruning is disabled. " + - "Default is off until cross-workload benchmarks confirm zero regression " + - "on non-pruning queries.") + "range predicates. When false (default), the serializer still writes " + + "the V3 per-column payload with empty stats so projected cache reads " + + "can lazily materialize only requested columns, while partition pruning " + + "is disabled.") .booleanConf .createWithDefault(false) diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala index f8b61bed484..887070b6d63 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala @@ -37,7 +37,7 @@ class GlutenCachedTableSuite sql("CACHE TABLE testData") spark.table("testData").queryExecution.withCachedData.collect { case cached: InMemoryRelation => - assert(cached.stats.sizeInBytes === 1132) + assert(cached.stats.sizeInBytes === 1130) } } } diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala index f8b61bed484..887070b6d63 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala @@ -37,7 +37,7 @@ class GlutenCachedTableSuite sql("CACHE TABLE testData") spark.table("testData").queryExecution.withCachedData.collect { case cached: InMemoryRelation => - assert(cached.stats.sizeInBytes === 1132) + assert(cached.stats.sizeInBytes === 1130) } } } diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala index 0afabae6e5f..54035b5ff93 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala @@ -40,7 +40,7 @@ class GlutenCachedTableSuite sql("CACHE TABLE testData") spark.table("testData").queryExecution.withCachedData.collect { case cached: InMemoryRelation => - assert(cached.stats.sizeInBytes === 1132) + assert(cached.stats.sizeInBytes === 1130) } } diff --git a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala index f4388126594..4cb296499f5 100644 --- a/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala +++ b/gluten-ut/spark40/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala @@ -49,7 +49,7 @@ class GlutenCachedTableSuite sql("CACHE TABLE testData") spark.table("testData").queryExecution.withCachedData.collect { case cached: InMemoryRelation => - assert(cached.stats.sizeInBytes === 1132) + assert(cached.stats.sizeInBytes === 1130) } } diff --git a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala index f4388126594..4cb296499f5 100644 --- a/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala +++ b/gluten-ut/spark41/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala @@ -49,7 +49,7 @@ class GlutenCachedTableSuite sql("CACHE TABLE testData") spark.table("testData").queryExecution.withCachedData.collect { case cached: InMemoryRelation => - assert(cached.stats.sizeInBytes === 1132) + assert(cached.stats.sizeInBytes === 1130) } }