diff --git a/docs/compatibility.md b/docs/compatibility.md
index d7ef0005508..85946b3c4ad 100644
--- a/docs/compatibility.md
+++ b/docs/compatibility.md
@@ -283,14 +283,7 @@ will produce a different result compared to the plugin.
Due to inconsistencies between how CSV data is parsed CSV parsing is off by default.
Each data type can be enabled or disabled independently using the following configs.
- * [spark.rapids.sql.csv.read.bool.enabled](configs.md#sql.csv.read.bool.enabled)
- * [spark.rapids.sql.csv.read.byte.enabled](configs.md#sql.csv.read.byte.enabled)
* [spark.rapids.sql.csv.read.date.enabled](configs.md#sql.csv.read.date.enabled)
- * [spark.rapids.sql.csv.read.double.enabled](configs.md#sql.csv.read.double.enabled)
- * [spark.rapids.sql.csv.read.float.enabled](configs.md#sql.csv.read.float.enabled)
- * [spark.rapids.sql.csv.read.integer.enabled](configs.md#sql.csv.read.integer.enabled)
- * [spark.rapids.sql.csv.read.long.enabled](configs.md#sql.csv.read.long.enabled)
- * [spark.rapids.sql.csv.read.short.enabled](configs.md#sql.csv.read.short.enabled)
* [spark.rapids.sql.csvTimestamps.enabled](configs.md#sql.csvTimestamps.enabled)
If you know that your particular data type will be parsed correctly enough, you may enable each
@@ -307,14 +300,6 @@ default. The plugin will strip leading and trailing space for all values except
There are also discrepancies/issues with specific types that are detailed below.
-### CSV Boolean
-
-Invalid values like `BAD` show up as `true` as described by this
-[issue](https://github.com/NVIDIA/spark-rapids/issues/2071)
-
-This is the same for all other types, but because that is the only issue with boolean parsing
-we have called it out specifically here.
-
### CSV Strings
Writing strings to a CSV file in general for Spark can be problematic unless you can ensure that
your data does not have any line deliminators in it. The GPU accelerated CSV parser handles quoted
@@ -383,10 +368,6 @@ Also parsing of some values will not produce bit for bit identical results to wh
They are within round-off errors except when they are close enough to overflow to Inf or -Inf which
then results in a number being returned when the CPU would have returned null.
-### CSV Integer
-
-Any number that overflows will not be turned into a null value.
-
## ORC
The ORC format has fairly complete support for both reads and writes. There are only a few known
@@ -475,18 +456,13 @@ The nested types(array, map and struct) are not supported yet in current version
Parsing floating-point values has the same limitations as [casting from string to float](#String-to-Float).
-The GPU JSON reader does not support `NaN` and `Inf` values with full compatibility with Spark.
+Prior to Spark 3.3.0, reading JSON strings such as `"+Infinity"` when specifying that the data type is `FloatType`
+or `DoubleType` caused these values to be parsed even when `allowNonNumericNumbers` is set to false. Also, Spark
+versions prior to 3.3.0 only supported the `"Infinity"` and `"-Infinity"` representations of infinity and did not
+support `"+INF"`, `"-INF"`, or `"+Infinity"`, which Spark considers valid when unquoted. The GPU JSON reader is
+consistent with the behavior in Spark 3.3.0 and later.
-The following are the only formats that are parsed consistently between CPU and GPU. Any other variation, including
-these formats when unquoted, will produce `null` on the CPU and may produce valid `NaN` and `Inf` results on the GPU.
-
-```json
-{ "number": "NaN" }
-{ "number": "Infinity" }
-{ "number": "-Infinity" }
-```
-
-Another limitation of the GPU JSON reader is that it will parse strings containing floating-point values where
+Another limitation of the GPU JSON reader is that it will parse strings containing boolean or numeric values where
Spark will treat them as invalid inputs and will just return `null`.
### JSON Schema discovery
@@ -513,6 +489,12 @@ unquoted control characters but Spark reads these entries incorrectly as null. H
and when the option is false, then RAPIDS Accelerator's behavior is same as Spark where an exception is thrown
as discussed in `JSON Schema discovery` section.
+- `allowNonNumericNumbers` - Allows `NaN` and `Infinity` values to be parsed (note that these are not valid numeric
+values in the [JSON specification](https://json.org)). Spark versions prior to 3.3.0 have inconsistent behavior and will
+parse some variants of `NaN` and `Infinity` even when this option is disabled
+([SPARK-38060](https://issues.apache.org/jira/browse/SPARK-38060)). The RAPIDS Accelerator behavior is consistent with
+Spark version 3.3.0 and later.
+
## Regular Expressions
The following Apache Spark regular expression functions and expressions are supported on the GPU:
diff --git a/docs/supported_ops.md b/docs/supported_ops.md
index 4f4292f97a6..a063f6ff02d 100644
--- a/docs/supported_ops.md
+++ b/docs/supported_ops.md
@@ -17698,7 +17698,7 @@ dates or timestamps, or for a lack of type coercion support.
S |
PS UTC is only supported TZ for TIMESTAMP |
S |
-NS |
+S |
|
NS |
|
@@ -17741,7 +17741,7 @@ dates or timestamps, or for a lack of type coercion support.
S |
PS UTC is only supported TZ for TIMESTAMP |
S |
-NS |
+S |
|
NS |
|
diff --git a/integration_tests/src/main/python/csv_test.py b/integration_tests/src/main/python/csv_test.py
index 67c5e7344b4..3067bb1ee5b 100644
--- a/integration_tests/src/main/python/csv_test.py
+++ b/integration_tests/src/main/python/csv_test.py
@@ -127,6 +127,12 @@
_double_schema = StructType([
StructField('number', DoubleType())])
+_decimal_10_2_schema = StructType([
+ StructField('number', DecimalType(10, 2))])
+
+_decimal_10_3_schema = StructType([
+ StructField('number', DecimalType(10, 3))])
+
_number_as_string_schema = StructType([
StructField('number', StringType())])
@@ -220,6 +226,9 @@ def read_impl(spark):
pytest.param('simple_int_values.csv', _long_schema, {'header': 'true'}),
('simple_int_values.csv', _float_schema, {'header': 'true'}),
('simple_int_values.csv', _double_schema, {'header': 'true'}),
+ ('simple_int_values.csv', _decimal_10_2_schema, {'header': 'true'}),
+ ('decimals.csv', _decimal_10_2_schema, {'header': 'true'}),
+ ('decimals.csv', _decimal_10_3_schema, {'header': 'true'}),
pytest.param('empty_int_values.csv', _empty_byte_schema, {'header': 'true'}),
pytest.param('empty_int_values.csv', _empty_short_schema, {'header': 'true'}),
pytest.param('empty_int_values.csv', _empty_int_schema, {'header': 'true'}),
@@ -235,6 +244,8 @@ def read_impl(spark):
pytest.param('simple_float_values.csv', _long_schema, {'header': 'true'}),
pytest.param('simple_float_values.csv', _float_schema, {'header': 'true'}),
pytest.param('simple_float_values.csv', _double_schema, {'header': 'true'}),
+ pytest.param('simple_float_values.csv', _decimal_10_2_schema, {'header': 'true'}),
+ pytest.param('simple_float_values.csv', _decimal_10_3_schema, {'header': 'true'}),
pytest.param('simple_boolean_values.csv', _bool_schema, {'header': 'true'}),
pytest.param('ints_with_whitespace.csv', _number_as_string_schema, {'header': 'true'}, marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/2069')),
pytest.param('ints_with_whitespace.csv', _byte_schema, {'header': 'true'}, marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/130'))
diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py
index 98daed3e037..eab1c1e0cc2 100644
--- a/integration_tests/src/main/python/json_test.py
+++ b/integration_tests/src/main/python/json_test.py
@@ -19,7 +19,7 @@
from conftest import is_databricks_runtime
from marks import approximate_float, allow_non_gpu, ignore_order
-from spark_session import with_cpu_session, with_gpu_session
+from spark_session import with_cpu_session, with_gpu_session, is_before_spark_330
json_supported_gens = [
# Spark does not escape '\r' or '\n' even though it uses it to mark end of record
@@ -60,6 +60,12 @@
_double_schema = StructType([
StructField('number', DoubleType())])
+_decimal_10_2_schema = StructType([
+ StructField('number', DecimalType(10, 2))])
+
+_decimal_10_3_schema = StructType([
+ StructField('number', DecimalType(10, 3))])
+
_string_schema = StructType([
StructField('a', StringType())])
@@ -191,13 +197,15 @@ def test_json_ts_formats_round_trip(spark_tmp_path, date_format, ts_part, v1_ena
'ints.json',
pytest.param('ints_invalid.json', marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/4793')),
'nan_and_inf.json',
- pytest.param('nan_and_inf_edge_cases.json', marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/4646')),
+ pytest.param('nan_and_inf_strings.json', marks=pytest.mark.skipif(is_before_spark_330(), reason='https://issues.apache.org/jira/browse/SPARK-38060 fixed in Spark 3.3.0')),
+ 'nan_and_inf_invalid.json',
'floats.json',
'floats_leading_zeros.json',
'floats_invalid.json',
pytest.param('floats_edge_cases.json', marks=pytest.mark.xfail(reason='https://github.com/NVIDIA/spark-rapids/issues/4647')),
+ 'decimals.json',
])
-@pytest.mark.parametrize('schema', [_bool_schema, _byte_schema, _short_schema, _int_schema, _long_schema, _float_schema, _double_schema])
+@pytest.mark.parametrize('schema', [_bool_schema, _byte_schema, _short_schema, _int_schema, _long_schema, _float_schema, _double_schema, _decimal_10_2_schema, _decimal_10_3_schema])
@pytest.mark.parametrize('read_func', [read_json_df, read_json_sql])
@pytest.mark.parametrize('allow_non_numeric_numbers', ["true", "false"])
@pytest.mark.parametrize('allow_numeric_leading_zeros', ["true"])
diff --git a/integration_tests/src/test/resources/decimals.csv b/integration_tests/src/test/resources/decimals.csv
new file mode 100644
index 00000000000..b8e83af6fff
--- /dev/null
+++ b/integration_tests/src/test/resources/decimals.csv
@@ -0,0 +1,16 @@
+"number"
+-1
+0
+1.
+0.12
+.12
++.12
+-.12
+1
+1.01
+12.34
+12.3456
+12345678.12
+33.545454
+33.454545
+
diff --git a/integration_tests/src/test/resources/decimals.json b/integration_tests/src/test/resources/decimals.json
new file mode 100644
index 00000000000..0a98fd05474
--- /dev/null
+++ b/integration_tests/src/test/resources/decimals.json
@@ -0,0 +1,14 @@
+{ "number": 0 }
+{ "number": 12 }
+{ "number": 12.0 }
+{ "number": 12. }
+{ "number": .34 }
+{ "number": +.34 }
+{ "number": -.34 }
+{ "number": 0.34 }
+{ "number": 12.34 }
+{ "number": 12.3456 }
+{ "number": 12.345678 }
+{ "number": 123456.78 }
+{ "number": 33.454545 }
+{ "number": 33.545454 }
diff --git a/integration_tests/src/test/resources/nan_and_inf.json b/integration_tests/src/test/resources/nan_and_inf.json
index e4aab168de4..ff012b53bec 100644
--- a/integration_tests/src/test/resources/nan_and_inf.json
+++ b/integration_tests/src/test/resources/nan_and_inf.json
@@ -1,3 +1,6 @@
-{ "number": "NaN" }
-{ "number": "Infinity" }
-{ "number": "-Infinity" }
\ No newline at end of file
+{ "number": NaN }
+{ "number": +INF }
+{ "number": -INF }
+{ "number": Infinity }
+{ "number": +Infinity }
+{ "number": -Infinity }
\ No newline at end of file
diff --git a/integration_tests/src/test/resources/nan_and_inf_edge_cases.json b/integration_tests/src/test/resources/nan_and_inf_edge_cases.json
deleted file mode 100644
index c27a2291626..00000000000
--- a/integration_tests/src/test/resources/nan_and_inf_edge_cases.json
+++ /dev/null
@@ -1,12 +0,0 @@
-{ "number": "NAN" }
-{ "number": "nan" }
-{ "number": "INF" }
-{ "number": "+INF" }
-{ "number": "-INF" }
-{ "number": INF }
-{ "number": +INF }
-{ "number": -INF }
-{ "number": "Inf" }
-{ "number": "+Inf" }
-{ "number": "-Inf" }
-{ "number": "+Infinity" }
\ No newline at end of file
diff --git a/integration_tests/src/test/resources/nan_and_inf_invalid.json b/integration_tests/src/test/resources/nan_and_inf_invalid.json
new file mode 100644
index 00000000000..03dbb41d2f5
--- /dev/null
+++ b/integration_tests/src/test/resources/nan_and_inf_invalid.json
@@ -0,0 +1,12 @@
+{ "number": NAN }
+{ "number": nan }
+{ "number": INF }
+{ "number": Inf }
+{ "number": +Inf }
+{ "number": -Inf }
+{ "number": "NAN" }
+{ "number": "nan" }
+{ "number": "INF" }
+{ "number": "Inf" }
+{ "number": "+Inf" }
+{ "number": "-Inf" }
\ No newline at end of file
diff --git a/integration_tests/src/test/resources/nan_and_inf_strings.json b/integration_tests/src/test/resources/nan_and_inf_strings.json
new file mode 100644
index 00000000000..3080f5147a4
--- /dev/null
+++ b/integration_tests/src/test/resources/nan_and_inf_strings.json
@@ -0,0 +1,6 @@
+{ "number": "NaN" }
+{ "number": "+INF" }
+{ "number": "-INF" }
+{ "number": "Infinity" }
+{ "number": "+Infinity" }
+{ "number": "-Infinity" }
\ No newline at end of file
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala
index d3a40991fd3..2ccc3d12bed 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala
@@ -804,7 +804,7 @@ object GpuOverrides extends Logging {
lazy val fileFormats: Map[FileFormatType, Map[FileFormatOp, FileFormatChecks]] = Map(
(CsvFormatType, FileFormatChecks(
- cudfRead = TypeSig.commonCudfTypes,
+ cudfRead = TypeSig.commonCudfTypes + TypeSig.DECIMAL_128,
cudfWrite = TypeSig.none,
sparkSig = TypeSig.cpuAtomics)),
(ParquetFormatType, FileFormatChecks(
@@ -823,7 +823,7 @@ object GpuOverrides extends Logging {
sparkSig = (TypeSig.cpuAtomics + TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP +
TypeSig.UDT).nested())),
(JsonFormatType, FileFormatChecks(
- cudfRead = TypeSig.commonCudfTypes,
+ cudfRead = TypeSig.commonCudfTypes + TypeSig.DECIMAL_128,
cudfWrite = TypeSig.none,
sparkSig = (TypeSig.cpuAtomics + TypeSig.STRUCT + TypeSig.ARRAY + TypeSig.MAP +
TypeSig.UDT).nested())))
diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala
index 7b079ee0618..88e7fef3970 100644
--- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala
+++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTextBasedPartitionReader.scala
@@ -28,7 +28,7 @@ import org.apache.spark.TaskContext
import org.apache.spark.sql.connector.read.PartitionReader
import org.apache.spark.sql.execution.QueryExecutionException
import org.apache.spark.sql.execution.datasources.{HadoopFileLinesReader, PartitionedFile}
-import org.apache.spark.sql.types.{DataTypes, StructField, StructType}
+import org.apache.spark.sql.types.{DataTypes, DecimalType, StructField, StructType}
import org.apache.spark.sql.vectorized.ColumnarBatch
/**
@@ -172,7 +172,7 @@ abstract class GpuTextBasedPartitionReader(
f.dataType match {
case DataTypes.BooleanType | DataTypes.ByteType | DataTypes.ShortType |
DataTypes.IntegerType | DataTypes.LongType | DataTypes.FloatType |
- DataTypes.DoubleType =>
+ DataTypes.DoubleType | _: DecimalType =>
f.copy(dataType = DataTypes.StringType)
case _ =>
f
@@ -196,8 +196,6 @@ abstract class GpuTextBasedPartitionReader(
// Table increases the ref counts on the columns so we have
// to close them after creating the table
withResource(columns) { _ =>
- // ansi mode does not apply to text inputs
- val ansiEnabled = false
for (i <- 0 until table.getNumberOfColumns) {
val castColumn = newReadDataSchema.fields(i).dataType match {
case DataTypes.BooleanType =>
@@ -211,9 +209,11 @@ abstract class GpuTextBasedPartitionReader(
case DataTypes.LongType =>
castStringToInt(table.getColumn(i), DType.INT64)
case DataTypes.FloatType =>
- GpuCast.castStringToFloats(table.getColumn(i), ansiEnabled, DType.FLOAT32)
+ castStringToFloat(table.getColumn(i), DType.FLOAT32)
case DataTypes.DoubleType =>
- GpuCast.castStringToFloats(table.getColumn(i), ansiEnabled, DType.FLOAT64)
+ castStringToFloat(table.getColumn(i), DType.FLOAT64)
+ case dt: DecimalType =>
+ castStringToDecimal(table.getColumn(i), dt)
case _ =>
table.getColumn(i).incRefCount()
}
@@ -232,6 +232,14 @@ abstract class GpuTextBasedPartitionReader(
def castStringToBool(input: ColumnVector): ColumnVector
+ def castStringToFloat(input: ColumnVector, dt: DType): ColumnVector = {
+ GpuCast.castStringToFloats(input, ansiEnabled = false, dt)
+ }
+
+ def castStringToDecimal(input: ColumnVector, dt: DecimalType): ColumnVector = {
+ GpuCast.castStringToDecimal(input, ansiEnabled = false, dt)
+ }
+
def castStringToInt(input: ColumnVector, intType: DType): ColumnVector = {
withResource(input.isInteger(intType)) { isInt =>
withResource(input.castTo(intType)) { asInt =>
diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala
index 795a7d8eeb3..2815bbbdd64 100644
--- a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala
+++ b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala
@@ -38,7 +38,7 @@ import org.apache.spark.sql.execution.datasources.{PartitionedFile, Partitioning
import org.apache.spark.sql.execution.datasources.v2.{FilePartitionReaderFactory, FileScan, TextBasedFileScan}
import org.apache.spark.sql.execution.datasources.v2.json.JsonScan
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.types.{DateType, StringType, StructType, TimestampType}
+import org.apache.spark.sql.types.{DateType, DecimalType, StringType, StructType, TimestampType}
import org.apache.spark.sql.util.CaseInsensitiveStringMap
import org.apache.spark.sql.vectorized.ColumnarBatch
import org.apache.spark.util.SerializableConfiguration
@@ -370,4 +370,47 @@ class JsonPartitionReader(
}
}
+ /**
+ * JSON has strict rules about valid numeric formats. See https://www.json.org/ for specification.
+ *
+ * Spark then has its own rules for supporting NaN and Infinity, which are not
+ * valid numbers in JSON.
+ */
+ private def sanitizeNumbers(input: ColumnVector): ColumnVector = {
+ // Note that this is not 100% consistent with Spark versions prior to Spark 3.3.0
+ // due to https://issues.apache.org/jira/browse/SPARK-38060
+ // cuDF `isFloat` supports some inputs that are not valid JSON numbers, such as `.1`, `1.`,
+ // and `+1` so we use a regular expression to match valid JSON numbers instead
+ val jsonNumberRegexp = "^-?[0-9]+(?:\\.[0-9]+)?(?:[eE][\\-\\+]?[0-9]+)?$"
+ val isValid = if (parsedOptions.allowNonNumericNumbers) {
+ withResource(ColumnVector.fromStrings("NaN", "+INF", "-INF", "+Infinity",
+ "Infinity", "-Infinity")) { nonNumeric =>
+ withResource(input.matchesRe(jsonNumberRegexp)) { isJsonNumber =>
+ withResource(input.contains(nonNumeric)) { nonNumeric =>
+ isJsonNumber.or(nonNumeric)
+ }
+ }
+ }
+ } else {
+ input.matchesRe(jsonNumberRegexp)
+ }
+ withResource(isValid) { _ =>
+ withResource(Scalar.fromNull(DType.STRING)) { nullString =>
+ isValid.ifElse(input, nullString)
+ }
+ }
+ }
+
+ override def castStringToFloat(input: ColumnVector, dt: DType): ColumnVector = {
+ withResource(sanitizeNumbers(input)) { sanitizedInput =>
+ super.castStringToFloat(sanitizedInput, dt)
+ }
+ }
+
+ override def castStringToDecimal(input: ColumnVector, dt: DecimalType): ColumnVector = {
+ withResource(sanitizeNumbers(input)) { sanitizedInput =>
+ super.castStringToDecimal(sanitizedInput, dt)
+ }
+ }
+
}
diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/unit/DecimalUnitTest.scala b/tests/src/test/scala/com/nvidia/spark/rapids/unit/DecimalUnitTest.scala
index 79ded960a0a..98e145bec57 100644
--- a/tests/src/test/scala/com/nvidia/spark/rapids/unit/DecimalUnitTest.scala
+++ b/tests/src/test/scala/com/nvidia/spark/rapids/unit/DecimalUnitTest.scala
@@ -26,8 +26,6 @@ import com.nvidia.spark.rapids.{GpuAlias, GpuBatchScanExec, GpuColumnVector, Gpu
import org.apache.spark.SparkConf
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Literal}
-import org.apache.spark.sql.execution.FileSourceScanExec
-import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.rapids.GpuFileSourceScanExec
import org.apache.spark.sql.types.{Decimal, DecimalType, IntegerType, LongType, StructField, StructType}
@@ -257,7 +255,7 @@ class DecimalUnitTest extends GpuUnitTests {
var rootPlan = frameFromOrc("decimal-test.orc")(ss).queryExecution.executedPlan
assert(rootPlan.map(p => p).exists(_.isInstanceOf[GpuFileSourceScanExec]))
rootPlan = fromCsvDf("decimal-test.csv", decimalCsvStruct)(ss).queryExecution.executedPlan
- assert(rootPlan.map(p => p).exists(_.isInstanceOf[FileSourceScanExec]))
+ assert(rootPlan.map(p => p).exists(_.isInstanceOf[GpuFileSourceScanExec]))
rootPlan = frameFromParquet("decimal-test.parquet")(ss).queryExecution.executedPlan
assert(rootPlan.map(p => p).exists(_.isInstanceOf[GpuFileSourceScanExec]))
}, conf)
@@ -266,7 +264,7 @@ class DecimalUnitTest extends GpuUnitTests {
var rootPlan = frameFromOrc("decimal-test.orc")(ss).queryExecution.executedPlan
assert(rootPlan.map(p => p).exists(_.isInstanceOf[GpuBatchScanExec]))
rootPlan = fromCsvDf("decimal-test.csv", decimalCsvStruct)(ss).queryExecution.executedPlan
- assert(rootPlan.map(p => p).exists(_.isInstanceOf[BatchScanExec]))
+ assert(rootPlan.map(p => p).exists(_.isInstanceOf[GpuBatchScanExec]))
rootPlan = frameFromParquet("decimal-test.parquet")(ss).queryExecution.executedPlan
assert(rootPlan.map(p => p).exists(_.isInstanceOf[GpuBatchScanExec]))
}, conf.set(SQLConf.USE_V1_SOURCE_LIST.key, ""))
diff --git a/tools/src/main/resources/supportedDataSource.csv b/tools/src/main/resources/supportedDataSource.csv
index 4a42eede67c..81833809620 100644
--- a/tools/src/main/resources/supportedDataSource.csv
+++ b/tools/src/main/resources/supportedDataSource.csv
@@ -1,5 +1,5 @@
Format,Direction,BOOLEAN,BYTE,SHORT,INT,LONG,FLOAT,DOUBLE,DATE,TIMESTAMP,STRING,DECIMAL,NULL,BINARY,CALENDAR,ARRAY,MAP,STRUCT,UDT
-CSV,read,S,S,S,S,S,S,S,CO,CO,S,NS,NA,NS,NA,NA,NA,NA,NA
+CSV,read,S,S,S,S,S,S,S,CO,CO,S,S,NA,NS,NA,NA,NA,NA,NA
JSON,read,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO,CO
ORC,read,S,S,S,S,S,S,S,S,PS,S,S,NA,NS,NA,PS,PS,PS,NS
ORC,write,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA