diff --git a/fips-pom.xml b/fips-pom.xml
index 999fde92..b3b51300 100644
--- a/fips-pom.xml
+++ b/fips-pom.xml
@@ -4,7 +4,7 @@
4.0.0
com.snowflake
snowpark-fips
- 1.9.0-SNAPSHOT
+ 1.9.0
${project.artifactId}
Snowflake's DataFrame API
https://www.snowflake.com/
diff --git a/java_doc.xml b/java_doc.xml
index c2827d21..6210cabe 100644
--- a/java_doc.xml
+++ b/java_doc.xml
@@ -4,7 +4,7 @@
4.0.0
com.snowflake
snowpark-java
- 1.9.0-SNAPSHOT
+ 1.9.0
${project.artifactId}
Snowflake's DataFrame API
https://www.snowflake.com/
diff --git a/pom.xml b/pom.xml
index b06a1f60..57f273ba 100644
--- a/pom.xml
+++ b/pom.xml
@@ -4,7 +4,7 @@
4.0.0
com.snowflake
snowpark
- 1.9.0-SNAPSHOT
+ 1.9.0-coveo-1
${project.artifactId}
Snowflake's DataFrame API
https://www.snowflake.com/
diff --git a/src/main/java/com/snowflake/snowpark_java/Functions.java b/src/main/java/com/snowflake/snowpark_java/Functions.java
index 74cc39a8..06477bb0 100644
--- a/src/main/java/com/snowflake/snowpark_java/Functions.java
+++ b/src/main/java/com/snowflake/snowpark_java/Functions.java
@@ -79,7 +79,7 @@ public static Column toScalar(DataFrame df) {
* @return The result column
*/
public static Column lit(Object literal) {
- return new Column(com.snowflake.snowpark.functions.lit(literal));
+ return new Column(com.snowflake.snowpark.functions.lit(JavaUtils.toScala(literal)));
}
/**
diff --git a/src/main/scala/com/snowflake/snowpark/internal/JavaUtils.scala b/src/main/scala/com/snowflake/snowpark/internal/JavaUtils.scala
index 08a92b6b..6d817507 100644
--- a/src/main/scala/com/snowflake/snowpark/internal/JavaUtils.scala
+++ b/src/main/scala/com/snowflake/snowpark/internal/JavaUtils.scala
@@ -414,4 +414,15 @@ object JavaUtils {
}
}
+ def toScala(element: Any): Any = {
+ import collection.JavaConverters._
+ element match {
+ case map: java.util.Map[_, _] => mapAsScalaMap(map).map {
+ case (k, v) => toScala(k) -> toScala(v)
+ }.toMap
+ case iterable: java.lang.Iterable[_] => iterableAsScalaIterable(iterable).map(toScala)
+ case iterator: java.util.Iterator[_] => asScalaIterator(iterator).map(toScala)
+ case _ => element
+ }
+ }
}
diff --git a/src/main/scala/com/snowflake/snowpark/internal/Utils.scala b/src/main/scala/com/snowflake/snowpark/internal/Utils.scala
index faa53e45..789cc572 100644
--- a/src/main/scala/com/snowflake/snowpark/internal/Utils.scala
+++ b/src/main/scala/com/snowflake/snowpark/internal/Utils.scala
@@ -14,7 +14,7 @@ import scala.collection.mutable.ArrayBuffer
import scala.util.Random
object Utils extends Logging {
- val Version: String = "1.9.0-SNAPSHOT"
+ val Version: String = "1.9.0"
// Package name of snowpark on server side
val SnowparkPackageName = "com.snowflake:snowpark"
val PackageNameDelimiter = ":"
diff --git a/src/main/scala/com/snowflake/snowpark/internal/analyzer/DataTypeMapper.scala b/src/main/scala/com/snowflake/snowpark/internal/analyzer/DataTypeMapper.scala
index 598dd166..1993fd61 100644
--- a/src/main/scala/com/snowflake/snowpark/internal/analyzer/DataTypeMapper.scala
+++ b/src/main/scala/com/snowflake/snowpark/internal/analyzer/DataTypeMapper.scala
@@ -1,22 +1,22 @@
package com.snowflake.snowpark.internal.analyzer
+
import com.snowflake.snowpark.internal.Utils
+import com.snowflake.snowpark.types._
+import net.snowflake.client.jdbc.internal.snowflake.common.core.SnowflakeDateTimeFormat
+import java.math.{BigDecimal => JBigDecimal}
import java.sql.{Date, Timestamp}
import java.util.TimeZone
-import java.math.{BigDecimal => JBigDecimal}
-
-import com.snowflake.snowpark.types._
-import com.snowflake.snowpark.types.convertToSFType
import javax.xml.bind.DatatypeConverter
-import net.snowflake.client.jdbc.internal.snowflake.common.core.SnowflakeDateTimeFormat
object DataTypeMapper {
// milliseconds per day
private val MILLIS_PER_DAY = 24 * 3600 * 1000L
// microseconds per millisecond
private val MICROS_PER_MILLIS = 1000L
+
private[analyzer] def stringToSql(str: String): String =
- // Escapes all backslashes, single quotes and new line.
+ // Escapes all backslashes, single quotes and new line.
"'" + str
.replaceAll("\\\\", "\\\\\\\\")
.replaceAll("'", "''")
@@ -25,63 +25,77 @@ object DataTypeMapper {
/*
* Convert a value with DataType to a snowflake compatible sql
*/
- private[analyzer] def toSql(value: Any, dataType: Option[DataType]): String = {
- dataType match {
- case None => "NULL"
- case Some(dt) =>
- (value, dt) match {
- case (_, _: ArrayType | _: MapType | _: StructType | GeographyType) if value == null =>
- "NULL"
- case (_, IntegerType) if value == null => "NULL :: int"
- case (_, ShortType) if value == null => "NULL :: smallint"
- case (_, ByteType) if value == null => "NULL :: tinyint"
- case (_, LongType) if value == null => "NULL :: bigint"
- case (_, FloatType) if value == null => "NULL :: float"
- case (_, StringType) if value == null => "NULL :: string"
- case (_, DoubleType) if value == null => "NULL :: double"
- case (_, BooleanType) if value == null => "NULL :: boolean"
- case (_, BinaryType) if value == null => "NULL :: binary"
- case _ if value == null => "NULL"
- case (v: String, StringType) => stringToSql(v)
- case (v: Byte, ByteType) => v + s" :: tinyint"
- case (v: Short, ShortType) => v + s" :: smallint"
- case (v: Any, IntegerType) => v + s" :: int"
- case (v: Long, LongType) => v + s" :: bigint"
- case (v: Boolean, BooleanType) => s"$v :: boolean"
- // Float type doesn't have a suffix
- case (v: Float, FloatType) =>
- val castedValue = v match {
- case _ if v.isNaN => "'NaN'"
- case Float.PositiveInfinity => "'Infinity'"
- case Float.NegativeInfinity => "'-Infinity'"
- case _ => s"'$v'"
- }
- s"$castedValue :: FLOAT"
- case (v: Double, DoubleType) =>
- v match {
- case _ if v.isNaN => "'NaN'"
- case Double.PositiveInfinity => "'Infinity'"
- case Double.NegativeInfinity => "'-Infinity'"
- case _ => v + "::DOUBLE"
- }
- case (v: BigDecimal, t: DecimalType) => v + s" :: ${number(t.precision, t.scale)}"
- case (v: JBigDecimal, t: DecimalType) => v + s" :: ${number(t.precision, t.scale)}"
- case (v: Int, DateType) =>
- s"DATE '${SnowflakeDateTimeFormat
- .fromSqlFormat(Utils.DateInputFormat)
- .format(new Date(v * MILLIS_PER_DAY), TimeZone.getTimeZone("GMT"))}'"
- case (v: Long, TimestampType) =>
- s"TIMESTAMP '${SnowflakeDateTimeFormat
- .fromSqlFormat(Utils.TimestampInputFormat)
- .format(new Timestamp(v / MICROS_PER_MILLIS), TimeZone.getDefault, 3)}'"
- case (v: Array[Byte], BinaryType) =>
- s"'${DatatypeConverter.printHexBinary(v)}' :: binary"
- case _ =>
- throw new UnsupportedOperationException(
- s"Unsupported datatype by ToSql: ${value.getClass.getName} => $dataType")
+ private[analyzer] def toSql(literal: TLiteral): String = {
+ literal match {
+ case Literal(value, dataType) => (value, dataType) match {
+ case (_, None) => "NULL"
+ case (value, Some(dt)) =>
+ (value, dt) match {
+ case (_, _: ArrayType | _: MapType | _: StructType | GeographyType) if value == null =>
+ "NULL"
+ case (_, IntegerType) if value == null => "NULL :: int"
+ case (_, ShortType) if value == null => "NULL :: smallint"
+ case (_, ByteType) if value == null => "NULL :: tinyint"
+ case (_, LongType) if value == null => "NULL :: bigint"
+ case (_, FloatType) if value == null => "NULL :: float"
+ case (_, StringType) if value == null => "NULL :: string"
+ case (_, DoubleType) if value == null => "NULL :: double"
+ case (_, BooleanType) if value == null => "NULL :: boolean"
+ case (_, BinaryType) if value == null => "NULL :: binary"
+ case _ if value == null => "NULL"
+ case (v: String, StringType) => stringToSql(v)
+ case (v: Byte, ByteType) => v + s" :: tinyint"
+ case (v: Short, ShortType) => v + s" :: smallint"
+ case (v: Any, IntegerType) => v + s" :: int"
+ case (v: Long, LongType) => v + s" :: bigint"
+ case (v: Boolean, BooleanType) => s"$v :: boolean"
+ // Float type doesn't have a suffix
+ case (v: Float, FloatType) =>
+ val castedValue = v match {
+ case _ if v.isNaN => "'NaN'"
+ case Float.PositiveInfinity => "'Infinity'"
+ case Float.NegativeInfinity => "'-Infinity'"
+ case _ => s"'$v'"
+ }
+ s"$castedValue :: FLOAT"
+ case (v: Double, DoubleType) =>
+ v match {
+ case _ if v.isNaN => "'NaN'"
+ case Double.PositiveInfinity => "'Infinity'"
+ case Double.NegativeInfinity => "'-Infinity'"
+ case _ => v + "::DOUBLE"
+ }
+ case (v: BigDecimal, t: DecimalType) => v + s" :: ${number(t.precision, t.scale)}"
+ case (v: JBigDecimal, t: DecimalType) => v + s" :: ${number(t.precision, t.scale)}"
+ case (v: Int, DateType) =>
+ s"DATE '${
+ SnowflakeDateTimeFormat
+ .fromSqlFormat(Utils.DateInputFormat)
+ .format(new Date(v * MILLIS_PER_DAY), TimeZone.getTimeZone("GMT"))
+ }'"
+ case (v: Long, TimestampType) =>
+ s"TIMESTAMP '${
+ SnowflakeDateTimeFormat
+ .fromSqlFormat(Utils.TimestampInputFormat)
+ .format(new Timestamp(v / MICROS_PER_MILLIS), TimeZone.getDefault, 3)
+ }'"
+ case _ =>
+ throw new UnsupportedOperationException(
+ s"Unsupported datatype by ToSql: ${value.getClass.getName} => $dataType")
+ }
+ }
+ case arrayLiteral: ArrayLiteral =>
+ if (arrayLiteral.dataTypeOption == Some(BinaryType)) {
+ val bytes = arrayLiteral.value.asInstanceOf[Seq[Byte]].toArray
+ s"'${DatatypeConverter.printHexBinary(bytes)}' :: binary"
+ } else {
+ "ARRAY_CONSTRUCT" + arrayLiteral.elementsLiterals.map(toSql).mkString("(", ", ", ")")
}
+ case mapLiteral: MapLiteral =>
+ "OBJECT_CONSTRUCT" + mapLiteral.entriesLiterals.flatMap { case (keyLiteral, valueLiteral) =>
+ Seq(toSql(keyLiteral), toSql(valueLiteral))
+ }.mkString("(", ", ", ")")
}
-
}
private[analyzer] def schemaExpression(dataType: DataType, isNullable: Boolean): String =
diff --git a/src/main/scala/com/snowflake/snowpark/internal/analyzer/Literal.scala b/src/main/scala/com/snowflake/snowpark/internal/analyzer/Literal.scala
index 69fb3eda..86804508 100644
--- a/src/main/scala/com/snowflake/snowpark/internal/analyzer/Literal.scala
+++ b/src/main/scala/com/snowflake/snowpark/internal/analyzer/Literal.scala
@@ -2,12 +2,11 @@ package com.snowflake.snowpark.internal.analyzer
import com.snowflake.snowpark.internal.ErrorMessage
import com.snowflake.snowpark.types._
+
import java.math.{BigDecimal => JavaBigDecimal}
import java.sql.{Date, Timestamp}
import java.time.{Instant, LocalDate}
-import scala.math.BigDecimal
-
private[snowpark] object Literal {
// Snowflake max precision for decimal is 38
private lazy val bigDecimalRoundContext = new java.math.MathContext(DecimalType.MAX_PRECISION)
@@ -16,7 +15,7 @@ private[snowpark] object Literal {
decimal.round(bigDecimalRoundContext)
}
- def apply(v: Any): Literal = v match {
+ def apply(v: Any): TLiteral = v match {
case i: Int => Literal(i, Option(IntegerType))
case l: Long => Literal(l, Option(LongType))
case d: Double => Literal(d, Option(DoubleType))
@@ -36,7 +35,8 @@ private[snowpark] object Literal {
case t: Timestamp => Literal(DateTimeUtils.javaTimestampToMicros(t), Option(TimestampType))
case ld: LocalDate => Literal(DateTimeUtils.localDateToDays(ld), Option(DateType))
case d: Date => Literal(DateTimeUtils.javaDateToDays(d), Option(DateType))
- case a: Array[Byte] => Literal(a, Option(BinaryType))
+ case s: Seq[Any] => ArrayLiteral(s)
+ case m: Map[Any, Any] => MapLiteral(m)
case null => Literal(null, None)
case v: Literal => v
case _ =>
@@ -45,10 +45,48 @@ private[snowpark] object Literal {
}
-private[snowpark] case class Literal private (value: Any, dataTypeOption: Option[DataType])
- extends Expression {
+private[snowpark] trait TLiteral extends Expression {
+ def value: Any
+ def dataTypeOption: Option[DataType]
+
override def children: Seq[Expression] = Seq.empty
override protected def createAnalyzedExpression(analyzedChildren: Seq[Expression]): Expression =
this
}
+
+private[snowpark] case class Literal (value: Any, dataTypeOption: Option[DataType]) extends TLiteral
+
+private[snowpark] case class ArrayLiteral(value: Seq[Any]) extends TLiteral {
+ val elementsLiterals: Seq[TLiteral] = value.map(Literal(_))
+ val dataTypeOption = inferArrayType
+
+ private[analyzer] def inferArrayType(): Option[DataType] = {
+ elementsLiterals.flatMap(_.dataTypeOption).distinct match {
+ case Seq() => None
+ case Seq(ByteType) => Some(BinaryType)
+ case Seq(dt) => Some(ArrayType(dt))
+ case Seq(_, _*) => Some(ArrayType(VariantType))
+ }
+ }
+}
+
+private[snowpark] case class MapLiteral(value: Map[Any, Any]) extends TLiteral {
+ val entriesLiterals = value.map { case (k, v) => Literal(k) -> Literal(v) }
+ val dataTypeOption = inferMapType
+
+ private[analyzer] def inferMapType(): Option[MapType] = {
+ entriesLiterals.keys.flatMap(_.dataTypeOption).toSeq.distinct match {
+ case Seq() => None
+ case Seq(StringType) =>
+ val valuesTypes = entriesLiterals.values.flatMap(_.dataTypeOption).toSeq.distinct
+ valuesTypes match {
+ case Seq() => None
+ case Seq(dt) => Some(MapType(StringType, dt))
+ case Seq(_, _*) => Some(MapType(StringType, VariantType))
+ }
+ case _ =>
+ throw ErrorMessage.PLAN_CANNOT_CREATE_LITERAL(value.getClass.getCanonicalName, s"$value")
+ }
+ }
+}
diff --git a/src/main/scala/com/snowflake/snowpark/internal/analyzer/SqlGenerator.scala b/src/main/scala/com/snowflake/snowpark/internal/analyzer/SqlGenerator.scala
index a7a5f655..058d00a4 100644
--- a/src/main/scala/com/snowflake/snowpark/internal/analyzer/SqlGenerator.scala
+++ b/src/main/scala/com/snowflake/snowpark/internal/analyzer/SqlGenerator.scala
@@ -203,8 +203,8 @@ private object SqlGenerator extends Logging {
case UnspecifiedFrame => ""
case SpecialFrameBoundaryExtractor(str) => str
- case Literal(value, dataType) =>
- DataTypeMapper.toSql(value, dataType)
+ case l: TLiteral =>
+ DataTypeMapper.toSql(l)
case attr: Attribute => quoteName(attr.name)
// unresolved expression
case UnresolvedAttribute(name) => name
diff --git a/src/main/scala/com/snowflake/snowpark/internal/analyzer/package.scala b/src/main/scala/com/snowflake/snowpark/internal/analyzer/package.scala
index a6af91aa..ca7edc41 100644
--- a/src/main/scala/com/snowflake/snowpark/internal/analyzer/package.scala
+++ b/src/main/scala/com/snowflake/snowpark/internal/analyzer/package.scala
@@ -3,7 +3,7 @@ package com.snowflake.snowpark.internal
import com.snowflake.snowpark.FileOperationCommand._
import com.snowflake.snowpark.Row
import com.snowflake.snowpark.internal.Utils.{TempObjectType, randomNameForTempObject}
-import com.snowflake.snowpark.types.{DataType, convertToSFType}
+import com.snowflake.snowpark.types.{ArrayType, DataType, MapType, convertToSFType}
package object analyzer {
// constant string
@@ -446,7 +446,9 @@ package object analyzer {
val types = output.map(_.dataType)
val rows = data.map { row =>
val cells = row.toSeq.zip(types).map {
- case (v, dType) => DataTypeMapper.toSql(v, Option(dType))
+ case (v: Seq[Any], _: ArrayType) => DataTypeMapper.toSql(ArrayLiteral(v))
+ case (v: Map[Any, Any], _: MapType) => DataTypeMapper.toSql(MapLiteral(v))
+ case (v, dType) => DataTypeMapper.toSql(Literal(v, Option(dType)))
}
cells.mkString(_LeftParenthesis, _Comma, _RightParenthesis)
}
diff --git a/src/test/java/com/snowflake/snowpark_test/JavaFunctionSuite.java b/src/test/java/com/snowflake/snowpark_test/JavaFunctionSuite.java
index f74dc440..6af340dc 100644
--- a/src/test/java/com/snowflake/snowpark_test/JavaFunctionSuite.java
+++ b/src/test/java/com/snowflake/snowpark_test/JavaFunctionSuite.java
@@ -1,11 +1,23 @@
package com.snowflake.snowpark_test;
+import com.snowflake.snowpark.internal.JavaUtils;
+import com.snowflake.snowpark.internal.analyzer.Literal;
import com.snowflake.snowpark_java.*;
import java.sql.Date;
import java.sql.Time;
import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
+import jdk.jshell.spi.ExecutionControl;
import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+
public class JavaFunctionSuite extends TestBase {
@Test
@@ -17,6 +29,66 @@ public void toScalar() {
checkAnswer(df1.select(Functions.col("c1"), Functions.col(df2)), expected, false);
checkAnswer(df1.select(Functions.col("c1"), Functions.toScalar(df2)), expected, false);
}
+
+ @Test
+ public void lit() {
+ DataFrame df = getSession().sql("select * from values (1),(2),(3) as T(a)");
+
+ // Empty array is supported
+ Row[] expectedEmptyArray = new Row[3];
+ Arrays.fill(expectedEmptyArray, Row.create("[]"));
+ checkAnswer(df.select(Functions.lit(Collections.EMPTY_LIST)), expectedEmptyArray, false);
+
+ // Empty map is supported
+ Row[] expectedEmptyMap = new Row[3];
+ Arrays.fill(expectedEmptyMap, Row.create("{}"));
+ checkAnswer(df.select(Functions.lit(Collections.EMPTY_MAP)), expectedEmptyMap, false);
+
+ // Array with only bytes should be considered Binary
+ Row[] expectedBinary = new Row[3];
+ Arrays.fill(expectedBinary, Row.create(new byte[]{(byte) 1, (byte) 2, (byte) 3}));
+
+ DataFrame actualBinary = df.select(Functions.lit(List.of((byte) 1, (byte) 2, (byte) 3)));
+
+ checkAnswer(actualBinary, expectedBinary);
+
+ // Array and Map results type are not supported, they are instead always converted to String.
+ // Hence, we need to test by comparing results Strings.
+ Function rowsToString = (Row[] rows) -> Arrays.stream(rows)
+ .map((Row row) -> row.getString(0).replaceAll("\n| ", ""))
+ .toArray();
+
+ // Array with different types of elements
+ String[] expectedArrays = new String[3];
+ Arrays.fill(expectedArrays, "[1,\"3\",[\"2023-08-25\"]]");
+
+ Row[] actualArraysRows = df.select(Functions.lit(List.of(
+ 1,
+ "3",
+ List.of(Date.valueOf("2023-08-25"))
+ ))).collect();
+ Object[] actualArrays = rowsToString.apply(actualArraysRows);
+
+ assertEquals(expectedArrays, actualArrays);
+
+ // One or more map keys are not of the String type. Should throw an exception.
+ assertThrows(
+ scala.NotImplementedError.class,
+ () -> df.select(Functions.lit(Map.of("1", 1, 2, 2)))
+ );
+
+ // Map with different type of elements
+ String[] expectedMaps = new String[3];
+ Arrays.fill(expectedMaps, "{\"key1\":{\"nestedKey\":42},\"key2\":\"2023-08-25\"}");
+
+ Row[] actualMapsRows = df.select(Functions.lit(Map.of(
+ "key1", Map.of("nestedKey", 42),
+ "key2", Date.valueOf("2023-08-25"))
+ )).collect();
+ Object[] actualMaps = rowsToString.apply(actualMapsRows);
+
+ assertEquals(expectedMaps, actualMaps);
+ }
@Test
public void sqlText() {
diff --git a/src/test/scala/com/snowflake/snowpark_test/AsyncJobSuite.scala b/src/test/scala/com/snowflake/snowpark_test/AsyncJobSuite.scala
index 7eb31354..84ad53bf 100644
--- a/src/test/scala/com/snowflake/snowpark_test/AsyncJobSuite.scala
+++ b/src/test/scala/com/snowflake/snowpark_test/AsyncJobSuite.scala
@@ -30,12 +30,13 @@ class AsyncJobSuite extends TestData with BeforeAndAfterEach {
super.beforeAll()
// create temporary stage to store the file
runQuery(s"CREATE TEMPORARY STAGE $tmpStageName", session)
- runQuery(s"CREATE TEMPORARY STAGE $tmpStageName", newSession)
// Create temp target stage for writing DF to file test.
runQuery(s"CREATE TEMPORARY STAGE $targetStageName", session)
// upload the file to stage
uploadFileToStage(tmpStageName, testFileCsv, compress = false)
if (!isStoredProc(session)) {
+ // create temporary stage to store the file
+ runQuery(s"CREATE TEMPORARY STAGE $tmpStageName", newSession)
TestUtils.addDepsToClassPath(session, Some(tmpStageName))
// In stored procs mode, there is only one session
TestUtils.addDepsToClassPath(newSession, Some(tmpStageName))
@@ -45,7 +46,9 @@ class AsyncJobSuite extends TestData with BeforeAndAfterEach {
override def afterAll(): Unit = {
// drop the temporary stages
runQuery(s"DROP STAGE IF EXISTS $tmpStageName", session)
- runQuery(s"DROP STAGE IF EXISTS $tmpStageName", newSession)
+ if (!isStoredProc(session)) {
+ runQuery(s"DROP STAGE IF EXISTS $tmpStageName", newSession)
+ }
dropTable(tableName)
dropTable(tableName1)
dropTable(tableName2)
diff --git a/src/test/scala/com/snowflake/snowpark_test/DataFrameSuite.scala b/src/test/scala/com/snowflake/snowpark_test/DataFrameSuite.scala
index 6efa0809..65e141dc 100644
--- a/src/test/scala/com/snowflake/snowpark_test/DataFrameSuite.scala
+++ b/src/test/scala/com/snowflake/snowpark_test/DataFrameSuite.scala
@@ -441,7 +441,7 @@ trait DataFrameSuite extends TestData with BeforeAndAfterEach {
assert(double2.stat.cov("a", "b").get == 0.010000000000000037)
}
- test("df.stat.approxQuantile") {
+ test("df.stat.approxQuantile", JavaStoredProcExclude) {
assert(approxNumbers.stat.approxQuantile("a", Array(0.5))(0).get == 4.5)
assert(
approxNumbers.stat.approxQuantile("a", Array(0, 0.1, 0.4, 0.6, 1)).deep ==