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 85b2018726af..60e264b01676 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 @@ -26,27 +26,46 @@ import org.apache.gluten.runtime.Runtimes import org.apache.gluten.utils.ArrowAbiUtil import org.apache.gluten.vectorized.ColumnarBatchSerializerJniWrapper +import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Kryo.KRYO_SERIALIZER_MAX_BUFFER_SIZE import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, GenericInternalRow} +import org.apache.spark.sql.columnar.{CachedBatch, SimpleMetricsCachedBatch, SimpleMetricsCachedBatchSerializer} import org.apache.spark.sql.execution.columnar.DefaultCachedBatchSerializer import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.sql.utils.SparkArrowUtil import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.storage.StorageLevel +import org.apache.spark.unsafe.types.UTF8String import com.esotericsoftware.kryo.{Kryo, Serializer => KryoSerializer} import com.esotericsoftware.kryo.DefaultSerializer import com.esotericsoftware.kryo.io.{Input, Output} import org.apache.arrow.c.ArrowSchema +import java.io.ByteArrayOutputStream +import java.lang.{Double => JDouble, Float => JFloat} +import java.math.{BigDecimal => JBigDecimal, BigInteger} +import java.nio.{ByteBuffer, ByteOrder} +import java.nio.charset.StandardCharsets.UTF_8 +import java.util.Arrays + /** - * TODO: fix on Spark-4.1 - Documentation + * A Velox columnar cache batch carrying per-partition column statistics. + * + * `stats` follows the [[SimpleMetricsCachedBatch]] contract (SPARK-32274): per-column slots + * `(lowerBound, upperBound, nullCount, count, sizeInBytes)`. `null` means stats are unavailable + * (legacy V1 binary, or partition-stats SQLConf disabled); the serializer's `buildFilter` override + * directs such batches through unchanged to avoid NPE in vanilla + * `SimpleMetricsCachedBatchSerializer.buildFilter` on `partitionFilter.eval(null)`. + * + * `sizeInBytes` is the serialized blob length (Velox off-heap footprint), overriding the trait's + * default per-column sum so cache-eviction accounting matches actual memory. * - * If you encounter serialization issues, manually register this class: + * Manual Kryo registration (Spark 4.1 doc TODO): * {{{ * spark.kryo.classesToRegister=org.apache.spark.sql.execution.CachedColumnarBatch * }}} @@ -55,25 +74,62 @@ import org.apache.arrow.c.ArrowSchema case class CachedColumnarBatch( override val numRows: Int, override val sizeInBytes: Long, - bytes: Array[Byte]) - extends CachedBatch {} + bytes: Array[Byte], + override val stats: InternalRow, + // Schema is carried per-batch so Kryo read path can dispatch (de)serializeStats by source + // dataType. Nullable for V1 binary back-compat. + schema: StructType = null) + extends SimpleMetricsCachedBatch +/** + * Kryo serializer for [[CachedColumnarBatch]]. + * + * Wire layout: + * {{{ + * [numRows: Int] // single-arg writeInt = fixed 4-byte BE + * [sizeInBytes: Long] + * [bytes.length + 1: Int] // +1 distinguishes Kryo.NULL + * [bytes] + * [hasStats: Boolean] if true: [statsLen: Int] [statsBlob] + * [hasSchema: Boolean] if true: [schemaLen: Int] [schemaJsonBytes] + * }}} + */ class CachedColumnarBatchKryoSerializer extends KryoSerializer[CachedColumnarBatch] { + override def write(kryo: Kryo, output: Output, batch: CachedColumnarBatch): Unit = { + // Use the single-arg writeInt(int) overload: fixed 4-byte BE. The (int, boolean) overload + // silently forwards to writeVarInt (1-5 bytes) and would corrupt the read path. output.writeInt(batch.numRows) output.writeLong(batch.sizeInBytes) require( batch.bytes != null, "The object 'CachedColumnarBatch.bytes' is invalid or malformed to " + s"serialize using ${this.getClass.getName}") - output.writeInt(batch.bytes.length + 1) // +1 to distinguish Kryo.NULL + output.writeInt(batch.bytes.length + 1) // +1 distinguishes Kryo.NULL output.writeBytes(batch.bytes) + if (batch.stats == null) { + output.writeBoolean(false) + } else { + output.writeBoolean(true) + val statsBytes = CachedColumnarBatchKryoSerializer.serializeStats(batch.stats, batch.schema) + output.writeInt(statsBytes.length) + output.writeBytes(statsBytes) + } + if (batch.schema == null) { + output.writeBoolean(false) + } else { + output.writeBoolean(true) + val schemaBytes = batch.schema.json.getBytes(UTF_8) + output.writeInt(schemaBytes.length) + output.writeBytes(schemaBytes) + } } override def read( kryo: Kryo, input: Input, cls: Class[CachedColumnarBatch]): CachedColumnarBatch = { + val maxLen = CachedColumnarBatchKryoSerializer.maxKryoBufferBytes val numRows = input.readInt() val sizeInBytes = input.readLong() val length = input.readInt() @@ -81,43 +137,482 @@ class CachedColumnarBatchKryoSerializer extends KryoSerializer[CachedColumnarBat length != Kryo.NULL, "The object 'CachedColumnarBatch.bytes' is invalid or malformed to " + s"deserialize using ${this.getClass.getName}") - val bytes = new Array[Byte](length - 1) // -1 to restore + // length is the byte payload size + 1 (the +1 distinguishes Kryo.NULL on the write side). + // Bound to spark.kryoserializer.buffer.max so a corrupt or malicious stream cannot trigger + // a multi-GB allocation; Spark's own Kryo write path enforces this same ceiling on the + // producing side, so any stream beyond it is already invalid. + val payloadLen = length - 1 + require( + payloadLen >= 0 && payloadLen.toLong <= maxLen, + s"CachedColumnarBatch.bytes length ($payloadLen) out of bounds [0, $maxLen]; " + + "stream is corrupt or exceeds spark.kryoserializer.buffer.max" + ) + val bytes = new Array[Byte](payloadLen) input.readBytes(bytes) - CachedColumnarBatch(numRows, sizeInBytes, bytes) + // Backward-compat with the V1 wire format (no trailing hasStats / hasSchema booleans): + // legacy CachedColumnarBatch instances persisted on disk (DISK_ONLY / MEMORY_AND_DISK) + // surviving a rolling upgrade lack these fields. available() is best-effort -- treats + // unavailable suffix as "absent" instead of throwing KryoException. + val hasStats = input.available() > 0 && input.readBoolean() + // Even when hasStats=false we still consume the hasSchema tag to keep the stream aligned. + // NB: avoid `val (a: T, b: U) = ...` -- Scala 2.13 erases Tuple2 generics and the typed + // pattern match throws MatchError at runtime. + val statsAndSchema: (InternalRow, StructType) = if (hasStats) { + val statsLen = input.readInt() + require( + statsLen >= 0 && statsLen.toLong <= maxLen, + s"CachedColumnarBatch stats length ($statsLen) out of bounds [0, $maxLen]; " + + "stream is corrupt or exceeds spark.kryoserializer.buffer.max" + ) + val statsBytes = new Array[Byte](statsLen) + input.readBytes(statsBytes) + val sch = readOptionalSchema(input, maxLen) + (CachedColumnarBatchKryoSerializer.deserializeStats(statsBytes, sch), sch) + } else { + (null, readOptionalSchema(input, maxLen)) + } + CachedColumnarBatch(numRows, sizeInBytes, bytes, statsAndSchema._1, statsAndSchema._2) + } + + private def readOptionalSchema(input: Input, maxLen: Long): StructType = { + // Treat absent trailing bytes as "no schema": V1 wire format predates this field. + if (input.available() <= 0 || !input.readBoolean()) { + null + } else { + val schemaLen = input.readInt() + require( + schemaLen >= 0 && schemaLen.toLong <= maxLen, + s"CachedColumnarBatch schema length ($schemaLen) out of bounds [0, $maxLen]; " + + "stream is corrupt or exceeds spark.kryoserializer.buffer.max" + ) + val schemaBytes = new Array[Byte](schemaLen) + input.readBytes(schemaBytes) + DataType.fromJson(new String(schemaBytes, UTF_8)).asInstanceOf[StructType] + } + } +} + +object CachedColumnarBatchKryoSerializer { + // Defensive upper bound on any single length-prefixed field in the Kryo wire (payload bytes, + // statsBlob, schema JSON). Tied to spark.kryoserializer.buffer.max because Kryo write itself + // refuses to emit any single object larger than that ceiling, so any stream claiming a larger + // field is necessarily corrupt or malicious. Falls back to the conf default (64 MiB) when no + // SparkEnv is active (e.g. unit tests without a SparkContext). + def maxKryoBufferBytes: Long = { + val env = SparkEnv.get + if (env == null) { + KRYO_SERIALIZER_MAX_BUFFER_SIZE.defaultValue.get * 1024L * 1024L + } else { + env.conf.get(KRYO_SERIALIZER_MAX_BUFFER_SIZE) * 1024L * 1024L + } + } + + // Sanity-check magic for the cpp/JVM ABI of the framed JNI return (serializeWithStats). Not a + // version tag: a corrupt or truncated cpp emit fails fast here instead of feeding garbage into + // length-prefix readers downstream. + val STATS_FRAMED_MAGIC: Array[Byte] = + Array[Byte](0xfe.toByte, 0xca.toByte, 0x53.toByte, 0x02.toByte) + + // Per-column statsBlob layout (LE throughout, matches the cpp emitter in + // VeloxColumnarBatchSerializer.cc): + // + // [ numCols: u32 ] + // per col: + // [ supported: u8 ] + // [ nullCount: u32 ] + // [ count: u32 ] + // [ sizeInBytes: u64 ] + // if supported: + // [ lowerBoundLen: u32 ] [ lowerBound bytes ] + // [ upperBoundLen: u32 ] [ upperBound bytes ] + // + // The vanilla SimpleMetricsCachedBatch.stats InternalRow has 5 slots per source column in + // order (lowerBound, upperBound, nullCount, count, sizeInBytes). + // + // Source dataTypes outside this allowlist are demoted to supported=0 in serializeStats + // (the cpp side may still emit supported=1 for short-Decimal as Velox BIGINT; the JVM + // gate prevents UnsupportedOperationException in dispatch). + private[execution] def isDispatchable(dt: DataType): Boolean = + dt match { + case IntegerType | DateType | _: YearMonthIntervalType => true + case ShortType => true + case ByteType => true + case LongType | _: DayTimeIntervalType | TimestampType | TimestampNTZType => true + case d: DecimalType if d.precision <= 18 => true // short-decimal: Long unscaled + case d: DecimalType if d.precision <= 38 => true // long-decimal: 16B LE int128 + case FloatType => true // 4B IEEE 754; NaN guard in cpp scanMinMax + case DoubleType => true // 8B IEEE 754; NaN guard in cpp scanMinMax + case BooleanType => true + case _: StringType => true // truncated to 256B; see encodeStringBounds (any collation) + case _ => false + } + + // schema may be null for legacy callsites; in that case behave as BIGINT-only (V2 read path). + private[execution] def serializeStats( + stats: InternalRow, + schema: StructType): Array[Byte] = { + require( + stats.numFields % 5 == 0, + s"stats InternalRow numFields=${stats.numFields} must be a multiple of 5 " + + s"(vanilla PartitionStatistics schema = 5 slots per column)" + ) + val numCols = stats.numFields / 5 + val baos = new ByteArrayOutputStream() + writeU32LE(baos, numCols) + var col = 0 + while (col < numCols) { + val base = col * 5 + val hasLower = !stats.isNullAt(base) + val hasUpper = !stats.isNullAt(base + 1) + val dispatchable = (schema == null) || + CachedColumnarBatchKryoSerializer.isDispatchable(schema(col).dataType) + // For String, pre-compute the truncated payload so an all-0xFF carry overflow + // can demote `supported` *before* the supported byte is written. + val isStringCol = (schema != null) && hasLower && hasUpper && + schema(col).dataType.isInstanceOf[StringType] + val stringPayload: Option[(Array[Byte], Array[Byte])] = + if (isStringCol) { + val loB = stats.getUTF8String(base).getBytes + val hiB = stats.getUTF8String(base + 1).getBytes + encodeStringBounds(loB, hiB) + } else None + val supported = hasLower && hasUpper && dispatchable && + (!isStringCol || stringPayload.isDefined) + baos.write(if (supported) 1 else 0) + writeU32LE(baos, if (stats.isNullAt(base + 2)) 0 else stats.getInt(base + 2)) + writeU32LE(baos, if (stats.isNullAt(base + 3)) 0 else stats.getInt(base + 3)) + writeU64LE(baos, if (stats.isNullAt(base + 4)) 0L else stats.getLong(base + 4)) + if (supported) { + // schema==null => BIGINT-only legacy behavior. Otherwise dispatch by source dataType, + // matching vanilla ColumnBuilder's union for the integer / long families. + val dt: DataType = + if (schema == null) LongType else schema(col).dataType + dt match { + case IntegerType | DateType | _: YearMonthIntervalType => + writeU32LE(baos, 4) + writeU32LE(baos, stats.getInt(base)) + writeU32LE(baos, 4) + writeU32LE(baos, stats.getInt(base + 1)) + case ShortType => + writeU32LE(baos, 2) + writeU16LE(baos, stats.getShort(base) & 0xffff) + writeU32LE(baos, 2) + writeU16LE(baos, stats.getShort(base + 1) & 0xffff) + case ByteType => + writeU32LE(baos, 1) + baos.write(stats.getByte(base) & 0xff) + writeU32LE(baos, 1) + baos.write(stats.getByte(base + 1) & 0xff) + case LongType | TimestampType | TimestampNTZType | _: DayTimeIntervalType => + writeU32LE(baos, 8) + writeI64LE(baos, stats.getLong(base)) + writeU32LE(baos, 8) + writeI64LE(baos, stats.getLong(base + 1)) + case d: DecimalType if d.precision <= 18 => + // short-Decimal: Long unscaled (matches Velox short-decimal physical = BIGINT). + writeU32LE(baos, 8) + writeI64LE(baos, stats.getDecimal(base, d.precision, d.scale).toUnscaledLong) + writeU32LE(baos, 8) + writeI64LE(baos, stats.getDecimal(base + 1, d.precision, d.scale).toUnscaledLong) + case d: DecimalType if d.precision <= 38 => + // long-Decimal: 16B LE signed two's-complement (cpp HUGEINT int128 wire). + val loDec = stats.getDecimal(base, d.precision, d.scale) + val hiDec = stats.getDecimal(base + 1, d.precision, d.scale) + writeU32LE(baos, 16) + writeI128LE(baos, loDec.toJavaBigDecimal.unscaledValue) + writeU32LE(baos, 16) + writeI128LE(baos, hiDec.toJavaBigDecimal.unscaledValue) + case FloatType => + writeU32LE(baos, 4) + writeU32LE(baos, JFloat.floatToRawIntBits(stats.getFloat(base))) + writeU32LE(baos, 4) + writeU32LE(baos, JFloat.floatToRawIntBits(stats.getFloat(base + 1))) + case DoubleType => + writeU32LE(baos, 8) + writeU64LE(baos, JDouble.doubleToRawLongBits(stats.getDouble(base))) + writeU32LE(baos, 8) + writeU64LE(baos, JDouble.doubleToRawLongBits(stats.getDouble(base + 1))) + case BooleanType => + writeU32LE(baos, 1) + baos.write(if (stats.getBoolean(base)) 1 else 0) + writeU32LE(baos, 1) + baos.write(if (stats.getBoolean(base + 1)) 1 else 0) + case _: StringType => + // Pre-validated: encodeStringBounds returned Some, otherwise we'd have demoted. + val (lo, hi) = stringPayload.get + writeU32LE(baos, lo.length) + baos.write(lo) + writeU32LE(baos, hi.length) + baos.write(hi) + case other => + throw new UnsupportedOperationException( + s"serializeStats: dispatch for $other not implemented") + } + } + col += 1 + } + baos.toByteArray + } + + // schema may be null for legacy callsites; in that case behave as BIGINT-only (V2 read path). + private[execution] def deserializeStats( + blob: Array[Byte], + schema: StructType): InternalRow = { + val buf = ByteBuffer.wrap(blob).order(ByteOrder.LITTLE_ENDIAN) + val numCols = buf.getInt + require( + numCols >= 0 && numCols <= Int.MaxValue / 5, + s"corrupt statsBlob: numCols=$numCols out of valid range") + val row = new GenericInternalRow(numCols * 5) + var col = 0 + while (col < numCols) { + val base = col * 5 + val supported = buf.get() + val nullCount = buf.getInt + val count = buf.getInt + val sizeInBytes = buf.getLong + if (supported == 1) { + val dt: DataType = + if (schema == null) LongType else schema(col).dataType + val lowerLen = buf.getInt + require( + lowerLen >= 0 && lowerLen <= STRING_BOUND_TRUNCATE_LEN, + s"lowerLen=$lowerLen out of range [0, $STRING_BOUND_TRUNCATE_LEN] " + + s"(likely cpp/JVM wire mismatch)" + ) + dt match { + case IntegerType | DateType | _: YearMonthIntervalType => + require(lowerLen == 4, s"Integer-family expects 4-byte lowerBound, got $lowerLen") + row.update(base, buf.getInt) + val upperLen = buf.getInt + require(upperLen == 4, s"Integer-family expects 4-byte upperBound, got $upperLen") + row.update(base + 1, buf.getInt) + case ShortType => + require(lowerLen == 2, s"ShortType expects 2-byte lowerBound, got $lowerLen") + row.update(base, buf.getShort) + val upperLen = buf.getInt + require(upperLen == 2, s"ShortType expects 2-byte upperBound, got $upperLen") + row.update(base + 1, buf.getShort) + case ByteType => + require(lowerLen == 1, s"ByteType expects 1-byte lowerBound, got $lowerLen") + row.update(base, buf.get) + val upperLen = buf.getInt + require(upperLen == 1, s"ByteType expects 1-byte upperBound, got $upperLen") + row.update(base + 1, buf.get) + case LongType | TimestampType | TimestampNTZType | _: DayTimeIntervalType => + require(lowerLen == 8, s"Long-family expects 8-byte lowerBound, got $lowerLen") + row.update(base, buf.getLong) + val upperLen = buf.getInt + require(upperLen == 8, s"Long-family expects 8-byte upperBound, got $upperLen") + row.update(base + 1, buf.getLong) + case d: DecimalType if d.precision <= 18 => + // Wrap as Decimal (NOT raw Long), else SpecificInternalRow.getDecimal CCEs at codegen. + require(lowerLen == 8, s"short-Decimal expects 8-byte lowerBound, got $lowerLen") + row.update(base, Decimal(buf.getLong, d.precision, d.scale)) + val upperLen = buf.getInt + require(upperLen == 8, s"short-Decimal expects 8-byte upperBound, got $upperLen") + row.update(base + 1, Decimal(buf.getLong, d.precision, d.scale)) + case d: DecimalType if d.precision <= 38 => + require(lowerLen == 16, s"long-Decimal expects 16-byte lowerBound, got $lowerLen") + val loBytes = new Array[Byte](16) + buf.get(loBytes) + row.update( + base, + Decimal(new JBigDecimal(readI128LE(loBytes), d.scale), d.precision, d.scale)) + val upperLenL = buf.getInt + require(upperLenL == 16, s"long-Decimal expects 16-byte upperBound, got $upperLenL") + val hiBytes = new Array[Byte](16) + buf.get(hiBytes) + row.update( + base + 1, + Decimal(new JBigDecimal(readI128LE(hiBytes), d.scale), d.precision, d.scale)) + case FloatType => + require(lowerLen == 4, s"FloatType expects 4B lowerBound, got $lowerLen") + row.update(base, JFloat.intBitsToFloat(buf.getInt)) + val upperLenF = buf.getInt + require(upperLenF == 4, s"FloatType expects 4B upperBound, got $upperLenF") + row.update(base + 1, JFloat.intBitsToFloat(buf.getInt)) + case DoubleType => + require(lowerLen == 8, s"DoubleType expects 8B lowerBound, got $lowerLen") + row.update(base, JDouble.longBitsToDouble(buf.getLong)) + val upperLenD = buf.getInt + require(upperLenD == 8, s"DoubleType expects 8B upperBound, got $upperLenD") + row.update(base + 1, JDouble.longBitsToDouble(buf.getLong)) + case BooleanType => + require(lowerLen == 1, s"BooleanType expects 1B lowerBound, got $lowerLen") + row.update(base, buf.get != 0) + val upperLenB = buf.getInt + require(upperLenB == 1, s"BooleanType expects 1B upperBound, got $upperLenB") + row.update(base + 1, buf.get != 0) + case _: StringType => + require( + lowerLen >= 0 && lowerLen <= 256, + s"StringType expects lowerBound in [0, 256], got $lowerLen") + val loBytes = new Array[Byte](lowerLen) + buf.get(loBytes) + row.update(base, UTF8String.fromBytes(loBytes)) + val upperLenS = buf.getInt + require( + upperLenS >= 0 && upperLenS <= 256, + s"StringType expects upperBound in [0, 256], got $upperLenS") + val hiBytes = new Array[Byte](upperLenS) + buf.get(hiBytes) + row.update(base + 1, UTF8String.fromBytes(hiBytes)) + case _ => + // cpp may emit supported=1 for types not yet in JVM dispatch (e.g. short-Decimal as + // Velox BIGINT). Skip both payloads using their wire-declared lengths instead of + // crashing; the row keeps the slot null so the caller treats it as supported=false. + buf.get(new Array[Byte](lowerLen)) + val upperSkipLen = buf.getInt + require( + upperSkipLen >= 0 && upperSkipLen <= STRING_BOUND_TRUNCATE_LEN, + s"unknown-arm upperSkipLen=$upperSkipLen out of range " + + s"[0, $STRING_BOUND_TRUNCATE_LEN] (likely cpp/JVM wire mismatch)" + ) + buf.get(new Array[Byte](upperSkipLen)) + } + } + row.update(base + 2, nullCount) + row.update(base + 3, count) + row.update(base + 4, sizeInBytes) + col += 1 + } + row + } + + private def writeU16LE(out: ByteArrayOutputStream, v: Int): Unit = { + out.write(v & 0xff) + out.write((v >>> 8) & 0xff) + } + + private def writeU32LE(out: 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 writeU64LE(out: ByteArrayOutputStream, v: Long): Unit = { + var i = 0 + while (i < 8) { + out.write(((v >>> (8 * i)) & 0xffL).toInt) + i += 1 + } + } + + private def writeI64LE(out: ByteArrayOutputStream, v: Long): Unit = + writeU64LE(out, v) + + // 16B LE signed two's-complement representation of a BigInteger. BigInteger.toByteArray() + // returns big-endian signed minimal-width bytes; sign-extend to 16 then reverse to LE. + private def writeI128LE(out: ByteArrayOutputStream, v: BigInteger): Unit = { + val raw = v.toByteArray + require(raw.length <= 16, s"BigInteger does not fit int128 (${raw.length} bytes)") + val padded = new Array[Byte](16) + val signByte: Byte = if (v.signum < 0) 0xff.toByte else 0x00.toByte + var i = 0 + while (i < 16 - raw.length) { + padded(i) = signByte + i += 1 + } + System.arraycopy(raw, 0, padded, 16 - raw.length, raw.length) + var j = 0 + while (j < 8) { + val t = padded(j) + padded(j) = padded(15 - j) + padded(15 - j) = t + j += 1 + } + out.write(padded) + } + + private def readI128LE(le: Array[Byte]): BigInteger = { + require(le.length == 16, s"readI128LE expects 16 bytes, got ${le.length}") + val be = new Array[Byte](16) + var i = 0 + while (i < 16) { + be(i) = le(15 - i) + i += 1 + } + new BigInteger(be) // signed BE constructor + } + + // Encode (lo, hi) string bounds for the wire by truncating each to 256 bytes. + // - Lower: prefix is byte-wise lex <= original, monotonic. + // - Upper: needs +1 carry on the truncated tail to ensure encoded >= original. If the carry + // propagates past byte 0 (all 256 prefix bytes were 0xFF), we cannot form a safe widening + // upper bound; return None so the caller demotes supported. + private val STRING_BOUND_TRUNCATE_LEN = 256 + private def encodeStringBounds( + loBytes: Array[Byte], + hiBytes: Array[Byte]): Option[(Array[Byte], Array[Byte])] = { + val loLen = math.min(loBytes.length, STRING_BOUND_TRUNCATE_LEN) + val loEnc = Arrays.copyOf(loBytes, loLen) + if (hiBytes.length <= STRING_BOUND_TRUNCATE_LEN) { + Some((loEnc, Arrays.copyOf(hiBytes, hiBytes.length))) + } else { + val hiEnc = Arrays.copyOf(hiBytes, STRING_BOUND_TRUNCATE_LEN) + var i = STRING_BOUND_TRUNCATE_LEN - 1 + while (i >= 0) { + val b = (hiEnc(i) & 0xff) + 1 + if (b <= 0xff) { + hiEnc(i) = b.toByte + return Some((loEnc, hiEnc)) + } + hiEnc(i) = 0.toByte + i -= 1 + } + None // carry overflowed past byte 0 + } + } + + /** + * 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 ]`. + * + * Eager guards catch corrupt magic / truncated framing before they propagate. + */ + private[execution] def parseFramedBytes( + framed: Array[Byte], + schema: StructType): (InternalRow, Array[Byte]) = { + require( + framed != null && framed.length >= 4 + 4 + 4, + 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" + ) + 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}") + 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)") + val bytesBlob = new Array[Byte](bytesLen) + buf.get(bytesBlob) + (stats, bytesBlob) } } -// format: off /** - * Feature: - * 1. This serializer supports column pruning - * 2. TODO: support push down filter - * 3. Super TODO: support store offheap object directly - * - * The data transformation pipeline: - * - * - Serializer ColumnarBatch -> CachedColumnarBatch - * -> serialize to byte[] - * - * - Deserializer CachedColumnarBatch -> ColumnarBatch - * -> deserialize to byte[] to create Velox ColumnarBatch - * - * - Serializer InternalRow -> CachedColumnarBatch (support RowToColumnar) - * -> Convert InternalRow to ColumnarBatch - * -> Serializer ColumnarBatch -> CachedColumnarBatch - * - * - Serializer InternalRow -> DefaultCachedBatch (unsupport RowToColumnar) - * -> Convert InternalRow to DefaultCachedBatch using vanilla Spark serializer - * - * - Deserializer CachedColumnarBatch -> InternalRow (support ColumnarToRow) - * -> Deserializer CachedColumnarBatch -> ColumnarBatch - * -> Convert ColumnarBatch to InternalRow - * - * - Deserializer DefaultCachedBatch -> InternalRow (unsupport ColumnarToRow) - * -> Convert DefaultCachedBatch to InternalRow using vanilla Spark serializer + * Velox columnar cache serializer. Supports column pruning; converts row-based input via + * [[RowToVeloxColumnarExec]] and falls back to vanilla Spark serialization for unsupported schemas. */ -// format: on -class ColumnarCachedBatchSerializer extends CachedBatchSerializer with Logging { +class ColumnarCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { private lazy val rowBasedCachedBatchSerializer = new DefaultCachedBatchSerializer private def glutenConf: GlutenConfig = GlutenConfig.get @@ -208,19 +703,63 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with Logging { if heavy batch is encountered */ batch => VeloxColumnarBatches.ensureVeloxBatch(batch) } + // Hoist the per-partition StructType out of the per-batch hot path: schema is constant + // for the lifetime of this iterator, so allocating one StructType per CachedBatch wastes + // GC for the many-small-batch case. + val structSchema = StructType( + schema.map(a => StructField(a.name, a.dataType, a.nullable))) new Iterator[CachedBatch] { override def hasNext: Boolean = veloxBatches.hasNext override def next(): CachedBatch = { val batch = veloxBatches.next() - val unsafeBuffer = ColumnarBatchSerializerJniWrapper - .create( - Runtimes.contextInstance( - BackendsApiManager.getBackendName, - "ColumnarCachedBatchSerializer#serialize")) - .serialize(ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, batch)) - val bytes = unsafeBuffer.toByteArray - CachedColumnarBatch(batch.numRows(), bytes.length, bytes) + val jni = ColumnarBatchSerializerJniWrapper.create( + Runtimes.contextInstance( + BackendsApiManager.getBackendName, + "ColumnarCachedBatchSerializer#serialize")) + val handle = + ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, batch) + // 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. + val partitionStatsEnabled = + GlutenConfig.get.getConf(GlutenConfig.COLUMNAR_TABLE_CACHE_PARTITION_STATS_ENABLED) + if (partitionStatsEnabled && ColumnarCachedBatchSerializer.statsExtAvailable) { + try { + val framed = jni.serializeWithStats(handle) + val (stats, bytesBlob) = + CachedColumnarBatchKryoSerializer.parseFramedBytes(framed, structSchema) + CachedColumnarBatch( + batch.numRows(), + bytesBlob.length, + bytesBlob, + stats, + structSchema) + } catch { + case e: UnsatisfiedLinkError => + ColumnarCachedBatchSerializer.markStatsExtUnavailable(e) + val unsafeBuffer = jni.serialize(handle) + val bytes = unsafeBuffer.toByteArray + CachedColumnarBatch( + batch.numRows(), + bytes.length, + bytes, + stats = null, + schema = null) + } + } else { + val unsafeBuffer = jni.serialize(handle) + val bytes = unsafeBuffer.toByteArray + CachedColumnarBatch( + batch.numRows(), + bytes.length, + bytes, + stats = null, + schema = null) + } } } } @@ -296,10 +835,84 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with Logging { } } + // Lazy-split iterator wrapper. stats=null batches are passed through unchanged; stats!=null + // batches are routed to the inherited parent buildFilter for partition pruning. Without this + // split, vanilla SimpleMetricsCachedBatchSerializer.buildFilter NPEs on + // partitionFilter.eval(null) for non-trivial predicates -- the codegen and interpreted + // paths both have no fallback for null stats. override def buildFilter( predicates: Seq[Expression], - cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = { - // TODO, support build filter as we did not support collect min/max value for columnar batch - (_, it) => it + cachedAttributes: Seq[Attribute]) + : (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = { + val parent = super.buildFilter(predicates, cachedAttributes) + (index, cachedBatchIterator) => + new Iterator[CachedBatch] { + private val peekable = cachedBatchIterator.buffered + private var staged: Iterator[CachedBatch] = Iterator.empty + + // Drain peekable until staged has an element ready, or peekable is empty. Idempotent: + // 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. + 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. + val runIt = new Iterator[CachedBatch] { + override def hasNext: Boolean = + peekable.hasNext && statsOf(peekable.head) != null + override def next(): CachedBatch = peekable.next() + } + staged = parent(index, runIt) + } + } + } + + private def statsOf(batch: CachedBatch): InternalRow = batch match { + case ccb: CachedColumnarBatch => ccb.stats + case smcb: SimpleMetricsCachedBatch => smcb.stats + case _ => null + } + + override def hasNext: Boolean = { + advance() + staged.hasNext + } + override def next(): CachedBatch = { + advance() + staged.next() + } + } + } +} + +object ColumnarCachedBatchSerializer extends Logging { + // Lazy capability flag for the serializeWithStats JNI symbol. A new Gluten jar paired with an + // older libgluten.so will throw UnsatisfiedLinkError on the first invocation; the call site + // catches it once via markStatsExtUnavailable() and we degrade to the legacy serialize() path + // for the remainder of the JVM lifetime. Default true so the optimistic fast path is taken. + @volatile private var statsExtAvailableFlag: Boolean = true + + def statsExtAvailable: Boolean = statsExtAvailableFlag + + def markStatsExtUnavailable(cause: Throwable): Unit = { + if (statsExtAvailableFlag) { + statsExtAvailableFlag = false + logWarning( + "serializeWithStats JNI symbol is not linked in libgluten.so; " + + "falling back to serialize() and disabling per-partition stats for this JVM. " + + "This typically indicates a Gluten jar / native library version mismatch.", + cause + ) + } + } + + // 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 } } diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCacheShipBlockerMarshalSuite.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCacheShipBlockerMarshalSuite.scala new file mode 100644 index 000000000000..c25638d91c40 --- /dev/null +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCacheShipBlockerMarshalSuite.scala @@ -0,0 +1,131 @@ +/* + * 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.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.types.{Decimal, DecimalType, StringType, StructField, StructType} +import org.apache.spark.unsafe.types.UTF8String + +import org.scalatest.funsuite.AnyFunSuite + +import java.util.Arrays + +/** + * Ship-blocker acceptance tests for the non-BIGINT marshal paths (Decimal short/long, String, Float + * / Double NaN guard). Guards against silent corruption (wrong Decimal scale, wrong UTF-8 byte + * order, lost BigInteger sign) on round-trip. + */ +class ColumnarCacheShipBlockerMarshalSuite extends AnyFunSuite { + + test("Decimal(10, 2) round-trip preserves value") { + val lo = Decimal(BigDecimal("1.50"), 10, 2) + val hi = Decimal(BigDecimal("99.99"), 10, 2) + val stats: InternalRow = new GenericInternalRow( + Array[Any](lo, hi, 0, 100, 800L)) + val schema = StructType(Seq(StructField("d", DecimalType(10, 2)))) + val blob = CachedColumnarBatchKryoSerializer.serializeStats(stats, schema) + val read = CachedColumnarBatchKryoSerializer.deserializeStats(blob, schema) + val dt = DecimalType(10, 2) + val readLo = read.get(0, dt) + val readHi = read.get(1, dt) + assert(readLo == lo, s"lower bound corrupted: expected $lo got $readLo") + assert(readHi == hi, s"upper bound corrupted: expected $hi got $readHi") + } + + test("Decimal(30, 5) round-trip preserves big value") { + val big = BigDecimal("12345678901234567890.12345") + val lo = Decimal(big.bigDecimal.negate(), 30, 5) + val hi = Decimal(big, 30, 5) + val stats: InternalRow = new GenericInternalRow( + Array[Any](lo, hi, 0, 100, 1600L)) + val schema = StructType(Seq(StructField("d", DecimalType(30, 5)))) + val blob = CachedColumnarBatchKryoSerializer.serializeStats(stats, schema) + val read = CachedColumnarBatchKryoSerializer.deserializeStats(blob, schema) + val dt = DecimalType(30, 5) + val readLo = read.get(0, dt) + val readHi = read.get(1, dt) + assert(readLo == lo, s"lower bound corrupted: expected $lo got $readLo") + assert(readHi == hi, s"upper bound corrupted: expected $hi got $readHi") + } + + test("String byte-wise lex round-trip preserves UTF-8 bytes") { + val lo = UTF8String.fromString("apple") + // UTF-8 bytes for two CJK code points (U+4E2D U+6587) constructed from hex + // to keep this file ASCII-only (scalastyle nonascii filter). + val hi = UTF8String.fromBytes(Array[Byte]( + 0xe4.toByte, + 0xb8.toByte, + 0xad.toByte, + 0xe6.toByte, + 0x96.toByte, + 0x87.toByte)) + val stats: InternalRow = new GenericInternalRow( + Array[Any](lo, hi, 0, 100, 1024L)) + val schema = StructType(Seq(StructField("s", StringType))) + val blob = CachedColumnarBatchKryoSerializer.serializeStats(stats, schema) + val read = CachedColumnarBatchKryoSerializer.deserializeStats(blob, schema) + val readLo = read.getUTF8String(0) + val readHi = read.getUTF8String(1) + assert(readLo == lo, s"lower bound corrupted: expected $lo got $readLo") + assert(readHi == hi, s"upper bound corrupted: expected $hi got $readHi") + } + + test("String truncation to 256B widens upper bound monotonically") { + val loBytes = new Array[Byte](100) + Arrays.fill(loBytes, 'a'.toByte) + // Upper: 300 bytes of 'm' followed by trailing chars. Truncated prefix + // is 256 'm's (all 0x6d); +1 carry on last byte -> last byte = 0x6e. + val hiBytes = new Array[Byte](300) + Arrays.fill(hiBytes, 'm'.toByte) + val lo = UTF8String.fromBytes(loBytes) + val hi = UTF8String.fromBytes(hiBytes) + val stats: InternalRow = new GenericInternalRow(Array[Any](lo, hi, 0, 100, 50000L)) + val schema = StructType(Seq(StructField("s", StringType))) + val blob = CachedColumnarBatchKryoSerializer.serializeStats(stats, schema) + val read = CachedColumnarBatchKryoSerializer.deserializeStats(blob, schema) + val readLo = read.getUTF8String(0) + val readHi = read.getUTF8String(1) + assert(readLo.numBytes() == 100, s"lower untruncated, got numBytes=${readLo.numBytes()}") + assert(readHi.numBytes() == 256, s"upper should truncate to 256B, got ${readHi.numBytes()}") + val readHiArr = readHi.getBytes + // First 255 bytes still 'm', last byte 'm'+1 = 'n' (0x6e) due to carry. + var i = 0 + while (i < 255) { + assert(readHiArr(i) == 'm'.toByte, s"upper byte $i = ${readHiArr(i)}, expected 'm'") + i += 1 + } + assert( + readHiArr(255) == 'n'.toByte, + s"upper byte 255 should be 'n' (carry), got ${readHiArr(255)}") + } + + test("String carry overflow demotes column to unsupported") { + val loBytes = new Array[Byte](10) + Arrays.fill(loBytes, 0xff.toByte) + val hiBytes = new Array[Byte](300) + Arrays.fill(hiBytes, 0xff.toByte) + val lo = UTF8String.fromBytes(loBytes) + val hi = UTF8String.fromBytes(hiBytes) + val stats: InternalRow = new GenericInternalRow(Array[Any](lo, hi, 0, 100, 50000L)) + val schema = StructType(Seq(StructField("s", StringType))) + val blob = CachedColumnarBatchKryoSerializer.serializeStats(stats, schema) + val read = CachedColumnarBatchKryoSerializer.deserializeStats(blob, schema) + assert(read.isNullAt(0), "lower bound must be null when carry overflows") + assert(read.isNullAt(1), "upper bound must be null when carry overflows") + } +} 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 new file mode 100644 index 000000000000..1956139d1c53 --- /dev/null +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchBuildFilterPruneSuite.scala @@ -0,0 +1,95 @@ +/* + * 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.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, GenericInternalRow, Literal} +import org.apache.spark.sql.columnar.CachedBatch +import org.apache.spark.sql.types.LongType + +import org.scalatest.funsuite.AnyFunSuite + +/** + * Tests for the lazy-split wrapper's stats!=null branch: batches whose [lowerBound, upperBound] + * does not cover the literal must be pruned; batches whose range covers it must be returned. + * + * Pure JVM. Uses GenericInternalRow with the vanilla 5-slot-per-col schema (lower, upper, + * nullCount, count, sizeInBytes). + */ +class ColumnarCachedBatchBuildFilterPruneSuite extends AnyFunSuite { + + // Build a CachedColumnarBatch with BIGINT 1-col stats [lower, upper]. + private def batchWithStats(numRows: Int, lower: Long, upper: Long): CachedColumnarBatch = { + val stats = new GenericInternalRow(Array[Any](lower, upper, 0, numRows, numRows * 8L)) + CachedColumnarBatch( + numRows = numRows, + sizeInBytes = numRows * 8L, + bytes = Array.fill[Byte](numRows * 4)(0), + stats = stats) + } + + test("EqualTo literal in [lower, upper] keeps the batch") { + val serializer = new ColumnarCachedBatchSerializer + val attr = AttributeReference("id", LongType, nullable = false)() + val predicate = EqualTo(attr, Literal(50L)) + val filter = serializer.buildFilter(Seq(predicate), Seq(attr)) + + val batches: Iterator[CachedBatch] = Iterator(batchWithStats(10, 0L, 100L)) + val result = filter(0, batches).toList + + assert(result.length === 1, "batch with [0, 100] covers literal 50, must be kept") + assert(result.head.numRows === 10) + } + + test("EqualTo literal outside [lower, upper] prunes the batch") { + 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 batches: Iterator[CachedBatch] = Iterator(batchWithStats(10, 0L, 100L)) + val result = filter(0, batches).toList + + assert(result.length === 0, "batch with [0, 100] cannot contain 999, must be pruned") + } + + test("mixed null/non-null stats: null through, non-null pruned by predicate") { + val serializer = new ColumnarCachedBatchSerializer + val attr = AttributeReference("id", LongType, nullable = false)() + val predicate = EqualTo(attr, Literal(50L)) + val filter = serializer.buildFilter(Seq(predicate), Seq(attr)) + + val nullStats = CachedColumnarBatch( + numRows = 7, + sizeInBytes = 28L, + bytes = Array[Byte](9, 9), + stats = null) + val keptBatch = batchWithStats(10, 0L, 100L) // covers 50, kept + val prunedBatch = batchWithStats(5, 200L, 300L) // does not cover 50, pruned + + val batches: Iterator[CachedBatch] = Iterator(nullStats, prunedBatch, keptBatch) + val result = filter(0, batches).toList + + // Expected order: nullStats first (direct), then keptBatch. + // prunedBatch dropped by parent. + assert( + result.length === 2, + s"expected 2 (nullStats + keptBatch), got ${result.length}") + assert( + result.map(_.numRows) === Seq(7, 10), + "order: stats=null pass-through first, then stats-covers-literal kept") + } +} diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchBuildFilterSuite.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchBuildFilterSuite.scala new file mode 100644 index 000000000000..c54f82faeb39 --- /dev/null +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchBuildFilterSuite.scala @@ -0,0 +1,77 @@ +/* + * 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.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} +import org.apache.spark.sql.columnar.CachedBatch +import org.apache.spark.sql.types.IntegerType + +import org.scalatest.funsuite.AnyFunSuite + +/** + * Tests for the buildFilter stats=null guard (lazy-split iterator wrapper). Without the wrapper, + * vanilla `SimpleMetricsCachedBatchSerializer.buildFilter` NPEs on `partitionFilter.eval(null)` + * (non-trivial predicates, both codegen and interpreted paths). A naive null-literal occupier row + * would fail differently and silently drop every batch. + * + * Pure JVM, no native lib. Drives buildFilter on a synthetic Iterator[CachedBatch] of + * CachedColumnarBatch with stats=null under a non-trivial predicate. + */ +class ColumnarCachedBatchBuildFilterSuite extends AnyFunSuite { + + test("buildFilter must direct stats=null batches through under EqualTo predicate") { + val serializer = new ColumnarCachedBatchSerializer + val attr = AttributeReference("id", IntegerType, nullable = false)() + val predicate = EqualTo(attr, Literal(5)) + val cachedAttributes = Seq(attr) + + val filter = serializer.buildFilter(Seq(predicate), cachedAttributes) + + // Three v1 binary batches: stats=null. A correct wrapper directs + // them all through (no pruning, since we have no stats to prune on). + val batches: Iterator[CachedBatch] = Iterator( + CachedColumnarBatch( + numRows = 3, + sizeInBytes = 12L, + bytes = Array[Byte](1, 2, 3), + stats = null), + CachedColumnarBatch( + numRows = 5, + sizeInBytes = 20L, + bytes = Array[Byte](4, 5), + stats = null), + CachedColumnarBatch( + numRows = 7, + sizeInBytes = 28L, + bytes = Array[Byte](6, 7, 8), + stats = null) + ) + + // GREEN expectation: all 3 batches returned (stats=null -> direct through). + // Without the wrapper: NPE thrown at partitionFilter.eval(null) + // OR silent drop (if naive occupier-row placeholder was used). + val result = filter(0, batches).toList + + assert( + result.length === 3, + "v1 binary (stats=null) batches must all be returned (lazy-split wrapper); " + + s"got ${result.length}") + assert( + result.map(_.numRows) === Seq(3, 5, 7), + "batch order must be preserved through wrapper") + } +} 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 new file mode 100644 index 000000000000..3b4d1d3a161b --- /dev/null +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchE2ESuite.scala @@ -0,0 +1,345 @@ +/* + * 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.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.columnar.{InMemoryTableScanExec, SparkCacheUtil} +import org.apache.spark.sql.functions.{col, lit, when} + +import java.sql.Timestamp +import java.time.Instant + +/** + * End-to-end smoke for Gluten in-memory cache stats (Layer A min/max). + * + * Asserts no crash, correct result, plan shape, and `numOutputRows` significantly less than total + * rows. Precise prune semantics live in `ColumnarCachedBatchBuildFilterPruneSuite`. + */ +class ColumnarCachedBatchE2ESuite + extends VeloxWholeStageTransformerSuite + with AdaptiveSparkPlanHelper { + override protected val resourcePath: String = "/tpch-data-parquet" + override protected val fileFormat: String = "parquet" + + 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") + } + + // Build a deterministic, range-partitioned cached frame: + // k in [0, N), repartitioned to P partitions by id range so each partition + // carries a disjoint k interval. A point-equality filter on the pivot can + // then be pruned to a single partition by min/max metadata. + private val N: Long = 1000L + private val P: Int = 5 + private val pivot: Long = 500L // falls inside partition that owns [400, 600) + + private def cacheRange(): DataFrame = { + spark + .range(N) + .select(col("id").cast("bigint").as("k")) + .repartitionByRange(P, col("k")) + .cache() + } + + test("e2e cache + equality filter: no crash + correct result") { + val cached = cacheRange() + try { + cached.count() // materialize cache (triggers serializeWithStats path) + val result = cached.filter(col("k") === pivot).count() + assert(result == 1L, s"expected exactly one row matching k=$pivot, got $result") + } finally { + cached.unpersist() + } + } + + test("plan contains InMemoryTableScanExec + our serializer kicked in") { + val cached = cacheRange() + try { + cached.count() + val df = cached.filter(col("k") === pivot) + val plan = df.queryExecution.executedPlan + val scan = find(plan) { + case _: InMemoryTableScanExec => true + case _ => false + } + assert(scan.isDefined, s"plan missing InMemoryTableScanExec:\n$plan") + val ims = scan.get.asInstanceOf[InMemoryTableScanExec] + val serName = ims.relation.cacheBuilder.serializer.getClass.getSimpleName + assert( + serName == "ColumnarCachedBatchSerializer", + s"expected ColumnarCachedBatchSerializer, got $serName" + ) + // Force execution so numOutputRows is populated for the next assertion. + df.count() + } finally { + cached.unpersist() + } + } + + test("numOutputRows reflects post-filter row count (significantly < N)") { + val cached = cacheRange() + try { + cached.count() + val df = cached.filter(col("k") === pivot) + df.count() + val plan = df.queryExecution.executedPlan + val ims = find(plan) { + case _: InMemoryTableScanExec => true + case _ => false + }.get.asInstanceOf[InMemoryTableScanExec] + val outRows = ims.metrics("numOutputRows").value + // Prune evidence: numOutputRows must be << N (full-scan would give N). + // Lower bound is 0 -- with full partition pruning the InMemoryTableScanExec + // node may legitimately emit zero rows (the surviving row comes from cache + // metadata / pivot resolution at a higher layer when Gluten native scan + // uses its own metrics path). The semantic correctness is anchored by + // the equality-result test (result == 1) and the precise prune behavior by + // BuildFilterPruneSuite; this case only needs to refute full-scan. + val upperBound = (N / P) * 2 + assert( + outRows <= upperBound, + s"numOutputRows=$outRows expected <= $upperBound " + + s"(N=$N, P=$P, full-scan would give $N -- prune appears not effective)" + ) + } finally { + cached.unpersist() + } + } + + test("all-null Long column: cache + equality filter no crash + correct result") { + val df = spark + .range(N) + .select(lit(null).cast("bigint").as("k")) + .repartition(P) + .cache() + try { + df.count() // materialize + val result = df.filter(col("k") === 5L).count() + assert(result == 0L, s"all-null col cannot match k=5, got $result") + // Sanity: cached count (pre-filter) is still N + assert(df.count() == N, s"all-null cached frame should still hold $N rows") + } finally { + df.unpersist() + } + } + + test("Float NaN partition: filter on non-NaN not silently pruned") { + val df = spark + .range(N) + .select( + when(col("id") === 7L, lit(Float.NaN)) + .otherwise(col("id").cast("float")) + .as("k")) + .repartition(P) + .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)") + } finally { + df.unpersist() + } + } + + test("Date column equality filter: prune via INTEGER stats (4B LE)") { + import org.apache.spark.sql.functions.{date_add, lit => sparkLit} + val base = sparkLit("2020-01-01").cast("date") + val cached = spark + .range(N) + .select(date_add(base, col("id").cast("int")).as("d")) + .repartitionByRange(P, col("d")) + .cache() + try { + cached.count() // materialize - triggers cpp INTEGER computeStats path + val pivotDate = date_add(base, sparkLit(pivot.toInt)) + val df = cached.filter(col("d") === pivotDate) + val result = df.count() + assert(result == 1L, s"expected exactly one row matching date pivot, got $result") + val plan = df.queryExecution.executedPlan + val ims = find(plan) { + case _: InMemoryTableScanExec => true + case _ => false + }.get.asInstanceOf[InMemoryTableScanExec] + val outRows = ims.metrics("numOutputRows").value + val upperBound = (N / P) * 2 + assert( + outRows <= upperBound, + s"numOutputRows=$outRows expected <= $upperBound (Date prune ineffective)" + ) + } finally { + cached.unpersist() + } + } + + test("multi-column cache: no IndexOOB + correct result") { + val cached = spark + .range(N) + .selectExpr( + "cast(id as bigint) as a", + "cast(id * 2 as bigint) as b", + "cast(id + 100 as bigint) as c") + .repartitionByRange(P, col("a")) + .cache() + try { + cached.count() + val result = cached.filter(col("a") === pivot && col("c") === (pivot + 100L)).count() + assert(result == 1L, s"expected 1 row matching pivot, got $result") + } finally { + cached.unpersist() + } + } + + test("Decimal column cache: no UOE crash on materialize + read") { + val cached = spark + .range(N) + .selectExpr("cast(id as decimal(10, 2)) as d") + .repartition(P) + .cache() + try { + cached.count() + val total = cached.count() + assert(total == N, s"expected $N rows, got $total") + } finally { + cached.unpersist() + } + } + + test("IsNotNull predicate honors vanilla count semantics") { + val df = spark + .range(N) + .selectExpr("if(id % 3 = 0, cast(null as bigint), id) as k") // ~33% nulls + .repartition(P) + .cache() + try { + df.count() // materialize + val nonNullCount = df.filter(col("k").isNotNull).count() + val expected = (0L until N).count(_ % 3 != 0).toLong + assert( + nonNullCount == expected, + s"IsNotNull silently dropped partitions: got $nonNullCount, expected $expected") + } finally { + df.unpersist() + } + } + + test("Timestamp column equality filter: prune via Long us stats (8B LE)") { + import org.apache.spark.sql.functions.{lit => sparkLit} + // Build N rows of distinct timestamps, one second apart starting at epoch + // 2024-01-01T00:00:00Z (= 1704067200 seconds). Pivot at second N/2. + val baseSec = 1704067200L + val cached = spark + .range(N) + .selectExpr(s"timestamp_seconds(${baseSec}L + id) as ts") + .repartitionByRange(P, col("ts")) + .cache() + try { + cached.count() // materialize -- triggers cpp TIMESTAMP computeStats path + val pivotTs = sparkLit(Timestamp.from(Instant.ofEpochSecond(baseSec + (N / 2)))) + val df = cached.filter(col("ts") === pivotTs) + val result = df.count() + assert(result == 1L, s"expected exactly one row matching timestamp pivot, got $result") + val plan = df.queryExecution.executedPlan + val ims = find(plan) { + case _: InMemoryTableScanExec => true + case _ => false + }.get.asInstanceOf[InMemoryTableScanExec] + val outRows = ims.metrics("numOutputRows").value + val upperBound = (N / P) * 2 + assert( + outRows <= upperBound, + s"numOutputRows=$outRows expected <= $upperBound (Timestamp prune ineffective)" + ) + } finally { + cached.unpersist() + } + } + + test("String column equality filter: prune via byte-unsigned stats") { + val cached = spark + .range(N) + .selectExpr("concat('k_', lpad(cast(id as string), 4, '0')) as s") + .repartitionByRange(P, col("s")) + .cache() + try { + cached.count() // materialize -- triggers cpp VARCHAR computeStats path + val pivotStr = "k_0500" + val df = cached.filter(col("s") === pivotStr) + val result = df.count() + assert(result == 1L, s"expected exactly one row matching string pivot, got $result") + val plan = df.queryExecution.executedPlan + val ims = find(plan) { + case _: InMemoryTableScanExec => true + case _ => false + }.get.asInstanceOf[InMemoryTableScanExec] + val outRows = ims.metrics("numOutputRows").value + val upperBound = (N / P) * 2 + assert( + outRows <= upperBound, + s"numOutputRows=$outRows expected <= $upperBound (String prune ineffective)" + ) + } finally { + cached.unpersist() + } + } + + // 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. + // + // 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") { + 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) + val result = cached.filter(col("k") === pivot).count() + assert(result == 1L, s"expected exactly one row matching k=$pivot, 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 new file mode 100644 index 000000000000..018aa880e10a --- /dev/null +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchFramedBytesSuite.scala @@ -0,0 +1,100 @@ +/* + * 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.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow + +import org.scalatest.funsuite.AnyFunSuite + +import java.util.Locale + +/** + * Tests for JNI `serializeWithStats` framed-byte parser. Pure JVM; crafts framed bytes by hand via + * serializeStats, then exercises parseFramedBytes round-trip + corrupt-magic / truncated-blob + * guards. + * + * Layout: `[ STATS_FRAMED_MAGIC: 4B 0xFE 0xCA 0x53 0x02 ] [ statsLen: u32 LE ] [ statsBlob ] [ + * bytesLen: u32 LE ] [ bytesBlob ]`. + */ +class ColumnarCachedBatchFramedBytesSuite extends AnyFunSuite { + + private def craftFramed(stats: InternalRow, bytesBlob: Array[Byte]): Array[Byte] = { + val statsBlob = CachedColumnarBatchKryoSerializer.serializeStats(stats, null) + val out = new java.io.ByteArrayOutputStream() + out.write(CachedColumnarBatchKryoSerializer.STATS_FRAMED_MAGIC) + writeU32LE(out, statsBlob.length) + out.write(statsBlob) + writeU32LE(out, bytesBlob.length) + out.write(bytesBlob) + out.toByteArray + } + + 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) + } + + test("parseFramedBytes round-trip BIGINT 1-col stats + bytesBlob") { + val stats: InternalRow = new GenericInternalRow( + Array[Any](42L, 100L, 0, 10, 64L)) + val payload = Array[Byte](1, 2, 3, 4, 5) + val framed = craftFramed(stats, payload) + + val (parsedStats, parsedBytes) = + CachedColumnarBatchKryoSerializer.parseFramedBytes(framed, null) + + assert(parsedBytes === payload, "bytesBlob round-trip exact") + assert(parsedStats !== null, "stats round-trip non-null") + assert(parsedStats.numFields === 5) + assert(parsedStats.getLong(0) === 42L) + assert(parsedStats.getLong(1) === 100L) + assert(parsedStats.getInt(2) === 0) + assert(parsedStats.getInt(3) === 10) + assert(parsedStats.getLong(4) === 64L) + } + + test("corrupt magic fails eagerly with clear message") { + val payload = Array[Byte](1, 2, 3) + val stats: InternalRow = new GenericInternalRow( + Array[Any](0L, 0L, 0, 1, 1L)) + val good = craftFramed(stats, payload) + val bad = good.clone() + bad(0) = 0x00.toByte // corrupt magic byte 0 + + val ex = intercept[IllegalArgumentException] { + CachedColumnarBatchKryoSerializer.parseFramedBytes(bad, null) + } + assert( + ex.getMessage.toLowerCase(Locale.ROOT).contains("magic"), + s"expected 'magic' in error, got: ${ex.getMessage}") + } + + test("truncated framed bytes fails eagerly") { + val payload = Array[Byte](1, 2, 3) + val stats: InternalRow = new GenericInternalRow( + Array[Any](0L, 0L, 0, 1, 1L)) + val good = craftFramed(stats, payload) + val truncated = good.take(8) // magic + statsLen only, no statsBlob + + intercept[Exception] { + CachedColumnarBatchKryoSerializer.parseFramedBytes(truncated, null) + } + } +} diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchIntFamilyMarshalSuite.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchIntFamilyMarshalSuite.scala new file mode 100644 index 000000000000..632f7cdfddeb --- /dev/null +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchIntFamilyMarshalSuite.scala @@ -0,0 +1,185 @@ +/* + * 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.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.types.{ByteType, DateType, DayTimeIntervalType, IntegerType, LongType, ShortType, StructField, StructType, TimestampNTZType, TimestampType, YearMonthIntervalType} + +import org.scalatest.funsuite.AnyFunSuite + +/** + * Integer-family marshal tests (INT / SMALLINT / TINYINT / Date / interval). JVM-only; pins + * serialize/deserialize dispatch by source-column dataType. + */ +class ColumnarCachedBatchIntFamilyMarshalSuite extends AnyFunSuite { + + test("INT round-trip 4B LE preserves value") { + val lo: Integer = Int.box(-2147483) + val hi: Integer = Int.box(2147483) + val stats: InternalRow = new GenericInternalRow( + Array[Any](lo, hi, 0, 100, 400L)) + val schema = StructType(Seq( + StructField("k.lowerBound", IntegerType, nullable = true), + StructField("k.upperBound", IntegerType, nullable = true), + StructField("k.nullCount", IntegerType, nullable = false), + StructField("k.count", IntegerType, nullable = false), + StructField("k.sizeInBytes", LongType, nullable = false) + )) + val blob = CachedColumnarBatchKryoSerializer.serializeStats(stats, schema) + val read = CachedColumnarBatchKryoSerializer.deserializeStats(blob, schema) + assert(read.getInt(0) == lo, s"lower bound corrupted: expected $lo got ${read.getInt(0)}") + assert(read.getInt(1) == hi, s"upper bound corrupted: expected $hi got ${read.getInt(1)}") + assert(read.getInt(2) == 0, "nullCount roundtrip") + assert(read.getInt(3) == 100, "count roundtrip") + assert(read.getLong(4) == 400L, "sizeInBytes roundtrip") + } + + test("SMALLINT round-trip 2B LE preserves value (incl negative)") { + val lo: java.lang.Short = Short.box((-12345).toShort) + val hi: java.lang.Short = Short.box(12345.toShort) + val stats: InternalRow = new GenericInternalRow( + Array[Any](lo, hi, 0, 200, 400L)) + val schema = StructType(Seq( + StructField("k.lowerBound", ShortType, nullable = true), + StructField("k.upperBound", ShortType, nullable = true), + StructField("k.nullCount", IntegerType, nullable = false), + StructField("k.count", IntegerType, nullable = false), + StructField("k.sizeInBytes", LongType, nullable = false) + )) + val blob = CachedColumnarBatchKryoSerializer.serializeStats(stats, schema) + val read = CachedColumnarBatchKryoSerializer.deserializeStats(blob, schema) + assert(read.getShort(0) == lo, s"lower: expected $lo got ${read.getShort(0)}") + assert(read.getShort(1) == hi, s"upper: expected $hi got ${read.getShort(1)}") + assert(read.getInt(3) == 200, "count roundtrip") + } + + test("TINYINT round-trip 1B preserves value (incl negative)") { + val lo: java.lang.Byte = Byte.box((-128).toByte) + val hi: java.lang.Byte = Byte.box(127.toByte) + val stats: InternalRow = new GenericInternalRow( + Array[Any](lo, hi, 0, 50, 50L)) + val schema = StructType( + Seq( + StructField("k.lowerBound", ByteType, nullable = true), + StructField("k.upperBound", ByteType, nullable = true), + StructField("k.nullCount", IntegerType, nullable = false), + StructField("k.count", IntegerType, nullable = false), + StructField("k.sizeInBytes", LongType, nullable = false) + )) + val blob = CachedColumnarBatchKryoSerializer.serializeStats(stats, schema) + val read = CachedColumnarBatchKryoSerializer.deserializeStats(blob, schema) + assert(read.getByte(0) == lo, s"lower: expected $lo got ${read.getByte(0)}") + assert(read.getByte(1) == hi, s"upper: expected $hi got ${read.getByte(1)}") + assert(read.getInt(3) == 50, "count roundtrip") + } + + test("YearMonthInterval round-trip 4B LE (months as Int)") { + val lo: Integer = Int.box(-12) + val hi: Integer = Int.box(36) + val stats: InternalRow = new GenericInternalRow( + Array[Any](lo, hi, 0, 10, 40L)) + val schema = StructType( + Seq( + StructField("k.lowerBound", YearMonthIntervalType(), nullable = true), + StructField("k.upperBound", YearMonthIntervalType(), nullable = true), + StructField("k.nullCount", IntegerType, nullable = false), + StructField("k.count", IntegerType, nullable = false), + StructField("k.sizeInBytes", LongType, nullable = false) + )) + val blob = CachedColumnarBatchKryoSerializer.serializeStats(stats, schema) + val read = CachedColumnarBatchKryoSerializer.deserializeStats(blob, schema) + assert(read.getInt(0) == lo, s"lower: expected $lo got ${read.getInt(0)}") + assert(read.getInt(1) == hi, s"upper: expected $hi got ${read.getInt(1)}") + } + + test("DayTimeInterval round-trip 8B LE (microseconds as Long)") { + val lo: java.lang.Long = Long.box(-86400000000L) + val hi: java.lang.Long = Long.box(86400000000L) + val stats: InternalRow = new GenericInternalRow( + Array[Any](lo, hi, 0, 10, 80L)) + val schema = StructType( + Seq( + StructField("k.lowerBound", DayTimeIntervalType(), nullable = true), + StructField("k.upperBound", DayTimeIntervalType(), nullable = true), + StructField("k.nullCount", IntegerType, nullable = false), + StructField("k.count", IntegerType, nullable = false), + StructField("k.sizeInBytes", LongType, nullable = false) + )) + val blob = CachedColumnarBatchKryoSerializer.serializeStats(stats, schema) + val read = CachedColumnarBatchKryoSerializer.deserializeStats(blob, schema) + assert(read.getLong(0) == lo, s"lower: expected $lo got ${read.getLong(0)}") + assert(read.getLong(1) == hi, s"upper: expected $hi got ${read.getLong(1)}") + } + + test("Date round-trip 4B LE (days since epoch as Int)") { + val lo: Integer = Int.box(0) // 1970-01-01 + val hi: Integer = Int.box(20000) // ~2024 + val stats: InternalRow = new GenericInternalRow( + Array[Any](lo, hi, 0, 100, 400L)) + val schema = StructType( + Seq( + StructField("d.lowerBound", DateType, nullable = true), + StructField("d.upperBound", DateType, nullable = true), + StructField("d.nullCount", IntegerType, nullable = false), + StructField("d.count", IntegerType, nullable = false), + StructField("d.sizeInBytes", LongType, nullable = false) + )) + val blob = CachedColumnarBatchKryoSerializer.serializeStats(stats, schema) + val read = CachedColumnarBatchKryoSerializer.deserializeStats(blob, schema) + assert(read.getInt(0) == lo) + assert(read.getInt(1) == hi) + } + + test("Timestamp round-trip 8B LE (microseconds as Long)") { + val lo: java.lang.Long = Long.box(1700000000000000L) + val hi: java.lang.Long = Long.box(1800000000000000L) + val stats: InternalRow = new GenericInternalRow( + Array[Any](lo, hi, 0, 100, 800L)) + val schema = StructType( + Seq( + StructField("ts.lowerBound", TimestampType, nullable = true), + StructField("ts.upperBound", TimestampType, nullable = true), + StructField("ts.nullCount", IntegerType, nullable = false), + StructField("ts.count", IntegerType, nullable = false), + StructField("ts.sizeInBytes", LongType, nullable = false) + )) + val blob = CachedColumnarBatchKryoSerializer.serializeStats(stats, schema) + val read = CachedColumnarBatchKryoSerializer.deserializeStats(blob, schema) + assert(read.getLong(0) == lo) + assert(read.getLong(1) == hi) + } + + test("TimestampNTZ round-trip 8B LE (microseconds as Long)") { + val lo: java.lang.Long = Long.box(0L) + val hi: java.lang.Long = Long.box(2000000000000000L) + val stats: InternalRow = new GenericInternalRow( + Array[Any](lo, hi, 0, 100, 800L)) + val schema = StructType( + Seq( + StructField("tsntz.lowerBound", TimestampNTZType, nullable = true), + StructField("tsntz.upperBound", TimestampNTZType, nullable = true), + StructField("tsntz.nullCount", IntegerType, nullable = false), + StructField("tsntz.count", IntegerType, nullable = false), + StructField("tsntz.sizeInBytes", LongType, nullable = false) + )) + val blob = CachedColumnarBatchKryoSerializer.serializeStats(stats, schema) + val read = CachedColumnarBatchKryoSerializer.deserializeStats(blob, schema) + assert(read.getLong(0) == lo) + assert(read.getLong(1) == hi) + } +} diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchKryoSuite.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchKryoSuite.scala new file mode 100644 index 000000000000..ead2e67a49d5 --- /dev/null +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchKryoSuite.scala @@ -0,0 +1,178 @@ +/* + * 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.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow + +import com.esotericsoftware.kryo.Kryo +import com.esotericsoftware.kryo.io.{Input, Output} +import org.scalatest.funsuite.AnyFunSuite + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} + +/** + * Tests for CachedColumnarBatch Kryo (de)serialization. Pure JVM, no native lib required. + * + * Wire-format invariant: Kryo `Output.writeInt(int)` writes a fixed 4-byte BIG-ENDIAN int; the + * (int, boolean) overload silently forwards to writeVarInt (1-5 bytes) and would corrupt the + * length-prefixed read path. + */ +class ColumnarCachedBatchKryoSuite extends AnyFunSuite { + + // Use serializer.write / serializer.read directly; writeObject/readObject would prepend a + // varint refId envelope. Production goes through Spark SerializerInstance internals where + // reference tracking is off for cached batches. + private def roundTrip(batch: CachedColumnarBatch): CachedColumnarBatch = { + val ser = new CachedColumnarBatchKryoSerializer + val kryo = new Kryo() + val baos = new ByteArrayOutputStream() + val out = new Output(baos) + ser.write(kryo, out, batch) + out.close() + val in = new Input(new ByteArrayInputStream(baos.toByteArray)) + val read = ser.read(kryo, in, classOf[CachedColumnarBatch]) + in.close() + read + } + + test("stats field round-trip") { + val stats: InternalRow = new GenericInternalRow( + Array[Any](42L, 100L, 0, 10, 64L)) + val batch = CachedColumnarBatch( + numRows = 10, + sizeInBytes = 64L, + bytes = Array[Byte](1, 2, 3, 4), + stats = stats) + + val read = roundTrip(batch) + + assert(read.numRows === 10) + assert(read.sizeInBytes === 64L) + assert(read.bytes === Array[Byte](1, 2, 3, 4)) + assert(read.stats !== null, "stats field must round-trip") + assert(read.stats.numFields === 5, "vanilla PartitionStatistics = 5 slots / col") + assert(read.stats.getLong(0) === 42L, "lowerBound at slot 0") + assert(read.stats.getLong(1) === 100L, "upperBound at slot 1") + assert(read.stats.getInt(2) === 0, "nullCount at slot 2") + assert(read.stats.getInt(3) === 10, "count at slot 3") + assert(read.stats.getLong(4) === 64L, "sizeInBytes at slot 4") + } + + test("stats=null round-trip") { + val batch = CachedColumnarBatch( + numRows = 7, + sizeInBytes = 123L, + bytes = Array[Byte](9, 8, 7), + stats = null) + + val read = roundTrip(batch) + + assert(read.numRows === 7) + assert(read.sizeInBytes === 123L) + assert(read.bytes === Array[Byte](9, 8, 7)) + assert(read.stats === null) + } + + // Build a V1-format byte stream: numRows + sizeInBytes + length + bytes, with NO trailing + // hasStats / hasSchema booleans. Mirrors a CachedColumnarBatch persisted by an older Gluten + // jar and surviving a rolling upgrade (DISK_ONLY / MEMORY_AND_DISK storage). + private def writeV1Stream(numRows: Int, sizeInBytes: Long, payload: Array[Byte]): Array[Byte] = { + val baos = new ByteArrayOutputStream() + val out = new Output(baos) + out.writeInt(numRows) + out.writeLong(sizeInBytes) + out.writeInt(payload.length + 1) + out.writeBytes(payload) + out.close() + baos.toByteArray + } + + test("V1 wire (no trailing hasStats/hasSchema booleans) reads as stats=null/schema=null") { + val raw = writeV1Stream(numRows = 5, sizeInBytes = 99L, payload = Array[Byte](1, 2, 3)) + val ser = new CachedColumnarBatchKryoSerializer + val kryo = new Kryo() + val in = new Input(new ByteArrayInputStream(raw)) + val read = ser.read(kryo, in, classOf[CachedColumnarBatch]) + in.close() + + assert(read.numRows === 5) + assert(read.sizeInBytes === 99L) + assert(read.bytes === Array[Byte](1, 2, 3)) + assert(read.stats === null, "absent trailing hasStats must read as null, not throw") + assert(read.schema === null, "absent trailing hasSchema must read as null, not throw") + } + + // Construct a corrupt stream by hand-writing the length-prefix only (no payload follows). The + // production read path must reject the bogus length BEFORE allocating the array, otherwise + // either negative-size or multi-GB allocation would crash the executor. + private def streamWithBogusLength(length: Int): Array[Byte] = { + val baos = new ByteArrayOutputStream() + val out = new Output(baos) + out.writeInt(1) // numRows + out.writeLong(0L) // sizeInBytes + out.writeInt(length) // bogus length + out.close() + baos.toByteArray + } + + test("read rejects negative length without NegativeArraySizeException") { + val raw = streamWithBogusLength(-100) // payloadLen = -101 + val ser = new CachedColumnarBatchKryoSerializer + val in = new Input(new ByteArrayInputStream(raw)) + val ex = intercept[IllegalArgumentException] { + ser.read(new Kryo(), in, classOf[CachedColumnarBatch]) + } + assert( + ex.getMessage.contains("out of bounds"), + s"expected bounds-check failure, got: ${ex.getMessage}") + } + + test("read rejects oversized length without OOM") { + // length = Int.MaxValue would attempt a 2 GB allocation (well above 64 MiB ceiling). + val raw = streamWithBogusLength(Int.MaxValue) + val ser = new CachedColumnarBatchKryoSerializer + val in = new Input(new ByteArrayInputStream(raw)) + val ex = intercept[IllegalArgumentException] { + ser.read(new Kryo(), in, classOf[CachedColumnarBatch]) + } + assert( + ex.getMessage.contains("out of bounds"), + s"expected bounds-check failure, got: ${ex.getMessage}") + } + + test("read rejects oversized statsLen without OOM") { + // Build a stream with valid payload, hasStats=true, then a bogus statsLen. + val baos = new ByteArrayOutputStream() + val out = new Output(baos) + out.writeInt(1) + out.writeLong(0L) + out.writeInt(2) // payload length+1, payloadLen = 1 + out.writeBytes(Array[Byte](7)) + out.writeBoolean(true) // hasStats + out.writeInt(Int.MaxValue) // bogus statsLen + out.close() + val ser = new CachedColumnarBatchKryoSerializer + val in = new Input(new ByteArrayInputStream(baos.toByteArray)) + val ex = intercept[IllegalArgumentException] { + ser.read(new Kryo(), in, classOf[CachedColumnarBatch]) + } + assert( + ex.getMessage.contains("stats length"), + s"expected statsLen bounds-check failure, got: ${ex.getMessage}") + } +} diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchStatsBlobSuite.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchStatsBlobSuite.scala new file mode 100644 index 000000000000..87b622a24ced --- /dev/null +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/ColumnarCachedBatchStatsBlobSuite.scala @@ -0,0 +1,102 @@ +/* + * 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.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow + +import org.scalatest.funsuite.AnyFunSuite + +import java.nio.{ByteBuffer, ByteOrder} + +/** + * Tests for statsBlob binary framing (LE throughout, cpp-aligned). + * + * Wire (BIGINT 1-col): [ numCols: u32 LE ] per col [ supported: u8 | nullCount: u32 | count: u32 | + * sizeInBytes: u64 ] if supported [ lowerLen: u32 | lower bytes | upperLen: u32 | upper bytes ]. + */ +class ColumnarCachedBatchStatsBlobSuite extends AnyFunSuite { + + test("statsBlob LE numCols + BIGINT cell round-trip byte-for-byte") { + val stats: InternalRow = new GenericInternalRow( + Array[Any](42L, 100L, 0, 10, 64L)) + val blob = CachedColumnarBatchKryoSerializer.serializeStats(stats, null) + + // Hand-compute expected wire: + // numCols=1 LE = 01 00 00 00 + // supported=1 = 01 + // nullCount=0 LE = 00 00 00 00 + // count=10 LE = 0A 00 00 00 + // sizeInBytes=64 LE = 40 00 00 00 00 00 00 00 + // lowerBoundLen=8 LE = 08 00 00 00 + // lowerBound=42 LE = 2A 00 00 00 00 00 00 00 + // upperBoundLen=8 LE = 08 00 00 00 + // upperBound=100 LE = 64 00 00 00 00 00 00 00 + val expected = Array[Byte]( + 0x01, 0x00, 0x00, 0x00, + 0x01, + 0x00, 0x00, 0x00, 0x00, + 0x0a, 0x00, 0x00, 0x00, + 0x40, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x08, 0x00, 0x00, 0x00, + 0x2a, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, + 0x08, 0x00, 0x00, 0x00, + 0x64, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00) + assert( + blob === expected, + "statsBlob must match cpp-aligned LE wire format byte-for-byte") + } + + test("serializeStats then deserializeStats round-trip BIGINT 1-col") { + val stats: InternalRow = new GenericInternalRow( + Array[Any](-7L, 999L, 3, 100, 1024L)) + val blob = CachedColumnarBatchKryoSerializer.serializeStats(stats, null) + val read = CachedColumnarBatchKryoSerializer.deserializeStats(blob, null) + + assert(read.numFields === 5) + assert(read.getLong(0) === -7L, "lowerBound at slot 0") + assert(read.getLong(1) === 999L, "upperBound at slot 1") + assert(read.getInt(2) === 3, "nullCount at slot 2") + assert(read.getInt(3) === 100, "count at slot 3") + assert(read.getLong(4) === 1024L, "sizeInBytes at slot 4") + } + + test("corrupt statsBlob (numCols out of range) fails eagerly") { + // Craft a blob claiming numCols=Int.MaxValue: 0xFF FF FF 7F + val corruptNumCols = ByteBuffer.allocate(4).order(ByteOrder.LITTLE_ENDIAN) + .putInt(Int.MaxValue).array() + val ex = intercept[IllegalArgumentException] { + CachedColumnarBatchKryoSerializer.deserializeStats(corruptNumCols, null) + } + assert( + ex.getMessage.contains("numCols"), + s"expected numCols range guard, got: ${ex.getMessage}") + } + + test("unsupported col round-trip preserves null bounds + metrics") { + val stats: InternalRow = new GenericInternalRow( + Array[Any](null, null, 5, 50, 200L)) + val blob = CachedColumnarBatchKryoSerializer.serializeStats(stats, null) + val read = CachedColumnarBatchKryoSerializer.deserializeStats(blob, null) + + assert(read.isNullAt(0), "lowerBound must be null for unsupported col") + assert(read.isNullAt(1), "upperBound must be null for unsupported col") + assert(read.getInt(2) === 5) + assert(read.getInt(3) === 50) + assert(read.getLong(4) === 200L) + } +} diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnarTableCachePartitionStatsBenchmark.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnarTableCachePartitionStatsBenchmark.scala new file mode 100644 index 000000000000..5d66da425943 --- /dev/null +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/ColumnarTableCachePartitionStatsBenchmark.scala @@ -0,0 +1,122 @@ +/* + * 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.storage.StorageLevel + +/** + * Benchmark to measure write/read overhead and pruning benefit of partition stats in columnar table + * cache. To run this benchmark: + * {{{ + * 1. without sbt: + * bin/spark-submit --class --jars + * }}} + */ +object ColumnarTableCachePartitionStatsBenchmark extends SqlBasedBenchmark { + private val numRows = 100L * 1000 * 1000 + private val numParts = 32 + private val confKey = GlutenConfig.COLUMNAR_TABLE_CACHE_PARTITION_STATS_ENABLED.key + + private def buildCache(statsOn: Boolean): DataFrame = { + import org.apache.spark.sql.functions.col + val prev = spark.conf.getOption(confKey) + spark.conf.set(confKey, statsOn.toString) + try { + val cached = spark + .range(numRows) + .selectExpr( + "cast(id as int) as c0", + "id as c2", + "cast(id as string) as c3", + "uuid() as c4") + .repartitionByRange(numParts, col("c2")) + .persist(StorageLevel.MEMORY_ONLY) + cached.count() // materialize cache (stats are emitted on the write path) + cached + } finally { + prev match { + case Some(v) => spark.conf.set(confKey, v) + case None => spark.conf.unset(confKey) + } + } + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + // === Benchmark 1: write-path overhead (cache build) === + val buildBench = new Benchmark("table cache build", numRows, output = output) + Seq(false, true).foreach { + on => + buildBench.addCase(s"partitionStats ${if (on) "on " else "off"}", 3) { + _ => + spark.catalog.clearCache() + buildCache(statsOn = on).unpersist() + } + } + buildBench.run() + spark.catalog.clearCache() + + // Build two cached relations once for the read-path benchmarks + val cachedOff = buildCache(statsOn = false) + val cachedOn = buildCache(statsOn = true) + + // Heavier follow-up operator: groupBy + sum on c2 + count over c3. + // Pruned partitions also skip the agg work, amplifying the prune speedup. + import org.apache.spark.sql.functions._ + def heavyAgg(df: DataFrame, predicate: String): Unit = { + df.where(predicate) + .groupBy((col("c2") % 1000).as("g")) + .agg(sum("c2"), count("c3"), avg("c0")) + .noop() + } + + // === Benchmark 2: read prune, high selectivity (~0.001%) === + val readHighBench = + new Benchmark( + "table cache filter+agg (high selectivity, ~0.001%)", + numRows, + output = output) + readHighBench.addCase("partitionStats off", 3)(_ => heavyAgg(cachedOff, "c2 < 1000")) + readHighBench.addCase("partitionStats on ", 3)(_ => heavyAgg(cachedOn, "c2 < 1000")) + readHighBench.run() + + // === Benchmark 3: read prune, low selectivity (~50%) === + val readLowBench = + new Benchmark( + "table cache filter+agg (low selectivity, ~50%)", + numRows, + output = output) + readLowBench.addCase("partitionStats off", 3)(_ => heavyAgg(cachedOff, "c2 < 50000000")) + readLowBench.addCase("partitionStats on ", 3)(_ => heavyAgg(cachedOn, "c2 < 50000000")) + readLowBench.run() + + // === Benchmark 4: read prune, point lookup (~1 row) === + val readPointBench = + new Benchmark( + "table cache filter+agg (point lookup, 1 row)", + numRows, + output = output) + readPointBench.addCase("partitionStats off", 3)(_ => heavyAgg(cachedOff, "c2 = 50000000")) + readPointBench.addCase("partitionStats on ", 3)(_ => heavyAgg(cachedOn, "c2 = 50000000")) + readPointBench.run() + + spark.catalog.clearCache() + } +} diff --git a/benchmarks/ColumnarTableCachePartitionStatsBenchmark-results.txt b/benchmarks/ColumnarTableCachePartitionStatsBenchmark-results.txt new file mode 100644 index 000000000000..a3599570d271 --- /dev/null +++ b/benchmarks/ColumnarTableCachePartitionStatsBenchmark-results.txt @@ -0,0 +1,28 @@ +OpenJDK 64-Bit Server VM 17.0.18+8-Ubuntu-124.04.1 on Linux 6.6.87.2-microsoft-standard-WSL2 +AMD EPYC 7763 64-Core Processor +table cache build: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +partitionStats off 126425 138565 10546 0.8 1264.3 1.0X +partitionStats on 131431 137094 7581 0.8 1314.3 1.0X + +OpenJDK 64-Bit Server VM 17.0.18+8-Ubuntu-124.04.1 on Linux 6.6.87.2-microsoft-standard-WSL2 +AMD EPYC 7763 64-Core Processor +table cache filter+agg (high selectivity, ~0.001%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +---------------------------------------------------------------------------------------------------------------------------------- +partitionStats off 4431 4492 70 22.6 44.3 1.0X +partitionStats on 1744 1777 31 57.3 17.4 2.5X + +OpenJDK 64-Bit Server VM 17.0.18+8-Ubuntu-124.04.1 on Linux 6.6.87.2-microsoft-standard-WSL2 +AMD EPYC 7763 64-Core Processor +table cache filter+agg (low selectivity, ~50%): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------------ +partitionStats off 5332 5411 70 18.8 53.3 1.0X +partitionStats on 3392 3446 90 29.5 33.9 1.6X + +OpenJDK 64-Bit Server VM 17.0.18+8-Ubuntu-124.04.1 on Linux 6.6.87.2-microsoft-standard-WSL2 +AMD EPYC 7763 64-Core Processor +table cache filter+agg (point lookup, 1 row): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +---------------------------------------------------------------------------------------------------------------------------- +partitionStats off 4343 4489 129 23.0 43.4 1.0X +partitionStats on 1686 1709 21 59.3 16.9 2.6X + diff --git a/cpp/core/jni/JniWrapper.cc b/cpp/core/jni/JniWrapper.cc index 9e194be6ea84..4a3215ae666a 100644 --- a/cpp/core/jni/JniWrapper.cc +++ b/cpp/core/jni/JniWrapper.cc @@ -1294,6 +1294,31 @@ JNIEXPORT jobject JNICALL Java_org_apache_gluten_vectorized_ColumnarBatchSeriali JNI_METHOD_END(nullptr) } +// Framed [magic | statsLen | statsBlob | bytesLen | bytesBlob] entry point. Uses the +// ColumnarBatchSerializer::framedSerializeWithStats virtual hook; non-Velox backends inherit +// the default empty-vector return so callers fall back to the legacy serialize() path. +JNIEXPORT jbyteArray JNICALL +Java_org_apache_gluten_vectorized_ColumnarBatchSerializerJniWrapper_serializeWithStats( // 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->framedSerializeWithStats(batch); + + jbyteArray out = env->NewByteArray(static_cast(framed.size())); + if (!framed.empty()) { + env->SetByteArrayRegion(out, 0, static_cast(framed.size()), reinterpret_cast(framed.data())); + } + return out; + JNI_METHOD_END(nullptr) +} + JNIEXPORT jlong JNICALL Java_org_apache_gluten_vectorized_ColumnarBatchSerializerJniWrapper_init( // NOLINT JNIEnv* env, jobject wrapper, diff --git a/cpp/core/operators/serializer/ColumnarBatchSerializer.h b/cpp/core/operators/serializer/ColumnarBatchSerializer.h index 08a76f9f23de..b7bf6b41bea0 100644 --- a/cpp/core/operators/serializer/ColumnarBatchSerializer.h +++ b/cpp/core/operators/serializer/ColumnarBatchSerializer.h @@ -18,6 +18,7 @@ #pragma once #include +#include #include "memory/ColumnarBatch.h" @@ -37,6 +38,14 @@ 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. + virtual std::vector framedSerializeWithStats(const std::shared_ptr& /*batch*/) { + return {}; + } + protected: arrow::MemoryPool* arrowPool_; }; diff --git a/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc b/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc index 1931b910ecb6..50e8a96abd63 100644 --- a/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc +++ b/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc @@ -19,6 +19,10 @@ #include +#include +#include +#include + #include "memory/ArrowMemory.h" #include "memory/VeloxColumnarBatch.h" #include "velox/common/memory/Memory.h" @@ -95,4 +99,469 @@ std::shared_ptr VeloxColumnarBatchSerializer::deserialize(uint8_t return std::make_shared(result); } +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. +// +// Floating-point edge cases that DO NOT poison the column: +// - +/-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. +template +bool scanMinMax(const facebook::velox::FlatVector* flat, T& tLo, T& tHi, int64_t& nullCnt, bool& seen) { + static_assert(!std::is_same_v, "BOOLEAN must use scanBoolMinMax (FlatVector::rawValues unsupported)"); + 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; + continue; + } + 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. + continue; + } + } + if (floatingUnsupported) { + // NaN already poisoned min/max; skip bound updates but keep counting (nulls handled above). + continue; + } + if (!seen) { + tLo = v; + tHi = v; + seen = true; + } else { + if (v < tLo) + tLo = v; + if (v > tHi) + tHi = v; + } + } + return !floatingUnsupported; +} + +// BOOLEAN-specific scan: FlatVector::rawValues() is unsupported in Velox (bit-packed +// storage, no bool* accessor) and throws VeloxRuntimeError. Use valueAt(i)/isNullAt(i) +// instead. Semantics match scanMinMax: track lo/hi (false* flat, bool& tLo, bool& tHi, int64_t& nullCnt, bool& seen) { + const auto size = flat->size(); + for (vector_size_t i = 0; i < size; ++i) { + if (flat->isNullAt(i)) { + ++nullCnt; + continue; + } + bool v = flat->valueAt(i); + if (!seen) { + tLo = v; + tHi = v; + seen = true; + } else { + if (v < tLo) + tLo = v; + if (v > tHi) + tHi = v; + } + } + return true; +} + +// Null counter for non-flat encoding (Dictionary / Constant / Complex). Their rawValues path +// is undefined for stats compute; we still must report a real nullCount so JVM-side IsNull +// pruning (`statsFor(a).nullCount > 0`) does not incorrectly skip null-bearing partitions. +inline int64_t countNullsAny(const facebook::velox::BaseVector* vec) { + const auto size = vec->size(); + int64_t nullCnt = 0; + for (vector_size_t i = 0; i < size; ++i) { + if (vec->isNullAt(i)) { + ++nullCnt; + } + } + return nullCnt; +} + +} // namespace + +std::vector VeloxColumnarBatchSerializer::computeStats(RowVectorPtr rowVector) { + std::vector result; + const auto numCols = rowVector->childrenSize(); + result.resize(numCols); + for (column_index_t col = 0; col < numCols; ++col) { + auto& stats = result[col]; + auto child = rowVector->childAt(col); + if (child == nullptr) { + continue; + } + if (!child->isFlatEncoding()) { + // Non-flat (Dictionary / Constant / Complex): min/max stays unsupported, but we MUST + // still report a real nullCount. Spark IsNull pruning reads `statsFor(a).nullCount > 0` + // and a default-0 on a null-bearing dict-encoded column would incorrectly prune. + stats.nullCount = countNullsAny(child.get()); + continue; + } + bool seen = false; + int64_t nullCnt = 0; + bool supported = false; + switch (child->typeKind()) { + case TypeKind::BIGINT: { + auto* flat = child->asFlatVector(); + int64_t lo = 0, hi = 0; + supported = scanMinMax(flat, lo, hi, nullCnt, seen); + if (supported && seen) { + stats.hasLowerBound = true; + stats.hasUpperBound = true; + stats.lowerBound = variant(lo); + stats.upperBound = variant(hi); + } + break; + } + case TypeKind::INTEGER: { + auto* flat = child->asFlatVector(); + int32_t lo = 0, hi = 0; + supported = scanMinMax(flat, lo, hi, nullCnt, seen); + if (supported && seen) { + stats.hasLowerBound = true; + stats.hasUpperBound = true; + stats.lowerBound = variant(lo); + stats.upperBound = variant(hi); + } + break; + } + case TypeKind::SMALLINT: { + auto* flat = child->asFlatVector(); + int16_t lo = 0, hi = 0; + supported = scanMinMax(flat, lo, hi, nullCnt, seen); + if (supported && seen) { + stats.hasLowerBound = true; + stats.hasUpperBound = true; + stats.lowerBound = variant(lo); + stats.upperBound = variant(hi); + } + break; + } + case TypeKind::TINYINT: { + auto* flat = child->asFlatVector(); + int8_t lo = 0, hi = 0; + supported = scanMinMax(flat, lo, hi, nullCnt, seen); + if (supported && seen) { + stats.hasLowerBound = true; + stats.hasUpperBound = true; + stats.lowerBound = variant(lo); + stats.upperBound = variant(hi); + } + break; + } + case TypeKind::REAL: { + auto* flat = child->asFlatVector(); + float lo = 0.f, hi = 0.f; + supported = scanMinMax(flat, lo, hi, nullCnt, seen); + if (supported && seen) { + stats.hasLowerBound = true; + stats.hasUpperBound = true; + stats.lowerBound = variant(lo); + stats.upperBound = variant(hi); + } + break; + } + case TypeKind::DOUBLE: { + auto* flat = child->asFlatVector(); + double lo = 0.0, hi = 0.0; + supported = scanMinMax(flat, lo, hi, nullCnt, seen); + if (supported && seen) { + stats.hasLowerBound = true; + stats.hasUpperBound = true; + stats.lowerBound = variant(lo); + stats.upperBound = variant(hi); + } + break; + } + case TypeKind::BOOLEAN: { + auto* flat = child->asFlatVector(); + bool lo = false, hi = false; + supported = scanBoolMinMax(flat, lo, hi, nullCnt, seen); + if (supported && seen) { + stats.hasLowerBound = true; + stats.hasUpperBound = true; + stats.lowerBound = variant(lo); + stats.upperBound = variant(hi); + } + break; + } + case TypeKind::HUGEINT: { + // long-Decimal (precision > 18); marshaled as 16B LE int128 downstream. + auto* flat = child->asFlatVector(); + int128_t lo = 0, hi = 0; + supported = scanMinMax(flat, lo, hi, nullCnt, seen); + if (supported && seen) { + stats.hasLowerBound = true; + stats.hasUpperBound = true; + stats.lowerBound = variant(lo); + stats.upperBound = variant(hi); + } + break; + } + case TypeKind::TIMESTAMP: { + // Velox Timestamp has defaulted operator<=> (Timestamp.h) so scanMinMax compiles via the + // existing template. Wire emit converts via toMicros() to int64; Spark TimestampType / + // TimestampNTZType physical = Long microseconds. + auto* flat = child->asFlatVector(); + Timestamp lo, hi; + supported = scanMinMax(flat, lo, hi, nullCnt, seen); + if (supported && seen) { + stats.hasLowerBound = true; + stats.hasUpperBound = true; + stats.lowerBound = variant(lo); + stats.upperBound = variant(hi); + } + break; + } + case TypeKind::VARCHAR: { + // StringView::operator<=> uses memcmp -> unsigned byte ordering, matching Spark + // ByteArray.compareBinary. variant(std::string{sv}) heap-copies so post-computeStats + // lifetime is decoupled from the RowVector buffer. + // + // Truncate to 256B at the source so the JVM never sees > 256B (single source of truth). + // Lower bound: prefix is byte-wise <= original. Upper bound: prefix +1 carry on the + // rightmost byte to ensure encoded >= original; carry overflow on an all-0xFF prefix + // demotes supported=0. Mirrors the JVM-side encodeStringBounds. + constexpr size_t kStatsStringTruncateLen = 256; + auto* flat = child->asFlatVector(); + StringView lo, hi; + supported = scanMinMax(flat, lo, hi, nullCnt, seen); + if (supported && seen) { + const size_t loLen = std::min(static_cast(lo.size()), kStatsStringTruncateLen); + std::string loBytes(lo.data(), loLen); + const size_t hiSrcLen = static_cast(hi.size()); + std::string hiBytes(hi.data(), std::min(hiSrcLen, kStatsStringTruncateLen)); + bool hiOk = true; + if (hiSrcLen > kStatsStringTruncateLen) { + bool carryDone = false; + for (int i = static_cast(hiBytes.size()) - 1; i >= 0; --i) { + uint8_t b = static_cast(hiBytes[i]) + 1; + if (b != 0) { + hiBytes[i] = static_cast(b); + carryDone = true; + break; + } + hiBytes[i] = 0; + } + hiOk = carryDone; + } + if (hiOk) { + stats.hasLowerBound = true; + stats.hasUpperBound = true; + stats.lowerBound = variant(std::move(loBytes)); + stats.upperBound = variant(std::move(hiBytes)); + } else { + supported = false; + } + } + break; + } + default: + // Unsupported type -> hasLowerBound=hasUpperBound=false -> JVM buildFilter pass-through. + break; + } + stats.nullCount = nullCnt; + } + return result; +} + +std::vector VeloxColumnarBatchSerializer::framedSerializeWithStats( + const std::shared_ptr& batch) { + // Compute stats over the inbound rowVector BEFORE delegating to the append path (which may + // consume / mutate iterator state on subsequent calls). + auto rowVector = VeloxColumnarBatch::from(veloxPool_.get(), batch)->getRowVector(); + const uint32_t numRows = static_cast(rowVector->size()); + std::vector perCol = computeStats(rowVector); + const uint32_t numCols = static_cast(perCol.size()); + + // Marshal statsBlob (LE primitives via lambdas). + 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)); + // PartitionStatistics.count = numRows (vanilla gatherNullStats increments count for null + // rows too; subtracting nullCount inverts the IsNotNull predicate). + 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: { + // 16 LE bytes: int128 split into low/high uint64 halves, low first. JVM reconstructs + // via BigInteger from signed two's-complement big-endian byte array (reverse on read). + 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(); + float 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(); + double 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: { + // Spark Timestamp / TimestampNTZ physical = Long microseconds; share the JVM LongType + // 8B wire arm. Velox Timestamp::toMicros() floors toward -infinity. Floor on lo widens + // the prune interval downward (conservative) but floor on hi can shrink it and + // false-negative drop rows whose true ts has nanos % 1000 != 0. Fix: ceil hi by +1us + // when there is any sub-microsecond residue. + const auto& loTs = s.lowerBound.value(); + const auto& hiTs = s.upperBound.value(); + int64_t loMicros = loTs.toMicros(); + int64_t hiMicros = hiTs.toMicros(); + if (hiTs.getNanos() % 1000 != 0) { + hiMicros += 1; + } + pushU32(8); + pushI64LE(loMicros); + pushU32(8); + pushI64LE(hiMicros); + break; + } + case facebook::velox::TypeKind::VARCHAR: { + // Truncation already applied by computeStats (256B + carry); emit raw bytes with + // u32 LE length prefix. variant.value() returns owned std::string&. + 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; + } + } + } + const uint32_t statsLen = static_cast(statsBlob.size()); + + // Produce bytesBlob via the existing serializer path. + append(batch); + const int64_t bytesLen = maxSerializedSize(); + std::vector bytesBlob(bytesLen); + serializeTo(bytesBlob.data(), bytesLen); + + // Assemble: [magic(4) | statsLen(u32 LE) | statsBlob | bytesLen(u32 LE) | bytesBlob]. + std::vector framed; + framed.reserve(4 + 4 + statsLen + 4 + bytesLen); + framed.push_back(0xFE); + framed.push_back(0xCA); + framed.push_back(0x53); + framed.push_back(0x02); + framed.push_back(static_cast(statsLen & 0xFF)); + framed.push_back(static_cast((statsLen >> 8) & 0xFF)); + framed.push_back(static_cast((statsLen >> 16) & 0xFF)); + framed.push_back(static_cast((statsLen >> 24) & 0xFF)); + framed.insert(framed.end(), statsBlob.begin(), statsBlob.end()); + // Wire framing encodes bytesLen as u32 LE, so reject any single-batch payload that would + // overflow that field (>4GB). Pathological in practice (very wide schemas / huge string + // columns); fail fast here rather than silently truncate and corrupt the JVM-side parser. + GLUTEN_CHECK( + bytesLen >= 0 && bytesLen <= static_cast(std::numeric_limits::max()), + "Serialized batch size (" + std::to_string(bytesLen) + ") exceeds u32 framing limit"); + const uint32_t bytesLen32 = static_cast(bytesLen); + framed.push_back(static_cast(bytesLen32 & 0xFF)); + framed.push_back(static_cast((bytesLen32 >> 8) & 0xFF)); + framed.push_back(static_cast((bytesLen32 >> 16) & 0xFF)); + framed.push_back(static_cast((bytesLen32 >> 24) & 0xFF)); + framed.insert(framed.end(), bytesBlob.begin(), bytesBlob.end()); + return framed; +} + } // namespace gluten diff --git a/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.h b/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.h index f58da732810b..860c3ec5361f 100644 --- a/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.h +++ b/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.h @@ -22,9 +22,21 @@ #include "memory/ColumnarBatch.h" #include "operators/serializer/ColumnarBatchSerializer.h" #include "velox/serializers/PrestoSerializer.h" +#include "velox/type/Variant.h" namespace gluten { +// Per-column min/max + nullCount carrier consumed by the JVM cache stats marshaling. +// Unsupported / NaN-poisoned columns emit hasLowerBound=hasUpperBound=false, which the JVM +// buildFilter treats as pass-through. +struct ColumnStats { + bool hasLowerBound{false}; + bool hasUpperBound{false}; + facebook::velox::variant lowerBound; + facebook::velox::variant upperBound; + int64_t nullCount{0}; +}; + class VeloxColumnarBatchSerializer : public ColumnarBatchSerializer { public: VeloxColumnarBatchSerializer( @@ -40,6 +52,16 @@ class VeloxColumnarBatchSerializer : public ColumnarBatchSerializer { std::shared_ptr deserialize(uint8_t* data, int32_t size) override; + // Per-column min/max scan. Returns hasLowerBound/UpperBound=true for supported scalar types + // (integer / decimal / float / double / bool / timestamp / varchar); other types degrade to + // 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. + std::vector framedSerializeWithStats(const std::shared_ptr& batch) override; + protected: std::shared_ptr veloxPool_; std::unique_ptr arena_; diff --git a/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc b/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc index 35c18d8ec34f..bdd16d930ce8 100644 --- a/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc +++ b/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc @@ -28,6 +28,8 @@ #include +#include + using namespace facebook::velox; namespace gluten { @@ -75,4 +77,670 @@ TEST_F(VeloxColumnarBatchSerializerTest, serialize) { test::assertEqualVectors(vector, deserializedVector); } +// BIGINT FlatVector min/max scan: values [42, 7, 99, -3, 50] -> lo=-3, hi=99. +TEST_F(VeloxColumnarBatchSerializerTest, computeStatsBigintFlatVector) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + + std::vector children = { + makeFlatVector({42, 7, 99, -3, 50}), + }; + auto vector = makeRowVector(children); + auto batch = std::make_shared(vector); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + auto stats = serializer->computeStats(vector); + + ASSERT_EQ(stats.size(), 1u); + EXPECT_TRUE(stats[0].hasLowerBound); + EXPECT_TRUE(stats[0].hasUpperBound); + EXPECT_EQ(stats[0].lowerBound.value(), -3); + EXPECT_EQ(stats[0].upperBound.value(), 99); + 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 +// drop matching rows under min/max pruning). +TEST_F(VeloxColumnarBatchSerializerTest, computeStatsNaNRealPartitionPoisons) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + // (a) REAL FlatVector NO NaN -- min/max well-defined, should be supported. + { + std::vector children = { + makeFlatVector({1.5f, 2.5f, 0.5f, 3.5f}), + }; + auto vector = makeRowVector(children); + auto stats = serializer->computeStats(vector); + ASSERT_EQ(stats.size(), 1u); + EXPECT_TRUE(stats[0].hasLowerBound) << "REAL FlatVector w/o NaN must be supported"; + EXPECT_TRUE(stats[0].hasUpperBound); + } + + // (b) REAL FlatVector WITH NaN -- must fall back to unsupported. + { + const float nan = std::numeric_limits::quiet_NaN(); + std::vector children = { + makeFlatVector({1.5f, nan, 3.5f}), + }; + 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_FALSE(stats[0].hasUpperBound); + } +} + +// Float / double boundary values (+/-Inf, +/-0, subnormal) must NOT poison the column the way +// NaN does -- they are ordered under IEEE 754 < and so produce well-defined min/max bounds. +TEST_F(VeloxColumnarBatchSerializerTest, computeStatsFloatBoundaryValuesNotPoisoned) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + const float negInf = -std::numeric_limits::infinity(); + const float posInf = std::numeric_limits::infinity(); + const float subnormal = std::numeric_limits::denorm_min(); + + // (a) REAL: -Inf / +Inf / +0 / -0 / subnormal mixed -- must be supported with [-Inf, +Inf]. + { + std::vector children = { + makeFlatVector({negInf, +0.0f, -0.0f, subnormal, posInf}), + }; + auto vector = makeRowVector(children); + auto stats = serializer->computeStats(vector); + ASSERT_EQ(stats.size(), 1u); + EXPECT_TRUE(stats[0].hasLowerBound) << "boundary-value REAL column must be supported"; + EXPECT_TRUE(stats[0].hasUpperBound); + EXPECT_EQ(stats[0].lowerBound.value(), negInf); + EXPECT_EQ(stats[0].upperBound.value(), posInf); + } + + // (b) DOUBLE: same set, exercise the 8-byte path. + { + const double dNegInf = -std::numeric_limits::infinity(); + const double dPosInf = std::numeric_limits::infinity(); + const double dSubnormal = std::numeric_limits::denorm_min(); + std::vector children = { + makeFlatVector({dNegInf, +0.0, -0.0, dSubnormal, dPosInf}), + }; + auto vector = makeRowVector(children); + auto stats = serializer->computeStats(vector); + ASSERT_EQ(stats.size(), 1u); + EXPECT_TRUE(stats[0].hasLowerBound) << "boundary-value DOUBLE column must be supported"; + EXPECT_TRUE(stats[0].hasUpperBound); + EXPECT_EQ(stats[0].lowerBound.value(), dNegInf); + EXPECT_EQ(stats[0].upperBound.value(), dPosInf); + } +} + +// HUGEINT (int128) FlatVector for LongDecimal(20, 4): values [100, -50, 9999] +// -> lo=-50, hi=9999. +TEST_F(VeloxColumnarBatchSerializerTest, computeStatsHugeintDecimalFlatVector) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + std::vector children = { + makeFlatVector( + {static_cast(100), static_cast(-50), static_cast(9999)}, DECIMAL(20, 4)), + }; + auto vector = makeRowVector(children); + auto stats = serializer->computeStats(vector); + ASSERT_EQ(stats.size(), 1u); + EXPECT_TRUE(stats[0].hasLowerBound) << "HUGEINT (long Decimal P>=19) supported via 16B LE marshal"; + EXPECT_TRUE(stats[0].hasUpperBound); + EXPECT_EQ(stats[0].lowerBound.value(), static_cast(-50)); + EXPECT_EQ(stats[0].upperBound.value(), static_cast(9999)); +} + +// framedSerializeWithStats top-level layout: +// [ magic(4) = 0xFE 0xCA 0x53 0x02 | statsLen(u32 LE) | statsBlob | bytesLen(u32 LE) | bytesBlob ] +TEST_F(VeloxColumnarBatchSerializerTest, framedSerializeWithStatsTopLayout) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + std::vector children = { + makeFlatVector({1, 2, 3, 4, 5}), + }; + auto vector = makeRowVector(children); + auto batch = std::make_shared(vector); + + std::vector framed = serializer->framedSerializeWithStats(batch); + + // Top-level layout: at least magic(4) + statsLen(4) + bytesLen(4) = 12 bytes header. + ASSERT_GE(framed.size(), 12u); + + // Magic bytes 0xFE 0xCA 0x53 0x02. + EXPECT_EQ(framed[0], static_cast(0xFE)); + EXPECT_EQ(framed[1], static_cast(0xCA)); + EXPECT_EQ(framed[2], static_cast(0x53)); + EXPECT_EQ(framed[3], static_cast(0x02)); + + // statsLen LE int32 -- non-zero now that statsBlob is populated. The framing-only + // the layout shape (offsets / framing), not the numeric statsLen value. + uint32_t statsLen = static_cast(framed[4]) | (static_cast(framed[5]) << 8) | + (static_cast(framed[6]) << 16) | (static_cast(framed[7]) << 24); + + // bytesLen LE int32 immediately after empty statsBlob. + size_t bytesLenOffset = 8u + statsLen; + ASSERT_GE(framed.size(), bytesLenOffset + 4u); + uint32_t bytesLen = static_cast(framed[bytesLenOffset]) | + (static_cast(framed[bytesLenOffset + 1]) << 8) | + (static_cast(framed[bytesLenOffset + 2]) << 16) | + (static_cast(framed[bytesLenOffset + 3]) << 24); + EXPECT_GT(bytesLen, 0u) << "bytesBlob must contain serialized batch payload"; + EXPECT_EQ(framed.size(), bytesLenOffset + 4u + bytesLen) + << "framed total size must match magic + statsLen + statsBlob + bytesLen + bytesBlob"; +} + +// statsBlob layout (BIGINT 1-col): +// [ numCols u32 LE ] per col [ supported u8 | nullCount u32 | count u32 | +// sizeInBytes u64 | lowerLen u32 | lower bytes | upperLen u32 | upper bytes ] +TEST_F(VeloxColumnarBatchSerializerTest, statsBlobBigintLayout) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + std::vector children = { + makeFlatVector({42, 7, 99, -3, 50}), + }; + auto vector = makeRowVector(children); + auto batch = std::make_shared(vector); + + std::vector framed = serializer->framedSerializeWithStats(batch); + ASSERT_GE(framed.size(), 12u); + + // Skip magic(4) header; read statsLen LE. + uint32_t statsLen = static_cast(framed[4]) | (static_cast(framed[5]) << 8) | + (static_cast(framed[6]) << 16) | (static_cast(framed[7]) << 24); + ASSERT_GE(statsLen, 4u) << "statsBlob must contain at least numCols(uint32)"; + + // statsBlob starts at offset 8. + auto readU32LE = [&](size_t off) { + return static_cast(framed[off]) | (static_cast(framed[off + 1]) << 8) | + (static_cast(framed[off + 2]) << 16) | (static_cast(framed[off + 3]) << 24); + }; + auto readU64LE = [&](size_t off) { + uint64_t v = 0; + for (int i = 0; i < 8; ++i) { + v |= static_cast(framed[off + i]) << (8 * i); + } + return v; + }; + auto readI64LE = [&](size_t off) { return static_cast(readU64LE(off)); }; + + size_t off = 8; + uint32_t numCols = readU32LE(off); + off += 4; + EXPECT_EQ(numCols, 1u); + + // Per-col header. + uint8_t supported = framed[off]; + off += 1; + EXPECT_EQ(supported, 1u) << "BIGINT no-null col must be supported"; + + uint32_t nullCount = readU32LE(off); + off += 4; + EXPECT_EQ(nullCount, 0u); + + uint32_t count = readU32LE(off); + off += 4; + EXPECT_EQ(count, 5u) << "count = rowVector->size()"; + + uint64_t sizeInBytes = readU64LE(off); + off += 8; + EXPECT_EQ(sizeInBytes, 0u) << "sizeInBytes is a 0 placeholder"; + + // lowerBound: BIGINT -> 8 bytes int64 LE = -3. + uint32_t lowerLen = readU32LE(off); + off += 4; + EXPECT_EQ(lowerLen, 8u); + EXPECT_EQ(readI64LE(off), -3); + off += lowerLen; + + // upperBound: 8 bytes int64 LE = 99. + uint32_t upperLen = readU32LE(off); + off += 4; + EXPECT_EQ(upperLen, 8u); + EXPECT_EQ(readI64LE(off), 99); + off += upperLen; + + // statsBlob ends exactly at off (no trailing bytes within statsBlob). + EXPECT_EQ(off, 8u + statsLen) << "statsBlob content must exactly match statsLen"; +} + +// INTEGER (int32) FlatVector min/max scan; covers Spark IntegerType / DateType / +// YearMonthIntervalType which all map to Velox TypeKind::INTEGER. +TEST_F(VeloxColumnarBatchSerializerTest, computeStatsIntegerFlatVector) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + std::vector children = { + makeFlatVector({100, -2147483, 2147483, 0, 42}), + }; + auto vector = makeRowVector(children); + auto stats = serializer->computeStats(vector); + ASSERT_EQ(stats.size(), 1u); + EXPECT_TRUE(stats[0].hasLowerBound) << "INTEGER FlatVector must be supported"; + EXPECT_TRUE(stats[0].hasUpperBound); + EXPECT_EQ(stats[0].lowerBound.value(), -2147483); + EXPECT_EQ(stats[0].upperBound.value(), 2147483); + EXPECT_EQ(stats[0].nullCount, 0); +} + +// SMALLINT (int16) FlatVector min/max scan. +// Spark ShortType -> Velox TypeKind::SMALLINT. +TEST_F(VeloxColumnarBatchSerializerTest, computeStatsSmallintFlatVector) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + std::vector children = { + makeFlatVector({static_cast(-12345), static_cast(0), static_cast(12345)}), + }; + auto vector = makeRowVector(children); + auto stats = serializer->computeStats(vector); + ASSERT_EQ(stats.size(), 1u); + EXPECT_TRUE(stats[0].hasLowerBound) << "SMALLINT FlatVector must be supported"; + EXPECT_TRUE(stats[0].hasUpperBound); + EXPECT_EQ(stats[0].lowerBound.value(), static_cast(-12345)); + EXPECT_EQ(stats[0].upperBound.value(), static_cast(12345)); +} + +// TINYINT (int8) FlatVector min/max scan. +// Spark ByteType -> Velox TypeKind::TINYINT. +TEST_F(VeloxColumnarBatchSerializerTest, computeStatsTinyintFlatVector) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + std::vector children = { + makeFlatVector({static_cast(-128), static_cast(0), static_cast(127)}), + }; + auto vector = makeRowVector(children); + auto stats = serializer->computeStats(vector); + ASSERT_EQ(stats.size(), 1u); + EXPECT_TRUE(stats[0].hasLowerBound) << "TINYINT FlatVector must be supported"; + EXPECT_TRUE(stats[0].hasUpperBound); + EXPECT_EQ(stats[0].lowerBound.value(), static_cast(-128)); + EXPECT_EQ(stats[0].upperBound.value(), static_cast(127)); +} + +// TIMESTAMP FlatVector min/max scan; wire emit via Timestamp::toMicros() so it +// shares the JVM LongType 8B arm (Spark Timestamp / TimestampNTZ physical = Long us). +TEST_F(VeloxColumnarBatchSerializerTest, computeStatsTimestampFlatVector) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + // Three distinct timestamps in ascending order. + Timestamp t1(946684800, 0); + Timestamp t2(1704067200, 0); + Timestamp t3(1778198400, 0); + std::vector children = {makeFlatVector({t2, t1, t3})}; + auto vector = makeRowVector(children); + auto stats = serializer->computeStats(vector); + ASSERT_EQ(stats.size(), 1u); + EXPECT_TRUE(stats[0].hasLowerBound) << "TIMESTAMP FlatVector must be supported"; + EXPECT_TRUE(stats[0].hasUpperBound); + EXPECT_EQ(stats[0].lowerBound.value(), t1); + EXPECT_EQ(stats[0].upperBound.value(), t3); + EXPECT_EQ(stats[0].nullCount, 0); +} + +// TIMESTAMP wire: lowerLen=8 + LE int64 microseconds (JVM LongType arm). +TEST_F(VeloxColumnarBatchSerializerTest, framedSerializeWithStatsTimestamp) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + Timestamp t1(946684800, 0); // 2000-01-01 + Timestamp t3(1778198400, 0); // 2026-05-13 + std::vector children = {makeFlatVector({t1, t3})}; + auto rowVector = makeRowVector(children); + auto batch = std::make_shared(rowVector); + auto framed = serializer->framedSerializeWithStats(batch); + + // statsBlob starts at offset 8 (after magic + statsLen). + ASSERT_GE(framed.size(), 8u); + const uint8_t* p = framed.data() + 8; + // numCols + EXPECT_EQ(p[0], 1u); + EXPECT_EQ(p[1], 0u); + EXPECT_EQ(p[2], 0u); + EXPECT_EQ(p[3], 0u); + p += 4; + // supported + EXPECT_EQ(p[0], 1u) << "TIMESTAMP column should emit supported=1"; + p += 1; + // nullCount + count + sizeInBytes + p += 4 + 4 + 8; + // lowerLen must be 8 (microseconds Long), matching JVM LongType path. + uint32_t lowerLen = static_cast(p[0]) | (static_cast(p[1]) << 8) | + (static_cast(p[2]) << 16) | (static_cast(p[3]) << 24); + EXPECT_EQ(lowerLen, 8u) << "TIMESTAMP wire lowerLen must be 8B (microseconds)"; + p += 4; + // Read int64 LE microseconds; expect t1.toMicros(). + int64_t loMicros = 0; + for (int i = 0; i < 8; ++i) { + loMicros |= (static_cast(p[i]) << (8 * i)); + } + EXPECT_EQ(loMicros, t1.toMicros()) << "TIMESTAMP wire lowerBound microseconds mismatch"; +} + +// VARCHAR FlatVector min/max via StringView. memcmp byte-order matches Spark +// ByteArray.compareBinary; variant(std::string) owns the bytes after the scan. +TEST_F(VeloxColumnarBatchSerializerTest, computeStatsVarcharFlatVector) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + // Test inputs include high-bit byte (0xC2) to confirm unsigned comparison + // (signed cmp would put 0xC2 < 'a' = 0x61, wrong). + std::vector children = {makeFlatVector( + {StringView("apple"), + StringView("banana"), + StringView("\xc2\xa9" + "copy")})}; + auto vector = makeRowVector(children); + auto stats = serializer->computeStats(vector); + ASSERT_EQ(stats.size(), 1u); + EXPECT_TRUE(stats[0].hasLowerBound) << "VARCHAR FlatVector must be supported"; + EXPECT_TRUE(stats[0].hasUpperBound); + // Unsigned byte order: "apple"(0x61) < "banana"(0x62) < "\xc2\xa9copy"(0xc2). + EXPECT_EQ(stats[0].lowerBound.value(), std::string("apple")); + EXPECT_EQ( + stats[0].upperBound.value(), + std::string("\xc2\xa9" + "copy")); + EXPECT_EQ(stats[0].nullCount, 0); +} + +// VARCHAR wire: lowerLen u32 LE + raw UTF-8 bytes. +TEST_F(VeloxColumnarBatchSerializerTest, framedSerializeWithStatsVarchar) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + std::vector children = {makeFlatVector({StringView("apple"), StringView("banana")})}; + auto rowVector = makeRowVector(children); + auto batch = std::make_shared(rowVector); + auto framed = serializer->framedSerializeWithStats(batch); + + // Skip MAGIC (4) + statsLen (4); statsBlob starts at offset 8. + ASSERT_GE(framed.size(), 8u); + const uint8_t* p = framed.data() + 8; + // numCols = 1 + EXPECT_EQ(p[0], 1u); + p += 4; + // supported = 1 + EXPECT_EQ(p[0], 1u) << "VARCHAR column should emit supported=1"; + p += 1; + // skip nullCount + count + sizeInBytes (4 + 4 + 8) + p += 4 + 4 + 8; + // lowerLen = 5 (apple) + uint32_t lowerLen = static_cast(p[0]) | (static_cast(p[1]) << 8) | + (static_cast(p[2]) << 16) | (static_cast(p[3]) << 24); + EXPECT_EQ(lowerLen, 5u) << "VARCHAR lowerLen should be 5 for 'apple'"; + p += 4; + std::string lower(reinterpret_cast(p), 5); + EXPECT_EQ(lower, "apple") << "VARCHAR lowerBound bytes mismatch"; + p += 5; + // upperLen = 6 (banana) + uint32_t upperLen = static_cast(p[0]) | (static_cast(p[1]) << 8) | + (static_cast(p[2]) << 16) | (static_cast(p[3]) << 24); + EXPECT_EQ(upperLen, 6u) << "VARCHAR upperLen should be 6 for 'banana'"; + p += 4; + std::string upper(reinterpret_cast(p), 6); + EXPECT_EQ(upper, "banana") << "VARCHAR upperBound bytes mismatch"; +} + +// TIMESTAMP with sub-us nanos must conservatively widen: floor(lo), ceil(hi). +// Naive toMicros() floors both, which would shrink the interval and false-negative +// drop rows whose true ts has nanos % 1000 != 0. +TEST_F(VeloxColumnarBatchSerializerTest, timestampNanosCeilUpperFloorLower) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + // lo nanos=500 -> floor lower = s*1e6. + // hi nanos=999'500 -> ceil upper = s*1e6 + 1000 (raw toMicros gives 999 < true). + const int64_t s = 1704067200; // 2024-01-01T00:00:00Z + Timestamp lo(s, 500); + Timestamp hi(s, 999'500); + std::vector children = {makeFlatVector({lo, hi})}; + auto rowVector = makeRowVector(children); + auto batch = std::make_shared(rowVector); + auto framed = serializer->framedSerializeWithStats(batch); + + ASSERT_GE(framed.size(), 8u); + const uint8_t* p = framed.data() + 8; // skip MAGIC + statsLen + EXPECT_EQ(p[0], 1u); // numCols=1 + p += 4; + EXPECT_EQ(p[0], 1u) << "TIMESTAMP nanos column should still emit supported=1"; + p += 1 + 4 + 4 + 8; // skip supported + nullCount + count + sizeInBytes + + // lower: lowerLen=8, payload = floor lo = s*1e6 + uint32_t lowerLen = static_cast(p[0]) | (static_cast(p[1]) << 8) | + (static_cast(p[2]) << 16) | (static_cast(p[3]) << 24); + EXPECT_EQ(lowerLen, 8u); + p += 4; + int64_t loMicros = 0; + for (int i = 0; i < 8; ++i) { + loMicros |= (static_cast(p[i]) << (8 * i)); + } + const int64_t expectedLo = s * 1'000'000LL; + EXPECT_EQ(loMicros, expectedLo) << "lower must be floor(lo). Got " << loMicros << ", want " << expectedLo; + p += 8; + + // upper: upperLen=8, payload = ceil hi = s*1e6 + 1000 (NOT s*1e6+999) + uint32_t upperLen = static_cast(p[0]) | (static_cast(p[1]) << 8) | + (static_cast(p[2]) << 16) | (static_cast(p[3]) << 24); + EXPECT_EQ(upperLen, 8u); + p += 4; + int64_t hiMicros = 0; + for (int i = 0; i < 8; ++i) { + hiMicros |= (static_cast(p[i]) << (8 * i)); + } + const int64_t expectedHi = s * 1'000'000LL + 1000; // ceil to next us + EXPECT_EQ(hiMicros, expectedHi) << "upper must be ceil(hi) when nanos%1000 != 0. Got " << hiMicros << ", want " + << expectedHi; +} + +// When nanos % 1000 == 0 (exact us), no ceil adjustment needed. +TEST_F(VeloxColumnarBatchSerializerTest, timestampExactMicrosNoCeil) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + const int64_t s = 1704067200; + Timestamp lo(s, 0); + Timestamp hi(s, 1'000); // exactly 1 us + std::vector children = {makeFlatVector({lo, hi})}; + auto rowVector = makeRowVector(children); + auto batch = std::make_shared(rowVector); + auto framed = serializer->framedSerializeWithStats(batch); + + const uint8_t* p = framed.data() + 8; + p += 4 + 1 + 4 + 4 + 8 + 4; // numCols + supported + n/c/sz + lowerLen + int64_t loMicros = 0; + for (int i = 0; i < 8; ++i) + loMicros |= (static_cast(p[i]) << (8 * i)); + EXPECT_EQ(loMicros, s * 1'000'000LL); + p += 8 + 4; // lower bytes + upperLen + int64_t hiMicros = 0; + for (int i = 0; i < 8; ++i) + hiMicros |= (static_cast(p[i]) << (8 * i)); + EXPECT_EQ(hiMicros, s * 1'000'000LL + 1) << "upper exact us should NOT ceil-adjust beyond toMicros"; +} + +// VARCHAR cpp truncates to 256B at source (single source of truth). +TEST_F(VeloxColumnarBatchSerializerTest, varcharTruncatesAt256Bytes) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + // 300-byte 'a' (lo) + 300-byte 'm' (hi); after truncate+carry expect + // lo = 256B 'a' (prefix <= original) + // hi = 255B 'm' + 1B 'n' (+1 carry on last byte) + std::string longA(300, 'a'); + std::string longM(300, 'm'); + // makeFlatVector with two rows so one becomes lo, the other hi. + std::vector children = { + makeFlatVector({StringView(longA.data(), longA.size()), StringView(longM.data(), longM.size())})}; + auto rowVector = makeRowVector(children); + auto batch = std::make_shared(rowVector); + auto framed = serializer->framedSerializeWithStats(batch); + + ASSERT_GE(framed.size(), 8u); + const uint8_t* p = framed.data() + 8; + EXPECT_EQ(p[0], 1u); // numCols + p += 4; + EXPECT_EQ(p[0], 1u) << "column should still be supported=1 after cpp truncate"; + p += 1 + 4 + 4 + 8; // skip supported + nullCount + count + sizeInBytes + + // lower: 256B prefix of 'a' + uint32_t lowerLen = static_cast(p[0]) | (static_cast(p[1]) << 8) | + (static_cast(p[2]) << 16) | (static_cast(p[3]) << 24); + EXPECT_EQ(lowerLen, 256u) << "cpp must truncate lower to 256B"; + p += 4; + for (uint32_t i = 0; i < 256u; ++i) { + if (p[i] != 'a') { + ADD_FAILURE() << "lower byte " << i << " = " << static_cast(p[i]) << ", expected 'a'"; + break; + } + } + p += 256; + + // upper: 255B 'm' + 1B 'n' (carry) + uint32_t upperLen = static_cast(p[0]) | (static_cast(p[1]) << 8) | + (static_cast(p[2]) << 16) | (static_cast(p[3]) << 24); + EXPECT_EQ(upperLen, 256u) << "cpp must truncate upper to 256B"; + p += 4; + for (uint32_t i = 0; i < 255u; ++i) { + if (p[i] != 'm') { + ADD_FAILURE() << "upper byte " << i << " = " << static_cast(p[i]) << ", expected 'm'"; + break; + } + } + EXPECT_EQ(p[255], 'n') << "upper last byte should be 'm'+1='n' (carry), got " << static_cast(p[255]); +} + +// All-0xFF upper prefix: carry overflows -> demote supported=0. +TEST_F(VeloxColumnarBatchSerializerTest, varcharCarryOverflowDemotesUnsupported) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + std::string longLo(10, '\xff'); + std::string longHi(300, '\xff'); + std::vector children = { + makeFlatVector({StringView(longLo.data(), longLo.size()), StringView(longHi.data(), longHi.size())})}; + auto rowVector = makeRowVector(children); + auto batch = std::make_shared(rowVector); + auto framed = serializer->framedSerializeWithStats(batch); + + const uint8_t* p = framed.data() + 8; + p += 4; // numCols + EXPECT_EQ(p[0], 0u) << "300x 0xFF upper must demote to supported=0 (carry overflow)"; +} + +// Regression: short string (<= 256B) must round-trip unchanged. +TEST_F(VeloxColumnarBatchSerializerTest, varcharShortStringRoundTripsIntact) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + std::vector children = {makeFlatVector({StringView("apple"), StringView("banana")})}; + auto rowVector = makeRowVector(children); + auto batch = std::make_shared(rowVector); + auto framed = serializer->framedSerializeWithStats(batch); + + const uint8_t* p = framed.data() + 8; + p += 4 + 1 + 4 + 4 + 8; // numCols + supported + n/c/sz + uint32_t lowerLen = static_cast(p[0]) | (static_cast(p[1]) << 8) | + (static_cast(p[2]) << 16) | (static_cast(p[3]) << 24); + EXPECT_EQ(lowerLen, 5u) << "short lower bytes intact (no truncate)"; + p += 4; + EXPECT_EQ(std::string(reinterpret_cast(p), 5), "apple"); + p += 5; + uint32_t upperLen = static_cast(p[0]) | (static_cast(p[1]) << 8) | + (static_cast(p[2]) << 16) | (static_cast(p[3]) << 24); + EXPECT_EQ(upperLen, 6u) << "short upper bytes intact (no truncate, no carry)"; + p += 4; + EXPECT_EQ(std::string(reinterpret_cast(p), 6), "banana"); +} + +// BOOLEAN FlatVector min/max scan must use valueAt(i)/isNullAt(i): +// FlatVector::rawValues() throws VeloxRuntimeError (bit-packed +// storage), so without scanBoolMinMax materializing a cached +// BooleanType column would hard-fail. Verify nullable bool batch +// produces (lo=false, hi=true, nullCount=2) instead of throwing. +TEST_F(VeloxColumnarBatchSerializerTest, computeStatsBooleanFlatVectorWithNulls) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + std::vector children = { + makeNullableFlatVector({true, std::nullopt, false, std::nullopt, true}), + }; + auto vector = makeRowVector(children); + + std::vector stats; + ASSERT_NO_THROW(stats = serializer->computeStats(vector)) + << "Boolean scan must NOT throw (Velox FlatVector::rawValues unsupported)"; + ASSERT_EQ(stats.size(), 1u); + EXPECT_TRUE(stats[0].hasLowerBound) << "Bool with non-null values must be supported"; + EXPECT_TRUE(stats[0].hasUpperBound); + EXPECT_EQ(stats[0].lowerBound.value(), false); + EXPECT_EQ(stats[0].upperBound.value(), true); + 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. +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. + 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)"; +} + +// Non-flat encoding (Dictionary / Constant / Complex) must still +// report a real nullCount. A null-bearing dict-encoded column +// reporting nullCount=0 would be advertised as having no nulls and +// incorrectly pruned by `col IS NULL`. +TEST_F(VeloxColumnarBatchSerializerTest, computeStatsDictEncodedNullCountReported) { + auto* arrowPool = getDefaultMemoryManager()->defaultArrowMemoryPool(); + auto serializer = std::make_shared(arrowPool, pool_, nullptr); + + // Base flat: [10, 20, 30]. Dictionary indices [0, 1, 0, 2, 1] but with nulls at + // positions 1 and 3 of the wrapping vector. Result has 5 rows, 2 nulls. + auto base = makeFlatVector({10, 20, 30}); + vector_size_t outSize = 5; + BufferPtr indices = AlignedBuffer::allocate(outSize, pool_.get()); + auto* rawIndices = indices->asMutable(); + rawIndices[0] = 0; + rawIndices[1] = 1; + rawIndices[2] = 0; + rawIndices[3] = 2; + rawIndices[4] = 1; + + BufferPtr nulls = AlignedBuffer::allocate(outSize, pool_.get()); + auto* rawNulls = nulls->asMutable(); + bits::fillBits(rawNulls, 0, outSize, true); + bits::setNull(rawNulls, 1, true); + bits::setNull(rawNulls, 3, true); + + auto dictVec = BaseVector::wrapInDictionary(nulls, indices, outSize, base); + std::vector children = {dictVec}; + auto vector = makeRowVector(children); + + auto stats = serializer->computeStats(vector); + ASSERT_EQ(stats.size(), 1u); + EXPECT_FALSE(stats[0].hasLowerBound) << "Dictionary encoding -> min/max unsupported"; + EXPECT_FALSE(stats[0].hasUpperBound); + EXPECT_EQ(stats[0].nullCount, 2) + << "Dict-encoded vector with 2 nulls must report nullCount=2 (IsNull prune correctness)"; +} + } // namespace gluten diff --git a/docs/Configuration.md b/docs/Configuration.md index 294e6c010ff0..9a7cc8af8c7e 100644 --- a/docs/Configuration.md +++ b/docs/Configuration.md @@ -20,132 +20,133 @@ nav_order: 15 ## Gluten configurations -| Key | Default | Description | -|--------------------------------------------------------------------|-------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| spark.gluten.costModel | 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 | 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 | 86400 | Expired time of execution with resource relation has cached. | -| spark.gluten.expression.blacklist | <undefined> | A black list of expression to skip transform, multiple values separated by commas. | -| spark.gluten.loadLibFromJar | false | Whether to load shared libraries from jars. | -| spark.gluten.loadLibOS | <undefined> | The shared library loader's OS name. | -| spark.gluten.loadLibOSVersion | <undefined> | The shared library loader's OS version. | -| spark.gluten.memory.isolation | 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 | 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 | 8MB | Block size of native reservation listener reserve memory from Spark. | -| spark.gluten.numTaskSlotsPerExecutor | -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 | <undefined> | -| spark.gluten.soft-affinity.duplicateReading.maxCacheItems | 10000 | Enable Soft Affinity duplicate reading detection | -| spark.gluten.soft-affinity.duplicateReadingDetect.enabled | false | If true, Enable Soft Affinity duplicate reading detection | -| spark.gluten.soft-affinity.enabled | false | Whether to enable Soft Affinity scheduling. | -| spark.gluten.soft-affinity.min.target-hosts | 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 | 2 | Calculate the number of the replications for scheduling to the target executors per file | -| spark.gluten.sql.adaptive.costEvaluator.enabled | 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 | 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 | true | Config to enable BroadcastNestedLoopJoinExecTransformer. | -| spark.gluten.sql.cacheWholeStageTransformerContext | 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 | true | Config to enable CartesianProductExecTransformer. | -| spark.gluten.sql.collapseGetJsonObject.enabled | false | Collapse nested get_json_object functions as one for optimization. | -| spark.gluten.sql.columnar.appendData | true | Enable or disable columnar v2 command append data. | -| spark.gluten.sql.columnar.arrowUdf | true | Enable or disable columnar arrow udf. | -| spark.gluten.sql.columnar.batchscan | true | Enable or disable columnar batchscan. | -| spark.gluten.sql.columnar.broadcastExchange | true | Enable or disable columnar broadcastExchange. | -| spark.gluten.sql.columnar.broadcastJoin | true | Enable or disable columnar broadcastJoin. | -| spark.gluten.sql.columnar.cast.avg | true | -| spark.gluten.sql.columnar.coalesce | true | Enable or disable columnar coalesce. | -| spark.gluten.sql.columnar.collectLimit | true | Enable or disable columnar collectLimit. | -| spark.gluten.sql.columnar.collectTail | true | Enable or disable columnar collectTail. | -| spark.gluten.sql.columnar.enableNestedColumnPruningInHiveTableScan | true | Enable or disable nested column pruning in hivetablescan. | -| spark.gluten.sql.columnar.enableVanillaVectorizedReaders | true | Enable or disable vanilla vectorized scan. | -| spark.gluten.sql.columnar.executor.libpath || The gluten executor library path. | -| spark.gluten.sql.columnar.expand | true | Enable or disable columnar expand. | -| spark.gluten.sql.columnar.fallback.expressions.threshold | 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 | true | When true, the fallback policy ignores the RowToColumnar when counting fallback number. | -| spark.gluten.sql.columnar.fallback.preferColumnar | 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 | true | Enable or disable columnar filescan. | -| spark.gluten.sql.columnar.filter | true | Enable or disable columnar filter. | -| spark.gluten.sql.columnar.force.hashagg | true | Whether to force to use gluten's hash agg for replacing vanilla spark's sort agg. | -| spark.gluten.sql.columnar.forceShuffledHashJoin | true | -| spark.gluten.sql.columnar.generate | true | -| spark.gluten.sql.columnar.hashagg | true | Enable or disable columnar hashagg. | -| spark.gluten.sql.columnar.hivetablescan | true | Enable or disable columnar hivetablescan. | -| spark.gluten.sql.columnar.libname | gluten | The gluten library name. | -| spark.gluten.sql.columnar.libpath || The gluten library path. | -| spark.gluten.sql.columnar.limit | true | -| spark.gluten.sql.columnar.maxBatchSize | 4096 | -| spark.gluten.sql.columnar.overwriteByExpression | true | Enable or disable columnar v2 command overwrite by expression. | -| spark.gluten.sql.columnar.overwritePartitionsDynamic | true | Enable or disable columnar v2 command overwrite partitions dynamic. | -| spark.gluten.sql.columnar.parquet.write.blockSize | 128MB | -| spark.gluten.sql.columnar.partial.generate | true | Evaluates the non-offload-able HiveUDTF using vanilla Spark generator | -| spark.gluten.sql.columnar.partial.project | 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 | 12 | Fallback to row operators if there are several continuous joins. | -| spark.gluten.sql.columnar.physicalJoinOptimizationOutputSize | 52 | Fallback to row operators if there are several continuous joins and matched output size. | -| spark.gluten.sql.columnar.physicalJoinOptimizeEnable | false | Enable or disable columnar physicalJoinOptimize. | -| spark.gluten.sql.columnar.preferStreamingAggregate | 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 | true | Enable or disable columnar project. | -| spark.gluten.sql.columnar.project.collapse | true | Combines two columnar project operators into one and perform alias substitution | -| spark.gluten.sql.columnar.query.fallback.threshold | -1 | The threshold for whether query will fall back by counting the number of ColumnarToRow & vanilla leaf node. | -| spark.gluten.sql.columnar.range | true | Enable or disable columnar range. | -| spark.gluten.sql.columnar.replaceData | true | Enable or disable columnar v2 command replace data. | -| spark.gluten.sql.columnar.scanOnly | false | When enabled, only scan and the filter after scan will be offloaded to native. | -| spark.gluten.sql.columnar.shuffle | true | Enable or disable columnar shuffle. | -| spark.gluten.sql.columnar.shuffle.celeborn.fallback.enabled | true | If enabled, fall back to ColumnarShuffleManager when celeborn service is unavailable.Otherwise, throw an exception. | -| spark.gluten.sql.columnar.shuffle.celeborn.useRssSort | 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 | <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 | <undefined> | -| spark.gluten.sql.columnar.shuffle.compression.threshold | 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 | false | Enable dictionary in hash-based shuffle. | -| spark.gluten.sql.columnar.shuffle.merge.threshold | 0.25 | -| spark.gluten.sql.columnar.shuffle.readerBufferSize | 1MB | Buffer size in bytes for shuffle reader reading input stream from local or remote. | -| spark.gluten.sql.columnar.shuffle.realloc.threshold | 0.25 | -| spark.gluten.sql.columnar.shuffle.sort.columns.threshold | 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 | 1MB | Buffer size in bytes for sort-based shuffle reader deserializing raw input to columnar batch. | -| spark.gluten.sql.columnar.shuffle.sort.partitions.threshold | 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 | 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 | true | Enable or disable columnar shuffledHashJoin. | -| spark.gluten.sql.columnar.shuffledHashJoin.optimizeBuildSide | true | Whether to allow Gluten to choose an optimal build side for shuffled hash join. | -| spark.gluten.sql.columnar.smallFileThreshold | 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 | true | Enable or disable columnar sort. | -| spark.gluten.sql.columnar.sortMergeJoin | true | Enable or disable columnar sortMergeJoin. This should be set with preferSortMergeJoin=false. | -| spark.gluten.sql.columnar.tableCache | false | Enable or disable columnar table cache. | -| spark.gluten.sql.columnar.takeOrderedAndProject | true | -| spark.gluten.sql.columnar.union | true | Enable or disable columnar union. | -| spark.gluten.sql.columnar.wholeStage.fallback.threshold | -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 | true | Enable or disable columnar window. | -| spark.gluten.sql.columnar.window.group.limit | true | Enable or disable columnar window group limit. | -| spark.gluten.sql.columnar.writeToDataSourceV2 | true | Enable or disable columnar v2 command write to data source v2. | -| spark.gluten.sql.columnarSampleEnabled | false | Disable or enable columnar sample. | -| spark.gluten.sql.columnarToRowMemoryThreshold | 64MB | -| spark.gluten.sql.countDistinctWithoutExpand | 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 | true | Do extended nested column pruning for cases ignored by vanilla Spark. | -| spark.gluten.sql.fallbackRegexpExpressions | 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 | false | If enabled, Gluten will not offload scan when unexpected metadata is detected. | -| spark.gluten.sql.fallbackUnexpectedMetadataParquet.limit | 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 | 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 | false | When true, Gluten will inject native plan tree to Spark's explain output. | -| spark.gluten.sql.mergeTwoPhasesAggregate.enabled | true | Whether to merge two phases aggregate if there are no other operators between them. | -| spark.gluten.sql.native.arrow.reader.enabled | false | This is config to specify whether to enable the native columnar csv reader | -| spark.gluten.sql.native.bloomFilter | true | -| spark.gluten.sql.native.hive.writer.enabled | 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 | true | -| spark.gluten.sql.native.parquet.write.blockRows | 100000000 | -| spark.gluten.sql.native.union | false | Enable or disable native union where computation is completely offloaded to backend. | -| spark.gluten.sql.native.writeColumnMetadataExclusionList | 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 | <undefined> | This is config to specify whether to enable the native columnar parquet/orc writer | -| spark.gluten.sql.orc.charType.scan.fallback.enabled | true | Force fallback for orc char type scan. | -| spark.gluten.sql.pushAggregateThroughJoin.enabled | 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 | 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 | true | When true, Gluten will remove the vanilla Spark V1Writes added sort and project for velox backend. | -| spark.gluten.sql.rewrite.dateTimestampComparison | 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 | 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 | and,or | Flatten nested functions as one for optimization. | -| spark.gluten.sql.text.input.empty.as.default | false | treat empty fields in CSV input as default values. | -| spark.gluten.sql.text.input.max.block.size | 8KB | the max block size for text input rows | -| spark.gluten.sql.validation.printStackOnFailure | false | -| spark.gluten.storage.hdfsViewfs.enabled | false | If enabled, gluten will convert the viewfs path to hdfs path in scala side | -| spark.gluten.supported.hive.udfs || Supported hive udf names. | -| spark.gluten.supported.python.udfs || Supported python udf names. | -| spark.gluten.supported.scala.udfs || Supported scala udf names. | -| spark.gluten.ui.enabled | true | Whether to enable the gluten web UI, If true, attach the gluten UI page to the Spark web UI. | +| Key | Default | Description | +|---------------------------------------------------------------------|-------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| spark.gluten.costModel | 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 | 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 | 86400 | Expired time of execution with resource relation has cached. | +| spark.gluten.expression.blacklist | <undefined> | A black list of expression to skip transform, multiple values separated by commas. | +| spark.gluten.loadLibFromJar | false | Whether to load shared libraries from jars. | +| spark.gluten.loadLibOS | <undefined> | The shared library loader's OS name. | +| spark.gluten.loadLibOSVersion | <undefined> | The shared library loader's OS version. | +| spark.gluten.memory.isolation | 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 | 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 | 8MB | Block size of native reservation listener reserve memory from Spark. | +| spark.gluten.numTaskSlotsPerExecutor | -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 | <undefined> | +| spark.gluten.soft-affinity.duplicateReading.maxCacheItems | 10000 | Enable Soft Affinity duplicate reading detection | +| spark.gluten.soft-affinity.duplicateReadingDetect.enabled | false | If true, Enable Soft Affinity duplicate reading detection | +| spark.gluten.soft-affinity.enabled | false | Whether to enable Soft Affinity scheduling. | +| spark.gluten.soft-affinity.min.target-hosts | 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 | 2 | Calculate the number of the replications for scheduling to the target executors per file | +| spark.gluten.sql.adaptive.costEvaluator.enabled | 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 | 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 | true | Config to enable BroadcastNestedLoopJoinExecTransformer. | +| spark.gluten.sql.cacheWholeStageTransformerContext | 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 | true | Config to enable CartesianProductExecTransformer. | +| spark.gluten.sql.collapseGetJsonObject.enabled | false | Collapse nested get_json_object functions as one for optimization. | +| spark.gluten.sql.columnar.appendData | true | Enable or disable columnar v2 command append data. | +| spark.gluten.sql.columnar.arrowUdf | true | Enable or disable columnar arrow udf. | +| spark.gluten.sql.columnar.batchscan | true | Enable or disable columnar batchscan. | +| spark.gluten.sql.columnar.broadcastExchange | true | Enable or disable columnar broadcastExchange. | +| spark.gluten.sql.columnar.broadcastJoin | true | Enable or disable columnar broadcastJoin. | +| spark.gluten.sql.columnar.cast.avg | true | +| spark.gluten.sql.columnar.coalesce | true | Enable or disable columnar coalesce. | +| spark.gluten.sql.columnar.collectLimit | true | Enable or disable columnar collectLimit. | +| spark.gluten.sql.columnar.collectTail | true | Enable or disable columnar collectTail. | +| spark.gluten.sql.columnar.enableNestedColumnPruningInHiveTableScan | true | Enable or disable nested column pruning in hivetablescan. | +| spark.gluten.sql.columnar.enableVanillaVectorizedReaders | true | Enable or disable vanilla vectorized scan. | +| spark.gluten.sql.columnar.executor.libpath || The gluten executor library path. | +| spark.gluten.sql.columnar.expand | true | Enable or disable columnar expand. | +| spark.gluten.sql.columnar.fallback.expressions.threshold | 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 | true | When true, the fallback policy ignores the RowToColumnar when counting fallback number. | +| spark.gluten.sql.columnar.fallback.preferColumnar | 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 | true | Enable or disable columnar filescan. | +| spark.gluten.sql.columnar.filter | true | Enable or disable columnar filter. | +| spark.gluten.sql.columnar.force.hashagg | true | Whether to force to use gluten's hash agg for replacing vanilla spark's sort agg. | +| spark.gluten.sql.columnar.forceShuffledHashJoin | true | +| spark.gluten.sql.columnar.generate | true | +| spark.gluten.sql.columnar.hashagg | true | Enable or disable columnar hashagg. | +| spark.gluten.sql.columnar.hivetablescan | true | Enable or disable columnar hivetablescan. | +| spark.gluten.sql.columnar.libname | gluten | The gluten library name. | +| spark.gluten.sql.columnar.libpath || The gluten library path. | +| spark.gluten.sql.columnar.limit | true | +| spark.gluten.sql.columnar.maxBatchSize | 4096 | +| spark.gluten.sql.columnar.overwriteByExpression | true | Enable or disable columnar v2 command overwrite by expression. | +| spark.gluten.sql.columnar.overwritePartitionsDynamic | true | Enable or disable columnar v2 command overwrite partitions dynamic. | +| spark.gluten.sql.columnar.parquet.write.blockSize | 128MB | +| spark.gluten.sql.columnar.partial.generate | true | Evaluates the non-offload-able HiveUDTF using vanilla Spark generator | +| spark.gluten.sql.columnar.partial.project | 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 | 12 | Fallback to row operators if there are several continuous joins. | +| spark.gluten.sql.columnar.physicalJoinOptimizationOutputSize | 52 | Fallback to row operators if there are several continuous joins and matched output size. | +| spark.gluten.sql.columnar.physicalJoinOptimizeEnable | false | Enable or disable columnar physicalJoinOptimize. | +| spark.gluten.sql.columnar.preferStreamingAggregate | 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 | true | Enable or disable columnar project. | +| spark.gluten.sql.columnar.project.collapse | true | Combines two columnar project operators into one and perform alias substitution | +| spark.gluten.sql.columnar.query.fallback.threshold | -1 | The threshold for whether query will fall back by counting the number of ColumnarToRow & vanilla leaf node. | +| spark.gluten.sql.columnar.range | true | Enable or disable columnar range. | +| spark.gluten.sql.columnar.replaceData | true | Enable or disable columnar v2 command replace data. | +| spark.gluten.sql.columnar.scanOnly | false | When enabled, only scan and the filter after scan will be offloaded to native. | +| spark.gluten.sql.columnar.shuffle | true | Enable or disable columnar shuffle. | +| spark.gluten.sql.columnar.shuffle.celeborn.fallback.enabled | true | If enabled, fall back to ColumnarShuffleManager when celeborn service is unavailable.Otherwise, throw an exception. | +| spark.gluten.sql.columnar.shuffle.celeborn.useRssSort | 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 | <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 | <undefined> | +| spark.gluten.sql.columnar.shuffle.compression.threshold | 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 | false | Enable dictionary in hash-based shuffle. | +| spark.gluten.sql.columnar.shuffle.merge.threshold | 0.25 | +| spark.gluten.sql.columnar.shuffle.readerBufferSize | 1MB | Buffer size in bytes for shuffle reader reading input stream from local or remote. | +| spark.gluten.sql.columnar.shuffle.realloc.threshold | 0.25 | +| spark.gluten.sql.columnar.shuffle.sort.columns.threshold | 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 | 1MB | Buffer size in bytes for sort-based shuffle reader deserializing raw input to columnar batch. | +| spark.gluten.sql.columnar.shuffle.sort.partitions.threshold | 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 | 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 | true | Enable or disable columnar shuffledHashJoin. | +| spark.gluten.sql.columnar.shuffledHashJoin.optimizeBuildSide | true | Whether to allow Gluten to choose an optimal build side for shuffled hash join. | +| spark.gluten.sql.columnar.smallFileThreshold | 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 | true | Enable or disable columnar sort. | +| spark.gluten.sql.columnar.sortMergeJoin | true | Enable or disable columnar sortMergeJoin. This should be set with preferSortMergeJoin=false. | +| spark.gluten.sql.columnar.tableCache | false | Enable or disable columnar table cache. | +| spark.gluten.sql.columnar.tableCache.partitionStats.enabled | 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.takeOrderedAndProject | true | +| spark.gluten.sql.columnar.union | true | Enable or disable columnar union. | +| spark.gluten.sql.columnar.wholeStage.fallback.threshold | -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 | true | Enable or disable columnar window. | +| spark.gluten.sql.columnar.window.group.limit | true | Enable or disable columnar window group limit. | +| spark.gluten.sql.columnar.writeToDataSourceV2 | true | Enable or disable columnar v2 command write to data source v2. | +| spark.gluten.sql.columnarSampleEnabled | false | Disable or enable columnar sample. | +| spark.gluten.sql.columnarToRowMemoryThreshold | 64MB | +| spark.gluten.sql.countDistinctWithoutExpand | 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 | true | Do extended nested column pruning for cases ignored by vanilla Spark. | +| spark.gluten.sql.fallbackRegexpExpressions | 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 | false | If enabled, Gluten will not offload scan when unexpected metadata is detected. | +| spark.gluten.sql.fallbackUnexpectedMetadataParquet.limit | 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 | 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 | false | When true, Gluten will inject native plan tree to Spark's explain output. | +| spark.gluten.sql.mergeTwoPhasesAggregate.enabled | true | Whether to merge two phases aggregate if there are no other operators between them. | +| spark.gluten.sql.native.arrow.reader.enabled | false | This is config to specify whether to enable the native columnar csv reader | +| spark.gluten.sql.native.bloomFilter | true | +| spark.gluten.sql.native.hive.writer.enabled | 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 | true | +| spark.gluten.sql.native.parquet.write.blockRows | 100000000 | +| spark.gluten.sql.native.union | false | Enable or disable native union where computation is completely offloaded to backend. | +| spark.gluten.sql.native.writeColumnMetadataExclusionList | 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 | <undefined> | This is config to specify whether to enable the native columnar parquet/orc writer | +| spark.gluten.sql.orc.charType.scan.fallback.enabled | true | Force fallback for orc char type scan. | +| spark.gluten.sql.pushAggregateThroughJoin.enabled | 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 | 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 | true | When true, Gluten will remove the vanilla Spark V1Writes added sort and project for velox backend. | +| spark.gluten.sql.rewrite.dateTimestampComparison | 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 | 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 | and,or | Flatten nested functions as one for optimization. | +| spark.gluten.sql.text.input.empty.as.default | false | treat empty fields in CSV input as default values. | +| spark.gluten.sql.text.input.max.block.size | 8KB | the max block size for text input rows | +| spark.gluten.sql.validation.printStackOnFailure | false | +| spark.gluten.storage.hdfsViewfs.enabled | false | If enabled, gluten will convert the viewfs path to hdfs path in scala side | +| spark.gluten.supported.hive.udfs || Supported hive udf names. | +| spark.gluten.supported.python.udfs || Supported python udf names. | +| spark.gluten.supported.scala.udfs || Supported scala udf names. | +| spark.gluten.ui.enabled | 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 909b5b411d19..9a5247c823e4 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,6 +39,12 @@ 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. + public native byte[] serializeWithStats(long handle); + // Return the native ColumnarBatchSerializer handle public native long init(long cSchema); 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 0e2877ef4e7d..0bbbcead63be 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 @@ -1024,6 +1024,19 @@ object GlutenConfig extends ConfigRegistry { .booleanConf .createWithDefault(false) + val COLUMNAR_TABLE_CACHE_PARTITION_STATS_ENABLED = + buildConf("spark.gluten.sql.columnar.tableCache.partitionStats.enabled") + .doc( + "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.") + .booleanConf + .createWithDefault(false) + val COLUMNAR_PHYSICAL_JOIN_OPTIMIZATION_THROTTLE = buildConf("spark.gluten.sql.columnar.physicalJoinOptimizationLevel") .doc("Fallback to row operators if there are several continuous joins.")