nsivabalan commented on code in PR #12993:
URL: https://github.com/apache/hudi/pull/12993#discussion_r2012689008
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/ColumnStatsIndexSupport.scala:
##########
@@ -455,10 +456,10 @@ object ColumnStatsIndexSupport {
case w: LongWrapper => w.getValue
case w: FloatWrapper => w.getValue
case w: DoubleWrapper => w.getValue
+ case w: DecimalWrapper => w.getValue // Moved above BytesWrapper to
ensure proper matching
Review Comment:
got it.
##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala:
##########
@@ -1056,4 +1060,61 @@ class TestColumnStatsIndex extends
ColumnStatIndexTestBase {
assertTrue(r.getMinValue.asInstanceOf[Comparable[Object]].compareTo(r.getMaxValue.asInstanceOf[Object])
<= 0)
})
}
+
+ @Test
+ def testDeserializeFromByteBuffer(): Unit = {
+ // Original decimal value: 123.45 (scale 2)
+ val original: JBigDecimal = new JBigDecimal("123.45")
+ // Get the unscaled value (12345) as a byte array.
+ val schema: Schema = DecimalWrapper.SCHEMA$.getField("value").schema()
+ val buffer: ByteBuffer = ColumnStatsIndexSupport.decConv.toBytes(original,
schema, schema.getLogicalType)
+
+ // Create a dummy DecimalWrapper that returns a ByteBuffer.
+ val wrapper: DecimalWrapper = new DecimalWrapper(buffer)
+ // Call tryUnpackValueWrapper – it should match the DecimalWrapper case
and return the ByteBuffer.
+ val unwrapped: Any = ColumnStatsIndexSupport.tryUnpackValueWrapper(wrapper)
+ assertTrue(unwrapped.isInstanceOf[ByteBuffer], "Expected a ByteBuffer")
+
+ // Now deserialize the ByteBuffer to a BigDecimal.
+ val dt = DecimalType(10, 2)
+ val deserialized: Any = ColumnStatsIndexSupport.deserialize(unwrapped, dt)
+ assertTrue(deserialized.isInstanceOf[JBigDecimal], "Deserialized value
should be a java.math.BigDecimal")
+ assertEquals(original, deserialized.asInstanceOf[JBigDecimal], "Decimal
value from ByteBuffer does not match")
+ }
+
+ @Test
+ def testDeserializeFromJavaBigDecimal(): Unit = {
+ // Original decimal value: 543.21
+ val original: JBigDecimal = new JBigDecimal("543.21")
+ // Create an anonymous DecimalWrapper that returns a ByteBuffer of
java.math.BigDecimal directly.
+ val wrapper: DecimalWrapper = new DecimalWrapper {
+ val schema: Schema = DecimalWrapper.SCHEMA$.getField("value").schema()
+
+ override def getValue: ByteBuffer =
ColumnStatsIndexSupport.decConv.toBytes(original, schema, schema.getLogicalType)
+ }
+
+ val dt = DecimalType(10, 2)
+ val deserialized: Any =
ColumnStatsIndexSupport.deserialize(ColumnStatsIndexSupport.tryUnpackValueWrapper(wrapper),
dt)
+ assertTrue(deserialized.isInstanceOf[JBigDecimal], "Deserialized value
should be a java.math.BigDecimal")
+ assertEquals(original, deserialized.asInstanceOf[JBigDecimal], "Decimal
value from java.math.BigDecimal does not match")
+ }
+
+ @Test
+ def testDeserializeFromScalaBigDecimal(): Unit = {
Review Comment:
is it possible to make all 3 into just 1 test and make it parametrized.
input value, expected values etc.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]