Skip to content

Instantly share code, notes, and snippets.

@zhouyuan
Created September 21, 2023 02:48
Show Gist options
  • Save zhouyuan/0232b2a085571e5023699888d0de164d to your computer and use it in GitHub Desktop.
Save zhouyuan/0232b2a085571e5023699888d0de164d to your computer and use it in GitHub Desktop.
This file has been truncated, but you can view the full file.
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/java/org/apache/spark/sql/api/java/UDF23Test.java sql/core/src/test/java/org/apache/spark/sql/api/java/UDF23Test.java
--- /mnt/nvme1/git/spark/sql/core/src/test/java/org/apache/spark/sql/api/java/UDF23Test.java 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/java/org/apache/spark/sql/api/java/UDF23Test.java 2023-09-18 15:00:42.998708419 +0800
@@ -0,0 +1,35 @@
+/*
+ * 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.api.java;
+
+import java.io.Serializable;
+
+import org.apache.spark.annotation.Stable;
+
+/**
+ * A Spark SQL UDF that has 23 arguments for test.
+ */
+@Stable
+public interface UDF23Test<T1, T2, T3, T4, T5, T6, T7, T8, T9, T10, T11, T12,
+ T13, T14, T15, T16, T17, T18, T19, T20, T21, T22, T23, R> extends Serializable {
+
+ R call(
+ T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10,
+ T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18,
+ T19 t19, T20 t20, T21 t21, T22 t22, T23 t23) throws Exception;
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaOrderAndPartitionAwareDataSource.java sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaOrderAndPartitionAwareDataSource.java
--- /mnt/nvme1/git/spark/sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaOrderAndPartitionAwareDataSource.java 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/java/test/org/apache/spark/sql/connector/JavaOrderAndPartitionAwareDataSource.java 2023-09-18 15:00:42.998708419 +0800
@@ -0,0 +1,160 @@
+/*
+ * 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 test.org.apache.spark.sql.connector;
+
+import java.util.Arrays;
+
+import org.apache.spark.sql.connector.catalog.Table;
+import org.apache.spark.sql.connector.expressions.*;
+import org.apache.spark.sql.connector.read.*;
+import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+public class JavaOrderAndPartitionAwareDataSource extends JavaPartitionAwareDataSource {
+
+ static class MyScanBuilder extends JavaPartitionAwareDataSource.MyScanBuilder
+ implements SupportsReportOrdering {
+
+ private final Partitioning partitioning;
+ private final SortOrder[] ordering;
+
+ MyScanBuilder(String partitionKeys, String orderKeys) {
+ if (partitionKeys != null) {
+ String[] keys = partitionKeys.split(",");
+ Expression[] clustering = new Transform[keys.length];
+ for (int i = 0; i < keys.length; i++) {
+ clustering[i] = Expressions.identity(keys[i]);
+ }
+ this.partitioning = new KeyGroupedPartitioning(clustering, 2);
+ } else {
+ this.partitioning = new UnknownPartitioning(2);
+ }
+
+ if (orderKeys != null) {
+ String[] keys = orderKeys.split(",");
+ this.ordering = new SortOrder[keys.length];
+ for (int i = 0; i < keys.length; i++) {
+ this.ordering[i] = new MySortOrder(keys[i]);
+ }
+ } else {
+ this.ordering = new SortOrder[0];
+ }
+ }
+
+ @Override
+ public InputPartition[] planInputPartitions() {
+ InputPartition[] partitions = new InputPartition[2];
+ partitions[0] = new SpecificInputPartition(new int[]{1, 1, 3}, new int[]{4, 5, 5});
+ partitions[1] = new SpecificInputPartition(new int[]{2, 4, 4}, new int[]{6, 1, 2});
+ return partitions;
+ }
+
+ @Override
+ public Partitioning outputPartitioning() {
+ return this.partitioning;
+ }
+
+ @Override
+ public SortOrder[] outputOrdering() {
+ return this.ordering;
+ }
+ }
+
+ @Override
+ public Table getTable(CaseInsensitiveStringMap options) {
+ return new JavaSimpleBatchTable() {
+ @Override
+ public Transform[] partitioning() {
+ String partitionKeys = options.get("partitionKeys");
+ if (partitionKeys == null) {
+ return new Transform[0];
+ } else {
+ return (Transform[]) Arrays.stream(partitionKeys.split(","))
+ .map(Expressions::identity).toArray();
+ }
+ }
+
+ @Override
+ public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) {
+ return new MyScanBuilder(options.get("partitionKeys"), options.get("orderKeys"));
+ }
+ };
+ }
+
+ static class MySortOrder implements SortOrder {
+ private final Expression expression;
+
+ MySortOrder(String columnName) {
+ this.expression = new MyIdentityTransform(new MyNamedReference(columnName));
+ }
+
+ @Override
+ public Expression expression() {
+ return expression;
+ }
+
+ @Override
+ public SortDirection direction() {
+ return SortDirection.ASCENDING;
+ }
+
+ @Override
+ public NullOrdering nullOrdering() {
+ return NullOrdering.NULLS_FIRST;
+ }
+ }
+
+ static class MyNamedReference implements NamedReference {
+ private final String[] parts;
+
+ MyNamedReference(String part) {
+ this.parts = new String[] { part };
+ }
+
+ @Override
+ public String[] fieldNames() {
+ return this.parts;
+ }
+ }
+
+ static class MyIdentityTransform implements Transform {
+ private final Expression[] args;
+
+ MyIdentityTransform(NamedReference namedReference) {
+ this.args = new Expression[] { namedReference };
+ }
+
+ @Override
+ public String name() {
+ return "identity";
+ }
+
+ @Override
+ public NamedReference[] references() {
+ return new NamedReference[0];
+ }
+
+ @Override
+ public Expression[] arguments() {
+ return this.args;
+ }
+ }
+
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java
--- /mnt/nvme1/git/spark/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java 2023-09-21 10:45:41.951461262 +0800
+++ sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java 2023-09-18 15:00:42.998708419 +0800
@@ -186,18 +186,9 @@
Dataset<Row> dataFrame = spark.createDataFrame(inputRows, schema);
- try {
- dataFrame.as(encoder).collect();
- Assert.fail("Expected AnalysisException, but passed.");
- } catch (Throwable e) {
- // Here we need to handle weird case: compiler complains AnalysisException never be thrown
- // in try statement, but it can be thrown actually. Maybe Scala-Java interop issue?
- if (e instanceof AnalysisException) {
- Assert.assertTrue(e.getMessage().contains("Cannot up cast "));
- } else {
- throw e;
- }
- }
+ AnalysisException e = Assert.assertThrows(AnalysisException.class,
+ () -> dataFrame.as(encoder).collect());
+ Assert.assertTrue(e.getMessage().contains("Cannot up cast "));
}
private static Row createRecordSpark22000Row(Long index) {
@@ -599,9 +590,9 @@
.reduceGroups(rf);
List<Tuple2<String, Item>> expectedRecords = Arrays.asList(
- new Tuple2("a", new Item("a", 8)),
- new Tuple2("b", new Item("b", 3)),
- new Tuple2("c", new Item("c", 2)));
+ new Tuple2<>("a", new Item("a", 8)),
+ new Tuple2<>("b", new Item("b", 3)),
+ new Tuple2<>("c", new Item("c", 2)));
List<Tuple2<String, Item>> result = finalDs.collectAsList();
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java
--- /mnt/nvme1/git/spark/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java 2023-09-21 10:45:41.951461262 +0800
+++ sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java 2023-09-18 15:00:42.998708419 +0800
@@ -17,19 +17,23 @@
package test.org.apache.spark.sql;
+import java.util.*;
+
+import com.google.common.collect.Maps;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
import org.apache.spark.api.java.function.FilterFunction;
+import org.apache.spark.sql.AnalysisException;
import org.apache.spark.sql.Column;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.RowFactory;
import org.apache.spark.sql.test.TestSparkSession;
import org.apache.spark.sql.types.StructType;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.*;
import static org.apache.spark.sql.types.DataTypes.*;
@@ -79,14 +83,13 @@
createStructField("a", IntegerType, false),
createStructField("b", createArrayType(IntegerType, false), false)));
Dataset<Row> df = spark.createDataFrame(rows, schema);
- try {
- df.filter(df.col("a").isInCollection(Arrays.asList(new Column("b"))));
- Assert.fail("Expected org.apache.spark.sql.AnalysisException");
- } catch (Exception e) {
- Arrays.asList("cannot resolve",
- "due to data type mismatch: Arguments must be same type but were")
- .forEach(s -> Assert.assertTrue(
- e.getMessage().toLowerCase(Locale.ROOT).contains(s.toLowerCase(Locale.ROOT))));
- }
+ AnalysisException e = Assert.assertThrows(AnalysisException.class,
+ () -> df.filter(df.col("a").isInCollection(Arrays.asList(new Column("b")))));
+ Assert.assertTrue(e.getErrorClass().equals("DATATYPE_MISMATCH.DATA_DIFF_TYPES"));
+ Map<String, String> messageParameters = new HashMap<>();
+ messageParameters.put("functionName", "`in`");
+ messageParameters.put("dataType", "[\"INT\", \"ARRAY<INT>\"]");
+ messageParameters.put("sqlExpr", "\"(a IN (b))\"");
+ Assert.assertTrue(Maps.difference(e.getMessageParameters(), messageParameters).areEqual());
}
}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java
--- /mnt/nvme1/git/spark/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java 2023-09-21 10:45:41.951461262 +0800
+++ sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java 2023-09-18 15:00:42.998708419 +0800
@@ -523,10 +523,11 @@
// Checks a simple case for DataFrame here and put exhaustive tests for the issue
// of circular references in `JavaDatasetSuite`.
- @Test(expected = UnsupportedOperationException.class)
+ @Test
public void testCircularReferenceBean() {
CircularReference1Bean bean = new CircularReference1Bean();
- spark.createDataFrame(Arrays.asList(bean), CircularReference1Bean.class);
+ Assert.assertThrows(UnsupportedOperationException.class,
+ () -> spark.createDataFrame(Arrays.asList(bean), CircularReference1Bean.class));
}
@Test
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java
--- /mnt/nvme1/git/spark/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java 2023-09-21 10:45:41.951461262 +0800
+++ sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java 2023-09-18 15:00:42.998708419 +0800
@@ -302,7 +302,7 @@
}
@Test
- public void testGroupBy() {
+ public void testGroupByKey() {
List<String> data = Arrays.asList("a", "foo", "bar");
Dataset<String> ds = spark.createDataset(data, Encoders.STRING());
KeyValueGroupedDataset<Integer, String> grouped =
@@ -327,9 +327,21 @@
}
return Collections.singletonList(sb.toString()).iterator();
},
- Encoders.STRING());
+ Encoders.STRING());
Assert.assertEquals(asSet("1a", "3foobar"), toSet(flatMapped.collectAsList()));
+ Dataset<String> flatMapSorted = grouped.flatMapSortedGroups(
+ new Column[] { ds.col("value") },
+ (FlatMapGroupsFunction<Integer, String, String>) (key, values) -> {
+ StringBuilder sb = new StringBuilder(key.toString());
+ while (values.hasNext()) {
+ sb.append(values.next());
+ }
+ return Collections.singletonList(sb.toString()).iterator();
+ },
+ Encoders.STRING());
+
+ Assert.assertEquals(asSet("1a", "3barfoo"), toSet(flatMapSorted.collectAsList()));
Dataset<String> mapped2 = grouped.mapGroupsWithState(
(MapGroupsWithStateFunction<Integer, String, Long, String>) (key, values, s) -> {
@@ -352,10 +364,10 @@
}
return Collections.singletonList(sb.toString()).iterator();
},
- OutputMode.Append(),
- Encoders.LONG(),
- Encoders.STRING(),
- GroupStateTimeout.NoTimeout());
+ OutputMode.Append(),
+ Encoders.LONG(),
+ Encoders.STRING(),
+ GroupStateTimeout.NoTimeout());
Assert.assertEquals(asSet("1a", "3foobar"), toSet(flatMapped2.collectAsList()));
@@ -366,7 +378,7 @@
asSet(tuple2(1, "a"), tuple2(3, "foobar")),
toSet(reduced.collectAsList()));
- List<Integer> data2 = Arrays.asList(2, 6, 10);
+ List<Integer> data2 = Arrays.asList(2, 6, 7, 10);
Dataset<Integer> ds2 = spark.createDataset(data2, Encoders.INT());
KeyValueGroupedDataset<Integer, Integer> grouped2 = ds2.groupByKey(
(MapFunction<Integer, Integer>) v -> v / 2,
@@ -387,7 +399,26 @@
},
Encoders.STRING());
- Assert.assertEquals(asSet("1a#2", "3foobar#6", "5#10"), toSet(cogrouped.collectAsList()));
+ Assert.assertEquals(asSet("1a#2", "3foobar#67", "5#10"), toSet(cogrouped.collectAsList()));
+
+ Dataset<String> cogroupSorted = grouped.cogroupSorted(
+ grouped2,
+ new Column[] { ds.col("value") },
+ new Column[] { ds2.col("value").desc() },
+ (CoGroupFunction<Integer, String, Integer, String>) (key, left, right) -> {
+ StringBuilder sb = new StringBuilder(key.toString());
+ while (left.hasNext()) {
+ sb.append(left.next());
+ }
+ sb.append("#");
+ while (right.hasNext()) {
+ sb.append(right.next());
+ }
+ return Collections.singletonList(sb.toString()).iterator();
+ },
+ Encoders.STRING());
+
+ Assert.assertEquals(asSet("1a#2", "3barfoo#76", "5#10"), toSet(cogroupSorted.collectAsList()));
}
@Test
@@ -608,6 +639,14 @@
}
@Test
+ public void testLocalDateTimeEncoder() {
+ Encoder<LocalDateTime> encoder = Encoders.LOCALDATETIME();
+ List<LocalDateTime> data = Arrays.asList(LocalDateTime.of(1, 1, 1, 1, 1));
+ Dataset<LocalDateTime> ds = spark.createDataset(data, encoder);
+ Assert.assertEquals(data, ds.collectAsList());
+ }
+
+ @Test
public void testDurationEncoder() {
Encoder<Duration> encoder = Encoders.DURATION();
List<Duration> data = Arrays.asList(Duration.ofDays(0));
@@ -699,14 +738,16 @@
*/
private static class PrivateClassTest { }
- @Test(expected = UnsupportedOperationException.class)
+ @Test
public void testJavaEncoderErrorMessageForPrivateClass() {
- Encoders.javaSerialization(PrivateClassTest.class);
+ Assert.assertThrows(UnsupportedOperationException.class,
+ () -> Encoders.javaSerialization(PrivateClassTest.class));
}
- @Test(expected = UnsupportedOperationException.class)
+ @Test
public void testKryoEncoderErrorMessageForPrivateClass() {
- Encoders.kryo(PrivateClassTest.class);
+ Assert.assertThrows(UnsupportedOperationException.class,
+ () -> Encoders.kryo(PrivateClassTest.class));
}
public static class SimpleJavaBean implements Serializable {
@@ -1739,29 +1780,33 @@
}
}
- @Test(expected = UnsupportedOperationException.class)
+ @Test
public void testCircularReferenceBean1() {
CircularReference1Bean bean = new CircularReference1Bean();
- spark.createDataset(Arrays.asList(bean), Encoders.bean(CircularReference1Bean.class));
+ Assert.assertThrows(UnsupportedOperationException.class,
+ () -> spark.createDataset(Arrays.asList(bean), Encoders.bean(CircularReference1Bean.class)));
}
- @Test(expected = UnsupportedOperationException.class)
+ @Test
public void testCircularReferenceBean2() {
CircularReference3Bean bean = new CircularReference3Bean();
- spark.createDataset(Arrays.asList(bean), Encoders.bean(CircularReference3Bean.class));
+ Assert.assertThrows(UnsupportedOperationException.class,
+ () -> spark.createDataset(Arrays.asList(bean), Encoders.bean(CircularReference3Bean.class)));
}
- @Test(expected = UnsupportedOperationException.class)
+ @Test
public void testCircularReferenceBean3() {
CircularReference4Bean bean = new CircularReference4Bean();
- spark.createDataset(Arrays.asList(bean), Encoders.bean(CircularReference4Bean.class));
+ Assert.assertThrows(UnsupportedOperationException.class,
+ () -> spark.createDataset(Arrays.asList(bean), Encoders.bean(CircularReference4Bean.class)));
}
- @Test(expected = RuntimeException.class)
+ @Test
public void testNullInTopLevelBean() {
NestedSmallBean bean = new NestedSmallBean();
// We cannot set null in top-level bean
- spark.createDataset(Arrays.asList(bean, null), Encoders.bean(NestedSmallBean.class));
+ Assert.assertThrows(RuntimeException.class,
+ () -> spark.createDataset(Arrays.asList(bean, null), Encoders.bean(NestedSmallBean.class)));
}
@Test
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/java/test/org/apache/spark/sql/JavaSparkSessionSuite.java sql/core/src/test/java/test/org/apache/spark/sql/JavaSparkSessionSuite.java
--- /mnt/nvme1/git/spark/sql/core/src/test/java/test/org/apache/spark/sql/JavaSparkSessionSuite.java 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/java/test/org/apache/spark/sql/JavaSparkSessionSuite.java 2023-09-18 15:00:42.998708419 +0800
@@ -0,0 +1,57 @@
+/*
+ * 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 test.org.apache.spark.sql;
+
+import org.apache.spark.sql.*;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class JavaSparkSessionSuite {
+ private SparkSession spark;
+
+ @After
+ public void tearDown() {
+ spark.stop();
+ spark = null;
+ }
+
+ @Test
+ public void config() {
+ // SPARK-40163: SparkSession.config(Map)
+ Map<String, Object> map = new HashMap<String, Object>() {{
+ put("string", "");
+ put("boolean", true);
+ put("double", 0.0);
+ put("long", 0L);
+ }};
+
+ spark = SparkSession.builder()
+ .master("local[*]")
+ .appName("testing")
+ .config(map)
+ .getOrCreate();
+
+ for (Map.Entry<String, Object> e : map.entrySet()) {
+ Assert.assertEquals(spark.conf().get(e.getKey()), e.getValue().toString());
+ }
+ }
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java
--- /mnt/nvme1/git/spark/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java 2023-09-21 10:45:41.951461262 +0800
+++ sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java 2023-09-18 15:00:42.998708419 +0800
@@ -107,10 +107,11 @@
Assert.assertEquals(55, sum);
}
- @Test(expected = AnalysisException.class)
+ @Test
public void udf5Test() {
spark.udf().register("inc", (Long i) -> i + 1, DataTypes.LongType);
- List<Row> results = spark.sql("SELECT inc(1, 5)").collectAsList();
+ Assert.assertThrows(AnalysisException.class,
+ () -> spark.sql("SELECT inc(1, 5)").collectAsList());
}
@Test
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-functions/sql-expression-schema.md sql/core/src/test/resources/sql-functions/sql-expression-schema.md
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-functions/sql-expression-schema.md 2023-09-21 10:45:41.951461262 +0800
+++ sql/core/src/test/resources/sql-functions/sql-expression-schema.md 2023-09-18 15:00:42.998708419 +0800
@@ -1,8 +1,4 @@
<!-- Automatically generated by ExpressionsSchemaSuite -->
-## Summary
- - Number of queries: 388
- - Number of expressions that missing example: 12
- - Expressions missing examples: bigint,binary,boolean,date,decimal,double,float,int,smallint,string,timestamp,tinyint
## Schema of Built-in Functions
| Class name | Function name or alias | Query example | Output schema |
| ---------- | ---------------------- | ------------- | ------------- |
@@ -15,12 +11,16 @@
| org.apache.spark.sql.catalyst.expressions.AesEncrypt | aes_encrypt | SELECT hex(aes_encrypt('Spark', '0000111122223333')) | struct<hex(aes_encrypt(Spark, 0000111122223333, GCM, DEFAULT)):string> |
| org.apache.spark.sql.catalyst.expressions.And | and | SELECT true and true | struct<(true AND true):boolean> |
| org.apache.spark.sql.catalyst.expressions.ArrayAggregate | aggregate | SELECT aggregate(array(1, 2, 3), 0, (acc, x) -> acc + x) | struct<aggregate(array(1, 2, 3), 0, lambdafunction((namedlambdavariable() + namedlambdavariable()), namedlambdavariable(), namedlambdavariable()), lambdafunction(namedlambdavariable(), namedlambdavariable())):int> |
+| org.apache.spark.sql.catalyst.expressions.ArrayAggregate | reduce | SELECT reduce(array(1, 2, 3), 0, (acc, x) -> acc + x) | struct<reduce(array(1, 2, 3), 0, lambdafunction((namedlambdavariable() + namedlambdavariable()), namedlambdavariable(), namedlambdavariable()), lambdafunction(namedlambdavariable(), namedlambdavariable())):int> |
+| org.apache.spark.sql.catalyst.expressions.ArrayAppend | array_append | SELECT array_append(array('b', 'd', 'c', 'a'), 'd') | struct<array_append(array(b, d, c, a), d):array<string>> |
+| org.apache.spark.sql.catalyst.expressions.ArrayCompact | array_compact | SELECT array_compact(array(1, 2, 3, null)) | struct<array_compact(array(1, 2, 3, NULL)):array<int>> |
| org.apache.spark.sql.catalyst.expressions.ArrayContains | array_contains | SELECT array_contains(array(1, 2, 3), 2) | struct<array_contains(array(1, 2, 3), 2):boolean> |
| org.apache.spark.sql.catalyst.expressions.ArrayDistinct | array_distinct | SELECT array_distinct(array(1, 2, 3, null, 3)) | struct<array_distinct(array(1, 2, 3, NULL, 3)):array<int>> |
| org.apache.spark.sql.catalyst.expressions.ArrayExcept | array_except | SELECT array_except(array(1, 2, 3), array(1, 3, 5)) | struct<array_except(array(1, 2, 3), array(1, 3, 5)):array<int>> |
| org.apache.spark.sql.catalyst.expressions.ArrayExists | exists | SELECT exists(array(1, 2, 3), x -> x % 2 == 0) | struct<exists(array(1, 2, 3), lambdafunction(((namedlambdavariable() % 2) = 0), namedlambdavariable())):boolean> |
| org.apache.spark.sql.catalyst.expressions.ArrayFilter | filter | SELECT filter(array(1, 2, 3), x -> x % 2 == 1) | struct<filter(array(1, 2, 3), lambdafunction(((namedlambdavariable() % 2) = 1), namedlambdavariable())):array<int>> |
| org.apache.spark.sql.catalyst.expressions.ArrayForAll | forall | SELECT forall(array(1, 2, 3), x -> x % 2 == 0) | struct<forall(array(1, 2, 3), lambdafunction(((namedlambdavariable() % 2) = 0), namedlambdavariable())):boolean> |
+| org.apache.spark.sql.catalyst.expressions.ArrayInsert | array_insert | SELECT array_insert(array(1, 2, 3, 4), 5, 5) | struct<array_insert(array(1, 2, 3, 4), 5, 5):array<int>> |
| org.apache.spark.sql.catalyst.expressions.ArrayIntersect | array_intersect | SELECT array_intersect(array(1, 2, 3), array(1, 3, 5)) | struct<array_intersect(array(1, 2, 3), array(1, 3, 5)):array<int>> |
| org.apache.spark.sql.catalyst.expressions.ArrayJoin | array_join | SELECT array_join(array('hello', 'world'), ' ') | struct<array_join(array(hello, world), ):string> |
| org.apache.spark.sql.catalyst.expressions.ArrayMax | array_max | SELECT array_max(array(1, 20, null, 3)) | struct<array_max(array(1, 20, NULL, 3)):int> |
@@ -90,17 +90,23 @@
| org.apache.spark.sql.catalyst.expressions.Csc | csc | SELECT csc(1) | struct<CSC(1):double> |
| org.apache.spark.sql.catalyst.expressions.CsvToStructs | from_csv | SELECT from_csv('1, 0.8', 'a INT, b DOUBLE') | struct<from_csv(1, 0.8):struct<a:int,b:double>> |
| org.apache.spark.sql.catalyst.expressions.CumeDist | cume_dist | SELECT a, b, cume_dist() OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b) | struct<a:string,b:int,cume_dist() OVER (PARTITION BY a ORDER BY b ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW):double> |
+| org.apache.spark.sql.catalyst.expressions.CurDateExpressionBuilder | curdate | SELECT curdate() | struct<current_date():date> |
| org.apache.spark.sql.catalyst.expressions.CurrentCatalog | current_catalog | SELECT current_catalog() | struct<current_catalog():string> |
| org.apache.spark.sql.catalyst.expressions.CurrentDatabase | current_database | SELECT current_database() | struct<current_database():string> |
+| org.apache.spark.sql.catalyst.expressions.CurrentDatabase | current_schema | SELECT current_schema() | struct<current_database():string> |
| org.apache.spark.sql.catalyst.expressions.CurrentDate | current_date | SELECT current_date() | struct<current_date():date> |
| org.apache.spark.sql.catalyst.expressions.CurrentTimeZone | current_timezone | SELECT current_timezone() | struct<current_timezone():string> |
| org.apache.spark.sql.catalyst.expressions.CurrentTimestamp | current_timestamp | SELECT current_timestamp() | struct<current_timestamp():timestamp> |
| org.apache.spark.sql.catalyst.expressions.CurrentUser | current_user | SELECT current_user() | struct<current_user():string> |
+| org.apache.spark.sql.catalyst.expressions.CurrentUser | user | SELECT user() | struct<current_user():string> |
| org.apache.spark.sql.catalyst.expressions.DateAdd | date_add | SELECT date_add('2016-07-30', 1) | struct<date_add(2016-07-30, 1):date> |
+| org.apache.spark.sql.catalyst.expressions.DateAdd | dateadd | SELECT dateadd('2016-07-30', 1) | struct<date_add(2016-07-30, 1):date> |
+| org.apache.spark.sql.catalyst.expressions.DateDiff | date_diff | SELECT date_diff('2009-07-31', '2009-07-30') | struct<date_diff(2009-07-31, 2009-07-30):int> |
| org.apache.spark.sql.catalyst.expressions.DateDiff | datediff | SELECT datediff('2009-07-31', '2009-07-30') | struct<datediff(2009-07-31, 2009-07-30):int> |
| org.apache.spark.sql.catalyst.expressions.DateFormatClass | date_format | SELECT date_format('2016-04-08', 'y') | struct<date_format(2016-04-08, y):string> |
| org.apache.spark.sql.catalyst.expressions.DateFromUnixDate | date_from_unix_date | SELECT date_from_unix_date(1) | struct<date_from_unix_date(1):date> |
| org.apache.spark.sql.catalyst.expressions.DatePartExpressionBuilder | date_part | SELECT date_part('YEAR', TIMESTAMP '2019-08-12 01:00:00.123456') | struct<date_part(YEAR, TIMESTAMP '2019-08-12 01:00:00.123456'):int> |
+| org.apache.spark.sql.catalyst.expressions.DatePartExpressionBuilder | datepart | SELECT datepart('YEAR', TIMESTAMP '2019-08-12 01:00:00.123456') | struct<datepart(YEAR FROM TIMESTAMP '2019-08-12 01:00:00.123456'):int> |
| org.apache.spark.sql.catalyst.expressions.DateSub | date_sub | SELECT date_sub('2016-07-30', 1) | struct<date_sub(2016-07-30, 1):date> |
| org.apache.spark.sql.catalyst.expressions.DayOfMonth | day | SELECT day('2009-07-30') | struct<day(2009-07-30):int> |
| org.apache.spark.sql.catalyst.expressions.DayOfMonth | dayofmonth | SELECT dayofmonth('2009-07-30') | struct<dayofmonth(2009-07-30):int> |
@@ -113,6 +119,7 @@
| org.apache.spark.sql.catalyst.expressions.Elt | elt | SELECT elt(1, 'scala', 'java') | struct<elt(1, scala, java):string> |
| org.apache.spark.sql.catalyst.expressions.Encode | encode | SELECT encode('abc', 'utf-8') | struct<encode(abc, utf-8):binary> |
| org.apache.spark.sql.catalyst.expressions.EndsWithExpressionBuilder | endswith | SELECT endswith('Spark SQL', 'SQL') | struct<endswith(Spark SQL, SQL):boolean> |
+| org.apache.spark.sql.catalyst.expressions.EqualNull | equal_null | SELECT equal_null(3, 3) | struct<equal_null(3, 3):boolean> |
| org.apache.spark.sql.catalyst.expressions.EqualNullSafe | <=> | SELECT 2 <=> 2 | struct<(2 <=> 2):boolean> |
| org.apache.spark.sql.catalyst.expressions.EqualTo | = | SELECT 2 = 2 | struct<(2 = 2):boolean> |
| org.apache.spark.sql.catalyst.expressions.EqualTo | == | SELECT 2 == 2 | struct<(2 = 2):boolean> |
@@ -131,6 +138,7 @@
| org.apache.spark.sql.catalyst.expressions.FormatString | printf | SELECT printf("Hello World %d %s", 100, "days") | struct<printf(Hello World %d %s, 100, days):string> |
| org.apache.spark.sql.catalyst.expressions.FromUTCTimestamp | from_utc_timestamp | SELECT from_utc_timestamp('2016-08-31', 'Asia/Seoul') | struct<from_utc_timestamp(2016-08-31, Asia/Seoul):timestamp> |
| org.apache.spark.sql.catalyst.expressions.FromUnixTime | from_unixtime | SELECT from_unixtime(0, 'yyyy-MM-dd HH:mm:ss') | struct<from_unixtime(0, yyyy-MM-dd HH:mm:ss):string> |
+| org.apache.spark.sql.catalyst.expressions.Get | get | SELECT get(array(1, 2, 3), 0) | struct<get(array(1, 2, 3), 0):int> |
| org.apache.spark.sql.catalyst.expressions.GetJsonObject | get_json_object | SELECT get_json_object('{"a":"b"}', '$.a') | struct<get_json_object({"a":"b"}, $.a):string> |
| org.apache.spark.sql.catalyst.expressions.GreaterThan | > | SELECT 2 > 1 | struct<(2 > 1):boolean> |
| org.apache.spark.sql.catalyst.expressions.GreaterThanOrEqual | >= | SELECT 2 >= 1 | struct<(2 >= 1):boolean> |
@@ -164,6 +172,7 @@
| org.apache.spark.sql.catalyst.expressions.Left | left | SELECT left('Spark SQL', 3) | struct<left(Spark SQL, 3):string> |
| org.apache.spark.sql.catalyst.expressions.Length | char_length | SELECT char_length('Spark SQL ') | struct<char_length(Spark SQL ):int> |
| org.apache.spark.sql.catalyst.expressions.Length | character_length | SELECT character_length('Spark SQL ') | struct<character_length(Spark SQL ):int> |
+| org.apache.spark.sql.catalyst.expressions.Length | len | SELECT len('Spark SQL ') | struct<len(Spark SQL ):int> |
| org.apache.spark.sql.catalyst.expressions.Length | length | SELECT length('Spark SQL ') | struct<length(Spark SQL ):int> |
| org.apache.spark.sql.catalyst.expressions.LengthOfJsonArray | json_array_length | SELECT json_array_length('[1,2,3,4]') | struct<json_array_length([1,2,3,4]):int> |
| org.apache.spark.sql.catalyst.expressions.LessThan | < | SELECT 1 < 2 | struct<(1 < 2):boolean> |
@@ -194,6 +203,7 @@
| org.apache.spark.sql.catalyst.expressions.MapKeys | map_keys | SELECT map_keys(map(1, 'a', 2, 'b')) | struct<map_keys(map(1, a, 2, b)):array<int>> |
| org.apache.spark.sql.catalyst.expressions.MapValues | map_values | SELECT map_values(map(1, 'a', 2, 'b')) | struct<map_values(map(1, a, 2, b)):array<string>> |
| org.apache.spark.sql.catalyst.expressions.MapZipWith | map_zip_with | SELECT map_zip_with(map(1, 'a', 2, 'b'), map(1, 'x', 2, 'y'), (k, v1, v2) -> concat(v1, v2)) | struct<map_zip_with(map(1, a, 2, b), map(1, x, 2, y), lambdafunction(concat(namedlambdavariable(), namedlambdavariable()), namedlambdavariable(), namedlambdavariable(), namedlambdavariable())):map<int,string>> |
+| org.apache.spark.sql.catalyst.expressions.Mask | mask | SELECT mask('abcd-EFGH-8765-4321') | struct<mask(abcd-EFGH-8765-4321, X, x, n, NULL):string> |
| org.apache.spark.sql.catalyst.expressions.Md5 | md5 | SELECT md5('Spark') | struct<md5(Spark):string> |
| org.apache.spark.sql.catalyst.expressions.MicrosToTimestamp | timestamp_micros | SELECT timestamp_micros(1230219000123123) | struct<timestamp_micros(1230219000123123):timestamp> |
| org.apache.spark.sql.catalyst.expressions.MillisToTimestamp | timestamp_millis | SELECT timestamp_millis(1230219000123) | struct<timestamp_millis(1230219000123):timestamp> |
@@ -239,9 +249,12 @@
| org.apache.spark.sql.catalyst.expressions.Rand | random | SELECT random() | struct<rand():double> |
| org.apache.spark.sql.catalyst.expressions.Randn | randn | SELECT randn() | struct<randn():double> |
| org.apache.spark.sql.catalyst.expressions.Rank | rank | SELECT a, b, rank(b) OVER (PARTITION BY a ORDER BY b) FROM VALUES ('A1', 2), ('A1', 1), ('A2', 3), ('A1', 1) tab(a, b) | struct<a:string,b:int,RANK() OVER (PARTITION BY a ORDER BY b ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW):int> |
+| org.apache.spark.sql.catalyst.expressions.RegExpCount | regexp_count | SELECT regexp_count('Steven Jones and Stephen Smith are the best players', 'Ste(v&#124;ph)en') | struct<regexp_count(Steven Jones and Stephen Smith are the best players, Ste(v&#124;ph)en):int> |
| org.apache.spark.sql.catalyst.expressions.RegExpExtract | regexp_extract | SELECT regexp_extract('100-200', '(\\d+)-(\\d+)', 1) | struct<regexp_extract(100-200, (\d+)-(\d+), 1):string> |
| org.apache.spark.sql.catalyst.expressions.RegExpExtractAll | regexp_extract_all | SELECT regexp_extract_all('100-200, 300-400', '(\\d+)-(\\d+)', 1) | struct<regexp_extract_all(100-200, 300-400, (\d+)-(\d+), 1):array<string>> |
+| org.apache.spark.sql.catalyst.expressions.RegExpInStr | regexp_instr | SELECT regexp_instr('user@spark.apache.org', '@[^.]*') | struct<regexp_instr(user@spark.apache.org, @[^.]*, 0):int> |
| org.apache.spark.sql.catalyst.expressions.RegExpReplace | regexp_replace | SELECT regexp_replace('100-200', '(\\d+)', 'num') | struct<regexp_replace(100-200, (\d+), num, 1):string> |
+| org.apache.spark.sql.catalyst.expressions.RegExpSubStr | regexp_substr | SELECT regexp_substr('Steven Jones and Stephen Smith are the best players', 'Ste(v&#124;ph)en') | struct<regexp_substr(Steven Jones and Stephen Smith are the best players, Ste(v&#124;ph)en):string> |
| org.apache.spark.sql.catalyst.expressions.Remainder | % | SELECT 2 % 1.8 | struct<(2 % 1.8):decimal(2,1)> |
| org.apache.spark.sql.catalyst.expressions.Remainder | mod | SELECT 2 % 1.8 | struct<(2 % 1.8):decimal(2,1)> |
| org.apache.spark.sql.catalyst.expressions.Reverse | reverse | SELECT reverse('Spark SQL') | struct<reverse(Spark SQL):string> |
@@ -302,6 +315,7 @@
| org.apache.spark.sql.catalyst.expressions.Tanh | tanh | SELECT tanh(0) | struct<TANH(0):double> |
| org.apache.spark.sql.catalyst.expressions.TimeWindow | window | SELECT a, window.start, window.end, count(*) as cnt FROM VALUES ('A1', '2021-01-01 00:00:00'), ('A1', '2021-01-01 00:04:30'), ('A1', '2021-01-01 00:06:00'), ('A2', '2021-01-01 00:01:00') AS tab(a, b) GROUP by a, window(b, '5 minutes') ORDER BY a, start | struct<a:string,start:timestamp,end:timestamp,cnt:bigint> |
| org.apache.spark.sql.catalyst.expressions.ToBinary | to_binary | SELECT to_binary('abc', 'utf-8') | struct<to_binary(abc, utf-8):binary> |
+| org.apache.spark.sql.catalyst.expressions.ToCharacter | to_char | SELECT to_char(454, '999') | struct<to_char(454, 999):string> |
| org.apache.spark.sql.catalyst.expressions.ToDegrees | degrees | SELECT degrees(3.141592653589793) | struct<DEGREES(3.141592653589793):double> |
| org.apache.spark.sql.catalyst.expressions.ToNumber | to_number | SELECT to_number('454', '999') | struct<to_number(454, 999):decimal(3,0)> |
| org.apache.spark.sql.catalyst.expressions.ToRadians | radians | SELECT radians(180) | struct<RADIANS(180):double> |
@@ -318,6 +332,7 @@
| org.apache.spark.sql.catalyst.expressions.TrySubtract | try_subtract | SELECT try_subtract(2, 1) | struct<try_subtract(2, 1):int> |
| org.apache.spark.sql.catalyst.expressions.TryToBinary | try_to_binary | SELECT try_to_binary('abc', 'utf-8') | struct<try_to_binary(abc, utf-8):binary> |
| org.apache.spark.sql.catalyst.expressions.TryToNumber | try_to_number | SELECT try_to_number('454', '999') | struct<try_to_number(454, 999):decimal(3,0)> |
+| org.apache.spark.sql.catalyst.expressions.TryToTimestampExpressionBuilder | try_to_timestamp | SELECT try_to_timestamp('2016-12-31 00:12:00') | struct<try_to_timestamp(2016-12-31 00:12:00):timestamp> |
| org.apache.spark.sql.catalyst.expressions.TypeOf | typeof | SELECT typeof(1) | struct<typeof(1):string> |
| org.apache.spark.sql.catalyst.expressions.UnBase64 | unbase64 | SELECT unbase64('U3BhcmsgU1FM') | struct<unbase64(U3BhcmsgU1FM):binary> |
| org.apache.spark.sql.catalyst.expressions.UnaryMinus | negative | SELECT negative(1) | struct<negative(1):int> |
@@ -330,13 +345,17 @@
| org.apache.spark.sql.catalyst.expressions.UnixTimestamp | unix_timestamp | SELECT unix_timestamp() | struct<unix_timestamp(current_timestamp(), yyyy-MM-dd HH:mm:ss):bigint> |
| org.apache.spark.sql.catalyst.expressions.Upper | ucase | SELECT ucase('SparkSql') | struct<ucase(SparkSql):string> |
| org.apache.spark.sql.catalyst.expressions.Upper | upper | SELECT upper('SparkSql') | struct<upper(SparkSql):string> |
+| org.apache.spark.sql.catalyst.expressions.UrlDecode | url_decode | SELECT url_decode('https%3A%2F%2Fspark.apache.org') | struct<url_decode(https%3A%2F%2Fspark.apache.org):string> |
+| org.apache.spark.sql.catalyst.expressions.UrlEncode | url_encode | SELECT url_encode('https://spark.apache.org') | struct<url_encode(https://spark.apache.org):string> |
| org.apache.spark.sql.catalyst.expressions.Uuid | uuid | SELECT uuid() | struct<uuid():string> |
| org.apache.spark.sql.catalyst.expressions.WeekDay | weekday | SELECT weekday('2009-07-30') | struct<weekday(2009-07-30):int> |
| org.apache.spark.sql.catalyst.expressions.WeekOfYear | weekofyear | SELECT weekofyear('2008-02-20') | struct<weekofyear(2008-02-20):int> |
| org.apache.spark.sql.catalyst.expressions.WidthBucket | width_bucket | SELECT width_bucket(5.3, 0.2, 10.6, 5) | struct<width_bucket(5.3, 0.2, 10.6, 5):bigint> |
+| org.apache.spark.sql.catalyst.expressions.WindowTime | window_time | SELECT a, window.start as start, window.end as end, window_time(window), cnt FROM (SELECT a, window, count(*) as cnt FROM VALUES ('A1', '2021-01-01 00:00:00'), ('A1', '2021-01-01 00:04:30'), ('A1', '2021-01-01 00:06:00'), ('A2', '2021-01-01 00:01:00') AS tab(a, b) GROUP by a, window(b, '5 minutes') ORDER BY a, window.start) | struct<a:string,start:timestamp,end:timestamp,window_time(window):timestamp,cnt:bigint> |
| org.apache.spark.sql.catalyst.expressions.XxHash64 | xxhash64 | SELECT xxhash64('Spark', array(123), 2) | struct<xxhash64(Spark, array(123), 2):bigint> |
| org.apache.spark.sql.catalyst.expressions.Year | year | SELECT year('2016-07-30') | struct<year(2016-07-30):int> |
| org.apache.spark.sql.catalyst.expressions.ZipWith | zip_with | SELECT zip_with(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, x)) | struct<zip_with(array(1, 2, 3), array(a, b, c), lambdafunction(named_struct(y, namedlambdavariable(), x, namedlambdavariable()), namedlambdavariable(), namedlambdavariable())):array<struct<y:string,x:int>>> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.AnyValue | any_value | SELECT any_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct<any_value(col):int> |
| org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile | approx_percentile | SELECT approx_percentile(col, array(0.5, 0.4, 0.1), 100) FROM VALUES (0), (1), (2), (10) AS tab(col) | struct<approx_percentile(col, array(0.5, 0.4, 0.1), 100):array<int>> |
| org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile | percentile_approx | SELECT percentile_approx(col, array(0.5, 0.4, 0.1), 100) FROM VALUES (0), (1), (2), (10) AS tab(col) | struct<percentile_approx(col, array(0.5, 0.4, 0.1), 100):array<int>> |
| org.apache.spark.sql.catalyst.expressions.aggregate.Average | avg | SELECT avg(col) FROM VALUES (1), (2), (3) AS tab(col) | struct<avg(col):double> |
@@ -367,21 +386,28 @@
| org.apache.spark.sql.catalyst.expressions.aggregate.Last | last_value | SELECT last_value(col) FROM VALUES (10), (5), (20) AS tab(col) | struct<last_value(col):int> |
| org.apache.spark.sql.catalyst.expressions.aggregate.Max | max | SELECT max(col) FROM VALUES (10), (50), (20) AS tab(col) | struct<max(col):int> |
| org.apache.spark.sql.catalyst.expressions.aggregate.MaxBy | max_by | SELECT max_by(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y) | struct<max_by(x, y):string> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.Median | median | SELECT median(col) FROM VALUES (0), (10) AS tab(col) | struct<median(col):double> |
| org.apache.spark.sql.catalyst.expressions.aggregate.Min | min | SELECT min(col) FROM VALUES (10), (-1), (20) AS tab(col) | struct<min(col):int> |
| org.apache.spark.sql.catalyst.expressions.aggregate.MinBy | min_by | SELECT min_by(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y) | struct<min_by(x, y):string> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.Mode | mode | SELECT mode(col) FROM VALUES (0), (10), (10) AS tab(col) | struct<mode(col):int> |
| org.apache.spark.sql.catalyst.expressions.aggregate.Percentile | percentile | SELECT percentile(col, 0.3) FROM VALUES (0), (10) AS tab(col) | struct<percentile(col, 0.3, 1):double> |
| org.apache.spark.sql.catalyst.expressions.aggregate.RegrAvgX | regr_avgx | SELECT regr_avgx(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x) | struct<regr_avgx(y, x):double> |
| org.apache.spark.sql.catalyst.expressions.aggregate.RegrAvgY | regr_avgy | SELECT regr_avgy(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x) | struct<regr_avgy(y, x):double> |
| org.apache.spark.sql.catalyst.expressions.aggregate.RegrCount | regr_count | SELECT regr_count(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x) | struct<regr_count(y, x):bigint> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.RegrIntercept | regr_intercept | SELECT regr_intercept(y, x) FROM VALUES (1,1), (2,2), (3,3) AS tab(y, x) | struct<regr_intercept(y, x):double> |
| org.apache.spark.sql.catalyst.expressions.aggregate.RegrR2 | regr_r2 | SELECT regr_r2(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x) | struct<regr_r2(y, x):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.RegrSXX | regr_sxx | SELECT regr_sxx(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x) | struct<regr_sxx(y, x):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.RegrSXY | regr_sxy | SELECT regr_sxy(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x) | struct<regr_sxy(y, x):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.RegrSYY | regr_syy | SELECT regr_syy(y, x) FROM VALUES (1, 2), (2, 2), (2, 3), (2, 4) AS tab(y, x) | struct<regr_syy(y, x):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.RegrSlope | regr_slope | SELECT regr_slope(y, x) FROM VALUES (1,1), (2,2), (3,3) AS tab(y, x) | struct<regr_slope(y, x):double> |
| org.apache.spark.sql.catalyst.expressions.aggregate.Skewness | skewness | SELECT skewness(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col) | struct<skewness(col):double> |
| org.apache.spark.sql.catalyst.expressions.aggregate.StddevPop | stddev_pop | SELECT stddev_pop(col) FROM VALUES (1), (2), (3) AS tab(col) | struct<stddev_pop(col):double> |
| org.apache.spark.sql.catalyst.expressions.aggregate.StddevSamp | std | SELECT std(col) FROM VALUES (1), (2), (3) AS tab(col) | struct<std(col):double> |
| org.apache.spark.sql.catalyst.expressions.aggregate.StddevSamp | stddev | SELECT stddev(col) FROM VALUES (1), (2), (3) AS tab(col) | struct<stddev(col):double> |
| org.apache.spark.sql.catalyst.expressions.aggregate.StddevSamp | stddev_samp | SELECT stddev_samp(col) FROM VALUES (1), (2), (3) AS tab(col) | struct<stddev_samp(col):double> |
| org.apache.spark.sql.catalyst.expressions.aggregate.Sum | sum | SELECT sum(col) FROM VALUES (5), (10), (15) AS tab(col) | struct<sum(col):bigint> |
-| org.apache.spark.sql.catalyst.expressions.aggregate.TryAverage | try_avg | SELECT try_avg(col) FROM VALUES (1), (2), (3) AS tab(col) | struct<try_avg(col):double> |
-| org.apache.spark.sql.catalyst.expressions.aggregate.TrySum | try_sum | SELECT try_sum(col) FROM VALUES (5), (10), (15) AS tab(col) | struct<try_sum(col):bigint> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.TryAverageExpressionBuilder | try_avg | SELECT try_avg(col) FROM VALUES (1), (2), (3) AS tab(col) | struct<try_avg(col):double> |
+| org.apache.spark.sql.catalyst.expressions.aggregate.TrySumExpressionBuilder | try_sum | SELECT try_sum(col) FROM VALUES (5), (10), (15) AS tab(col) | struct<try_sum(col):bigint> |
| org.apache.spark.sql.catalyst.expressions.aggregate.VariancePop | var_pop | SELECT var_pop(col) FROM VALUES (1), (2), (3) AS tab(col) | struct<var_pop(col):double> |
| org.apache.spark.sql.catalyst.expressions.aggregate.VarianceSamp | var_samp | SELECT var_samp(col) FROM VALUES (1), (2), (3) AS tab(col) | struct<var_samp(col):double> |
| org.apache.spark.sql.catalyst.expressions.aggregate.VarianceSamp | variance | SELECT variance(col) FROM VALUES (1), (2), (3) AS tab(col) | struct<variance(col):double> |
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-count-bug.sql.out sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-count-bug.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-count-bug.sql.out 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-count-bug.sql.out 2023-09-18 15:00:42.998708419 +0800
@@ -0,0 +1,228 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temp view l (a, b)
+as values
+ (1, 2.0),
+ (1, 2.0),
+ (2, 1.0),
+ (2, 1.0),
+ (3, 3.0),
+ (null, null),
+ (null, 5.0),
+ (6, null)
+-- !query analysis
+CreateViewCommand `l`, [(a,None), (b,None)], values
+ (1, 2.0),
+ (1, 2.0),
+ (2, 1.0),
+ (2, 1.0),
+ (3, 3.0),
+ (null, null),
+ (null, 5.0),
+ (6, null), false, false, LocalTempView, true
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+create temp view r (c, d)
+as values
+ (2, 3.0),
+ (2, 3.0),
+ (3, 2.0),
+ (4, 1.0),
+ (null, null),
+ (null, 5.0),
+ (6, null)
+-- !query analysis
+CreateViewCommand `r`, [(c,None), (d,None)], values
+ (2, 3.0),
+ (2, 3.0),
+ (3, 2.0),
+ (4, 1.0),
+ (null, null),
+ (null, 5.0),
+ (6, null), false, false, LocalTempView, true
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+select *, (select count(*) from r where l.a = r.c) from l
+-- !query analysis
+Project [a#x, b#x, scalar-subquery#x [a#x] AS scalarsubquery(a)#xL]
+: +- Aggregate [count(1) AS count(1)#xL]
+: +- Filter (outer(a#x) = c#x)
+: +- SubqueryAlias r
+: +- View (`r`, [c#x,d#x])
+: +- Project [cast(col1#x as int) AS c#x, cast(col2#x as decimal(2,1)) AS d#x]
+: +- LocalRelation [col1#x, col2#x]
++- SubqueryAlias l
+ +- View (`l`, [a#x,b#x])
+ +- Project [cast(col1#x as int) AS a#x, cast(col2#x as decimal(2,1)) AS b#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+select *, (select count(*) from r where l.a = r.c group by c) from l
+-- !query analysis
+Project [a#x, b#x, scalar-subquery#x [a#x] AS scalarsubquery(a)#xL]
+: +- Aggregate [c#x], [count(1) AS count(1)#xL]
+: +- Filter (outer(a#x) = c#x)
+: +- SubqueryAlias r
+: +- View (`r`, [c#x,d#x])
+: +- Project [cast(col1#x as int) AS c#x, cast(col2#x as decimal(2,1)) AS d#x]
+: +- LocalRelation [col1#x, col2#x]
++- SubqueryAlias l
+ +- View (`l`, [a#x,b#x])
+ +- Project [cast(col1#x as int) AS a#x, cast(col2#x as decimal(2,1)) AS b#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+select *, (select count(*) from r where l.a = r.c group by 'constant') from l
+-- !query analysis
+Project [a#x, b#x, scalar-subquery#x [a#x] AS scalarsubquery(a)#xL]
+: +- Aggregate [constant], [count(1) AS count(1)#xL]
+: +- Filter (outer(a#x) = c#x)
+: +- SubqueryAlias r
+: +- View (`r`, [c#x,d#x])
+: +- Project [cast(col1#x as int) AS c#x, cast(col2#x as decimal(2,1)) AS d#x]
+: +- LocalRelation [col1#x, col2#x]
++- SubqueryAlias l
+ +- View (`l`, [a#x,b#x])
+ +- Project [cast(col1#x as int) AS a#x, cast(col2#x as decimal(2,1)) AS b#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+select *, (
+ select (count(*)) is null
+ from r
+ where l.a = r.c)
+from l
+-- !query analysis
+Project [a#x, b#x, scalar-subquery#x [a#x] AS scalarsubquery(a)#x]
+: +- Aggregate [isnull(count(1)) AS (count(1) IS NULL)#x]
+: +- Filter (outer(a#x) = c#x)
+: +- SubqueryAlias r
+: +- View (`r`, [c#x,d#x])
+: +- Project [cast(col1#x as int) AS c#x, cast(col2#x as decimal(2,1)) AS d#x]
+: +- LocalRelation [col1#x, col2#x]
++- SubqueryAlias l
+ +- View (`l`, [a#x,b#x])
+ +- Project [cast(col1#x as int) AS a#x, cast(col2#x as decimal(2,1)) AS b#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+select *, (
+ select (count(*)) is null
+ from r
+ where l.a = r.c
+ group by r.c)
+from l
+-- !query analysis
+Project [a#x, b#x, scalar-subquery#x [a#x] AS scalarsubquery(a)#x]
+: +- Aggregate [c#x], [isnull(count(1)) AS (count(1) IS NULL)#x]
+: +- Filter (outer(a#x) = c#x)
+: +- SubqueryAlias r
+: +- View (`r`, [c#x,d#x])
+: +- Project [cast(col1#x as int) AS c#x, cast(col2#x as decimal(2,1)) AS d#x]
+: +- LocalRelation [col1#x, col2#x]
++- SubqueryAlias l
+ +- View (`l`, [a#x,b#x])
+ +- Project [cast(col1#x as int) AS a#x, cast(col2#x as decimal(2,1)) AS b#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+select *, (select count(*) from r where l.a = r.c having count(*) <= 1) from l
+-- !query analysis
+Project [a#x, b#x, scalar-subquery#x [a#x] AS scalarsubquery(a)#xL]
+: +- Filter (count(1)#xL <= cast(1 as bigint))
+: +- Aggregate [count(1) AS count(1)#xL]
+: +- Filter (outer(a#x) = c#x)
+: +- SubqueryAlias r
+: +- View (`r`, [c#x,d#x])
+: +- Project [cast(col1#x as int) AS c#x, cast(col2#x as decimal(2,1)) AS d#x]
+: +- LocalRelation [col1#x, col2#x]
++- SubqueryAlias l
+ +- View (`l`, [a#x,b#x])
+ +- Project [cast(col1#x as int) AS a#x, cast(col2#x as decimal(2,1)) AS b#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+select *, (select count(*) from r where l.a = r.c having count(*) >= 2) from l
+-- !query analysis
+Project [a#x, b#x, scalar-subquery#x [a#x] AS scalarsubquery(a)#xL]
+: +- Filter (count(1)#xL >= cast(2 as bigint))
+: +- Aggregate [count(1) AS count(1)#xL]
+: +- Filter (outer(a#x) = c#x)
+: +- SubqueryAlias r
+: +- View (`r`, [c#x,d#x])
+: +- Project [cast(col1#x as int) AS c#x, cast(col2#x as decimal(2,1)) AS d#x]
+: +- LocalRelation [col1#x, col2#x]
++- SubqueryAlias l
+ +- View (`l`, [a#x,b#x])
+ +- Project [cast(col1#x as int) AS a#x, cast(col2#x as decimal(2,1)) AS b#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+set spark.sql.optimizer.decorrelateSubqueryLegacyIncorrectCountHandling.enabled = true
+-- !query analysis
+SetCommand (spark.sql.optimizer.decorrelateSubqueryLegacyIncorrectCountHandling.enabled,Some(true))
+
+
+-- !query
+select *, (select count(*) from r where l.a = r.c) from l
+-- !query analysis
+Project [a#x, b#x, scalar-subquery#x [a#x] AS scalarsubquery(a)#xL]
+: +- Aggregate [count(1) AS count(1)#xL]
+: +- Filter (outer(a#x) = c#x)
+: +- SubqueryAlias r
+: +- View (`r`, [c#x,d#x])
+: +- Project [cast(col1#x as int) AS c#x, cast(col2#x as decimal(2,1)) AS d#x]
+: +- LocalRelation [col1#x, col2#x]
++- SubqueryAlias l
+ +- View (`l`, [a#x,b#x])
+ +- Project [cast(col1#x as int) AS a#x, cast(col2#x as decimal(2,1)) AS b#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+select *, (select count(*) from r where l.a = r.c group by c) from l
+-- !query analysis
+Project [a#x, b#x, scalar-subquery#x [a#x] AS scalarsubquery(a)#xL]
+: +- Aggregate [c#x], [count(1) AS count(1)#xL]
+: +- Filter (outer(a#x) = c#x)
+: +- SubqueryAlias r
+: +- View (`r`, [c#x,d#x])
+: +- Project [cast(col1#x as int) AS c#x, cast(col2#x as decimal(2,1)) AS d#x]
+: +- LocalRelation [col1#x, col2#x]
++- SubqueryAlias l
+ +- View (`l`, [a#x,b#x])
+ +- Project [cast(col1#x as int) AS a#x, cast(col2#x as decimal(2,1)) AS b#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+select *, (select count(*) from r where l.a = r.c group by 'constant') from l
+-- !query analysis
+Project [a#x, b#x, scalar-subquery#x [a#x] AS scalarsubquery(a)#xL]
+: +- Aggregate [constant], [count(1) AS count(1)#xL]
+: +- Filter (outer(a#x) = c#x)
+: +- SubqueryAlias r
+: +- View (`r`, [c#x,d#x])
+: +- Project [cast(col1#x as int) AS c#x, cast(col2#x as decimal(2,1)) AS d#x]
+: +- LocalRelation [col1#x, col2#x]
++- SubqueryAlias l
+ +- View (`l`, [a#x,b#x])
+ +- Project [cast(col1#x as int) AS a#x, cast(col2#x as decimal(2,1)) AS b#x]
+ +- LocalRelation [col1#x, col2#x]
+
+
+-- !query
+reset spark.sql.optimizer.decorrelateSubqueryLegacyIncorrectCountHandling.enabled
+-- !query analysis
+ResetCommand spark.sql.optimizer.decorrelateSubqueryLegacyIncorrectCountHandling.enabled
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/ansi/array.sql sql/core/src/test/resources/sql-tests/inputs/ansi/array.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/ansi/array.sql 2023-09-21 10:45:41.951461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/ansi/array.sql 2023-09-18 15:00:42.998708419 +0800
@@ -1,17 +1 @@
---IMPORT array.sql
-
--- index out of range for array elements
--- return null results if array index in [] operator is out of bound
-set spark.sql.ansi.strictIndexOperator=false;
-select array(1, 2, 3)[5];
-select array(1, 2, 3)[-1];
-
--- the configuration spark.sql.ansi.strictIndexOperator doesn't affect the function element_at
-select element_at(array(1, 2, 3), 5);
-select element_at(array(1, 2, 3), -5);
-select element_at(array(1, 2, 3), 0);
-
--- -- the configuration spark.sql.ansi.strictIndexOperator doesn't affect the function elt
-select elt(4, '123', '456');
-select elt(0, '123', '456');
-select elt(-1, '123', '456');
+--IMPORT array.sql
\ No newline at end of file
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/ansi/decimalArithmeticOperations.sql sql/core/src/test/resources/sql-tests/inputs/ansi/decimalArithmeticOperations.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/ansi/decimalArithmeticOperations.sql 2023-09-15 11:42:05.962317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/ansi/decimalArithmeticOperations.sql 2023-09-18 15:00:42.998708419 +0800
@@ -29,4 +29,13 @@
select 123456789123456789.1234567890 * 1.123456789123456789;
select 12345678912345.123456789123 / 0.000000012345678;
+select 1.0123456789012345678901234567890123456e36BD / 0.1;
+select 1.0123456789012345678901234567890123456e35BD / 1.0;
+select 1.0123456789012345678901234567890123456e34BD / 1.0;
+select 1.0123456789012345678901234567890123456e33BD / 1.0;
+select 1.0123456789012345678901234567890123456e32BD / 1.0;
+select 1.0123456789012345678901234567890123456e31BD / 1.0;
+select 1.0123456789012345678901234567890123456e31BD / 0.1;
+select 1.0123456789012345678901234567890123456e31BD / 10.0;
+
drop table decimals_test;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/ansi/double-quoted-identifiers-disabled.sql sql/core/src/test/resources/sql-tests/inputs/ansi/double-quoted-identifiers-disabled.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/ansi/double-quoted-identifiers-disabled.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/ansi/double-quoted-identifiers-disabled.sql 2023-09-18 15:00:42.998708419 +0800
@@ -0,0 +1,2 @@
+--SET spark.sql.ansi.doubleQuotedIdentifiers=false
+--IMPORT double-quoted-identifiers.sql
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/ansi/double-quoted-identifiers-enabled.sql sql/core/src/test/resources/sql-tests/inputs/ansi/double-quoted-identifiers-enabled.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/ansi/double-quoted-identifiers-enabled.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/ansi/double-quoted-identifiers-enabled.sql 2023-09-18 15:00:42.998708419 +0800
@@ -0,0 +1,3 @@
+--SET spark.sql.ansi.doubleQuotedIdentifiers=true
+--IMPORT double-quoted-identifiers.sql
+
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/ansi/map.sql sql/core/src/test/resources/sql-tests/inputs/ansi/map.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/ansi/map.sql 2023-09-21 10:45:41.951461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/ansi/map.sql 2023-09-18 15:00:42.998708419 +0800
@@ -1,9 +1 @@
--IMPORT map.sql
-
--- key does not exist
--- return null results if the map key in [] operator doesn't exist
-set spark.sql.ansi.strictIndexOperator=false;
-select map(1, 'a', 2, 'b')[5];
--- the configuration spark.sql.ansi.strictIndexOperator doesn't affect the function element_at
-select element_at(map(1, 'a', 2, 'b'), 5);
-select element_at(map('a', 1, 'b', 2), 'c');
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/ansi/math.sql sql/core/src/test/resources/sql-tests/inputs/ansi/math.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/ansi/math.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/ansi/math.sql 2023-09-18 15:00:42.998708419 +0800
@@ -0,0 +1 @@
+--IMPORT math.sql
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/ansi/try_datetime_functions.sql sql/core/src/test/resources/sql-tests/inputs/ansi/try_datetime_functions.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/ansi/try_datetime_functions.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/ansi/try_datetime_functions.sql 2023-09-18 15:00:42.998708419 +0800
@@ -0,0 +1 @@
+--IMPORT try_datetime_functions.sql
\ No newline at end of file
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/array.sql sql/core/src/test/resources/sql-tests/inputs/array.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/array.sql 2023-09-21 10:45:41.951461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/array.sql 2023-09-18 15:00:42.998708419 +0800
@@ -113,3 +113,50 @@
select array_size(array(2, 1));
select array_size(NULL);
select array_size(map('a', 1, 'b', 2));
+
+-- size(arrays_zip)
+select size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10)));
+select size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10)));
+select size(arrays_zip(array(1, 2, 3), array(4), null, array(7, 8, 9, 10)));
+
+-- isnotnull(arrays_zip)
+select isnotnull(arrays_zip(array(), array(4), array(7, 8, 9, 10)));
+select isnotnull(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10)));
+select isnotnull(arrays_zip(array(1, 2, 3), NULL, array(4), array(7, 8, 9, 10)));
+
+-- function get()
+select get(array(1, 2, 3), 0);
+select get(array(1, 2, 3), 3);
+select get(array(1, 2, 3), null);
+select get(array(1, 2, 3), -1);
+
+-- function array_insert()
+select array_insert(array(1, 2, 3), 3, 4);
+select array_insert(array(2, 3, 4), 0, 1);
+select array_insert(array(2, 3, 4), 1, 1);
+select array_insert(array(1, 3, 4), -2, 2);
+select array_insert(array(1, 2, 3), 3, "4");
+select array_insert(cast(NULL as ARRAY<INT>), 1, 1);
+select array_insert(array(1, 2, 3, NULL), cast(NULL as INT), 4);
+select array_insert(array(1, 2, 3, NULL), 4, cast(NULL as INT));
+select array_insert(array(2, 3, NULL, 4), 5, 5);
+select array_insert(array(2, 3, NULL, 4), -5, 1);
+
+-- function array_compact
+select array_compact(id) from values (1) as t(id);
+select array_compact(array("1", null, "2", null));
+select array_compact(array("a", "b", "c"));
+select array_compact(array(1D, null, 2D, null));
+select array_compact(array(array(1, 2, 3, null), null, array(4, null, 6)));
+select array_compact(array(null));
+
+-- function array_append
+select array_append(array(1, 2, 3), 4);
+select array_append(array('a', 'b', 'c'), 'd');
+select array_append(array(1, 2, 3, NULL), NULL);
+select array_append(array('a', 'b', 'c', NULL), NULL);
+select array_append(CAST(null AS ARRAY<String>), 'a');
+select array_append(CAST(null AS ARRAY<String>), CAST(null as String));
+select array_append(array(), 1);
+select array_append(CAST(array() AS ARRAY<String>), CAST(NULL AS String));
+select array_append(array(CAST(NULL AS String)), CAST(NULL AS String));
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/cast.sql sql/core/src/test/resources/sql-tests/inputs/cast.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/cast.sql 2023-09-21 10:45:41.951461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/cast.sql 2023-09-18 15:00:42.998708419 +0800
@@ -104,3 +104,45 @@
select cast(cast('inf' as double) as timestamp);
select cast(cast('inf' as float) as timestamp);
+
+-- cast ANSI intervals to integrals
+select cast(interval '1' year as tinyint);
+select cast(interval '-10-2' year to month as smallint);
+select cast(interval '1000' month as int);
+select cast(interval -'10.123456' second as tinyint);
+select cast(interval '23:59:59' hour to second as smallint);
+select cast(interval -'1 02:03:04.123' day to second as int);
+select cast(interval '10' day as bigint);
+
+select cast(interval '-1000' month as tinyint);
+select cast(interval '1000000' second as smallint);
+
+-- cast integrals to ANSI intervals
+select cast(1Y as interval year);
+select cast(-122S as interval year to month);
+select cast(ym as interval year to month) from values(-122S) as t(ym);
+select cast(1000 as interval month);
+select cast(-10L as interval second);
+select cast(100Y as interval hour to second);
+select cast(dt as interval hour to second) from values(100Y) as t(dt);
+select cast(-1000S as interval day to second);
+select cast(10 as interval day);
+
+select cast(2147483647 as interval year);
+select cast(-9223372036854775808L as interval day);
+
+-- cast ANSI intervals to decimals
+select cast(interval '-1' year as decimal(10, 0));
+select cast(interval '1.000001' second as decimal(10, 6));
+select cast(interval '08:11:10.001' hour to second as decimal(10, 4));
+select cast(interval '1 01:02:03.1' day to second as decimal(8, 1));
+select cast(interval '10.123' second as decimal(4, 2));
+select cast(interval '10.005' second as decimal(4, 2));
+select cast(interval '10.123' second as decimal(5, 2));
+select cast(interval '10.123' second as decimal(1, 0));
+
+-- cast decimals to ANSI intervals
+select cast(10.123456BD as interval day to second);
+select cast(80.654321BD as interval hour to minute);
+select cast(-10.123456BD as interval year to month);
+select cast(10.654321BD as interval month);
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/column-resolution-aggregate.sql sql/core/src/test/resources/sql-tests/inputs/column-resolution-aggregate.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/column-resolution-aggregate.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/column-resolution-aggregate.sql 2023-09-18 15:00:42.998708419 +0800
@@ -0,0 +1,33 @@
+-- Tests covering column resolution priority in Aggregate.
+
+CREATE TEMPORARY VIEW v1 AS VALUES (1, 1, 1), (2, 2, 1) AS t(a, b, k);
+CREATE TEMPORARY VIEW v2 AS VALUES (1, 1, 1), (2, 2, 1) AS t(x, y, all);
+
+-- Relation output columns have higher priority than lateral column alias. This query
+-- should fail as `b` is not in GROUP BY.
+SELECT max(a) AS b, b FROM v1 GROUP BY k;
+
+-- Lateral column alias has higher priority than outer reference.
+SELECT a FROM v1 WHERE (12, 13) IN (SELECT max(x + 10) AS a, a + 1 FROM v2);
+
+-- Relation output columns have higher priority than GROUP BY alias. This query should
+-- fail as `a` is not in GROUP BY.
+SELECT a AS k FROM v1 GROUP BY k;
+
+-- Relation output columns have higher priority than GROUP BY ALL. This query should
+-- fail as `x` is not in GROUP BY.
+SELECT x FROM v2 GROUP BY all;
+
+-- GROUP BY alias has higher priority than GROUP BY ALL, this query fails as `b` is not in GROUP BY.
+SELECT a AS all, b FROM v1 GROUP BY all;
+
+-- GROUP BY alias/ALL does not support lateral column alias.
+SELECT k AS lca, lca + 1 AS col FROM v1 GROUP BY k, col;
+SELECT k AS lca, lca + 1 AS col FROM v1 GROUP BY all;
+
+-- GROUP BY alias still works if it does not directly reference lateral column alias.
+SELECT k AS lca, lca + 1 AS col FROM v1 GROUP BY lca;
+
+-- GROUP BY ALL has higher priority than outer reference. This query should run as `a` and `b` are
+-- in GROUP BY due to the GROUP BY ALL resolution.
+SELECT * FROM v2 WHERE EXISTS (SELECT a, b FROM v1 GROUP BY all);
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/columnresolution-negative.sql sql/core/src/test/resources/sql-tests/inputs/columnresolution-negative.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/columnresolution-negative.sql 2023-09-15 11:42:05.962317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/columnresolution-negative.sql 2023-09-18 15:00:42.998708419 +0800
@@ -31,6 +31,20 @@
USE mydb2;
SELECT mydb1.t1.i1 FROM t1;
+-- Negative tests: view cannot resolve column after incompatible schema change
+USE mydb1;
+CREATE VIEW v1 AS SELECT * FROM t1;
+DROP TABLE t1;
+CREATE TABLE t1 USING parquet AS SELECT 1 AS i2;
+SELECT * FROM v1;
+
+-- Negative tests: temp view cannot resolve column after incompatible schema change
+USE mydb2;
+CREATE TEMP VIEW v2 AS SELECT * FROM t1;
+DROP TABLE t1;
+CREATE TABLE t1 USING parquet AS SELECT 1 AS i2;
+SELECT * FROM v2;
+
-- reset
DROP DATABASE mydb1 CASCADE;
DROP DATABASE mydb2 CASCADE;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/column-resolution-sort.sql sql/core/src/test/resources/sql-tests/inputs/column-resolution-sort.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/column-resolution-sort.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/column-resolution-sort.sql 2023-09-18 15:00:42.998708419 +0800
@@ -0,0 +1,20 @@
+--SET spark.sql.leafNodeDefaultParallelism=1
+-- Tests covering column resolution priority in Sort.
+
+CREATE TEMPORARY VIEW v1 AS VALUES (1, 2, 2), (2, 1, 1) AS t(a, b, k);
+CREATE TEMPORARY VIEW v2 AS VALUES (1, 2, 2), (2, 1, 1) AS t(a, b, all);
+
+-- Relation output columns have higher priority than missing reference.
+-- Query will fail if we order by the column `v1.b`, as it's not in GROUP BY.
+-- Actually results are [1, 2] as we order by `max(a) AS b`.
+SELECT max(a) AS b FROM v1 GROUP BY k ORDER BY b;
+
+-- Missing reference has higher priority than ORDER BY ALL.
+-- Results will be [1, 2] if we order by `max(a)`.
+-- Actually results are [2, 1] as we order by the grouping column `v2.all`.
+SELECT max(a) FROM v2 GROUP BY all ORDER BY all;
+
+-- ORDER BY ALL has higher priority than outer reference.
+-- Results will be [1, 1] if we order by outer reference 'v2.all'.
+-- Actually results are [2, 2] as we order by column `v1.b`
+SELECT (SELECT b FROM v1 ORDER BY all LIMIT 1) FROM v2;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/csv-functions.sql sql/core/src/test/resources/sql-tests/inputs/csv-functions.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/csv-functions.sql 2023-09-15 11:42:05.962317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/csv-functions.sql 2023-09-18 15:00:42.998708419 +0800
@@ -4,6 +4,7 @@
-- Check if errors handled
select from_csv('1', 1);
select from_csv('1', 'a InvalidType');
+select from_csv('1', 'Array<int>');
select from_csv('1', 'a INT', named_struct('mode', 'PERMISSIVE'));
select from_csv('1', 'a INT', map('mode', 1));
select from_csv();
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/cte-nested.sql sql/core/src/test/resources/sql-tests/inputs/cte-nested.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/cte-nested.sql 2023-09-21 10:45:41.951461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/cte-nested.sql 2023-09-18 15:00:42.998708419 +0800
@@ -17,6 +17,16 @@
SELECT * FROM t
);
+-- Make sure CTE in subquery is scoped to that subquery rather than global
+-- the 2nd half of the union should fail because the cte is scoped to the first half
+SELECT * FROM
+ (
+ WITH cte AS (SELECT * FROM range(10))
+ SELECT * FROM cte WHERE id = 8
+ ) a
+UNION
+SELECT * FROM cte;
+
-- CTE in CTE definition shadows outer
WITH
t AS (SELECT 1),
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/cte.sql sql/core/src/test/resources/sql-tests/inputs/cte.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/cte.sql 2023-09-15 11:42:05.962317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/cte.sql 2023-09-18 15:00:42.998708419 +0800
@@ -49,6 +49,10 @@
t(x) AS (SELECT 2)
SELECT * FROM t;
+-- invalid CTE relation should fail the query even if it's not referenced
+WITH t AS (SELECT 1 FROM non_existing_table)
+SELECT 2;
+
-- Clean up
DROP VIEW IF EXISTS t;
DROP VIEW IF EXISTS t2;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/current_database_catalog.sql sql/core/src/test/resources/sql-tests/inputs/current_database_catalog.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/current_database_catalog.sql 2023-09-15 11:42:05.962317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/current_database_catalog.sql 2023-09-18 15:00:42.998708419 +0800
@@ -1,2 +1,2 @@
--- get current_datebase and current_catalog
-select current_database(), current_catalog();
+-- get current_database/current_schema and current_catalog
+select current_database(), current_schema(), current_catalog();
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/date.sql sql/core/src/test/resources/sql-tests/inputs/date.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/date.sql 2023-09-21 10:45:41.951461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/date.sql 2023-09-18 15:00:42.998708419 +0800
@@ -19,6 +19,7 @@
select current_date = current_date;
-- under ANSI mode, `current_date` can't be a function name.
select current_date() = current_date();
+select curdate(1);
-- conversions between date and unix_date (number of days from epoch)
select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null);
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql 2023-09-15 11:42:05.962317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql 2023-09-18 15:00:42.998708419 +0800
@@ -54,6 +54,36 @@
select 123456789123456789.1234567890 * 1.123456789123456789;
select 12345678912345.123456789123 / 0.000000012345678;
+-- union decimal type
+
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(3, 1)) + CAST(90 AS DECIMAL(3, 1));
+
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(3, 1)) - CAST(-90 AS DECIMAL(3, 1));
+
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(3, 1)) * CAST(10 AS DECIMAL(3, 1));
+
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(3, 1)) / CAST(10 AS DECIMAL(3, 1));
+
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(10, 2)) % CAST(3 AS DECIMAL(5, 1));
+
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT pmod(CAST(10 AS DECIMAL(10, 2)), CAST(3 AS DECIMAL(5, 1)));
+
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(10, 2)) div CAST(3 AS DECIMAL(5, 1));
+
-- return NULL instead of rounding, according to old Spark versions' behavior
set spark.sql.decimalOperations.allowPrecisionLoss=false;
@@ -83,4 +113,13 @@
select 123456789123456789.1234567890 * 1.123456789123456789;
select 12345678912345.123456789123 / 0.000000012345678;
+select 1.0123456789012345678901234567890123456e36BD / 0.1;
+select 1.0123456789012345678901234567890123456e35BD / 1.0;
+select 1.0123456789012345678901234567890123456e34BD / 1.0;
+select 1.0123456789012345678901234567890123456e33BD / 1.0;
+select 1.0123456789012345678901234567890123456e32BD / 1.0;
+select 1.0123456789012345678901234567890123456e31BD / 1.0;
+select 1.0123456789012345678901234567890123456e31BD / 0.1;
+select 1.0123456789012345678901234567890123456e31BD / 10.0;
+
drop table decimals_test;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/describe.sql sql/core/src/test/resources/sql-tests/inputs/describe.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/describe.sql 2023-09-15 11:42:05.962317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/describe.sql 2023-09-18 15:00:43.002708419 +0800
@@ -1,8 +1,8 @@
CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet
- OPTIONS (a '1', b '2')
+ OPTIONS (a '1', b '2', password 'password')
PARTITIONED BY (c, d) CLUSTERED BY (a) SORTED BY (b ASC) INTO 2 BUCKETS
COMMENT 'table_comment'
- TBLPROPERTIES (t 'test');
+ TBLPROPERTIES (t 'test', password 'password');
CREATE TEMPORARY VIEW temp_v AS SELECT * FROM t;
@@ -97,3 +97,25 @@
DROP VIEW temp_Data_Source_View;
DROP VIEW v;
+
+-- Show column default values
+CREATE TABLE d (a STRING DEFAULT 'default-value', b INT DEFAULT 42) USING parquet COMMENT 'table_comment';
+
+DESC d;
+
+DESC EXTENDED d;
+
+DESC TABLE EXTENDED d;
+
+DESC FORMATTED d;
+
+-- Show column default values with newlines in the string
+CREATE TABLE e (a STRING DEFAULT CONCAT('a\n b\n ', 'c\n d'), b INT DEFAULT 42) USING parquet COMMENT 'table_comment';
+
+DESC e;
+
+DESC EXTENDED e;
+
+DESC TABLE EXTENDED e;
+
+DESC FORMATTED e;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/describe-table-column.sql sql/core/src/test/resources/sql-tests/inputs/describe-table-column.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/describe-table-column.sql 2023-09-15 11:42:05.962317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/describe-table-column.sql 1970-01-01 08:00:00.000000000 +0800
@@ -1,70 +0,0 @@
--- Test temp table
-CREATE TEMPORARY VIEW desc_col_temp_view (key int COMMENT 'column_comment', col struct<x:int, y:string>) USING PARQUET;
-
-DESC desc_col_temp_view key;
-
-DESC EXTENDED desc_col_temp_view key;
-
-DESC FORMATTED desc_col_temp_view key;
-
--- Describe a column with qualified name
-DESC FORMATTED desc_col_temp_view desc_col_temp_view.key;
-
--- Describe a non-existent column
-DESC desc_col_temp_view key1;
-
--- Describe a nested column
-DESC desc_col_temp_view col.x;
-
--- Test persistent table
-CREATE TABLE desc_col_table (key int COMMENT 'column_comment') USING PARQUET;
-
-ANALYZE TABLE desc_col_table COMPUTE STATISTICS FOR COLUMNS key;
-
-DESC desc_col_table key;
-
-DESC EXTENDED desc_col_table key;
-
-DESC FORMATTED desc_col_table key;
-
--- Describe a non-existent column
-DESC desc_col_table key1;
-
--- Test complex columns
-CREATE TABLE desc_complex_col_table (`a.b` int, col struct<x:int, y:string>) USING PARQUET;
-
-DESC FORMATTED desc_complex_col_table `a.b`;
-
-DESC FORMATTED desc_complex_col_table col;
-
--- Describe a nested column
-DESC FORMATTED desc_complex_col_table col.x;
-
--- Test output for histogram statistics
-SET spark.sql.statistics.histogram.enabled=true;
-SET spark.sql.statistics.histogram.numBins=2;
-
-INSERT INTO desc_col_table values 1, 2, 3, 4;
-
-ANALYZE TABLE desc_col_table COMPUTE STATISTICS FOR COLUMNS key;
-
-DESC EXTENDED desc_col_table key;
-
-DROP VIEW desc_col_temp_view;
-
-DROP TABLE desc_col_table;
-
-DROP TABLE desc_complex_col_table;
-
---Test case insensitive
-
-CREATE TABLE customer(CName STRING) USING PARQUET;
-
-INSERT INTO customer VALUES('Maria');
-
-ANALYZE TABLE customer COMPUTE STATISTICS FOR COLUMNS cname;
-
-DESC EXTENDED customer cname;
-
-DROP TABLE customer;
-
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/double-quoted-identifiers.sql sql/core/src/test/resources/sql-tests/inputs/double-quoted-identifiers.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/double-quoted-identifiers.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/double-quoted-identifiers.sql 2023-09-18 15:00:43.002708419 +0800
@@ -0,0 +1,57 @@
+-- All these should error out in the parser
+SELECT 1 FROM "not_exist";
+
+USE SCHEMA "not_exist";
+
+ALTER TABLE "not_exist" ADD COLUMN not_exist int;
+
+ALTER TABLE not_exist ADD COLUMN "not_exist" int;
+
+SELECT 1 AS "not_exist" FROM not_exist;
+
+SELECT 1 FROM not_exist AS X("hello");
+
+SELECT "not_exist"();
+
+SELECT "not_exist".not_exist();
+
+-- All these should error out in analysis
+SELECT 1 FROM `hello`;
+
+USE SCHEMA `not_exist`;
+
+ALTER TABLE `not_exist` ADD COLUMN not_exist int;
+
+ALTER TABLE not_exist ADD COLUMN `not_exist` int;
+
+SELECT 1 AS `not_exist` FROM `not_exist`;
+
+SELECT 1 FROM not_exist AS X(`hello`);
+
+SELECT `not_exist`();
+
+SELECT `not_exist`.not_exist();
+
+-- Strings in various situations all work
+SELECT "hello";
+
+CREATE TEMPORARY VIEW v(c1 COMMENT "hello") AS SELECT 1;
+DROP VIEW v;
+
+SELECT INTERVAL "1" YEAR;
+
+-- Single ticks still work
+SELECT 'hello';
+
+CREATE TEMPORARY VIEW v(c1 COMMENT 'hello') AS SELECT 1;
+DROP VIEW v;
+
+SELECT INTERVAL '1' YEAR;
+
+-- A whole scenario
+CREATE SCHEMA "myschema";
+CREATE TEMPORARY VIEW "myview"("c1") AS
+ WITH "v"("a") AS (SELECT 1) SELECT "a" FROM "v";
+SELECT "a1" AS "a2" FROM "myview" AS "atab"("a1");
+DROP TABLE "myview";
+DROP SCHEMA "myschema";
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/explain.sql sql/core/src/test/resources/sql-tests/inputs/explain.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/explain.sql 2023-09-15 11:42:05.962317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/explain.sql 2023-09-18 15:00:43.002708419 +0800
@@ -7,6 +7,7 @@
CREATE table explain_temp2 (key int, val int) USING PARQUET;
CREATE table explain_temp3 (key int, val int) USING PARQUET;
CREATE table explain_temp4 (key int, val string) USING PARQUET;
+CREATE table explain_temp5 (key int) USING PARQUET PARTITIONED BY(val string);
SET spark.sql.codegen.wholeStage = true;
@@ -119,11 +120,15 @@
FROM explain_temp4
GROUP BY key;
+-- V1 Write
+EXPLAIN EXTENDED INSERT INTO TABLE explain_temp5 SELECT * FROM explain_temp4;
+
-- cleanup
DROP TABLE explain_temp1;
DROP TABLE explain_temp2;
DROP TABLE explain_temp3;
DROP TABLE explain_temp4;
+DROP TABLE explain_temp5;
-- SPARK-35479: Format PartitionFilters IN strings in scan nodes
CREATE table t(v array<string>) USING PARQUET;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/extract.sql sql/core/src/test/resources/sql-tests/inputs/extract.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/extract.sql 2023-09-15 11:42:05.962317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/extract.sql 2023-09-18 15:00:43.002708419 +0800
@@ -161,3 +161,7 @@
select extract(MONTH from interval '123 12:34:56.789123123' DAY TO SECOND);
select date_part('not_supported', interval '123 12:34:56.789123123' DAY TO SECOND);
+
+-- alias for date_part
+select datepart('year', c), datepart('year', ntz), datepart('year', i) from t;
+select datepart('DAY', interval '123 12:34:56.789123123' DAY TO SECOND);
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/group-by-all-duckdb.sql sql/core/src/test/resources/sql-tests/inputs/group-by-all-duckdb.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/group-by-all-duckdb.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/group-by-all-duckdb.sql 2023-09-18 15:00:43.002708419 +0800
@@ -0,0 +1,29 @@
+-- group by all
+-- additional test cases from DuckDB, given to us by Mosha
+
+create temporary view integers as select * from values
+ (0, 1),
+ (0, 2),
+ (1, 3),
+ (1, NULL)
+ as integers(g, i);
+
+
+SELECT g, SUM(i) FROM integers GROUP BY ALL ORDER BY 1;
+
+SELECT g, SUM(i), COUNT(*), COUNT(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1;
+
+SELECT i%2, SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1;
+
+SELECT (g+i)%2, SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1;
+
+SELECT (g+i)%2 + SUM(i), SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1;
+
+SELECT g, i, g%2, SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1, 2, 3, 4;
+
+SELECT c0 FROM (SELECT 1 c0) t0 GROUP BY ALL HAVING c0>0;
+
+SELECT c0 FROM (SELECT 1 c0, 1 c1 UNION ALL SELECT 1, 2) t0 GROUP BY ALL ORDER BY c0;
+
+SELECT c0 FROM (SELECT 1 c0, 1 c1 UNION ALL SELECT 1, 2) t0 GROUP BY ALL HAVING c1>0 ORDER BY c0;
+
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/group-by-all-mosha.sql sql/core/src/test/resources/sql-tests/inputs/group-by-all-mosha.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/group-by-all-mosha.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/group-by-all-mosha.sql 2023-09-18 15:00:43.002708419 +0800
@@ -0,0 +1,32 @@
+-- group by all
+-- additional group by star test cases from Mosha
+create temporary view stuff as select * from values
+ (42, 9.75, 'hello world', '1970-08-07', '13.37', array(1,20,300)),
+ (1337, 1.2345, 'oh no', '2000-01-01', '42.0', array(4000,50000,600000)),
+ (42, 13.37, 'test', '1970-08-07', '1234567890', array(7000000,80000000,900000000))
+ as stuff(i, f, s, t, d, a);
+
+SELECT 100 * SUM(i) + SUM(f) / COUNT(s) AS f1, i AS f2 FROM stuff GROUP BY ALL ORDER BY f2;
+
+SELECT i + 1 AS i1, COUNT(i - 2) ci, f / i AS fi, SUM(i + f) sif FROM stuff GROUP BY ALL ORDER BY 1, 3;
+
+SELECT i AS i, COUNT(i) ci, f AS f, SUM(i + f) sif FROM stuff GROUP BY ALL ORDER BY 1, i, 2, ci, 3, f, 4, sif;
+
+SELECT i + 1, f / i, substring(s, 2, 3), extract(year from t), d / 2, size(a) FROM stuff
+GROUP BY ALL ORDER BY 1, 3, 4, 5, 6, 2;
+
+-- unlike Mosha, I'm failing this case because IMO it is too implicit to automatically group by i.
+SELECT i + SUM(f) FROM stuff GROUP BY ALL;
+
+SELECT s AS s, COUNT(*) c FROM stuff GROUP BY ALL HAVING SUM(f) > 0 ORDER BY s;
+
+SELECT SUM(i) si FROM stuff GROUP BY ALL HAVING si > 2;
+
+SELECT SUM(i) si FROM stuff GROUP BY ALL HAVING si < 2;
+
+-- negative test, i shouldn't propagate through the aggregate so the having should fail
+SELECT SUM(i) si FROM stuff GROUP BY ALL HAVING i > 2;
+
+-- negative test, i shouldn't propagate through the aggregate so the order by should fail
+SELECT SUM(i) si FROM stuff GROUP BY ALL ORDER BY i DESC;
+
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/group-by-all.sql sql/core/src/test/resources/sql-tests/inputs/group-by-all.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/group-by-all.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/group-by-all.sql 2023-09-18 15:00:43.002708419 +0800
@@ -0,0 +1,85 @@
+-- group by all
+-- see https://www.linkedin.com/posts/mosha_duckdb-firebolt-snowflake-activity-7009615821006131200-VQ0o
+
+create temporary view data as select * from values
+ ("USA", "San Francisco", "Reynold", 1, 11.0),
+ ("USA", "San Francisco", "Matei", 2, 12.0),
+ ("USA", "Berkeley", "Xiao", 3, 13.0),
+ ("China", "Hangzhou", "Wenchen", 4, 14.0),
+ ("China", "Shanghai", "Shanghaiese", 5, 15.0),
+ ("Korea", "Seoul", "Hyukjin", 6, 16.0),
+ ("UK", "London", "Sean", 7, 17.0)
+ as data(country, city, name, id, power);
+
+-- basic
+select country, count(*) from data group by ALL;
+
+-- different case
+select country, count(*) from data group by aLl;
+
+-- a column named "all" would still work
+select all, city, count(*) from (select country as all, city, id from data) group by all, city;
+
+-- a column named "all" should take precedence over the normal group by all expansion
+-- if all refers to the column, then the following should return 3 rows.
+-- if all refers to the global aggregate, then 1 row.
+SELECT count(1) FROM VALUES(1), (2), (3) AS T(all) GROUP BY all;
+
+-- two grouping columns and two aggregates
+select country, city, count(*), sum(power) from data group by all;
+
+-- different ordering
+select count(*), country, city, sum(power) from data group by all;
+
+-- alias in grouping column
+select country as con, count(*) from data group by all;
+
+
+-- alias in aggregate column
+select country, count(*) as cnt from data group by all;
+
+-- scalar expression in grouping column
+select upper(country), count(*) as powerup from data group by all;
+
+-- scalar expression in aggregate column
+select country, sum(power) + 10 as powerup from data group by all;
+
+-- group by all without aggregate, which should just become a distinct
+select country, city from data group by all;
+
+-- make sure aliases are propagated through correctly
+select con, powerup from
+ (select country as con, sum(power) + 10 as powerup from data group by all);
+
+-- having
+select country, count(id) as cnt from data group by all having cnt > 1;
+
+-- no grouping column
+select count(id) from data group by all;
+
+-- a more complex no grouping column case
+select count(id + power / 2) * 3 from data group by all;
+
+-- no grouping column on an empty relation
+-- this should still return one row because we rewrite this to a global aggregate, as opposed to
+-- returning zero row (grouping by a constant).
+select count(*) from (select * from data where country = "DNS") group by all;
+
+-- complex cases that we choose not to infer; fail with a useful error message
+select id + count(*) from data group by all;
+
+-- an even more complex case that we choose not to infer; fail with a useful error message
+select (id + id) / 2 + count(*) * 2 from data group by all;
+
+-- uncorrelated subquery should work
+select country, (select count(*) from data) as cnt, count(id) as cnt_id from data group by all;
+
+-- correlated subquery should also work
+select country, (select count(*) from data d1 where d1.country = d2.country), count(id) from data d2 group by all;
+
+-- correlated subquery together with aggregate function doesn't work.
+-- make sure we report the right error UNRESOLVED_ALL_IN_GROUP_BY, rather than some random subquery error.
+select (select count(*) from data d1 where d1.country = d2.country) + count(id) from data d2 group by all;
+
+-- SELECT list contains unresolved column, should not report UNRESOLVED_ALL_IN_GROUP_BY
+select non_exist from data group by all;
\ No newline at end of file
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/group-by.sql sql/core/src/test/resources/sql-tests/inputs/group-by.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/group-by.sql 2023-09-21 10:45:41.951461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/group-by.sql 2023-09-18 15:00:43.002708419 +0800
@@ -7,12 +7,6 @@
CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null)
AS testData(a, b);
-CREATE OR REPLACE TEMPORARY VIEW testRegression AS SELECT * FROM VALUES
-(1, 10, null), (2, 10, 11), (2, 20, 22), (2, 25, null), (2, 30, 35)
-AS testRegression(k, y, x);
-CREATE OR REPLACE TEMPORARY VIEW aggr AS SELECT * FROM VALUES
-(0, 0), (0, 10), (0, 20), (0, 30), (0, 40), (1, 10), (1, 20), (2, 10), (2, 20), (2, 25), (2, 30), (3, 60), (4, null)
-AS aggr(k, v);
-- Aggregate with empty GroupBy expressions.
SELECT a, COUNT(b) FROM testData;
@@ -40,6 +34,9 @@
SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1;
SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1;
+-- struct() in group by
+SELECT count(1) FROM testData GROUP BY struct(a + 0.1 AS aa);
+
-- Aggregate with nulls.
SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a)
FROM testData;
@@ -237,19 +234,6 @@
SELECT histogram_numeric(col, 3)
FROM VALUES (CAST(NULL AS INT)), (CAST(NULL AS INT)), (CAST(NULL AS INT)) AS tab(col);
-
--- SPARK-37613: Support ANSI Aggregate Function: regr_count
-SELECT regr_count(y, x) FROM testRegression;
-SELECT regr_count(y, x) FROM testRegression WHERE x IS NOT NULL;
-SELECT k, count(*), regr_count(y, x) FROM testRegression GROUP BY k;
-SELECT k, count(*) FILTER (WHERE x IS NOT NULL), regr_count(y, x) FROM testRegression GROUP BY k;
-
--- SPARK-37613: Support ANSI Aggregate Function: regr_r2
-SELECT regr_r2(y, x) FROM testRegression;
-SELECT regr_r2(y, x) FROM testRegression WHERE x IS NOT NULL;
-SELECT k, corr(y, x), regr_r2(y, x) FROM testRegression GROUP BY k;
-SELECT k, corr(y, x) FILTER (WHERE x IS NOT NULL), regr_r2(y, x) FROM testRegression GROUP BY k;
-
-- SPARK-27974: Support ANSI Aggregate Function: array_agg
SELECT
collect_list(col),
@@ -264,34 +248,6 @@
(1,4),(2,3),(1,4),(2,4) AS v(a,b)
GROUP BY a;
--- SPARK-37614: Support ANSI Aggregate Function: regr_avgx & regr_avgy
-SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression;
-SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL;
-SELECT k, avg(x), avg(y), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k;
-SELECT k, avg(x) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), avg(y) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k;
-
--- SPARK-37676: Support ANSI Aggregation Function: percentile_cont
-SELECT
- percentile_cont(0.25) WITHIN GROUP (ORDER BY v),
- percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC)
-FROM aggr;
-SELECT
- k,
- percentile_cont(0.25) WITHIN GROUP (ORDER BY v),
- percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC)
-FROM aggr
-GROUP BY k
-ORDER BY k;
--- SPARK-37691: Support ANSI Aggregation Function: percentile_disc
-SELECT
- percentile_disc(0.25) WITHIN GROUP (ORDER BY v),
- percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC)
-FROM aggr;
-SELECT
- k,
- percentile_disc(0.25) WITHIN GROUP (ORDER BY v),
- percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC)
-FROM aggr
-GROUP BY k
-ORDER BY k;
+SELECT mode(a), mode(b) FROM testData;
+SELECT a, mode(b) FROM testData GROUP BY a ORDER BY a;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/having.sql sql/core/src/test/resources/sql-tests/inputs/having.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/having.sql 2023-09-15 11:42:05.962317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/having.sql 2023-09-18 15:00:43.002708419 +0800
@@ -11,6 +11,9 @@
-- having condition contains grouping column
SELECT count(k) FROM hav GROUP BY v + 1 HAVING v + 1 = 2;
+-- invalid having condition contains grouping column
+SELECT count(k) FROM hav GROUP BY v HAVING v = array(1);
+
-- SPARK-11032: resolve having correctly
SELECT MIN(t.v) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(COUNT(1) > 0);
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql 2023-09-15 11:42:05.962317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql 2023-09-18 15:00:43.002708419 +0800
@@ -51,6 +51,12 @@
-- Aggregate a null array
select aggregate(cast(null as array<int>), 0, (a, y) -> a + y + 1, a -> a + 2) as v;
+-- alias for Aggregate.
+select reduce(ys, 0, (y, a) -> y + a + x) as v from nested;
+select reduce(ys, (0 as sum, 0 as n), (acc, x) -> (acc.sum + x, acc.n + 1), acc -> acc.sum / acc.n) as v from nested;
+select transform(zs, z -> reduce(z, 1, (acc, val) -> acc * val * size(z))) as v from nested;
+select reduce(cast(null as array<int>), 0, (a, y) -> a + y + 1, a -> a + 2) as v;
+
-- Check for element existence
select exists(ys, y -> y > 30) as v from nested;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/join-lateral.sql sql/core/src/test/resources/sql-tests/inputs/join-lateral.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/join-lateral.sql 2023-09-21 10:45:41.951461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/join-lateral.sql 2023-09-18 15:00:43.002708419 +0800
@@ -2,6 +2,8 @@
CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2);
CREATE VIEW t2(c1, c2) AS VALUES (0, 2), (0, 3);
+CREATE VIEW t3(c1, c2) AS VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4));
+CREATE VIEW t4(c1, c2) AS VALUES (0, 1), (0, 2), (1, 1), (1, 3);
-- lateral join with single column select
SELECT * FROM t1, LATERAL (SELECT c1);
@@ -44,6 +46,9 @@
-- lateral join with correlated non-equality predicates
SELECT * FROM t1, LATERAL (SELECT c2 FROM t2 WHERE t1.c2 < t2.c2);
+-- SPARK-36114: lateral join with aggregation and correlated non-equality predicates
+SELECT * FROM t1, LATERAL (SELECT max(c2) AS m FROM t2 WHERE t1.c2 < t2.c2);
+
-- lateral join can reference preceding FROM clause items
SELECT * FROM t1 JOIN t2 JOIN LATERAL (SELECT t1.c2 + t2.c2);
-- expect error: cannot resolve `t2.c1`
@@ -167,6 +172,197 @@
)
SELECT * FROM cte2;
+-- SPARK-41441: lateral join with outer references in Generate
+SELECT * FROM t3 JOIN LATERAL (SELECT EXPLODE(c2));
+SELECT * FROM t3 JOIN LATERAL (SELECT EXPLODE_OUTER(c2));
+SELECT * FROM t3 JOIN LATERAL (SELECT EXPLODE(c2)) t(c3) ON c1 = c3;
+SELECT * FROM t3 LEFT JOIN LATERAL (SELECT EXPLODE(c2)) t(c3) ON c1 = c3;
+
+-- Window func - unsupported
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT sum(t2.c2) over (order by t2.c1)
+ FROM t2
+ WHERE t2.c1 >= t1.c1);
+
+-- lateral join with union
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT t2.c2
+ FROM t2
+ WHERE t2.c1 = t1.c1
+ UNION ALL
+ SELECT t4.c2
+ FROM t4
+ WHERE t4.c1 = t1.c1);
+
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT t2.c2
+ FROM t2
+ WHERE t2.c1 = t1.c1
+ UNION DISTINCT
+ SELECT t4.c2
+ FROM t4
+ WHERE t4.c1 > t1.c2);
+
+-- COUNT bug with UNION in subquery
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT COUNT(t2.c2)
+ FROM t2
+ WHERE t2.c1 = t1.c1
+ UNION DISTINCT
+ SELECT COUNT(t4.c2)
+ FROM t4
+ WHERE t4.c1 > t1.c2);
+
+-- Both correlated and uncorrelated children
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT t2.c1, t2.c2
+ FROM t2
+ WHERE t2.c1 = t1.c1
+ UNION ALL
+ SELECT t4.c2, t4.c1
+ FROM t4
+ WHERE t4.c1 = t1.c1);
+
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT t2.c2
+ FROM t2
+ WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2
+ UNION ALL
+ SELECT t4.c2
+ FROM t4);
+
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT t2.c2
+ FROM t2
+ WHERE t2.c1 = t1.c1
+ UNION ALL
+ SELECT t4.c2
+ FROM t4);
+
+-- Correlation under group by
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT t2.c2
+ FROM t2
+ WHERE t2.c1 = t1.c1
+ GROUP BY t2.c2
+ UNION ALL
+ SELECT t4.c2
+ FROM t4
+ WHERE t4.c1 > t1.c2
+ GROUP BY t4.c2);
+
+-- Correlation in group by
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT t2.c1 - t1.c1
+ FROM t2
+ GROUP BY t2.c1 - t1.c1
+ UNION ALL
+ SELECT t4.c2
+ FROM t4
+ WHERE t4.c1 > t1.c2
+ GROUP BY t4.c2);
+
+-- Window func - unsupported
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT sum(t2.c2) over (order by t2.c1)
+ FROM t2
+ WHERE t2.c1 >= t1.c1
+ UNION ALL
+ SELECT t4.c2
+ FROM t4);
+
+-- lateral join under union
+(SELECT * FROM t1 JOIN LATERAL (SELECT * FROM t2 WHERE t2.c1 = t1.c1))
+UNION ALL
+(SELECT * FROM t1 JOIN t4);
+
+-- union above and below lateral join
+(SELECT * FROM t1 JOIN LATERAL
+ (SELECT t2.c2
+ FROM t2
+ WHERE t2.c1 = t1.c1
+ UNION ALL
+ SELECT t4.c2
+ FROM t4
+ WHERE t4.c1 = t1.c1))
+UNION ALL
+(SELECT * FROM t2 JOIN LATERAL
+ (SELECT t1.c2
+ FROM t1
+ WHERE t2.c1 <= t1.c1
+ UNION ALL
+ SELECT t4.c2
+ FROM t4
+ WHERE t4.c1 < t2.c1));
+
+-- SPARK-41961: lateral join with table-valued functions
+SELECT * FROM LATERAL EXPLODE(ARRAY(1, 2));
+SELECT * FROM t1, LATERAL RANGE(3);
+SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3);
+SELECT * FROM t3, LATERAL EXPLODE(c2) t2(v);
+SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v);
+SELECT * FROM EXPLODE(ARRAY(1, 2)) t(v), LATERAL (SELECT v + 1);
+
+-- lateral join with table-valued functions and join conditions
+SELECT * FROM t1 JOIN LATERAL EXPLODE(ARRAY(c1, c2)) t(c3) ON t1.c1 = c3;
+SELECT * FROM t3 JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3;
+SELECT * FROM t3 LEFT JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3;
+
+-- lateral join with table-valued functions in lateral subqueries
+SELECT * FROM t1, LATERAL (SELECT * FROM EXPLODE(ARRAY(c1, c2)));
+SELECT * FROM t1, LATERAL (SELECT t1.c1 + c3 FROM EXPLODE(ARRAY(c1, c2)) t(c3));
+SELECT * FROM t1, LATERAL (SELECT t1.c1 + c3 FROM EXPLODE(ARRAY(c1, c2)) t(c3) WHERE t1.c2 > 1);
+SELECT * FROM t1, LATERAL (SELECT * FROM EXPLODE(ARRAY(c1, c2)) l(x) JOIN EXPLODE(ARRAY(c2, c1)) r(y) ON x = y);
+
+-- SPARK-42119: lateral join with table-valued functions inline and inline_outer;
+CREATE OR REPLACE TEMPORARY VIEW array_struct(id, arr) AS VALUES
+ (1, ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))),
+ (2, ARRAY()),
+ (3, ARRAY(STRUCT(3, 'c')));
+SELECT * FROM t1, LATERAL INLINE(ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b')));
+SELECT c1, t.* FROM t1, LATERAL INLINE(ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))) t(x, y);
+SELECT * FROM array_struct JOIN LATERAL INLINE(arr);
+SELECT * FROM array_struct LEFT JOIN LATERAL INLINE(arr) t(k, v) ON id = k;
+SELECT * FROM array_struct JOIN LATERAL INLINE_OUTER(arr);
+DROP VIEW array_struct;
+
+-- lateral join with table-valued functions posexplode and posexplode_outer
+SELECT * FROM LATERAL posexplode(ARRAY(1, 2));
+SELECT * FROM t1, LATERAL posexplode(ARRAY(c1, c2)) t2(pos, c3);
+SELECT * FROM t1 JOIN LATERAL posexplode(ARRAY(c1, c2)) t(pos, c3) ON t1.c1 = c3;
+SELECT * FROM t3, LATERAL posexplode(c2) t2(pos, v);
+SELECT * FROM t3 JOIN LATERAL posexplode(c2) t(pos, c3) ON t3.c1 = c3;
+SELECT * FROM t3, LATERAL posexplode_outer(c2) t2(pos, v);
+SELECT * FROM t3 LEFT JOIN LATERAL posexplode(c2) t(pos, c3) ON t3.c1 = c3;
+SELECT * FROM t3 LEFT JOIN LATERAL posexplode_outer(c2) t(pos, c3) ON t3.c1 = c3;
+
+-- lateral join with table-valued function json_tuple
+CREATE OR REPLACE TEMP VIEW json_table(key, jstring) AS VALUES
+ ('1', '{"f1": "1", "f2": "2", "f3": 3, "f5": 5.23}'),
+ ('2', '{"f1": "1", "f3": "3", "f2": 2, "f4": 4.01}'),
+ ('3', '{"f1": 3, "f4": "4", "f3": "3", "f2": 2, "f5": 5.01}'),
+ ('4', cast(null as string)),
+ ('5', '{"f1": null, "f5": ""}'),
+ ('6', '[invalid JSON string]');
+SELECT t1.key, t2.* FROM json_table t1, LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2;
+SELECT t1.key, t2.* FROM json_table t1, LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2 WHERE t2.c0 IS NOT NULL;
+SELECT t1.key, t2.* FROM json_table t1
+ JOIN LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2(f1, f2, f3, f4, f5)
+ ON t1.key = t2.f1;
+SELECT t1.key, t2.* FROM json_table t1
+ LEFT JOIN LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2(f1, f2, f3, f4, f5)
+ ON t1.key = t2.f1;
+DROP VIEW json_table;
+
+-- lateral join with table-valued function stack
+SELECT t.* FROM t1, LATERAL stack(2, 'Key', c1, 'Value', c2) t;
+SELECT t.* FROM t1 JOIN LATERAL stack(1, c1, c2) t(x, y);
+SELECT t.* FROM t1 JOIN t3 ON t1.c1 = t3.c1 JOIN LATERAL stack(1, t1.c2, t3.c2) t;
+-- expect error
+SELECT t.* FROM t1, LATERAL stack(c1, c2);
+
-- clean up
DROP VIEW t1;
DROP VIEW t2;
+DROP VIEW t3;
+DROP VIEW t4;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/linear-regression.sql sql/core/src/test/resources/sql-tests/inputs/linear-regression.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/linear-regression.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/linear-regression.sql 2023-09-18 15:00:43.002708419 +0800
@@ -0,0 +1,52 @@
+-- Test data.
+CREATE OR REPLACE TEMPORARY VIEW testRegression AS SELECT * FROM VALUES
+(1, 10, null), (2, 10, 11), (2, 20, 22), (2, 25, null), (2, 30, 35)
+AS testRegression(k, y, x);
+
+-- SPARK-37613: Support ANSI Aggregate Function: regr_count
+SELECT regr_count(y, x) FROM testRegression;
+SELECT regr_count(y, x) FROM testRegression WHERE x IS NOT NULL;
+SELECT k, count(*), regr_count(y, x) FROM testRegression GROUP BY k;
+SELECT k, count(*) FILTER (WHERE x IS NOT NULL), regr_count(y, x) FROM testRegression GROUP BY k;
+
+-- SPARK-37613: Support ANSI Aggregate Function: regr_r2
+SELECT regr_r2(y, x) FROM testRegression;
+SELECT regr_r2(y, x) FROM testRegression WHERE x IS NOT NULL;
+SELECT k, corr(y, x), regr_r2(y, x) FROM testRegression GROUP BY k;
+SELECT k, corr(y, x) FILTER (WHERE x IS NOT NULL), regr_r2(y, x) FROM testRegression GROUP BY k;
+
+-- SPARK-37614: Support ANSI Aggregate Function: regr_avgx & regr_avgy
+SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression;
+SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL;
+SELECT k, avg(x), avg(y), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k;
+SELECT k, avg(x) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), avg(y) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k;
+
+-- SPARK-37672: Support ANSI Aggregate Function: regr_sxx
+SELECT regr_sxx(y, x) FROM testRegression;
+SELECT regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL;
+SELECT k, regr_sxx(y, x) FROM testRegression GROUP BY k;
+SELECT k, regr_sxx(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k;
+
+-- SPARK-37681: Support ANSI Aggregate Function: regr_sxy
+SELECT regr_sxy(y, x) FROM testRegression;
+SELECT regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL;
+SELECT k, regr_sxy(y, x) FROM testRegression GROUP BY k;
+SELECT k, regr_sxy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k;
+
+-- SPARK-37702: Support ANSI Aggregate Function: regr_syy
+SELECT regr_syy(y, x) FROM testRegression;
+SELECT regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL;
+SELECT k, regr_syy(y, x) FROM testRegression GROUP BY k;
+SELECT k, regr_syy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k;
+
+-- SPARK-39230: Support ANSI Aggregate Function: regr_slope
+SELECT regr_slope(y, x) FROM testRegression;
+SELECT regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL;
+SELECT k, regr_slope(y, x) FROM testRegression GROUP BY k;
+SELECT k, regr_slope(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k;
+
+-- SPARK-37623: Support ANSI Aggregate Function: regr_intercept
+SELECT regr_intercept(y, x) FROM testRegression;
+SELECT regr_intercept(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL;
+SELECT k, regr_intercept(y, x) FROM testRegression GROUP BY k;
+SELECT k, regr_intercept(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL GROUP BY k;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/mask-functions.sql sql/core/src/test/resources/sql-tests/inputs/mask-functions.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/mask-functions.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/mask-functions.sql 2023-09-18 15:00:43.002708419 +0800
@@ -0,0 +1,58 @@
+-- mask function
+SELECT mask('AbCD123-@$#');
+SELECT mask('AbCD123-@$#', 'Q');
+SELECT mask('AbCD123-@$#', 'Q', 'q');
+SELECT mask('AbCD123-@$#', 'Q', 'q', 'd');
+SELECT mask('AbCD123-@$#', 'Q', 'q', 'd', 'o');
+SELECT mask('AbCD123-@$#', 'Qa', 'qa', 'da', 'oa');
+SELECT mask('AbCD123-@$#', NULL, 'q', 'd', 'o');
+SELECT mask('AbCD123-@$#', NULL, NULL, 'd', 'o');
+SELECT mask('AbCD123-@$#', NULL, NULL, NULL, 'o');
+SELECT mask('AbCD123-@$#', NULL, NULL, NULL, NULL);
+SELECT mask(NULL);
+SELECT mask(NULL, NULL, 'q', 'd', 'o');
+SELECT mask(NULL, NULL, NULL, 'd', 'o');
+SELECT mask(NULL, NULL, NULL, NULL, 'o');
+SELECT mask('AbCD123-@$#', NULL, NULL, NULL, NULL);
+SELECT mask(c1) from values ('AbCD123-@$#') as tab(c1);
+SELECT mask(c1, 'Q') from values ('AbCD123-@$#') as tab(c1);
+SELECT mask(c1, 'Q', 'q')from values ('AbCD123-@$#') as tab(c1);
+SELECT mask(c1, 'Q', 'q', 'd') from values ('AbCD123-@$#') as tab(c1);
+SELECT mask(c1, 'Q', 'q', 'd', 'o') from values ('AbCD123-@$#') as tab(c1);
+SELECT mask(c1, NULL, 'q', 'd', 'o') from values ('AbCD123-@$#') as tab(c1);
+SELECT mask(c1, NULL, NULL, 'd', 'o') from values ('AbCD123-@$#') as tab(c1);
+SELECT mask(c1, NULL, NULL, NULL, 'o') from values ('AbCD123-@$#') as tab(c1);
+SELECT mask(c1, NULL, NULL, NULL, NULL) from values ('AbCD123-@$#') as tab(c1);
+SELECT mask(c1, NULL, 'q', 'd', 'o') from values ('AbCD123-@$#') as tab(c1);
+SELECT mask(c1, 'Q', NULL, 'd', 'o') from values ('AbCD123-@$#') as tab(c1);
+SELECT mask(c1, 'Q', 'q', NULL, 'o') from values ('AbCD123-@$#') as tab(c1);
+SELECT mask(c1, 'Q', 'q', 'd', NULL) from values ('AbCD123-@$#') as tab(c1);
+SELECT mask(NULL, 'Q', 'q', 'd', NULL) from values ('AbCD123-@$#') as tab(c1);
+SELECT mask('abcd-EFGH-8765-4321');
+SELECT mask('abcd-EFGH-8765-4321', 'Q');
+SELECT mask('abcd-EFGH-8765-4321', 'Q', 'q');
+SELECT mask('abcd-EFGH-8765-4321', 'Q', 'q', 'd');
+SELECT mask('abcd-EFGH-8765-4321', 'Q', 'q', 'd', '*');
+SELECT mask('abcd-EFGH-8765-4321', NULL, 'q', 'd', '*');
+SELECT mask('abcd-EFGH-8765-4321', NULL, NULL, 'd', '*');
+SELECT mask('abcd-EFGH-8765-4321', NULL, NULL, NULL, '*');
+SELECT mask('abcd-EFGH-8765-4321', NULL, NULL, NULL, NULL);
+SELECT mask(NULL);
+SELECT mask(NULL, NULL, 'q', 'd', '*');
+SELECT mask(NULL, NULL, NULL, 'd', '*');
+SELECT mask(NULL, NULL, NULL, NULL, '*');
+SELECT mask(NULL, NULL, NULL, NULL, NULL);
+SELECT mask(c1) from values ('abcd-EFGH-8765-4321') as tab(c1);
+SELECT mask(c1, 'Q') from values ('abcd-EFGH-8765-4321') as tab(c1);
+SELECT mask(c1, 'Q', 'q')from values ('abcd-EFGH-8765-4321') as tab(c1);
+SELECT mask(c1, 'Q', 'q', 'd') from values ('abcd-EFGH-8765-4321') as tab(c1);
+SELECT mask(c1, 'Q', 'q', 'd', '*') from values ('abcd-EFGH-8765-4321') as tab(c1);
+SELECT mask(c1, NULL, 'q', 'd', '*') from values ('abcd-EFGH-8765-4321') as tab(c1);
+SELECT mask(c1, NULL, NULL, 'd', '*') from values ('abcd-EFGH-8765-4321') as tab(c1);
+SELECT mask(c1, NULL, NULL, NULL, '*') from values ('abcd-EFGH-8765-4321') as tab(c1);
+SELECT mask(c1, NULL, NULL, NULL, NULL) from values ('abcd-EFGH-8765-4321') as tab(c1);
+SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 'a') as t(c1, replaceArg);
+SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 'ABC') as t(c1, replaceArg);
+SELECT mask(c1, replaceArg) from values('abcd-EFGH-8765-4321', 123) as t(c1, replaceArg);
+SELECT mask('abcd-EFGH-8765-4321', 'A', 'w', '');
+SELECT mask('abcd-EFGH-8765-4321', 'A', 'abc');
\ No newline at end of file
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/math.sql sql/core/src/test/resources/sql-tests/inputs/math.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/math.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/math.sql 2023-09-18 15:00:43.002708419 +0800
@@ -0,0 +1,79 @@
+-- Round with Byte input
+SELECT round(25y, 1);
+SELECT round(25y, 0);
+SELECT round(25y, -1);
+SELECT round(25y, -2);
+SELECT round(25y, -3);
+SELECT round(127y, -1);
+SELECT round(-128y, -1);
+
+-- Round with short integer input
+SELECT round(525s, 1);
+SELECT round(525s, 0);
+SELECT round(525s, -1);
+SELECT round(525s, -2);
+SELECT round(525s, -3);
+SELECT round(32767s, -1);
+SELECT round(-32768s, -1);
+
+-- Round with integer input
+SELECT round(525, 1);
+SELECT round(525, 0);
+SELECT round(525, -1);
+SELECT round(525, -2);
+SELECT round(525, -3);
+SELECT round(2147483647, -1);
+SELECT round(-2147483647, -1);
+
+-- Round with big integer input
+SELECT round(525L, 1);
+SELECT round(525L, 0);
+SELECT round(525L, -1);
+SELECT round(525L, -2);
+SELECT round(525L, -3);
+SELECT round(9223372036854775807L, -1);
+SELECT round(-9223372036854775808L, -1);
+
+-- Bround with byte input
+SELECT bround(25y, 1);
+SELECT bround(25y, 0);
+SELECT bround(25y, -1);
+SELECT bround(25y, -2);
+SELECT bround(25y, -3);
+SELECT bround(127y, -1);
+SELECT bround(-128y, -1);
+
+-- Bround with Short input
+SELECT bround(525s, 1);
+SELECT bround(525s, 0);
+SELECT bround(525s, -1);
+SELECT bround(525s, -2);
+SELECT bround(525s, -3);
+SELECT bround(32767s, -1);
+SELECT bround(-32768s, -1);
+
+-- Bround with integer input
+SELECT bround(525, 1);
+SELECT bround(525, 0);
+SELECT bround(525, -1);
+SELECT bround(525, -2);
+SELECT bround(525, -3);
+SELECT bround(2147483647, -1);
+SELECT bround(-2147483647, -1);
+
+-- Bround with big integer input
+SELECT bround(525L, 1);
+SELECT bround(525L, 0);
+SELECT bround(525L, -1);
+SELECT bround(525L, -2);
+SELECT bround(525L, -3);
+SELECT bround(9223372036854775807L, -1);
+SELECT bround(-9223372036854775808L, -1);
+
+-- Conv
+SELECT conv('100', 2, 10);
+SELECT conv(-10, 16, -10);
+SELECT conv('9223372036854775808', 10, 16);
+SELECT conv('92233720368547758070', 10, 16);
+SELECT conv('9223372036854775807', 36, 10);
+SELECT conv('-9223372036854775807', 36, 10);
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/non-excludable-rule.sql sql/core/src/test/resources/sql-tests/inputs/non-excludable-rule.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/non-excludable-rule.sql 2023-09-21 10:45:41.951461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/non-excludable-rule.sql 2023-09-18 15:00:43.002708419 +0800
@@ -4,3 +4,16 @@
(SELECT min(id) FROM range(10)),
(SELECT sum(id) FROM range(10)),
(SELECT count(distinct id) FROM range(10));
+
+-- SPARK-39444
+SET spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.Optimizer$OptimizeSubqueries;
+WITH tmp AS (
+ SELECT id FROM range(2)
+ INTERSECT
+ SELECT id FROM range(4)
+)
+SELECT id FROM range(3) WHERE id > (SELECT max(id) FROM tmp);
+
+-- SPARK-36979
+SET spark.sql.optimizer.excludedRules=org.apache.spark.sql.catalyst.optimizer.RewriteLateralSubquery;
+SELECT * FROM testData, LATERAL (SELECT * FROM testData) LIMIT 1;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/null-handling.sql sql/core/src/test/resources/sql-tests/inputs/null-handling.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/null-handling.sql 2023-09-15 11:42:05.962317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/null-handling.sql 2023-09-18 15:00:43.002708419 +0800
@@ -45,4 +45,7 @@
select a+140 from t1 where not (b<10 AND c=1);
select a+150 from t1 where not (c=1 AND b<10);
+-- null-safe comparisons
+select b, c, equal_null(b, c), equal_null(c, b) from t1;
+
drop table t1;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/order-by-all.sql sql/core/src/test/resources/sql-tests/inputs/order-by-all.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/order-by-all.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/order-by-all.sql 2023-09-18 15:00:43.002708419 +0800
@@ -0,0 +1,44 @@
+create temporary view data as select * from values
+ (0, 1),
+ (0, 2),
+ (1, 3),
+ (1, NULL)
+ as data(g, i);
+
+-- most basic test with only 1 column
+select g from data order by all;
+
+-- two columns
+select * from data order by all;
+
+-- test case insensitive
+select * from data order by aLl;
+
+-- asc/desc
+select * from data order by all asc;
+select * from data order by all desc;
+
+-- nulls first / last
+select * from data order by all nulls first;
+select * from data order by all nulls last;
+
+-- combining nulls first/last/asc/desc
+select * from data order by all asc nulls first;
+select * from data order by all desc nulls first;
+select * from data order by all asc nulls last;
+select * from data order by all desc nulls last;
+
+-- set operations from duckdb
+select * from data union all select * from data order by all;
+select * from data union select * from data order by all;
+
+-- limit
+select * from data order by all limit 2;
+
+-- precedence: if there's a column already named all, reference that, instead of expanding.
+-- result should be 1, 2, 3, and not 3, 2, 1
+select * from values("z", 1), ("y", 2), ("x", 3) AS T(col1, all) order by all;
+
+-- shouldn't work in window functions
+select name, dept, rank() over (partition by dept order by all) as rank
+from values('Lisa', 'Sales', 10000, 35) as T(name, dept, salary, age);
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/percentiles.sql sql/core/src/test/resources/sql-tests/inputs/percentiles.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/percentiles.sql 2023-09-21 10:45:41.951461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/percentiles.sql 2023-09-18 15:00:43.002708419 +0800
@@ -1,74 +1,302 @@
--- SPARK-44871: Fix percentile_disc behaviour
+-- Test data.
+CREATE OR REPLACE TEMPORARY VIEW aggr AS SELECT * FROM VALUES
+(0, 0), (0, 10), (0, 20), (0, 30), (0, 40), (1, 10), (1, 20), (2, 10), (2, 20), (2, 25), (2, 30), (3, 60), (4, null)
+AS aggr(k, v);
+
+CREATE OR REPLACE TEMPORARY VIEW basic_pays AS SELECT * FROM VALUES
+('Diane Murphy','Accounting',8435),
+('Mary Patterson','Accounting',9998),
+('Jeff Firrelli','Accounting',8992),
+('William Patterson','Accounting',8870),
+('Gerard Bondur','Accounting',11472),
+('Anthony Bow','Accounting',6627),
+('Leslie Jennings','IT',8113),
+('Leslie Thompson','IT',5186),
+('Julie Firrelli','Sales',9181),
+('Steve Patterson','Sales',9441),
+('Foon Yue Tseng','Sales',6660),
+('George Vanauf','Sales',10563),
+('Loui Bondur','SCM',10449),
+('Gerard Hernandez','SCM',6949),
+('Pamela Castillo','SCM',11303),
+('Larry Bott','SCM',11798),
+('Barry Jones','SCM',10586)
+AS basic_pays(employee_name, department, salary);
+
+SELECT
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY v),
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0),
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC),
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0)
+FROM aggr;
+
+SELECT
+ k,
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY v),
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0),
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC),
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0)
+FROM aggr
+GROUP BY k
+ORDER BY k;
+
+SELECT
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY v),
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0),
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC),
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0)
+FROM aggr;
+
+SELECT
+ k,
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY v),
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY v) FILTER (WHERE k > 0),
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC),
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC) FILTER (WHERE k > 0)
+FROM aggr
+GROUP BY k
+ORDER BY k;
+
+SELECT
+ median(v),
+ percentile(v, 0.5),
+ percentile_cont(0.5) WITHIN GROUP (ORDER BY v)
+FROM aggr;
+
+SELECT
+ k,
+ median(v),
+ percentile(v, 0.5),
+ percentile_cont(0.5) WITHIN GROUP (ORDER BY v)
+FROM aggr
+GROUP BY k
+ORDER BY k;
+
+SELECT
+ employee_name,
+ department,
+ salary,
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department),
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department),
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department),
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department)
+FROM basic_pays
+ORDER BY salary;
+
+SELECT
+ employee_name,
+ department,
+ salary,
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary),
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ORDER BY salary)
+FROM basic_pays
+ORDER BY salary;
+
+SELECT
+ employee_name,
+ department,
+ salary,
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary),
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ORDER BY salary)
+FROM basic_pays
+ORDER BY salary;
+
+SELECT
+ employee_name,
+ department,
+ salary,
+ median(salary) OVER (PARTITION BY department ORDER BY salary)
+FROM basic_pays
+ORDER BY salary;
+
+SELECT
+ employee_name,
+ department,
+ salary,
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING),
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)
+FROM basic_pays
+ORDER BY salary;
+
+SELECT
+ employee_name,
+ department,
+ salary,
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING),
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)
+FROM basic_pays
+ORDER BY salary;
+
+SELECT
+ employee_name,
+ department,
+ salary,
+ median(salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)
+FROM basic_pays
+ORDER BY salary;
+
+SELECT
+ employee_name,
+ department,
+ salary,
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w,
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w,
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w,
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w
+FROM basic_pays
+WINDOW w AS (PARTITION BY department)
+ORDER BY salary;
+
SELECT
- percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0,
- percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1,
- percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2,
- percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3,
- percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4,
- percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5,
- percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6,
- percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7,
- percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8,
- percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9,
- percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10
-FROM VALUES (0) AS v(a);
-
-SELECT
- percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0,
- percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1,
- percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2,
- percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3,
- percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4,
- percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5,
- percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6,
- percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7,
- percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8,
- percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9,
- percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10
-FROM VALUES (0), (1) AS v(a);
-
-SELECT
- percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0,
- percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1,
- percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2,
- percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3,
- percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4,
- percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5,
- percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6,
- percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7,
- percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8,
- percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9,
- percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10
-FROM VALUES (0), (1), (2) AS v(a);
-
-SELECT
- percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0,
- percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1,
- percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2,
- percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3,
- percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4,
- percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5,
- percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6,
- percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7,
- percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8,
- percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9,
- percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10
-FROM VALUES (0), (1), (2), (3), (4) AS v(a);
-
-SET spark.sql.legacy.percentileDiscCalculation = true;
-
-SELECT
- percentile_disc(0.0) WITHIN GROUP (ORDER BY a) as p0,
- percentile_disc(0.1) WITHIN GROUP (ORDER BY a) as p1,
- percentile_disc(0.2) WITHIN GROUP (ORDER BY a) as p2,
- percentile_disc(0.3) WITHIN GROUP (ORDER BY a) as p3,
- percentile_disc(0.4) WITHIN GROUP (ORDER BY a) as p4,
- percentile_disc(0.5) WITHIN GROUP (ORDER BY a) as p5,
- percentile_disc(0.6) WITHIN GROUP (ORDER BY a) as p6,
- percentile_disc(0.7) WITHIN GROUP (ORDER BY a) as p7,
- percentile_disc(0.8) WITHIN GROUP (ORDER BY a) as p8,
- percentile_disc(0.9) WITHIN GROUP (ORDER BY a) as p9,
- percentile_disc(1.0) WITHIN GROUP (ORDER BY a) as p10
-FROM VALUES (0), (1), (2), (3), (4) AS v(a);
+ employee_name,
+ department,
+ salary,
+ median(salary) OVER w,
+ percentile_cont(0.5) WITHIN GROUP (ORDER BY salary) OVER w,
+ percentile_disc(0.5) WITHIN GROUP (ORDER BY salary) OVER w,
+ percentile_cont(0.5) WITHIN GROUP (ORDER BY salary DESC) OVER w,
+ percentile_disc(0.5) WITHIN GROUP (ORDER BY salary DESC) OVER w
+FROM basic_pays
+WHERE salary > 8900
+WINDOW w AS (PARTITION BY department)
+ORDER BY salary;
-SET spark.sql.legacy.percentileDiscCalculation = false;
+SELECT
+ employee_name,
+ department,
+ salary,
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w,
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w
+FROM basic_pays
+WINDOW w AS (PARTITION BY department ORDER BY salary)
+ORDER BY salary;
+
+SELECT
+ employee_name,
+ department,
+ salary,
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w,
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w
+FROM basic_pays
+WINDOW w AS (PARTITION BY department ORDER BY salary)
+ORDER BY salary;
+
+SELECT
+ employee_name,
+ department,
+ salary,
+ median(salary) OVER w
+FROM basic_pays
+WINDOW w AS (PARTITION BY department ORDER BY salary)
+ORDER BY salary;
+
+SELECT
+ employee_name,
+ department,
+ salary,
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w,
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w
+FROM basic_pays
+WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)
+ORDER BY salary;
+
+SELECT
+ employee_name,
+ department,
+ salary,
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w,
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w
+FROM basic_pays
+WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)
+ORDER BY salary;
+
+SELECT
+ employee_name,
+ department,
+ salary,
+ median(salary) OVER w
+FROM basic_pays
+WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)
+ORDER BY salary;
+
+CREATE OR REPLACE TEMPORARY VIEW intervals AS SELECT * FROM VALUES
+(0, INTERVAL '0' MONTH, INTERVAL '0' SECOND, INTERVAL '0' MINUTE),
+(0, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE),
+(0, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE),
+(0, INTERVAL '30' MONTH, INTERVAL '30' SECOND, INTERVAL '30' MINUTE),
+(0, INTERVAL '40' MONTH, INTERVAL '40' SECOND, INTERVAL '40' MINUTE),
+(1, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE),
+(1, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE),
+(2, INTERVAL '10' MONTH, INTERVAL '10' SECOND, INTERVAL '10' MINUTE),
+(2, INTERVAL '20' MONTH, INTERVAL '20' SECOND, INTERVAL '20' MINUTE),
+(2, INTERVAL '25' MONTH, INTERVAL '25' SECOND, INTERVAL '25' MINUTE),
+(2, INTERVAL '30' MONTH, INTERVAL '30' SECOND, INTERVAL '30' MINUTE),
+(3, INTERVAL '60' MONTH, INTERVAL '60' SECOND, INTERVAL '60' MINUTE),
+(4, null, null, null)
+AS intervals(k, dt, ym, dt2);
+
+SELECT
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY dt),
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY dt DESC)
+FROM intervals;
+
+SELECT
+ k,
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY ym),
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY ym DESC)
+FROM intervals
+GROUP BY k
+ORDER BY k;
+
+SELECT
+ k,
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY dt2),
+ percentile_cont(0.25) WITHIN GROUP (ORDER BY dt2 DESC)
+FROM intervals
+GROUP BY k
+ORDER BY k;
+
+SELECT
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY dt),
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY dt DESC)
+FROM intervals;
+
+SELECT
+ k,
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY ym),
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY ym DESC)
+FROM intervals
+GROUP BY k
+ORDER BY k;
+
+SELECT
+ k,
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY dt2),
+ percentile_disc(0.25) WITHIN GROUP (ORDER BY dt2 DESC)
+FROM intervals
+GROUP BY k
+ORDER BY k;
+
+SELECT
+ median(dt),
+ percentile(dt, 0.5),
+ percentile_cont(0.5) WITHIN GROUP (ORDER BY dt)
+FROM intervals;
+
+SELECT
+ k,
+ median(ym),
+ percentile(ym, 0.5),
+ percentile_cont(0.5) WITHIN GROUP (ORDER BY ym)
+FROM intervals
+GROUP BY k
+ORDER BY k;
+
+SELECT
+ k,
+ median(dt2),
+ percentile(dt2, 0.5),
+ percentile_cont(0.5) WITHIN GROUP (ORDER BY dt2)
+FROM intervals
+GROUP BY k
+ORDER BY k;
\ No newline at end of file
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql 2023-09-18 15:00:43.002708419 +0800
@@ -81,29 +81,28 @@
-- SQL2003 binary aggregates [SPARK-23907]
SELECT regr_count(b, a) FROM aggtest;
--- SELECT regr_sxx(b, a) FROM aggtest;
--- SELECT regr_syy(b, a) FROM aggtest;
--- SELECT regr_sxy(b, a) FROM aggtest;
+SELECT regr_sxx(b, a) FROM aggtest;
+SELECT regr_syy(b, a) FROM aggtest;
+SELECT regr_sxy(b, a) FROM aggtest;
SELECT regr_avgx(b, a), regr_avgy(b, a) FROM aggtest;
SELECT regr_r2(b, a) FROM aggtest;
--- SELECT regr_slope(b, a), regr_intercept(b, a) FROM aggtest;
+SELECT regr_slope(b, a), regr_intercept(b, a) FROM aggtest;
SELECT covar_pop(b, a), covar_samp(b, a) FROM aggtest;
SELECT corr(b, a) FROM aggtest;
-- test accum and combine functions directly [SPARK-23907]
--- CREATE TABLE regr_test (x float8, y float8);
--- INSERT INTO regr_test VALUES (10,150),(20,250),(30,350),(80,540),(100,200);
--- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x)
--- FROM regr_test WHERE x IN (10,20,30,80);
--- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x)
--- FROM regr_test;
+CREATE TEMPORARY VIEW regr_test AS SELECT * FROM VALUES (10,150),(20,250),(30,350),(80,540),(100,200) AS regr_test (x, y);
+SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x)
+FROM regr_test WHERE x IN (10,20,30,80);
+SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x)
+FROM regr_test;
-- SELECT float8_accum('{4,140,2900}'::float8[], 100);
-- SELECT float8_regr_accum('{4,140,2900,1290,83075,15050}'::float8[], 200, 100);
--- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x)
--- FROM regr_test WHERE x IN (10,20,30);
--- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x)
--- FROM regr_test WHERE x IN (80,100);
+SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x)
+FROM regr_test WHERE x IN (10,20,30);
+SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x)
+FROM regr_test WHERE x IN (80,100);
-- SELECT float8_combine('{3,60,200}'::float8[], '{0,0,0}'::float8[]);
-- SELECT float8_combine('{0,0,0}'::float8[], '{2,180,200}'::float8[]);
-- SELECT float8_combine('{3,60,200}'::float8[], '{2,180,200}'::float8[]);
@@ -113,7 +112,7 @@
-- '{2,180,200,740,57800,-3400}'::float8[]);
-- SELECT float8_regr_combine('{3,60,200,750,20000,2000}'::float8[],
-- '{2,180,200,740,57800,-3400}'::float8[]);
--- DROP TABLE regr_test;
+DROP VIEW regr_test;
-- test count, distinct
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql 2023-09-15 11:42:05.966317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql 2023-09-18 15:00:43.002708419 +0800
@@ -101,27 +101,26 @@
SELECT min(q1), min(q2) FROM INT8_TBL;
SELECT max(q1), max(q2) FROM INT8_TBL;
--- [SPARK-28137] Missing Data Type Formatting Functions
-- TO_CHAR()
---
--- SELECT '' AS to_char_1, to_char(q1, '9G999G999G999G999G999'), to_char(q2, '9,999,999,999,999,999')
--- FROM INT8_TBL;
+-- some queries are commented out as the format string is not supported by Spark
+SELECT '' AS to_char_1, to_char(q1, '9G999G999G999G999G999'), to_char(q2, '9,999,999,999,999,999')
+FROM INT8_TBL;
-- SELECT '' AS to_char_2, to_char(q1, '9G999G999G999G999G999D999G999'), to_char(q2, '9,999,999,999,999,999.999,999')
--- FROM INT8_TBL;
+-- FROM INT8_TBL;
--- SELECT '' AS to_char_3, to_char( (q1 * -1), '9999999999999999PR'), to_char( (q2 * -1), '9999999999999999.999PR')
--- FROM INT8_TBL;
+SELECT '' AS to_char_3, to_char( (q1 * -1), '9999999999999999PR'), to_char( (q2 * -1), '9999999999999999.999PR')
+FROM INT8_TBL;
--- SELECT '' AS to_char_4, to_char( (q1 * -1), '9999999999999999S'), to_char( (q2 * -1), 'S9999999999999999')
--- FROM INT8_TBL;
+SELECT '' AS to_char_4, to_char( (q1 * -1), '9999999999999999S'), to_char( (q2 * -1), 'S9999999999999999')
+FROM INT8_TBL;
--- SELECT '' AS to_char_5, to_char(q2, 'MI9999999999999999') FROM INT8_TBL;
+SELECT '' AS to_char_5, to_char(q2, 'MI9999999999999999') FROM INT8_TBL;
-- SELECT '' AS to_char_6, to_char(q2, 'FMS9999999999999999') FROM INT8_TBL;
-- SELECT '' AS to_char_7, to_char(q2, 'FM9999999999999999THPR') FROM INT8_TBL;
-- SELECT '' AS to_char_8, to_char(q2, 'SG9999999999999999th') FROM INT8_TBL;
--- SELECT '' AS to_char_9, to_char(q2, '0999999999999999') FROM INT8_TBL;
--- SELECT '' AS to_char_10, to_char(q2, 'S0999999999999999') FROM INT8_TBL;
+SELECT '' AS to_char_9, to_char(q2, '0999999999999999') FROM INT8_TBL;
+SELECT '' AS to_char_10, to_char(q2, 'S0999999999999999') FROM INT8_TBL;
-- SELECT '' AS to_char_11, to_char(q2, 'FM0999999999999999') FROM INT8_TBL;
-- SELECT '' AS to_char_12, to_char(q2, 'FM9999999999999999.000') FROM INT8_TBL;
-- SELECT '' AS to_char_13, to_char(q2, 'L9999999999999999.000') FROM INT8_TBL;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/limit.sql sql/core/src/test/resources/sql-tests/inputs/postgreSQL/limit.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/limit.sql 2023-09-15 11:42:05.966317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/postgreSQL/limit.sql 2023-09-18 15:00:43.002708419 +0800
@@ -12,25 +12,24 @@
SELECT '' AS two, unique1, unique2, stringu1
FROM onek WHERE unique1 > 60 AND unique1 < 63
ORDER BY unique1 LIMIT 5;
--- [SPARK-28330] ANSI SQL: Top-level <result offset clause> in <query expression>
--- SELECT '' AS three, unique1, unique2, stringu1
--- FROM onek WHERE unique1 > 100
--- ORDER BY unique1 LIMIT 3 OFFSET 20;
--- SELECT '' AS zero, unique1, unique2, stringu1
--- FROM onek WHERE unique1 < 50
--- ORDER BY unique1 DESC LIMIT 8 OFFSET 99;
--- SELECT '' AS eleven, unique1, unique2, stringu1
--- FROM onek WHERE unique1 < 50
--- ORDER BY unique1 DESC LIMIT 20 OFFSET 39;
--- SELECT '' AS ten, unique1, unique2, stringu1
--- FROM onek
--- ORDER BY unique1 OFFSET 990;
+SELECT '' AS three, unique1, unique2, stringu1
+ FROM onek WHERE unique1 > 100
+ ORDER BY unique1 LIMIT 3 OFFSET 20;
+SELECT '' AS zero, unique1, unique2, stringu1
+ FROM onek WHERE unique1 < 50
+ ORDER BY unique1 DESC LIMIT 8 OFFSET 99;
+SELECT '' AS eleven, unique1, unique2, stringu1
+ FROM onek WHERE unique1 < 50
+ ORDER BY unique1 DESC LIMIT 20 OFFSET 39;
+SELECT '' AS ten, unique1, unique2, stringu1
+ FROM onek
+ ORDER BY unique1 OFFSET 990;
-- SELECT '' AS five, unique1, unique2, stringu1
-- FROM onek
-- ORDER BY unique1 OFFSET 990 LIMIT 5;
--- SELECT '' AS five, unique1, unique2, stringu1
--- FROM onek
--- ORDER BY unique1 LIMIT 5 OFFSET 900;
+SELECT '' AS five, unique1, unique2, stringu1
+ FROM onek
+ ORDER BY unique1 LIMIT 5 OFFSET 900;
CREATE OR REPLACE TEMPORARY VIEW INT8_TBL AS SELECT * FROM
(VALUES
@@ -45,8 +44,7 @@
-- constant, so to ensure executor is exercised, do this:
-- [SPARK-29650] Discard a NULL constant in LIMIT
select * from int8_tbl limit (case when random() < 0.5 then bigint(null) end);
--- [SPARK-28330] ANSI SQL: Top-level <result offset clause> in <query expression>
--- select * from int8_tbl offset (case when random() < 0.5 then bigint(null) end);
+select * from int8_tbl offset (case when random() < 0.5 then bigint(null) end);
-- Test assorted cases involving backwards fetch from a LIMIT plan node
-- [SPARK-20965] Support PREPARE/EXECUTE/DECLARE/FETCH statements
@@ -90,7 +88,7 @@
-- Stress test for variable LIMIT in conjunction with bounded-heap sorting
--- [SPARK-28330] ANSI SQL: Top-level <result offset clause> in <query expression>
+-- [SPARK-27767] Built-in function: generate_series
-- SELECT
-- (SELECT n
-- FROM (VALUES (1)) AS x,
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/postgreSQL/numeric.sql 2023-09-18 15:00:43.002708419 +0800
@@ -840,40 +840,33 @@
DROP TABLE width_bucket_test;
--- [SPARK-28137] Missing Data Type Formatting Functions: TO_CHAR
-- TO_CHAR()
---
--- SELECT '' AS to_char_1, to_char(val, '9G999G999G999G999G999')
--- FROM num_data;
-
--- SELECT '' AS to_char_2, to_char(val, '9G999G999G999G999G999D999G999G999G999G999')
--- FROM num_data;
-
--- SELECT '' AS to_char_3, to_char(val, '9999999999999999.999999999999999PR')
--- FROM num_data;
-
--- SELECT '' AS to_char_4, to_char(val, '9999999999999999.999999999999999S')
--- FROM num_data;
-
--- SELECT '' AS to_char_5, to_char(val, 'MI9999999999999999.999999999999999') FROM num_data;
--- SELECT '' AS to_char_6, to_char(val, 'FMS9999999999999999.999999999999999') FROM num_data;
--- SELECT '' AS to_char_7, to_char(val, 'FM9999999999999999.999999999999999THPR') FROM num_data;
--- SELECT '' AS to_char_8, to_char(val, 'SG9999999999999999.999999999999999th') FROM num_data;
--- SELECT '' AS to_char_9, to_char(val, '0999999999999999.999999999999999') FROM num_data;
--- SELECT '' AS to_char_10, to_char(val, 'S0999999999999999.999999999999999') FROM num_data;
--- SELECT '' AS to_char_11, to_char(val, 'FM0999999999999999.999999999999999') FROM num_data;
--- SELECT '' AS to_char_12, to_char(val, 'FM9999999999999999.099999999999999') FROM num_data;
--- SELECT '' AS to_char_13, to_char(val, 'FM9999999999990999.990999999999999') FROM num_data;
--- SELECT '' AS to_char_14, to_char(val, 'FM0999999999999999.999909999999999') FROM num_data;
--- SELECT '' AS to_char_15, to_char(val, 'FM9999999990999999.099999999999999') FROM num_data;
--- SELECT '' AS to_char_16, to_char(val, 'L9999999999999999.099999999999999') FROM num_data;
--- SELECT '' AS to_char_17, to_char(val, 'FM9999999999999999.99999999999999') FROM num_data;
--- SELECT '' AS to_char_18, to_char(val, 'S 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9') FROM num_data;
--- SELECT '' AS to_char_19, to_char(val, 'FMS 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9') FROM num_data;
--- SELECT '' AS to_char_20, to_char(val, E'99999 "text" 9999 "9999" 999 "\\"text between quote marks\\"" 9999') FROM num_data;
--- SELECT '' AS to_char_21, to_char(val, '999999SG9999999999') FROM num_data;
--- SELECT '' AS to_char_22, to_char(val, 'FM9999999999999999.999999999999999') FROM num_data;
--- SELECT '' AS to_char_23, to_char(val, '9.999EEEE') FROM num_data;
+-- some queries are commented out as the format string is not supported by Spark
+SELECT '' AS to_char_3, to_char(val, '9999999999999999.999999999999999PR'), val
+FROM num_data;
+
+SELECT '' AS to_char_4, to_char(val, '9999999999999999.999999999999999S'), val
+FROM num_data;
+
+SELECT '' AS to_char_5, to_char(val, 'MI9999999999999999.999999999999999'), val FROM num_data;
+-- SELECT '' AS to_char_6, to_char(val, 'FMS9999999999999999.999999999999999'), val FROM num_data;
+-- SELECT '' AS to_char_7, to_char(val, 'FM9999999999999999.999999999999999THPR'), val FROM num_data;
+-- SELECT '' AS to_char_8, to_char(val, 'SG9999999999999999.999999999999999th'), val FROM num_data;
+SELECT '' AS to_char_9, to_char(val, '0999999999999999.999999999999999'), val FROM num_data;
+SELECT '' AS to_char_10, to_char(val, 'S0999999999999999.999999999999999'), val FROM num_data;
+-- SELECT '' AS to_char_11, to_char(val, 'FM0999999999999999.999999999999999'), val FROM num_data;
+-- SELECT '' AS to_char_12, to_char(val, 'FM9999999999999999.099999999999999'), val FROM num_data;
+-- SELECT '' AS to_char_13, to_char(val, 'FM9999999999990999.990999999999999'), val FROM num_data;
+-- SELECT '' AS to_char_14, to_char(val, 'FM0999999999999999.999909999999999'), val FROM num_data;
+-- SELECT '' AS to_char_15, to_char(val, 'FM9999999990999999.099999999999999'), val FROM num_data;
+-- SELECT '' AS to_char_16, to_char(val, 'L9999999999999999.099999999999999'), val FROM num_data;
+-- SELECT '' AS to_char_17, to_char(val, 'FM9999999999999999.99999999999999'), val FROM num_data;
+-- SELECT '' AS to_char_18, to_char(val, 'S 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9'), val FROM num_data;
+-- SELECT '' AS to_char_19, to_char(val, 'FMS 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9'), val FROM num_data;
+-- SELECT '' AS to_char_20, to_char(val, E'99999 "text" 9999 "9999" 999 "\\"text between quote marks\\"" 9999'), val FROM num_data;
+-- SELECT '' AS to_char_21, to_char(val, '999999SG9999999999'), val FROM num_data;
+-- SELECT '' AS to_char_22, to_char(val, 'FM9999999999999999.999999999999999'), val FROM num_data;
+-- SELECT '' AS to_char_23, to_char(val, '9.999EEEE'), val FROM num_data;
-- SELECT '' AS to_char_24, to_char('100'::numeric, 'FM999.9');
-- SELECT '' AS to_char_25, to_char('100'::numeric, 'FM999.');
@@ -893,7 +886,7 @@
-- [SPARK-28137] Missing Data Type Formatting Functions: TO_NUMBER
-- TO_NUMBER()
---
+-- some queries are commented out as the format string is not supported by Spark
-- SET lc_numeric = 'C';
SELECT '' AS to_number_1, to_number('-34,338,492', '99G999G999');
SELECT '' AS to_number_2, to_number('-34,338,492.654,878', '99G999G999D999G999');
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql 2023-09-15 11:42:05.966317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/regexp-functions.sql 2023-09-18 15:00:43.002708419 +0800
@@ -53,3 +53,30 @@
SELECT regexp('1a 2b 14m', '[a-z]+b');
SELECT rlike('1a 2b 14m', '\\d+b');
SELECT rlike('1a 2b 14m', '[a-z]+b');
+
+-- regexp_count
+SELECT regexp_count('1a 2b 14m', '\\d+');
+SELECT regexp_count('1a 2b 14m', 'mmm');
+SELECT regexp_count('the fox', 'FOX');
+SELECT regexp_count('the fox', '(?i)FOX');
+SELECT regexp_count('passwd7 plain A1234 a1234', '(?=[^ ]*[a-z])(?=[^ ]*[0-9])[^ ]+');
+SELECT regexp_count(null, 'abc');
+SELECT regexp_count('abc', null);
+
+-- regexp_substr
+SELECT regexp_substr('1a 2b 14m', '\\d+');
+SELECT regexp_substr('1a 2b 14m', '\\d+ ');
+SELECT regexp_substr('1a 2b 14m', '\\d+(a|b|m)');
+SELECT regexp_substr('1a 2b 14m', '\\d{2}(a|b|m)');
+SELECT regexp_substr('1a 2b 14m', '');
+SELECT regexp_substr('Spark', null);
+SELECT regexp_substr(null, '.*');
+
+-- regexp_instr
+SELECT regexp_instr('abc', 'b');
+SELECT regexp_instr('abc', 'x');
+SELECT regexp_instr('ABC', '(?-i)b');
+SELECT regexp_instr('1a 2b 14m', '\\d{2}(a|b|m)');
+SELECT regexp_instr('abc', null);
+SELECT regexp_instr(null, 'b');
+SELECT regexp_instr('abc', col0, 1) FROM VALUES(') ?') AS t(col0);
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/show-create-table.sql sql/core/src/test/resources/sql-tests/inputs/show-create-table.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/show-create-table.sql 2023-09-15 11:42:05.966317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/show-create-table.sql 2023-09-18 15:00:43.002708419 +0800
@@ -7,7 +7,7 @@
-- options
CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet
-OPTIONS ('a' 1);
+OPTIONS ('a' 1, 'password' = 'password');
SHOW CREATE TABLE tbl;
DROP TABLE tbl;
@@ -45,6 +45,14 @@
DROP TABLE tbl;
+-- default column values
+CREATE TABLE tbl (a INT DEFAULT 42, b STRING DEFAULT 'abc, def', c INT DEFAULT 42) USING parquet
+COMMENT 'This is a comment';
+
+SHOW CREATE TABLE tbl;
+DROP TABLE tbl;
+
+
-- comment
CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet
COMMENT 'This is a comment';
@@ -55,7 +63,7 @@
-- tblproperties
CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet
-TBLPROPERTIES ('a' = '1');
+TBLPROPERTIES ('a' = '1', 'password' = 'password');
SHOW CREATE TABLE tbl;
DROP TABLE tbl;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/show-tblproperties.sql sql/core/src/test/resources/sql-tests/inputs/show-tblproperties.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/show-tblproperties.sql 2023-09-15 11:42:05.966317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/show-tblproperties.sql 2023-09-18 15:00:43.002708419 +0800
@@ -1,6 +1,6 @@
-- create a table with properties
CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet
-TBLPROPERTIES('p1'='v1', 'p2'='v2');
+TBLPROPERTIES('p1'='v1', 'p2'='v2', password = 'password');
SHOW TBLPROPERTIES tbl;
SHOW TBLPROPERTIES tbl("p1");
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql sql/core/src/test/resources/sql-tests/inputs/string-functions.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/string-functions.sql 2023-09-18 15:00:43.002708419 +0800
@@ -26,6 +26,10 @@
-- split function
SELECT split('aa1cc2ee3', '[1-9]+');
SELECT split('aa1cc2ee3', '[1-9]+', 2);
+SELECT split('hello', '');
+SELECT split('', '');
+SELECT split('abc', null);
+SELECT split(null, 'b');
-- split_part function
SELECT split_part('11.12.13', '.', 2);
@@ -176,14 +180,43 @@
select to_number('<00,454.8>', '00,000.9PR');
-- to_binary
-select to_binary('abc');
-select to_binary('abc', 'utf-8');
-select to_binary('abc', 'base64');
-select to_binary('abc', 'hex');
+-- base64 valid
+select to_binary('', 'base64');
+select to_binary(' ', 'base64');
+select to_binary(' ab cd ', 'base64');
+select to_binary(' ab c=', 'base64');
+select to_binary(' ab cdef= = ', 'base64');
+select to_binary(
+ concat(' b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0',
+ 'd2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl'), 'base64');
+-- base64 invalid
+select to_binary('a', 'base64');
+select to_binary('a?', 'base64');
+select to_binary('abcde', 'base64');
+select to_binary('abcd=', 'base64');
+select to_binary('a===', 'base64');
+select to_binary('ab==f', 'base64');
+-- utf-8
+select to_binary(
+ '∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β)', 'utf-8');
+select to_binary('大千世界', 'utf8');
+select to_binary('', 'utf-8');
+select to_binary(' ', 'utf8');
+-- hex valid
+select to_binary('737472696E67');
+select to_binary('737472696E67', 'hex');
+select to_binary('');
+select to_binary('1', 'hex');
+select to_binary('FF');
select to_binary('123', 'hex');
select to_binary('12345', 'hex');
+-- hex invalid
+select to_binary('GG');
+select to_binary('01 AF', 'hex');
-- 'format' parameter can be any foldable string value, not just literal.
select to_binary('abc', concat('utf', '-8'));
+select to_binary(' ab cdef= = ', substr('base64whynot', 0, 6));
+select to_binary(' ab cdef= = ', replace('HEX0', '0'));
-- 'format' parameter is case insensitive.
select to_binary('abc', 'Hex');
-- null inputs lead to null result.
@@ -191,10 +224,10 @@
select to_binary(null, 'utf-8');
select to_binary(null, null);
select to_binary(null, cast(null as string));
--- 'format' parameter must be string type or void type.
-select to_binary(null, cast(null as int));
-select to_binary('abc', 1);
-- invalid format
+select to_binary('abc', 1);
select to_binary('abc', 'invalidFormat');
--- invalid string input
-select to_binary('a!', 'base64');
+CREATE TEMPORARY VIEW fmtTable(fmtField) AS SELECT * FROM VALUES ('invalidFormat');
+SELECT to_binary('abc', fmtField) FROM fmtTable;
+-- Clean up
+DROP VIEW IF EXISTS fmtTable;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-joins-and-set-ops.sql sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-joins-and-set-ops.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-joins-and-set-ops.sql 2023-09-15 11:42:05.966317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-joins-and-set-ops.sql 2023-09-18 15:00:43.002708419 +0800
@@ -238,3 +238,23 @@
WHERE dept_id >= 30
AND dept_id <= 50);
+-- Correlated predicates under UNION - unsupported
+SELECT *
+FROM emp
+WHERE EXISTS (SELECT *
+ FROM dept
+ WHERE dept_id = emp.dept_id and state = "CA"
+ UNION
+ SELECT *
+ FROM dept
+ WHERE dept_id = emp.dept_id and state = "TX");
+
+SELECT *
+FROM emp
+WHERE NOT EXISTS (SELECT *
+ FROM dept
+ WHERE dept_id = emp.dept_id and state = "CA"
+ UNION
+ SELECT *
+ FROM dept
+ WHERE dept_id = emp.dept_id and state = "TX");
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql 2023-09-15 11:42:05.966317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-orderby-limit.sql 2023-09-18 15:00:43.002708419 +0800
@@ -121,3 +121,81 @@
WHERE dept.dept_id > 100
GROUP BY state
LIMIT 1);
+
+-- limit and offset in the exists subquery block.
+-- TC.03.01
+SELECT *
+FROM emp
+WHERE EXISTS (SELECT dept.dept_name
+ FROM dept
+ WHERE dept.dept_id > 10
+ LIMIT 1
+ OFFSET 2);
+
+-- limit and offset in the exists subquery block with aggregate.
+-- TC.03.02
+SELECT *
+FROM emp
+WHERE EXISTS (SELECT max(dept.dept_id)
+ FROM dept
+ GROUP BY state
+ LIMIT 1
+ OFFSET 2);
+
+-- limit and offset in the not exists subquery block.
+-- TC.03.03
+SELECT *
+FROM emp
+WHERE NOT EXISTS (SELECT dept.dept_name
+ FROM dept
+ WHERE dept.dept_id > 100
+ LIMIT 1
+ OFFSET 2);
+
+-- limit and offset in the not exists subquery block with aggregates.
+-- TC.03.04
+SELECT *
+FROM emp
+WHERE NOT EXISTS (SELECT max(dept.dept_id)
+ FROM dept
+ WHERE dept.dept_id > 100
+ GROUP BY state
+ LIMIT 1
+ OFFSET 2);
+
+-- offset in the exists subquery block.
+-- TC.04.01
+SELECT *
+FROM emp
+WHERE EXISTS (SELECT dept.dept_name
+ FROM dept
+ WHERE dept.dept_id > 10
+ OFFSET 2);
+
+-- offset in the exists subquery block with aggregate.
+-- TC.04.02
+SELECT *
+FROM emp
+WHERE EXISTS (SELECT max(dept.dept_id)
+ FROM dept
+ GROUP BY state
+ OFFSET 2);
+
+-- limit in the not exists subquery block.
+-- TC.04.03
+SELECT *
+FROM emp
+WHERE NOT EXISTS (SELECT dept.dept_name
+ FROM dept
+ WHERE dept.dept_id > 100
+ OFFSET 2);
+
+-- limit in the not exists subquery block with aggregates.
+-- TC.04.04
+SELECT *
+FROM emp
+WHERE NOT EXISTS (SELECT max(dept.dept_id)
+ FROM dept
+ WHERE dept.dept_id > 100
+ GROUP BY state
+ OFFSET 2);
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-outside-filter.sql sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-outside-filter.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-outside-filter.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/subquery/exists-subquery/exists-outside-filter.sql 2023-09-18 15:00:43.002708419 +0800
@@ -0,0 +1,156 @@
+-- Tests EXISTS subquery support where the subquery is used outside the WHERE clause.
+
+
+CREATE TEMPORARY VIEW EMP AS SELECT * FROM VALUES
+ (100, "emp 1", date "2005-01-01", 100.00D, 10),
+ (100, "emp 1", date "2005-01-01", 100.00D, 10),
+ (200, "emp 2", date "2003-01-01", 200.00D, 10),
+ (300, "emp 3", date "2002-01-01", 300.00D, 20),
+ (400, "emp 4", date "2005-01-01", 400.00D, 30),
+ (500, "emp 5", date "2001-01-01", 400.00D, NULL),
+ (600, "emp 6 - no dept", date "2001-01-01", 400.00D, 100),
+ (700, "emp 7", date "2010-01-01", 400.00D, 100),
+ (800, "emp 8", date "2016-01-01", 150.00D, 70)
+AS EMP(id, emp_name, hiredate, salary, dept_id);
+
+CREATE TEMPORARY VIEW DEPT AS SELECT * FROM VALUES
+ (10, "dept 1", "CA"),
+ (20, "dept 2", "NY"),
+ (30, "dept 3", "TX"),
+ (40, "dept 4 - unassigned", "OR"),
+ (50, "dept 5 - unassigned", "NJ"),
+ (70, "dept 7", "FL")
+AS DEPT(dept_id, dept_name, state);
+
+CREATE TEMPORARY VIEW BONUS AS SELECT * FROM VALUES
+ ("emp 1", 10.00D),
+ ("emp 1", 20.00D),
+ ("emp 2", 300.00D),
+ ("emp 2", 100.00D),
+ ("emp 3", 300.00D),
+ ("emp 4", 100.00D),
+ ("emp 5", 1000.00D),
+ ("emp 6 - no dept", 500.00D)
+AS BONUS(emp_name, bonus_amt);
+
+-- uncorrelated select exist
+-- TC.01.01
+SELECT
+ emp_name,
+ EXISTS (SELECT 1
+ FROM dept
+ WHERE dept.dept_id > 10
+ AND dept.dept_id < 30)
+FROM emp;
+
+-- correlated select exist
+-- TC.01.02
+SELECT
+ emp_name,
+ EXISTS (SELECT 1
+ FROM dept
+ WHERE emp.dept_id = dept.dept_id)
+FROM emp;
+
+-- uncorrelated exist in aggregate filter
+-- TC.01.03
+SELECT
+ sum(salary),
+ sum(salary) FILTER (WHERE EXISTS (SELECT 1
+ FROM dept
+ WHERE dept.dept_id > 10
+ AND dept.dept_id < 30))
+FROM emp;
+
+-- correlated exist in aggregate filter
+-- TC.01.04
+SELECT
+ sum(salary),
+ sum(salary) FILTER (WHERE EXISTS (SELECT 1
+ FROM dept
+ WHERE emp.dept_id = dept.dept_id))
+FROM emp;
+
+-- Multiple correlated exist in aggregate filter
+-- TC.01.05
+SELECT
+ sum(salary),
+ sum(salary) FILTER (WHERE EXISTS (SELECT 1
+ FROM dept
+ WHERE emp.dept_id = dept.dept_id)
+ OR EXISTS (SELECT 1
+ FROM bonus
+ WHERE emp.emp_name = bonus.emp_name))
+FROM emp;
+
+-- correlated exist in DISTINCT aggregate filter
+-- TC.01.06
+SELECT
+ sum(DISTINCT salary),
+ count(DISTINCT hiredate) FILTER (WHERE EXISTS (SELECT 1
+ FROM dept
+ WHERE emp.dept_id = dept.dept_id))
+FROM emp;
+
+-- correlated exist in group by of an aggregate
+-- TC.01.07
+SELECT
+ count(hiredate),
+ sum(salary)
+FROM emp
+GROUP BY EXISTS (SELECT 1
+ FROM dept
+ WHERE emp.dept_id = dept.dept_id);
+
+-- correlated exist in group by of a distinct aggregate
+-- TC.01.08
+SELECT
+ count(DISTINCT hiredate),
+ sum(DISTINCT salary)
+FROM emp
+GROUP BY EXISTS (SELECT 1
+ FROM dept
+ WHERE emp.dept_id = dept.dept_id);
+
+-- uncorrelated exist in aggregate function
+-- TC.01.09
+SELECT
+ count(CASE WHEN EXISTS (SELECT 1
+ FROM dept
+ WHERE dept.dept_id > 10
+ AND dept.dept_id < 30) THEN 1 END),
+ sum(CASE WHEN EXISTS (SELECT 1
+ FROM dept
+ WHERE dept.dept_id > 10
+ AND dept.dept_id < 30) THEN salary END)
+FROM emp;
+
+-- correlated exist in aggregate function
+-- TC.01.10
+SELECT
+ count(CASE WHEN EXISTS (SELECT 1
+ FROM dept
+ WHERE emp.dept_id = dept.dept_id) THEN 1 END),
+ sum(CASE WHEN EXISTS (SELECT 1
+ FROM dept
+ WHERE emp.dept_id = dept.dept_id) THEN salary END)
+FROM emp;
+
+-- uncorrelated exist in window
+-- TC.01.11
+SELECT
+ emp_name,
+ sum(salary) OVER (PARTITION BY EXISTS (SELECT 1
+ FROM dept
+ WHERE dept.dept_id > 10
+ AND dept.dept_id < 30))
+FROM emp;
+
+-- correlated exist in window
+-- TC.01.12
+SELECT
+ emp_name,
+ sum(salary) OVER (PARTITION BY EXISTS (SELECT 1
+ FROM dept
+ WHERE emp.dept_id = dept.dept_id))
+FROM emp;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql 2023-09-15 11:42:05.966317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql 2023-09-18 15:00:43.002708419 +0800
@@ -100,4 +100,113 @@
LIMIT 1)
GROUP BY t1b
ORDER BY t1b NULLS last
-LIMIT 1;
\ No newline at end of file
+LIMIT 1;
+
+-- LIMIT and OFFSET in parent side
+-- TC 02.01
+SELECT *
+FROM t1
+WHERE t1a IN (SELECT t2a
+ FROM t2
+ WHERE t1d = t2d)
+LIMIT 2
+OFFSET 2;
+
+-- TC 02.02
+SELECT *
+FROM t1
+WHERE t1c IN (SELECT t2c
+ FROM t2
+ WHERE t2b >= 8
+ LIMIT 2
+ OFFSET 2)
+LIMIT 4
+OFFSET 2;
+
+-- TC 02.03
+SELECT Count(DISTINCT( t1a )),
+ t1b
+FROM t1
+WHERE t1d IN (SELECT t2d
+ FROM t2
+ ORDER BY t2c, t2d
+ LIMIT 2)
+GROUP BY t1b
+ORDER BY t1b DESC NULLS FIRST
+LIMIT 1
+OFFSET 1;
+
+-- LIMIT with NOT IN
+-- TC 02.04
+SELECT *
+FROM t1
+WHERE t1b NOT IN (SELECT t2b
+ FROM t2
+ WHERE t2b > 6
+ LIMIT 2
+ OFFSET 2);
+
+-- TC 02.05
+SELECT Count(DISTINCT( t1a )),
+ t1b
+FROM t1
+WHERE t1d NOT IN (SELECT t2d
+ FROM t2
+ ORDER BY t2b DESC nulls first, t2d
+ LIMIT 1
+ OFFSET 1)
+GROUP BY t1b
+ORDER BY t1b NULLS last
+LIMIT 1
+OFFSET 1;
+
+-- OFFSET in parent side
+-- TC 03.01
+SELECT *
+FROM t1
+WHERE t1a IN (SELECT t2a
+ FROM t2
+ WHERE t1d = t2d)
+OFFSET 2;
+
+-- TC 03.02
+SELECT *
+FROM t1
+WHERE t1c IN (SELECT t2c
+ FROM t2
+ WHERE t2b >= 8
+ OFFSET 2)
+OFFSET 4;
+
+-- TC 03.03
+SELECT Count(DISTINCT( t1a )),
+ t1b
+FROM t1
+WHERE t1d IN (SELECT t2d
+ FROM t2
+ ORDER BY t2c, t2d
+ OFFSET 2)
+GROUP BY t1b
+ORDER BY t1b DESC NULLS FIRST
+OFFSET 1;
+
+-- OFFSET with NOT IN
+-- TC 03.04
+SELECT *
+FROM t1
+WHERE t1b NOT IN (SELECT t2b
+ FROM t2
+ WHERE t2b > 6
+ OFFSET 2);
+
+-- TC 03.05
+SELECT Count(DISTINCT( t1a )),
+ t1b
+FROM t1
+WHERE t1d NOT IN (SELECT t2d
+ FROM t2
+ ORDER BY t2b DESC nulls first, t2d
+ OFFSET 1)
+GROUP BY t1b
+ORDER BY t1b NULLS last
+OFFSET 1;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-count-bug.sql sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-count-bug.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-count-bug.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-count-bug.sql 2023-09-18 15:00:43.002708419 +0800
@@ -0,0 +1,61 @@
+--CONFIG_DIM1 spark.sql.optimizer.decorrelateInnerQuery.enabled=true
+--CONFIG_DIM1 spark.sql.optimizer.decorrelateInnerQuery.enabled=false
+
+create temp view l (a, b)
+as values
+ (1, 2.0),
+ (1, 2.0),
+ (2, 1.0),
+ (2, 1.0),
+ (3, 3.0),
+ (null, null),
+ (null, 5.0),
+ (6, null);
+
+create temp view r (c, d)
+as values
+ (2, 3.0),
+ (2, 3.0),
+ (3, 2.0),
+ (4, 1.0),
+ (null, null),
+ (null, 5.0),
+ (6, null);
+
+-- count bug, empty groups should evaluate to 0
+select *, (select count(*) from r where l.a = r.c) from l;
+
+-- no count bug, empty groups should evaluate to null
+select *, (select count(*) from r where l.a = r.c group by c) from l;
+select *, (select count(*) from r where l.a = r.c group by 'constant') from l;
+
+-- count bug, empty groups should evaluate to false - but this case is wrong due to bug SPARK-43156
+select *, (
+ select (count(*)) is null
+ from r
+ where l.a = r.c)
+from l;
+
+-- no count bug, empty groups should evaluate to null
+select *, (
+ select (count(*)) is null
+ from r
+ where l.a = r.c
+ group by r.c)
+from l;
+
+-- Empty groups should evaluate to 0, and groups filtered by HAVING should evaluate to NULL
+select *, (select count(*) from r where l.a = r.c having count(*) <= 1) from l;
+
+-- Empty groups are filtered by HAVING and should evaluate to null
+select *, (select count(*) from r where l.a = r.c having count(*) >= 2) from l;
+
+
+set spark.sql.optimizer.decorrelateSubqueryLegacyIncorrectCountHandling.enabled = true;
+
+-- With legacy behavior flag set, both cases evaluate to 0
+select *, (select count(*) from r where l.a = r.c) from l;
+select *, (select count(*) from r where l.a = r.c group by c) from l;
+select *, (select count(*) from r where l.a = r.c group by 'constant') from l;
+
+reset spark.sql.optimizer.decorrelateSubqueryLegacyIncorrectCountHandling.enabled;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-predicate.sql sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-predicate.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-predicate.sql 2023-09-15 11:42:05.966317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-predicate.sql 2023-09-18 15:00:43.002708419 +0800
@@ -279,3 +279,132 @@
FROM t2
WHERE t2c = t1c
GROUP BY t2c);
+
+-- Set operations in correlation path
+
+CREATE OR REPLACE TEMP VIEW t0(t0a, t0b) AS VALUES (1, 1), (2, 0);
+CREATE OR REPLACE TEMP VIEW t1(t1a, t1b, t1c) AS VALUES (1, 1, 3);
+CREATE OR REPLACE TEMP VIEW t2(t2a, t2b, t2c) AS VALUES (1, 1, 5), (2, 2, 7);
+
+SELECT * FROM t0 WHERE t0a <
+(SELECT sum(c) FROM
+ (SELECT t1c as c
+ FROM t1
+ WHERE t1a = t0a
+ UNION ALL
+ SELECT t2c as c
+ FROM t2
+ WHERE t2b = t0b)
+);
+
+SELECT * FROM t0 WHERE t0a <
+(SELECT sum(c) FROM
+ (SELECT t1c as c
+ FROM t1
+ WHERE t1a = t0a
+ UNION ALL
+ SELECT t2c as c
+ FROM t2
+ WHERE t2a = t0a)
+);
+
+SELECT * FROM t0 WHERE t0a <
+(SELECT sum(c) FROM
+ (SELECT t1c as c
+ FROM t1
+ WHERE t1a > t0a
+ UNION ALL
+ SELECT t2c as c
+ FROM t2
+ WHERE t2b <= t0b)
+);
+
+SELECT * FROM t0 WHERE t0a <
+(SELECT sum(t1c) FROM
+ (SELECT t1c
+ FROM t1
+ WHERE t1a = t0a
+ UNION ALL
+ SELECT t2c
+ FROM t2
+ WHERE t2b = t0b)
+);
+
+SELECT * FROM t0 WHERE t0a <
+(SELECT sum(t1c) FROM
+ (SELECT t1c
+ FROM t1
+ WHERE t1a = t0a
+ UNION DISTINCT
+ SELECT t2c
+ FROM t2
+ WHERE t2b = t0b)
+);
+
+-- Tests for column aliasing
+SELECT * FROM t0 WHERE t0a <
+(SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM
+ (SELECT t1c as t1a, t1a as t1b, t0a as t1c
+ FROM t1
+ WHERE t1a = t0a
+ UNION ALL
+ SELECT t0a as t2b, t2c as t1a, t0b as t2c
+ FROM t2
+ WHERE t2b = t0b)
+);
+
+-- Test handling of COUNT bug
+SELECT * FROM t0 WHERE t0a <
+(SELECT count(t1c) FROM
+ (SELECT t1c
+ FROM t1
+ WHERE t1a = t0a
+ UNION DISTINCT
+ SELECT t2c
+ FROM t2
+ WHERE t2b = t0b)
+);
+
+-- Correlated references in project
+SELECT * FROM t0 WHERE t0a <
+(SELECT sum(d) FROM
+ (SELECT t1a - t0a as d
+ FROM t1
+ UNION ALL
+ SELECT t2a - t0a as d
+ FROM t2)
+);
+
+-- Correlated references in aggregate - unsupported
+SELECT * FROM t0 WHERE t0a <
+(SELECT sum(d) FROM
+ (SELECT sum(t0a) as d
+ FROM t1
+ UNION ALL
+ SELECT sum(t2a) + t0a as d
+ FROM t2)
+);
+
+-- In HAVING clause
+SELECT t0a, t0b FROM t0
+GROUP BY t0a, t0b
+HAVING t0a <
+(SELECT sum(c) FROM
+ (SELECT t1c as c
+ FROM t1
+ WHERE t1a > t0a
+ UNION ALL
+ SELECT t2c as c
+ FROM t2
+ WHERE t2b <= t0b)
+);
+
+-- SPARK-43760: the result of the subquery can be NULL.
+select *
+from range(1, 3) t1
+where (select sum(c) from (
+ select t2.id * t2.id c
+ from range (1, 2) t2 where t1.id = t2.id
+ group by t2.id
+ )
+) is not null;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql 2023-09-15 11:42:05.966317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql 2023-09-18 15:00:43.002708419 +0800
@@ -187,3 +187,177 @@
UNION SELECT * FROM v3 WHERE c2 > 0
) WHERE c1 = v1.c1
) FROM v1;
+
+-- Multi-value subquery error
+SELECT (SELECT a FROM (SELECT 1 AS a UNION ALL SELECT 2 AS a) t) AS b;
+
+-- SPARK-36114: Support correlated non-equality predicates
+CREATE OR REPLACE TEMP VIEW t1(c1, c2) AS (VALUES (0, 1), (1, 2));
+CREATE OR REPLACE TEMP VIEW t2(c1, c2) AS (VALUES (0, 2), (0, 3));
+
+-- Neumann example Q2
+CREATE OR REPLACE TEMP VIEW students(id, name, major, year) AS (VALUES
+ (0, 'A', 'CS', 2022),
+ (1, 'B', 'CS', 2022),
+ (2, 'C', 'Math', 2022));
+CREATE OR REPLACE TEMP VIEW exams(sid, course, curriculum, grade, date) AS (VALUES
+ (0, 'C1', 'CS', 4, 2020),
+ (0, 'C2', 'CS', 3, 2021),
+ (1, 'C1', 'CS', 2, 2020),
+ (1, 'C2', 'CS', 1, 2021));
+
+SELECT students.name, exams.course
+FROM students, exams
+WHERE students.id = exams.sid
+ AND (students.major = 'CS' OR students.major = 'Games Eng')
+ AND exams.grade >= (
+ SELECT avg(exams.grade) + 1
+ FROM exams
+ WHERE students.id = exams.sid
+ OR (exams.curriculum = students.major AND students.year > exams.date));
+
+-- Correlated non-equality predicates
+SELECT (SELECT min(c2) FROM t2 WHERE t1.c1 > t2.c1) FROM t1;
+SELECT (SELECT min(c2) FROM t2 WHERE t1.c1 >= t2.c1 AND t1.c2 < t2.c2) FROM t1;
+
+-- Correlated non-equality predicates with the COUNT bug.
+SELECT (SELECT count(*) FROM t2 WHERE t1.c1 > t2.c1) FROM t1;
+
+-- Correlated equality predicates that are not supported after SPARK-35080
+SELECT c, (
+ SELECT count(*)
+ FROM (VALUES ('ab'), ('abc'), ('bc')) t2(c)
+ WHERE t1.c = substring(t2.c, 1, 1)
+) FROM (VALUES ('a'), ('b')) t1(c);
+
+SELECT c, (
+ SELECT count(*)
+ FROM (VALUES (0, 6), (1, 5), (2, 4), (3, 3)) t1(a, b)
+ WHERE a + b = c
+) FROM (VALUES (6)) t2(c);
+
+-- SPARK-43156: scalar subquery with Literal result like `COUNT(1) is null`
+SELECT *, (SELECT count(1) is null FROM t2 WHERE t1.c1 = t2.c1) FROM t1;
+
+select (select f from (select false as f, max(c2) from t1 where t1.c1 = t1.c1)) from t2;
+
+-- Set operations in correlation path
+
+CREATE OR REPLACE TEMP VIEW t0(t0a, t0b) AS VALUES (1, 1), (2, 0);
+CREATE OR REPLACE TEMP VIEW t1(t1a, t1b, t1c) AS VALUES (1, 1, 3);
+CREATE OR REPLACE TEMP VIEW t2(t2a, t2b, t2c) AS VALUES (1, 1, 5), (2, 2, 7);
+
+SELECT t0a, (SELECT sum(c) FROM
+ (SELECT t1c as c
+ FROM t1
+ WHERE t1a = t0a
+ UNION ALL
+ SELECT t2c as c
+ FROM t2
+ WHERE t2b = t0b)
+)
+FROM t0;
+
+SELECT t0a, (SELECT sum(c) FROM
+ (SELECT t1c as c
+ FROM t1
+ WHERE t1a = t0a
+ UNION ALL
+ SELECT t2c as c
+ FROM t2
+ WHERE t2a = t0a)
+)
+FROM t0;
+
+SELECT t0a, (SELECT sum(c) FROM
+ (SELECT t1c as c
+ FROM t1
+ WHERE t1a > t0a
+ UNION ALL
+ SELECT t2c as c
+ FROM t2
+ WHERE t2b <= t0b)
+)
+FROM t0;
+
+SELECT t0a, (SELECT sum(t1c) FROM
+ (SELECT t1c
+ FROM t1
+ WHERE t1a = t0a
+ UNION ALL
+ SELECT t2c
+ FROM t2
+ WHERE t2b = t0b)
+)
+FROM t0;
+
+SELECT t0a, (SELECT sum(t1c) FROM
+ (SELECT t1c
+ FROM t1
+ WHERE t1a = t0a
+ UNION DISTINCT
+ SELECT t2c
+ FROM t2
+ WHERE t2b = t0b)
+)
+FROM t0;
+
+-- Tests for column aliasing
+SELECT t0a, (SELECT sum(t1a + 3 * t1b + 5 * t1c) FROM
+ (SELECT t1c as t1a, t1a as t1b, t0a as t1c
+ FROM t1
+ WHERE t1a = t0a
+ UNION ALL
+ SELECT t0a as t2b, t2c as t1a, t0b as t2c
+ FROM t2
+ WHERE t2b = t0b)
+)
+FROM t0;
+
+-- Test handling of COUNT bug
+SELECT t0a, (SELECT count(t1c) FROM
+ (SELECT t1c
+ FROM t1
+ WHERE t1a = t0a
+ UNION DISTINCT
+ SELECT t2c
+ FROM t2
+ WHERE t2b = t0b)
+)
+FROM t0;
+
+-- Correlated references in project
+SELECT t0a, (SELECT sum(d) FROM
+ (SELECT t1a - t0a as d
+ FROM t1
+ UNION ALL
+ SELECT t2a - t0a as d
+ FROM t2)
+)
+FROM t0;
+
+-- Correlated references in aggregate - unsupported
+SELECT t0a, (SELECT sum(d) FROM
+ (SELECT sum(t0a) as d
+ FROM t1
+ UNION ALL
+ SELECT sum(t2a) + t0a as d
+ FROM t2)
+)
+FROM t0;
+
+-- SPARK-43760: the result of the subquery can be NULL.
+select *
+from
+(
+ select t1.id c1, (
+ select sum(c)
+ from (
+ select t2.id * t2.id c
+ from range (1, 2) t2 where t1.id = t2.id
+ group by t2.id
+ )
+ ) c2
+ from range (1, 3) t1
+) t
+where t.c2 is not null;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql 2023-09-15 11:42:05.966317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql 2023-09-18 15:00:43.002708419 +0800
@@ -27,3 +27,97 @@
-- range call with a mixed-case function name
select * from RaNgE(2);
+
+-- range call with alias
+select i from range(0, 2) t(i);
+
+-- explode
+select * from explode(array(1, 2));
+select * from explode(map('a', 1, 'b', 2));
+
+-- explode with empty values
+select * from explode(array());
+select * from explode(map());
+
+-- explode with column aliases
+select * from explode(array(1, 2)) t(c1);
+select * from explode(map('a', 1, 'b', 2)) t(k, v);
+
+-- explode with erroneous input
+select * from explode(null);
+select * from explode(null) t(c1);
+select * from explode(1);
+select * from explode(1, 2);
+select * from explode(explode(array(1)));
+select * from explode(array(1, 2)) t(c1, c2);
+
+-- explode_outer
+select * from explode_outer(array(1, 2));
+select * from explode_outer(map('a', 1, 'b', 2));
+select * from explode_outer(array());
+select * from explode_outer(map());
+
+-- table-valued functions with join
+select * from range(2) join explode(array(1, 2));
+select * from range(2) join explode_outer(array());
+
+-- inline
+select * from inline(array(struct(1, 'a'), struct(2, 'b')));
+select * from inline(array(struct(1, 'a'), struct(2, 'b'))) t(x, y);
+select * from inline(array_remove(array(struct(1, 'a')), struct(1, 'a')));
+
+-- inline with erroneous input
+select * from inline(null);
+select * from inline(array(struct(1, 2), struct(2, 3))) t(a, b, c);
+
+-- inline_outer
+select * from inline_outer(array(struct(1, 'a'), struct(2, 'b')));
+select * from inline_outer(array_remove(array(struct(1, 'a')), struct(1, 'a')));
+
+-- posexplode
+select * from posexplode(array());
+select * from posexplode(array(1, 2));
+select * from posexplode(array(1, 2)) t(pos, x);
+select * from posexplode(map());
+select * from posexplode(map('a', 1, 'b', 2));
+select * from posexplode(map('a', 1, 'b', 2)) t(pos, k, v);
+
+-- posexplode with erroneous input
+select * from posexplode(1);
+select * from posexplode(1, 2);
+select * from posexplode(explode(array(1)));
+select * from posexplode(array(1, 2)) t(x);
+
+-- posexplode
+select * from posexplode_outer(array());
+select * from posexplode_outer(array(1, 2));
+select * from posexplode_outer(map());
+select * from posexplode_outer(map('a', 1, 'b', 2));
+
+-- json_tuple
+select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b');
+select * from json_tuple('{"a": 1, "b": 2}', 'a', 'c');
+select * from json_tuple('{"a": 1, "b": 2}', 'a', 'a');
+select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x, y);
+select * from json_tuple('{"a": bad, "b": string}', 'a', 'b');
+
+-- json_tuple with erroneous input
+select * from json_tuple();
+select * from json_tuple('{"a": 1}');
+select * from json_tuple('{"a": 1}', 1);
+select * from json_tuple('{"a": 1}', null);
+select * from json_tuple('{"a": 1, "b": 2}', 'a', 'b') AS t(x);
+
+-- stack
+select * from stack(1, 1, 2, 3);
+select * from stack(2, 1, 2, 3);
+select * from stack(3, 1, 2, 3) t(x);
+select * from stack(4, 1, 2, 3) t(x);
+select * from stack(2, 1, 1.1, 'a', 2, 2.2, 'b') t(a, b, c);
+select * from stack(2, 1, 1.1, null, 2, null, 'b') t(a, b, c);
+
+-- stack with erroneous input
+select * from stack();
+select * from stack(2, 1, 2, 3) t(a, b, c);
+select * from stack(2, 1, '1.1', 'a', 2, 2.2, 'b');
+select * from stack(2, explode(array(1, 2, 3)));
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/timestamp-ntz.sql sql/core/src/test/resources/sql-tests/inputs/timestamp-ntz.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/timestamp-ntz.sql 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/timestamp-ntz.sql 2023-09-18 15:00:43.006708418 +0800
@@ -24,3 +24,10 @@
select timestampdiff(HOUR, timestamp_ntz'2022-02-14 01:02:03', timestamp_ltz'2022-02-14 02:03:04');
select timestampdiff(YEAR, date'2022-02-15', timestamp_ntz'2023-02-15 10:11:12');
select timestampdiff(MILLISECOND, timestamp_ntz'2022-02-14 23:59:59.123', date'2022-02-15');
+
+select timestamp_ntz'2022-01-01 00:00:00' = date'2022-01-01';
+select timestamp_ntz'2022-01-01 00:00:00' > date'2022-01-01';
+select timestamp_ntz'2022-01-01 00:00:00' < date'2022-01-01';
+select timestamp_ntz'2022-01-01 00:00:00' = timestamp_ltz'2022-01-01 00:00:00';
+select timestamp_ntz'2022-01-01 00:00:00' > timestamp_ltz'2022-01-01 00:00:00';
+select timestamp_ntz'2022-01-01 00:00:00' < timestamp_ltz'2022-01-01 00:00:00';
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/try_aggregates.sql sql/core/src/test/resources/sql-tests/inputs/try_aggregates.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/try_aggregates.sql 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/try_aggregates.sql 2023-09-18 15:00:43.006708418 +0800
@@ -12,6 +12,15 @@
SELECT try_sum(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col);
SELECT try_sum(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col);
+-- errors in child should be shown in ANSI mode
+SELECT try_sum(col / 0) FROM VALUES (5), (10), (15) AS tab(col);
+SELECT try_sum(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col);
+SELECT try_sum(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col);
+SELECT try_sum(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col);
+
+SELECT try_sum(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col);
+SELECT try_sum(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col);
+
-- try_avg
SELECT try_avg(col) FROM VALUES (5), (10), (15) AS tab(col);
SELECT try_avg(col) FROM VALUES (5.0), (10.0), (15.0) AS tab(col);
@@ -25,3 +34,12 @@
SELECT try_avg(col) FROM VALUES (interval '2147483647 months'), (interval '1 months') AS tab(col);
SELECT try_avg(col) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col);
SELECT try_avg(col) FROM VALUES (interval '106751991 DAYS'), (interval '1 DAYS') AS tab(col);
+
+-- errors in child should be shown in ANSI mode
+SELECT try_avg(col / 0) FROM VALUES (5), (10), (15) AS tab(col);
+SELECT try_avg(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col);
+SELECT try_avg(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col);
+SELECT try_avg(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col);
+
+SELECT try_avg(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col);
+SELECT try_avg(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col);
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/try_arithmetic.sql sql/core/src/test/resources/sql-tests/inputs/try_arithmetic.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/try_arithmetic.sql 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/try_arithmetic.sql 2023-09-18 15:00:43.006708418 +0800
@@ -4,6 +4,9 @@
SELECT try_add(-2147483648, -1);
SELECT try_add(9223372036854775807L, 1);
SELECT try_add(-9223372036854775808L, -1);
+SELECT try_add(1, (2147483647 + 1));
+SELECT try_add(1L, (9223372036854775807L + 1L));
+SELECT try_add(1, 1.0 / 0.0);
-- Date + Integer
SELECT try_add(date'2021-01-01', 1);
@@ -32,6 +35,9 @@
SELECT try_divide(1, 0.5);
SELECT try_divide(1, 0);
SELECT try_divide(0, 0);
+SELECT try_divide(1, (2147483647 + 1));
+SELECT try_divide(1L, (9223372036854775807L + 1L));
+SELECT try_divide(1, 1.0 / 0.0);
-- Interval / Numeric
SELECT try_divide(interval 2 year, 2);
@@ -47,6 +53,9 @@
SELECT try_subtract(-2147483648, 1);
SELECT try_subtract(9223372036854775807L, -1);
SELECT try_subtract(-9223372036854775808L, 1);
+SELECT try_subtract(1, (2147483647 + 1));
+SELECT try_subtract(1L, (9223372036854775807L + 1L));
+SELECT try_subtract(1, 1.0 / 0.0);
-- Interval - Interval
SELECT try_subtract(interval 2 year, interval 3 year);
@@ -60,6 +69,9 @@
SELECT try_multiply(-2147483648, 2);
SELECT try_multiply(9223372036854775807L, 2);
SELECT try_multiply(-9223372036854775808L, -2);
+SELECT try_multiply(1, (2147483647 + 1));
+SELECT try_multiply(1L, (9223372036854775807L + 1L));
+SELECT try_multiply(1, 1.0 / 0.0);
-- Interval * Numeric
SELECT try_multiply(interval 2 year, 2);
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/try_datetime_functions.sql sql/core/src/test/resources/sql-tests/inputs/try_datetime_functions.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/try_datetime_functions.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/try_datetime_functions.sql 2023-09-18 15:00:43.006708418 +0800
@@ -0,0 +1,6 @@
+select try_to_timestamp(null), try_to_timestamp('2016-12-31 00:12:00'), try_to_timestamp('2016-12-31', 'yyyy-MM-dd');
+select try_to_timestamp(1);
+select try_to_timestamp('2016-12-31 abc');
+select try_to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]');
+select try_to_timestamp("02-29", "MM-dd");
+select try_to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE');
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/try-string-functions.sql sql/core/src/test/resources/sql-tests/inputs/try-string-functions.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/try-string-functions.sql 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/try-string-functions.sql 2023-09-18 15:00:43.006708418 +0800
@@ -1,12 +1,41 @@
-- try_to_binary
-select try_to_binary('abc');
-select try_to_binary('abc', 'utf-8');
-select try_to_binary('abc', 'base64');
-select try_to_binary('abc', 'hex');
+-- base64 valid
+select try_to_binary('', 'base64');
+select try_to_binary(' ', 'base64');
+select try_to_binary(' ab cd ', 'base64');
+select try_to_binary(' ab c=', 'base64');
+select try_to_binary(' ab cdef= = ', 'base64');
+select try_to_binary(
+ concat(' b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0',
+ 'd2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl'), 'base64');
+-- base64 invalid
+select try_to_binary('a', 'base64');
+select try_to_binary('a?', 'base64');
+select try_to_binary('abcde', 'base64');
+select try_to_binary('abcd=', 'base64');
+select try_to_binary('a===', 'base64');
+select try_to_binary('ab==f', 'base64');
+-- utf-8
+select try_to_binary(
+ '∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β)', 'utf-8');
+select try_to_binary('大千世界', 'utf8');
+select try_to_binary('', 'utf-8');
+select try_to_binary(' ', 'utf8');
+-- hex valid
+select try_to_binary('737472696E67');
+select try_to_binary('737472696E67', 'hex');
+select try_to_binary('');
+select try_to_binary('1', 'hex');
+select try_to_binary('FF');
select try_to_binary('123');
select try_to_binary('12345');
+-- hex invalid
+select try_to_binary('GG');
+select try_to_binary('01 AF', 'hex');
-- 'format' parameter can be any foldable string value, not just literal.
select try_to_binary('abc', concat('utf', '-8'));
+select try_to_binary(' ab cdef= = ', substr('base64whynot', 0, 6));
+select try_to_binary(' ab cdef= = ', replace('HEX0', '0'));
-- 'format' parameter is case insensitive.
select try_to_binary('abc', 'Hex');
-- null inputs lead to null result.
@@ -14,10 +43,6 @@
select try_to_binary(null, 'utf-8');
select try_to_binary(null, null);
select try_to_binary(null, cast(null as string));
--- 'format' parameter must be string type or void type.
-select try_to_binary(null, cast(null as int));
-select try_to_binary('abc', 1);
-- invalid format
-select try_to_binary('abc', 'invalidFormat');
--- invalid string input
-select try_to_binary('a!', 'base64');
+select try_to_binary('abc', 1);
+select try_to_binary('abc', 'invalidFormat');
\ No newline at end of file
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/udaf/udaf-group-analytics.sql sql/core/src/test/resources/sql-tests/inputs/udaf/udaf-group-analytics.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/udaf/udaf-group-analytics.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/udaf/udaf-group-analytics.sql 2023-09-18 15:00:43.006708418 +0800
@@ -0,0 +1,70 @@
+CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
+(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2)
+AS testData(a, b);
+
+-- CUBE on overlapping columns
+SELECT a + b, b, udaf(a - b) FROM testData GROUP BY a + b, b WITH CUBE;
+
+SELECT a, b, udaf(b) FROM testData GROUP BY a, b WITH CUBE;
+
+-- ROLLUP on overlapping columns
+SELECT a + b, b, udaf(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP;
+
+SELECT a, b, udaf(b) FROM testData GROUP BY a, b WITH ROLLUP;
+
+CREATE OR REPLACE TEMPORARY VIEW courseSales AS SELECT * FROM VALUES
+("dotNET", 2012, 10000), ("Java", 2012, 20000), ("dotNET", 2012, 5000), ("dotNET", 2013, 48000), ("Java", 2013, 30000)
+AS courseSales(course, year, earnings);
+
+-- ROLLUP
+SELECT course, year, udaf(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY course, year;
+SELECT course, year, udaf(earnings) FROM courseSales GROUP BY ROLLUP(course, year, (course, year)) ORDER BY course, year;
+SELECT course, year, udaf(earnings) FROM courseSales GROUP BY ROLLUP(course, year, (course, year), ()) ORDER BY course, year;
+
+-- CUBE
+SELECT course, year, udaf(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, year;
+SELECT course, year, udaf(earnings) FROM courseSales GROUP BY CUBE(course, year, (course, year)) ORDER BY course, year;
+SELECT course, year, udaf(earnings) FROM courseSales GROUP BY CUBE(course, year, (course, year), ()) ORDER BY course, year;
+
+-- GROUPING SETS
+SELECT course, year, udaf(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year);
+SELECT course, year, udaf(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year, ());
+SELECT course, year, udaf(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course);
+SELECT course, year, udaf(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year);
+
+-- Partial ROLLUP/CUBE/GROUPING SETS
+SELECT course, year, udaf(earnings) FROM courseSales GROUP BY course, CUBE(course, year) ORDER BY course, year;
+SELECT course, year, udaf(earnings) FROM courseSales GROUP BY CUBE(course, year), ROLLUP(course, year) ORDER BY course, year;
+SELECT course, year, udaf(earnings) FROM courseSales GROUP BY CUBE(course, year), ROLLUP(course, year), GROUPING SETS(course, year) ORDER BY course, year;
+
+-- GROUPING SETS with aggregate functions containing groupBy columns
+SELECT course, udaf(earnings) AS sum FROM courseSales
+GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum;
+SELECT course, udaf(earnings) AS sum, GROUPING_ID(course, earnings) FROM courseSales
+GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, sum;
+
+-- Aliases in SELECT could be used in ROLLUP/CUBE/GROUPING SETS
+SELECT a + b AS k1, b AS k2, udaf(a - b) FROM testData GROUP BY CUBE(k1, k2);
+SELECT a + b AS k, b, udaf(a - b) FROM testData GROUP BY ROLLUP(k, b);
+SELECT a + b, b AS k, udaf(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k);
+
+-- GROUP BY use mixed Separate columns and CUBE/ROLLUP/Gr
+SELECT a, b, udaf(1) FROM testData GROUP BY a, b, CUBE(a, b);
+SELECT a, b, udaf(1) FROM testData GROUP BY a, b, ROLLUP(a, b);
+SELECT a, b, udaf(1) FROM testData GROUP BY CUBE(a, b), ROLLUP(a, b);
+SELECT a, b, udaf(1) FROM testData GROUP BY a, CUBE(a, b), ROLLUP(b);
+SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS((a, b), (a), ());
+SELECT a, b, udaf(1) FROM testData GROUP BY a, CUBE(a, b), GROUPING SETS((a, b), (a), ());
+SELECT a, b, udaf(1) FROM testData GROUP BY a, CUBE(a, b), ROLLUP(a, b), GROUPING SETS((a, b), (a), ());
+
+-- Support nested CUBE/ROLLUP/GROUPING SETS in GROUPING SETS
+SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS(ROLLUP(a, b));
+SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b), (a), ()));
+
+SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS((a, b), GROUPING SETS(ROLLUP(a, b)));
+SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS((a, b, a, b), (a, b, a), (a, b));
+SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b, a, b), (a, b, a), (a, b)));
+
+SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS(ROLLUP(a, b), CUBE(a, b));
+SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS(GROUPING SETS((a, b), (a), ()), GROUPING SETS((a, b), (a), (b), ()));
+SELECT a, b, udaf(1) FROM testData GROUP BY a, GROUPING SETS((a, b), (a), (), (a, b), (a), (b), ());
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/udaf/udaf-group-by-ordinal.sql sql/core/src/test/resources/sql-tests/inputs/udaf/udaf-group-by-ordinal.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/udaf/udaf-group-by-ordinal.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/udaf/udaf-group-by-ordinal.sql 2023-09-18 15:00:43.006708418 +0800
@@ -0,0 +1,88 @@
+-- group by ordinal positions
+
+create temporary view data as select * from values
+ (1, 1),
+ (1, 2),
+ (2, 1),
+ (2, 2),
+ (3, 1),
+ (3, 2)
+ as data(a, b);
+
+-- basic case
+select a, udaf(b) from data group by 1;
+
+-- constant case
+select 1, 2, udaf(b) from data group by 1, 2;
+
+-- duplicate group by column
+select a, 1, udaf(b) from data group by a, 1;
+select a, 1, udaf(b) from data group by 1, 2;
+
+-- group by a non-aggregate expression's ordinal
+select a, b + 2, udaf(2) from data group by a, 2;
+
+-- with alias
+select a as aa, b + 2 as bb, udaf(2) from data group by 1, 2;
+
+-- foldable non-literal: this should be the same as no grouping.
+select udaf(b) from data group by 1 + 0;
+
+-- negative case: position is an aggregate expression
+select a, b, udaf(b) from data group by 3;
+select a, b, udaf(b) + 2 from data group by 3;
+
+-- negative case: nondeterministic expression
+select a, rand(0), udaf(b)
+from
+(select /*+ REPARTITION(1) */ a, b from data) group by a, 2;
+
+-- group by ordinal followed by order by
+select a, udaf(a) from (select 1 as a) tmp group by 1 order by 1;
+
+-- group by ordinal followed by having
+select udaf(a), a from (select 1 as a) tmp group by 2 having a > 0;
+
+-- mixed cases: group-by ordinals and aliases
+select a, a AS k, udaf(b) from data group by k, 1;
+
+-- can use ordinal in CUBE
+select a, b, udaf(1) from data group by cube(1, 2);
+
+-- mixed cases: can use ordinal in CUBE
+select a, b, udaf(1) from data group by cube(1, b);
+
+-- can use ordinal with cube
+select a, b, udaf(1) from data group by 1, 2 with cube;
+
+-- can use ordinal in ROLLUP
+select a, b, udaf(1) from data group by rollup(1, 2);
+
+-- mixed cases: can use ordinal in ROLLUP
+select a, b, udaf(1) from data group by rollup(1, b);
+
+-- can use ordinal with rollup
+select a, b, udaf(1) from data group by 1, 2 with rollup;
+
+-- can use ordinal in GROUPING SETS
+select a, b, udaf(1) from data group by grouping sets((1), (2), (1, 2));
+
+-- mixed cases: can use ordinal in GROUPING SETS
+select a, b, udaf(1) from data group by grouping sets((1), (b), (a, 2));
+
+select a, b, udaf(1) from data group by a, 2 grouping sets((1), (b), (a, 2));
+
+-- range error
+select a, b, udaf(1) from data group by a, -1;
+
+select a, b, udaf(1) from data group by a, 3;
+
+select a, b, udaf(1) from data group by cube(-1, 2);
+
+select a, b, udaf(1) from data group by cube(1, 3);
+
+-- turn off group by ordinal
+set spark.sql.groupByOrdinal=false;
+
+-- can now group by negative literal
+select udaf(b) from data group by -1;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/udaf/udaf-group-by.sql sql/core/src/test/resources/sql-tests/inputs/udaf/udaf-group-by.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/udaf/udaf-group-by.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/udaf/udaf-group-by.sql 2023-09-18 15:00:43.006708418 +0800
@@ -0,0 +1,110 @@
+-- Test aggregate operator with codegen on and off.
+--CONFIG_DIM1 spark.sql.codegen.wholeStage=true
+--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY
+--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN
+
+-- Test data.
+CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
+(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null)
+AS testData(a, b);
+
+-- Aggregate with empty GroupBy expressions.
+SELECT a, udaf(b) FROM testData;
+SELECT udaf(a), udaf(b) FROM testData;
+
+-- Aggregate with non-empty GroupBy expressions.
+SELECT a, udaf(b) FROM testData GROUP BY a;
+SELECT a, udaf(b) FROM testData GROUP BY b;
+SELECT udaf(a), udaf(b) FROM testData GROUP BY a;
+
+-- Aggregate grouped by literals.
+SELECT 'foo', udaf(a) FROM testData GROUP BY 1;
+
+-- Aggregate grouped by literals (hash aggregate).
+SELECT 'foo', udaf(a) FROM testData WHERE a = 0 GROUP BY 1;
+
+-- Aggregate grouped by literals (sort aggregate).
+SELECT 'foo', udaf(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1;
+
+-- Aggregate with complex GroupBy expressions.
+SELECT a + b, udaf(b) FROM testData GROUP BY a + b;
+SELECT a + 2, udaf(b) FROM testData GROUP BY a + 1;
+SELECT a + 1 + 1, udaf(b) FROM testData GROUP BY a + 1;
+
+-- Aggregate with nulls.
+SELECT SKEWNESS(a), KURTOSIS(a), udaf(a), udaf(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), udaf(a)
+FROM testData;
+
+-- Aggregate with foldable input and multiple distinct groups.
+SELECT udaf(DISTINCT b), udaf(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a;
+
+-- Aliases in SELECT could be used in GROUP BY
+SELECT a AS k, udaf(b) FROM testData GROUP BY k;
+SELECT a AS k, udaf(b) FROM testData GROUP BY k HAVING k > 1;
+
+-- GROUP BY alias with invalid col in SELECT list
+SELECT a AS k, udaf(non_existing) FROM testData GROUP BY k;
+
+-- Test data.
+CREATE OR REPLACE TEMPORARY VIEW testDataHasSameNameWithAlias AS SELECT * FROM VALUES
+(1, 1, 3), (1, 2, 1) AS testDataHasSameNameWithAlias(k, a, v);
+SELECT k AS a, udaf(v) FROM testDataHasSameNameWithAlias GROUP BY a;
+
+-- turn off group by aliases
+set spark.sql.groupByAliases=false;
+
+-- Check analysis exceptions
+SELECT a AS k, udaf(b) FROM testData GROUP BY k;
+
+-- Aggregate with empty input and non-empty GroupBy expressions.
+SELECT a, udaf(1) FROM testData WHERE false GROUP BY a;
+
+-- Aggregate with empty input and empty GroupBy expressions.
+SELECT udaf(1) FROM testData WHERE false;
+SELECT 1 FROM (SELECT udaf(1) FROM testData WHERE false) t;
+
+-- Aggregate with empty GroupBy expressions and filter on top
+SELECT 1 from (
+ SELECT 1 AS z,
+ udaf(a.x)
+ FROM (select 1 as x) a
+ WHERE false
+) b
+where b.z != b.z;
+
+-- SPARK-25708 HAVING without GROUP BY means global aggregate
+SELECT 1 FROM range(10) HAVING udaf(id) > 0;
+
+-- Test data
+CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES
+ (1, true), (1, false),
+ (2, true),
+ (3, false), (3, null),
+ (4, null), (4, null),
+ (5, null), (5, true), (5, false) AS test_agg(k, v);
+
+-- empty table
+SELECT udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE 1 = 0;
+
+-- all null values
+SELECT udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 4;
+
+-- aggregates are null Filtering
+SELECT udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg WHERE k = 5;
+
+-- group by
+SELECT k, udaf(v), some(v), any(v), bool_and(v), bool_or(v) FROM test_agg GROUP BY k;
+
+-- having
+SELECT k, udaf(v) FROM test_agg GROUP BY k HAVING udaf(v) = false;
+SELECT k, udaf(v) FROM test_agg GROUP BY k HAVING udaf(v) IS NULL;
+
+-- basic subquery path to make sure rewrite happens in both parent and child plans.
+SELECT k,
+ udaf(v) AS count
+FROM test_agg
+WHERE k = 2
+ AND v IN (SELECT Any(v)
+ FROM test_agg
+ WHERE k = 1)
+GROUP BY k;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/udaf/udaf-grouping-set.sql sql/core/src/test/resources/sql-tests/inputs/udaf/udaf-grouping-set.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/udaf/udaf-grouping-set.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/udaf/udaf-grouping-set.sql 2023-09-18 15:00:43.006708418 +0800
@@ -0,0 +1,47 @@
+CREATE TEMPORARY VIEW grouping AS SELECT * FROM VALUES
+ ("1", "2", "3", 1),
+ ("4", "5", "6", 1),
+ ("7", "8", "9", 1)
+ as grouping(a, b, c, d);
+
+-- SPARK-17849: grouping set throws NPE #1
+SELECT a, b, c, udaf(d) FROM grouping GROUP BY a, b, c GROUPING SETS (());
+
+-- SPARK-17849: grouping set throws NPE #2
+SELECT a, b, c, udaf(d) FROM grouping GROUP BY a, b, c GROUPING SETS ((a));
+
+-- SPARK-17849: grouping set throws NPE #3
+SELECT a, b, c, udaf(d) FROM grouping GROUP BY a, b, c GROUPING SETS ((c));
+
+-- Group sets without explicit group by
+SELECT c1, udaf(c2) FROM (VALUES ('x', 10, 0), ('y', 20, 0)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1);
+
+-- Group sets without group by and with grouping
+SELECT c1, udaf(c2), grouping(c1) FROM (VALUES ('x', 10, 0), ('y', 20, 0)) AS t (c1, c2, c3) GROUP BY GROUPING SETS (c1);
+
+-- Mutiple grouping within a grouping set
+SELECT c1, c2, udaf(c3), grouping__id
+FROM (VALUES ('x', 'a', 10), ('y', 'b', 20) ) AS t (c1, c2, c3)
+GROUP BY GROUPING SETS ( ( c1 ), ( c2 ) )
+HAVING GROUPING__ID > 1;
+
+-- complex expression in grouping sets
+SELECT a + b, b, udaf(c) FROM (VALUES (1,1,1),(2,2,2)) AS t(a,b,c) GROUP BY GROUPING SETS ( (a + b), (b));
+
+-- complex expression in grouping sets
+SELECT a + b, b, udaf(c) FROM (VALUES (1,1,1),(2,2,2)) AS t(a,b,c) GROUP BY GROUPING SETS ( (a + b), (b + a), (b));
+
+-- negative tests - must have at least one grouping expression
+SELECT a, b, c, udaf(d) FROM grouping GROUP BY WITH ROLLUP;
+
+SELECT a, b, c, udaf(d) FROM grouping GROUP BY WITH CUBE;
+
+-- duplicate entries in grouping sets
+SELECT k1, k2, udaf(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1));
+
+SELECT grouping__id, k1, k2, udaf(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1));
+
+SELECT grouping(k1), k1, k2, udaf(v) FROM (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY GROUPING SETS ((k1),(k1,k2),(k2,k1));
+
+-- grouping_id function
+SELECT grouping_id(k1, k2), udaf(v) from (VALUES (1,1,1),(2,2,2)) AS t(k1,k2,v) GROUP BY k1, k2 GROUPING SETS ((k2, k1), k1);
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/udaf/udaf.sql sql/core/src/test/resources/sql-tests/inputs/udaf/udaf.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/udaf/udaf.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/udaf/udaf.sql 2023-09-18 15:00:43.006708418 +0800
@@ -0,0 +1,21 @@
+-- Test aggregate operator and UDAF with codegen on and off.
+--CONFIG_DIM1 spark.sql.codegen.wholeStage=true
+--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY
+--CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN
+
+CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES
+(1), (2), (3), (4)
+as t1(int_col1);
+
+CREATE FUNCTION myDoubleAvg AS 'test.org.apache.spark.sql.MyDoubleAvg';
+
+SELECT default.myDoubleAvg(int_col1) as my_avg from t1;
+
+SELECT default.myDoubleAvg(int_col1, 3) as my_avg from t1;
+
+CREATE FUNCTION udaf1 AS 'test.non.existent.udaf';
+
+SELECT default.udaf1(int_col1) as udaf1 from t1;
+
+DROP FUNCTION myDoubleAvg;
+DROP FUNCTION udaf1;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/udaf.sql sql/core/src/test/resources/sql-tests/inputs/udaf.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/udaf.sql 2023-09-15 11:42:05.966317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/udaf.sql 1970-01-01 08:00:00.000000000 +0800
@@ -1,21 +0,0 @@
--- Test aggregate operator and UDAF with codegen on and off.
---CONFIG_DIM1 spark.sql.codegen.wholeStage=true
---CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY
---CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=NO_CODEGEN
-
-CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES
-(1), (2), (3), (4)
-as t1(int_col1);
-
-CREATE FUNCTION myDoubleAvg AS 'test.org.apache.spark.sql.MyDoubleAvg';
-
-SELECT default.myDoubleAvg(int_col1) as my_avg from t1;
-
-SELECT default.myDoubleAvg(int_col1, 3) as my_avg from t1;
-
-CREATE FUNCTION udaf1 AS 'test.non.existent.udaf';
-
-SELECT default.udaf1(int_col1) as udaf1 from t1;
-
-DROP FUNCTION myDoubleAvg;
-DROP FUNCTION udaf1;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part1.sql sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part1.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part1.sql 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-aggregates_part1.sql 2023-09-18 15:00:43.006708418 +0800
@@ -81,29 +81,28 @@
-- SQL2003 binary aggregates [SPARK-23907]
SELECT regr_count(b, a) FROM aggtest;
--- SELECT regr_sxx(b, a) FROM aggtest;
--- SELECT regr_syy(b, a) FROM aggtest;
--- SELECT regr_sxy(b, a) FROM aggtest;
+SELECT regr_sxx(b, a) FROM aggtest;
+SELECT regr_syy(b, a) FROM aggtest;
+SELECT regr_sxy(b, a) FROM aggtest;
SELECT regr_avgx(b, a), regr_avgy(b, a) FROM aggtest;
SELECT regr_r2(b, a) FROM aggtest;
--- SELECT regr_slope(b, a), regr_intercept(b, a) FROM aggtest;
+SELECT regr_slope(b, a), regr_intercept(b, a) FROM aggtest;
SELECT udf(covar_pop(b, udf(a))), covar_samp(udf(b), a) FROM aggtest;
SELECT corr(b, udf(a)) FROM aggtest;
-- test accum and combine functions directly [SPARK-23907]
--- CREATE TABLE regr_test (x float8, y float8);
--- INSERT INTO regr_test VALUES (10,150),(20,250),(30,350),(80,540),(100,200);
--- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x)
--- FROM regr_test WHERE x IN (10,20,30,80);
--- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x)
--- FROM regr_test;
+CREATE TEMPORARY VIEW regr_test AS SELECT * FROM VALUES (10,150),(20,250),(30,350),(80,540),(100,200) AS regr_test (x, y);
+SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x)
+FROM regr_test WHERE x IN (10,20,30,80);
+SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x)
+FROM regr_test;
-- SELECT float8_accum('{4,140,2900}'::float8[], 100);
-- SELECT float8_regr_accum('{4,140,2900,1290,83075,15050}'::float8[], 200, 100);
--- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x)
--- FROM regr_test WHERE x IN (10,20,30);
--- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x)
--- FROM regr_test WHERE x IN (80,100);
+SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x)
+FROM regr_test WHERE x IN (10,20,30);
+SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x)
+FROM regr_test WHERE x IN (80,100);
-- SELECT float8_combine('{3,60,200}'::float8[],ELECT CAST(udf(covar_pop(b, udf(a))) AS '{0,0,0}'::float8[]);
-- SELECT float8_combine('{0,0,0}'::float8[], '{2,180,200}'::float8[]);
-- SELECT float8_combine('{3,60,200}'::float8[], '{2,180,200}'::float8[]);
@@ -113,7 +112,7 @@
-- '{2,180,200,740,57800,-3400}'::float8[]);
-- SELECT float8_regr_combine('{3,60,200,750,20000,2000}'::float8[],
-- '{2,180,200,740,57800,-3400}'::float8[]);
--- DROP TABLE regr_test;
+DROP VIEW regr_test;
-- test count, distinct
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql 2023-09-15 11:42:05.966317944 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql 2023-09-18 15:00:43.006708418 +0800
@@ -66,6 +66,9 @@
first_value(udf(val)) OVER w AS first_value,
first_value(udf(val), true) OVER w AS first_value_ignore_null,
first_value(udf(val), false) OVER w AS first_value_contain_null,
+any_value(udf(val)) OVER w AS any_value,
+any_value(udf(val), true) OVER w AS any_value_ignore_null,
+any_value(udf(val), false) OVER w AS any_value_contain_null,
last_value(udf(val)) OVER w AS last_value,
last_value(udf(val), true) OVER w AS last_value_ignore_null,
last_value(udf(val), false) OVER w AS last_value_contain_null,
@@ -99,11 +102,14 @@
-- Over clause is empty
SELECT udf(val), cate, sum(val) OVER(), avg(val) OVER() FROM testData ORDER BY cate, val;
--- first_value()/last_value() over ()
+-- first_value()/last_value()/any_value() over ()
SELECT udf(val), cate,
first_value(false) OVER w AS first_value,
first_value(true, true) OVER w AS first_value_ignore_null,
first_value(false, false) OVER w AS first_value_contain_null,
+any_value(false) OVER w AS any_value,
+any_value(true, true) OVER w AS any_value_ignore_null,
+any_value(false, false) OVER w AS any_value_contain_null,
last_value(false) OVER w AS last_value,
last_value(true, true) OVER w AS last_value_ignore_null,
last_value(false, false) OVER w AS last_value_contain_null
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/unpivot.sql sql/core/src/test/resources/sql-tests/inputs/unpivot.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/unpivot.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/unpivot.sql 2023-09-18 15:00:43.006708418 +0800
@@ -0,0 +1,44 @@
+create temporary view courseEarnings as select * from values
+ ("dotNET", 15000, 48000, 22500),
+ ("Java", 20000, 30000, NULL)
+ as courseEarnings(course, `2012`, `2013`, `2014`);
+
+SELECT * FROM courseEarnings
+UNPIVOT (
+ earningsYear FOR year IN (`2012`, `2013`, `2014`)
+);
+
+-- NULL values excluded by default, include them explicitly
+SELECT * FROM courseEarnings
+UNPIVOT INCLUDE NULLS (
+ earningsYear FOR year IN (`2012`, `2013`, `2014`)
+);
+
+-- alias for column names
+SELECT * FROM courseEarnings
+UNPIVOT (
+ earningsYear FOR year IN (`2012` as `twenty-twelve`, `2013` as `twenty-thirteen`, `2014` as `twenty-fourteen`)
+);
+
+
+create temporary view courseEarningsAndSales as select * from values
+ ("dotNET", 15000, NULL, 48000, 1, 22500, 1),
+ ("Java", 20000, 1, 30000, 2, NULL, NULL)
+ as courseEarningsAndSales(course, earnings2012, sales2012, earnings2013, sales2013, earnings2014, sales2014);
+
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+ (earnings, sales) FOR year IN ((earnings2012, sales2012), (earnings2013, sales2013), (earnings2014, sales2014))
+);
+
+-- NULL values excluded by default, include them explicitly
+SELECT * FROM courseEarningsAndSales
+UNPIVOT INCLUDE NULLS (
+ (earnings, sales) FOR year IN ((earnings2012, sales2012), (earnings2013, sales2013), (earnings2014, sales2014))
+);
+
+-- alias for column names
+SELECT * FROM courseEarningsAndSales
+UNPIVOT (
+ (earnings, sales) FOR year IN ((earnings2012, sales2012) as `2012`, (earnings2013, sales2013) as `2013`, (earnings2014, sales2014) as `2014`)
+);
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/url-functions.sql sql/core/src/test/resources/sql-tests/inputs/url-functions.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/url-functions.sql 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/url-functions.sql 2023-09-18 15:00:43.006708418 +0800
@@ -0,0 +1,20 @@
+-- parse_url function
+select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'HOST');
+select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'PATH');
+select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'QUERY');
+select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'REF');
+select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'PROTOCOL');
+select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'FILE');
+select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'AUTHORITY');
+select parse_url('http://userinfo@spark.apache.org/path?query=1#Ref', 'USERINFO');
+
+-- url_encode function
+select url_encode('https://spark.apache.org');
+select url_encode('inva lid://user:pass@host/file\\;param?query\\;p2');
+select url_encode(null);
+
+-- url_decode function
+select url_decode('https%3A%2F%2Fspark.apache.org');
+select url_decode('http%3A%2F%2spark.apache.org');
+select url_decode('inva lid://user:pass@host/file\\;param?query\\;p2');
+select url_decode(null);
\ No newline at end of file
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/window.sql sql/core/src/test/resources/sql-tests/inputs/window.sql
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/inputs/window.sql 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/inputs/window.sql 2023-09-18 15:00:43.006708418 +0800
@@ -123,6 +123,9 @@
first_value(val) OVER w AS first_value,
first_value(val, true) OVER w AS first_value_ignore_null,
first_value(val, false) OVER w AS first_value_contain_null,
+any_value(val) OVER w AS any_value,
+any_value(val, true) OVER w AS any_value_ignore_null,
+any_value(val, false) OVER w AS any_value_contain_null,
last_value(val) OVER w AS last_value,
last_value(val, true) OVER w AS last_value_ignore_null,
last_value(val, false) OVER w AS last_value_contain_null,
@@ -156,11 +159,14 @@
-- Over clause is empty
SELECT val, cate, sum(val) OVER(), avg(val) OVER() FROM testData ORDER BY cate, val;
--- first_value()/last_value() over ()
+-- first_value()/last_value()/any_value() over ()
SELECT val, cate,
first_value(false) OVER w AS first_value,
first_value(true, true) OVER w AS first_value_ignore_null,
first_value(false, false) OVER w AS first_value_contain_null,
+any_value(false) OVER w AS any_value,
+any_value(true, true) OVER w AS any_value_ignore_null,
+any_value(false, false) OVER w AS any_value_contain_null,
last_value(false) OVER w AS last_value,
last_value(true, true) OVER w AS last_value_ignore_null,
last_value(false, false) OVER w AS last_value_contain_null
@@ -179,11 +185,12 @@
count(val) FILTER (WHERE val > 1) OVER(PARTITION BY cate)
FROM testData ORDER BY cate, val;
--- nth_value()/first_value() over ()
+-- nth_value()/first_value()/any_value() over ()
SELECT
employee_name,
salary,
first_value(employee_name) OVER w highest_salary,
+ any_value(employee_name) OVER w highest_salary,
nth_value(employee_name, 2) OVER w second_highest_salary
FROM
basic_pays
@@ -194,6 +201,7 @@
employee_name,
salary,
first_value(employee_name) OVER w highest_salary,
+ any_value(employee_name) OVER w highest_salary,
nth_value(employee_name, 2) OVER w second_highest_salary
FROM
basic_pays
@@ -204,6 +212,7 @@
employee_name,
salary,
first_value(employee_name) OVER w highest_salary,
+ any_value(employee_name) OVER w highest_salary,
nth_value(employee_name, 2) OVER w second_highest_salary
FROM
basic_pays
@@ -214,6 +223,7 @@
employee_name,
salary,
first_value(employee_name) OVER w highest_salary,
+ any_value(employee_name) OVER w highest_salary,
nth_value(employee_name, 2) OVER w second_highest_salary
FROM
basic_pays
@@ -224,6 +234,7 @@
employee_name,
salary,
first_value(employee_name) OVER w highest_salary,
+ any_value(employee_name) OVER w highest_salary,
nth_value(employee_name, 2) OVER w second_highest_salary
FROM
basic_pays
@@ -234,6 +245,7 @@
employee_name,
salary,
first_value(employee_name) OVER w highest_salary,
+ any_value(employee_name) OVER w highest_salary,
nth_value(employee_name, 2) OVER w second_highest_salary
FROM
basic_pays
@@ -244,6 +256,7 @@
employee_name,
salary,
first_value(employee_name) OVER w highest_salary,
+ any_value(employee_name) OVER w highest_salary,
nth_value(employee_name, 2) OVER w second_highest_salary
FROM
basic_pays
@@ -254,6 +267,7 @@
employee_name,
salary,
first_value(employee_name) OVER w highest_salary,
+ any_value(employee_name) OVER w highest_salary,
nth_value(employee_name, 2) OVER w second_highest_salary
FROM
basic_pays
@@ -264,6 +278,7 @@
employee_name,
salary,
first_value(employee_name) OVER w highest_salary,
+ any_value(employee_name) OVER w highest_salary,
nth_value(employee_name, 2) OVER w second_highest_salary
FROM
basic_pays
@@ -289,6 +304,7 @@
employee_name,
salary,
first_value(employee_name) OVER w highest_salary,
+ any_value(employee_name) OVER w highest_salary,
nth_value(employee_name, 2) OVER w second_highest_salary
FROM
basic_pays
@@ -313,6 +329,7 @@
nth_value(v, 2) IGNORE NULLS OVER w nth_value_2,
nth_value(v, 3) IGNORE NULLS OVER w nth_value_3,
first_value(v) IGNORE NULLS OVER w first_value,
+ any_value(v) IGNORE NULLS OVER w any_value,
last_value(v) IGNORE NULLS OVER w last_value
FROM
test_ignore_null
@@ -327,6 +344,7 @@
nth_value(v, 2) IGNORE NULLS OVER w nth_value_2,
nth_value(v, 3) IGNORE NULLS OVER w nth_value_3,
first_value(v) IGNORE NULLS OVER w first_value,
+ any_value(v) IGNORE NULLS OVER w any_value,
last_value(v) IGNORE NULLS OVER w last_value
FROM
test_ignore_null
@@ -341,6 +359,7 @@
nth_value(v, 2) IGNORE NULLS OVER w nth_value_2,
nth_value(v, 3) IGNORE NULLS OVER w nth_value_3,
first_value(v) IGNORE NULLS OVER w first_value,
+ any_value(v) IGNORE NULLS OVER w any_value,
last_value(v) IGNORE NULLS OVER w last_value
FROM
test_ignore_null
@@ -355,6 +374,7 @@
nth_value(v, 2) IGNORE NULLS OVER w nth_value_2,
nth_value(v, 3) IGNORE NULLS OVER w nth_value_3,
first_value(v) IGNORE NULLS OVER w first_value,
+ any_value(v) IGNORE NULLS OVER w any_value,
last_value(v) IGNORE NULLS OVER w last_value
FROM
test_ignore_null
@@ -369,6 +389,7 @@
nth_value(v, 2) IGNORE NULLS OVER w nth_value_2,
nth_value(v, 3) IGNORE NULLS OVER w nth_value_3,
first_value(v) IGNORE NULLS OVER w first_value,
+ any_value(v) IGNORE NULLS OVER w any_value,
last_value(v) IGNORE NULLS OVER w last_value
FROM
test_ignore_null
@@ -383,6 +404,7 @@
nth_value(v, 2) IGNORE NULLS OVER w nth_value_2,
nth_value(v, 3) IGNORE NULLS OVER w nth_value_3,
first_value(v) IGNORE NULLS OVER w first_value,
+ any_value(v) IGNORE NULLS OVER w any_value,
last_value(v) IGNORE NULLS OVER w last_value
FROM
test_ignore_null
@@ -397,6 +419,7 @@
nth_value(v, 2) IGNORE NULLS OVER w nth_value_2,
nth_value(v, 3) IGNORE NULLS OVER w nth_value_3,
first_value(v) IGNORE NULLS OVER w first_value,
+ any_value(v) IGNORE NULLS OVER w any_value,
last_value(v) IGNORE NULLS OVER w last_value
FROM
test_ignore_null
@@ -411,6 +434,7 @@
nth_value(v, 2) IGNORE NULLS OVER w nth_value_2,
nth_value(v, 3) IGNORE NULLS OVER w nth_value_3,
first_value(v) IGNORE NULLS OVER w first_value,
+ any_value(v) IGNORE NULLS OVER w any_value,
last_value(v) IGNORE NULLS OVER w last_value
FROM
test_ignore_null
@@ -425,6 +449,7 @@
nth_value(v, 2) IGNORE NULLS OVER w nth_value_2,
nth_value(v, 3) IGNORE NULLS OVER w nth_value_3,
first_value(v) IGNORE NULLS OVER w first_value,
+ any_value(v) IGNORE NULLS OVER w any_value,
last_value(v) IGNORE NULLS OVER w last_value
FROM
test_ignore_null
@@ -440,115 +465,3 @@
SUM(salary) OVER w sum_salary
FROM
basic_pays;
-
-SELECT
- employee_name,
- department,
- salary,
- percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department),
- percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department),
- percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department),
- percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department)
-FROM basic_pays
-ORDER BY salary;
-
-SELECT
- employee_name,
- department,
- salary,
- percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary),
- percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ORDER BY salary)
-FROM basic_pays
-ORDER BY salary;
-
-SELECT
- employee_name,
- department,
- salary,
- percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ORDER BY salary),
- percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ORDER BY salary)
-FROM basic_pays
-ORDER BY salary;
-
-SELECT
- employee_name,
- department,
- salary,
- percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING),
- percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)
-FROM basic_pays
-ORDER BY salary;
-
-SELECT
- employee_name,
- department,
- salary,
- percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING),
- percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)
-FROM basic_pays
-ORDER BY salary;
-
-SELECT
- employee_name,
- department,
- salary,
- percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w,
- percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w,
- percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w,
- percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w
-FROM basic_pays
-WINDOW w AS (PARTITION BY department)
-ORDER BY salary;
-
-SELECT
- employee_name,
- department,
- salary,
- percentile_cont(0.5) WITHIN GROUP (ORDER BY salary) OVER w,
- percentile_disc(0.5) WITHIN GROUP (ORDER BY salary) OVER w,
- percentile_cont(0.5) WITHIN GROUP (ORDER BY salary DESC) OVER w,
- percentile_disc(0.5) WITHIN GROUP (ORDER BY salary DESC) OVER w
-FROM basic_pays
-WHERE salary > 8900
-WINDOW w AS (PARTITION BY department)
-ORDER BY salary;
-
-SELECT
- employee_name,
- department,
- salary,
- percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w,
- percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w
-FROM basic_pays
-WINDOW w AS (PARTITION BY department ORDER BY salary)
-ORDER BY salary;
-
-SELECT
- employee_name,
- department,
- salary,
- percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w,
- percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w
-FROM basic_pays
-WINDOW w AS (PARTITION BY department ORDER BY salary)
-ORDER BY salary;
-
-SELECT
- employee_name,
- department,
- salary,
- percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w,
- percentile_cont(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w
-FROM basic_pays
-WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)
-ORDER BY salary;
-
-SELECT
- employee_name,
- department,
- salary,
- percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w,
- percentile_disc(0.25) WITHIN GROUP (ORDER BY salary DESC) OVER w
-FROM basic_pays
-WINDOW w AS (PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)
-ORDER BY salary;
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/array.sql.out 2023-09-18 15:00:43.006708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 38
-
-
-- !query
create temporary view data as select * from values
("one", array(11, 12, 13), array(array(111, 112, 113), array(121, 122, 123))),
@@ -131,7 +128,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'sort_array(array('b', 'd'), '1')' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "2",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"sort_array(array(b, d), 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 39,
+ "fragment" : "sort_array(array('b', 'd'), '1')"
+ } ]
+}
-- !query
@@ -140,7 +154,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'sort_array(array('b', 'd'), CAST(NULL AS BOOLEAN))' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"CAST(NULL AS BOOLEAN)\"",
+ "inputType" : "\"BOOLEAN\"",
+ "paramIndex" : "2",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"sort_array(array(b, d), CAST(NULL AS BOOLEAN))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 57,
+ "fragment" : "sort_array(array('b', 'd'), cast(NULL as boolean))"
+ } ]
+}
-- !query
@@ -168,7 +199,22 @@
struct<>
-- !query output
org.apache.spark.SparkArrayIndexOutOfBoundsException
-The index 5 is out of bounds. The array has 3 elements. Use `try_element_at` to tolerate accessing element at invalid index and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
+{
+ "errorClass" : "INVALID_ARRAY_INDEX_IN_ELEMENT_AT",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "arraySize" : "3",
+ "indexValue" : "5"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 36,
+ "fragment" : "element_at(array(1, 2, 3), 5)"
+ } ]
+}
-- !query
@@ -177,7 +223,22 @@
struct<>
-- !query output
org.apache.spark.SparkArrayIndexOutOfBoundsException
-The index -5 is out of bounds. The array has 3 elements. Use `try_element_at` to tolerate accessing element at invalid index and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
+{
+ "errorClass" : "INVALID_ARRAY_INDEX_IN_ELEMENT_AT",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "arraySize" : "3",
+ "indexValue" : "-5"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 37,
+ "fragment" : "element_at(array(1, 2, 3), -5)"
+ } ]
+}
-- !query
@@ -185,8 +246,18 @@
-- !query schema
struct<>
-- !query output
-java.lang.ArrayIndexOutOfBoundsException
-SQL array indices start at 1
+org.apache.spark.SparkRuntimeException
+{
+ "errorClass" : "INVALID_INDEX_OF_ZERO",
+ "sqlState" : "22003",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 36,
+ "fragment" : "element_at(array(1, 2, 3), 0)"
+ } ]
+}
-- !query
@@ -195,7 +266,22 @@
struct<>
-- !query output
org.apache.spark.SparkArrayIndexOutOfBoundsException
-The index 4 is out of bounds. The array has 2 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
+{
+ "errorClass" : "INVALID_ARRAY_INDEX",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "arraySize" : "2",
+ "indexValue" : "4"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 27,
+ "fragment" : "elt(4, '123', '456')"
+ } ]
+}
-- !query
@@ -204,7 +290,22 @@
struct<>
-- !query output
org.apache.spark.SparkArrayIndexOutOfBoundsException
-The index 0 is out of bounds. The array has 2 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
+{
+ "errorClass" : "INVALID_ARRAY_INDEX",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "arraySize" : "2",
+ "indexValue" : "0"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 27,
+ "fragment" : "elt(0, '123', '456')"
+ } ]
+}
-- !query
@@ -213,7 +314,22 @@
struct<>
-- !query output
org.apache.spark.SparkArrayIndexOutOfBoundsException
-The index -1 is out of bounds. The array has 2 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
+{
+ "errorClass" : "INVALID_ARRAY_INDEX",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "arraySize" : "2",
+ "indexValue" : "-1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "elt(-1, '123', '456')"
+ } ]
+}
-- !query
@@ -254,7 +370,22 @@
struct<>
-- !query output
org.apache.spark.SparkArrayIndexOutOfBoundsException
-The index 5 is out of bounds. The array has 3 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
+{
+ "errorClass" : "INVALID_ARRAY_INDEX",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "arraySize" : "3",
+ "indexValue" : "5"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 24,
+ "fragment" : "array(1, 2, 3)[5]"
+ } ]
+}
-- !query
@@ -263,7 +394,22 @@
struct<>
-- !query output
org.apache.spark.SparkArrayIndexOutOfBoundsException
-The index -1 is out of bounds. The array has 3 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
+{
+ "errorClass" : "INVALID_ARRAY_INDEX",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "arraySize" : "3",
+ "indexValue" : "-1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 25,
+ "fragment" : "array(1, 2, 3)[-1]"
+ } ]
+}
-- !query
@@ -304,82 +450,348 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'array_size(map('a', 1, 'b', 2))' due to data type mismatch: argument 1 requires array type, however, 'map('a', 1, 'b', 2)' is of map<string,int> type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"map(a, 1, b, 2)\"",
+ "inputType" : "\"MAP<STRING, INT>\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"ARRAY\"",
+ "sqlExpr" : "\"array_size(map(a, 1, b, 2))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 38,
+ "fragment" : "array_size(map('a', 1, 'b', 2))"
+ } ]
+}
-- !query
-set spark.sql.ansi.strictIndexOperator=false
+select size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10)))
-- !query schema
-struct<key:string,value:string>
+struct<size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10))):int>
-- !query output
-spark.sql.ansi.strictIndexOperator false
+4
-- !query
-select array(1, 2, 3)[5]
+select size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10)))
+-- !query schema
+struct<size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10))):int>
+-- !query output
+4
+
+
+-- !query
+select size(arrays_zip(array(1, 2, 3), array(4), null, array(7, 8, 9, 10)))
-- !query schema
-struct<array(1, 2, 3)[5]:int>
+struct<size(arrays_zip(array(1, 2, 3), array(4), NULL, array(7, 8, 9, 10))):int>
-- !query output
NULL
-- !query
-select array(1, 2, 3)[-1]
+select isnotnull(arrays_zip(array(), array(4), array(7, 8, 9, 10)))
+-- !query schema
+struct<(arrays_zip(array(), array(4), array(7, 8, 9, 10)) IS NOT NULL):boolean>
+-- !query output
+true
+
+
+-- !query
+select isnotnull(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10)))
+-- !query schema
+struct<(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10)) IS NOT NULL):boolean>
+-- !query output
+true
+
+
+-- !query
+select isnotnull(arrays_zip(array(1, 2, 3), NULL, array(4), array(7, 8, 9, 10)))
+-- !query schema
+struct<(arrays_zip(array(1, 2, 3), NULL, array(4), array(7, 8, 9, 10)) IS NOT NULL):boolean>
+-- !query output
+false
+
+
+-- !query
+select get(array(1, 2, 3), 0)
+-- !query schema
+struct<get(array(1, 2, 3), 0):int>
+-- !query output
+1
+
+
+-- !query
+select get(array(1, 2, 3), 3)
-- !query schema
-struct<array(1, 2, 3)[-1]:int>
+struct<get(array(1, 2, 3), 3):int>
-- !query output
NULL
-- !query
-select element_at(array(1, 2, 3), 5)
+select get(array(1, 2, 3), null)
-- !query schema
-struct<>
+struct<get(array(1, 2, 3), NULL):int>
-- !query output
-org.apache.spark.SparkArrayIndexOutOfBoundsException
-The index 5 is out of bounds. The array has 3 elements. Use `try_element_at` to tolerate accessing element at invalid index and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
+NULL
-- !query
-select element_at(array(1, 2, 3), -5)
+select get(array(1, 2, 3), -1)
-- !query schema
-struct<>
+struct<get(array(1, 2, 3), -1):int>
-- !query output
-org.apache.spark.SparkArrayIndexOutOfBoundsException
-The index -5 is out of bounds. The array has 3 elements. Use `try_element_at` to tolerate accessing element at invalid index and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
+NULL
-- !query
-select element_at(array(1, 2, 3), 0)
+select array_insert(array(1, 2, 3), 3, 4)
-- !query schema
-struct<>
+struct<array_insert(array(1, 2, 3), 3, 4):array<int>>
-- !query output
-java.lang.ArrayIndexOutOfBoundsException
-SQL array indices start at 1
+[1,2,4,3]
-- !query
-select elt(4, '123', '456')
+select array_insert(array(2, 3, 4), 0, 1)
-- !query schema
struct<>
-- !query output
-org.apache.spark.SparkArrayIndexOutOfBoundsException
-The index 4 is out of bounds. The array has 2 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
+org.apache.spark.SparkRuntimeException
+{
+ "errorClass" : "INVALID_INDEX_OF_ZERO",
+ "sqlState" : "22003",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 41,
+ "fragment" : "array_insert(array(2, 3, 4), 0, 1)"
+ } ]
+}
-- !query
-select elt(0, '123', '456')
+select array_insert(array(2, 3, 4), 1, 1)
+-- !query schema
+struct<array_insert(array(2, 3, 4), 1, 1):array<int>>
+-- !query output
+[1,2,3,4]
+
+
+-- !query
+select array_insert(array(1, 3, 4), -2, 2)
+-- !query schema
+struct<array_insert(array(1, 3, 4), -2, 2):array<int>>
+-- !query output
+[1,2,3,4]
+
+
+-- !query
+select array_insert(array(1, 2, 3), 3, "4")
-- !query schema
struct<>
-- !query output
-org.apache.spark.SparkArrayIndexOutOfBoundsException
-The index 0 is out of bounds. The array has 2 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "dataType" : "\"ARRAY\"",
+ "functionName" : "`array_insert`",
+ "leftType" : "\"ARRAY<INT>\"",
+ "rightType" : "\"STRING\"",
+ "sqlExpr" : "\"array_insert(array(1, 2, 3), 3, 4)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "array_insert(array(1, 2, 3), 3, \"4\")"
+ } ]
+}
-- !query
-select elt(-1, '123', '456')
+select array_insert(cast(NULL as ARRAY<INT>), 1, 1)
+-- !query schema
+struct<array_insert(NULL, 1, 1):array<int>>
+-- !query output
+NULL
+
+
+-- !query
+select array_insert(array(1, 2, 3, NULL), cast(NULL as INT), 4)
+-- !query schema
+struct<array_insert(array(1, 2, 3, NULL), CAST(NULL AS INT), 4):array<int>>
+-- !query output
+NULL
+
+
+-- !query
+select array_insert(array(1, 2, 3, NULL), 4, cast(NULL as INT))
+-- !query schema
+struct<array_insert(array(1, 2, 3, NULL), 4, CAST(NULL AS INT)):array<int>>
+-- !query output
+[1,2,3,null,null]
+
+
+-- !query
+select array_insert(array(2, 3, NULL, 4), 5, 5)
+-- !query schema
+struct<array_insert(array(2, 3, NULL, 4), 5, 5):array<int>>
+-- !query output
+[2,3,null,4,5]
+
+
+-- !query
+select array_insert(array(2, 3, NULL, 4), -5, 1)
+-- !query schema
+struct<array_insert(array(2, 3, NULL, 4), -5, 1):array<int>>
+-- !query output
+[1,null,2,3,null,4]
+
+
+-- !query
+select array_compact(id) from values (1) as t(id)
-- !query schema
struct<>
-- !query output
-org.apache.spark.SparkArrayIndexOutOfBoundsException
-The index -1 is out of bounds. The array has 2 elements. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"id\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"ARRAY\"",
+ "sqlExpr" : "\"array_compact(id)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 24,
+ "fragment" : "array_compact(id)"
+ } ]
+}
+
+
+-- !query
+select array_compact(array("1", null, "2", null))
+-- !query schema
+struct<array_compact(array(1, NULL, 2, NULL)):array<string>>
+-- !query output
+["1","2"]
+
+
+-- !query
+select array_compact(array("a", "b", "c"))
+-- !query schema
+struct<array_compact(array(a, b, c)):array<string>>
+-- !query output
+["a","b","c"]
+
+
+-- !query
+select array_compact(array(1D, null, 2D, null))
+-- !query schema
+struct<array_compact(array(1.0, NULL, 2.0, NULL)):array<double>>
+-- !query output
+[1.0,2.0]
+
+
+-- !query
+select array_compact(array(array(1, 2, 3, null), null, array(4, null, 6)))
+-- !query schema
+struct<array_compact(array(array(1, 2, 3, NULL), NULL, array(4, NULL, 6))):array<array<int>>>
+-- !query output
+[[1,2,3,null],[4,null,6]]
+
+
+-- !query
+select array_compact(array(null))
+-- !query schema
+struct<array_compact(array(NULL)):array<void>>
+-- !query output
+[]
+
+
+-- !query
+select array_append(array(1, 2, 3), 4)
+-- !query schema
+struct<array_append(array(1, 2, 3), 4):array<int>>
+-- !query output
+[1,2,3,4]
+
+
+-- !query
+select array_append(array('a', 'b', 'c'), 'd')
+-- !query schema
+struct<array_append(array(a, b, c), d):array<string>>
+-- !query output
+["a","b","c","d"]
+
+
+-- !query
+select array_append(array(1, 2, 3, NULL), NULL)
+-- !query schema
+struct<array_append(array(1, 2, 3, NULL), NULL):array<int>>
+-- !query output
+[1,2,3,null,null]
+
+
+-- !query
+select array_append(array('a', 'b', 'c', NULL), NULL)
+-- !query schema
+struct<array_append(array(a, b, c, NULL), NULL):array<string>>
+-- !query output
+["a","b","c",null,null]
+
+
+-- !query
+select array_append(CAST(null AS ARRAY<String>), 'a')
+-- !query schema
+struct<array_append(NULL, a):array<string>>
+-- !query output
+NULL
+
+
+-- !query
+select array_append(CAST(null AS ARRAY<String>), CAST(null as String))
+-- !query schema
+struct<array_append(NULL, CAST(NULL AS STRING)):array<string>>
+-- !query output
+NULL
+
+
+-- !query
+select array_append(array(), 1)
+-- !query schema
+struct<array_append(array(), 1):array<int>>
+-- !query output
+[1]
+
+
+-- !query
+select array_append(CAST(array() AS ARRAY<String>), CAST(NULL AS String))
+-- !query schema
+struct<array_append(array(), CAST(NULL AS STRING)):array<string>>
+-- !query output
+[null]
+
+
+-- !query
+select array_append(array(CAST(NULL AS String)), CAST(NULL AS String))
+-- !query schema
+struct<array_append(array(CAST(NULL AS STRING)), CAST(NULL AS STRING)):array<string>>
+-- !query output
+[null,null]
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/cast.sql.out sql/core/src/test/resources/sql-tests/results/ansi/cast.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/cast.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/cast.sql.out 2023-09-18 15:00:43.006708418 +0800
@@ -1,17 +1,27 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 72
-
-
-- !query
SELECT CAST('1.23' AS int)
-- !query schema
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '1.23' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('1.23' AS int)
- ^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'1.23'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"INT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 26,
+ "fragment" : "CAST('1.23' AS int)"
+ } ]
+}
-- !query
@@ -20,10 +30,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '1.23' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('1.23' AS long)
- ^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'1.23'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"BIGINT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 27,
+ "fragment" : "CAST('1.23' AS long)"
+ } ]
+}
-- !query
@@ -32,10 +55,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '-4.56' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('-4.56' AS int)
- ^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'-4.56'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"INT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 27,
+ "fragment" : "CAST('-4.56' AS int)"
+ } ]
+}
-- !query
@@ -44,10 +80,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '-4.56' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('-4.56' AS long)
- ^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'-4.56'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"BIGINT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "CAST('-4.56' AS long)"
+ } ]
+}
-- !query
@@ -56,10 +105,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value 'abc' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('abc' AS int)
- ^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'abc'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"INT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 25,
+ "fragment" : "CAST('abc' AS int)"
+ } ]
+}
-- !query
@@ -68,10 +130,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value 'abc' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('abc' AS long)
- ^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'abc'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"BIGINT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 26,
+ "fragment" : "CAST('abc' AS long)"
+ } ]
+}
-- !query
@@ -80,10 +155,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value 'abc' of the type "STRING" cannot be cast to "FLOAT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('abc' AS float)
- ^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'abc'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"FLOAT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 27,
+ "fragment" : "CAST('abc' AS float)"
+ } ]
+}
-- !query
@@ -92,10 +180,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value 'abc' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('abc' AS double)
- ^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'abc'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"DOUBLE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "CAST('abc' AS double)"
+ } ]
+}
-- !query
@@ -104,10 +205,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '1234567890123' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('1234567890123' AS int)
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'1234567890123'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"INT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 35,
+ "fragment" : "CAST('1234567890123' AS int)"
+ } ]
+}
-- !query
@@ -116,10 +230,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '12345678901234567890123' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('12345678901234567890123' AS long)
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'12345678901234567890123'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"BIGINT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 46,
+ "fragment" : "CAST('12345678901234567890123' AS long)"
+ } ]
+}
-- !query
@@ -128,10 +255,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('' AS int)
- ^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "''",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"INT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 22,
+ "fragment" : "CAST('' AS int)"
+ } ]
+}
-- !query
@@ -140,10 +280,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('' AS long)
- ^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "''",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"BIGINT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 23,
+ "fragment" : "CAST('' AS long)"
+ } ]
+}
-- !query
@@ -152,10 +305,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '' of the type "STRING" cannot be cast to "FLOAT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('' AS float)
- ^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "''",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"FLOAT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 24,
+ "fragment" : "CAST('' AS float)"
+ } ]
+}
-- !query
@@ -164,10 +330,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('' AS double)
- ^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "''",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"DOUBLE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 25,
+ "fragment" : "CAST('' AS double)"
+ } ]
+}
-- !query
@@ -192,10 +371,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '123.a' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('123.a' AS int)
- ^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'123.a'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"INT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 27,
+ "fragment" : "CAST('123.a' AS int)"
+ } ]
+}
-- !query
@@ -204,10 +396,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '123.a' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('123.a' AS long)
- ^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'123.a'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"BIGINT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "CAST('123.a' AS long)"
+ } ]
+}
-- !query
@@ -216,10 +421,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '123.a' of the type "STRING" cannot be cast to "FLOAT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('123.a' AS float)
- ^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'123.a'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"FLOAT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "CAST('123.a' AS float)"
+ } ]
+}
-- !query
@@ -228,10 +446,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '123.a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('123.a' AS double)
- ^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'123.a'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"DOUBLE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "CAST('123.a' AS double)"
+ } ]
+}
-- !query
@@ -248,10 +479,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '-2147483649' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('-2147483649' AS int)
- ^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'-2147483649'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"INT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 33,
+ "fragment" : "CAST('-2147483649' AS int)"
+ } ]
+}
-- !query
@@ -268,10 +512,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '2147483648' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('2147483648' AS int)
- ^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'2147483648'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"INT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 32,
+ "fragment" : "CAST('2147483648' AS int)"
+ } ]
+}
-- !query
@@ -288,10 +545,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '-9223372036854775809' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('-9223372036854775809' AS long)
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'-9223372036854775809'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"BIGINT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "CAST('-9223372036854775809' AS long)"
+ } ]
+}
-- !query
@@ -308,10 +578,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '9223372036854775808' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT CAST('9223372036854775808' AS long)
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'9223372036854775808'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"BIGINT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 42,
+ "fragment" : "CAST('9223372036854775808' AS long)"
+ } ]
+}
-- !query
@@ -328,10 +611,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(CAST(123 AS TINYINT) AS BINARY)' due to data type mismatch:
- cannot cast tinyint to binary with ANSI mode on.
- If you have to cast tinyint to binary, you can set spark.sql.ansi.enabled as false.
-; line 1 pos 11
+{
+ "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "configVal" : "'false'",
+ "sqlExpr" : "\"CAST(CAST(123 AS TINYINT) AS BINARY)\"",
+ "srcType" : "\"TINYINT\"",
+ "targetType" : "\"BINARY\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 12,
+ "stopIndex" : 44,
+ "fragment" : "CAST(CAST(123 AS byte) AS binary)"
+ } ]
+}
-- !query
@@ -340,10 +637,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(CAST(-123 AS TINYINT) AS BINARY)' due to data type mismatch:
- cannot cast tinyint to binary with ANSI mode on.
- If you have to cast tinyint to binary, you can set spark.sql.ansi.enabled as false.
-; line 1 pos 11
+{
+ "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "configVal" : "'false'",
+ "sqlExpr" : "\"CAST(CAST(-123 AS TINYINT) AS BINARY)\"",
+ "srcType" : "\"TINYINT\"",
+ "targetType" : "\"BINARY\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 12,
+ "stopIndex" : 45,
+ "fragment" : "CAST(CAST(-123 AS byte) AS binary)"
+ } ]
+}
-- !query
@@ -352,10 +663,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(123S AS BINARY)' due to data type mismatch:
- cannot cast smallint to binary with ANSI mode on.
- If you have to cast smallint to binary, you can set spark.sql.ansi.enabled as false.
-; line 1 pos 11
+{
+ "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "configVal" : "'false'",
+ "sqlExpr" : "\"CAST(123 AS BINARY)\"",
+ "srcType" : "\"SMALLINT\"",
+ "targetType" : "\"BINARY\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 12,
+ "stopIndex" : 31,
+ "fragment" : "CAST(123S AS binary)"
+ } ]
+}
-- !query
@@ -364,10 +689,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(-123S AS BINARY)' due to data type mismatch:
- cannot cast smallint to binary with ANSI mode on.
- If you have to cast smallint to binary, you can set spark.sql.ansi.enabled as false.
-; line 1 pos 11
+{
+ "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "configVal" : "'false'",
+ "sqlExpr" : "\"CAST(-123 AS BINARY)\"",
+ "srcType" : "\"SMALLINT\"",
+ "targetType" : "\"BINARY\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 12,
+ "stopIndex" : 32,
+ "fragment" : "CAST(-123S AS binary)"
+ } ]
+}
-- !query
@@ -376,10 +715,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(123 AS BINARY)' due to data type mismatch:
- cannot cast int to binary with ANSI mode on.
- If you have to cast int to binary, you can set spark.sql.ansi.enabled as false.
-; line 1 pos 11
+{
+ "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "configVal" : "'false'",
+ "sqlExpr" : "\"CAST(123 AS BINARY)\"",
+ "srcType" : "\"INT\"",
+ "targetType" : "\"BINARY\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 12,
+ "stopIndex" : 30,
+ "fragment" : "CAST(123 AS binary)"
+ } ]
+}
-- !query
@@ -388,10 +741,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(-123 AS BINARY)' due to data type mismatch:
- cannot cast int to binary with ANSI mode on.
- If you have to cast int to binary, you can set spark.sql.ansi.enabled as false.
-; line 1 pos 11
+{
+ "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "configVal" : "'false'",
+ "sqlExpr" : "\"CAST(-123 AS BINARY)\"",
+ "srcType" : "\"INT\"",
+ "targetType" : "\"BINARY\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 12,
+ "stopIndex" : 31,
+ "fragment" : "CAST(-123 AS binary)"
+ } ]
+}
-- !query
@@ -400,10 +767,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(123L AS BINARY)' due to data type mismatch:
- cannot cast bigint to binary with ANSI mode on.
- If you have to cast bigint to binary, you can set spark.sql.ansi.enabled as false.
-; line 1 pos 11
+{
+ "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "configVal" : "'false'",
+ "sqlExpr" : "\"CAST(123 AS BINARY)\"",
+ "srcType" : "\"BIGINT\"",
+ "targetType" : "\"BINARY\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 12,
+ "stopIndex" : 31,
+ "fragment" : "CAST(123L AS binary)"
+ } ]
+}
-- !query
@@ -412,10 +793,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'CAST(-123L AS BINARY)' due to data type mismatch:
- cannot cast bigint to binary with ANSI mode on.
- If you have to cast bigint to binary, you can set spark.sql.ansi.enabled as false.
-; line 1 pos 11
+{
+ "errorClass" : "DATATYPE_MISMATCH.CAST_WITH_CONF_SUGGESTION",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "configVal" : "'false'",
+ "sqlExpr" : "\"CAST(-123 AS BINARY)\"",
+ "srcType" : "\"BIGINT\"",
+ "targetType" : "\"BINARY\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 12,
+ "stopIndex" : 32,
+ "fragment" : "CAST(-123L AS binary)"
+ } ]
+}
-- !query
@@ -473,12 +868,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot mix year-month and day-time fields: interval 3 month 1 hour(line 1, pos 12)
-
-== SQL ==
-SELECT CAST(interval 3 month 1 hour AS string)
-------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0029",
+ "messageParameters" : {
+ "literal" : "interval 3 month 1 hour"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 13,
+ "stopIndex" : 35,
+ "fragment" : "interval 3 month 1 hour"
+ } ]
+}
-- !query
@@ -567,10 +969,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '1中文' of the type "STRING" cannot be cast to "TINYINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select cast('1中文' as tinyint)
- ^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'1中文'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"TINYINT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "cast('1中文' as tinyint)"
+ } ]
+}
-- !query
@@ -579,10 +994,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '1中文' of the type "STRING" cannot be cast to "SMALLINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select cast('1中文' as smallint)
- ^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'1中文'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"SMALLINT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "cast('1中文' as smallint)"
+ } ]
+}
-- !query
@@ -591,10 +1019,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '1中文' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select cast('1中文' as INT)
- ^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'1中文'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"INT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 25,
+ "fragment" : "cast('1中文' as INT)"
+ } ]
+}
-- !query
@@ -603,10 +1044,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '中文1' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select cast('中文1' as bigint)
- ^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'中文1'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"BIGINT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "cast('中文1' as bigint)"
+ } ]
+}
-- !query
@@ -615,10 +1069,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '1中文' of the type "STRING" cannot be cast to "BIGINT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select cast('1中文' as bigint)
- ^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'1中文'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"BIGINT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "cast('1中文' as bigint)"
+ } ]
+}
-- !query
@@ -645,11 +1112,23 @@
struct<>
-- !query output
org.apache.spark.SparkRuntimeException
-The value '
- xyz ' of the type "STRING" cannot be cast to "BOOLEAN" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select cast('\t\n xyz \t\r' as boolean)
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'\t\n xyz \t\r'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"BOOLEAN\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 39,
+ "fragment" : "cast('\\t\\n xyz \\t\\r' as boolean)"
+ } ]
+}
-- !query
@@ -666,10 +1145,23 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-Decimal(expanded, 123.45, 5, 2) cannot be represented as Decimal(4, 2). If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select cast('123.45' as decimal(4, 2))
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "precision" : "4",
+ "scale" : "2",
+ "value" : "123.45"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 38,
+ "fragment" : "cast('123.45' as decimal(4, 2))"
+ } ]
+}
-- !query
@@ -678,10 +1170,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value 'xyz' of the type "STRING" cannot be cast to "DECIMAL(4,2)" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select cast('xyz' as decimal(4, 2))
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'xyz'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"DECIMAL(4,2)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 35,
+ "fragment" : "cast('xyz' as decimal(4, 2))"
+ } ]
+}
-- !query
@@ -698,10 +1203,23 @@
struct<>
-- !query output
org.apache.spark.SparkDateTimeException
-The value 'a' of the type "STRING" cannot be cast to "DATE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select cast('a' as date)
- ^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'a'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 24,
+ "fragment" : "cast('a' as date)"
+ } ]
+}
-- !query
@@ -718,10 +1236,23 @@
struct<>
-- !query output
org.apache.spark.SparkDateTimeException
-The value 'a' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select cast('a' as timestamp)
- ^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'a'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "cast('a' as timestamp)"
+ } ]
+}
-- !query
@@ -738,10 +1269,23 @@
struct<>
-- !query output
org.apache.spark.SparkDateTimeException
-The value 'a' of the type "STRING" cannot be cast to "TIMESTAMP_NTZ" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select cast('a' as timestamp_ntz)
- ^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'a'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"TIMESTAMP_NTZ\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 33,
+ "fragment" : "cast('a' as timestamp_ntz)"
+ } ]
+}
-- !query
@@ -750,10 +1294,23 @@
struct<>
-- !query output
org.apache.spark.SparkDateTimeException
-The value Infinity of the type "DOUBLE" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select cast(cast('inf' as double) as timestamp)
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "Infinity",
+ "sourceType" : "\"DOUBLE\"",
+ "targetType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 47,
+ "fragment" : "cast(cast('inf' as double) as timestamp)"
+ } ]
+}
-- !query
@@ -762,7 +1319,343 @@
struct<>
-- !query output
org.apache.spark.SparkDateTimeException
-The value Infinity of the type "DOUBLE" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select cast(cast('inf' as float) as timestamp)
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "Infinity",
+ "sourceType" : "\"DOUBLE\"",
+ "targetType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 46,
+ "fragment" : "cast(cast('inf' as float) as timestamp)"
+ } ]
+}
+
+
+-- !query
+select cast(interval '1' year as tinyint)
+-- !query schema
+struct<CAST(INTERVAL '1' YEAR AS TINYINT):tinyint>
+-- !query output
+1
+
+
+-- !query
+select cast(interval '-10-2' year to month as smallint)
+-- !query schema
+struct<CAST(INTERVAL '-10-2' YEAR TO MONTH AS SMALLINT):smallint>
+-- !query output
+-122
+
+
+-- !query
+select cast(interval '1000' month as int)
+-- !query schema
+struct<CAST(INTERVAL '1000' MONTH AS INT):int>
+-- !query output
+1000
+
+
+-- !query
+select cast(interval -'10.123456' second as tinyint)
+-- !query schema
+struct<CAST(INTERVAL '-10.123456' SECOND AS TINYINT):tinyint>
+-- !query output
+-10
+
+
+-- !query
+select cast(interval '23:59:59' hour to second as smallint)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "CAST_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "sourceType" : "\"INTERVAL HOUR TO SECOND\"",
+ "targetType" : "\"SMALLINT\"",
+ "value" : "INTERVAL '23:59:59' HOUR TO SECOND"
+ }
+}
+
+
+-- !query
+select cast(interval -'1 02:03:04.123' day to second as int)
+-- !query schema
+struct<CAST(INTERVAL '-1 02:03:04.123' DAY TO SECOND AS INT):int>
+-- !query output
+-93784
+
+
+-- !query
+select cast(interval '10' day as bigint)
+-- !query schema
+struct<CAST(INTERVAL '10' DAY AS BIGINT):bigint>
+-- !query output
+10
+
+
+-- !query
+select cast(interval '-1000' month as tinyint)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "CAST_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "sourceType" : "\"INTERVAL MONTH\"",
+ "targetType" : "\"TINYINT\"",
+ "value" : "INTERVAL '-1000' MONTH"
+ }
+}
+
+
+-- !query
+select cast(interval '1000000' second as smallint)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "CAST_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "sourceType" : "\"INTERVAL SECOND\"",
+ "targetType" : "\"SMALLINT\"",
+ "value" : "INTERVAL '1000000' SECOND"
+ }
+}
+
+
+-- !query
+select cast(1Y as interval year)
+-- !query schema
+struct<CAST(1 AS INTERVAL YEAR):interval year>
+-- !query output
+1-0
+
+
+-- !query
+select cast(-122S as interval year to month)
+-- !query schema
+struct<CAST(-122 AS INTERVAL YEAR TO MONTH):interval year to month>
+-- !query output
+-10-2
+
+
+-- !query
+select cast(ym as interval year to month) from values(-122S) as t(ym)
+-- !query schema
+struct<ym:interval year to month>
+-- !query output
+-10-2
+
+
+-- !query
+select cast(1000 as interval month)
+-- !query schema
+struct<CAST(1000 AS INTERVAL MONTH):interval month>
+-- !query output
+83-4
+
+
+-- !query
+select cast(-10L as interval second)
+-- !query schema
+struct<CAST(-10 AS INTERVAL SECOND):interval second>
+-- !query output
+-0 00:00:10.000000000
+
+
+-- !query
+select cast(100Y as interval hour to second)
+-- !query schema
+struct<CAST(100 AS INTERVAL HOUR TO SECOND):interval hour to second>
+-- !query output
+0 00:01:40.000000000
+
+
+-- !query
+select cast(dt as interval hour to second) from values(100Y) as t(dt)
+-- !query schema
+struct<dt:interval hour to second>
+-- !query output
+0 00:01:40.000000000
+
+
+-- !query
+select cast(-1000S as interval day to second)
+-- !query schema
+struct<CAST(-1000 AS INTERVAL DAY TO SECOND):interval day to second>
+-- !query output
+-0 00:16:40.000000000
+
+
+-- !query
+select cast(10 as interval day)
+-- !query schema
+struct<CAST(10 AS INTERVAL DAY):interval day>
+-- !query output
+10 00:00:00.000000000
+
+
+-- !query
+select cast(2147483647 as interval year)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "CAST_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "sourceType" : "\"INT\"",
+ "targetType" : "\"INTERVAL YEAR\"",
+ "value" : "2147483647"
+ }
+}
+
+
+-- !query
+select cast(-9223372036854775808L as interval day)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "CAST_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "sourceType" : "\"BIGINT\"",
+ "targetType" : "\"INTERVAL DAY\"",
+ "value" : "-9223372036854775808L"
+ }
+}
+
+
+-- !query
+select cast(interval '-1' year as decimal(10, 0))
+-- !query schema
+struct<CAST(INTERVAL '-1' YEAR AS DECIMAL(10,0)):decimal(10,0)>
+-- !query output
+-1
+
+
+-- !query
+select cast(interval '1.000001' second as decimal(10, 6))
+-- !query schema
+struct<CAST(INTERVAL '01.000001' SECOND AS DECIMAL(10,6)):decimal(10,6)>
+-- !query output
+1.000001
+
+
+-- !query
+select cast(interval '08:11:10.001' hour to second as decimal(10, 4))
+-- !query schema
+struct<CAST(INTERVAL '08:11:10.001' HOUR TO SECOND AS DECIMAL(10,4)):decimal(10,4)>
+-- !query output
+29470.0010
+
+
+-- !query
+select cast(interval '1 01:02:03.1' day to second as decimal(8, 1))
+-- !query schema
+struct<CAST(INTERVAL '1 01:02:03.1' DAY TO SECOND AS DECIMAL(8,1)):decimal(8,1)>
+-- !query output
+90123.1
+
+
+-- !query
+select cast(interval '10.123' second as decimal(4, 2))
+-- !query schema
+struct<CAST(INTERVAL '10.123' SECOND AS DECIMAL(4,2)):decimal(4,2)>
+-- !query output
+10.12
+
+
+-- !query
+select cast(interval '10.005' second as decimal(4, 2))
+-- !query schema
+struct<CAST(INTERVAL '10.005' SECOND AS DECIMAL(4,2)):decimal(4,2)>
+-- !query output
+10.01
+
+
+-- !query
+select cast(interval '10.123' second as decimal(5, 2))
+-- !query schema
+struct<CAST(INTERVAL '10.123' SECOND AS DECIMAL(5,2)):decimal(5,2)>
+-- !query output
+10.12
+
+
+-- !query
+select cast(interval '10.123' second as decimal(1, 0))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "precision" : "1",
+ "scale" : "0",
+ "value" : "10.123000"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 54,
+ "fragment" : "cast(interval '10.123' second as decimal(1, 0))"
+ } ]
+}
+
+
+-- !query
+select cast(10.123456BD as interval day to second)
+-- !query schema
+struct<CAST(10.123456 AS INTERVAL DAY TO SECOND):interval day to second>
+-- !query output
+0 00:00:10.123456000
+
+
+-- !query
+select cast(80.654321BD as interval hour to minute)
+-- !query schema
+struct<CAST(80.654321 AS INTERVAL HOUR TO MINUTE):interval hour to minute>
+-- !query output
+0 01:20:00.000000000
+
+
+-- !query
+select cast(-10.123456BD as interval year to month)
+-- !query schema
+struct<CAST(-10.123456 AS INTERVAL YEAR TO MONTH):interval year to month>
+-- !query output
+-0-10
+
+
+-- !query
+select cast(10.654321BD as interval month)
+-- !query schema
+struct<CAST(10.654321 AS INTERVAL MONTH):interval month>
+-- !query output
+0-11
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/conditional-functions.sql.out sql/core/src/test/resources/sql-tests/results/ansi/conditional-functions.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/conditional-functions.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/conditional-functions.sql.out 2023-09-18 15:00:43.006708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 14
-
-
-- !query
CREATE TABLE conditional_t USING PARQUET AS SELECT c1, c2 FROM VALUES(1d, 0),(2d, 1),(null, 1),(CAST('NaN' AS DOUBLE), 0) AS t(c1, c2)
-- !query schema
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/date.sql.out sql/core/src/test/resources/sql-tests/results/ansi/date.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/date.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/date.sql.out 2023-09-18 15:00:43.006708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 97
-
-
-- !query
create temporary view date_view as select '2011-11-11' date_str, '1' int_str
-- !query schema
@@ -24,12 +21,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the DATE value: 2020-01-01中文(line 1, pos 7)
-
-== SQL ==
-select date '2020-01-01中文'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2020-01-01中文'",
+ "valueType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 26,
+ "fragment" : "date '2020-01-01中文'"
+ } ]
+}
-- !query
@@ -45,8 +51,14 @@
-- !query schema
struct<>
-- !query output
-java.time.DateTimeException
-Invalid value for MonthOfYear (valid values 1 - 12): 13. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_2000",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Invalid value for MonthOfYear (valid values 1 - 12): 13"
+ }
+}
-- !query
@@ -54,8 +66,14 @@
-- !query schema
struct<>
-- !query output
-java.time.DateTimeException
-Invalid value for DayOfMonth (valid values 1 - 28/31): 33. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_2000",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Invalid value for DayOfMonth (valid values 1 - 28/31): 33"
+ }
+}
-- !query
@@ -64,12 +82,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the DATE value: 015(line 1, pos 7)
-
-== SQL ==
-select date'015'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'015'",
+ "valueType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 16,
+ "fragment" : "date'015'"
+ } ]
+}
-- !query
@@ -78,12 +105,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the DATE value: 2021-4294967297-11(line 1, pos 7)
-
-== SQL ==
-select date'2021-4294967297-11'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2021-4294967297-11'",
+ "valueType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 31,
+ "fragment" : "date'2021-4294967297-11'"
+ } ]
+}
-- !query
@@ -103,6 +139,31 @@
-- !query
+select curdate(1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "1",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "0",
+ "functionName" : "`curdate`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 17,
+ "fragment" : "curdate(1)"
+ } ]
+}
+
+
+-- !query
select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null)
-- !query schema
struct<date_from_unix_date(0):date,date_from_unix_date(1000):date,date_from_unix_date(NULL):date>
@@ -139,8 +200,15 @@
-- !query schema
struct<>
-- !query output
-java.time.DateTimeException
-Invalid date 'February 29' as '1970' is not a leap year. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Invalid date 'February 29' as '1970' is not a leap year"
+ }
+}
-- !query
@@ -198,8 +266,14 @@
-- !query schema
struct<>
-- !query output
-java.lang.IllegalArgumentException
-Illegal input for day of week: xx. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkIllegalArgumentException
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_2000",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Illegal input for day of week: xx"
+ }
+}
-- !query
@@ -232,10 +306,23 @@
struct<>
-- !query output
org.apache.spark.SparkDateTimeException
-The value 'xx' of the type "STRING" cannot be cast to "DATE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select next_day("xx", "Mon")
- ^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'xx'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "next_day(\"xx\", \"Mon\")"
+ } ]
+}
-- !query
@@ -292,7 +379,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 1L)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1L' is of bigint type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"BIGINT\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 33,
+ "fragment" : "date_add('2011-11-11', 1L)"
+ } ]
+}
-- !query
@@ -301,7 +405,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 1.0BD)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1.0BD' is of decimal(2,1) type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1.0\"",
+ "inputType" : "\"DECIMAL(2,1)\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, 1.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_add('2011-11-11', 1.0)"
+ } ]
+}
-- !query
@@ -310,7 +431,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 10.0D)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '10.0D' is of double type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"10.0\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, 10.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_add('2011-11-11', 1E1)"
+ } ]
+}
-- !query
@@ -327,10 +465,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '1.2' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select date_add('2011-11-11', '1.2')
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'1.2'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"INT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 36,
+ "fragment" : "date_add('2011-11-11', '1.2')"
+ } ]
+}
-- !query
@@ -403,7 +554,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_sub(CAST('2011-11-11' AS DATE), 1L)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1L' is of bigint type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"BIGINT\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 33,
+ "fragment" : "date_sub('2011-11-11', 1L)"
+ } ]
+}
-- !query
@@ -412,7 +580,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_sub(CAST('2011-11-11' AS DATE), 1.0BD)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1.0BD' is of decimal(2,1) type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1.0\"",
+ "inputType" : "\"DECIMAL(2,1)\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, 1.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_sub('2011-11-11', 1.0)"
+ } ]
+}
-- !query
@@ -421,7 +606,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_sub(CAST('2011-11-11' AS DATE), 10.0D)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '10.0D' is of double type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"10.0\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, 10.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_sub('2011-11-11', 1E1)"
+ } ]
+}
-- !query
@@ -438,10 +640,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value '1.2' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select date_sub(date'2011-11-11', '1.2')
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'1.2'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"INT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 40,
+ "fragment" : "date_sub(date'2011-11-11', '1.2')"
+ } ]
+}
-- !query
@@ -514,7 +729,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(DATE '2011-11-11', 10.0D)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '10.0D' is of double type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"10.0\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(DATE '2011-11-11', 10.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "date '2011-11-11' + 1E1"
+ } ]
+}
-- !query
@@ -603,7 +835,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(DATE '2011-11-11', CAST('1' AS DATE))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST('1' AS DATE)' is of date type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"DATE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "date'2011-11-11' + '1'"
+ } ]
+}
-- !query
@@ -612,7 +861,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(CAST('1' AS DATE), DATE '2011-11-11')' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'DATE '2011-11-11'' is of date type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"DATE '2011-11-11'\"",
+ "inputType" : "\"DATE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(1, DATE '2011-11-11')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "'1' + date'2011-11-11'"
+ } ]
+}
-- !query
@@ -650,7 +916,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd/MMMMM/yyyy'"
+ }
+}
-- !query
@@ -659,7 +933,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd/MMMMM/yyyy'"
+ }
+}
-- !query
@@ -668,7 +950,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd/MMMMM/yyyy'"
+ }
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/datetime-parsing-invalid.sql.out 2023-09-18 15:00:43.006708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 28
-
-
-- !query
select to_timestamp('294248', 'y')
-- !query schema
@@ -17,7 +14,14 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "datetime" : "'1'"
+ }
+}
-- !query
@@ -25,8 +29,15 @@
-- !query schema
struct<>
-- !query output
-java.time.format.DateTimeParseException
-Text '-12' could not be parsed at index 0. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '-12' could not be parsed at index 0"
+ }
+}
-- !query
@@ -35,7 +46,14 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '123' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "datetime" : "'123'"
+ }
+}
-- !query
@@ -44,7 +62,14 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "datetime" : "'1'"
+ }
+}
-- !query
@@ -53,7 +78,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'yyyyyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'yyyyyyy'"
+ }
+}
-- !query
@@ -61,8 +94,15 @@
-- !query schema
struct<>
-- !query output
-java.time.DateTimeException
-Invalid date 'DayOfYear 366' as '1970' is not a leap year. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Invalid date 'DayOfYear 366' as '1970' is not a leap year"
+ }
+}
-- !query
@@ -71,7 +111,14 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '9' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "datetime" : "'9'"
+ }
+}
-- !query
@@ -80,7 +127,14 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '9' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "datetime" : "'9'"
+ }
+}
-- !query
@@ -89,7 +143,14 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '99' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "datetime" : "'99'"
+ }
+}
-- !query
@@ -97,8 +158,15 @@
-- !query schema
struct<>
-- !query output
-java.time.DateTimeException
-Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31."
+ }
+}
-- !query
@@ -106,8 +174,15 @@
-- !query schema
struct<>
-- !query output
-java.time.DateTimeException
-Conflict found: Field MonthOfYear 11 differs from MonthOfYear 12 derived from 1970-12-31. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Conflict found: Field MonthOfYear 11 differs from MonthOfYear 12 derived from 1970-12-31."
+ }
+}
-- !query
@@ -115,8 +190,15 @@
-- !query schema
struct<>
-- !query output
-java.time.format.DateTimeParseException
-Text '2019-366' could not be parsed: Invalid date 'DayOfYear 366' as '2019' is not a leap year. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '2019-366' could not be parsed: Invalid date 'DayOfYear 366' as '2019' is not a leap year"
+ }
+}
-- !query
@@ -124,8 +206,15 @@
-- !query schema
struct<>
-- !query output
-java.time.DateTimeException
-Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Conflict found: Field DayOfMonth 30 differs from DayOfMonth 31 derived from 1970-12-31."
+ }
+}
-- !query
@@ -133,8 +222,15 @@
-- !query schema
struct<>
-- !query output
-java.time.format.DateTimeParseException
-Text '2020-01-365' could not be parsed: Conflict found: Field DayOfMonth 30 differs from DayOfMonth 1 derived from 2020-12-30. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '2020-01-365' could not be parsed: Conflict found: Field DayOfMonth 30 differs from DayOfMonth 1 derived from 2020-12-30"
+ }
+}
-- !query
@@ -142,8 +238,15 @@
-- !query schema
struct<>
-- !query output
-java.time.format.DateTimeParseException
-Text '2020-10-350' could not be parsed: Conflict found: Field MonthOfYear 12 differs from MonthOfYear 10 derived from 2020-12-15. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '2020-10-350' could not be parsed: Conflict found: Field MonthOfYear 12 differs from MonthOfYear 10 derived from 2020-12-15"
+ }
+}
-- !query
@@ -151,8 +254,15 @@
-- !query schema
struct<>
-- !query output
-java.time.format.DateTimeParseException
-Text '2020-11-31-366' could not be parsed: Invalid date 'NOVEMBER 31'. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '2020-11-31-366' could not be parsed: Invalid date 'NOVEMBER 31'"
+ }
+}
-- !query
@@ -161,7 +271,14 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '2018-366' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "datetime" : "'2018-366'"
+ }
+}
-- !query
@@ -169,8 +286,15 @@
-- !query schema
struct<>
-- !query output
-java.time.format.DateTimeParseException
-Text '2020-01-27T20:06:11.847' could not be parsed at index 10. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10"
+ }
+}
-- !query
@@ -178,8 +302,15 @@
-- !query schema
struct<>
-- !query output
-java.time.format.DateTimeParseException
-Text 'Unparseable' could not be parsed at index 0. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text 'Unparseable' could not be parsed at index 0"
+ }
+}
-- !query
@@ -187,8 +318,15 @@
-- !query schema
struct<>
-- !query output
-java.time.format.DateTimeParseException
-Text '2020-01-27T20:06:11.847' could not be parsed at index 10. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10"
+ }
+}
-- !query
@@ -196,8 +334,15 @@
-- !query schema
struct<>
-- !query output
-java.time.format.DateTimeParseException
-Text 'Unparseable' could not be parsed at index 0. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text 'Unparseable' could not be parsed at index 0"
+ }
+}
-- !query
@@ -205,8 +350,15 @@
-- !query schema
struct<>
-- !query output
-java.time.format.DateTimeParseException
-Text '2020-01-27T20:06:11.847' could not be parsed at index 10. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10"
+ }
+}
-- !query
@@ -214,8 +366,15 @@
-- !query schema
struct<>
-- !query output
-java.time.format.DateTimeParseException
-Text 'Unparseable' could not be parsed at index 0. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text 'Unparseable' could not be parsed at index 0"
+ }
+}
-- !query
@@ -223,8 +382,15 @@
-- !query schema
struct<>
-- !query output
-java.time.format.DateTimeParseException
-Text '2020-01-27T20:06:11.847' could not be parsed at index 10. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '2020-01-27T20:06:11.847' could not be parsed at index 10"
+ }
+}
-- !query
@@ -232,8 +398,15 @@
-- !query schema
struct<>
-- !query output
-java.time.format.DateTimeParseException
-Text 'Unparseable' could not be parsed at index 0. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text 'Unparseable' could not be parsed at index 0"
+ }
+}
-- !query
@@ -242,10 +415,23 @@
struct<>
-- !query output
org.apache.spark.SparkDateTimeException
-The value 'Unparseable' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select cast("Unparseable" as timestamp)
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'Unparseable'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 39,
+ "fragment" : "cast(\"Unparseable\" as timestamp)"
+ } ]
+}
-- !query
@@ -254,7 +440,20 @@
struct<>
-- !query output
org.apache.spark.SparkDateTimeException
-The value 'Unparseable' of the type "STRING" cannot be cast to "DATE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select cast("Unparseable" as date)
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'Unparseable'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "cast(\"Unparseable\" as date)"
+ } ]
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/datetime-special.sql.out sql/core/src/test/resources/sql-tests/results/ansi/datetime-special.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/datetime-special.sql.out 2023-09-15 11:42:05.966317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/datetime-special.sql.out 2023-09-18 15:00:43.006708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 4
-
-
-- !query
select date'999999-03-18', date'-0001-1-28', date'0015'
-- !query schema
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/decimalArithmeticOperations.sql.out 2023-09-18 15:00:43.006708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 16
-
-
-- !query
create table decimals_test(id int, a decimal(38,18), b decimal(38,18)) using parquet
-- !query schema
@@ -76,10 +73,23 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-Decimal(expanded, 10000000000000000000000000000000000000.1, 39, 1) cannot be represented as Decimal(38, 1). If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select (5e36BD + 0.1) + 5e36BD
- ^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "precision" : "38",
+ "scale" : "1",
+ "value" : "10000000000000000000000000000000000000.1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "(5e36BD + 0.1) + 5e36BD"
+ } ]
+}
-- !query
@@ -88,10 +98,23 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-Decimal(expanded, -11000000000000000000000000000000000000.1, 39, 1) cannot be represented as Decimal(38, 1). If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select (-4e36BD - 0.1) - 7e36BD
- ^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "precision" : "38",
+ "scale" : "1",
+ "value" : "-11000000000000000000000000000000000000.1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 31,
+ "fragment" : "(-4e36BD - 0.1) - 7e36BD"
+ } ]
+}
-- !query
@@ -100,10 +123,23 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-Decimal(expanded, 152415787532388367501905199875019052100, 39, 0) cannot be represented as Decimal(38, 2). If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select 12345678901234567890.0 * 12345678901234567890.0
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "precision" : "38",
+ "scale" : "2",
+ "value" : "152415787532388367501905199875019052100"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 54,
+ "fragment" : "12345678901234567890.0 * 12345678901234567890.0"
+ } ]
+}
-- !query
@@ -112,10 +148,23 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-Decimal(expanded, 1000000000000000000000000000000000000, 37, 0) cannot be represented as Decimal(38, 6). If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select 1e35BD / 0.1
- ^^^^^^^^^^^^
+{
+ "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "precision" : "38",
+ "scale" : "6",
+ "value" : "1000000000000000000000000000000000000.00000000000000000000000000000000000000"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 19,
+ "fragment" : "1e35BD / 0.1"
+ } ]
+}
-- !query
@@ -143,6 +192,172 @@
-- !query
+select 1.0123456789012345678901234567890123456e36BD / 0.1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "precision" : "38",
+ "scale" : "6",
+ "value" : "10123456789012345678901234567890123456.00000000000000000000000000000000000000"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 57,
+ "fragment" : "1.0123456789012345678901234567890123456e36BD / 0.1"
+ } ]
+}
+
+
+-- !query
+select 1.0123456789012345678901234567890123456e35BD / 1.0
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "precision" : "38",
+ "scale" : "6",
+ "value" : "101234567890123456789012345678901234.56000000000000000000000000000000000000"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 57,
+ "fragment" : "1.0123456789012345678901234567890123456e35BD / 1.0"
+ } ]
+}
+
+
+-- !query
+select 1.0123456789012345678901234567890123456e34BD / 1.0
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "precision" : "38",
+ "scale" : "6",
+ "value" : "10123456789012345678901234567890123.45600000000000000000000000000000000000"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 57,
+ "fragment" : "1.0123456789012345678901234567890123456e34BD / 1.0"
+ } ]
+}
+
+
+-- !query
+select 1.0123456789012345678901234567890123456e33BD / 1.0
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "precision" : "38",
+ "scale" : "6",
+ "value" : "1012345678901234567890123456789012.34560000000000000000000000000000000000"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 57,
+ "fragment" : "1.0123456789012345678901234567890123456e33BD / 1.0"
+ } ]
+}
+
+
+-- !query
+select 1.0123456789012345678901234567890123456e32BD / 1.0
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "precision" : "38",
+ "scale" : "6",
+ "value" : "101234567890123456789012345678901.23456000000000000000000000000000000000"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 57,
+ "fragment" : "1.0123456789012345678901234567890123456e32BD / 1.0"
+ } ]
+}
+
+
+-- !query
+select 1.0123456789012345678901234567890123456e31BD / 1.0
+-- !query schema
+struct<(10123456789012345678901234567890.123456 / 1.0):decimal(38,6)>
+-- !query output
+10123456789012345678901234567890.123456
+
+
+-- !query
+select 1.0123456789012345678901234567890123456e31BD / 0.1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "precision" : "38",
+ "scale" : "6",
+ "value" : "101234567890123456789012345678901.23456000000000000000000000000000000000"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 57,
+ "fragment" : "1.0123456789012345678901234567890123456e31BD / 0.1"
+ } ]
+}
+
+
+-- !query
+select 1.0123456789012345678901234567890123456e31BD / 10.0
+-- !query schema
+struct<(10123456789012345678901234567890.123456 / 10.0):decimal(38,6)>
+-- !query output
+1012345678901234567890123456789.012346
+
+
+-- !query
drop table decimals_test
-- !query schema
struct<>
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-disabled.sql.out sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-disabled.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-disabled.sql.out 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-disabled.sql.out 2023-09-18 15:00:43.006708418 +0800
@@ -0,0 +1,443 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+SELECT 1 FROM "not_exist"
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"not_exist\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+USE SCHEMA "not_exist"
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"not_exist\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+ALTER TABLE "not_exist" ADD COLUMN not_exist int
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"not_exist\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+ALTER TABLE not_exist ADD COLUMN "not_exist" int
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"not_exist\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+SELECT 1 AS "not_exist" FROM not_exist
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"not_exist\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+SELECT 1 FROM not_exist AS X("hello")
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"hello\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+SELECT "not_exist"()
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"not_exist\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+SELECT "not_exist".not_exist()
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"not_exist\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+SELECT 1 FROM `hello`
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`hello`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 21,
+ "fragment" : "`hello`"
+ } ]
+}
+
+
+-- !query
+USE SCHEMA `not_exist`
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
+{
+ "errorClass" : "SCHEMA_NOT_FOUND",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "schemaName" : "`not_exist`"
+ }
+}
+
+
+-- !query
+ALTER TABLE `not_exist` ADD COLUMN not_exist int
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`not_exist`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 13,
+ "stopIndex" : 23,
+ "fragment" : "`not_exist`"
+ } ]
+}
+
+
+-- !query
+ALTER TABLE not_exist ADD COLUMN `not_exist` int
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`not_exist`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 13,
+ "stopIndex" : 21,
+ "fragment" : "not_exist"
+ } ]
+}
+
+
+-- !query
+SELECT 1 AS `not_exist` FROM `not_exist`
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`not_exist`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 30,
+ "stopIndex" : 40,
+ "fragment" : "`not_exist`"
+ } ]
+}
+
+
+-- !query
+SELECT 1 FROM not_exist AS X(`hello`)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`not_exist`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 37,
+ "fragment" : "not_exist AS X(`hello`)"
+ } ]
+}
+
+
+-- !query
+SELECT `not_exist`()
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVED_ROUTINE",
+ "sqlState" : "42883",
+ "messageParameters" : {
+ "routineName" : "`not_exist`",
+ "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 20,
+ "fragment" : "`not_exist`()"
+ } ]
+}
+
+
+-- !query
+SELECT `not_exist`.not_exist()
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVED_ROUTINE",
+ "sqlState" : "42883",
+ "messageParameters" : {
+ "routineName" : "`not_exist`.`not_exist`",
+ "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "`not_exist`.not_exist()"
+ } ]
+}
+
+
+-- !query
+SELECT "hello"
+-- !query schema
+struct<hello:string>
+-- !query output
+hello
+
+
+-- !query
+CREATE TEMPORARY VIEW v(c1 COMMENT "hello") AS SELECT 1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DROP VIEW v
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT INTERVAL "1" YEAR
+-- !query schema
+struct<INTERVAL '1' YEAR:interval year>
+-- !query output
+1-0
+
+
+-- !query
+SELECT 'hello'
+-- !query schema
+struct<hello:string>
+-- !query output
+hello
+
+
+-- !query
+CREATE TEMPORARY VIEW v(c1 COMMENT 'hello') AS SELECT 1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DROP VIEW v
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT INTERVAL '1' YEAR
+-- !query schema
+struct<INTERVAL '1' YEAR:interval year>
+-- !query output
+1-0
+
+
+-- !query
+CREATE SCHEMA "myschema"
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"myschema\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+CREATE TEMPORARY VIEW "myview"("c1") AS
+ WITH "v"("a") AS (SELECT 1) SELECT "a" FROM "v"
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"myview\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+SELECT "a1" AS "a2" FROM "myview" AS "atab"("a1")
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"a2\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+DROP TABLE "myview"
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"myview\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+DROP SCHEMA "myschema"
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"myschema\"'",
+ "hint" : ""
+ }
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out 2023-09-18 15:00:43.006708418 +0800
@@ -0,0 +1,483 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+SELECT 1 FROM "not_exist"
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`not_exist`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 25,
+ "fragment" : "\"not_exist\""
+ } ]
+}
+
+
+-- !query
+USE SCHEMA "not_exist"
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
+{
+ "errorClass" : "SCHEMA_NOT_FOUND",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "schemaName" : "`not_exist`"
+ }
+}
+
+
+-- !query
+ALTER TABLE "not_exist" ADD COLUMN not_exist int
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`not_exist`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 13,
+ "stopIndex" : 23,
+ "fragment" : "\"not_exist\""
+ } ]
+}
+
+
+-- !query
+ALTER TABLE not_exist ADD COLUMN "not_exist" int
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`not_exist`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 13,
+ "stopIndex" : 21,
+ "fragment" : "not_exist"
+ } ]
+}
+
+
+-- !query
+SELECT 1 AS "not_exist" FROM not_exist
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`not_exist`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 30,
+ "stopIndex" : 38,
+ "fragment" : "not_exist"
+ } ]
+}
+
+
+-- !query
+SELECT 1 FROM not_exist AS X("hello")
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`not_exist`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 37,
+ "fragment" : "not_exist AS X(\"hello\")"
+ } ]
+}
+
+
+-- !query
+SELECT "not_exist"()
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVED_ROUTINE",
+ "sqlState" : "42883",
+ "messageParameters" : {
+ "routineName" : "`not_exist`",
+ "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 20,
+ "fragment" : "\"not_exist\"()"
+ } ]
+}
+
+
+-- !query
+SELECT "not_exist".not_exist()
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVED_ROUTINE",
+ "sqlState" : "42883",
+ "messageParameters" : {
+ "routineName" : "`not_exist`.`not_exist`",
+ "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "\"not_exist\".not_exist()"
+ } ]
+}
+
+
+-- !query
+SELECT 1 FROM `hello`
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`hello`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 21,
+ "fragment" : "`hello`"
+ } ]
+}
+
+
+-- !query
+USE SCHEMA `not_exist`
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
+{
+ "errorClass" : "SCHEMA_NOT_FOUND",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "schemaName" : "`not_exist`"
+ }
+}
+
+
+-- !query
+ALTER TABLE `not_exist` ADD COLUMN not_exist int
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`not_exist`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 13,
+ "stopIndex" : 23,
+ "fragment" : "`not_exist`"
+ } ]
+}
+
+
+-- !query
+ALTER TABLE not_exist ADD COLUMN `not_exist` int
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`not_exist`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 13,
+ "stopIndex" : 21,
+ "fragment" : "not_exist"
+ } ]
+}
+
+
+-- !query
+SELECT 1 AS `not_exist` FROM `not_exist`
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`not_exist`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 30,
+ "stopIndex" : 40,
+ "fragment" : "`not_exist`"
+ } ]
+}
+
+
+-- !query
+SELECT 1 FROM not_exist AS X(`hello`)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`not_exist`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 37,
+ "fragment" : "not_exist AS X(`hello`)"
+ } ]
+}
+
+
+-- !query
+SELECT `not_exist`()
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVED_ROUTINE",
+ "sqlState" : "42883",
+ "messageParameters" : {
+ "routineName" : "`not_exist`",
+ "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 20,
+ "fragment" : "`not_exist`()"
+ } ]
+}
+
+
+-- !query
+SELECT `not_exist`.not_exist()
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVED_ROUTINE",
+ "sqlState" : "42883",
+ "messageParameters" : {
+ "routineName" : "`not_exist`.`not_exist`",
+ "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "`not_exist`.not_exist()"
+ } ]
+}
+
+
+-- !query
+SELECT "hello"
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION",
+ "sqlState" : "42703",
+ "messageParameters" : {
+ "objectName" : "`hello`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 14,
+ "fragment" : "\"hello\""
+ } ]
+}
+
+
+-- !query
+CREATE TEMPORARY VIEW v(c1 COMMENT "hello") AS SELECT 1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"hello\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+DROP VIEW v
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.analysis.NoSuchTableException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`spark_catalog`.`default`.`v`"
+ }
+}
+
+
+-- !query
+SELECT INTERVAL "1" YEAR
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"1\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+SELECT 'hello'
+-- !query schema
+struct<hello:string>
+-- !query output
+hello
+
+
+-- !query
+CREATE TEMPORARY VIEW v(c1 COMMENT 'hello') AS SELECT 1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DROP VIEW v
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT INTERVAL '1' YEAR
+-- !query schema
+struct<INTERVAL '1' YEAR:interval year>
+-- !query output
+1-0
+
+
+-- !query
+CREATE SCHEMA "myschema"
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE TEMPORARY VIEW "myview"("c1") AS
+ WITH "v"("a") AS (SELECT 1) SELECT "a" FROM "v"
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT "a1" AS "a2" FROM "myview" AS "atab"("a1")
+-- !query schema
+struct<a2:int>
+-- !query output
+1
+
+
+-- !query
+DROP TABLE "myview"
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DROP SCHEMA "myschema"
+-- !query schema
+struct<>
+-- !query output
+
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/higher-order-functions.sql.out 2023-09-18 15:00:43.006708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 30
-
-
-- !query
create or replace temporary view nested as values
(1, array(32, 97), array(array(12, 99), array(123, 42), array(1))),
@@ -20,7 +17,19 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-A lambda function should only be used in a higher order function. However, its class is org.apache.spark.sql.catalyst.expressions.Upper, which is not a higher order function.; line 1 pos 7
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_2306",
+ "messageParameters" : {
+ "class" : "org.apache.spark.sql.catalyst.expressions.Upper"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 20,
+ "fragment" : "upper(x -> x)"
+ } ]
+}
-- !query
@@ -148,6 +157,44 @@
-- !query
+select reduce(ys, 0, (y, a) -> y + a + x) as v from nested
+-- !query schema
+struct<v:int>
+-- !query output
+131
+15
+5
+
+
+-- !query
+select reduce(ys, (0 as sum, 0 as n), (acc, x) -> (acc.sum + x, acc.n + 1), acc -> acc.sum / acc.n) as v from nested
+-- !query schema
+struct<v:double>
+-- !query output
+0.5
+12.0
+64.5
+
+
+-- !query
+select transform(zs, z -> reduce(z, 1, (acc, val) -> acc * val * size(z))) as v from nested
+-- !query schema
+struct<v:array<int>>
+-- !query output
+[1010880,8]
+[17]
+[4752,20664,1]
+
+
+-- !query
+select reduce(cast(null as array<int>), 0, (a, y) -> a + y + 1, a -> a + 2) as v
+-- !query schema
+struct<v:int>
+-- !query output
+NULL
+
+
+-- !query
select exists(ys, y -> y > 30) as v from nested
-- !query schema
struct<v:boolean>
@@ -277,4 +324,4 @@
-- !query schema
struct<aggregate(split(abcdefgh, , -1), array(array()), lambdafunction(array(array(namedlambdavariable())), namedlambdavariable(), namedlambdavariable()), lambdafunction(namedlambdavariable(), namedlambdavariable())):array<array<string>>>
-- !query output
-[[""]]
+[["h"]]
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out 2023-09-18 15:00:43.006708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 285
-
-
-- !query
select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15')
-- !query schema
@@ -16,12 +13,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot mix year-month and day-time fields: interval 4 month 2 weeks 3 microseconds(line 1, pos 7)
-
-== SQL ==
-select interval 4 month 2 weeks 3 microseconds * 1.5
--------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0029",
+ "messageParameters" : {
+ "literal" : "interval 4 month 2 weeks 3 microseconds"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 46,
+ "fragment" : "interval 4 month 2 weeks 3 microseconds"
+ } ]
+}
-- !query
@@ -122,10 +126,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value 'a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select interval 2 second * 'a'
- ^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'a'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"DOUBLE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "interval 2 second * 'a'"
+ } ]
+}
-- !query
@@ -134,10 +151,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value 'a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select interval 2 second / 'a'
- ^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'a'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"DOUBLE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "interval 2 second / 'a'"
+ } ]
+}
-- !query
@@ -146,10 +176,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value 'a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select interval 2 year * 'a'
- ^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'a'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"DOUBLE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "interval 2 year * 'a'"
+ } ]
+}
-- !query
@@ -158,10 +201,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value 'a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select interval 2 year / 'a'
- ^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'a'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"DOUBLE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "interval 2 year / 'a'"
+ } ]
+}
-- !query
@@ -186,10 +242,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value 'a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select 'a' * interval 2 second
- ^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'a'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"DOUBLE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "'a' * interval 2 second"
+ } ]
+}
-- !query
@@ -198,10 +267,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value 'a' of the type "STRING" cannot be cast to "DOUBLE" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select 'a' * interval 2 year
- ^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'a'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"DOUBLE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "'a' * interval 2 year"
+ } ]
+}
-- !query
@@ -210,7 +292,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '('2' / INTERVAL '02' SECOND)' due to data type mismatch: differing types in '('2' / INTERVAL '02' SECOND)' (string and interval second).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"STRING\"",
+ "right" : "\"INTERVAL SECOND\"",
+ "sqlExpr" : "\"(2 / INTERVAL '02' SECOND)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "'2' / interval 2 second"
+ } ]
+}
-- !query
@@ -219,7 +316,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '('2' / INTERVAL '2' YEAR)' due to data type mismatch: differing types in '('2' / INTERVAL '2' YEAR)' (string and interval year).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"STRING\"",
+ "right" : "\"INTERVAL YEAR\"",
+ "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "'2' / interval 2 year"
+ } ]
+}
-- !query
@@ -228,10 +340,17 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error.
-== SQL(line 1, position 8) ==
-select interval '2 seconds' / 0
- ^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "INTERVAL_DIVIDED_BY_ZERO",
+ "sqlState" : "22012",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 31,
+ "fragment" : "interval '2 seconds' / 0"
+ } ]
+}
-- !query
@@ -264,10 +383,17 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error.
-== SQL(line 1, position 8) ==
-select interval '2' year / 0
- ^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "INTERVAL_DIVIDED_BY_ZERO",
+ "sqlState" : "22012",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "interval '2' year / 0"
+ } ]
+}
-- !query
@@ -300,7 +426,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(2 / INTERVAL '2' YEAR)' due to data type mismatch: differing types in '(2 / INTERVAL '2' YEAR)' (int and interval year).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INT\"",
+ "right" : "\"INTERVAL YEAR\"",
+ "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "2 / interval '2' year"
+ } ]
+}
-- !query
@@ -309,7 +450,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(2 / INTERVAL '02' HOUR)' due to data type mismatch: differing types in '(2 / INTERVAL '02' HOUR)' (int and interval hour).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INT\"",
+ "right" : "\"INTERVAL HOUR\"",
+ "sqlExpr" : "\"(2 / INTERVAL '02' HOUR)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "2 / interval '2' hour"
+ } ]
+}
-- !query
@@ -318,7 +474,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(NULL / INTERVAL '2' YEAR)' due to data type mismatch: differing types in '(NULL / INTERVAL '2' YEAR)' (void and interval year).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"VOID\"",
+ "right" : "\"INTERVAL YEAR\"",
+ "sqlExpr" : "\"(NULL / INTERVAL '2' YEAR)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 31,
+ "fragment" : "null / interval '2' year"
+ } ]
+}
-- !query
@@ -327,7 +498,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(NULL / INTERVAL '02' HOUR)' due to data type mismatch: differing types in '(NULL / INTERVAL '02' HOUR)' (void and interval hour).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"VOID\"",
+ "right" : "\"INTERVAL HOUR\"",
+ "sqlExpr" : "\"(NULL / INTERVAL '02' HOUR)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 31,
+ "fragment" : "null / interval '2' hour"
+ } ]
+}
-- !query
@@ -336,12 +522,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot mix year-month and day-time fields: interval '-1 month 1 day -1 second'(line 1, pos 8)
-
-== SQL ==
-select -interval '-1 month 1 day -1 second'
---------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0029",
+ "messageParameters" : {
+ "literal" : "interval '-1 month 1 day -1 second'"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 9,
+ "stopIndex" : 43,
+ "fragment" : "interval '-1 month 1 day -1 second'"
+ } ]
+}
-- !query
@@ -366,12 +559,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot mix year-month and day-time fields: interval -1 month 1 day -1 second(line 1, pos 8)
-
-== SQL ==
-select -interval -1 month 1 day -1 second
---------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0029",
+ "messageParameters" : {
+ "literal" : "interval -1 month 1 day -1 second"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 9,
+ "stopIndex" : 41,
+ "fragment" : "interval -1 month 1 day -1 second"
+ } ]
+}
-- !query
@@ -396,12 +596,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot mix year-month and day-time fields: interval '-1 month 1 day -1 second'(line 1, pos 8)
-
-== SQL ==
-select +interval '-1 month 1 day -1 second'
---------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0029",
+ "messageParameters" : {
+ "literal" : "interval '-1 month 1 day -1 second'"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 9,
+ "stopIndex" : 43,
+ "fragment" : "interval '-1 month 1 day -1 second'"
+ } ]
+}
-- !query
@@ -426,12 +633,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot mix year-month and day-time fields: interval -1 month 1 day -1 second(line 1, pos 8)
-
-== SQL ==
-select +interval -1 month 1 day -1 second
---------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0029",
+ "messageParameters" : {
+ "literal" : "interval -1 month 1 day -1 second"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 9,
+ "stopIndex" : 41,
+ "fragment" : "interval -1 month 1 day -1 second"
+ } ]
+}
-- !query
@@ -664,10 +878,23 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-Decimal(expanded, 1234567890123456789, 20, 0) cannot be represented as Decimal(18, 6). If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789)
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "precision" : "18",
+ "scale" : "6",
+ "value" : "1234567890123456789"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 59,
+ "fragment" : "make_interval(0, 0, 0, 0, 0, 0, 1234567890123456789)"
+ } ]
+}
-- !query
@@ -847,12 +1074,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot mix year-month and day-time fields: interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond(line 1, pos 7)
-
-== SQL ==
-select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond
--------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0029",
+ "messageParameters" : {
+ "literal" : "interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 97,
+ "fragment" : "interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond"
+ } ]
+}
-- !query
@@ -885,12 +1119,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot mix year-month and day-time fields: interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second(line 1, pos 7)
-
-== SQL ==
-select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second
--------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0029",
+ "messageParameters" : {
+ "literal" : "interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 92,
+ "fragment" : "interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second"
+ } ]
+}
-- !query
@@ -1019,12 +1260,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Interval string does not match day-time format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval day to hour: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16)
-
-== SQL ==
-select interval '20 15:40:32.99899999' day to hour
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "Interval string does not match day-time format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval day to hour: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 50,
+ "fragment" : "'20 15:40:32.99899999' day to hour"
+ } ]
+}
-- !query
@@ -1033,12 +1281,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Interval string does not match day-time format of `[+|-]d h:m`, `INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE` when cast to interval day to minute: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16)
-
-== SQL ==
-select interval '20 15:40:32.99899999' day to minute
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "Interval string does not match day-time format of `[+|-]d h:m`, `INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE` when cast to interval day to minute: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 52,
+ "fragment" : "'20 15:40:32.99899999' day to minute"
+ } ]
+}
-- !query
@@ -1047,12 +1302,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Interval string does not match day-time format of `[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE` when cast to interval hour to minute: 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16)
-
-== SQL ==
-select interval '15:40:32.99899999' hour to minute
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "Interval string does not match day-time format of `[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE` when cast to interval hour to minute: 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 50,
+ "fragment" : "'15:40:32.99899999' hour to minute"
+ } ]
+}
-- !query
@@ -1061,12 +1323,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16)
-
-== SQL ==
-select interval '15:40.99899999' hour to second
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 47,
+ "fragment" : "'15:40.99899999' hour to second"
+ } ]
+}
-- !query
@@ -1075,12 +1344,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16)
-
-== SQL ==
-select interval '15:40' hour to second
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 38,
+ "fragment" : "'15:40' hour to second"
+ } ]
+}
-- !query
@@ -1089,12 +1365,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Interval string does not match day-time format of `[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND` when cast to interval minute to second: 20 40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16)
-
-== SQL ==
-select interval '20 40:32.99899999' minute to second
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "Interval string does not match day-time format of `[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND` when cast to interval minute to second: 20 40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 52,
+ "fragment" : "'20 40:32.99899999' minute to second"
+ } ]
+}
-- !query
@@ -1103,12 +1386,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Error parsing ' 10 nanoseconds' to interval, invalid unit 'nanoseconds'(line 1, pos 16)
-
-== SQL ==
-select interval 10 nanoseconds
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0062",
+ "messageParameters" : {
+ "msg" : "Error parsing ' 10 nanoseconds' to interval, invalid unit 'nanoseconds'"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 30,
+ "fragment" : "10 nanoseconds"
+ } ]
+}
-- !query
@@ -1197,12 +1487,14 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Error parsing ' 1 fake_unit' to interval, invalid unit 'fake_unit'(line 1, pos 16)
-
-== SQL ==
-select interval 1 fake_unit
-----------------^^^
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'fake_unit'",
+ "hint" : ""
+ }
+}
-- !query
@@ -1211,12 +1503,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-The value of from-to unit must be a string(line 1, pos 16)
-
-== SQL ==
-select interval 1 year to month
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0027",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 31,
+ "fragment" : "1 year to month"
+ } ]
+}
-- !query
@@ -1225,12 +1521,20 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Intervals FROM year TO second are not supported.(line 1, pos 16)
-
-== SQL ==
-select interval '1' year to second
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0028",
+ "messageParameters" : {
+ "from" : "year",
+ "to" : "second"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 34,
+ "fragment" : "'1' year to second"
+ } ]
+}
-- !query
@@ -1239,12 +1543,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Can only have a single from-to unit in the interval literal syntax(line 1, pos 37)
-
-== SQL ==
-select interval '10-9' year to month '2-1' year to month
--------------------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0024",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 56,
+ "fragment" : "interval '10-9' year to month '2-1' year to month"
+ } ]
+}
-- !query
@@ -1253,12 +1561,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Can only have a single from-to unit in the interval literal syntax(line 1, pos 37)
-
-== SQL ==
-select interval '10-9' year to month '12:11:10' hour to second
--------------------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0024",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 62,
+ "fragment" : "interval '10-9' year to month '12:11:10' hour to second"
+ } ]
+}
-- !query
@@ -1267,12 +1579,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Can only have a single from-to unit in the interval literal syntax(line 1, pos 40)
-
-== SQL ==
-select interval '1 15:11' day to minute '12:11:10' hour to second
-----------------------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0024",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 65,
+ "fragment" : "interval '1 15:11' day to minute '12:11:10' hour to second"
+ } ]
+}
-- !query
@@ -1281,12 +1597,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Can only have a single from-to unit in the interval literal syntax(line 1, pos 23)
-
-== SQL ==
-select interval 1 year '2-1' year to month
------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0024",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 42,
+ "fragment" : "interval 1 year '2-1' year to month"
+ } ]
+}
-- !query
@@ -1295,12 +1615,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Can only have a single from-to unit in the interval literal syntax(line 1, pos 23)
-
-== SQL ==
-select interval 1 year '12:11:10' hour to second
------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0024",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 48,
+ "fragment" : "interval 1 year '12:11:10' hour to second"
+ } ]
+}
-- !query
@@ -1309,12 +1633,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Can only have a single from-to unit in the interval literal syntax(line 1, pos 37)
-
-== SQL ==
-select interval '10-9' year to month '1' year
--------------------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0024",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 45,
+ "fragment" : "interval '10-9' year to month '1' year"
+ } ]
+}
-- !query
@@ -1323,12 +1651,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Can only have a single from-to unit in the interval literal syntax(line 1, pos 42)
-
-== SQL ==
-select interval '12:11:10' hour to second '1' year
-------------------------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0024",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 50,
+ "fragment" : "interval '12:11:10' hour to second '1' year"
+ } ]
+}
-- !query
@@ -1337,7 +1669,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Undefined function: interval. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.interval.; line 1 pos 7
+{
+ "errorClass" : "UNRESOLVED_ROUTINE",
+ "sqlState" : "42883",
+ "messageParameters" : {
+ "routineName" : "`interval`",
+ "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 21,
+ "fragment" : "interval (-30)"
+ } ]
+}
-- !query
@@ -1346,7 +1692,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Undefined function: interval. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.interval.; line 1 pos 7
+{
+ "errorClass" : "UNRESOLVED_ROUTINE",
+ "sqlState" : "42883",
+ "messageParameters" : {
+ "routineName" : "`interval`",
+ "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 23,
+ "fragment" : "interval (a + 1)"
+ } ]
+}
-- !query
@@ -1355,12 +1715,14 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Syntax error at or near 'day': extra input 'day'(line 1, pos 27)
-
-== SQL ==
-select interval 30 day day day
----------------------------^^^
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'day'",
+ "hint" : ": extra input 'day'"
+ }
+}
-- !query
@@ -1369,7 +1731,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Undefined function: interval. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.interval.; line 1 pos 7
+{
+ "errorClass" : "UNRESOLVED_ROUTINE",
+ "sqlState" : "42883",
+ "messageParameters" : {
+ "routineName" : "`interval`",
+ "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 21,
+ "fragment" : "interval (-30)"
+ } ]
+}
-- !query
@@ -1378,7 +1754,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Undefined function: interval. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.interval.; line 1 pos 7
+{
+ "errorClass" : "UNRESOLVED_ROUTINE",
+ "sqlState" : "42883",
+ "messageParameters" : {
+ "routineName" : "`interval`",
+ "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 23,
+ "fragment" : "interval (a + 1)"
+ } ]
+}
-- !query
@@ -1387,12 +1777,14 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Syntax error at or near 'days': extra input 'days'(line 1, pos 29)
-
-== SQL ==
-select interval 30 days days days
------------------------------^^^
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'days'",
+ "hint" : ": extra input 'days'"
+ }
+}
-- !query
@@ -1409,12 +1801,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Error parsing interval year-month string: integer overflow(line 1, pos 16)
-
-== SQL ==
-SELECT INTERVAL '178956970-8' YEAR TO MONTH
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "Error parsing interval year-month string: integer overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 43,
+ "fragment" : "'178956970-8' YEAR TO MONTH"
+ } ]
+}
-- !query
@@ -1459,7 +1858,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '2' YEAR + '3-3 year to month')' due to data type mismatch: differing types in '(INTERVAL '2' YEAR + '3-3 year to month')' (interval year and string).; line 2 pos 2
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL YEAR\"",
+ "right" : "\"STRING\"",
+ "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3 year to month)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 10,
+ "stopIndex" : 48,
+ "fragment" : "interval '2' year + '3-3 year to month'"
+ } ]
+}
-- !query
@@ -1484,7 +1898,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '2' YEAR + '3-3')' due to data type mismatch: differing types in '(INTERVAL '2' YEAR + '3-3')' (interval year and string).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL YEAR\"",
+ "right" : "\"STRING\"",
+ "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 32,
+ "fragment" : "interval '2' year + '3-3'"
+ } ]
+}
-- !query
@@ -1493,7 +1922,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '2' YEAR - '4')' due to data type mismatch: differing types in '(INTERVAL '2' YEAR - '4')' (interval year and string).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL YEAR\"",
+ "right" : "\"STRING\"",
+ "sqlExpr" : "\"(INTERVAL '2' YEAR - 4)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "interval '2' year - '4'"
+ } ]
+}
-- !query
@@ -1502,10 +1946,23 @@
struct<>
-- !query output
org.apache.spark.SparkDateTimeException
-The value '4 11:11' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select '4 11:11' - interval '4 22:12' day to minute
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'4 11:11'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 51,
+ "fragment" : "'4 11:11' - interval '4 22:12' day to minute"
+ } ]
+}
-- !query
@@ -1514,10 +1971,23 @@
struct<>
-- !query output
org.apache.spark.SparkDateTimeException
-The value '4 12:12:12' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select '4 12:12:12' + interval '4 22:12' day to minute
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'4 12:12:12'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 54,
+ "fragment" : "'4 12:12:12' + interval '4 22:12' day to minute"
+ } ]
+}
-- !query
@@ -1534,7 +2004,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '2' YEAR + interval_view.str)' due to data type mismatch: differing types in '(INTERVAL '2' YEAR + interval_view.str)' (interval year and string).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL YEAR\"",
+ "right" : "\"STRING\"",
+ "sqlExpr" : "\"(INTERVAL '2' YEAR + str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "interval '2' year + str"
+ } ]
+}
-- !query
@@ -1543,7 +2028,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '2' YEAR - interval_view.str)' due to data type mismatch: differing types in '(INTERVAL '2' YEAR - interval_view.str)' (interval year and string).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL YEAR\"",
+ "right" : "\"STRING\"",
+ "sqlExpr" : "\"(INTERVAL '2' YEAR - str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "interval '2' year - str"
+ } ]
+}
-- !query
@@ -1552,10 +2052,23 @@
struct<>
-- !query output
org.apache.spark.SparkDateTimeException
-The value '1' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select str - interval '4 22:12' day to minute from interval_view
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'1'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 45,
+ "fragment" : "str - interval '4 22:12' day to minute"
+ } ]
+}
-- !query
@@ -1564,10 +2077,23 @@
struct<>
-- !query output
org.apache.spark.SparkDateTimeException
-The value '1' of the type "STRING" cannot be cast to "TIMESTAMP" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select str + interval '4 22:12' day to minute from interval_view
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'1'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 45,
+ "fragment" : "str + interval '4 22:12' day to minute"
+ } ]
+}
-- !query
@@ -1576,7 +2102,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' DAY' due to data type mismatch: argument 1 requires (timestamp or timestamp without time zone) type, however, 'INTERVAL '2-2' YEAR TO MONTH' is of interval year to month type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"",
+ "inputType" : "\"INTERVAL YEAR TO MONTH\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' DAY\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 54,
+ "fragment" : "interval '2-2' year to month + interval '3' day"
+ } ]
+}
-- !query
@@ -1585,7 +2128,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' DAY' due to data type mismatch: argument 1 requires (timestamp or timestamp without time zone) type, however, 'INTERVAL '2-2' YEAR TO MONTH' is of interval year to month type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"",
+ "inputType" : "\"INTERVAL YEAR TO MONTH\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' DAY\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 54,
+ "fragment" : "interval '3' day + interval '2-2' year to month"
+ } ]
+}
-- !query
@@ -1594,7 +2154,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'INTERVAL '2-2' YEAR TO MONTH + (- INTERVAL '3' DAY)' due to data type mismatch: argument 1 requires (timestamp or timestamp without time zone) type, however, 'INTERVAL '2-2' YEAR TO MONTH' is of interval year to month type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"",
+ "inputType" : "\"INTERVAL YEAR TO MONTH\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + (- INTERVAL '3' DAY)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 54,
+ "fragment" : "interval '2-2' year to month - interval '3' day"
+ } ]
+}
-- !query
@@ -1603,7 +2180,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '3' DAY - INTERVAL '2-2' YEAR TO MONTH)' due to data type mismatch: differing types in '(INTERVAL '3' DAY - INTERVAL '2-2' YEAR TO MONTH)' (interval day and interval year to month).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL DAY\"",
+ "right" : "\"INTERVAL YEAR TO MONTH\"",
+ "sqlExpr" : "\"(INTERVAL '3' DAY - INTERVAL '2-2' YEAR TO MONTH)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 54,
+ "fragment" : "interval '3' day - interval '2-2' year to month"
+ } ]
+}
-- !query
@@ -1612,7 +2204,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '1 + (- INTERVAL '02' SECOND)' due to data type mismatch: argument 1 requires (timestamp or timestamp without time zone) type, however, '1' is of int type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"1 + (- INTERVAL '02' SECOND)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "1 - interval '2' second"
+ } ]
+}
-- !query
@@ -1621,7 +2230,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(1 + INTERVAL '2' MONTH)' due to data type mismatch: differing types in '(1 + INTERVAL '2' MONTH)' (int and interval month).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INT\"",
+ "right" : "\"INTERVAL MONTH\"",
+ "sqlExpr" : "\"(1 + INTERVAL '2' MONTH)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "1 + interval '2' month"
+ } ]
+}
-- !query
@@ -1630,7 +2254,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '1 + INTERVAL '02' SECOND' due to data type mismatch: argument 1 requires (timestamp or timestamp without time zone) type, however, '1' is of int type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"1 + INTERVAL '02' SECOND\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "interval '2' second + 1"
+ } ]
+}
-- !query
@@ -1639,7 +2280,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '2' MONTH - 1)' due to data type mismatch: differing types in '(INTERVAL '2' MONTH - 1)' (interval month and int).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL MONTH\"",
+ "right" : "\"INT\"",
+ "sqlExpr" : "\"(INTERVAL '2' MONTH - 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "interval '2' month - 1"
+ } ]
+}
-- !query
@@ -1696,12 +2352,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Interval string does not match year-month format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval year to month: - 2-2 (line 1, pos 16)
-
-== SQL ==
-select interval '-\t2-2\t' year to month
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "Interval string does not match year-month format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval year to month: -\t2-2\t"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 40,
+ "fragment" : "'-\\t2-2\\t' year to month"
+ } ]
+}
-- !query
@@ -1718,13 +2381,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Interval string does not match day-time format of `[+|-]d h:m:s.n`, `INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND` when cast to interval day to second:
-- 10 12:34:46.789 , set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16)
-
-== SQL ==
-select interval '\n-\t10\t 12:34:46.789\t' day to second
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "Interval string does not match day-time format of `[+|-]d h:m:s.n`, `INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND` when cast to interval day to second: \n-\t10\t 12:34:46.789\t, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 56,
+ "fragment" : "'\\n-\\t10\\t 12:34:46.789\\t' day to second"
+ } ]
+}
-- !query
@@ -1733,12 +2402,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: 中文 interval 1 day(line 1, pos 7)
-
-== SQL ==
-select interval '中文 interval 1 day'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'中文 interval 1 day'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 35,
+ "fragment" : "interval '中文 interval 1 day'"
+ } ]
+}
-- !query
@@ -1747,12 +2425,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: interval中文 1 day(line 1, pos 7)
-
-== SQL ==
-select interval 'interval中文 1 day'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'interval中文 1 day'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "interval 'interval中文 1 day'"
+ } ]
+}
-- !query
@@ -1761,12 +2448,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: interval 1中文day(line 1, pos 7)
-
-== SQL ==
-select interval 'interval 1中文day'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'interval 1中文day'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 33,
+ "fragment" : "interval 'interval 1中文day'"
+ } ]
+}
-- !query
@@ -1775,7 +2471,14 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error.
+{
+ "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW",
+ "sqlState" : "22015",
+ "messageParameters" : {
+ "alternative" : "",
+ "message" : "integer overflow"
+ }
+}
-- !query
@@ -1784,7 +2487,14 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error.
+{
+ "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW",
+ "sqlState" : "22015",
+ "messageParameters" : {
+ "alternative" : " Use 'try_subtract' to tolerate overflow and return NULL instead.",
+ "message" : "integer overflow"
+ }
+}
-- !query
@@ -1793,7 +2503,14 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error.
+{
+ "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW",
+ "sqlState" : "22015",
+ "messageParameters" : {
+ "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.",
+ "message" : "integer overflow"
+ }
+}
-- !query
@@ -1872,12 +2589,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: +(line 1, pos 7)
-
-== SQL ==
-select interval '+'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'+'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 19,
+ "fragment" : "interval '+'"
+ } ]
+}
-- !query
@@ -1886,12 +2612,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: +.(line 1, pos 7)
-
-== SQL ==
-select interval '+.'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'+.'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 20,
+ "fragment" : "interval '+.'"
+ } ]
+}
-- !query
@@ -1900,12 +2635,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: 1(line 1, pos 7)
-
-== SQL ==
-select interval '1'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'1'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 19,
+ "fragment" : "interval '1'"
+ } ]
+}
-- !query
@@ -1914,12 +2658,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: 1.2(line 1, pos 7)
-
-== SQL ==
-select interval '1.2'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'1.2'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 21,
+ "fragment" : "interval '1.2'"
+ } ]
+}
-- !query
@@ -1928,12 +2681,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: - 2(line 1, pos 7)
-
-== SQL ==
-select interval '- 2'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'- 2'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 21,
+ "fragment" : "interval '- 2'"
+ } ]
+}
-- !query
@@ -1942,12 +2704,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: 1 day -(line 1, pos 7)
-
-== SQL ==
-select interval '1 day -'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'1 day -'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 25,
+ "fragment" : "interval '1 day -'"
+ } ]
+}
-- !query
@@ -1956,12 +2727,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: 1 day 1(line 1, pos 7)
-
-== SQL ==
-select interval '1 day 1'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'1 day 1'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 25,
+ "fragment" : "interval '1 day 1'"
+ } ]
+}
-- !query
@@ -1970,12 +2750,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Can only use numbers in the interval value part for multiple unit value pairs interval form, but got invalid value: 1 day 2(line 1, pos 16)
-
-== SQL ==
-select interval '1 day 2' day
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0026",
+ "messageParameters" : {
+ "value" : "1 day 2"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 29,
+ "fragment" : "'1 day 2' day"
+ } ]
+}
-- !query
@@ -1984,12 +2771,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Can only use numbers in the interval value part for multiple unit value pairs interval form, but got invalid value: interval 1(line 1, pos 16)
-
-== SQL ==
-select interval 'interval 1' day
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0026",
+ "messageParameters" : {
+ "value" : "interval 1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 32,
+ "fragment" : "'interval 1' day"
+ } ]
+}
-- !query
@@ -2022,10 +2816,21 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW",
+ "sqlState" : "22015",
+ "messageParameters" : {
+ "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.",
+ "message" : "Interval value overflows after being divided by -1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 51,
+ "fragment" : "(INTERVAL '-178956970-8' YEAR TO MONTH) / -1"
+ } ]
+}
-- !query
@@ -2034,10 +2839,21 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1L
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW",
+ "sqlState" : "22015",
+ "messageParameters" : {
+ "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.",
+ "message" : "Interval value overflows after being divided by -1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 52,
+ "fragment" : "(INTERVAL '-178956970-8' YEAR TO MONTH) / -1L"
+ } ]
+}
-- !query
@@ -2080,10 +2896,21 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW",
+ "sqlState" : "22015",
+ "messageParameters" : {
+ "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.",
+ "message" : "Interval value overflows after being divided by -1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 65,
+ "fragment" : "(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1"
+ } ]
+}
-- !query
@@ -2092,10 +2919,21 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1L
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW",
+ "sqlState" : "22015",
+ "messageParameters" : {
+ "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.",
+ "message" : "Interval value overflows after being divided by -1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 66,
+ "fragment" : "(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1L"
+ } ]
+}
-- !query
@@ -2218,12 +3056,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-requirement failed: day 106751992 outside range [0, 106751991](line 1, pos 16)
-
-== SQL ==
-SELECT INTERVAL '106751992 04' DAY TO HOUR
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "requirement failed: day 106751992 outside range [0, 106751991]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 42,
+ "fragment" : "'106751992 04' DAY TO HOUR"
+ } ]
+}
-- !query
@@ -2232,12 +3077,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-requirement failed: day 106751992 outside range [0, 106751991](line 1, pos 16)
-
-== SQL ==
-SELECT INTERVAL '-106751992 04' DAY TO HOUR
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "requirement failed: day 106751992 outside range [0, 106751991]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 43,
+ "fragment" : "'-106751992 04' DAY TO HOUR"
+ } ]
+}
-- !query
@@ -2246,12 +3098,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-requirement failed: hour 2562047789 outside range [0, 2562047788](line 1, pos 16)
-
-== SQL ==
-SELECT INTERVAL '2562047789:00' HOUR TO MINUTE
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "requirement failed: hour 2562047789 outside range [0, 2562047788]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 46,
+ "fragment" : "'2562047789:00' HOUR TO MINUTE"
+ } ]
+}
-- !query
@@ -2260,12 +3119,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-requirement failed: hour 2562047789 outside range [0, 2562047788](line 1, pos 16)
-
-== SQL ==
-SELECT INTERVAL '-2562047789:00' HOUR TO MINUTE
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "requirement failed: hour 2562047789 outside range [0, 2562047788]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 47,
+ "fragment" : "'-2562047789:00' HOUR TO MINUTE"
+ } ]
+}
-- !query
@@ -2274,12 +3140,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-requirement failed: minute 153722867281 outside range [0, 153722867280](line 1, pos 16)
-
-== SQL ==
-SELECT INTERVAL '153722867281:54.775808' MINUTE TO SECOND
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "requirement failed: minute 153722867281 outside range [0, 153722867280]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 57,
+ "fragment" : "'153722867281:54.775808' MINUTE TO SECOND"
+ } ]
+}
-- !query
@@ -2288,12 +3161,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-requirement failed: minute 153722867281 outside range [0, 153722867280](line 1, pos 16)
-
-== SQL ==
-SELECT INTERVAL '-153722867281:54.775808' MINUTE TO SECOND
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "requirement failed: minute 153722867281 outside range [0, 153722867280]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 58,
+ "fragment" : "'-153722867281:54.775808' MINUTE TO SECOND"
+ } ]
+}
-- !query
@@ -2430,7 +3310,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '1' MONTH > INTERVAL '20' DAY)' due to data type mismatch: differing types in '(INTERVAL '1' MONTH > INTERVAL '20' DAY)' (interval month and interval day).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL MONTH\"",
+ "right" : "\"INTERVAL DAY\"",
+ "sqlExpr" : "\"(INTERVAL '1' MONTH > INTERVAL '20' DAY)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 42,
+ "fragment" : "INTERVAL 1 MONTH > INTERVAL 20 DAYS"
+ } ]
+}
-- !query
@@ -2439,7 +3334,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '1' DAY < '1')' due to data type mismatch: differing types in '(INTERVAL '1' DAY < '1')' (interval day and string).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL DAY\"",
+ "right" : "\"STRING\"",
+ "sqlExpr" : "\"(INTERVAL '1' DAY < 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "INTERVAL '1' DAY < '1'"
+ } ]
+}
-- !query
@@ -2448,7 +3358,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '1' DAY = '1')' due to data type mismatch: differing types in '(INTERVAL '1' DAY = '1')' (interval day and string).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL DAY\"",
+ "right" : "\"STRING\"",
+ "sqlExpr" : "\"(INTERVAL '1' DAY = 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "INTERVAL '1' DAY = '1'"
+ } ]
+}
-- !query
@@ -2457,7 +3382,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '1' DAY > '1')' due to data type mismatch: differing types in '(INTERVAL '1' DAY > '1')' (interval day and string).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL DAY\"",
+ "right" : "\"STRING\"",
+ "sqlExpr" : "\"(INTERVAL '1' DAY > 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "INTERVAL '1' DAY > '1'"
+ } ]
+}
-- !query
@@ -2466,7 +3406,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '('1' < INTERVAL '1' DAY)' due to data type mismatch: differing types in '('1' < INTERVAL '1' DAY)' (string and interval day).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"STRING\"",
+ "right" : "\"INTERVAL DAY\"",
+ "sqlExpr" : "\"(1 < INTERVAL '1' DAY)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "'1' < INTERVAL '1' DAY"
+ } ]
+}
-- !query
@@ -2475,7 +3430,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '('1' = INTERVAL '1' DAY)' due to data type mismatch: differing types in '('1' = INTERVAL '1' DAY)' (string and interval day).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"STRING\"",
+ "right" : "\"INTERVAL DAY\"",
+ "sqlExpr" : "\"(1 = INTERVAL '1' DAY)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "'1' = INTERVAL '1' DAY"
+ } ]
+}
-- !query
@@ -2484,7 +3454,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '('1' > INTERVAL '1' DAY)' due to data type mismatch: differing types in '('1' > INTERVAL '1' DAY)' (string and interval day).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"STRING\"",
+ "right" : "\"INTERVAL DAY\"",
+ "sqlExpr" : "\"(1 > INTERVAL '1' DAY)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "'1' > INTERVAL '1' DAY"
+ } ]
+}
-- !query
@@ -2493,7 +3478,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '1' YEAR < '1')' due to data type mismatch: differing types in '(INTERVAL '1' YEAR < '1')' (interval year and string).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL YEAR\"",
+ "right" : "\"STRING\"",
+ "sqlExpr" : "\"(INTERVAL '1' YEAR < 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "INTERVAL '1' YEAR < '1'"
+ } ]
+}
-- !query
@@ -2502,7 +3502,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '1' YEAR = '1')' due to data type mismatch: differing types in '(INTERVAL '1' YEAR = '1')' (interval year and string).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL YEAR\"",
+ "right" : "\"STRING\"",
+ "sqlExpr" : "\"(INTERVAL '1' YEAR = 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "INTERVAL '1' YEAR = '1'"
+ } ]
+}
-- !query
@@ -2511,7 +3526,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '1' YEAR > '1')' due to data type mismatch: differing types in '(INTERVAL '1' YEAR > '1')' (interval year and string).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL YEAR\"",
+ "right" : "\"STRING\"",
+ "sqlExpr" : "\"(INTERVAL '1' YEAR > 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "INTERVAL '1' YEAR > '1'"
+ } ]
+}
-- !query
@@ -2520,7 +3550,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '('1' < INTERVAL '1' YEAR)' due to data type mismatch: differing types in '('1' < INTERVAL '1' YEAR)' (string and interval year).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"STRING\"",
+ "right" : "\"INTERVAL YEAR\"",
+ "sqlExpr" : "\"(1 < INTERVAL '1' YEAR)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "'1' < INTERVAL '1' YEAR"
+ } ]
+}
-- !query
@@ -2529,7 +3574,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '('1' = INTERVAL '1' YEAR)' due to data type mismatch: differing types in '('1' = INTERVAL '1' YEAR)' (string and interval year).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"STRING\"",
+ "right" : "\"INTERVAL YEAR\"",
+ "sqlExpr" : "\"(1 = INTERVAL '1' YEAR)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "'1' = INTERVAL '1' YEAR"
+ } ]
+}
-- !query
@@ -2538,7 +3598,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '('1' > INTERVAL '1' YEAR)' due to data type mismatch: differing types in '('1' > INTERVAL '1' YEAR)' (string and interval year).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"STRING\"",
+ "right" : "\"INTERVAL YEAR\"",
+ "sqlExpr" : "\"(1 > INTERVAL '1' YEAR)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "'1' > INTERVAL '1' YEAR"
+ } ]
+}
-- !query
@@ -2563,7 +3638,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'array(INTERVAL '1' MONTH, INTERVAL '20' DAY)' due to data type mismatch: input to function array should all be the same type, but it's [interval month, interval day]; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "dataType" : "(\"INTERVAL MONTH\" or \"INTERVAL DAY\")",
+ "functionName" : "`array`",
+ "sqlExpr" : "\"array(INTERVAL '1' MONTH, INTERVAL '20' DAY)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 48,
+ "fragment" : "array(INTERVAL 1 MONTH, INTERVAL 20 DAYS)"
+ } ]
+}
-- !query
@@ -2588,7 +3678,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'coalesce(INTERVAL '1' MONTH, INTERVAL '20' DAY)' due to data type mismatch: input to function coalesce should all be the same type, but it's [interval month, interval day]; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "dataType" : "(\"INTERVAL MONTH\" or \"INTERVAL DAY\")",
+ "functionName" : "`coalesce`",
+ "sqlExpr" : "\"coalesce(INTERVAL '1' MONTH, INTERVAL '20' DAY)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 51,
+ "fragment" : "coalesce(INTERVAL 1 MONTH, INTERVAL 20 DAYS)"
+ } ]
+}
-- !query
@@ -2645,7 +3750,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '1' MONTH div INTERVAL '-1' DAY)' due to data type mismatch: differing types in '(INTERVAL '1' MONTH div INTERVAL '-1' DAY)' (interval month and interval day).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL MONTH\"",
+ "right" : "\"INTERVAL DAY\"",
+ "sqlExpr" : "\"(INTERVAL '1' MONTH div INTERVAL '-1' DAY)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 49,
+ "fragment" : "div(INTERVAL '1' MONTH, INTERVAL '-1' DAY)"
+ } ]
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out 2023-09-18 15:00:43.006708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 54
-
-
-- !query
select null, Null, nUll
-- !query schema
@@ -40,12 +37,22 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Numeric literal 128 does not fit in range [-128, 127] for type tinyint(line 1, pos 7)
-
-== SQL ==
-select 128Y
--------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0023",
+ "messageParameters" : {
+ "maxValue" : "127",
+ "minValue" : "-128",
+ "rawStrippedQualifier" : "128",
+ "typeName" : "tinyint"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 11,
+ "fragment" : "128Y"
+ } ]
+}
-- !query
@@ -70,12 +77,22 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Numeric literal 32768 does not fit in range [-32768, 32767] for type smallint(line 1, pos 7)
-
-== SQL ==
-select 32768S
--------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0023",
+ "messageParameters" : {
+ "maxValue" : "32767",
+ "minValue" : "-32768",
+ "rawStrippedQualifier" : "32768",
+ "typeName" : "smallint"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 13,
+ "fragment" : "32768S"
+ } ]
+}
-- !query
@@ -100,12 +117,22 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Numeric literal 9223372036854775808 does not fit in range [-9223372036854775808, 9223372036854775807] for type bigint(line 1, pos 7)
-
-== SQL ==
-select 9223372036854775808L
--------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0023",
+ "messageParameters" : {
+ "maxValue" : "9223372036854775807",
+ "minValue" : "-9223372036854775808",
+ "rawStrippedQualifier" : "9223372036854775808",
+ "typeName" : "bigint"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 27,
+ "fragment" : "9223372036854775808L"
+ } ]
+}
-- !query
@@ -145,11 +172,15 @@
-- !query schema
struct<>
-- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-
-decimal can only support precision up to 38
-== SQL ==
-select 1234567890123456789012345678901234567890
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "maxPrecision" : "38",
+ "precision" : "40"
+ }
+}
-- !query
@@ -157,11 +188,15 @@
-- !query schema
struct<>
-- !query output
-org.apache.spark.sql.catalyst.parser.ParseException
-
-decimal can only support precision up to 38
-== SQL ==
-select 1234567890123456789012345678901234567890.0
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "maxPrecision" : "38",
+ "precision" : "41"
+ }
+}
-- !query
@@ -186,12 +221,22 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Numeric literal -3.4028235E39 does not fit in range [-3.4028234663852886E+38, 3.4028234663852886E+38] for type float(line 1, pos 7)
-
-== SQL ==
-select -3.4028235E39f
--------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0023",
+ "messageParameters" : {
+ "maxValue" : "3.4028234663852886E+38",
+ "minValue" : "-3.4028234663852886E+38",
+ "rawStrippedQualifier" : "-3.4028235E39",
+ "typeName" : "float"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 21,
+ "fragment" : "-3.4028235E39f"
+ } ]
+}
-- !query
@@ -216,12 +261,14 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Syntax error at or near '.'(line 1, pos 7)
-
-== SQL ==
-select .e3
--------^^^
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'.'",
+ "hint" : ""
+ }
+}
-- !query
@@ -230,12 +277,22 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Numeric literal 1E309 does not fit in range [-1.7976931348623157E+308, 1.7976931348623157E+308] for type double(line 1, pos 7)
-
-== SQL ==
-select 1E309, -1E309
--------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0023",
+ "messageParameters" : {
+ "maxValue" : "1.7976931348623157E+308",
+ "minValue" : "-1.7976931348623157E+308",
+ "rawStrippedQualifier" : "1E309",
+ "typeName" : "double"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 12,
+ "fragment" : "1E309"
+ } ]
+}
-- !query
@@ -334,12 +391,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the DATE value: mar 11 2016(line 1, pos 7)
-
-== SQL ==
-select date 'mar 11 2016'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'mar 11 2016'",
+ "valueType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 25,
+ "fragment" : "date 'mar 11 2016'"
+ } ]
+}
-- !query
@@ -356,12 +422,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the TIMESTAMP value: 2016-33-11 20:54:00.000(line 1, pos 7)
-
-== SQL ==
-select timestamp '2016-33-11 20:54:00.000'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2016-33-11 20:54:00.000'",
+ "valueType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 42,
+ "fragment" : "timestamp '2016-33-11 20:54:00.000'"
+ } ]
+}
-- !query
@@ -370,12 +445,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Literals of type 'GEO' are currently not supported.(line 1, pos 7)
-
-== SQL ==
-select GEO '(10,-6)'
--------^^^
+{
+ "errorClass" : "UNSUPPORTED_TYPED_LITERAL",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "supportedTypes" : "\"DATE\", \"TIMESTAMP_NTZ\", \"TIMESTAMP_LTZ\", \"TIMESTAMP\", \"INTERVAL\", \"X\"",
+ "unsupportedType" : "\"GEO\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 20,
+ "fragment" : "GEO '(10,-6)'"
+ } ]
+}
-- !query
@@ -392,12 +476,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-decimal can only support precision up to 38(line 1, pos 7)
-
-== SQL ==
-select 1.20E-38BD
--------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0061",
+ "messageParameters" : {
+ "msg" : "[DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION] Decimal precision 40 exceeds max precision 38."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 17,
+ "fragment" : "1.20E-38BD"
+ } ]
+}
-- !query
@@ -414,12 +505,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-contains illegal character for hexBinary: 0XuZ(line 1, pos 7)
-
-== SQL ==
-select X'XuZ'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'XuZ'",
+ "valueType" : "\"X\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 13,
+ "fragment" : "X'XuZ'"
+ } ]
+}
-- !query
@@ -436,7 +536,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(+ DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval day to second or interval year to month or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"DATE '1999-01-01'\"",
+ "inputType" : "\"DATE\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")",
+ "sqlExpr" : "\"(+ DATE '1999-01-01')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 25,
+ "fragment" : "+date '1999-01-01'"
+ } ]
+}
-- !query
@@ -445,7 +562,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(+ TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval day to second or interval year to month or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"TIMESTAMP '1999-01-01 00:00:00'\"",
+ "inputType" : "\"TIMESTAMP\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")",
+ "sqlExpr" : "\"(+ TIMESTAMP '1999-01-01 00:00:00')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "+timestamp '1999-01-01'"
+ } ]
+}
-- !query
@@ -462,7 +596,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(+ map(1, 2))' due to data type mismatch: argument 1 requires (numeric or interval day to second or interval year to month or interval) type, however, 'map(1, 2)' is of map<int,int> type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"map(1, 2)\"",
+ "inputType" : "\"MAP<INT, INT>\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")",
+ "sqlExpr" : "\"(+ map(1, 2))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 17,
+ "fragment" : "+map(1, 2)"
+ } ]
+}
-- !query
@@ -471,7 +622,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(+ array(1, 2))' due to data type mismatch: argument 1 requires (numeric or interval day to second or interval year to month or interval) type, however, 'array(1, 2)' is of array<int> type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"array(1, 2)\"",
+ "inputType" : "\"ARRAY<INT>\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")",
+ "sqlExpr" : "\"(+ array(1, 2))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 18,
+ "fragment" : "+array(1,2)"
+ } ]
+}
-- !query
@@ -480,7 +648,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(+ named_struct('a', 1, 'b', 'spark'))' due to data type mismatch: argument 1 requires (numeric or interval day to second or interval year to month or interval) type, however, 'named_struct('a', 1, 'b', 'spark')' is of struct<a:int,b:string> type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"named_struct(a, 1, b, spark)\"",
+ "inputType" : "\"STRUCT<a: INT, b: STRING>\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")",
+ "sqlExpr" : "\"(+ named_struct(a, 1, b, spark))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 42,
+ "fragment" : "+named_struct('a', 1, 'b', 'spark')"
+ } ]
+}
-- !query
@@ -489,7 +674,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(+ X'01')' due to data type mismatch: argument 1 requires (numeric or interval day to second or interval year to month or interval) type, however, 'X'01'' is of binary type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"X'01'\"",
+ "inputType" : "\"BINARY\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")",
+ "sqlExpr" : "\"(+ X'01')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 12,
+ "fragment" : "+X'1'"
+ } ]
+}
-- !query
@@ -498,7 +700,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(- DATE '1999-01-01')' due to data type mismatch: argument 1 requires (numeric or interval day to second or interval year to month or interval) type, however, 'DATE '1999-01-01'' is of date type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"DATE '1999-01-01'\"",
+ "inputType" : "\"DATE\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")",
+ "sqlExpr" : "\"(- DATE '1999-01-01')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 25,
+ "fragment" : "-date '1999-01-01'"
+ } ]
+}
-- !query
@@ -507,7 +726,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(- TIMESTAMP '1999-01-01 00:00:00')' due to data type mismatch: argument 1 requires (numeric or interval day to second or interval year to month or interval) type, however, 'TIMESTAMP '1999-01-01 00:00:00'' is of timestamp type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"TIMESTAMP '1999-01-01 00:00:00'\"",
+ "inputType" : "\"TIMESTAMP\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")",
+ "sqlExpr" : "\"(- TIMESTAMP '1999-01-01 00:00:00')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "-timestamp '1999-01-01'"
+ } ]
+}
-- !query
@@ -516,4 +752,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(- X'2379ACFE')' due to data type mismatch: argument 1 requires (numeric or interval day to second or interval year to month or interval) type, however, 'X'2379ACFE'' is of binary type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"X'2379ACFE'\"",
+ "inputType" : "\"BINARY\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")",
+ "sqlExpr" : "\"(- X'2379ACFE')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 19,
+ "fragment" : "-x'2379ACFe'"
+ } ]
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/map.sql.out sql/core/src/test/resources/sql-tests/results/ansi/map.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/map.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/map.sql.out 2023-09-18 15:00:43.006708418 +0800
@@ -1,29 +1,18 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 14
-
-
-- !query
select element_at(map(1, 'a', 2, 'b'), 5)
-- !query schema
-struct<>
+struct<element_at(map(1, a, 2, b), 5):string>
-- !query output
-org.apache.spark.SparkNoSuchElementException
-Key 5 does not exist. Use `try_element_at` to tolerate non-existent key and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select element_at(map(1, 'a', 2, 'b'), 5)
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+NULL
-- !query
select map(1, 'a', 2, 'b')[5]
-- !query schema
-struct<>
+struct<map(1, a, 2, b)[5]:string>
-- !query output
-org.apache.spark.SparkNoSuchElementException
-Key 5 does not exist. Use `try_element_at` to tolerate non-existent key and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select map(1, 'a', 2, 'b')[5]
- ^^^^^^^^^^^^^^^^^^^^^^
+NULL
-- !query
@@ -80,7 +69,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'map_contains_key(map('1', 'a', '2', 'b'), 1)' due to data type mismatch: Input to function map_contains_key should have been map followed by a value with same key type, but it's [map<string,string>, int].; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "dataType" : "\"MAP\"",
+ "functionName" : "`map_contains_key`",
+ "leftType" : "\"MAP<STRING, STRING>\"",
+ "rightType" : "\"INT\"",
+ "sqlExpr" : "\"map_contains_key(map(1, a, 2, b), 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 51,
+ "fragment" : "map_contains_key(map('1', 'a', '2', 'b'), 1)"
+ } ]
+}
-- !query
@@ -89,44 +95,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'map_contains_key(map(1, 'a', 2, 'b'), '1')' due to data type mismatch: Input to function map_contains_key should have been map followed by a value with same key type, but it's [map<int,string>, string].; line 1 pos 7
-
-
--- !query
-set spark.sql.ansi.strictIndexOperator=false
--- !query schema
-struct<key:string,value:string>
--- !query output
-spark.sql.ansi.strictIndexOperator false
-
-
--- !query
-select map(1, 'a', 2, 'b')[5]
--- !query schema
-struct<map(1, a, 2, b)[5]:string>
--- !query output
-NULL
-
-
--- !query
-select element_at(map(1, 'a', 2, 'b'), 5)
--- !query schema
-struct<>
--- !query output
-org.apache.spark.SparkNoSuchElementException
-Key 5 does not exist. Use `try_element_at` to tolerate non-existent key and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select element_at(map(1, 'a', 2, 'b'), 5)
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
-
-
--- !query
-select element_at(map('a', 1, 'b', 2), 'c')
--- !query schema
-struct<>
--- !query output
-org.apache.spark.SparkNoSuchElementException
-Key 'c' does not exist. Use `try_element_at` to tolerate non-existent key and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-select element_at(map('a', 1, 'b', 2), 'c')
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "DATATYPE_MISMATCH.MAP_FUNCTION_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "dataType" : "\"MAP\"",
+ "functionName" : "`map_contains_key`",
+ "leftType" : "\"MAP<INT, STRING>\"",
+ "rightType" : "\"STRING\"",
+ "sqlExpr" : "\"map_contains_key(map(1, a, 2, b), 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 49,
+ "fragment" : "map_contains_key(map(1, 'a', 2, 'b'), '1')"
+ } ]
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/math.sql.out sql/core/src/test/resources/sql-tests/results/ansi/math.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/math.sql.out 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/math.sql.out 2023-09-18 15:00:43.006708418 +0800
@@ -0,0 +1,799 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+SELECT round(25y, 1)
+-- !query schema
+struct<round(25, 1):tinyint>
+-- !query output
+25
+
+
+-- !query
+SELECT round(25y, 0)
+-- !query schema
+struct<round(25, 0):tinyint>
+-- !query output
+25
+
+
+-- !query
+SELECT round(25y, -1)
+-- !query schema
+struct<round(25, -1):tinyint>
+-- !query output
+30
+
+
+-- !query
+SELECT round(25y, -2)
+-- !query schema
+struct<round(25, -2):tinyint>
+-- !query output
+0
+
+
+-- !query
+SELECT round(25y, -3)
+-- !query schema
+struct<round(25, -3):tinyint>
+-- !query output
+0
+
+
+-- !query
+SELECT round(127y, -1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 22,
+ "fragment" : "round(127y, -1)"
+ } ]
+}
+
+
+-- !query
+SELECT round(-128y, -1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 23,
+ "fragment" : "round(-128y, -1)"
+ } ]
+}
+
+
+-- !query
+SELECT round(525s, 1)
+-- !query schema
+struct<round(525, 1):smallint>
+-- !query output
+525
+
+
+-- !query
+SELECT round(525s, 0)
+-- !query schema
+struct<round(525, 0):smallint>
+-- !query output
+525
+
+
+-- !query
+SELECT round(525s, -1)
+-- !query schema
+struct<round(525, -1):smallint>
+-- !query output
+530
+
+
+-- !query
+SELECT round(525s, -2)
+-- !query schema
+struct<round(525, -2):smallint>
+-- !query output
+500
+
+
+-- !query
+SELECT round(525s, -3)
+-- !query schema
+struct<round(525, -3):smallint>
+-- !query output
+1000
+
+
+-- !query
+SELECT round(32767s, -1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 24,
+ "fragment" : "round(32767s, -1)"
+ } ]
+}
+
+
+-- !query
+SELECT round(-32768s, -1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 25,
+ "fragment" : "round(-32768s, -1)"
+ } ]
+}
+
+
+-- !query
+SELECT round(525, 1)
+-- !query schema
+struct<round(525, 1):int>
+-- !query output
+525
+
+
+-- !query
+SELECT round(525, 0)
+-- !query schema
+struct<round(525, 0):int>
+-- !query output
+525
+
+
+-- !query
+SELECT round(525, -1)
+-- !query schema
+struct<round(525, -1):int>
+-- !query output
+530
+
+
+-- !query
+SELECT round(525, -2)
+-- !query schema
+struct<round(525, -2):int>
+-- !query output
+500
+
+
+-- !query
+SELECT round(525, -3)
+-- !query schema
+struct<round(525, -3):int>
+-- !query output
+1000
+
+
+-- !query
+SELECT round(2147483647, -1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "round(2147483647, -1)"
+ } ]
+}
+
+
+-- !query
+SELECT round(-2147483647, -1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "round(-2147483647, -1)"
+ } ]
+}
+
+
+-- !query
+SELECT round(525L, 1)
+-- !query schema
+struct<round(525, 1):bigint>
+-- !query output
+525
+
+
+-- !query
+SELECT round(525L, 0)
+-- !query schema
+struct<round(525, 0):bigint>
+-- !query output
+525
+
+
+-- !query
+SELECT round(525L, -1)
+-- !query schema
+struct<round(525, -1):bigint>
+-- !query output
+530
+
+
+-- !query
+SELECT round(525L, -2)
+-- !query schema
+struct<round(525, -2):bigint>
+-- !query output
+500
+
+
+-- !query
+SELECT round(525L, -3)
+-- !query schema
+struct<round(525, -3):bigint>
+-- !query output
+1000
+
+
+-- !query
+SELECT round(9223372036854775807L, -1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 38,
+ "fragment" : "round(9223372036854775807L, -1)"
+ } ]
+}
+
+
+-- !query
+SELECT round(-9223372036854775808L, -1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 39,
+ "fragment" : "round(-9223372036854775808L, -1)"
+ } ]
+}
+
+
+-- !query
+SELECT bround(25y, 1)
+-- !query schema
+struct<bround(25, 1):tinyint>
+-- !query output
+25
+
+
+-- !query
+SELECT bround(25y, 0)
+-- !query schema
+struct<bround(25, 0):tinyint>
+-- !query output
+25
+
+
+-- !query
+SELECT bround(25y, -1)
+-- !query schema
+struct<bround(25, -1):tinyint>
+-- !query output
+20
+
+
+-- !query
+SELECT bround(25y, -2)
+-- !query schema
+struct<bround(25, -2):tinyint>
+-- !query output
+0
+
+
+-- !query
+SELECT bround(25y, -3)
+-- !query schema
+struct<bround(25, -3):tinyint>
+-- !query output
+0
+
+
+-- !query
+SELECT bround(127y, -1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 23,
+ "fragment" : "bround(127y, -1)"
+ } ]
+}
+
+
+-- !query
+SELECT bround(-128y, -1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 24,
+ "fragment" : "bround(-128y, -1)"
+ } ]
+}
+
+
+-- !query
+SELECT bround(525s, 1)
+-- !query schema
+struct<bround(525, 1):smallint>
+-- !query output
+525
+
+
+-- !query
+SELECT bround(525s, 0)
+-- !query schema
+struct<bround(525, 0):smallint>
+-- !query output
+525
+
+
+-- !query
+SELECT bround(525s, -1)
+-- !query schema
+struct<bround(525, -1):smallint>
+-- !query output
+520
+
+
+-- !query
+SELECT bround(525s, -2)
+-- !query schema
+struct<bround(525, -2):smallint>
+-- !query output
+500
+
+
+-- !query
+SELECT bround(525s, -3)
+-- !query schema
+struct<bround(525, -3):smallint>
+-- !query output
+1000
+
+
+-- !query
+SELECT bround(32767s, -1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 25,
+ "fragment" : "bround(32767s, -1)"
+ } ]
+}
+
+
+-- !query
+SELECT bround(-32768s, -1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 26,
+ "fragment" : "bround(-32768s, -1)"
+ } ]
+}
+
+
+-- !query
+SELECT bround(525, 1)
+-- !query schema
+struct<bround(525, 1):int>
+-- !query output
+525
+
+
+-- !query
+SELECT bround(525, 0)
+-- !query schema
+struct<bround(525, 0):int>
+-- !query output
+525
+
+
+-- !query
+SELECT bround(525, -1)
+-- !query schema
+struct<bround(525, -1):int>
+-- !query output
+520
+
+
+-- !query
+SELECT bround(525, -2)
+-- !query schema
+struct<bround(525, -2):int>
+-- !query output
+500
+
+
+-- !query
+SELECT bround(525, -3)
+-- !query schema
+struct<bround(525, -3):int>
+-- !query output
+1000
+
+
+-- !query
+SELECT bround(2147483647, -1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "bround(2147483647, -1)"
+ } ]
+}
+
+
+-- !query
+SELECT bround(-2147483647, -1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "bround(-2147483647, -1)"
+ } ]
+}
+
+
+-- !query
+SELECT bround(525L, 1)
+-- !query schema
+struct<bround(525, 1):bigint>
+-- !query output
+525
+
+
+-- !query
+SELECT bround(525L, 0)
+-- !query schema
+struct<bround(525, 0):bigint>
+-- !query output
+525
+
+
+-- !query
+SELECT bround(525L, -1)
+-- !query schema
+struct<bround(525, -1):bigint>
+-- !query output
+520
+
+
+-- !query
+SELECT bround(525L, -2)
+-- !query schema
+struct<bround(525, -2):bigint>
+-- !query output
+500
+
+
+-- !query
+SELECT bround(525L, -3)
+-- !query schema
+struct<bround(525, -3):bigint>
+-- !query output
+1000
+
+
+-- !query
+SELECT bround(9223372036854775807L, -1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 39,
+ "fragment" : "bround(9223372036854775807L, -1)"
+ } ]
+}
+
+
+-- !query
+SELECT bround(-9223372036854775808L, -1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 40,
+ "fragment" : "bround(-9223372036854775808L, -1)"
+ } ]
+}
+
+
+-- !query
+SELECT conv('100', 2, 10)
+-- !query schema
+struct<conv(100, 2, 10):string>
+-- !query output
+4
+
+
+-- !query
+SELECT conv(-10, 16, -10)
+-- !query schema
+struct<conv(-10, 16, -10):string>
+-- !query output
+-16
+
+
+-- !query
+SELECT conv('9223372036854775808', 10, 16)
+-- !query schema
+struct<conv(9223372036854775808, 10, 16):string>
+-- !query output
+8000000000000000
+
+
+-- !query
+SELECT conv('92233720368547758070', 10, 16)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow in function conv()"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "conv('92233720368547758070', 10, 16)"
+ } ]
+}
+
+
+-- !query
+SELECT conv('9223372036854775807', 36, 10)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow in function conv()"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 42,
+ "fragment" : "conv('9223372036854775807', 36, 10)"
+ } ]
+}
+
+
+-- !query
+SELECT conv('-9223372036854775807', 36, 10)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : "",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Overflow in function conv()"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "conv('-9223372036854775807', 36, 10)"
+ } ]
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/parse-schema-string.sql.out sql/core/src/test/resources/sql-tests/results/ansi/parse-schema-string.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/parse-schema-string.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/parse-schema-string.sql.out 2023-09-18 15:00:43.006708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 4
-
-
-- !query
select from_csv('1', 'create INT')
-- !query schema
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out 2023-09-18 15:00:43.006708418 +0800
@@ -1,14 +1,20 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 152
-
-
-- !query
select concat_ws()
-- !query schema
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-requirement failed: concat_ws requires at least one argument.; line 1 pos 7
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "0",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "> 0",
+ "functionName" : "`concat_ws`"
+ }
+}
-- !query
@@ -17,7 +23,16 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-requirement failed: format_string() should take at least 1 argument; line 1 pos 7
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "0",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "> 0",
+ "functionName" : "`format_string`"
+ }
+}
-- !query
@@ -82,10 +97,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value 'a' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 43) ==
-...t("abcd", -2), left("abcd", 0), left("abcd", 'a')
- ^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'a'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"INT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 43,
+ "stopIndex" : 59,
+ "fragment" : "left(\"abcd\", 'a')"
+ } ]
+}
-- !query
@@ -110,10 +138,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value 'a' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 45) ==
-...("abcd", -2), right("abcd", 0), right("abcd", 'a')
- ^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'a'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"INT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 45,
+ "stopIndex" : 62,
+ "fragment" : "right(\"abcd\", 'a')"
+ } ]
+}
-- !query
@@ -133,6 +174,38 @@
-- !query
+SELECT split('hello', '')
+-- !query schema
+struct<split(hello, , -1):array<string>>
+-- !query output
+["h","e","l","l","o"]
+
+
+-- !query
+SELECT split('', '')
+-- !query schema
+struct<split(, , -1):array<string>>
+-- !query output
+[""]
+
+
+-- !query
+SELECT split('abc', null)
+-- !query schema
+struct<split(abc, NULL, -1):array<string>>
+-- !query output
+NULL
+
+
+-- !query
+SELECT split(null, 'b')
+-- !query schema
+struct<split(NULL, b, -1):array<string>>
+-- !query output
+NULL
+
+
+-- !query
SELECT split_part('11.12.13', '.', 2)
-- !query schema
struct<split_part(11.12.13, ., 2):string>
@@ -177,8 +250,11 @@
-- !query schema
struct<>
-- !query output
-java.lang.ArrayIndexOutOfBoundsException
-SQL array indices start at 1
+org.apache.spark.SparkRuntimeException
+{
+ "errorClass" : "INVALID_INDEX_OF_ZERO",
+ "sqlState" : "22003"
+}
-- !query
@@ -427,10 +503,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value 'invalid_length' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT lpad('hi', 'invalid_length')
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'invalid_length'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"INT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 35,
+ "fragment" : "lpad('hi', 'invalid_length')"
+ } ]
+}
-- !query
@@ -439,10 +528,23 @@
struct<>
-- !query output
org.apache.spark.SparkNumberFormatException
-The value 'invalid_length' of the type "STRING" cannot be cast to "INT" because it is malformed. Correct the value as per the syntax, or change its target type. Use `try_cast` to tolerate malformed input and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT rpad('hi', 'invalid_length')
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "CAST_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "expression" : "'invalid_length'",
+ "sourceType" : "\"STRING\"",
+ "targetType" : "\"INT\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 35,
+ "fragment" : "rpad('hi', 'invalid_length')"
+ } ]
+}
-- !query
@@ -707,7 +809,23 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function decode. Expected: 2; Found: 0; line 1 pos 7
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "0",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "2",
+ "functionName" : "`decode`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 15,
+ "fragment" : "decode()"
+ } ]
+}
-- !query
@@ -716,7 +834,23 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function decode. Expected: 2; Found: 1; line 1 pos 7
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "1",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "2",
+ "functionName" : "`decode`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 37,
+ "fragment" : "decode(encode('abc', 'utf-8'))"
+ } ]
+}
-- !query
@@ -1112,36 +1246,243 @@
-- !query
-select to_binary('abc')
+select to_binary('', 'base64')
-- !query schema
-struct<to_binary(abc):binary>
+struct<to_binary(, base64):binary>
-- !query output
-�
-- !query
-select to_binary('abc', 'utf-8')
+select to_binary(' ', 'base64')
-- !query schema
-struct<to_binary(abc, utf-8):binary>
+struct<to_binary( , base64):binary>
-- !query output
-abc
+
+
+
+-- !query
+select to_binary(' ab cd ', 'base64')
+-- !query schema
+struct<to_binary( ab cd , base64):binary>
+-- !query output
+i�
-- !query
-select to_binary('abc', 'base64')
+select to_binary(' ab c=', 'base64')
-- !query schema
-struct<to_binary(abc, base64):binary>
+struct<to_binary( ab c=, base64):binary>
-- !query output
i�
-- !query
-select to_binary('abc', 'hex')
+select to_binary(' ab cdef= = ', 'base64')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkIllegalArgumentException
+{
+ "errorClass" : "CONVERSION_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "fmt" : "'BASE64'",
+ "str" : "' ab cdef= = '",
+ "suggestion" : "`try_to_binary`",
+ "targetType" : "\"BINARY\""
+ }
+}
+
+
+-- !query
+select to_binary(
+ concat(' b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0',
+ 'd2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl'), 'base64')
-- !query schema
-struct<to_binary(abc, hex):binary>
+struct<to_binary(concat( b25lIHR3byB0aHJlZSBmb3VyIGZpdmUgc2l4IHNldmVuIGVpZ2h0IG5pbmUgdGVuIGVsZXZlbiB0, d2VsdmUgdGhpcnRlZW4gZm91cnRlZW4gZml2dGVlbiBzaXh0ZWVuIHNldmVudGVlbiBlaWdodGVl), base64):binary>
-- !query output
+one two three four five six seven eight nine ten eleven twelve thirteen fourteen fivteen sixteen seventeen eightee
+
+-- !query
+select to_binary('a', 'base64')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkIllegalArgumentException
+{
+ "errorClass" : "CONVERSION_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "fmt" : "'BASE64'",
+ "str" : "'a'",
+ "suggestion" : "`try_to_binary`",
+ "targetType" : "\"BINARY\""
+ }
+}
+
+
+-- !query
+select to_binary('a?', 'base64')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkIllegalArgumentException
+{
+ "errorClass" : "CONVERSION_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "fmt" : "'BASE64'",
+ "str" : "'a?'",
+ "suggestion" : "`try_to_binary`",
+ "targetType" : "\"BINARY\""
+ }
+}
+
+
+-- !query
+select to_binary('abcde', 'base64')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkIllegalArgumentException
+{
+ "errorClass" : "CONVERSION_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "fmt" : "'BASE64'",
+ "str" : "'abcde'",
+ "suggestion" : "`try_to_binary`",
+ "targetType" : "\"BINARY\""
+ }
+}
+
+
+-- !query
+select to_binary('abcd=', 'base64')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkIllegalArgumentException
+{
+ "errorClass" : "CONVERSION_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "fmt" : "'BASE64'",
+ "str" : "'abcd='",
+ "suggestion" : "`try_to_binary`",
+ "targetType" : "\"BINARY\""
+ }
+}
+
+
+-- !query
+select to_binary('a===', 'base64')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkIllegalArgumentException
+{
+ "errorClass" : "CONVERSION_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "fmt" : "'BASE64'",
+ "str" : "'a==='",
+ "suggestion" : "`try_to_binary`",
+ "targetType" : "\"BINARY\""
+ }
+}
+
+
+-- !query
+select to_binary('ab==f', 'base64')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkIllegalArgumentException
+{
+ "errorClass" : "CONVERSION_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "fmt" : "'BASE64'",
+ "str" : "'ab==f'",
+ "suggestion" : "`try_to_binary`",
+ "targetType" : "\"BINARY\""
+ }
+}
+
+
+-- !query
+select to_binary(
+ '∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β)', 'utf-8')
+-- !query schema
+struct<to_binary(∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β), utf-8):binary>
+-- !query output
+∮ E⋅da = Q, n → ∞, ∑ f(i) = ∏ g(i), ∀x∈ℝ: ⌈x⌉ = −⌊−x⌋, α ∧ ¬β = ¬(¬α ∨ β)
+
+
+-- !query
+select to_binary('大千世界', 'utf8')
+-- !query schema
+struct<to_binary(大千世界, utf8):binary>
+-- !query output
+大千世界
+
+
+-- !query
+select to_binary('', 'utf-8')
+-- !query schema
+struct<to_binary(, utf-8):binary>
+-- !query output
+
+
+
+-- !query
+select to_binary(' ', 'utf8')
+-- !query schema
+struct<to_binary( , utf8):binary>
+-- !query output
+
+
+
+-- !query
+select to_binary('737472696E67')
+-- !query schema
+struct<to_binary(737472696E67):binary>
+-- !query output
+string
+
+
+-- !query
+select to_binary('737472696E67', 'hex')
+-- !query schema
+struct<to_binary(737472696E67, hex):binary>
+-- !query output
+string
+
+
+-- !query
+select to_binary('')
+-- !query schema
+struct<to_binary():binary>
+-- !query output
+
+
+
+-- !query
+select to_binary('1', 'hex')
+-- !query schema
+struct<to_binary(1, hex):binary>
+-- !query output
+
+
+
+-- !query
+select to_binary('FF')
+-- !query schema
+struct<to_binary(FF):binary>
+-- !query output
�
@@ -1162,6 +1503,42 @@
-- !query
+select to_binary('GG')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkIllegalArgumentException
+{
+ "errorClass" : "CONVERSION_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "fmt" : "'HEX'",
+ "str" : "'GG'",
+ "suggestion" : "`try_to_binary`",
+ "targetType" : "\"BINARY\""
+ }
+}
+
+
+-- !query
+select to_binary('01 AF', 'hex')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkIllegalArgumentException
+{
+ "errorClass" : "CONVERSION_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "fmt" : "'HEX'",
+ "str" : "'01 AF'",
+ "suggestion" : "`try_to_binary`",
+ "targetType" : "\"BINARY\""
+ }
+}
+
+
+-- !query
select to_binary('abc', concat('utf', '-8'))
-- !query schema
struct<to_binary(abc, concat(utf, -8)):binary>
@@ -1170,6 +1547,42 @@
-- !query
+select to_binary(' ab cdef= = ', substr('base64whynot', 0, 6))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkIllegalArgumentException
+{
+ "errorClass" : "CONVERSION_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "fmt" : "'BASE64'",
+ "str" : "' ab cdef= = '",
+ "suggestion" : "`try_to_binary`",
+ "targetType" : "\"BINARY\""
+ }
+}
+
+
+-- !query
+select to_binary(' ab cdef= = ', replace('HEX0', '0'))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkIllegalArgumentException
+{
+ "errorClass" : "CONVERSION_INVALID_INPUT",
+ "sqlState" : "22018",
+ "messageParameters" : {
+ "fmt" : "'HEX'",
+ "str" : "' ab cdef= = '",
+ "suggestion" : "`try_to_binary`",
+ "targetType" : "\"BINARY\""
+ }
+}
+
+
+-- !query
select to_binary('abc', 'Hex')
-- !query schema
struct<to_binary(abc, Hex):binary>
@@ -1211,36 +1624,93 @@
-- !query
-select to_binary(null, cast(null as int))
+select to_binary('abc', 1)
-- !query schema
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-The 'format' parameter of function 'to_binary' needs to be a string literal.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputName" : "fmt",
+ "inputValue" : "'1'",
+ "requireType" : "case-insensitive \"STRING\"",
+ "sqlExpr" : "\"to_binary(abc, 1)\"",
+ "validValues" : "'hex', 'utf-8', 'utf8', or 'base64'"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 26,
+ "fragment" : "to_binary('abc', 1)"
+ } ]
+}
-- !query
-select to_binary('abc', 1)
+select to_binary('abc', 'invalidFormat')
-- !query schema
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-The 'format' parameter of function 'to_binary' needs to be a string literal.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.INVALID_ARG_VALUE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputName" : "fmt",
+ "inputValue" : "'invalidformat'",
+ "requireType" : "case-insensitive \"STRING\"",
+ "sqlExpr" : "\"to_binary(abc, invalidFormat)\"",
+ "validValues" : "'hex', 'utf-8', 'utf8', or 'base64'"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 40,
+ "fragment" : "to_binary('abc', 'invalidFormat')"
+ } ]
+}
-- !query
-select to_binary('abc', 'invalidFormat')
+CREATE TEMPORARY VIEW fmtTable(fmtField) AS SELECT * FROM VALUES ('invalidFormat')
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT to_binary('abc', fmtField) FROM fmtTable
-- !query schema
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Invalid value for the 'format' parameter of function 'to_binary': invalidformat. The value has to be a case-insensitive string literal of 'hex', 'utf-8', or 'base64'.
+{
+ "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputExpr" : "\"fmtField\"",
+ "inputName" : "fmt",
+ "inputType" : "\"STRING\"",
+ "sqlExpr" : "\"to_binary(abc, fmtField)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 33,
+ "fragment" : "to_binary('abc', fmtField)"
+ } ]
+}
-- !query
-select to_binary('a!', 'base64')
+DROP VIEW IF EXISTS fmtTable
-- !query schema
struct<>
-- !query output
-java.lang.IllegalArgumentException
-Last unit does not have enough valid bits
+
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/timestamp.sql.out sql/core/src/test/resources/sql-tests/results/ansi/timestamp.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/timestamp.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/timestamp.sql.out 2023-09-18 15:00:43.006708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 98
-
-
-- !query
select timestamp '2019-01-01\t'
-- !query schema
@@ -16,12 +13,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the TIMESTAMP value: 2019-01-01中文(line 1, pos 7)
-
-== SQL ==
-select timestamp '2019-01-01中文'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2019-01-01中文'",
+ "valueType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 31,
+ "fragment" : "timestamp '2019-01-01中文'"
+ } ]
+}
-- !query
@@ -30,12 +36,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the TIMESTAMP value: 4294967297(line 1, pos 7)
-
-== SQL ==
-select timestamp'4294967297'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'4294967297'",
+ "valueType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "timestamp'4294967297'"
+ } ]
+}
-- !query
@@ -44,12 +59,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the TIMESTAMP value: 2021-01-01T12:30:4294967297.123456(line 1, pos 7)
-
-== SQL ==
-select timestamp'2021-01-01T12:30:4294967297.123456'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2021-01-01T12:30:4294967297.123456'",
+ "valueType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 52,
+ "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'"
+ } ]
+}
-- !query
@@ -98,7 +122,13 @@
struct<>
-- !query output
org.apache.spark.SparkDateTimeException
-The fraction of sec must be zero. Valid range is [0, 60]. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.
+{
+ "errorClass" : "INVALID_FRACTION_OF_SECOND",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\""
+ }
+}
-- !query
@@ -122,8 +152,14 @@
-- !query schema
struct<>
-- !query output
-java.time.DateTimeException
-Invalid value for SecondOfMinute (valid values 0 - 59): 61. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_2000",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Invalid value for SecondOfMinute (valid values 0 - 59): 61"
+ }
+}
-- !query
@@ -147,8 +183,14 @@
-- !query schema
struct<>
-- !query output
-java.time.DateTimeException
-Invalid value for SecondOfMinute (valid values 0 - 59): 99. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_2000",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Invalid value for SecondOfMinute (valid values 0 - 59): 99"
+ }
+}
-- !query
@@ -156,8 +198,14 @@
-- !query schema
struct<>
-- !query output
-java.time.DateTimeException
-Invalid value for SecondOfMinute (valid values 0 - 59): 999. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_2000",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Invalid value for SecondOfMinute (valid values 0 - 59): 999"
+ }
+}
-- !query
@@ -339,8 +387,15 @@
-- !query schema
struct<>
-- !query output
-java.time.format.DateTimeParseException
-Text '2019-10-06 10:11:12.' could not be parsed at index 20. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '2019-10-06 10:11:12.' could not be parsed at index 20"
+ }
+}
-- !query
@@ -404,8 +459,15 @@
-- !query schema
struct<>
-- !query output
-java.time.format.DateTimeParseException
-Text '2019-10-06 10:11:12.1234567PST' could not be parsed, unparsed text found at index 26. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '2019-10-06 10:11:12.1234567PST' could not be parsed, unparsed text found at index 26"
+ }
+}
-- !query
@@ -421,8 +483,15 @@
-- !query schema
struct<>
-- !query output
-java.time.format.DateTimeParseException
-Text '223456 2019-10-06 10:11:12.123456PST' could not be parsed at index 27. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '223456 2019-10-06 10:11:12.123456PST' could not be parsed at index 27"
+ }
+}
-- !query
@@ -486,8 +555,15 @@
-- !query schema
struct<>
-- !query output
-java.time.format.DateTimeParseException
-Text '12.1232019-10-06S10:11' could not be parsed at index 7. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 7"
+ }
+}
-- !query
@@ -495,8 +571,15 @@
-- !query schema
struct<>
-- !query output
-java.time.format.DateTimeParseException
-Text '12.1232019-10-06S10:11' could not be parsed at index 9. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Text '12.1232019-10-06S10:11' could not be parsed at index 9"
+ }
+}
-- !query
@@ -568,8 +651,15 @@
-- !query schema
struct<>
-- !query output
-java.time.DateTimeException
-Invalid date 'February 29' as '1970' is not a leap year. If necessary set spark.sql.ansi.enabled to false to bypass this error.
+org.apache.spark.SparkDateTimeException
+{
+ "errorClass" : "CANNOT_PARSE_TIMESTAMP",
+ "sqlState" : "22007",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "message" : "Invalid date 'February 29' as '1970' is not a leap year"
+ }
+}
-- !query
@@ -674,7 +764,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(TIMESTAMP '2011-11-11 11:11:11' + CAST('1' AS TIMESTAMP))' due to data type mismatch: '(TIMESTAMP '2011-11-11 11:11:11' + CAST('1' AS TIMESTAMP))' requires (numeric or interval day to second or interval year to month or interval) type, not timestamp; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "actualDataType" : "\"TIMESTAMP\"",
+ "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")",
+ "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "timestamp'2011-11-11 11:11:11' + '1'"
+ } ]
+}
-- !query
@@ -683,7 +788,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(CAST('1' AS TIMESTAMP) + TIMESTAMP '2011-11-11 11:11:11')' due to data type mismatch: '(CAST('1' AS TIMESTAMP) + TIMESTAMP '2011-11-11 11:11:11')' requires (numeric or interval day to second or interval year to month or interval) type, not timestamp; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_WRONG_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "actualDataType" : "\"TIMESTAMP\"",
+ "inputType" : "(\"NUMERIC\" or \"INTERVAL DAY TO SECOND\" or \"INTERVAL YEAR TO MONTH\" or \"INTERVAL\")",
+ "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "'1' + timestamp'2011-11-11 11:11:11'"
+ } ]
+}
-- !query
@@ -692,7 +812,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(TIMESTAMP '2011-11-11 11:11:11' + NULL)' due to data type mismatch: differing types in '(TIMESTAMP '2011-11-11 11:11:11' + NULL)' (timestamp and void).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"TIMESTAMP\"",
+ "right" : "\"VOID\"",
+ "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 44,
+ "fragment" : "timestamp'2011-11-11 11:11:11' + null"
+ } ]
+}
-- !query
@@ -701,7 +836,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(NULL + TIMESTAMP '2011-11-11 11:11:11')' due to data type mismatch: differing types in '(NULL + TIMESTAMP '2011-11-11 11:11:11')' (void and timestamp).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"VOID\"",
+ "right" : "\"TIMESTAMP\"",
+ "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 44,
+ "fragment" : "null + timestamp'2011-11-11 11:11:11'"
+ } ]
+}
-- !query
@@ -733,7 +883,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'yyyy-MM-dd GGGGG' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'yyyy-MM-dd GGGGG'"
+ }
+}
-- !query
@@ -742,7 +900,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd MM yyyy EEEEEE'"
+ }
+}
-- !query
@@ -751,7 +917,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd MM yyyy EEEEE'"
+ }
+}
-- !query
@@ -760,7 +934,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd MM yyyy EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd MM yyyy EEEEE'"
+ }
+}
-- !query
@@ -769,7 +951,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd/MMMMM/yyyy'"
+ }
+}
-- !query
@@ -778,7 +968,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd/MMMMM/yyyy'"
+ }
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/try_aggregates.sql.out sql/core/src/test/resources/sql-tests/results/ansi/try_aggregates.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/try_aggregates.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/try_aggregates.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 20
-
-
-- !query
SELECT try_sum(col) FROM VALUES (5), (10), (15) AS tab(col)
-- !query schema
@@ -83,6 +80,134 @@
-- !query
+SELECT try_sum(col / 0) FROM VALUES (5), (10), (15) AS tab(col)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "DIVIDE_BY_ZERO",
+ "sqlState" : "22012",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 16,
+ "stopIndex" : 22,
+ "fragment" : "col / 0"
+ } ]
+}
+
+
+-- !query
+SELECT try_sum(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "DIVIDE_BY_ZERO",
+ "sqlState" : "22012",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 16,
+ "stopIndex" : 22,
+ "fragment" : "col / 0"
+ } ]
+}
+
+
+-- !query
+SELECT try_sum(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "DIVIDE_BY_ZERO",
+ "sqlState" : "22012",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 16,
+ "stopIndex" : 22,
+ "fragment" : "col / 0"
+ } ]
+}
+
+
+-- !query
+SELECT try_sum(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "long overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 16,
+ "stopIndex" : 23,
+ "fragment" : "col + 1L"
+ } ]
+}
+
+
+-- !query
+SELECT try_sum(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "INTERVAL_DIVIDED_BY_ZERO",
+ "sqlState" : "22012",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 16,
+ "stopIndex" : 22,
+ "fragment" : "col / 0"
+ } ]
+}
+
+
+-- !query
+SELECT try_sum(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "INTERVAL_DIVIDED_BY_ZERO",
+ "sqlState" : "22012",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 16,
+ "stopIndex" : 22,
+ "fragment" : "col / 0"
+ } ]
+}
+
+
+-- !query
SELECT try_avg(col) FROM VALUES (5), (10), (15) AS tab(col)
-- !query schema
struct<try_avg(col):double>
@@ -160,3 +285,131 @@
struct<try_avg(col):interval day to second>
-- !query output
NULL
+
+
+-- !query
+SELECT try_avg(col / 0) FROM VALUES (5), (10), (15) AS tab(col)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "DIVIDE_BY_ZERO",
+ "sqlState" : "22012",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 16,
+ "stopIndex" : 22,
+ "fragment" : "col / 0"
+ } ]
+}
+
+
+-- !query
+SELECT try_avg(col / 0) FROM VALUES (5.0), (10.0), (15.0) AS tab(col)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "DIVIDE_BY_ZERO",
+ "sqlState" : "22012",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 16,
+ "stopIndex" : 22,
+ "fragment" : "col / 0"
+ } ]
+}
+
+
+-- !query
+SELECT try_avg(col / 0) FROM VALUES (NULL), (10), (15) AS tab(col)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "DIVIDE_BY_ZERO",
+ "sqlState" : "22012",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 16,
+ "stopIndex" : 22,
+ "fragment" : "col / 0"
+ } ]
+}
+
+
+-- !query
+SELECT try_avg(col + 1L) FROM VALUES (9223372036854775807L), (1L) AS tab(col)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "long overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 16,
+ "stopIndex" : 23,
+ "fragment" : "col + 1L"
+ } ]
+}
+
+
+-- !query
+SELECT try_avg(col / 0) FROM VALUES (interval '1 months'), (interval '1 months') AS tab(col)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "INTERVAL_DIVIDED_BY_ZERO",
+ "sqlState" : "22012",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 16,
+ "stopIndex" : 22,
+ "fragment" : "col / 0"
+ } ]
+}
+
+
+-- !query
+SELECT try_avg(col / 0) FROM VALUES (interval '1 seconds'), (interval '1 seconds') AS tab(col)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "INTERVAL_DIVIDED_BY_ZERO",
+ "sqlState" : "22012",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 16,
+ "stopIndex" : 22,
+ "fragment" : "col / 0"
+ } ]
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/try_arithmetic.sql.out sql/core/src/test/resources/sql-tests/results/ansi/try_arithmetic.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/try_arithmetic.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/try_arithmetic.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 49
-
-
-- !query
SELECT try_add(1, 1)
-- !query schema
@@ -43,6 +40,76 @@
-- !query
+SELECT try_add(1, (2147483647 + 1))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "integer overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 20,
+ "stopIndex" : 33,
+ "fragment" : "2147483647 + 1"
+ } ]
+}
+
+
+-- !query
+SELECT try_add(1L, (9223372036854775807L + 1L))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "long overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 21,
+ "stopIndex" : 45,
+ "fragment" : "9223372036854775807L + 1L"
+ } ]
+}
+
+
+-- !query
+SELECT try_add(1, 1.0 / 0.0)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "DIVIDE_BY_ZERO",
+ "sqlState" : "22012",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 19,
+ "stopIndex" : 27,
+ "fragment" : "1.0 / 0.0"
+ } ]
+}
+
+
+-- !query
SELECT try_add(date'2021-01-01', 1)
-- !query schema
struct<try_add(DATE '2021-01-01', 1):date>
@@ -144,7 +211,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'INTERVAL '2' YEAR + INTERVAL '02' SECOND' due to data type mismatch: argument 1 requires (timestamp or timestamp without time zone) type, however, 'INTERVAL '2' YEAR' is of interval year type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"INTERVAL '2' YEAR\"",
+ "inputType" : "\"INTERVAL YEAR\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"INTERVAL '2' YEAR + INTERVAL '02' SECOND\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 50,
+ "fragment" : "try_add(interval 2 year, interval 2 second)"
+ } ]
+}
-- !query
@@ -188,6 +272,76 @@
-- !query
+SELECT try_divide(1, (2147483647 + 1))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "integer overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 23,
+ "stopIndex" : 36,
+ "fragment" : "2147483647 + 1"
+ } ]
+}
+
+
+-- !query
+SELECT try_divide(1L, (9223372036854775807L + 1L))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "long overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 24,
+ "stopIndex" : 48,
+ "fragment" : "9223372036854775807L + 1L"
+ } ]
+}
+
+
+-- !query
+SELECT try_divide(1, 1.0 / 0.0)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "DIVIDE_BY_ZERO",
+ "sqlState" : "22012",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 22,
+ "stopIndex" : 30,
+ "fragment" : "1.0 / 0.0"
+ } ]
+}
+
+
+-- !query
SELECT try_divide(interval 2 year, 2)
-- !query schema
struct<try_divide(INTERVAL '2' YEAR, 2):interval year to month>
@@ -276,6 +430,76 @@
-- !query
+SELECT try_subtract(1, (2147483647 + 1))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "integer overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 25,
+ "stopIndex" : 38,
+ "fragment" : "2147483647 + 1"
+ } ]
+}
+
+
+-- !query
+SELECT try_subtract(1L, (9223372036854775807L + 1L))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "long overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 26,
+ "stopIndex" : 50,
+ "fragment" : "9223372036854775807L + 1L"
+ } ]
+}
+
+
+-- !query
+SELECT try_subtract(1, 1.0 / 0.0)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "DIVIDE_BY_ZERO",
+ "sqlState" : "22012",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 24,
+ "stopIndex" : 32,
+ "fragment" : "1.0 / 0.0"
+ } ]
+}
+
+
+-- !query
SELECT try_subtract(interval 2 year, interval 3 year)
-- !query schema
struct<try_subtract(INTERVAL '2' YEAR, INTERVAL '3' YEAR):interval year>
@@ -348,6 +572,76 @@
-- !query
+SELECT try_multiply(1, (2147483647 + 1))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "integer overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 25,
+ "stopIndex" : 38,
+ "fragment" : "2147483647 + 1"
+ } ]
+}
+
+
+-- !query
+SELECT try_multiply(1L, (9223372036854775807L + 1L))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "ARITHMETIC_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.",
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "message" : "long overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 26,
+ "stopIndex" : 50,
+ "fragment" : "9223372036854775807L + 1L"
+ } ]
+}
+
+
+-- !query
+SELECT try_multiply(1, 1.0 / 0.0)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "DIVIDE_BY_ZERO",
+ "sqlState" : "22012",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 24,
+ "stopIndex" : 32,
+ "fragment" : "1.0 / 0.0"
+ } ]
+}
+
+
+-- !query
SELECT try_multiply(interval 2 year, 2)
-- !query schema
struct<try_multiply(INTERVAL '2' YEAR, 2):interval year to month>
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/try_datetime_functions.sql.out sql/core/src/test/resources/sql-tests/results/ansi/try_datetime_functions.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/try_datetime_functions.sql.out 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/try_datetime_functions.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -0,0 +1,56 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+select try_to_timestamp(null), try_to_timestamp('2016-12-31 00:12:00'), try_to_timestamp('2016-12-31', 'yyyy-MM-dd')
+-- !query schema
+struct<try_to_timestamp(NULL):timestamp,try_to_timestamp(2016-12-31 00:12:00):timestamp,try_to_timestamp(2016-12-31, yyyy-MM-dd):timestamp>
+-- !query output
+NULL 2016-12-31 00:12:00 2016-12-31 00:00:00
+
+
+-- !query
+select try_to_timestamp(1)
+-- !query schema
+struct<try_to_timestamp(1):timestamp>
+-- !query output
+1969-12-31 16:00:01
+
+
+-- !query
+select try_to_timestamp('2016-12-31 abc')
+-- !query schema
+struct<try_to_timestamp(2016-12-31 abc):timestamp>
+-- !query output
+NULL
+
+
+-- !query
+select try_to_timestamp('2019-10-06 10:11:12.', 'yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]')
+-- !query schema
+struct<try_to_timestamp(2019-10-06 10:11:12., yyyy-MM-dd HH:mm:ss.SSSSSS[zzz]):timestamp>
+-- !query output
+NULL
+
+
+-- !query
+select try_to_timestamp("02-29", "MM-dd")
+-- !query schema
+struct<try_to_timestamp(02-29, MM-dd):timestamp>
+-- !query output
+NULL
+
+
+-- !query
+select try_to_timestamp('22 05 2020 Friday', 'dd MM yyyy EEEEEE')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkUpgradeException
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd MM yyyy EEEEEE'"
+ }
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/try_element_at.sql.out sql/core/src/test/resources/sql-tests/results/ansi/try_element_at.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ansi/try_element_at.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ansi/try_element_at.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,14 +1,14 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 8
-
-
-- !query
SELECT try_element_at(array(1, 2, 3), 0)
-- !query schema
struct<>
-- !query output
-java.lang.ArrayIndexOutOfBoundsException
-SQL array indices start at 1
+org.apache.spark.SparkRuntimeException
+{
+ "errorClass" : "INVALID_INDEX_OF_ZERO",
+ "sqlState" : "22003"
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/array.sql.out sql/core/src/test/resources/sql-tests/results/array.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/array.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/array.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 29
-
-
-- !query
create temporary view data as select * from values
("one", array(11, 12, 13), array(array(111, 112, 113), array(121, 122, 123))),
@@ -131,7 +128,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'sort_array(array('b', 'd'), '1')' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "2",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"sort_array(array(b, d), 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 39,
+ "fragment" : "sort_array(array('b', 'd'), '1')"
+ } ]
+}
-- !query
@@ -140,7 +154,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'sort_array(array('b', 'd'), CAST(NULL AS BOOLEAN))' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"CAST(NULL AS BOOLEAN)\"",
+ "inputType" : "\"BOOLEAN\"",
+ "paramIndex" : "2",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"sort_array(array(b, d), CAST(NULL AS BOOLEAN))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 57,
+ "fragment" : "sort_array(array('b', 'd'), cast(NULL as boolean))"
+ } ]
+}
-- !query
@@ -183,8 +214,11 @@
-- !query schema
struct<>
-- !query output
-java.lang.ArrayIndexOutOfBoundsException
-SQL array indices start at 1
+org.apache.spark.SparkRuntimeException
+{
+ "errorClass" : "INVALID_INDEX_OF_ZERO",
+ "sqlState" : "22003"
+}
-- !query
@@ -297,4 +331,348 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'array_size(map('a', 1, 'b', 2))' due to data type mismatch: argument 1 requires array type, however, 'map('a', 1, 'b', 2)' is of map<string,int> type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"map(a, 1, b, 2)\"",
+ "inputType" : "\"MAP<STRING, INT>\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"ARRAY\"",
+ "sqlExpr" : "\"array_size(map(a, 1, b, 2))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 38,
+ "fragment" : "array_size(map('a', 1, 'b', 2))"
+ } ]
+}
+
+
+-- !query
+select size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10)))
+-- !query schema
+struct<size(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10))):int>
+-- !query output
+4
+
+
+-- !query
+select size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10)))
+-- !query schema
+struct<size(arrays_zip(array(), array(1, 2, 3), array(4), array(7, 8, 9, 10))):int>
+-- !query output
+4
+
+
+-- !query
+select size(arrays_zip(array(1, 2, 3), array(4), null, array(7, 8, 9, 10)))
+-- !query schema
+struct<size(arrays_zip(array(1, 2, 3), array(4), NULL, array(7, 8, 9, 10))):int>
+-- !query output
+-1
+
+
+-- !query
+select isnotnull(arrays_zip(array(), array(4), array(7, 8, 9, 10)))
+-- !query schema
+struct<(arrays_zip(array(), array(4), array(7, 8, 9, 10)) IS NOT NULL):boolean>
+-- !query output
+true
+
+
+-- !query
+select isnotnull(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10)))
+-- !query schema
+struct<(arrays_zip(array(1, 2, 3), array(4), array(7, 8, 9, 10)) IS NOT NULL):boolean>
+-- !query output
+true
+
+
+-- !query
+select isnotnull(arrays_zip(array(1, 2, 3), NULL, array(4), array(7, 8, 9, 10)))
+-- !query schema
+struct<(arrays_zip(array(1, 2, 3), NULL, array(4), array(7, 8, 9, 10)) IS NOT NULL):boolean>
+-- !query output
+false
+
+
+-- !query
+select get(array(1, 2, 3), 0)
+-- !query schema
+struct<get(array(1, 2, 3), 0):int>
+-- !query output
+1
+
+
+-- !query
+select get(array(1, 2, 3), 3)
+-- !query schema
+struct<get(array(1, 2, 3), 3):int>
+-- !query output
+NULL
+
+
+-- !query
+select get(array(1, 2, 3), null)
+-- !query schema
+struct<get(array(1, 2, 3), NULL):int>
+-- !query output
+NULL
+
+
+-- !query
+select get(array(1, 2, 3), -1)
+-- !query schema
+struct<get(array(1, 2, 3), -1):int>
+-- !query output
+NULL
+
+
+-- !query
+select array_insert(array(1, 2, 3), 3, 4)
+-- !query schema
+struct<array_insert(array(1, 2, 3), 3, 4):array<int>>
+-- !query output
+[1,2,4,3]
+
+
+-- !query
+select array_insert(array(2, 3, 4), 0, 1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkRuntimeException
+{
+ "errorClass" : "INVALID_INDEX_OF_ZERO",
+ "sqlState" : "22003",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 41,
+ "fragment" : "array_insert(array(2, 3, 4), 0, 1)"
+ } ]
+}
+
+
+-- !query
+select array_insert(array(2, 3, 4), 1, 1)
+-- !query schema
+struct<array_insert(array(2, 3, 4), 1, 1):array<int>>
+-- !query output
+[1,2,3,4]
+
+
+-- !query
+select array_insert(array(1, 3, 4), -2, 2)
+-- !query schema
+struct<array_insert(array(1, 3, 4), -2, 2):array<int>>
+-- !query output
+[1,2,3,4]
+
+
+-- !query
+select array_insert(array(1, 2, 3), 3, "4")
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.ARRAY_FUNCTION_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "dataType" : "\"ARRAY\"",
+ "functionName" : "`array_insert`",
+ "leftType" : "\"ARRAY<INT>\"",
+ "rightType" : "\"STRING\"",
+ "sqlExpr" : "\"array_insert(array(1, 2, 3), 3, 4)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "array_insert(array(1, 2, 3), 3, \"4\")"
+ } ]
+}
+
+
+-- !query
+select array_insert(cast(NULL as ARRAY<INT>), 1, 1)
+-- !query schema
+struct<array_insert(NULL, 1, 1):array<int>>
+-- !query output
+NULL
+
+
+-- !query
+select array_insert(array(1, 2, 3, NULL), cast(NULL as INT), 4)
+-- !query schema
+struct<array_insert(array(1, 2, 3, NULL), CAST(NULL AS INT), 4):array<int>>
+-- !query output
+NULL
+
+
+-- !query
+select array_insert(array(1, 2, 3, NULL), 4, cast(NULL as INT))
+-- !query schema
+struct<array_insert(array(1, 2, 3, NULL), 4, CAST(NULL AS INT)):array<int>>
+-- !query output
+[1,2,3,null,null]
+
+
+-- !query
+select array_insert(array(2, 3, NULL, 4), 5, 5)
+-- !query schema
+struct<array_insert(array(2, 3, NULL, 4), 5, 5):array<int>>
+-- !query output
+[2,3,null,4,5]
+
+
+-- !query
+select array_insert(array(2, 3, NULL, 4), -5, 1)
+-- !query schema
+struct<array_insert(array(2, 3, NULL, 4), -5, 1):array<int>>
+-- !query output
+[1,null,2,3,null,4]
+
+
+-- !query
+select array_compact(id) from values (1) as t(id)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"id\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"ARRAY\"",
+ "sqlExpr" : "\"array_compact(id)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 24,
+ "fragment" : "array_compact(id)"
+ } ]
+}
+
+
+-- !query
+select array_compact(array("1", null, "2", null))
+-- !query schema
+struct<array_compact(array(1, NULL, 2, NULL)):array<string>>
+-- !query output
+["1","2"]
+
+
+-- !query
+select array_compact(array("a", "b", "c"))
+-- !query schema
+struct<array_compact(array(a, b, c)):array<string>>
+-- !query output
+["a","b","c"]
+
+
+-- !query
+select array_compact(array(1D, null, 2D, null))
+-- !query schema
+struct<array_compact(array(1.0, NULL, 2.0, NULL)):array<double>>
+-- !query output
+[1.0,2.0]
+
+
+-- !query
+select array_compact(array(array(1, 2, 3, null), null, array(4, null, 6)))
+-- !query schema
+struct<array_compact(array(array(1, 2, 3, NULL), NULL, array(4, NULL, 6))):array<array<int>>>
+-- !query output
+[[1,2,3,null],[4,null,6]]
+
+
+-- !query
+select array_compact(array(null))
+-- !query schema
+struct<array_compact(array(NULL)):array<void>>
+-- !query output
+[]
+
+
+-- !query
+select array_append(array(1, 2, 3), 4)
+-- !query schema
+struct<array_append(array(1, 2, 3), 4):array<int>>
+-- !query output
+[1,2,3,4]
+
+
+-- !query
+select array_append(array('a', 'b', 'c'), 'd')
+-- !query schema
+struct<array_append(array(a, b, c), d):array<string>>
+-- !query output
+["a","b","c","d"]
+
+
+-- !query
+select array_append(array(1, 2, 3, NULL), NULL)
+-- !query schema
+struct<array_append(array(1, 2, 3, NULL), NULL):array<int>>
+-- !query output
+[1,2,3,null,null]
+
+
+-- !query
+select array_append(array('a', 'b', 'c', NULL), NULL)
+-- !query schema
+struct<array_append(array(a, b, c, NULL), NULL):array<string>>
+-- !query output
+["a","b","c",null,null]
+
+
+-- !query
+select array_append(CAST(null AS ARRAY<String>), 'a')
+-- !query schema
+struct<array_append(NULL, a):array<string>>
+-- !query output
+NULL
+
+
+-- !query
+select array_append(CAST(null AS ARRAY<String>), CAST(null as String))
+-- !query schema
+struct<array_append(NULL, CAST(NULL AS STRING)):array<string>>
+-- !query output
+NULL
+
+
+-- !query
+select array_append(array(), 1)
+-- !query schema
+struct<array_append(array(), 1):array<int>>
+-- !query output
+[1]
+
+
+-- !query
+select array_append(CAST(array() AS ARRAY<String>), CAST(NULL AS String))
+-- !query schema
+struct<array_append(array(), CAST(NULL AS STRING)):array<string>>
+-- !query output
+[null]
+
+
+-- !query
+select array_append(array(CAST(NULL AS String)), CAST(NULL AS String))
+-- !query schema
+struct<array_append(array(CAST(NULL AS STRING)), CAST(NULL AS STRING)):array<string>>
+-- !query output
+[null,null]
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out sql/core/src/test/resources/sql-tests/results/bitwise.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/bitwise.sql.out 2023-09-15 11:42:05.966317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/bitwise.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 32
-
-
-- !query
select bit_count(null)
-- !query schema
@@ -152,7 +149,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'bit_count('bit count')' due to data type mismatch: argument 1 requires (integral or boolean) type, however, ''bit count'' is of string type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"bit count\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"INTEGRAL\" or \"BOOLEAN\")",
+ "sqlExpr" : "\"bit_count(bit count)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "bit_count(\"bit count\")"
+ } ]
+}
-- !query
@@ -161,7 +175,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'bit_count('a')' due to data type mismatch: argument 1 requires (integral or boolean) type, however, ''a'' is of string type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"a\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "1",
+ "requiredType" : "(\"INTEGRAL\" or \"BOOLEAN\")",
+ "sqlExpr" : "\"bit_count(a)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 21,
+ "fragment" : "bit_count('a')"
+ } ]
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/cast.sql.out sql/core/src/test/resources/sql-tests/results/cast.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/cast.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/cast.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 72
-
-
-- !query
SELECT CAST('1.23' AS int)
-- !query schema
@@ -353,12 +350,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot mix year-month and day-time fields: interval 3 month 1 hour(line 1, pos 12)
-
-== SQL ==
-SELECT CAST(interval 3 month 1 hour AS string)
-------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0029",
+ "messageParameters" : {
+ "literal" : "interval 3 month 1 hour"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 13,
+ "stopIndex" : 35,
+ "fragment" : "interval 3 month 1 hour"
+ } ]
+}
-- !query
@@ -594,3 +598,319 @@
struct<CAST(CAST(inf AS FLOAT) AS TIMESTAMP):timestamp>
-- !query output
NULL
+
+
+-- !query
+select cast(interval '1' year as tinyint)
+-- !query schema
+struct<CAST(INTERVAL '1' YEAR AS TINYINT):tinyint>
+-- !query output
+1
+
+
+-- !query
+select cast(interval '-10-2' year to month as smallint)
+-- !query schema
+struct<CAST(INTERVAL '-10-2' YEAR TO MONTH AS SMALLINT):smallint>
+-- !query output
+-122
+
+
+-- !query
+select cast(interval '1000' month as int)
+-- !query schema
+struct<CAST(INTERVAL '1000' MONTH AS INT):int>
+-- !query output
+1000
+
+
+-- !query
+select cast(interval -'10.123456' second as tinyint)
+-- !query schema
+struct<CAST(INTERVAL '-10.123456' SECOND AS TINYINT):tinyint>
+-- !query output
+-10
+
+
+-- !query
+select cast(interval '23:59:59' hour to second as smallint)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "CAST_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "sourceType" : "\"INTERVAL HOUR TO SECOND\"",
+ "targetType" : "\"SMALLINT\"",
+ "value" : "INTERVAL '23:59:59' HOUR TO SECOND"
+ }
+}
+
+
+-- !query
+select cast(interval -'1 02:03:04.123' day to second as int)
+-- !query schema
+struct<CAST(INTERVAL '-1 02:03:04.123' DAY TO SECOND AS INT):int>
+-- !query output
+-93784
+
+
+-- !query
+select cast(interval '10' day as bigint)
+-- !query schema
+struct<CAST(INTERVAL '10' DAY AS BIGINT):bigint>
+-- !query output
+10
+
+
+-- !query
+select cast(interval '-1000' month as tinyint)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "CAST_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "sourceType" : "\"INTERVAL MONTH\"",
+ "targetType" : "\"TINYINT\"",
+ "value" : "INTERVAL '-1000' MONTH"
+ }
+}
+
+
+-- !query
+select cast(interval '1000000' second as smallint)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "CAST_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "sourceType" : "\"INTERVAL SECOND\"",
+ "targetType" : "\"SMALLINT\"",
+ "value" : "INTERVAL '1000000' SECOND"
+ }
+}
+
+
+-- !query
+select cast(1Y as interval year)
+-- !query schema
+struct<CAST(1 AS INTERVAL YEAR):interval year>
+-- !query output
+1-0
+
+
+-- !query
+select cast(-122S as interval year to month)
+-- !query schema
+struct<CAST(-122 AS INTERVAL YEAR TO MONTH):interval year to month>
+-- !query output
+-10-2
+
+
+-- !query
+select cast(ym as interval year to month) from values(-122S) as t(ym)
+-- !query schema
+struct<ym:interval year to month>
+-- !query output
+-10-2
+
+
+-- !query
+select cast(1000 as interval month)
+-- !query schema
+struct<CAST(1000 AS INTERVAL MONTH):interval month>
+-- !query output
+83-4
+
+
+-- !query
+select cast(-10L as interval second)
+-- !query schema
+struct<CAST(-10 AS INTERVAL SECOND):interval second>
+-- !query output
+-0 00:00:10.000000000
+
+
+-- !query
+select cast(100Y as interval hour to second)
+-- !query schema
+struct<CAST(100 AS INTERVAL HOUR TO SECOND):interval hour to second>
+-- !query output
+0 00:01:40.000000000
+
+
+-- !query
+select cast(dt as interval hour to second) from values(100Y) as t(dt)
+-- !query schema
+struct<dt:interval hour to second>
+-- !query output
+0 00:01:40.000000000
+
+
+-- !query
+select cast(-1000S as interval day to second)
+-- !query schema
+struct<CAST(-1000 AS INTERVAL DAY TO SECOND):interval day to second>
+-- !query output
+-0 00:16:40.000000000
+
+
+-- !query
+select cast(10 as interval day)
+-- !query schema
+struct<CAST(10 AS INTERVAL DAY):interval day>
+-- !query output
+10 00:00:00.000000000
+
+
+-- !query
+select cast(2147483647 as interval year)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "CAST_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "sourceType" : "\"INT\"",
+ "targetType" : "\"INTERVAL YEAR\"",
+ "value" : "2147483647"
+ }
+}
+
+
+-- !query
+select cast(-9223372036854775808L as interval day)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "CAST_OVERFLOW",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+ "sourceType" : "\"BIGINT\"",
+ "targetType" : "\"INTERVAL DAY\"",
+ "value" : "-9223372036854775808L"
+ }
+}
+
+
+-- !query
+select cast(interval '-1' year as decimal(10, 0))
+-- !query schema
+struct<CAST(INTERVAL '-1' YEAR AS DECIMAL(10,0)):decimal(10,0)>
+-- !query output
+-1
+
+
+-- !query
+select cast(interval '1.000001' second as decimal(10, 6))
+-- !query schema
+struct<CAST(INTERVAL '01.000001' SECOND AS DECIMAL(10,6)):decimal(10,6)>
+-- !query output
+1.000001
+
+
+-- !query
+select cast(interval '08:11:10.001' hour to second as decimal(10, 4))
+-- !query schema
+struct<CAST(INTERVAL '08:11:10.001' HOUR TO SECOND AS DECIMAL(10,4)):decimal(10,4)>
+-- !query output
+29470.0010
+
+
+-- !query
+select cast(interval '1 01:02:03.1' day to second as decimal(8, 1))
+-- !query schema
+struct<CAST(INTERVAL '1 01:02:03.1' DAY TO SECOND AS DECIMAL(8,1)):decimal(8,1)>
+-- !query output
+90123.1
+
+
+-- !query
+select cast(interval '10.123' second as decimal(4, 2))
+-- !query schema
+struct<CAST(INTERVAL '10.123' SECOND AS DECIMAL(4,2)):decimal(4,2)>
+-- !query output
+10.12
+
+
+-- !query
+select cast(interval '10.005' second as decimal(4, 2))
+-- !query schema
+struct<CAST(INTERVAL '10.005' SECOND AS DECIMAL(4,2)):decimal(4,2)>
+-- !query output
+10.01
+
+
+-- !query
+select cast(interval '10.123' second as decimal(5, 2))
+-- !query schema
+struct<CAST(INTERVAL '10.123' SECOND AS DECIMAL(5,2)):decimal(5,2)>
+-- !query output
+10.12
+
+
+-- !query
+select cast(interval '10.123' second as decimal(1, 0))
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkArithmeticException
+{
+ "errorClass" : "NUMERIC_VALUE_OUT_OF_RANGE",
+ "sqlState" : "22003",
+ "messageParameters" : {
+ "config" : "\"spark.sql.ansi.enabled\"",
+ "precision" : "1",
+ "scale" : "0",
+ "value" : "10.123000"
+ }
+}
+
+
+-- !query
+select cast(10.123456BD as interval day to second)
+-- !query schema
+struct<CAST(10.123456 AS INTERVAL DAY TO SECOND):interval day to second>
+-- !query output
+0 00:00:10.123456000
+
+
+-- !query
+select cast(80.654321BD as interval hour to minute)
+-- !query schema
+struct<CAST(80.654321 AS INTERVAL HOUR TO MINUTE):interval hour to minute>
+-- !query output
+0 01:20:00.000000000
+
+
+-- !query
+select cast(-10.123456BD as interval year to month)
+-- !query schema
+struct<CAST(-10.123456 AS INTERVAL YEAR TO MONTH):interval year to month>
+-- !query output
+-0-10
+
+
+-- !query
+select cast(10.654321BD as interval month)
+-- !query schema
+struct<CAST(10.654321 AS INTERVAL MONTH):interval month>
+-- !query output
+0-11
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 28
-
-
-- !query
SELECT CEIL(2.5, 0)
-- !query schema
@@ -96,7 +93,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-The 'scale' parameter of function 'ceil' needs to be a int literal.; line 1 pos 7
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1100",
+ "messageParameters" : {
+ "argName" : "scale",
+ "funcName" : "ceil",
+ "requiredType" : "int"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 22,
+ "fragment" : "CEIL(2.5, null)"
+ } ]
+}
-- !query
@@ -105,7 +116,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-The 'scale' parameter of function 'ceil' needs to be a int literal.; line 1 pos 7
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1100",
+ "messageParameters" : {
+ "argName" : "scale",
+ "funcName" : "ceil",
+ "requiredType" : "int"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 21,
+ "fragment" : "CEIL(2.5, 'a')"
+ } ]
+}
-- !query
@@ -114,7 +139,23 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function ceil. Expected: 2; Found: 3; line 1 pos 7
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "3",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "2",
+ "functionName" : "`ceil`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 22,
+ "fragment" : "CEIL(2.5, 0, 0)"
+ } ]
+}
-- !query
@@ -211,7 +252,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-The 'scale' parameter of function 'floor' needs to be a int literal.; line 1 pos 7
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1100",
+ "messageParameters" : {
+ "argName" : "scale",
+ "funcName" : "floor",
+ "requiredType" : "int"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 23,
+ "fragment" : "FLOOR(2.5, null)"
+ } ]
+}
-- !query
@@ -220,7 +275,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-The 'scale' parameter of function 'floor' needs to be a int literal.; line 1 pos 7
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1100",
+ "messageParameters" : {
+ "argName" : "scale",
+ "funcName" : "floor",
+ "requiredType" : "int"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 22,
+ "fragment" : "FLOOR(2.5, 'a')"
+ } ]
+}
-- !query
@@ -229,4 +298,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function floor. Expected: 2; Found: 3; line 1 pos 7
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "3",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "2",
+ "functionName" : "`floor`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 23,
+ "fragment" : "FLOOR(2.5, 0, 0)"
+ } ]
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/change-column.sql.out sql/core/src/test/resources/sql-tests/results/change-column.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/change-column.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/change-column.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 28
-
-
-- !query
CREATE TABLE test_change(a INT, b STRING, c INT) using parquet
-- !query schema
@@ -26,12 +23,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Operation not allowed: ALTER TABLE table CHANGE COLUMN requires a TYPE, a SET/DROP, a COMMENT, or a FIRST/AFTER(line 1, pos 0)
-
-== SQL ==
-ALTER TABLE test_change CHANGE a
-^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0035",
+ "messageParameters" : {
+ "message" : "ALTER TABLE table CHANGE COLUMN requires a TYPE, a SET/DROP, a COMMENT, or a FIRST/AFTER"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 32,
+ "fragment" : "ALTER TABLE test_change CHANGE a"
+ } ]
+}
-- !query
@@ -50,7 +54,14 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-RENAME COLUMN is only supported with v2 tables.
+{
+ "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "operation" : "RENAME COLUMN",
+ "tableName" : "`spark_catalog`.`default`.`test_change`"
+ }
+}
-- !query
@@ -69,7 +80,15 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-ALTER TABLE CHANGE COLUMN is not supported for changing column 'a' with type 'IntegerType' to 'a' with type 'StringType'
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1245",
+ "messageParameters" : {
+ "newName" : "a",
+ "newType" : "StringType",
+ "originName" : "a",
+ "originType" : "IntegerType"
+ }
+}
-- !query
@@ -88,7 +107,14 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-ALTER COLUMN ... FIRST | ALTER is only supported with v2 tables.
+{
+ "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "operation" : "ALTER COLUMN ... FIRST | ALTER",
+ "tableName" : "`spark_catalog`.`default`.`test_change`"
+ }
+}
-- !query
@@ -97,7 +123,14 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-ALTER COLUMN ... FIRST | ALTER is only supported with v2 tables.
+{
+ "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "operation" : "ALTER COLUMN ... FIRST | ALTER",
+ "tableName" : "`spark_catalog`.`default`.`test_change`"
+ }
+}
-- !query
@@ -176,12 +209,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Missing field invalid_col in table spark_catalog.default.test_change with schema:
-root
- |-- a: integer (nullable = true)
- |-- b: string (nullable = true)
- |-- c: integer (nullable = true)
-; line 1 pos 0
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1331",
+ "messageParameters" : {
+ "fieldName" : "invalid_col",
+ "schema" : "root\n |-- a: integer (nullable = true)\n |-- b: string (nullable = true)\n |-- c: integer (nullable = true)\n",
+ "table" : "spark_catalog.default.test_change"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 51,
+ "fragment" : "ALTER TABLE test_change CHANGE invalid_col TYPE INT"
+ } ]
+}
-- !query
@@ -226,7 +268,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-temp_view is a temp view. 'ALTER TABLE ... CHANGE COLUMN' expects a table.; line 1 pos 12
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1013",
+ "messageParameters" : {
+ "cmd" : "ALTER TABLE ... CHANGE COLUMN",
+ "hintStr" : "",
+ "nameParts" : "temp_view",
+ "viewStr" : "temp view"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 13,
+ "stopIndex" : 21,
+ "fragment" : "temp_view"
+ } ]
+}
-- !query
@@ -243,7 +300,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-global_temp.global_temp_view is a temp view. 'ALTER TABLE ... CHANGE COLUMN' expects a table.; line 1 pos 12
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1013",
+ "messageParameters" : {
+ "cmd" : "ALTER TABLE ... CHANGE COLUMN",
+ "hintStr" : "",
+ "nameParts" : "global_temp.global_temp_view",
+ "viewStr" : "temp view"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 13,
+ "stopIndex" : 40,
+ "fragment" : "global_temp.global_temp_view"
+ } ]
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out 2023-09-21 10:45:41.955461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/charvarchar.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 85
-
-
-- !query
create table char_tbl(c char(5), v varchar(6)) using parquet
-- !query schema
@@ -19,6 +16,7 @@
v varchar(6)
# Detailed Table Information
+Catalog spark_catalog
Database default
Table char_tbl
Created Time [not included in comparison]
@@ -85,6 +83,7 @@
v varchar(6)
# Detailed Table Information
+Catalog spark_catalog
Database default
Table char_tbl2
Created Time [not included in comparison]
@@ -129,6 +128,7 @@
v varchar(6)
# Detailed Table Information
+Catalog spark_catalog
Database default
Table char_tbl3
Created Time [not included in comparison]
@@ -184,6 +184,7 @@
v varchar(6)
# Detailed Table Information
+Catalog spark_catalog
Database default
Table char_view
Created Time [not included in comparison]
@@ -241,6 +242,7 @@
v varchar(6)
# Detailed Table Information
+Catalog spark_catalog
Database default
Table char_tbl1
Created Time [not included in comparison]
@@ -257,7 +259,15 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-ALTER TABLE CHANGE COLUMN is not supported for changing column 'c' with type 'CharType(5)' to 'c' with type 'CharType(6)'
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1245",
+ "messageParameters" : {
+ "newName" : "c",
+ "newType" : "CharType(6)",
+ "originName" : "c",
+ "originType" : "CharType(5)"
+ }
+}
-- !query
@@ -277,6 +287,7 @@
v varchar(6)
# Detailed Table Information
+Catalog spark_catalog
Database default
Table char_tbl1
Created Time [not included in comparison]
@@ -305,6 +316,7 @@
d char(5)
# Detailed Table Information
+Catalog spark_catalog
Database default
Table char_tbl1
Created Time [not included in comparison]
@@ -332,6 +344,7 @@
v varchar(6)
# Detailed Table Information
+Catalog spark_catalog
Database default
Table char_view
Created Time [not included in comparison]
@@ -362,6 +375,7 @@
d char(5)
# Detailed Table Information
+Catalog spark_catalog
Database default
Table char_tbl1
Created Time [not included in comparison]
@@ -390,6 +404,7 @@
v varchar(6)
# Detailed Table Information
+Catalog spark_catalog
Database default
Table char_view
Created Time [not included in comparison]
@@ -421,6 +436,7 @@
d char(5)
# Detailed Table Information
+Catalog spark_catalog
Database default
Table char_tbl1
Created Time [not included in comparison]
@@ -448,6 +464,7 @@
v varchar(6)
# Detailed Table Information
+Catalog spark_catalog
Database default
Table char_view
Created Time [not included in comparison]
@@ -478,6 +495,7 @@
d char(5)
# Detailed Table Information
+Catalog spark_catalog
Database default
Table char_tbl1
Created Time [not included in comparison]
@@ -512,6 +530,7 @@
c2 char(2)
# Detailed Table Information
+Catalog spark_catalog
Database default
Table char_part
Created Time [not included in comparison]
@@ -546,6 +565,7 @@
c2 char(2)
# Detailed Table Information
+Catalog spark_catalog
Database default
Table char_part
Created Time [not included in comparison]
@@ -563,7 +583,14 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Partition spec is invalid. The spec (v2) must match the partition spec (v2, c2) defined in table '`default`.`char_part`'
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1232",
+ "messageParameters" : {
+ "partitionColumnNames" : "v2, c2",
+ "specKeys" : "v2",
+ "tableName" : "`spark_catalog`.`default`.`char_part`"
+ }
+}
-- !query
@@ -581,6 +608,7 @@
c2 char(2)
# Detailed Table Information
+Catalog spark_catalog
Database default
Table char_part
Created Time [not included in comparison]
@@ -615,6 +643,7 @@
c2 char(2)
# Detailed Table Information
+Catalog spark_catalog
Database default
Table char_part
Created Time [not included in comparison]
@@ -649,6 +678,7 @@
c2 char(2)
# Detailed Table Information
+Catalog spark_catalog
Database default
Table char_part
Created Time [not included in comparison]
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/column-resolution-aggregate.sql.out sql/core/src/test/resources/sql-tests/results/column-resolution-aggregate.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/column-resolution-aggregate.sql.out 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/results/column-resolution-aggregate.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -0,0 +1,129 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+CREATE TEMPORARY VIEW v1 AS VALUES (1, 1, 1), (2, 2, 1) AS t(a, b, k)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE TEMPORARY VIEW v2 AS VALUES (1, 1, 1), (2, 2, 1) AS t(x, y, all)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT max(a) AS b, b FROM v1 GROUP BY k
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "MISSING_AGGREGATION",
+ "sqlState" : "42803",
+ "messageParameters" : {
+ "expression" : "\"b\"",
+ "expressionAnyValue" : "\"any_value(b)\""
+ }
+}
+
+
+-- !query
+SELECT a FROM v1 WHERE (12, 13) IN (SELECT max(x + 10) AS a, a + 1 FROM v2)
+-- !query schema
+struct<a:int>
+-- !query output
+1
+2
+
+
+-- !query
+SELECT a AS k FROM v1 GROUP BY k
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "MISSING_AGGREGATION",
+ "sqlState" : "42803",
+ "messageParameters" : {
+ "expression" : "\"a\"",
+ "expressionAnyValue" : "\"any_value(a)\""
+ }
+}
+
+
+-- !query
+SELECT x FROM v2 GROUP BY all
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "MISSING_AGGREGATION",
+ "sqlState" : "42803",
+ "messageParameters" : {
+ "expression" : "\"x\"",
+ "expressionAnyValue" : "\"any_value(x)\""
+ }
+}
+
+
+-- !query
+SELECT a AS all, b FROM v1 GROUP BY all
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "MISSING_AGGREGATION",
+ "sqlState" : "42803",
+ "messageParameters" : {
+ "expression" : "\"b\"",
+ "expressionAnyValue" : "\"any_value(b)\""
+ }
+}
+
+
+-- !query
+SELECT k AS lca, lca + 1 AS col FROM v1 GROUP BY k, col
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_GROUP_BY",
+ "sqlState" : "0A000"
+}
+
+
+-- !query
+SELECT k AS lca, lca + 1 AS col FROM v1 GROUP BY all
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_GROUP_BY",
+ "sqlState" : "0A000"
+}
+
+
+-- !query
+SELECT k AS lca, lca + 1 AS col FROM v1 GROUP BY lca
+-- !query schema
+struct<lca:int,col:int>
+-- !query output
+1 2
+
+
+-- !query
+SELECT * FROM v2 WHERE EXISTS (SELECT a, b FROM v1 GROUP BY all)
+-- !query schema
+struct<x:int,y:int,all:int>
+-- !query output
+1 1 1
+2 2 1
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 28
-
-
-- !query
CREATE DATABASE mydb1
-- !query schema
@@ -72,7 +69,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Reference 'i1' is ambiguous, could be: spark_catalog.mydb1.t1.i1, spark_catalog.mydb1.t1.i1.; line 1 pos 7
+{
+ "errorClass" : "AMBIGUOUS_REFERENCE",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "name" : "`i1`",
+ "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb1`.`t1`.`i1`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 9,
+ "fragment" : "i1"
+ } ]
+}
-- !query
@@ -81,7 +92,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Reference 't1.i1' is ambiguous, could be: spark_catalog.mydb1.t1.i1, spark_catalog.mydb1.t1.i1.; line 1 pos 7
+{
+ "errorClass" : "AMBIGUOUS_REFERENCE",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "name" : "`t1`.`i1`",
+ "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb1`.`t1`.`i1`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 12,
+ "fragment" : "t1.i1"
+ } ]
+}
-- !query
@@ -90,7 +115,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Reference 'mydb1.t1.i1' is ambiguous, could be: spark_catalog.mydb1.t1.i1, spark_catalog.mydb1.t1.i1.; line 1 pos 7
+{
+ "errorClass" : "AMBIGUOUS_REFERENCE",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "name" : "`mydb1`.`t1`.`i1`",
+ "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb1`.`t1`.`i1`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 18,
+ "fragment" : "mydb1.t1.i1"
+ } ]
+}
-- !query
@@ -99,7 +138,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Reference 'i1' is ambiguous, could be: spark_catalog.mydb1.t1.i1, spark_catalog.mydb2.t1.i1.; line 1 pos 7
+{
+ "errorClass" : "AMBIGUOUS_REFERENCE",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "name" : "`i1`",
+ "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 9,
+ "fragment" : "i1"
+ } ]
+}
-- !query
@@ -108,7 +161,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Reference 't1.i1' is ambiguous, could be: spark_catalog.mydb1.t1.i1, spark_catalog.mydb2.t1.i1.; line 1 pos 7
+{
+ "errorClass" : "AMBIGUOUS_REFERENCE",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "name" : "`t1`.`i1`",
+ "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 12,
+ "fragment" : "t1.i1"
+ } ]
+}
-- !query
@@ -125,7 +192,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Reference 'i1' is ambiguous, could be: spark_catalog.mydb2.t1.i1, spark_catalog.mydb1.t1.i1.; line 1 pos 7
+{
+ "errorClass" : "AMBIGUOUS_REFERENCE",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "name" : "`i1`",
+ "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 9,
+ "fragment" : "i1"
+ } ]
+}
-- !query
@@ -134,7 +215,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Reference 't1.i1' is ambiguous, could be: spark_catalog.mydb2.t1.i1, spark_catalog.mydb1.t1.i1.; line 1 pos 7
+{
+ "errorClass" : "AMBIGUOUS_REFERENCE",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "name" : "`t1`.`i1`",
+ "referenceNames" : "[`spark_catalog`.`mydb1`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 12,
+ "fragment" : "t1.i1"
+ } ]
+}
-- !query
@@ -143,7 +238,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Reference 'i1' is ambiguous, could be: spark_catalog.mydb2.t1.i1, spark_catalog.mydb2.t1.i1.; line 1 pos 7
+{
+ "errorClass" : "AMBIGUOUS_REFERENCE",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "name" : "`i1`",
+ "referenceNames" : "[`spark_catalog`.`mydb2`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 9,
+ "fragment" : "i1"
+ } ]
+}
-- !query
@@ -152,7 +261,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Reference 't1.i1' is ambiguous, could be: spark_catalog.mydb2.t1.i1, spark_catalog.mydb2.t1.i1.; line 1 pos 7
+{
+ "errorClass" : "AMBIGUOUS_REFERENCE",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "name" : "`t1`.`i1`",
+ "referenceNames" : "[`spark_catalog`.`mydb2`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 12,
+ "fragment" : "t1.i1"
+ } ]
+}
-- !query
@@ -161,7 +284,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Column 'db1.t1.i1' does not exist. Did you mean one of the following? [spark_catalog.mydb2.t1.i1, spark_catalog.mydb2.t1.i1]; line 1 pos 7
+{
+ "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION",
+ "sqlState" : "42703",
+ "messageParameters" : {
+ "objectName" : "`db1`.`t1`.`i1`",
+ "proposal" : "`spark_catalog`.`mydb2`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 16,
+ "fragment" : "db1.t1.i1"
+ } ]
+}
-- !query
@@ -186,7 +323,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Column 'mydb1.t1' does not exist. Did you mean one of the following? [spark_catalog.mydb1.t1.i1]; line 1 pos 7
+{
+ "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION",
+ "sqlState" : "42703",
+ "messageParameters" : {
+ "objectName" : "`mydb1`.`t1`",
+ "proposal" : "`spark_catalog`.`mydb1`.`t1`.`i1`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 15,
+ "fragment" : "mydb1.t1"
+ } ]
+}
-- !query
@@ -195,7 +346,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 't1.x.y.*' given input columns 'i1'; line 1 pos 7
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1051",
+ "messageParameters" : {
+ "columns" : "i1",
+ "targetString" : "t1.x.y"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 15,
+ "fragment" : "t1.x.y.*"
+ } ]
+}
-- !query
@@ -204,7 +368,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Column 't1' does not exist. Did you mean one of the following? [spark_catalog.mydb1.t1.i1]; line 1 pos 7
+{
+ "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION",
+ "sqlState" : "42703",
+ "messageParameters" : {
+ "objectName" : "`t1`",
+ "proposal" : "`spark_catalog`.`mydb1`.`t1`.`i1`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 9,
+ "fragment" : "t1"
+ } ]
+}
-- !query
@@ -221,7 +399,121 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Column 'mydb1.t1.i1' does not exist. Did you mean one of the following? [spark_catalog.mydb2.t1.i1]; line 1 pos 7
+{
+ "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION",
+ "sqlState" : "42703",
+ "messageParameters" : {
+ "objectName" : "`mydb1`.`t1`.`i1`",
+ "proposal" : "`spark_catalog`.`mydb2`.`t1`.`i1`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 18,
+ "fragment" : "mydb1.t1.i1"
+ } ]
+}
+
+
+-- !query
+USE mydb1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE VIEW v1 AS SELECT * FROM t1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DROP TABLE t1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE TABLE t1 USING parquet AS SELECT 1 AS i2
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM v1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE",
+ "messageParameters" : {
+ "actualCols" : "[]",
+ "colName" : "i1",
+ "expectedNum" : "1",
+ "suggestion" : "CREATE OR REPLACE VIEW spark_catalog.mydb1.v1 AS SELECT * FROM t1",
+ "viewName" : "`spark_catalog`.`mydb1`.`v1`"
+ }
+}
+
+
+-- !query
+USE mydb2
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE TEMP VIEW v2 AS SELECT * FROM t1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DROP TABLE t1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE TABLE t1 USING parquet AS SELECT 1 AS i2
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM v2
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INCOMPATIBLE_VIEW_SCHEMA_CHANGE",
+ "messageParameters" : {
+ "actualCols" : "[]",
+ "colName" : "i1",
+ "expectedNum" : "1",
+ "suggestion" : "CREATE OR REPLACE TEMPORARY VIEW",
+ "viewName" : "`v2`"
+ }
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/column-resolution-sort.sql.out sql/core/src/test/resources/sql-tests/results/column-resolution-sort.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/column-resolution-sort.sql.out 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/results/column-resolution-sort.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -0,0 +1,42 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+CREATE TEMPORARY VIEW v1 AS VALUES (1, 2, 2), (2, 1, 1) AS t(a, b, k)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE TEMPORARY VIEW v2 AS VALUES (1, 2, 2), (2, 1, 1) AS t(a, b, all)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT max(a) AS b FROM v1 GROUP BY k ORDER BY b
+-- !query schema
+struct<b:int>
+-- !query output
+1
+2
+
+
+-- !query
+SELECT max(a) FROM v2 GROUP BY all ORDER BY all
+-- !query schema
+struct<max(a):int>
+-- !query output
+2
+1
+
+
+-- !query
+SELECT (SELECT b FROM v1 ORDER BY all LIMIT 1) FROM v2
+-- !query schema
+struct<scalarsubquery():int>
+-- !query output
+1
+1
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/columnresolution.sql.out sql/core/src/test/resources/sql-tests/results/columnresolution.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/columnresolution.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/columnresolution.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 55
-
-
-- !query
CREATE DATABASE mydb1
-- !query schema
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/columnresolution-views.sql.out sql/core/src/test/resources/sql-tests/results/columnresolution-views.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/columnresolution-views.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/columnresolution-views.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 17
-
-
-- !query
CREATE OR REPLACE TEMPORARY VIEW view1 AS SELECT 2 AS i1
-- !query schema
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/comments.sql.out sql/core/src/test/resources/sql-tests/results/comments.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/comments.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/comments.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 12
-
-
-- !query
/* This is the first example of bracketed comment.
SELECT 'ommented out content' AS first;
@@ -135,20 +132,10 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Unclosed bracketed comment(line 3, pos 0)
-
-== SQL ==
-/*abc*/
-select 1 as a
-/*
-^^^
-
-2 as b
-/*abc*/
-, 3 as c
-
-/**/
+{
+ "errorClass" : "UNCLOSED_BRACKETED_COMMENT",
+ "sqlState" : "42601"
+}
-- !query
@@ -166,18 +153,7 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Unclosed bracketed comment(line 3, pos 0)
-
-== SQL ==
-/*abc*/
-select 1 as a
-/*
-^^^
-
-2 as b
-/*abc*/
-, 3 as c
-
-/**/
-select 4 as d
+{
+ "errorClass" : "UNCLOSED_BRACKETED_COMMENT",
+ "sqlState" : "42601"
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/comparator.sql.out sql/core/src/test/resources/sql-tests/results/comparator.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/comparator.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/comparator.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 10
-
-
-- !query
select x'00' < x'0f'
-- !query schema
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/count.sql.out sql/core/src/test/resources/sql-tests/results/count.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/count.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/count.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 21
-
-
-- !query
CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null)
@@ -149,7 +146,18 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'count()' due to data type mismatch: count requires at least one argument. If you have to call the function count without arguments, set the legacy configuration `spark.sql.legacy.allowParameterlessCount` as true; line 1 pos 7
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITH_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "0",
+ "expectedNum" : " >= 1",
+ "functionName" : "`count`",
+ "legacyConfKey" : "\"spark.sql.legacy.allowParameterlessCount\"",
+ "legacyConfValue" : "\"true\"",
+ "legacyNum" : "0"
+ }
+}
-- !query
@@ -182,4 +190,9 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-count(testData.*) is not allowed. Please use count(*) or expand the columns manually, e.g. count(col1, col2)
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1021",
+ "messageParameters" : {
+ "targetString" : "testData"
+ }
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/cross-join.sql.out sql/core/src/test/resources/sql-tests/results/cross-join.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/cross-join.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/cross-join.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 13
-
-
-- !query
create temporary view nt1 as select * from values
("one", 1),
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 17
-
-
-- !query
select from_csv('1, 3.14', 'a INT, f FLOAT')
-- !query schema
@@ -24,7 +21,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-The expression '1' is not a valid schema string.; line 1 pos 7
+{
+ "errorClass" : "INVALID_SCHEMA.NON_STRING_LITERAL",
+ "sqlState" : "42K07",
+ "messageParameters" : {
+ "inputSchema" : "\"1\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 23,
+ "fragment" : "from_csv('1', 1)"
+ } ]
+}
-- !query
@@ -33,20 +43,44 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Cannot parse the data type:
-Syntax error at or near 'InvalidType': extra input 'InvalidType'(line 1, pos 2)
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'InvalidType'",
+ "hint" : ": extra input 'InvalidType'"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 37,
+ "fragment" : "from_csv('1', 'a InvalidType')"
+ } ]
+}
-== SQL ==
-a InvalidType
---^^^
-
-Failed fallback parsing:
-DataType invalidtype is not supported.(line 1, pos 2)
-
-== SQL ==
-a InvalidType
---^^^
-; line 1 pos 7
+
+-- !query
+select from_csv('1', 'Array<int>')
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "INVALID_SCHEMA.NON_STRUCT_TYPE",
+ "sqlState" : "42K07",
+ "messageParameters" : {
+ "dataType" : "\"ARRAY<INT>\"",
+ "inputSchema" : "\"Array<int>\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "from_csv('1', 'Array<int>')"
+ } ]
+}
-- !query
@@ -55,7 +89,17 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Must use a map() function for options; line 1 pos 7
+{
+ "errorClass" : "INVALID_OPTIONS.NON_MAP_FUNCTION",
+ "sqlState" : "42K06",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 65,
+ "fragment" : "from_csv('1', 'a INT', named_struct('mode', 'PERMISSIVE'))"
+ } ]
+}
-- !query
@@ -64,7 +108,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-A type of keys and values in map() must be string, but got map<string,int>; line 1 pos 7
+{
+ "errorClass" : "INVALID_OPTIONS.NON_STRING_TYPE",
+ "sqlState" : "42K06",
+ "messageParameters" : {
+ "mapType" : "\"MAP<STRING, INT>\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 45,
+ "fragment" : "from_csv('1', 'a INT', map('mode', 1))"
+ } ]
+}
-- !query
@@ -73,7 +130,23 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function from_csv. Expected: one of 2 and 3; Found: 0; line 1 pos 7
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "0",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "[2, 3]",
+ "functionName" : "`from_csv`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 17,
+ "fragment" : "from_csv()"
+ } ]
+}
-- !query
@@ -98,7 +171,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'schema_of_csv(NULL)' due to data type mismatch: The input csv should be a foldable string expression and not null; however, got NULL.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_NULL",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "exprName" : "csv",
+ "sqlExpr" : "\"schema_of_csv(NULL)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 26,
+ "fragment" : "schema_of_csv(null)"
+ } ]
+}
-- !query
@@ -115,7 +202,23 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'schema_of_csv(csvtable.csvField)' due to data type mismatch: The input csv should be a foldable string expression and not null; however, got csvtable.csvField.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputExpr" : "\"csvField\"",
+ "inputName" : "csv",
+ "inputType" : "\"STRING\"",
+ "sqlExpr" : "\"schema_of_csv(csvField)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "schema_of_csv(csvField)"
+ } ]
+}
-- !query
@@ -148,7 +251,17 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Must use a map() function for options; line 1 pos 7
+{
+ "errorClass" : "INVALID_OPTIONS.NON_MAP_FUNCTION",
+ "sqlState" : "42K06",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 79,
+ "fragment" : "to_csv(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE'))"
+ } ]
+}
-- !query
@@ -157,4 +270,17 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-A type of keys and values in map() must be string, but got map<string,int>; line 1 pos 7
+{
+ "errorClass" : "INVALID_OPTIONS.NON_STRING_TYPE",
+ "sqlState" : "42K06",
+ "messageParameters" : {
+ "mapType" : "\"MAP<STRING, INT>\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 59,
+ "fragment" : "to_csv(named_struct('a', 1, 'b', 2), map('mode', 1))"
+ } ]
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 21
-
-
-- !query
WITH t as (
WITH t2 AS (SELECT 1)
@@ -37,6 +34,34 @@
-- !query
+SELECT * FROM
+ (
+ WITH cte AS (SELECT * FROM range(10))
+ SELECT * FROM cte WHERE id = 8
+ ) a
+UNION
+SELECT * FROM cte
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`cte`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 120,
+ "stopIndex" : 122,
+ "fragment" : "cte"
+ } ]
+}
+
+
+-- !query
WITH
t AS (SELECT 1),
t2 AS (
@@ -235,7 +260,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Table or view not found: t1; line 5 pos 20
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`t1`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 73,
+ "stopIndex" : 74,
+ "fragment" : "t1"
+ } ]
+}
-- !query
@@ -273,7 +311,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Table or view not found: cte_outer; line 8 pos 22
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`cte_outer`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 146,
+ "stopIndex" : 154,
+ "fragment" : "cte_outer"
+ } ]
+}
-- !query
@@ -293,7 +344,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Table or view not found: cte_invisible_inner; line 9 pos 18
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`cte_invisible_inner`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 164,
+ "stopIndex" : 182,
+ "fragment" : "cte_invisible_inner"
+ } ]
+}
-- !query
@@ -315,4 +379,17 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Table or view not found: cte_invisible_inner; line 11 pos 18
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`cte_invisible_inner`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 194,
+ "stopIndex" : 212,
+ "fragment" : "cte_invisible_inner"
+ } ]
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/cte-nested.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 21
-
-
-- !query
WITH t as (
WITH t2 AS (SELECT 1)
@@ -37,6 +34,34 @@
-- !query
+SELECT * FROM
+ (
+ WITH cte AS (SELECT * FROM range(10))
+ SELECT * FROM cte WHERE id = 8
+ ) a
+UNION
+SELECT * FROM cte
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`cte`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 120,
+ "stopIndex" : 122,
+ "fragment" : "cte"
+ } ]
+}
+
+
+-- !query
WITH
t AS (SELECT 1),
t2 AS (
@@ -48,7 +73,13 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Name t is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedencePolicy to CORRECTED so that name defined in inner CTE takes precedence. If set it to LEGACY, outer CTE definitions will take precedence. See more details in SPARK-28228.
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1200",
+ "messageParameters" : {
+ "config" : "spark.sql.legacy.ctePrecedencePolicy",
+ "name" : "t"
+ }
+}
-- !query
@@ -85,7 +116,13 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Name t is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedencePolicy to CORRECTED so that name defined in inner CTE takes precedence. If set it to LEGACY, outer CTE definitions will take precedence. See more details in SPARK-28228.
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1200",
+ "messageParameters" : {
+ "config" : "spark.sql.legacy.ctePrecedencePolicy",
+ "name" : "t"
+ }
+}
-- !query
@@ -139,7 +176,13 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Name t is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedencePolicy to CORRECTED so that name defined in inner CTE takes precedence. If set it to LEGACY, outer CTE definitions will take precedence. See more details in SPARK-28228.
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1200",
+ "messageParameters" : {
+ "config" : "spark.sql.legacy.ctePrecedencePolicy",
+ "name" : "t"
+ }
+}
-- !query
@@ -154,7 +197,13 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Name t is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedencePolicy to CORRECTED so that name defined in inner CTE takes precedence. If set it to LEGACY, outer CTE definitions will take precedence. See more details in SPARK-28228.
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1200",
+ "messageParameters" : {
+ "config" : "spark.sql.legacy.ctePrecedencePolicy",
+ "name" : "t"
+ }
+}
-- !query
@@ -170,7 +219,13 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Name t is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedencePolicy to CORRECTED so that name defined in inner CTE takes precedence. If set it to LEGACY, outer CTE definitions will take precedence. See more details in SPARK-28228.
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1200",
+ "messageParameters" : {
+ "config" : "spark.sql.legacy.ctePrecedencePolicy",
+ "name" : "t"
+ }
+}
-- !query
@@ -184,7 +239,13 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Name t is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedencePolicy to CORRECTED so that name defined in inner CTE takes precedence. If set it to LEGACY, outer CTE definitions will take precedence. See more details in SPARK-28228.
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1200",
+ "messageParameters" : {
+ "config" : "spark.sql.legacy.ctePrecedencePolicy",
+ "name" : "t"
+ }
+}
-- !query
@@ -213,7 +274,13 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Name aBc is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedencePolicy to CORRECTED so that name defined in inner CTE takes precedence. If set it to LEGACY, outer CTE definitions will take precedence. See more details in SPARK-28228.
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1200",
+ "messageParameters" : {
+ "config" : "spark.sql.legacy.ctePrecedencePolicy",
+ "name" : "aBc"
+ }
+}
-- !query
@@ -226,7 +293,13 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Name aBc is ambiguous in nested CTE. Please set spark.sql.legacy.ctePrecedencePolicy to CORRECTED so that name defined in inner CTE takes precedence. If set it to LEGACY, outer CTE definitions will take precedence. See more details in SPARK-28228.
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1200",
+ "messageParameters" : {
+ "config" : "spark.sql.legacy.ctePrecedencePolicy",
+ "name" : "aBc"
+ }
+}
-- !query
@@ -299,7 +372,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Table or view not found: cte_invisible_inner; line 9 pos 18
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`cte_invisible_inner`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 164,
+ "stopIndex" : 182,
+ "fragment" : "cte_invisible_inner"
+ } ]
+}
-- !query
@@ -321,4 +407,17 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Table or view not found: cte_invisible_inner; line 11 pos 18
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`cte_invisible_inner`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 194,
+ "stopIndex" : 212,
+ "fragment" : "cte_invisible_inner"
+ } ]
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/cte-nonlegacy.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 21
-
-
-- !query
WITH t as (
WITH t2 AS (SELECT 1)
@@ -37,6 +34,34 @@
-- !query
+SELECT * FROM
+ (
+ WITH cte AS (SELECT * FROM range(10))
+ SELECT * FROM cte WHERE id = 8
+ ) a
+UNION
+SELECT * FROM cte
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`cte`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 120,
+ "stopIndex" : 122,
+ "fragment" : "cte"
+ } ]
+}
+
+
+-- !query
WITH
t AS (SELECT 1),
t2 AS (
@@ -291,7 +316,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Table or view not found: cte_invisible_inner; line 9 pos 18
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`cte_invisible_inner`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 164,
+ "stopIndex" : 182,
+ "fragment" : "cte_invisible_inner"
+ } ]
+}
-- !query
@@ -313,4 +351,17 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Table or view not found: cte_invisible_inner; line 11 pos 18
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`cte_invisible_inner`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 194,
+ "stopIndex" : 212,
+ "fragment" : "cte_invisible_inner"
+ } ]
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/cte.sql.out sql/core/src/test/resources/sql-tests/results/cte.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/cte.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/cte.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 15
-
-
-- !query
create temporary view t as select * from values 0, 1, 2 as t(id)
-- !query schema
@@ -24,7 +21,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Table or view not found: s; line 1 pos 25
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`s`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 26,
+ "stopIndex" : 26,
+ "fragment" : "s"
+ } ]
+}
-- !query
@@ -34,7 +44,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Table or view not found: r; line 1 pos 33
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`r`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 34,
+ "stopIndex" : 34,
+ "fragment" : "r"
+ } ]
+}
-- !query
@@ -53,7 +76,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Table or view not found: s2; line 1 pos 26
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`s2`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 27,
+ "stopIndex" : 28,
+ "fragment" : "s2"
+ } ]
+}
-- !query
@@ -132,13 +168,14 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Syntax error at or near ')'(line 1, pos 7)
-
-== SQL ==
-WITH t() AS (SELECT 1)
--------^^^
-SELECT * FROM t
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "')'",
+ "hint" : ""
+ }
+}
-- !query
@@ -150,15 +187,42 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0038",
+ "messageParameters" : {
+ "duplicateNames" : "'t'"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 63,
+ "fragment" : "WITH\n t(x) AS (SELECT 1),\n t(x) AS (SELECT 2)\nSELECT * FROM t"
+ } ]
+}
-CTE definition can't have duplicate names: 't'.(line 1, pos 0)
-== SQL ==
-WITH
-^^^
- t(x) AS (SELECT 1),
- t(x) AS (SELECT 2)
-SELECT * FROM t
+-- !query
+WITH t AS (SELECT 1 FROM non_existing_table)
+SELECT 2
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`non_existing_table`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 26,
+ "stopIndex" : 43,
+ "fragment" : "non_existing_table"
+ } ]
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/current_database_catalog.sql.out sql/core/src/test/resources/sql-tests/results/current_database_catalog.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/current_database_catalog.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/current_database_catalog.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,10 +1,7 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 1
-
-
-- !query
-select current_database(), current_catalog()
+select current_database(), current_schema(), current_catalog()
-- !query schema
-struct<current_database():string,current_catalog():string>
+struct<current_database():string,current_database():string,current_catalog():string>
-- !query output
-default spark_catalog
+default default spark_catalog
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/date.sql.out sql/core/src/test/resources/sql-tests/results/date.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/date.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/date.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 97
-
-
-- !query
create temporary view date_view as select '2011-11-11' date_str, '1' int_str
-- !query schema
@@ -24,12 +21,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the DATE value: 2020-01-01中文(line 1, pos 7)
-
-== SQL ==
-select date '2020-01-01中文'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2020-01-01中文'",
+ "valueType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 26,
+ "fragment" : "date '2020-01-01中文'"
+ } ]
+}
-- !query
@@ -62,12 +68,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the DATE value: 015(line 1, pos 7)
-
-== SQL ==
-select date'015'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'015'",
+ "valueType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 16,
+ "fragment" : "date'015'"
+ } ]
+}
-- !query
@@ -76,12 +91,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the DATE value: 2021-4294967297-11(line 1, pos 7)
-
-== SQL ==
-select date'2021-4294967297-11'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2021-4294967297-11'",
+ "valueType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 31,
+ "fragment" : "date'2021-4294967297-11'"
+ } ]
+}
-- !query
@@ -101,6 +125,31 @@
-- !query
+select curdate(1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "1",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "0",
+ "functionName" : "`curdate`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 17,
+ "fragment" : "curdate(1)"
+ } ]
+}
+
+
+-- !query
select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null)
-- !query schema
struct<date_from_unix_date(0):date,date_from_unix_date(1000):date,date_from_unix_date(NULL):date>
@@ -284,7 +333,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 1L)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1L' is of bigint type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"BIGINT\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 33,
+ "fragment" : "date_add('2011-11-11', 1L)"
+ } ]
+}
-- !query
@@ -293,7 +359,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 1.0BD)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1.0BD' is of decimal(2,1) type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1.0\"",
+ "inputType" : "\"DECIMAL(2,1)\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, 1.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_add('2011-11-11', 1.0)"
+ } ]
+}
-- !query
@@ -302,7 +385,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 10.0D)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '10.0D' is of double type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"10.0\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, 10.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_add('2011-11-11', 1E1)"
+ } ]
+}
-- !query
@@ -319,7 +419,13 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-The second argument of 'date_add' function needs to be an integer.
+{
+ "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "date_add"
+ }
+}
-- !query
@@ -392,7 +498,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_sub(CAST('2011-11-11' AS DATE), 1L)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1L' is of bigint type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"BIGINT\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 33,
+ "fragment" : "date_sub('2011-11-11', 1L)"
+ } ]
+}
-- !query
@@ -401,7 +524,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_sub(CAST('2011-11-11' AS DATE), 1.0BD)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1.0BD' is of decimal(2,1) type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1.0\"",
+ "inputType" : "\"DECIMAL(2,1)\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, 1.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_sub('2011-11-11', 1.0)"
+ } ]
+}
-- !query
@@ -410,7 +550,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_sub(CAST('2011-11-11' AS DATE), 10.0D)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '10.0D' is of double type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"10.0\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, 10.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_sub('2011-11-11', 1E1)"
+ } ]
+}
-- !query
@@ -427,7 +584,13 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-The second argument of 'date_sub' function needs to be an integer.
+{
+ "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "date_sub"
+ }
+}
-- !query
@@ -468,7 +631,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(CAST('2011-11-11' AS DATE), date_view.int_str)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'date_view.int_str' is of string type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"int_str\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, int_str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 38,
+ "fragment" : "date_add('2011-11-11', int_str)"
+ } ]
+}
-- !query
@@ -477,7 +657,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_sub(CAST('2011-11-11' AS DATE), date_view.int_str)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'date_view.int_str' is of string type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"int_str\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, int_str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 38,
+ "fragment" : "date_sub('2011-11-11', int_str)"
+ } ]
+}
-- !query
@@ -502,7 +699,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(DATE '2011-11-11', 10.0D)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '10.0D' is of double type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"10.0\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(DATE '2011-11-11', 10.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "date '2011-11-11' + 1E1"
+ } ]
+}
-- !query
@@ -543,7 +757,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_sub(DATE '2001-10-01', CAST('2001-09-28' AS DOUBLE))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST('2001-09-28' AS DOUBLE)' is of double type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"2001-09-28\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(DATE '2001-10-01', 2001-09-28)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 39,
+ "fragment" : "date '2001-10-01' - '2001-09-28'"
+ } ]
+}
-- !query
@@ -584,7 +815,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_sub(DATE '2001-09-28', CAST(date_view.date_str AS DOUBLE))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(date_view.date_str AS DOUBLE)' is of double type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"date_str\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(DATE '2001-09-28', date_str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 35,
+ "fragment" : "date '2001-09-28' - date_str"
+ } ]
+}
-- !query
@@ -593,7 +841,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(DATE '2011-11-11', CAST('1' AS DOUBLE))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST('1' AS DOUBLE)' is of double type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "date'2011-11-11' + '1'"
+ } ]
+}
-- !query
@@ -602,7 +867,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(DATE '2011-11-11', CAST('1' AS DOUBLE))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST('1' AS DOUBLE)' is of double type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "'1' + date'2011-11-11'"
+ } ]
+}
-- !query
@@ -640,7 +922,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd/MMMMM/yyyy'"
+ }
+}
-- !query
@@ -649,7 +939,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd/MMMMM/yyyy'"
+ }
+}
-- !query
@@ -658,7 +956,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'dd/MMMMM/yyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'dd/MMMMM/yyyy'"
+ }
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/datetime-formatting-invalid.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,14 +1,19 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 37
-
-
-- !query
select date_format('2018-11-17 13:33:33.333', 'GGGGG')
-- !query schema
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'GGGGG' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'GGGGG'"
+ }
+}
-- !query
@@ -17,7 +22,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'yyyyyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'yyyyyyy'"
+ }
+}
-- !query
@@ -44,7 +57,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'MMMMM' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'MMMMM'"
+ }
+}
-- !query
@@ -53,7 +74,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'LLLLL' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'LLLLL'"
+ }
+}
-- !query
@@ -62,7 +91,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'EEEEE' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'EEEEE'"
+ }
+}
-- !query
@@ -71,7 +108,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'FF' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'FF'"
+ }
+}
-- !query
@@ -80,7 +125,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'ddd' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'ddd'"
+ }
+}
-- !query
@@ -89,7 +142,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'DDDD' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'DDDD'"
+ }
+}
-- !query
@@ -98,7 +159,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'HHH' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'HHH'"
+ }
+}
-- !query
@@ -107,7 +176,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'hhh' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'hhh'"
+ }
+}
-- !query
@@ -116,7 +193,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'kkk' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'kkk'"
+ }
+}
-- !query
@@ -125,7 +210,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'KKK' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'KKK'"
+ }
+}
-- !query
@@ -134,7 +227,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'mmm' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'mmm'"
+ }
+}
-- !query
@@ -143,7 +244,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'sss' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'sss'"
+ }
+}
-- !query
@@ -152,7 +261,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'SSSSSSSSSS' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'SSSSSSSSSS'"
+ }
+}
-- !query
@@ -161,7 +278,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'aa' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'aa'"
+ }
+}
-- !query
@@ -179,7 +304,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'zzzzz' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'zzzzz'"
+ }
+}
-- !query
@@ -197,7 +330,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'ZZZZZZ' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'ZZZZZZ'"
+ }
+}
-- !query
@@ -260,7 +401,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'Y' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'Y'"
+ }
+}
-- !query
@@ -269,7 +418,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'w' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'w'"
+ }
+}
-- !query
@@ -278,7 +435,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'W' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'W'"
+ }
+}
-- !query
@@ -287,7 +452,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'u' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'u'"
+ }
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/datetime-formatting-legacy.sql.out sql/core/src/test/resources/sql-tests/results/datetime-formatting-legacy.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/datetime-formatting-legacy.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/datetime-formatting-legacy.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 31
-
-
-- !query
create temporary view v as select col from values
(timestamp '1582-06-01 11:33:33.123UTC+080000'),
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/datetime-formatting.sql.out sql/core/src/test/resources/sql-tests/results/datetime-formatting.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/datetime-formatting.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/datetime-formatting.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 31
-
-
-- !query
create temporary view v as select col from values
(timestamp '1582-06-01 11:33:33.123UTC+080000'),
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 195
-
-
-- !query
create temporary view date_view as select '2011-11-11' date_str, '1' int_str
-- !query schema
@@ -24,12 +21,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the DATE value: 2020-01-01中文(line 1, pos 7)
-
-== SQL ==
-select date '2020-01-01中文'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2020-01-01中文'",
+ "valueType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 26,
+ "fragment" : "date '2020-01-01中文'"
+ } ]
+}
-- !query
@@ -62,12 +68,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the DATE value: 015(line 1, pos 7)
-
-== SQL ==
-select date'015'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'015'",
+ "valueType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 16,
+ "fragment" : "date'015'"
+ } ]
+}
-- !query
@@ -76,12 +91,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the DATE value: 2021-4294967297-11(line 1, pos 7)
-
-== SQL ==
-select date'2021-4294967297-11'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2021-4294967297-11'",
+ "valueType" : "\"DATE\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 31,
+ "fragment" : "date'2021-4294967297-11'"
+ } ]
+}
-- !query
@@ -101,6 +125,31 @@
-- !query
+select curdate(1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "1",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "0",
+ "functionName" : "`curdate`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 17,
+ "fragment" : "curdate(1)"
+ } ]
+}
+
+
+-- !query
select DATE_FROM_UNIX_DATE(0), DATE_FROM_UNIX_DATE(1000), DATE_FROM_UNIX_DATE(null)
-- !query schema
struct<date_from_unix_date(0):date,date_from_unix_date(1000):date,date_from_unix_date(NULL):date>
@@ -284,7 +333,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 1L)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1L' is of bigint type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"BIGINT\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 33,
+ "fragment" : "date_add('2011-11-11', 1L)"
+ } ]
+}
-- !query
@@ -293,7 +359,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 1.0BD)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1.0BD' is of decimal(2,1) type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1.0\"",
+ "inputType" : "\"DECIMAL(2,1)\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, 1.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_add('2011-11-11', 1.0)"
+ } ]
+}
-- !query
@@ -302,7 +385,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(CAST('2011-11-11' AS DATE), 10.0D)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '10.0D' is of double type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"10.0\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, 10.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_add('2011-11-11', 1E1)"
+ } ]
+}
-- !query
@@ -319,7 +419,13 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-The second argument of 'date_add' function needs to be an integer.
+{
+ "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "date_add"
+ }
+}
-- !query
@@ -392,7 +498,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_sub(CAST('2011-11-11' AS DATE), 1L)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1L' is of bigint type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"BIGINT\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 33,
+ "fragment" : "date_sub('2011-11-11', 1L)"
+ } ]
+}
-- !query
@@ -401,7 +524,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_sub(CAST('2011-11-11' AS DATE), 1.0BD)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '1.0BD' is of decimal(2,1) type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1.0\"",
+ "inputType" : "\"DECIMAL(2,1)\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, 1.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_sub('2011-11-11', 1.0)"
+ } ]
+}
-- !query
@@ -410,7 +550,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_sub(CAST('2011-11-11' AS DATE), 10.0D)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '10.0D' is of double type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"10.0\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, 10.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "date_sub('2011-11-11', 1E1)"
+ } ]
+}
-- !query
@@ -427,7 +584,13 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-The second argument of 'date_sub' function needs to be an integer.
+{
+ "errorClass" : "SECOND_FUNCTION_ARGUMENT_NOT_INTEGER",
+ "sqlState" : "22023",
+ "messageParameters" : {
+ "functionName" : "date_sub"
+ }
+}
-- !query
@@ -468,7 +631,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(CAST('2011-11-11' AS DATE), date_view.int_str)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'date_view.int_str' is of string type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"int_str\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(2011-11-11, int_str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 38,
+ "fragment" : "date_add('2011-11-11', int_str)"
+ } ]
+}
-- !query
@@ -477,7 +657,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_sub(CAST('2011-11-11' AS DATE), date_view.int_str)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'date_view.int_str' is of string type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"int_str\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(2011-11-11, int_str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 38,
+ "fragment" : "date_sub('2011-11-11', int_str)"
+ } ]
+}
-- !query
@@ -502,7 +699,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(DATE '2011-11-11', 10.0D)' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, '10.0D' is of double type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"10.0\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(DATE '2011-11-11', 10.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "date '2011-11-11' + 1E1"
+ } ]
+}
-- !query
@@ -543,7 +757,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_sub(DATE '2001-10-01', CAST('2001-09-28' AS DOUBLE))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST('2001-09-28' AS DOUBLE)' is of double type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"2001-09-28\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(DATE '2001-10-01', 2001-09-28)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 39,
+ "fragment" : "date '2001-10-01' - '2001-09-28'"
+ } ]
+}
-- !query
@@ -584,7 +815,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_sub(DATE '2001-09-28', CAST(date_view.date_str AS DOUBLE))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST(date_view.date_str AS DOUBLE)' is of double type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"date_str\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_sub(DATE '2001-09-28', date_str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 35,
+ "fragment" : "date '2001-09-28' - date_str"
+ } ]
+}
-- !query
@@ -593,7 +841,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(DATE '2011-11-11', CAST('1' AS DOUBLE))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST('1' AS DOUBLE)' is of double type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "date'2011-11-11' + '1'"
+ } ]
+}
-- !query
@@ -602,7 +867,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'date_add(DATE '2011-11-11', CAST('1' AS DOUBLE))' due to data type mismatch: argument 2 requires (int or smallint or tinyint) type, however, 'CAST('1' AS DOUBLE)' is of double type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "2",
+ "requiredType" : "(\"INT\" or \"SMALLINT\" or \"TINYINT\")",
+ "sqlExpr" : "\"date_add(DATE '2011-11-11', 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "'1' + date'2011-11-11'"
+ } ]
+}
-- !query
@@ -832,12 +1114,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the TIMESTAMP value: 2019-01-01中文(line 1, pos 7)
-
-== SQL ==
-select timestamp '2019-01-01中文'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2019-01-01中文'",
+ "valueType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 31,
+ "fragment" : "timestamp '2019-01-01中文'"
+ } ]
+}
-- !query
@@ -846,12 +1137,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the TIMESTAMP value: 4294967297(line 1, pos 7)
-
-== SQL ==
-select timestamp'4294967297'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'4294967297'",
+ "valueType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "timestamp'4294967297'"
+ } ]
+}
-- !query
@@ -860,12 +1160,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the TIMESTAMP value: 2021-01-01T12:30:4294967297.123456(line 1, pos 7)
-
-== SQL ==
-select timestamp'2021-01-01T12:30:4294967297.123456'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'2021-01-01T12:30:4294967297.123456'",
+ "valueType" : "\"TIMESTAMP\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 52,
+ "fragment" : "timestamp'2021-01-01T12:30:4294967297.123456'"
+ } ]
+}
-- !query
@@ -1424,7 +1733,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(TIMESTAMP '2011-11-11 11:11:11' - '2011-11-11 11:11:10')' due to data type mismatch: argument 2 requires (timestamp or timestamp without time zone) type, however, ''2011-11-11 11:11:10'' is of string type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"2011-11-11 11:11:10\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "2",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - 2011-11-11 11:11:10)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 61,
+ "fragment" : "timestamp'2011-11-11 11:11:11' - '2011-11-11 11:11:10'"
+ } ]
+}
-- !query
@@ -1433,7 +1759,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '('2011-11-11 11:11:11' - TIMESTAMP '2011-11-11 11:11:10')' due to data type mismatch: argument 1 requires (timestamp or timestamp without time zone) type, however, ''2011-11-11 11:11:11'' is of string type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"2011-11-11 11:11:11\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"(2011-11-11 11:11:11 - TIMESTAMP '2011-11-11 11:11:10')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 61,
+ "fragment" : "'2011-11-11 11:11:11' - timestamp'2011-11-11 11:11:10'"
+ } ]
+}
-- !query
@@ -1466,7 +1809,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(ts_view.str - TIMESTAMP '2011-11-11 11:11:11')' due to data type mismatch: argument 1 requires (timestamp or timestamp without time zone) type, however, 'ts_view.str' is of string type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"str\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"(str - TIMESTAMP '2011-11-11 11:11:11')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "str - timestamp'2011-11-11 11:11:11'"
+ } ]
+}
-- !query
@@ -1475,7 +1835,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(TIMESTAMP '2011-11-11 11:11:11' - ts_view.str)' due to data type mismatch: argument 2 requires (timestamp or timestamp without time zone) type, however, 'ts_view.str' is of string type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"str\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "2",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' - str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "timestamp'2011-11-11 11:11:11' - str"
+ } ]
+}
-- !query
@@ -1484,7 +1861,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(TIMESTAMP '2011-11-11 11:11:11' + CAST('1' AS DOUBLE))' due to data type mismatch: differing types in '(TIMESTAMP '2011-11-11 11:11:11' + CAST('1' AS DOUBLE))' (timestamp and double).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"TIMESTAMP\"",
+ "right" : "\"DOUBLE\"",
+ "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "timestamp'2011-11-11 11:11:11' + '1'"
+ } ]
+}
-- !query
@@ -1493,7 +1885,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(CAST('1' AS DOUBLE) + TIMESTAMP '2011-11-11 11:11:11')' due to data type mismatch: differing types in '(CAST('1' AS DOUBLE) + TIMESTAMP '2011-11-11 11:11:11')' (double and timestamp).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"DOUBLE\"",
+ "right" : "\"TIMESTAMP\"",
+ "sqlExpr" : "\"(1 + TIMESTAMP '2011-11-11 11:11:11')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 43,
+ "fragment" : "'1' + timestamp'2011-11-11 11:11:11'"
+ } ]
+}
-- !query
@@ -1502,7 +1909,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(TIMESTAMP '2011-11-11 11:11:11' + NULL)' due to data type mismatch: differing types in '(TIMESTAMP '2011-11-11 11:11:11' + NULL)' (timestamp and void).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"TIMESTAMP\"",
+ "right" : "\"VOID\"",
+ "sqlExpr" : "\"(TIMESTAMP '2011-11-11 11:11:11' + NULL)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 44,
+ "fragment" : "timestamp'2011-11-11 11:11:11' + null"
+ } ]
+}
-- !query
@@ -1511,7 +1933,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(NULL + TIMESTAMP '2011-11-11 11:11:11')' due to data type mismatch: differing types in '(NULL + TIMESTAMP '2011-11-11 11:11:11')' (void and timestamp).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"VOID\"",
+ "right" : "\"TIMESTAMP\"",
+ "sqlExpr" : "\"(NULL + TIMESTAMP '2011-11-11 11:11:11')\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 44,
+ "fragment" : "null + timestamp'2011-11-11 11:11:11'"
+ } ]
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/datetime-parsing-invalid.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 28
-
-
-- !query
select to_timestamp('294248', 'y')
-- !query schema
@@ -17,7 +14,14 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "datetime" : "'1'"
+ }
+}
-- !query
@@ -34,7 +38,14 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '123' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "datetime" : "'123'"
+ }
+}
-- !query
@@ -43,7 +54,14 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '1' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "datetime" : "'1'"
+ }
+}
-- !query
@@ -52,7 +70,15 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to recognize 'yyyyyyy' pattern in the DateTimeFormatter. 1) You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0. 2) You can form a valid datetime pattern with the guide from https://spark.apache.org/docs/latest/sql-ref-datetime-pattern.html
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.DATETIME_PATTERN_RECOGNITION",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "pattern" : "'yyyyyyy'"
+ }
+}
-- !query
@@ -69,7 +95,14 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '9' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "datetime" : "'9'"
+ }
+}
-- !query
@@ -78,7 +111,14 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '9' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "datetime" : "'9'"
+ }
+}
-- !query
@@ -87,7 +127,14 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '99' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "datetime" : "'99'"
+ }
+}
-- !query
@@ -152,7 +199,14 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '2018-366' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "datetime" : "'2018-366'"
+ }
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/datetime-parsing-legacy.sql.out sql/core/src/test/resources/sql-tests/results/datetime-parsing-legacy.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/datetime-parsing-legacy.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/datetime-parsing-legacy.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 25
-
-
-- !query
select to_timestamp('1', 'y')
-- !query schema
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/datetime-parsing.sql.out sql/core/src/test/resources/sql-tests/results/datetime-parsing.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/datetime-parsing.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/datetime-parsing.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 25
-
-
-- !query
select to_timestamp('1', 'y')
-- !query schema
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/datetime-special.sql.out sql/core/src/test/resources/sql-tests/results/datetime-special.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/datetime-special.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/datetime-special.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 4
-
-
-- !query
select date'999999-03-18', date'-0001-1-28', date'0015'
-- !query schema
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 40
-
-
-- !query
CREATE TEMPORARY VIEW t AS SELECT 1.0 as a, 0.0 as b
-- !query schema
@@ -186,6 +183,83 @@
-- !query
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(3, 1)) + CAST(90 AS DECIMAL(3, 1))
+-- !query schema
+struct<CAST(20 AS DECIMAL(4,1)):decimal(4,1)>
+-- !query output
+100.0
+20.0
+
+
+-- !query
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(3, 1)) - CAST(-90 AS DECIMAL(3, 1))
+-- !query schema
+struct<CAST(20 AS DECIMAL(4,1)):decimal(4,1)>
+-- !query output
+100.0
+20.0
+
+
+-- !query
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(3, 1)) * CAST(10 AS DECIMAL(3, 1))
+-- !query schema
+struct<CAST(20 AS DECIMAL(4,1)):decimal(7,2)>
+-- !query output
+100.00
+20.00
+
+
+-- !query
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(3, 1)) / CAST(10 AS DECIMAL(3, 1))
+-- !query schema
+struct<CAST(20 AS DECIMAL(4,1)):decimal(9,6)>
+-- !query output
+1.000000
+20.000000
+
+
+-- !query
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(10, 2)) % CAST(3 AS DECIMAL(5, 1))
+-- !query schema
+struct<CAST(20 AS DECIMAL(4,1)):decimal(6,2)>
+-- !query output
+1.00
+20.00
+
+
+-- !query
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT pmod(CAST(10 AS DECIMAL(10, 2)), CAST(3 AS DECIMAL(5, 1)))
+-- !query schema
+struct<CAST(20 AS DECIMAL(4,1)):decimal(6,2)>
+-- !query output
+1.00
+20.00
+
+
+-- !query
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(10, 2)) div CAST(3 AS DECIMAL(5, 1))
+-- !query schema
+struct<CAST(20 AS DECIMAL(4,1)):decimal(21,1)>
+-- !query output
+20.0
+3.0
+
+
+-- !query
set spark.sql.decimalOperations.allowPrecisionLoss=false
-- !query schema
struct<key:string,value:string>
@@ -328,6 +402,70 @@
-- !query
+select 1.0123456789012345678901234567890123456e36BD / 0.1
+-- !query schema
+struct<(1012345678901234567890123456789012345.6 / 0.1):decimal(38,2)>
+-- !query output
+NULL
+
+
+-- !query
+select 1.0123456789012345678901234567890123456e35BD / 1.0
+-- !query schema
+struct<(101234567890123456789012345678901234.56 / 1.0):decimal(38,3)>
+-- !query output
+NULL
+
+
+-- !query
+select 1.0123456789012345678901234567890123456e34BD / 1.0
+-- !query schema
+struct<(10123456789012345678901234567890123.456 / 1.0):decimal(38,3)>
+-- !query output
+10123456789012345678901234567890123.456
+
+
+-- !query
+select 1.0123456789012345678901234567890123456e33BD / 1.0
+-- !query schema
+struct<(1012345678901234567890123456789012.3456 / 1.0):decimal(38,4)>
+-- !query output
+1012345678901234567890123456789012.3456
+
+
+-- !query
+select 1.0123456789012345678901234567890123456e32BD / 1.0
+-- !query schema
+struct<(101234567890123456789012345678901.23456 / 1.0):decimal(38,5)>
+-- !query output
+101234567890123456789012345678901.23456
+
+
+-- !query
+select 1.0123456789012345678901234567890123456e31BD / 1.0
+-- !query schema
+struct<(10123456789012345678901234567890.123456 / 1.0):decimal(38,6)>
+-- !query output
+10123456789012345678901234567890.123456
+
+
+-- !query
+select 1.0123456789012345678901234567890123456e31BD / 0.1
+-- !query schema
+struct<(10123456789012345678901234567890.123456 / 0.1):decimal(38,6)>
+-- !query output
+NULL
+
+
+-- !query
+select 1.0123456789012345678901234567890123456e31BD / 10.0
+-- !query schema
+struct<(10123456789012345678901234567890.123456 / 10.0):decimal(38,7)>
+-- !query output
+1012345678901234567890123456789.0123456
+
+
+-- !query
drop table decimals_test
-- !query schema
struct<>
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 15
-
-
-- !query
CREATE TABLE t (key STRING, value STRING, ds STRING, hr INT) USING parquet
PARTITIONED BY (ds, hr)
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/describe-query.sql.out sql/core/src/test/resources/sql-tests/results/describe-query.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/describe-query.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/describe-query.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 19
-
-
-- !query
CREATE table desc_temp1 (key int COMMENT 'column_comment', val string) USING PARQUET
-- !query schema
@@ -111,12 +108,14 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Syntax error at or near 'desc_temp1'(line 1, pos 21)
-
-== SQL ==
-DESCRIBE INSERT INTO desc_temp1 values (1, 'val1')
----------------------^^^
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'desc_temp1'",
+ "hint" : ""
+ }
+}
-- !query
@@ -125,12 +124,14 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Syntax error at or near 'desc_temp1'(line 1, pos 21)
-
-== SQL ==
-DESCRIBE INSERT INTO desc_temp1 SELECT * FROM desc_temp2
----------------------^^^
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'desc_temp1'",
+ "hint" : ""
+ }
+}
-- !query
@@ -142,15 +143,14 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Syntax error at or near 'insert'(line 3, pos 5)
-
-== SQL ==
-DESCRIBE
- FROM desc_temp1 a
- insert into desc_temp1 select *
------^^^
- insert into desc_temp2 select *
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'insert'",
+ "hint" : ""
+ }
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/describe.sql.out sql/core/src/test/resources/sql-tests/results/describe.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/describe.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/describe.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,13 +1,10 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 42
-
-
-- !query
CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet
- OPTIONS (a '1', b '2')
+ OPTIONS (a '1', b '2', password 'password')
PARTITIONED BY (c, d) CLUSTERED BY (a) SORTED BY (b ASC) INTO 2 BUCKETS
COMMENT 'table_comment'
- TBLPROPERTIES (t 'test')
+ TBLPROPERTIES (t 'test', password 'password')
-- !query schema
struct<>
-- !query output
@@ -119,6 +116,7 @@
d string
# Detailed Table Information
+Catalog spark_catalog
Database default
Table t
Created Time [not included in comparison]
@@ -130,9 +128,9 @@
Bucket Columns [`a`]
Sort Columns [`b`]
Comment table_comment
-Table Properties [e=3, t=test]
+Table Properties [e=3, password=*********(redacted), t=test]
Location [not included in comparison]/{warehouse_dir}/t
-Storage Properties [a=1, b=2]
+Storage Properties [a=1, b=2, password=*********(redacted)]
Partition Provider Catalog
@@ -151,6 +149,7 @@
d string
# Detailed Table Information
+Catalog spark_catalog
Database default
Table t
Created Time [not included in comparison]
@@ -162,9 +161,9 @@
Bucket Columns [`a`]
Sort Columns [`b`]
Comment table_comment
-Table Properties [e=3, t=test]
+Table Properties [e=3, password=*********(redacted), t=test]
Location [not included in comparison]/{warehouse_dir}/t
-Storage Properties [a=1, b=2]
+Storage Properties [a=1, b=2, password=*********(redacted)]
Partition Provider Catalog
@@ -191,6 +190,7 @@
d string
# Detailed Table Information
+Catalog spark_catalog
Database default
Table t
Created Time [not included in comparison]
@@ -202,9 +202,9 @@
Bucket Columns [`a`]
Sort Columns [`b`]
Comment table_comment
-Table Properties [t=test]
+Table Properties [password=*********(redacted), t=test]
Location [not included in comparison]/{warehouse_dir}/t
-Storage Properties [a=1, b=2]
+Storage Properties [a=1, b=2, password=*********(redacted)]
Partition Provider Catalog
@@ -231,6 +231,7 @@
d string
# Detailed Table Information
+Catalog spark_catalog
Database default
Table t
Created Time [not included in comparison]
@@ -241,9 +242,9 @@
Num Buckets 2
Bucket Columns [`a`]
Sort Columns [`b`]
-Table Properties [t=test]
+Table Properties [password=*********(redacted), t=test]
Location [not included in comparison]/{warehouse_dir}/t
-Storage Properties [a=1, b=2]
+Storage Properties [a=1, b=2, password=*********(redacted)]
Partition Provider Catalog
@@ -281,7 +282,7 @@
Table t
Partition Values [c=Us, d=1]
Location [not included in comparison]/{warehouse_dir}/t/c=Us/d=1
-Storage Properties [a=1, b=2]
+Storage Properties [a=1, b=2, password=*********(redacted)]
Created Time [not included in comparison]
Last Access [not included in comparison]
@@ -290,7 +291,7 @@
Bucket Columns [`a`]
Sort Columns [`b`]
Location [not included in comparison]/{warehouse_dir}/t
-Storage Properties [a=1, b=2]
+Storage Properties [a=1, b=2, password=*********(redacted)]
-- !query
@@ -312,7 +313,7 @@
Table t
Partition Values [c=Us, d=1]
Location [not included in comparison]/{warehouse_dir}/t/c=Us/d=1
-Storage Properties [a=1, b=2]
+Storage Properties [a=1, b=2, password=*********(redacted)]
Created Time [not included in comparison]
Last Access [not included in comparison]
@@ -321,7 +322,7 @@
Bucket Columns [`a`]
Sort Columns [`b`]
Location [not included in comparison]/{warehouse_dir}/t
-Storage Properties [a=1, b=2]
+Storage Properties [a=1, b=2, password=*********(redacted)]
-- !query
@@ -343,7 +344,7 @@
Table t
Partition Values [c=Us, d=1]
Location [not included in comparison]/{warehouse_dir}/t/c=Us/d=1
-Storage Properties [a=1, b=2]
+Storage Properties [a=1, b=2, password=*********(redacted)]
Created Time [not included in comparison]
Last Access [not included in comparison]
@@ -352,7 +353,7 @@
Bucket Columns [`a`]
Sort Columns [`b`]
Location [not included in comparison]/{warehouse_dir}/t
-Storage Properties [a=1, b=2]
+Storage Properties [a=1, b=2, password=*********(redacted)]
-- !query
@@ -361,9 +362,14 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException
-Partition not found in table 't' database 'default':
-c -> Us
-d -> 2
+{
+ "errorClass" : "PARTITIONS_NOT_FOUND",
+ "sqlState" : "428FT",
+ "messageParameters" : {
+ "partitionList" : "PARTITION (`c` = Us, `d` = 2)",
+ "tableName" : "`default`.`t`"
+ }
+}
-- !query
@@ -372,7 +378,14 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Partition spec is invalid. The spec (c) must match the partition spec (c, d) defined in table '`default`.`t`'
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1232",
+ "messageParameters" : {
+ "partitionColumnNames" : "c, d",
+ "specKeys" : "c",
+ "tableName" : "`spark_catalog`.`default`.`t`"
+ }
+}
-- !query
@@ -381,12 +394,20 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-PARTITION specification is incomplete: `d`(line 1, pos 0)
-
-== SQL ==
-DESC t PARTITION (c='Us', d)
-^^^
+{
+ "errorClass" : "INVALID_SQL_SYNTAX",
+ "sqlState" : "42000",
+ "messageParameters" : {
+ "inputString" : "PARTITION specification is incomplete: `d`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 28,
+ "fragment" : "DESC t PARTITION (c='Us', d)"
+ } ]
+}
-- !query
@@ -462,7 +483,15 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-DESC PARTITION is not allowed on a temporary view: temp_v
+{
+ "errorClass" : "FORBIDDEN_OPERATION",
+ "sqlState" : "42809",
+ "messageParameters" : {
+ "objectName" : "`temp_v`",
+ "objectType" : "TEMPORARY VIEW",
+ "statement" : "DESC PARTITION"
+ }
+}
-- !query
@@ -498,6 +527,7 @@
d string
# Detailed Table Information
+Catalog spark_catalog
Database default
Table v
Created Time [not included in comparison]
@@ -521,6 +551,7 @@
d string
# Detailed Table Information
+Catalog spark_catalog
Database default
Table v
Created Time [not included in comparison]
@@ -539,7 +570,15 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-DESC PARTITION is not allowed on a view: v
+{
+ "errorClass" : "FORBIDDEN_OPERATION",
+ "sqlState" : "42809",
+ "messageParameters" : {
+ "objectName" : "`v`",
+ "objectType" : "VIEW",
+ "statement" : "DESC PARTITION"
+ }
+}
-- !query
@@ -549,7 +588,7 @@
-- !query output
== Physical Plan ==
Execute DescribeTableCommand
- +- DescribeTableCommand `default`.`t`, false, [col_name#x, data_type#x, comment#x]
+ +- DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x]
-- !query
@@ -559,7 +598,7 @@
-- !query output
== Physical Plan ==
Execute DescribeTableCommand
- +- DescribeTableCommand `default`.`t`, true, [col_name#x, data_type#x, comment#x]
+ +- DescribeTableCommand `spark_catalog`.`default`.`t`, true, [col_name#x, data_type#x, comment#x]
-- !query
@@ -573,14 +612,14 @@
== Analyzed Logical Plan ==
col_name: string, data_type: string, comment: string
-DescribeTableCommand `default`.`t`, false, [col_name#x, data_type#x, comment#x]
+DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x]
== Optimized Logical Plan ==
-DescribeTableCommand `default`.`t`, false, [col_name#x, data_type#x, comment#x]
+DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x]
== Physical Plan ==
Execute DescribeTableCommand
- +- DescribeTableCommand `default`.`t`, false, [col_name#x, data_type#x, comment#x]
+ +- DescribeTableCommand `spark_catalog`.`default`.`t`, false, [col_name#x, data_type#x, comment#x]
-- !query
@@ -590,7 +629,7 @@
-- !query output
== Physical Plan ==
Execute DescribeColumnCommand
- +- DescribeColumnCommand `default`.`t`, [spark_catalog, default, t, b], false, [info_name#x, info_value#x]
+ +- DescribeColumnCommand `spark_catalog`.`default`.`t`, [spark_catalog, default, t, b], false, [info_name#x, info_value#x]
-- !query
@@ -600,7 +639,7 @@
-- !query output
== Physical Plan ==
Execute DescribeTableCommand
- +- DescribeTableCommand `default`.`t`, [c=Us, d=2], false, [col_name#x, data_type#x, comment#x]
+ +- DescribeTableCommand `spark_catalog`.`default`.`t`, [c=Us, d=2], false, [col_name#x, data_type#x, comment#x]
-- !query
@@ -633,3 +672,195 @@
struct<>
-- !query output
+
+
+-- !query
+CREATE TABLE d (a STRING DEFAULT 'default-value', b INT DEFAULT 42) USING parquet COMMENT 'table_comment'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DESC d
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+a string
+b int
+
+# Column Default Values
+a string 'default-value'
+b int 42
+
+
+-- !query
+DESC EXTENDED d
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+a string
+b int
+
+# Detailed Table Information
+Catalog spark_catalog
+Database default
+Table d
+Created Time [not included in comparison]
+Last Access [not included in comparison]
+Created By [not included in comparison]
+Type MANAGED
+Provider parquet
+Comment table_comment
+Location [not included in comparison]/{warehouse_dir}/d
+
+# Column Default Values
+a string 'default-value'
+b int 42
+
+
+-- !query
+DESC TABLE EXTENDED d
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+a string
+b int
+
+# Detailed Table Information
+Catalog spark_catalog
+Database default
+Table d
+Created Time [not included in comparison]
+Last Access [not included in comparison]
+Created By [not included in comparison]
+Type MANAGED
+Provider parquet
+Comment table_comment
+Location [not included in comparison]/{warehouse_dir}/d
+
+# Column Default Values
+a string 'default-value'
+b int 42
+
+
+-- !query
+DESC FORMATTED d
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+a string
+b int
+
+# Detailed Table Information
+Catalog spark_catalog
+Database default
+Table d
+Created Time [not included in comparison]
+Last Access [not included in comparison]
+Created By [not included in comparison]
+Type MANAGED
+Provider parquet
+Comment table_comment
+Location [not included in comparison]/{warehouse_dir}/d
+
+# Column Default Values
+a string 'default-value'
+b int 42
+
+
+-- !query
+CREATE TABLE e (a STRING DEFAULT CONCAT('a\n b\n ', 'c\n d'), b INT DEFAULT 42) USING parquet COMMENT 'table_comment'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DESC e
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+a string
+b int
+
+# Column Default Values
+a string CONCAT('a\n b\n ', 'c\n d')
+b int 42
+
+
+-- !query
+DESC EXTENDED e
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+a string
+b int
+
+# Detailed Table Information
+Catalog spark_catalog
+Database default
+Table e
+Created Time [not included in comparison]
+Last Access [not included in comparison]
+Created By [not included in comparison]
+Type MANAGED
+Provider parquet
+Comment table_comment
+Location [not included in comparison]/{warehouse_dir}/e
+
+# Column Default Values
+a string CONCAT('a\n b\n ', 'c\n d')
+b int 42
+
+
+-- !query
+DESC TABLE EXTENDED e
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+a string
+b int
+
+# Detailed Table Information
+Catalog spark_catalog
+Database default
+Table e
+Created Time [not included in comparison]
+Last Access [not included in comparison]
+Created By [not included in comparison]
+Type MANAGED
+Provider parquet
+Comment table_comment
+Location [not included in comparison]/{warehouse_dir}/e
+
+# Column Default Values
+a string CONCAT('a\n b\n ', 'c\n d')
+b int 42
+
+
+-- !query
+DESC FORMATTED e
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+a string
+b int
+
+# Detailed Table Information
+Catalog spark_catalog
+Database default
+Table e
+Created Time [not included in comparison]
+Last Access [not included in comparison]
+Created By [not included in comparison]
+Type MANAGED
+Provider parquet
+Comment table_comment
+Location [not included in comparison]/{warehouse_dir}/e
+
+# Column Default Values
+a string CONCAT('a\n b\n ', 'c\n d')
+b int 42
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/describe-table-after-alter-table.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 12
-
-
-- !query
CREATE TABLE table_with_comment (a STRING, b INT, c STRING, d STRING) USING parquet COMMENT 'added'
-- !query schema
@@ -21,6 +18,7 @@
d string
# Detailed Table Information
+Catalog spark_catalog
Database default
Table table_with_comment
Created Time [not included in comparison]
@@ -51,6 +49,7 @@
d string
# Detailed Table Information
+Catalog spark_catalog
Database default
Table table_with_comment
Created Time [not included in comparison]
@@ -88,6 +87,7 @@
b int
# Detailed Table Information
+Catalog spark_catalog
Database default
Table table_comment
Created Time [not included in comparison]
@@ -115,6 +115,7 @@
b int
# Detailed Table Information
+Catalog spark_catalog
Database default
Table table_comment
Created Time [not included in comparison]
@@ -143,6 +144,7 @@
b int
# Detailed Table Information
+Catalog spark_catalog
Database default
Table table_comment
Created Time [not included in comparison]
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/describe-table-column.sql.out sql/core/src/test/resources/sql-tests/results/describe-table-column.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/describe-table-column.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/describe-table-column.sql.out 1970-01-01 08:00:00.000000000 +0800
@@ -1,333 +0,0 @@
--- Automatically generated by SQLQueryTestSuite
--- Number of queries: 30
-
-
--- !query
-CREATE TEMPORARY VIEW desc_col_temp_view (key int COMMENT 'column_comment', col struct<x:int, y:string>) USING PARQUET
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-DESC desc_col_temp_view key
--- !query schema
-struct<info_name:string,info_value:string>
--- !query output
-col_name key
-data_type int
-comment column_comment
-
-
--- !query
-DESC EXTENDED desc_col_temp_view key
--- !query schema
-struct<info_name:string,info_value:string>
--- !query output
-col_name key
-data_type int
-comment column_comment
-min NULL
-max NULL
-num_nulls NULL
-distinct_count NULL
-avg_col_len NULL
-max_col_len NULL
-histogram NULL
-
-
--- !query
-DESC FORMATTED desc_col_temp_view key
--- !query schema
-struct<info_name:string,info_value:string>
--- !query output
-col_name key
-data_type int
-comment column_comment
-min NULL
-max NULL
-num_nulls NULL
-distinct_count NULL
-avg_col_len NULL
-max_col_len NULL
-histogram NULL
-
-
--- !query
-DESC FORMATTED desc_col_temp_view desc_col_temp_view.key
--- !query schema
-struct<info_name:string,info_value:string>
--- !query output
-col_name key
-data_type int
-comment column_comment
-min NULL
-max NULL
-num_nulls NULL
-distinct_count NULL
-avg_col_len NULL
-max_col_len NULL
-histogram NULL
-
-
--- !query
-DESC desc_col_temp_view key1
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.AnalysisException
-Column key1 does not exist
-
-
--- !query
-DESC desc_col_temp_view col.x
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.AnalysisException
-DESC TABLE COLUMN does not support nested column: col.x
-
-
--- !query
-CREATE TABLE desc_col_table (key int COMMENT 'column_comment') USING PARQUET
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-ANALYZE TABLE desc_col_table COMPUTE STATISTICS FOR COLUMNS key
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-DESC desc_col_table key
--- !query schema
-struct<info_name:string,info_value:string>
--- !query output
-col_name key
-data_type int
-comment column_comment
-
-
--- !query
-DESC EXTENDED desc_col_table key
--- !query schema
-struct<info_name:string,info_value:string>
--- !query output
-col_name key
-data_type int
-comment column_comment
-min NULL
-max NULL
-num_nulls 0
-distinct_count 0
-avg_col_len 4
-max_col_len 4
-histogram NULL
-
-
--- !query
-DESC FORMATTED desc_col_table key
--- !query schema
-struct<info_name:string,info_value:string>
--- !query output
-col_name key
-data_type int
-comment column_comment
-min NULL
-max NULL
-num_nulls 0
-distinct_count 0
-avg_col_len 4
-max_col_len 4
-histogram NULL
-
-
--- !query
-DESC desc_col_table key1
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.AnalysisException
-Column key1 does not exist
-
-
--- !query
-CREATE TABLE desc_complex_col_table (`a.b` int, col struct<x:int, y:string>) USING PARQUET
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-DESC FORMATTED desc_complex_col_table `a.b`
--- !query schema
-struct<info_name:string,info_value:string>
--- !query output
-col_name a.b
-data_type int
-comment NULL
-min NULL
-max NULL
-num_nulls NULL
-distinct_count NULL
-avg_col_len NULL
-max_col_len NULL
-histogram NULL
-
-
--- !query
-DESC FORMATTED desc_complex_col_table col
--- !query schema
-struct<info_name:string,info_value:string>
--- !query output
-col_name col
-data_type struct<x:int,y:string>
-comment NULL
-min NULL
-max NULL
-num_nulls NULL
-distinct_count NULL
-avg_col_len NULL
-max_col_len NULL
-histogram NULL
-
-
--- !query
-DESC FORMATTED desc_complex_col_table col.x
--- !query schema
-struct<>
--- !query output
-org.apache.spark.sql.AnalysisException
-DESC TABLE COLUMN does not support nested column: col.x
-
-
--- !query
-SET spark.sql.statistics.histogram.enabled=true
--- !query schema
-struct<key:string,value:string>
--- !query output
-spark.sql.statistics.histogram.enabled true
-
-
--- !query
-SET spark.sql.statistics.histogram.numBins=2
--- !query schema
-struct<key:string,value:string>
--- !query output
-spark.sql.statistics.histogram.numBins 2
-
-
--- !query
-INSERT INTO desc_col_table values 1, 2, 3, 4
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-ANALYZE TABLE desc_col_table COMPUTE STATISTICS FOR COLUMNS key
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-DESC EXTENDED desc_col_table key
--- !query schema
-struct<info_name:string,info_value:string>
--- !query output
-col_name key
-data_type int
-comment column_comment
-min 1
-max 4
-num_nulls 0
-distinct_count 4
-avg_col_len 4
-max_col_len 4
-histogram height: 2.0, num_of_bins: 2
-bin_0 lower_bound: 1.0, upper_bound: 2.0, distinct_count: 2
-bin_1 lower_bound: 2.0, upper_bound: 4.0, distinct_count: 2
-
-
--- !query
-DROP VIEW desc_col_temp_view
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-DROP TABLE desc_col_table
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-DROP TABLE desc_complex_col_table
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-CREATE TABLE customer(CName STRING) USING PARQUET
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-INSERT INTO customer VALUES('Maria')
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-ANALYZE TABLE customer COMPUTE STATISTICS FOR COLUMNS cname
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-DESC EXTENDED customer cname
--- !query schema
-struct<info_name:string,info_value:string>
--- !query output
-col_name cname
-data_type string
-comment NULL
-min NULL
-max NULL
-num_nulls 0
-distinct_count 1
-avg_col_len 5
-max_col_len 5
-histogram NULL
-
-
--- !query
-DROP TABLE customer
--- !query schema
-struct<>
--- !query output
-
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers.sql.out sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers.sql.out 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -0,0 +1,443 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+SELECT 1 FROM "not_exist"
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"not_exist\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+USE SCHEMA "not_exist"
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"not_exist\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+ALTER TABLE "not_exist" ADD COLUMN not_exist int
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"not_exist\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+ALTER TABLE not_exist ADD COLUMN "not_exist" int
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"not_exist\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+SELECT 1 AS "not_exist" FROM not_exist
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"not_exist\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+SELECT 1 FROM not_exist AS X("hello")
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"hello\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+SELECT "not_exist"()
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"not_exist\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+SELECT "not_exist".not_exist()
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"not_exist\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+SELECT 1 FROM `hello`
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`hello`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 21,
+ "fragment" : "`hello`"
+ } ]
+}
+
+
+-- !query
+USE SCHEMA `not_exist`
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException
+{
+ "errorClass" : "SCHEMA_NOT_FOUND",
+ "sqlState" : "42704",
+ "messageParameters" : {
+ "schemaName" : "`not_exist`"
+ }
+}
+
+
+-- !query
+ALTER TABLE `not_exist` ADD COLUMN not_exist int
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`not_exist`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 13,
+ "stopIndex" : 23,
+ "fragment" : "`not_exist`"
+ } ]
+}
+
+
+-- !query
+ALTER TABLE not_exist ADD COLUMN `not_exist` int
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`not_exist`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 13,
+ "stopIndex" : 21,
+ "fragment" : "not_exist"
+ } ]
+}
+
+
+-- !query
+SELECT 1 AS `not_exist` FROM `not_exist`
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`not_exist`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 30,
+ "stopIndex" : 40,
+ "fragment" : "`not_exist`"
+ } ]
+}
+
+
+-- !query
+SELECT 1 FROM not_exist AS X(`hello`)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "TABLE_OR_VIEW_NOT_FOUND",
+ "sqlState" : "42P01",
+ "messageParameters" : {
+ "relationName" : "`not_exist`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 37,
+ "fragment" : "not_exist AS X(`hello`)"
+ } ]
+}
+
+
+-- !query
+SELECT `not_exist`()
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVED_ROUTINE",
+ "sqlState" : "42883",
+ "messageParameters" : {
+ "routineName" : "`not_exist`",
+ "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 20,
+ "fragment" : "`not_exist`()"
+ } ]
+}
+
+
+-- !query
+SELECT `not_exist`.not_exist()
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVED_ROUTINE",
+ "sqlState" : "42883",
+ "messageParameters" : {
+ "routineName" : "`not_exist`.`not_exist`",
+ "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "`not_exist`.not_exist()"
+ } ]
+}
+
+
+-- !query
+SELECT "hello"
+-- !query schema
+struct<hello:string>
+-- !query output
+hello
+
+
+-- !query
+CREATE TEMPORARY VIEW v(c1 COMMENT "hello") AS SELECT 1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DROP VIEW v
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT INTERVAL "1" YEAR
+-- !query schema
+struct<INTERVAL '1' YEAR:interval year>
+-- !query output
+1-0
+
+
+-- !query
+SELECT 'hello'
+-- !query schema
+struct<hello:string>
+-- !query output
+hello
+
+
+-- !query
+CREATE TEMPORARY VIEW v(c1 COMMENT 'hello') AS SELECT 1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DROP VIEW v
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT INTERVAL '1' YEAR
+-- !query schema
+struct<INTERVAL '1' YEAR:interval year>
+-- !query output
+1-0
+
+
+-- !query
+CREATE SCHEMA "myschema"
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"myschema\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+CREATE TEMPORARY VIEW "myview"("c1") AS
+ WITH "v"("a") AS (SELECT 1) SELECT "a" FROM "v"
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"myview\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+SELECT "a1" AS "a2" FROM "myview" AS "atab"("a1")
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"a2\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+DROP TABLE "myview"
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"myview\"'",
+ "hint" : ""
+ }
+}
+
+
+-- !query
+DROP SCHEMA "myschema"
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'\"myschema\"'",
+ "hint" : ""
+ }
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/except-all.sql.out sql/core/src/test/resources/sql-tests/results/except-all.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/except-all.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/except-all.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 27
-
-
-- !query
CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES
(0), (1), (2), (2), (2), (2), (3), (null), (null) AS tab1(c1)
@@ -141,7 +138,25 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-ExceptAll can only be performed on tables with the compatible column types. The first column of the second table is array<int> type which is not compatible with int at same column of first table
+{
+ "errorClass" : "INCOMPATIBLE_COLUMN_TYPE",
+ "sqlState" : "42825",
+ "messageParameters" : {
+ "columnOrdinalNumber" : "first",
+ "dataType1" : "\"ARRAY<INT>\"",
+ "dataType2" : "\"INT\"",
+ "hint" : "",
+ "operator" : "EXCEPT ALL",
+ "tableOrdinalNumber" : "second"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 45,
+ "fragment" : "SELECT * FROM tab1\nEXCEPT ALL\nSELECT array(1)"
+ } ]
+}
-- !query
@@ -213,7 +228,23 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-ExceptAll can only be performed on tables with the same number of columns, but the first table has 1 columns and the second table has 2 columns
+{
+ "errorClass" : "NUM_COLUMNS_MISMATCH",
+ "sqlState" : "42826",
+ "messageParameters" : {
+ "firstNumColumns" : "1",
+ "invalidNumColumns" : "2",
+ "invalidOrdinalNum" : "second",
+ "operator" : "EXCEPT ALL"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 51,
+ "fragment" : "SELECT k FROM tab3\nEXCEPT ALL\nSELECT k, v FROM tab4"
+ } ]
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/except.sql.out sql/core/src/test/resources/sql-tests/results/except.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/except.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/except.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 15
-
-
-- !query
create temporary view t1 as select * from values
("one", 1),
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/explain-aqe.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 27
-
-
-- !query
CREATE table explain_temp1 (key int, val int) USING PARQUET
-- !query schema
@@ -35,6 +32,14 @@
-- !query
+CREATE table explain_temp5 (key int) USING PARQUET PARTITIONED BY(val string)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
SET spark.sql.codegen.wholeStage = true
-- !query schema
struct<key:string,value:string>
@@ -57,12 +62,12 @@
sum(DISTINCT val): bigint
Aggregate [sum(distinct val#x) AS sum(DISTINCT val)#xL]
+- SubqueryAlias spark_catalog.default.explain_temp1
- +- Relation default.explain_temp1[key#x,val#x] parquet
+ +- Relation spark_catalog.default.explain_temp1[key#x,val#x] parquet
== Optimized Logical Plan ==
Aggregate [sum(distinct val#x) AS sum(DISTINCT val)#xL]
+- Project [val#x]
- +- Relation default.explain_temp1[key#x,val#x] parquet
+ +- Relation spark_catalog.default.explain_temp1[key#x,val#x] parquet
== Physical Plan ==
AdaptiveSparkPlan isFinalPlan=false
@@ -72,7 +77,7 @@
+- HashAggregate(keys=[val#x], functions=[], output=[val#x])
+- Exchange hashpartitioning(val#x, 4), ENSURE_REQUIREMENTS, [plan_id=x]
+- HashAggregate(keys=[val#x], functions=[], output=[val#x])
- +- FileScan parquet default.explain_temp1[val#x] Batched: true, DataFilters: [], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp1], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<val:int>
+ +- FileScan parquet spark_catalog.default.explain_temp1[val#x] Batched: true, DataFilters: [], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp1], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<val:int>
-- !query
@@ -93,10 +98,10 @@
+- Exchange (4)
+- HashAggregate (3)
+- Filter (2)
- +- Scan parquet default.explain_temp1 (1)
+ +- Scan parquet spark_catalog.default.explain_temp1 (1)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -155,10 +160,10 @@
+- Exchange (4)
+- HashAggregate (3)
+- Filter (2)
- +- Scan parquet default.explain_temp1 (1)
+ +- Scan parquet spark_catalog.default.explain_temp1 (1)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -211,12 +216,12 @@
+- HashAggregate (6)
+- Union (5)
:- Filter (2)
- : +- Scan parquet default.explain_temp1 (1)
+ : +- Scan parquet spark_catalog.default.explain_temp1 (1)
+- Filter (4)
- +- Scan parquet default.explain_temp1 (3)
+ +- Scan parquet spark_catalog.default.explain_temp1 (3)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -227,7 +232,7 @@
Input [2]: [key#x, val#x]
Condition : (isnotnull(key#x) AND (key#x > 0))
-(3) Scan parquet default.explain_temp1
+(3) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -276,13 +281,13 @@
AdaptiveSparkPlan (7)
+- BroadcastHashJoin Inner BuildRight (6)
:- Filter (2)
- : +- Scan parquet default.explain_temp1 (1)
+ : +- Scan parquet spark_catalog.default.explain_temp1 (1)
+- BroadcastExchange (5)
+- Filter (4)
- +- Scan parquet default.explain_temp2 (3)
+ +- Scan parquet spark_catalog.default.explain_temp2 (3)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -293,7 +298,7 @@
Input [2]: [key#x, val#x]
Condition : isnotnull(key#x)
-(3) Scan parquet default.explain_temp2
+(3) Scan parquet spark_catalog.default.explain_temp2
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp2]
@@ -311,6 +316,7 @@
(6) BroadcastHashJoin
Left keys [1]: [key#x]
Right keys [1]: [key#x]
+Join type: Inner
Join condition: None
(7) AdaptiveSparkPlan
@@ -330,19 +336,19 @@
== Physical Plan ==
AdaptiveSparkPlan (6)
+- BroadcastHashJoin LeftOuter BuildRight (5)
- :- Scan parquet default.explain_temp1 (1)
+ :- Scan parquet spark_catalog.default.explain_temp1 (1)
+- BroadcastExchange (4)
+- Filter (3)
- +- Scan parquet default.explain_temp2 (2)
+ +- Scan parquet spark_catalog.default.explain_temp2 (2)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
ReadSchema: struct<key:int,val:int>
-(2) Scan parquet default.explain_temp2
+(2) Scan parquet spark_catalog.default.explain_temp2
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp2]
@@ -360,6 +366,7 @@
(5) BroadcastHashJoin
Left keys [1]: [key#x]
Right keys [1]: [key#x]
+Join type: LeftOuter
Join condition: None
(6) AdaptiveSparkPlan
@@ -384,10 +391,10 @@
== Physical Plan ==
AdaptiveSparkPlan (3)
+- Filter (2)
- +- Scan parquet default.explain_temp1 (1)
+ +- Scan parquet spark_catalog.default.explain_temp1 (1)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -411,10 +418,10 @@
+- HashAggregate (7)
+- Project (6)
+- Filter (5)
- +- Scan parquet default.explain_temp2 (4)
+ +- Scan parquet spark_catalog.default.explain_temp2 (4)
-(4) Scan parquet default.explain_temp2
+(4) Scan parquet spark_catalog.default.explain_temp2
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp2]
@@ -458,10 +465,10 @@
+- HashAggregate (14)
+- Project (13)
+- Filter (12)
- +- Scan parquet default.explain_temp3 (11)
+ +- Scan parquet spark_catalog.default.explain_temp3 (11)
-(11) Scan parquet default.explain_temp3
+(11) Scan parquet spark_catalog.default.explain_temp3
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp3]
@@ -516,10 +523,10 @@
== Physical Plan ==
AdaptiveSparkPlan (3)
+- Filter (2)
- +- Scan parquet default.explain_temp1 (1)
+ +- Scan parquet spark_catalog.default.explain_temp1 (1)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -542,10 +549,10 @@
+- HashAggregate (7)
+- Project (6)
+- Filter (5)
- +- Scan parquet default.explain_temp2 (4)
+ +- Scan parquet spark_catalog.default.explain_temp2 (4)
-(4) Scan parquet default.explain_temp2
+(4) Scan parquet spark_catalog.default.explain_temp2
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp2]
@@ -589,10 +596,10 @@
+- HashAggregate (14)
+- Project (13)
+- Filter (12)
- +- Scan parquet default.explain_temp3 (11)
+ +- Scan parquet spark_catalog.default.explain_temp3 (11)
-(11) Scan parquet default.explain_temp3
+(11) Scan parquet spark_catalog.default.explain_temp3
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp3]
@@ -640,10 +647,10 @@
== Physical Plan ==
AdaptiveSparkPlan (3)
+- Project (2)
- +- Scan parquet default.explain_temp1 (1)
+ +- Scan parquet spark_catalog.default.explain_temp1 (1)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output: []
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -664,10 +671,10 @@
+- HashAggregate (7)
+- Exchange (6)
+- HashAggregate (5)
- +- Scan parquet default.explain_temp1 (4)
+ +- Scan parquet spark_catalog.default.explain_temp1 (4)
-(4) Scan parquet default.explain_temp1
+(4) Scan parquet spark_catalog.default.explain_temp1
Output [1]: [key#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -700,10 +707,10 @@
+- HashAggregate (12)
+- Exchange (11)
+- HashAggregate (10)
- +- Scan parquet default.explain_temp1 (9)
+ +- Scan parquet spark_catalog.default.explain_temp1 (9)
-(9) Scan parquet default.explain_temp1
+(9) Scan parquet spark_catalog.default.explain_temp1
Output [1]: [key#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -747,13 +754,13 @@
AdaptiveSparkPlan (7)
+- BroadcastHashJoin Inner BuildRight (6)
:- Filter (2)
- : +- Scan parquet default.explain_temp1 (1)
+ : +- Scan parquet spark_catalog.default.explain_temp1 (1)
+- BroadcastExchange (5)
+- Filter (4)
- +- Scan parquet default.explain_temp1 (3)
+ +- Scan parquet spark_catalog.default.explain_temp1 (3)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -764,7 +771,7 @@
Input [2]: [key#x, val#x]
Condition : (isnotnull(key#x) AND (key#x > 10))
-(3) Scan parquet default.explain_temp1
+(3) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -782,6 +789,7 @@
(6) BroadcastHashJoin
Left keys [1]: [key#x]
Right keys [1]: [key#x]
+Join type: Inner
Join condition: None
(7) AdaptiveSparkPlan
@@ -808,16 +816,16 @@
: +- Exchange (4)
: +- HashAggregate (3)
: +- Filter (2)
- : +- Scan parquet default.explain_temp1 (1)
+ : +- Scan parquet spark_catalog.default.explain_temp1 (1)
+- BroadcastExchange (11)
+- HashAggregate (10)
+- Exchange (9)
+- HashAggregate (8)
+- Filter (7)
- +- Scan parquet default.explain_temp1 (6)
+ +- Scan parquet spark_catalog.default.explain_temp1 (6)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -846,7 +854,7 @@
Aggregate Attributes [1]: [max(val#x)#x]
Results [2]: [key#x, max(val#x)#x AS max(val)#x]
-(6) Scan parquet default.explain_temp1
+(6) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -882,6 +890,7 @@
(12) BroadcastHashJoin
Left keys [1]: [key#x]
Right keys [1]: [key#x]
+Join type: Inner
Join condition: None
(13) AdaptiveSparkPlan
@@ -908,22 +917,10 @@
Output: []
(2) CreateViewCommand
-Arguments: `default`.`explain_view`, SELECT key, val FROM explain_temp1, false, false, PersistedView, true
+Arguments: `spark_catalog`.`default`.`explain_view`, SELECT key, val FROM explain_temp1, false, false, PersistedView, true
(3) LogicalRelation
-Arguments: parquet, [key#x, val#x], CatalogTable(
-Database: default
-Table: explain_temp1
-Created Time [not included in comparison]
-Last Access [not included in comparison]
-Created By [not included in comparison]
-Type: MANAGED
-Provider: PARQUET
-Location [not included in comparison]/{warehouse_dir}/explain_temp1
-Schema: root
--- key: integer (nullable = true)
--- val: integer (nullable = true)
-), false
+Arguments: parquet, [key#x, val#x], `spark_catalog`.`default`.`explain_temp1`, false
(4) SubqueryAlias
Arguments: spark_catalog.default.explain_temp1
@@ -946,10 +943,10 @@
+- HashAggregate (4)
+- Exchange (3)
+- HashAggregate (2)
- +- Scan parquet default.explain_temp1 (1)
+ +- Scan parquet spark_catalog.default.explain_temp1 (1)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -991,10 +988,10 @@
+- ObjectHashAggregate (4)
+- Exchange (3)
+- ObjectHashAggregate (2)
- +- Scan parquet default.explain_temp4 (1)
+ +- Scan parquet spark_catalog.default.explain_temp4 (1)
-(1) Scan parquet default.explain_temp4
+(1) Scan parquet spark_catalog.default.explain_temp4
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp4]
@@ -1038,10 +1035,10 @@
+- Exchange (4)
+- SortAggregate (3)
+- Sort (2)
- +- Scan parquet default.explain_temp4 (1)
+ +- Scan parquet spark_catalog.default.explain_temp4 (1)
-(1) Scan parquet default.explain_temp4
+(1) Scan parquet spark_catalog.default.explain_temp4
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp4]
@@ -1079,6 +1076,38 @@
-- !query
+EXPLAIN EXTENDED INSERT INTO TABLE explain_temp5 SELECT * FROM explain_temp4
+-- !query schema
+struct<plan:string>
+-- !query output
+== Parsed Logical Plan ==
+'InsertIntoStatement 'UnresolvedRelation [explain_temp5], [], false, false, false
++- 'Project [*]
+ +- 'UnresolvedRelation [explain_temp4], [], false
+
+== Analyzed Logical Plan ==
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/explain_temp5, false, [val#x], Parquet, [path=file:[not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/explain_temp5), [key, val]
++- Project [key#x, val#x]
+ +- SubqueryAlias spark_catalog.default.explain_temp4
+ +- Relation spark_catalog.default.explain_temp4[key#x,val#x] parquet
+
+== Optimized Logical Plan ==
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/explain_temp5, false, [val#x], Parquet, [path=file:[not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/explain_temp5), [key, val]
++- WriteFiles
+ +- Sort [val#x ASC NULLS FIRST], false
+ +- Project [key#x, empty2null(val#x) AS val#x]
+ +- Relation spark_catalog.default.explain_temp4[key#x,val#x] parquet
+
+== Physical Plan ==
+Execute InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/explain_temp5, false, [val#x], Parquet, [path=file:[not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/explain_temp5), [key, val]
++- WriteFiles
+ +- *Sort [val#x ASC NULLS FIRST], false, 0
+ +- *Project [key#x, empty2null(val#x) AS val#x]
+ +- *ColumnarToRow
+ +- FileScan parquet spark_catalog.default.explain_temp4[key#x,val#x] Batched: true, DataFilters: [], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp4], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<key:int,val:string>
+
+
+-- !query
DROP TABLE explain_temp1
-- !query schema
struct<>
@@ -1111,6 +1140,14 @@
-- !query
+DROP TABLE explain_temp5
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
CREATE table t(v array<string>) USING PARQUET
-- !query schema
struct<>
@@ -1125,7 +1162,8 @@
-- !query output
== Physical Plan ==
*Filter v#x IN ([a],null)
-+- FileScan parquet default.t[v#x] Batched: false, DataFilters: [v#x IN ([a],null)], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/t], PartitionFilters: [], PushedFilters: [In(v, [[a],null])], ReadSchema: struct<v:array<string>>
++- *ColumnarToRow
+ +- FileScan parquet spark_catalog.default.t[v#x] Batched: true, DataFilters: [v#x IN ([a],null)], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/t], PartitionFilters: [], PushedFilters: [In(v, [[a],null])], ReadSchema: struct<v:array<string>>
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/explain-cbo.sql.out sql/core/src/test/resources/sql-tests/results/explain-cbo.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/explain-cbo.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/explain-cbo.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 7
-
-
-- !query
CREATE TABLE explain_temp1(a INT, b INT) USING PARQUET
-- !query schema
@@ -60,8 +57,8 @@
: +- Aggregate [sum(b#x) AS csales#xL], Statistics(sizeInBytes=16.0 B, rowCount=1)
: +- Project [b#x], Statistics(sizeInBytes=1.0 B, rowCount=0)
: +- Filter (isnotnull(a#x) AND (a#x < 100)), Statistics(sizeInBytes=1.0 B, rowCount=0)
- : +- Relation default.explain_temp1[a#x,b#x] parquet, Statistics(sizeInBytes=1.0 B, rowCount=0)
- +- Relation default.explain_temp2[c#x,d#x] parquet, Statistics(sizeInBytes=1.0 B, rowCount=0)
+ : +- Relation spark_catalog.default.explain_temp1[a#x,b#x] parquet, Statistics(sizeInBytes=1.0 B, rowCount=0)
+ +- Relation spark_catalog.default.explain_temp2[c#x,d#x] parquet, Statistics(sizeInBytes=1.0 B, rowCount=0)
== Physical Plan ==
AdaptiveSparkPlan isFinalPlan=false
@@ -76,8 +73,8 @@
: +- HashAggregate(keys=[], functions=[partial_sum(b#x)], output=[sum#xL])
: +- Project [b#x]
: +- Filter (isnotnull(a#x) AND (a#x < 100))
- : +- FileScan parquet default.explain_temp1[a#x,b#x] Batched: true, DataFilters: [isnotnull(a#x), (a#x < 100)], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp1], PartitionFilters: [], PushedFilters: [IsNotNull(a), LessThan(a,100)], ReadSchema: struct<a:int,b:int>
- +- FileScan parquet default.explain_temp2[c#x,d#x] Batched: true, DataFilters: [isnotnull(d#x)], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp2], PartitionFilters: [], PushedFilters: [IsNotNull(d)], ReadSchema: struct<c:int,d:int>
+ : +- FileScan parquet spark_catalog.default.explain_temp1[a#x,b#x] Batched: true, DataFilters: [isnotnull(a#x), (a#x < 100)], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp1], PartitionFilters: [], PushedFilters: [IsNotNull(a), LessThan(a,100)], ReadSchema: struct<a:int,b:int>
+ +- FileScan parquet spark_catalog.default.explain_temp2[c#x,d#x] Batched: true, DataFilters: [isnotnull(d#x)], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp2], PartitionFilters: [], PushedFilters: [IsNotNull(d)], ReadSchema: struct<c:int,d:int>
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/explain.sql.out sql/core/src/test/resources/sql-tests/results/explain.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/explain.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/explain.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 27
-
-
-- !query
CREATE table explain_temp1 (key int, val int) USING PARQUET
-- !query schema
@@ -35,6 +32,14 @@
-- !query
+CREATE table explain_temp5 (key int) USING PARQUET PARTITIONED BY(val string)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
SET spark.sql.codegen.wholeStage = true
-- !query schema
struct<key:string,value:string>
@@ -57,12 +62,12 @@
sum(DISTINCT val): bigint
Aggregate [sum(distinct val#x) AS sum(DISTINCT val)#xL]
+- SubqueryAlias spark_catalog.default.explain_temp1
- +- Relation default.explain_temp1[key#x,val#x] parquet
+ +- Relation spark_catalog.default.explain_temp1[key#x,val#x] parquet
== Optimized Logical Plan ==
Aggregate [sum(distinct val#x) AS sum(DISTINCT val)#xL]
+- Project [val#x]
- +- Relation default.explain_temp1[key#x,val#x] parquet
+ +- Relation spark_catalog.default.explain_temp1[key#x,val#x] parquet
== Physical Plan ==
*HashAggregate(keys=[], functions=[sum(distinct val#x)], output=[sum(DISTINCT val)#xL])
@@ -72,7 +77,7 @@
+- Exchange hashpartitioning(val#x, 4), ENSURE_REQUIREMENTS, [plan_id=x]
+- *HashAggregate(keys=[val#x], functions=[], output=[val#x])
+- *ColumnarToRow
- +- FileScan parquet default.explain_temp1[val#x] Batched: true, DataFilters: [], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp1], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<val:int>
+ +- FileScan parquet spark_catalog.default.explain_temp1[val#x] Batched: true, DataFilters: [], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp1], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<val:int>
-- !query
@@ -93,10 +98,10 @@
+- * HashAggregate (4)
+- * Filter (3)
+- * ColumnarToRow (2)
- +- Scan parquet default.explain_temp1 (1)
+ +- Scan parquet spark_catalog.default.explain_temp1 (1)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -154,10 +159,10 @@
+- * HashAggregate (4)
+- * Filter (3)
+- * ColumnarToRow (2)
- +- Scan parquet default.explain_temp1 (1)
+ +- Scan parquet spark_catalog.default.explain_temp1 (1)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -209,13 +214,13 @@
+- Union (7)
:- * Filter (3)
: +- * ColumnarToRow (2)
- : +- Scan parquet default.explain_temp1 (1)
+ : +- Scan parquet spark_catalog.default.explain_temp1 (1)
+- * Filter (6)
+- * ColumnarToRow (5)
- +- Scan parquet default.explain_temp1 (4)
+ +- Scan parquet spark_catalog.default.explain_temp1 (4)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -229,7 +234,7 @@
Input [2]: [key#x, val#x]
Condition : (isnotnull(key#x) AND (key#x > 0))
-(4) Scan parquet default.explain_temp1
+(4) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -277,14 +282,14 @@
* BroadcastHashJoin Inner BuildRight (8)
:- * Filter (3)
: +- * ColumnarToRow (2)
-: +- Scan parquet default.explain_temp1 (1)
+: +- Scan parquet spark_catalog.default.explain_temp1 (1)
+- BroadcastExchange (7)
+- * Filter (6)
+- * ColumnarToRow (5)
- +- Scan parquet default.explain_temp2 (4)
+ +- Scan parquet spark_catalog.default.explain_temp2 (4)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -298,7 +303,7 @@
Input [2]: [key#x, val#x]
Condition : isnotnull(key#x)
-(4) Scan parquet default.explain_temp2
+(4) Scan parquet spark_catalog.default.explain_temp2
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp2]
@@ -319,6 +324,7 @@
(8) BroadcastHashJoin [codegen id : 2]
Left keys [1]: [key#x]
Right keys [1]: [key#x]
+Join type: Inner
Join condition: None
@@ -334,14 +340,14 @@
== Physical Plan ==
* BroadcastHashJoin LeftOuter BuildRight (7)
:- * ColumnarToRow (2)
-: +- Scan parquet default.explain_temp1 (1)
+: +- Scan parquet spark_catalog.default.explain_temp1 (1)
+- BroadcastExchange (6)
+- * Filter (5)
+- * ColumnarToRow (4)
- +- Scan parquet default.explain_temp2 (3)
+ +- Scan parquet spark_catalog.default.explain_temp2 (3)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -350,7 +356,7 @@
(2) ColumnarToRow [codegen id : 2]
Input [2]: [key#x, val#x]
-(3) Scan parquet default.explain_temp2
+(3) Scan parquet spark_catalog.default.explain_temp2
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp2]
@@ -371,6 +377,7 @@
(7) BroadcastHashJoin [codegen id : 2]
Left keys [1]: [key#x]
Right keys [1]: [key#x]
+Join type: LeftOuter
Join condition: None
@@ -391,10 +398,10 @@
== Physical Plan ==
* Filter (3)
+- * ColumnarToRow (2)
- +- Scan parquet default.explain_temp1 (1)
+ +- Scan parquet spark_catalog.default.explain_temp1 (1)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -417,10 +424,10 @@
+- * Project (7)
+- * Filter (6)
+- * ColumnarToRow (5)
- +- Scan parquet default.explain_temp2 (4)
+ +- Scan parquet spark_catalog.default.explain_temp2 (4)
-(4) Scan parquet default.explain_temp2
+(4) Scan parquet spark_catalog.default.explain_temp2
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp2]
@@ -463,10 +470,10 @@
+- * Project (14)
+- * Filter (13)
+- * ColumnarToRow (12)
- +- Scan parquet default.explain_temp3 (11)
+ +- Scan parquet spark_catalog.default.explain_temp3 (11)
-(11) Scan parquet default.explain_temp3
+(11) Scan parquet spark_catalog.default.explain_temp3
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp3]
@@ -520,10 +527,10 @@
== Physical Plan ==
* Filter (3)
+- * ColumnarToRow (2)
- +- Scan parquet default.explain_temp1 (1)
+ +- Scan parquet spark_catalog.default.explain_temp1 (1)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -545,10 +552,10 @@
+- * Project (7)
+- * Filter (6)
+- * ColumnarToRow (5)
- +- Scan parquet default.explain_temp2 (4)
+ +- Scan parquet spark_catalog.default.explain_temp2 (4)
-(4) Scan parquet default.explain_temp2
+(4) Scan parquet spark_catalog.default.explain_temp2
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp2]
@@ -591,10 +598,10 @@
+- * Project (14)
+- * Filter (13)
+- * ColumnarToRow (12)
- +- Scan parquet default.explain_temp3 (11)
+ +- Scan parquet spark_catalog.default.explain_temp3 (11)
-(11) Scan parquet default.explain_temp3
+(11) Scan parquet spark_catalog.default.explain_temp3
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp3]
@@ -641,10 +648,10 @@
== Physical Plan ==
* Project (3)
+- * ColumnarToRow (2)
- +- Scan parquet default.explain_temp1 (1)
+ +- Scan parquet spark_catalog.default.explain_temp1 (1)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output: []
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -664,10 +671,10 @@
+- Exchange (7)
+- * HashAggregate (6)
+- * ColumnarToRow (5)
- +- Scan parquet default.explain_temp1 (4)
+ +- Scan parquet spark_catalog.default.explain_temp1 (4)
-(4) Scan parquet default.explain_temp1
+(4) Scan parquet spark_catalog.default.explain_temp1
Output [1]: [key#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -712,14 +719,14 @@
* BroadcastHashJoin Inner BuildRight (8)
:- * Filter (3)
: +- * ColumnarToRow (2)
-: +- Scan parquet default.explain_temp1 (1)
+: +- Scan parquet spark_catalog.default.explain_temp1 (1)
+- BroadcastExchange (7)
+- * Filter (6)
+- * ColumnarToRow (5)
- +- Scan parquet default.explain_temp1 (4)
+ +- Scan parquet spark_catalog.default.explain_temp1 (4)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -733,7 +740,7 @@
Input [2]: [key#x, val#x]
Condition : (isnotnull(key#x) AND (key#x > 10))
-(4) Scan parquet default.explain_temp1
+(4) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -754,6 +761,7 @@
(8) BroadcastHashJoin [codegen id : 2]
Left keys [1]: [key#x]
Right keys [1]: [key#x]
+Join type: Inner
Join condition: None
@@ -776,13 +784,13 @@
: +- * HashAggregate (4)
: +- * Filter (3)
: +- * ColumnarToRow (2)
-: +- Scan parquet default.explain_temp1 (1)
+: +- Scan parquet spark_catalog.default.explain_temp1 (1)
+- BroadcastExchange (9)
+- * HashAggregate (8)
+- ReusedExchange (7)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -831,6 +839,7 @@
(10) BroadcastHashJoin [codegen id : 4]
Left keys [1]: [key#x]
Right keys [1]: [key#x]
+Join type: Inner
Join condition: None
@@ -853,22 +862,10 @@
Output: []
(2) CreateViewCommand
-Arguments: `default`.`explain_view`, SELECT key, val FROM explain_temp1, false, false, PersistedView, true
+Arguments: `spark_catalog`.`default`.`explain_view`, SELECT key, val FROM explain_temp1, false, false, PersistedView, true
(3) LogicalRelation
-Arguments: parquet, [key#x, val#x], CatalogTable(
-Database: default
-Table: explain_temp1
-Created Time [not included in comparison]
-Last Access [not included in comparison]
-Created By [not included in comparison]
-Type: MANAGED
-Provider: PARQUET
-Location [not included in comparison]/{warehouse_dir}/explain_temp1
-Schema: root
--- key: integer (nullable = true)
--- val: integer (nullable = true)
-), false
+Arguments: parquet, [key#x, val#x], `spark_catalog`.`default`.`explain_temp1`, false
(4) SubqueryAlias
Arguments: spark_catalog.default.explain_temp1
@@ -891,10 +888,10 @@
+- Exchange (4)
+- * HashAggregate (3)
+- * ColumnarToRow (2)
- +- Scan parquet default.explain_temp1 (1)
+ +- Scan parquet spark_catalog.default.explain_temp1 (1)
-(1) Scan parquet default.explain_temp1
+(1) Scan parquet spark_catalog.default.explain_temp1
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp1]
@@ -935,10 +932,10 @@
+- Exchange (4)
+- ObjectHashAggregate (3)
+- * ColumnarToRow (2)
- +- Scan parquet default.explain_temp4 (1)
+ +- Scan parquet spark_catalog.default.explain_temp4 (1)
-(1) Scan parquet default.explain_temp4
+(1) Scan parquet spark_catalog.default.explain_temp4
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp4]
@@ -981,10 +978,10 @@
+- SortAggregate (4)
+- * Sort (3)
+- * ColumnarToRow (2)
- +- Scan parquet default.explain_temp4 (1)
+ +- Scan parquet spark_catalog.default.explain_temp4 (1)
-(1) Scan parquet default.explain_temp4
+(1) Scan parquet spark_catalog.default.explain_temp4
Output [2]: [key#x, val#x]
Batched: true
Location [not included in comparison]/{warehouse_dir}/explain_temp4]
@@ -1021,6 +1018,38 @@
-- !query
+EXPLAIN EXTENDED INSERT INTO TABLE explain_temp5 SELECT * FROM explain_temp4
+-- !query schema
+struct<plan:string>
+-- !query output
+== Parsed Logical Plan ==
+'InsertIntoStatement 'UnresolvedRelation [explain_temp5], [], false, false, false
++- 'Project [*]
+ +- 'UnresolvedRelation [explain_temp4], [], false
+
+== Analyzed Logical Plan ==
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/explain_temp5, false, [val#x], Parquet, [path=file:[not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/explain_temp5), [key, val]
++- Project [key#x, val#x]
+ +- SubqueryAlias spark_catalog.default.explain_temp4
+ +- Relation spark_catalog.default.explain_temp4[key#x,val#x] parquet
+
+== Optimized Logical Plan ==
+InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/explain_temp5, false, [val#x], Parquet, [path=file:[not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/explain_temp5), [key, val]
++- WriteFiles
+ +- Sort [val#x ASC NULLS FIRST], false
+ +- Project [key#x, empty2null(val#x) AS val#x]
+ +- Relation spark_catalog.default.explain_temp4[key#x,val#x] parquet
+
+== Physical Plan ==
+Execute InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/explain_temp5, false, [val#x], Parquet, [path=file:[not included in comparison]/{warehouse_dir}/explain_temp5], Append, `spark_catalog`.`default`.`explain_temp5`, org.apache.spark.sql.execution.datasources.CatalogFileIndex(file:[not included in comparison]/{warehouse_dir}/explain_temp5), [key, val]
++- WriteFiles
+ +- *Sort [val#x ASC NULLS FIRST], false, 0
+ +- *Project [key#x, empty2null(val#x) AS val#x]
+ +- *ColumnarToRow
+ +- FileScan parquet spark_catalog.default.explain_temp4[key#x,val#x] Batched: true, DataFilters: [], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/explain_temp4], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<key:int,val:string>
+
+
+-- !query
DROP TABLE explain_temp1
-- !query schema
struct<>
@@ -1053,6 +1082,14 @@
-- !query
+DROP TABLE explain_temp5
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
CREATE table t(v array<string>) USING PARQUET
-- !query schema
struct<>
@@ -1067,7 +1104,8 @@
-- !query output
== Physical Plan ==
*Filter v#x IN ([a],null)
-+- FileScan parquet default.t[v#x] Batched: false, DataFilters: [v#x IN ([a],null)], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/t], PartitionFilters: [], PushedFilters: [In(v, [[a],null])], ReadSchema: struct<v:array<string>>
++- *ColumnarToRow
+ +- FileScan parquet spark_catalog.default.t[v#x] Batched: true, DataFilters: [v#x IN ([a],null)], Format: Parquet, Location [not included in comparison]/{warehouse_dir}/t], PartitionFilters: [], PushedFilters: [In(v, [[a],null])], ReadSchema: struct<v:array<string>>
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/extract.sql.out sql/core/src/test/resources/sql-tests/results/extract.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/extract.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/extract.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 127
-
-
-- !query
CREATE TEMPORARY VIEW t AS select '2011-05-06 07:08:09.1234567' as c, to_timestamp_ntz('2011-05-06 07:08:09.1234567') as ntz, interval 10 year 20 month as i, interval 30 day 40 hour 50 minute 6.7890 second as j
-- !query schema
@@ -320,7 +317,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Literals of type 'not_supported' are currently not supported for the string type.; line 1 pos 7
+{
+ "errorClass" : "INVALID_EXTRACT_FIELD",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "expr" : "\"c\"",
+ "field" : "`not_supported`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 36,
+ "fragment" : "extract(not_supported from c)"
+ } ]
+}
-- !query
@@ -329,7 +340,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Literals of type 'not_supported' are currently not supported for the interval year to month type.; line 1 pos 7
+{
+ "errorClass" : "INVALID_EXTRACT_FIELD",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "expr" : "\"i\"",
+ "field" : "`not_supported`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 36,
+ "fragment" : "extract(not_supported from i)"
+ } ]
+}
-- !query
@@ -338,7 +363,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Literals of type 'not_supported' are currently not supported for the interval day to second type.; line 1 pos 7
+{
+ "errorClass" : "INVALID_EXTRACT_FIELD",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "expr" : "\"j\"",
+ "field" : "`not_supported`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 36,
+ "fragment" : "extract(not_supported from j)"
+ } ]
+}
-- !query
@@ -651,7 +690,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Literals of type 'not_supported' are currently not supported for the string type.; line 1 pos 7
+{
+ "errorClass" : "INVALID_EXTRACT_FIELD",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "expr" : "\"c\"",
+ "field" : "`not_supported`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 36,
+ "fragment" : "date_part('not_supported', c)"
+ } ]
+}
-- !query
@@ -660,7 +713,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-The 'field' parameter of function 'date_part' needs to be a string literal.; line 1 pos 7
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1100",
+ "messageParameters" : {
+ "argName" : "field",
+ "funcName" : "date_part",
+ "requiredType" : "string"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 22,
+ "fragment" : "date_part(c, c)"
+ } ]
+}
-- !query
@@ -677,7 +744,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-The 'field' parameter of function 'date_part' needs to be a string literal.; line 1 pos 7
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1100",
+ "messageParameters" : {
+ "argName" : "field",
+ "funcName" : "date_part",
+ "requiredType" : "string"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 22,
+ "fragment" : "date_part(i, i)"
+ } ]
+}
-- !query
@@ -886,7 +967,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Literals of type 'DAY' are currently not supported for the interval year to month type.; line 1 pos 7
+{
+ "errorClass" : "INVALID_EXTRACT_FIELD",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "expr" : "\"INTERVAL '2-1' YEAR TO MONTH\"",
+ "field" : "`DAY`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 53,
+ "fragment" : "extract(DAY from interval '2-1' YEAR TO MONTH)"
+ } ]
+}
-- !query
@@ -895,7 +990,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Literals of type 'DAY' are currently not supported for the interval year to month type.; line 1 pos 7
+{
+ "errorClass" : "INVALID_EXTRACT_FIELD",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "expr" : "\"INTERVAL '2-1' YEAR TO MONTH\"",
+ "field" : "`DAY`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 53,
+ "fragment" : "date_part('DAY', interval '2-1' YEAR TO MONTH)"
+ } ]
+}
-- !query
@@ -904,7 +1013,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Literals of type 'not_supported' are currently not supported for the interval year to month type.; line 1 pos 7
+{
+ "errorClass" : "INVALID_EXTRACT_FIELD",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "expr" : "\"INTERVAL '2-1' YEAR TO MONTH\"",
+ "field" : "`not_supported`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 63,
+ "fragment" : "date_part('not_supported', interval '2-1' YEAR TO MONTH)"
+ } ]
+}
-- !query
@@ -1017,7 +1140,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Literals of type 'MONTH' are currently not supported for the interval day to second type.; line 1 pos 7
+{
+ "errorClass" : "INVALID_EXTRACT_FIELD",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "expr" : "\"INTERVAL '123 12:34:56.789123' DAY TO SECOND\"",
+ "field" : "`MONTH`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 74,
+ "fragment" : "extract(MONTH from interval '123 12:34:56.789123123' DAY TO SECOND)"
+ } ]
+}
-- !query
@@ -1026,4 +1163,34 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Literals of type 'not_supported' are currently not supported for the interval day to second type.; line 1 pos 7
+{
+ "errorClass" : "INVALID_EXTRACT_FIELD",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "expr" : "\"INTERVAL '123 12:34:56.789123' DAY TO SECOND\"",
+ "field" : "`not_supported`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 82,
+ "fragment" : "date_part('not_supported', interval '123 12:34:56.789123123' DAY TO SECOND)"
+ } ]
+}
+
+
+-- !query
+select datepart('year', c), datepart('year', ntz), datepart('year', i) from t
+-- !query schema
+struct<datepart(year FROM c):int,datepart(year FROM ntz):int,datepart(year FROM i):int>
+-- !query output
+2011 2011 11
+
+
+-- !query
+select datepart('DAY', interval '123 12:34:56.789123123' DAY TO SECOND)
+-- !query schema
+struct<datepart(DAY FROM INTERVAL '123 12:34:56.789123' DAY TO SECOND):int>
+-- !query output
+123
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 52
-
-
-- !query
CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2)
@@ -134,12 +131,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Empty set in ROLLUP grouping sets is not supported.(line 1, pos 61)
-
-== SQL ==
-SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year, (course, year), ()) ORDER BY course, year
--------------------------------------------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0051",
+ "messageParameters" : {
+ "element" : "ROLLUP"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 53,
+ "stopIndex" : 101,
+ "fragment" : "GROUP BY ROLLUP(course, year, (course, year), ())"
+ } ]
+}
-- !query
@@ -196,12 +200,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Empty set in CUBE grouping sets is not supported.(line 1, pos 61)
-
-== SQL ==
-SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year, (course, year), ()) ORDER BY course, year
--------------------------------------------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0051",
+ "messageParameters" : {
+ "element" : "CUBE"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 53,
+ "stopIndex" : 99,
+ "fragment" : "GROUP BY CUBE(course, year, (course, year), ())"
+ } ]
+}
-- !query
@@ -454,7 +465,16 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-grouping() can only be used with GroupingSets/Cube/Rollup
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_2445",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 22,
+ "stopIndex" : 37,
+ "fragment" : "GROUPING(course)"
+ } ]
+}
-- !query
@@ -463,7 +483,16 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-grouping_id() can only be used with GroupingSets/Cube/Rollup
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_2407",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 22,
+ "stopIndex" : 46,
+ "fragment" : "GROUPING_ID(course, year)"
+ } ]
+}
-- !query
@@ -499,7 +528,9 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup
+{
+ "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION"
+}
-- !query
@@ -508,7 +539,9 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup
+{
+ "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION"
+}
-- !query
@@ -563,7 +596,9 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup
+{
+ "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION"
+}
-- !query
@@ -572,7 +607,9 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup
+{
+ "errorClass" : "UNSUPPORTED_GROUPING_EXPRESSION"
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/group-by-all-duckdb.sql.out sql/core/src/test/resources/sql-tests/results/group-by-all-duckdb.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/group-by-all-duckdb.sql.out 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/results/group-by-all-duckdb.sql.out 2023-09-18 15:00:43.010708418 +0800
@@ -0,0 +1,119 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temporary view integers as select * from values
+ (0, 1),
+ (0, 2),
+ (1, 3),
+ (1, NULL)
+ as integers(g, i)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT g, SUM(i) FROM integers GROUP BY ALL ORDER BY 1
+-- !query schema
+struct<g:int,sum(i):bigint>
+-- !query output
+0 3
+1 3
+
+
+-- !query
+SELECT g, SUM(i), COUNT(*), COUNT(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1
+-- !query schema
+struct<g:int,sum(i):bigint,count(1):bigint,count(i):bigint,sum(g):bigint>
+-- !query output
+0 3 2 2 0
+1 3 2 1 2
+
+
+-- !query
+SELECT i%2, SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1
+-- !query schema
+struct<(i % 2):int,sum(i):bigint,sum(g):bigint>
+-- !query output
+NULL NULL 1
+0 2 0
+1 4 1
+
+
+-- !query
+SELECT (g+i)%2, SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1
+-- !query schema
+struct<((g + i) % 2):int,sum(i):bigint,sum(g):bigint>
+-- !query output
+NULL NULL 1
+0 5 1
+1 1 0
+
+
+-- !query
+SELECT (g+i)%2 + SUM(i), SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY",
+ "sqlState" : "42803",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 55,
+ "stopIndex" : 66,
+ "fragment" : "GROUP BY ALL"
+ } ]
+}
+
+
+-- !query
+SELECT g, i, g%2, SUM(i), SUM(g) FROM integers GROUP BY ALL ORDER BY 1, 2, 3, 4
+-- !query schema
+struct<g:int,i:int,(g % 2):int,sum(i):bigint,sum(g):bigint>
+-- !query output
+0 1 0 1 0
+0 2 0 2 0
+1 NULL 1 NULL 1
+1 3 1 3 1
+
+
+-- !query
+SELECT c0 FROM (SELECT 1 c0) t0 GROUP BY ALL HAVING c0>0
+-- !query schema
+struct<c0:int>
+-- !query output
+1
+
+
+-- !query
+SELECT c0 FROM (SELECT 1 c0, 1 c1 UNION ALL SELECT 1, 2) t0 GROUP BY ALL ORDER BY c0
+-- !query schema
+struct<c0:int>
+-- !query output
+1
+
+
+-- !query
+SELECT c0 FROM (SELECT 1 c0, 1 c1 UNION ALL SELECT 1, 2) t0 GROUP BY ALL HAVING c1>0 ORDER BY c0
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION",
+ "sqlState" : "42703",
+ "messageParameters" : {
+ "objectName" : "`c1`",
+ "proposal" : "`t0`.`c0`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 81,
+ "stopIndex" : 82,
+ "fragment" : "c1"
+ } ]
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/group-by-all-mosha.sql.out sql/core/src/test/resources/sql-tests/results/group-by-all-mosha.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/group-by-all-mosha.sql.out 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/results/group-by-all-mosha.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -0,0 +1,142 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temporary view stuff as select * from values
+ (42, 9.75, 'hello world', '1970-08-07', '13.37', array(1,20,300)),
+ (1337, 1.2345, 'oh no', '2000-01-01', '42.0', array(4000,50000,600000)),
+ (42, 13.37, 'test', '1970-08-07', '1234567890', array(7000000,80000000,900000000))
+ as stuff(i, f, s, t, d, a)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT 100 * SUM(i) + SUM(f) / COUNT(s) AS f1, i AS f2 FROM stuff GROUP BY ALL ORDER BY f2
+-- !query schema
+struct<f1:decimal(38,17),f2:int>
+-- !query output
+8411.56000000000000000 42
+133701.23450000000000000 1337
+
+
+-- !query
+SELECT i + 1 AS i1, COUNT(i - 2) ci, f / i AS fi, SUM(i + f) sif FROM stuff GROUP BY ALL ORDER BY 1, 3
+-- !query schema
+struct<i1:int,ci:bigint,fi:decimal(17,15),sif:decimal(25,4)>
+-- !query output
+43 1 0.232142857142857 51.7500
+43 1 0.318333333333333 55.3700
+1338 1 0.000923335826477 1338.2345
+
+
+-- !query
+SELECT i AS i, COUNT(i) ci, f AS f, SUM(i + f) sif FROM stuff GROUP BY ALL ORDER BY 1, i, 2, ci, 3, f, 4, sif
+-- !query schema
+struct<i:int,ci:bigint,f:decimal(6,4),sif:decimal(25,4)>
+-- !query output
+42 1 9.7500 51.7500
+42 1 13.3700 55.3700
+1337 1 1.2345 1338.2345
+
+
+-- !query
+SELECT i + 1, f / i, substring(s, 2, 3), extract(year from t), d / 2, size(a) FROM stuff
+GROUP BY ALL ORDER BY 1, 3, 4, 5, 6, 2
+-- !query schema
+struct<(i + 1):int,(f / i):decimal(17,15),substring(s, 2, 3):string,extract(year FROM t):int,(d / 2):double,size(a):int>
+-- !query output
+43 0.232142857142857 ell 1970 6.685 3
+43 0.318333333333333 est 1970 6.17283945E8 3
+1338 0.000923335826477 h n 2000 21.0 3
+
+
+-- !query
+SELECT i + SUM(f) FROM stuff GROUP BY ALL
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY",
+ "sqlState" : "42803",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 30,
+ "stopIndex" : 41,
+ "fragment" : "GROUP BY ALL"
+ } ]
+}
+
+
+-- !query
+SELECT s AS s, COUNT(*) c FROM stuff GROUP BY ALL HAVING SUM(f) > 0 ORDER BY s
+-- !query schema
+struct<s:string,c:bigint>
+-- !query output
+hello world 1
+oh no 1
+test 1
+
+
+-- !query
+SELECT SUM(i) si FROM stuff GROUP BY ALL HAVING si > 2
+-- !query schema
+struct<si:bigint>
+-- !query output
+1421
+
+
+-- !query
+SELECT SUM(i) si FROM stuff GROUP BY ALL HAVING si < 2
+-- !query schema
+struct<si:bigint>
+-- !query output
+
+
+
+-- !query
+SELECT SUM(i) si FROM stuff GROUP BY ALL HAVING i > 2
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION",
+ "sqlState" : "42703",
+ "messageParameters" : {
+ "objectName" : "`i`",
+ "proposal" : "`si`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 49,
+ "stopIndex" : 49,
+ "fragment" : "i"
+ } ]
+}
+
+
+-- !query
+SELECT SUM(i) si FROM stuff GROUP BY ALL ORDER BY i DESC
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION",
+ "sqlState" : "42703",
+ "messageParameters" : {
+ "objectName" : "`i`",
+ "proposal" : "`si`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 51,
+ "stopIndex" : 51,
+ "fragment" : "i"
+ } ]
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/group-by-all.sql.out sql/core/src/test/resources/sql-tests/results/group-by-all.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/group-by-all.sql.out 1970-01-01 08:00:00.000000000 +0800
+++ sql/core/src/test/resources/sql-tests/results/group-by-all.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -0,0 +1,290 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+create temporary view data as select * from values
+ ("USA", "San Francisco", "Reynold", 1, 11.0),
+ ("USA", "San Francisco", "Matei", 2, 12.0),
+ ("USA", "Berkeley", "Xiao", 3, 13.0),
+ ("China", "Hangzhou", "Wenchen", 4, 14.0),
+ ("China", "Shanghai", "Shanghaiese", 5, 15.0),
+ ("Korea", "Seoul", "Hyukjin", 6, 16.0),
+ ("UK", "London", "Sean", 7, 17.0)
+ as data(country, city, name, id, power)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+select country, count(*) from data group by ALL
+-- !query schema
+struct<country:string,count(1):bigint>
+-- !query output
+China 2
+Korea 1
+UK 1
+USA 3
+
+
+-- !query
+select country, count(*) from data group by aLl
+-- !query schema
+struct<country:string,count(1):bigint>
+-- !query output
+China 2
+Korea 1
+UK 1
+USA 3
+
+
+-- !query
+select all, city, count(*) from (select country as all, city, id from data) group by all, city
+-- !query schema
+struct<all:string,city:string,count(1):bigint>
+-- !query output
+China Hangzhou 1
+China Shanghai 1
+Korea Seoul 1
+UK London 1
+USA Berkeley 1
+USA San Francisco 2
+
+
+-- !query
+SELECT count(1) FROM VALUES(1), (2), (3) AS T(all) GROUP BY all
+-- !query schema
+struct<count(1):bigint>
+-- !query output
+1
+1
+1
+
+
+-- !query
+select country, city, count(*), sum(power) from data group by all
+-- !query schema
+struct<country:string,city:string,count(1):bigint,sum(power):decimal(13,1)>
+-- !query output
+China Hangzhou 1 14.0
+China Shanghai 1 15.0
+Korea Seoul 1 16.0
+UK London 1 17.0
+USA Berkeley 1 13.0
+USA San Francisco 2 23.0
+
+
+-- !query
+select count(*), country, city, sum(power) from data group by all
+-- !query schema
+struct<count(1):bigint,country:string,city:string,sum(power):decimal(13,1)>
+-- !query output
+1 China Hangzhou 14.0
+1 China Shanghai 15.0
+1 Korea Seoul 16.0
+1 UK London 17.0
+1 USA Berkeley 13.0
+2 USA San Francisco 23.0
+
+
+-- !query
+select country as con, count(*) from data group by all
+-- !query schema
+struct<con:string,count(1):bigint>
+-- !query output
+China 2
+Korea 1
+UK 1
+USA 3
+
+
+-- !query
+select country, count(*) as cnt from data group by all
+-- !query schema
+struct<country:string,cnt:bigint>
+-- !query output
+China 2
+Korea 1
+UK 1
+USA 3
+
+
+-- !query
+select upper(country), count(*) as powerup from data group by all
+-- !query schema
+struct<upper(country):string,powerup:bigint>
+-- !query output
+CHINA 2
+KOREA 1
+UK 1
+USA 3
+
+
+-- !query
+select country, sum(power) + 10 as powerup from data group by all
+-- !query schema
+struct<country:string,powerup:decimal(14,1)>
+-- !query output
+China 39.0
+Korea 26.0
+UK 27.0
+USA 46.0
+
+
+-- !query
+select country, city from data group by all
+-- !query schema
+struct<country:string,city:string>
+-- !query output
+China Hangzhou
+China Shanghai
+Korea Seoul
+UK London
+USA Berkeley
+USA San Francisco
+
+
+-- !query
+select con, powerup from
+ (select country as con, sum(power) + 10 as powerup from data group by all)
+-- !query schema
+struct<con:string,powerup:decimal(14,1)>
+-- !query output
+China 39.0
+Korea 26.0
+UK 27.0
+USA 46.0
+
+
+-- !query
+select country, count(id) as cnt from data group by all having cnt > 1
+-- !query schema
+struct<country:string,cnt:bigint>
+-- !query output
+China 2
+USA 3
+
+
+-- !query
+select count(id) from data group by all
+-- !query schema
+struct<count(id):bigint>
+-- !query output
+7
+
+
+-- !query
+select count(id + power / 2) * 3 from data group by all
+-- !query schema
+struct<(count((id + (power / 2))) * 3):bigint>
+-- !query output
+21
+
+
+-- !query
+select count(*) from (select * from data where country = "DNS") group by all
+-- !query schema
+struct<count(1):bigint>
+-- !query output
+0
+
+
+-- !query
+select id + count(*) from data group by all
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY",
+ "sqlState" : "42803",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 32,
+ "stopIndex" : 43,
+ "fragment" : "group by all"
+ } ]
+}
+
+
+-- !query
+select (id + id) / 2 + count(*) * 2 from data group by all
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY",
+ "sqlState" : "42803",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 47,
+ "stopIndex" : 58,
+ "fragment" : "group by all"
+ } ]
+}
+
+
+-- !query
+select country, (select count(*) from data) as cnt, count(id) as cnt_id from data group by all
+-- !query schema
+struct<country:string,cnt:bigint,cnt_id:bigint>
+-- !query output
+China 7 2
+Korea 7 1
+UK 7 1
+USA 7 3
+
+
+-- !query
+select country, (select count(*) from data d1 where d1.country = d2.country), count(id) from data d2 group by all
+-- !query schema
+struct<country:string,scalarsubquery(country):bigint,count(id):bigint>
+-- !query output
+China 2 2
+Korea 1 1
+UK 1 1
+USA 3 3
+
+
+-- !query
+select (select count(*) from data d1 where d1.country = d2.country) + count(id) from data d2 group by all
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVED_ALL_IN_GROUP_BY",
+ "sqlState" : "42803",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 94,
+ "stopIndex" : 105,
+ "fragment" : "group by all"
+ } ]
+}
+
+
+-- !query
+select non_exist from data group by all
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION",
+ "sqlState" : "42703",
+ "messageParameters" : {
+ "objectName" : "`non_exist`",
+ "proposal" : "`data`.`city`, `data`.`id`, `data`.`name`, `data`.`power`, `data`.`country`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 16,
+ "fragment" : "non_exist"
+ } ]
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 76
-
-
-- !query
CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null)
@@ -51,7 +48,17 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-grouping expressions sequence is empty, and 'testdata.a' is not an aggregate function. Wrap '(count(testdata.b) FILTER (WHERE (testdata.a >= 2)) AS `count(b) FILTER (WHERE (a >= 2))`)' in windowing function(s) or wrap 'testdata.a' in first() (or first_value) if you don't care which value you get.
+{
+ "errorClass" : "MISSING_GROUP_BY",
+ "sqlState" : "42803",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 54,
+ "fragment" : "SELECT a, COUNT(b) FILTER (WHERE a >= 2) FROM testData"
+ } ]
+}
-- !query
@@ -231,7 +238,14 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-expression 'testdata.a' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.
+{
+ "errorClass" : "MISSING_AGGREGATION",
+ "sqlState" : "42803",
+ "messageParameters" : {
+ "expression" : "\"a\"",
+ "expressionAnyValue" : "\"any_value(a)\""
+ }
+}
-- !query
@@ -711,7 +725,14 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-expression 'testdata.a' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.
+{
+ "errorClass" : "MISSING_AGGREGATION",
+ "sqlState" : "42803",
+ "messageParameters" : {
+ "expression" : "\"a\"",
+ "expressionAnyValue" : "\"any_value(a)\""
+ }
+}
-- !query
@@ -788,24 +809,14 @@
FROM emp
GROUP BY dept_id
-- !query schema
-struct<>
+struct<dept_id:int,avg(salary):double,avg(salary) FILTER (WHERE exists(dept_id)):double>
-- !query output
-org.apache.spark.sql.AnalysisException
-IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, avg(salary#x) AS avg(salary)#x, avg(salary#x) FILTER (WHERE exists#x [dept_id#x]) AS avg(salary) FILTER (WHERE exists(dept_id))#x]
-: +- Project [state#x]
-: +- Filter (dept_id#x = outer(dept_id#x))
-: +- SubqueryAlias dept
-: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x])
-: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x]
-: +- Project [dept_id#x, dept_name#x, state#x]
-: +- SubqueryAlias DEPT
-: +- LocalRelation [dept_id#x, dept_name#x, state#x]
-+- SubqueryAlias emp
- +- View (`EMP`, [id#x,emp_name#x,hiredate#x,salary#x,dept_id#x])
- +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(hiredate#x as date) AS hiredate#x, cast(salary#x as double) AS salary#x, cast(dept_id#x as int) AS dept_id#x]
- +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x]
- +- SubqueryAlias EMP
- +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x]
+10 133.33333333333334 133.33333333333334
+100 400.0 NULL
+20 300.0 300.0
+30 400.0 400.0
+70 150.0 150.0
+NULL 400.0 NULL
-- !query
@@ -817,24 +828,14 @@
FROM emp
GROUP BY dept_id
-- !query schema
-struct<>
+struct<dept_id:int,sum(salary):double,sum(salary) FILTER (WHERE (NOT exists(dept_id))):double>
-- !query output
-org.apache.spark.sql.AnalysisException
-IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, sum(salary#x) AS sum(salary)#x, sum(salary#x) FILTER (WHERE NOT exists#x [dept_id#x]) AS sum(salary) FILTER (WHERE (NOT exists(dept_id)))#x]
-: +- Project [state#x]
-: +- Filter (dept_id#x = outer(dept_id#x))
-: +- SubqueryAlias dept
-: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x])
-: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x]
-: +- Project [dept_id#x, dept_name#x, state#x]
-: +- SubqueryAlias DEPT
-: +- LocalRelation [dept_id#x, dept_name#x, state#x]
-+- SubqueryAlias emp
- +- View (`EMP`, [id#x,emp_name#x,hiredate#x,salary#x,dept_id#x])
- +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(hiredate#x as date) AS hiredate#x, cast(salary#x as double) AS salary#x, cast(dept_id#x as int) AS dept_id#x]
- +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x]
- +- SubqueryAlias EMP
- +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x]
+10 400.0 NULL
+100 800.0 800.0
+20 300.0 NULL
+30 400.0 NULL
+70 150.0 NULL
+NULL 400.0 400.0
-- !query
@@ -845,24 +846,14 @@
FROM emp
GROUP BY dept_id
-- !query schema
-struct<>
+struct<dept_id:int,avg(salary):double,avg(salary) FILTER (WHERE (dept_id IN (listquery()))):double>
-- !query output
-org.apache.spark.sql.AnalysisException
-IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, avg(salary#x) AS avg(salary)#x, avg(salary#x) FILTER (WHERE dept_id#x IN (list#x [])) AS avg(salary) FILTER (WHERE (dept_id IN (listquery())))#x]
-: +- Distinct
-: +- Project [dept_id#x]
-: +- SubqueryAlias dept
-: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x])
-: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x]
-: +- Project [dept_id#x, dept_name#x, state#x]
-: +- SubqueryAlias DEPT
-: +- LocalRelation [dept_id#x, dept_name#x, state#x]
-+- SubqueryAlias emp
- +- View (`EMP`, [id#x,emp_name#x,hiredate#x,salary#x,dept_id#x])
- +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(hiredate#x as date) AS hiredate#x, cast(salary#x as double) AS salary#x, cast(dept_id#x as int) AS dept_id#x]
- +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x]
- +- SubqueryAlias EMP
- +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x]
+10 133.33333333333334 133.33333333333334
+100 400.0 NULL
+20 300.0 300.0
+30 400.0 400.0
+70 150.0 150.0
+NULL 400.0 NULL
-- !query
@@ -873,24 +864,14 @@
FROM emp
GROUP BY dept_id
-- !query schema
-struct<>
+struct<dept_id:int,sum(salary):double,sum(salary) FILTER (WHERE (NOT (dept_id IN (listquery())))):double>
-- !query output
-org.apache.spark.sql.AnalysisException
-IN/EXISTS predicate sub-queries can only be used in Filter/Join and a few commands: Aggregate [dept_id#x], [dept_id#x, sum(salary#x) AS sum(salary)#x, sum(salary#x) FILTER (WHERE NOT dept_id#x IN (list#x [])) AS sum(salary) FILTER (WHERE (NOT (dept_id IN (listquery()))))#x]
-: +- Distinct
-: +- Project [dept_id#x]
-: +- SubqueryAlias dept
-: +- View (`DEPT`, [dept_id#x,dept_name#x,state#x])
-: +- Project [cast(dept_id#x as int) AS dept_id#x, cast(dept_name#x as string) AS dept_name#x, cast(state#x as string) AS state#x]
-: +- Project [dept_id#x, dept_name#x, state#x]
-: +- SubqueryAlias DEPT
-: +- LocalRelation [dept_id#x, dept_name#x, state#x]
-+- SubqueryAlias emp
- +- View (`EMP`, [id#x,emp_name#x,hiredate#x,salary#x,dept_id#x])
- +- Project [cast(id#x as int) AS id#x, cast(emp_name#x as string) AS emp_name#x, cast(hiredate#x as date) AS hiredate#x, cast(salary#x as double) AS salary#x, cast(dept_id#x as int) AS dept_id#x]
- +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x]
- +- SubqueryAlias EMP
- +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x]
+10 400.0 NULL
+100 800.0 800.0
+20 300.0 NULL
+30 400.0 NULL
+70 150.0 NULL
+NULL 400.0 NULL
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 33
-
-
-- !query
create temporary view data as select * from values
(1, 1),
@@ -95,7 +92,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-GROUP BY position -1 is not in select list (valid range is [1, 2]); line 1 pos 31
+{
+ "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "-1",
+ "size" : "2"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 32,
+ "stopIndex" : 33,
+ "fragment" : "-1"
+ } ]
+}
-- !query
@@ -104,7 +115,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-GROUP BY position 0 is not in select list (valid range is [1, 2]); line 1 pos 31
+{
+ "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "0",
+ "size" : "2"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 32,
+ "stopIndex" : 32,
+ "fragment" : "0"
+ } ]
+}
-- !query
@@ -113,7 +138,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-GROUP BY position 3 is not in select list (valid range is [1, 2]); line 1 pos 31
+{
+ "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "3",
+ "size" : "2"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 32,
+ "stopIndex" : 32,
+ "fragment" : "3"
+ } ]
+}
-- !query
@@ -122,7 +161,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-GROUP BY 3 refers to an expression that is or contains an aggregate function. Aggregate functions are not allowed in GROUP BY, but got sum(data.b) AS `sum(b)`; line 1 pos 39
+{
+ "errorClass" : "GROUP_BY_POS_AGGREGATE",
+ "sqlState" : "42903",
+ "messageParameters" : {
+ "aggExpr" : "sum(data.b) AS `sum(b)`",
+ "index" : "3"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 40,
+ "stopIndex" : 40,
+ "fragment" : "3"
+ } ]
+}
-- !query
@@ -131,7 +184,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-GROUP BY 3 refers to an expression that is or contains an aggregate function. Aggregate functions are not allowed in GROUP BY, but got (sum(data.b) + CAST(2 AS BIGINT)) AS `(sum(b) + 2)`; line 1 pos 43
+{
+ "errorClass" : "GROUP_BY_POS_AGGREGATE",
+ "sqlState" : "42903",
+ "messageParameters" : {
+ "aggExpr" : "(sum(data.b) + CAST(2 AS BIGINT)) AS `(sum(b) + 2)`",
+ "index" : "3"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 44,
+ "stopIndex" : 44,
+ "fragment" : "3"
+ } ]
+}
-- !query
@@ -155,7 +222,10 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Star (*) is not allowed in select list when GROUP BY ordinal position is used
+{
+ "errorClass" : "STAR_GROUP_BY_POS",
+ "sqlState" : "0A000"
+}
-- !query
@@ -352,7 +422,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-GROUP BY position -1 is not in select list (valid range is [1, 3]); line 1 pos 44
+{
+ "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "-1",
+ "size" : "3"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 45,
+ "stopIndex" : 46,
+ "fragment" : "-1"
+ } ]
+}
-- !query
@@ -361,7 +445,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-GROUP BY 3 refers to an expression that is or contains an aggregate function. Aggregate functions are not allowed in GROUP BY, but got count(1) AS `count(1)`; line 1 pos 44
+{
+ "errorClass" : "GROUP_BY_POS_AGGREGATE",
+ "sqlState" : "42903",
+ "messageParameters" : {
+ "aggExpr" : "count(1) AS `count(1)`",
+ "index" : "3"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 45,
+ "stopIndex" : 45,
+ "fragment" : "3"
+ } ]
+}
-- !query
@@ -370,7 +468,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-GROUP BY position -1 is not in select list (valid range is [1, 3]); line 1 pos 46
+{
+ "errorClass" : "GROUP_BY_POS_OUT_OF_RANGE",
+ "sqlState" : "42805",
+ "messageParameters" : {
+ "index" : "-1",
+ "size" : "3"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 47,
+ "stopIndex" : 48,
+ "fragment" : "-1"
+ } ]
+}
-- !query
@@ -379,7 +491,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-GROUP BY 3 refers to an expression that is or contains an aggregate function. Aggregate functions are not allowed in GROUP BY, but got count(1) AS `count(1)`; line 1 pos 49
+{
+ "errorClass" : "GROUP_BY_POS_AGGREGATE",
+ "sqlState" : "42903",
+ "messageParameters" : {
+ "aggExpr" : "count(1) AS `count(1)`",
+ "index" : "3"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 50,
+ "stopIndex" : 50,
+ "fragment" : "3"
+ } ]
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/group-by.sql.out sql/core/src/test/resources/sql-tests/results/group-by.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/group-by.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/group-by.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 101
-
-
-- !query
CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES
(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null)
@@ -13,32 +10,22 @@
-- !query
-CREATE OR REPLACE TEMPORARY VIEW testRegression AS SELECT * FROM VALUES
-(1, 10, null), (2, 10, 11), (2, 20, 22), (2, 25, null), (2, 30, 35)
-AS testRegression(k, y, x)
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
-CREATE OR REPLACE TEMPORARY VIEW aggr AS SELECT * FROM VALUES
-(0, 0), (0, 10), (0, 20), (0, 30), (0, 40), (1, 10), (1, 20), (2, 10), (2, 20), (2, 25), (2, 30), (3, 60), (4, null)
-AS aggr(k, v)
--- !query schema
-struct<>
--- !query output
-
-
-
--- !query
SELECT a, COUNT(b) FROM testData
-- !query schema
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-grouping expressions sequence is empty, and 'testdata.a' is not an aggregate function. Wrap '(count(testdata.b) AS `count(b)`)' in windowing function(s) or wrap 'testdata.a' in first() (or first_value) if you don't care which value you get.
+{
+ "errorClass" : "MISSING_GROUP_BY",
+ "sqlState" : "42803",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 32,
+ "fragment" : "SELECT a, COUNT(b) FROM testData"
+ } ]
+}
-- !query
@@ -66,7 +53,14 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-expression 'testdata.a' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.
+{
+ "errorClass" : "MISSING_AGGREGATION",
+ "sqlState" : "42803",
+ "messageParameters" : {
+ "expression" : "\"a\"",
+ "expressionAnyValue" : "\"any_value(a)\""
+ }
+}
-- !query
@@ -130,7 +124,14 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-expression 'testdata.a' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.
+{
+ "errorClass" : "MISSING_AGGREGATION",
+ "sqlState" : "42803",
+ "messageParameters" : {
+ "expression" : "\"a\"",
+ "expressionAnyValue" : "\"any_value(a)\""
+ }
+}
-- !query
@@ -145,6 +146,17 @@
-- !query
+SELECT count(1) FROM testData GROUP BY struct(a + 0.1 AS aa)
+-- !query schema
+struct<count(1):bigint>
+-- !query output
+2
+2
+2
+3
+
+
+-- !query
SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a)
FROM testData
-- !query schema
@@ -187,7 +199,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Column 'non_existing' does not exist. Did you mean one of the following? [testdata.a, testdata.b]; line 1 pos 21
+{
+ "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION",
+ "sqlState" : "42703",
+ "messageParameters" : {
+ "objectName" : "`non_existing`",
+ "proposal" : "`testdata`.`a`, `testdata`.`b`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 22,
+ "stopIndex" : 33,
+ "fragment" : "non_existing"
+ } ]
+}
-- !query
@@ -196,7 +222,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-aggregate functions are not allowed in GROUP BY, but found count(testdata.b)
+{
+ "errorClass" : "GROUP_BY_AGGREGATE",
+ "sqlState" : "42903",
+ "messageParameters" : {
+ "sqlExpr" : "count(testdata.b)"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 15,
+ "fragment" : "COUNT(b)"
+ } ]
+}
-- !query
@@ -214,7 +253,14 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-expression 'testdatahassamenamewithalias.k' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.
+{
+ "errorClass" : "MISSING_AGGREGATION",
+ "sqlState" : "42803",
+ "messageParameters" : {
+ "expression" : "\"k\"",
+ "expressionAnyValue" : "\"any_value(k)\""
+ }
+}
-- !query
@@ -231,7 +277,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Column 'k' does not exist. Did you mean one of the following? [testdata.a, testdata.b]; line 1 pos 47
+{
+ "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION",
+ "sqlState" : "42703",
+ "messageParameters" : {
+ "objectName" : "`k`",
+ "proposal" : "`testdata`.`a`, `testdata`.`b`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 48,
+ "stopIndex" : 48,
+ "fragment" : "k"
+ } ]
+}
-- !query
@@ -303,7 +363,17 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-grouping expressions sequence is empty, and 'id' is not an aggregate function. Wrap '()' in windowing function(s) or wrap 'id' in first() (or first_value) if you don't care which value you get.
+{
+ "errorClass" : "MISSING_GROUP_BY",
+ "sqlState" : "42803",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 38,
+ "fragment" : "SELECT id FROM range(10) HAVING id > 0"
+ } ]
+}
-- !query
@@ -337,10 +407,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-
-Aggregate/Window/Generate expressions are not valid in where clause of the query.
-Expression in where clause: [(max(id) > CAST(0 AS BIGINT))]
-Invalid expressions: [max(id)]
+{
+ "errorClass" : "INVALID_WHERE_CONDITION",
+ "sqlState" : "42903",
+ "messageParameters" : {
+ "condition" : "\"(max(id) > 0)\"",
+ "expressionList" : "max(id)"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 42,
+ "fragment" : "SELECT 1 FROM range(10) HAVING MAX(id) > 0"
+ } ]
+}
-- !query
@@ -470,7 +551,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'every(1)' due to data type mismatch: argument 1 requires boolean type, however, '1' is of int type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"every(1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 15,
+ "fragment" : "every(1)"
+ } ]
+}
-- !query
@@ -479,7 +577,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'some(1S)' due to data type mismatch: argument 1 requires boolean type, however, '1S' is of smallint type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"SMALLINT\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"some(1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 15,
+ "fragment" : "some(1S)"
+ } ]
+}
-- !query
@@ -488,7 +603,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'any(1L)' due to data type mismatch: argument 1 requires boolean type, however, '1L' is of bigint type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"BIGINT\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"any(1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 14,
+ "fragment" : "any(1L)"
+ } ]
+}
-- !query
@@ -497,7 +629,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'every('true')' due to data type mismatch: argument 1 requires boolean type, however, ''true'' is of string type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"true\"",
+ "inputType" : "\"STRING\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"every(true)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 20,
+ "fragment" : "every(\"true\")"
+ } ]
+}
-- !query
@@ -506,7 +655,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'bool_and(1.0BD)' due to data type mismatch: argument 1 requires boolean type, however, '1.0BD' is of decimal(2,1) type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1.0\"",
+ "inputType" : "\"DECIMAL(2,1)\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"bool_and(1.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 20,
+ "fragment" : "bool_and(1.0)"
+ } ]
+}
-- !query
@@ -515,7 +681,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'bool_or(1.0D)' due to data type mismatch: argument 1 requires boolean type, however, '1.0D' is of double type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1.0\"",
+ "inputType" : "\"DOUBLE\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"BOOLEAN\"",
+ "sqlExpr" : "\"bool_or(1.0)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 20,
+ "fragment" : "bool_or(1.0D)"
+ } ]
+}
-- !query
@@ -635,10 +818,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-
-Aggregate/Window/Generate expressions are not valid in where clause of the query.
-Expression in where clause: [(count(1) > 1L)]
-Invalid expressions: [count(1)]
+{
+ "errorClass" : "INVALID_WHERE_CONDITION",
+ "sqlState" : "42903",
+ "messageParameters" : {
+ "condition" : "\"(count(1) > 1)\"",
+ "expressionList" : "count(1)"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 49,
+ "fragment" : "SELECT count(*) FROM test_agg WHERE count(*) > 1L"
+ } ]
+}
-- !query
@@ -647,10 +841,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-
-Aggregate/Window/Generate expressions are not valid in where clause of the query.
-Expression in where clause: [((count(1) + 1L) > 1L)]
-Invalid expressions: [count(1)]
+{
+ "errorClass" : "INVALID_WHERE_CONDITION",
+ "sqlState" : "42903",
+ "messageParameters" : {
+ "condition" : "\"((count(1) + 1) > 1)\"",
+ "expressionList" : "count(1)"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 54,
+ "fragment" : "SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L"
+ } ]
+}
-- !query
@@ -659,10 +864,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-
-Aggregate/Window/Generate expressions are not valid in where clause of the query.
-Expression in where clause: [(((test_agg.k = 1) OR (test_agg.k = 2)) OR (((count(1) + 1L) > 1L) OR (max(test_agg.k) > 1)))]
-Invalid expressions: [count(1), max(test_agg.k)]
+{
+ "errorClass" : "INVALID_WHERE_CONDITION",
+ "sqlState" : "42903",
+ "messageParameters" : {
+ "condition" : "\"(((k = 1) OR (k = 2)) OR (((count(1) + 1) > 1) OR (max(k) > 1)))\"",
+ "expressionList" : "count(1), max(test_agg.k)"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 86,
+ "fragment" : "SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1"
+ } ]
+}
-- !query
@@ -844,74 +1060,6 @@
-- !query
-SELECT regr_count(y, x) FROM testRegression
--- !query schema
-struct<regr_count(y, x):bigint>
--- !query output
-3
-
-
--- !query
-SELECT regr_count(y, x) FROM testRegression WHERE x IS NOT NULL
--- !query schema
-struct<regr_count(y, x):bigint>
--- !query output
-3
-
-
--- !query
-SELECT k, count(*), regr_count(y, x) FROM testRegression GROUP BY k
--- !query schema
-struct<k:int,count(1):bigint,regr_count(y, x):bigint>
--- !query output
-1 1 0
-2 4 3
-
-
--- !query
-SELECT k, count(*) FILTER (WHERE x IS NOT NULL), regr_count(y, x) FROM testRegression GROUP BY k
--- !query schema
-struct<k:int,count(1) FILTER (WHERE (x IS NOT NULL)):bigint,regr_count(y, x):bigint>
--- !query output
-1 0 0
-2 3 3
-
-
--- !query
-SELECT regr_r2(y, x) FROM testRegression
--- !query schema
-struct<regr_r2(y, x):double>
--- !query output
-0.997690531177829
-
-
--- !query
-SELECT regr_r2(y, x) FROM testRegression WHERE x IS NOT NULL
--- !query schema
-struct<regr_r2(y, x):double>
--- !query output
-0.997690531177829
-
-
--- !query
-SELECT k, corr(y, x), regr_r2(y, x) FROM testRegression GROUP BY k
--- !query schema
-struct<k:int,corr(y, x):double,regr_r2(y, x):double>
--- !query output
-1 NULL NULL
-2 0.9988445981121533 0.997690531177829
-
-
--- !query
-SELECT k, corr(y, x) FILTER (WHERE x IS NOT NULL), regr_r2(y, x) FROM testRegression GROUP BY k
--- !query schema
-struct<k:int,corr(y, x) FILTER (WHERE (x IS NOT NULL)):double,regr_r2(y, x):double>
--- !query output
-1 NULL NULL
-2 0.9988445981121533 0.997690531177829
-
-
--- !query
SELECT
collect_list(col),
array_agg(col)
@@ -939,92 +1087,19 @@
-- !query
-SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression
+SELECT mode(a), mode(b) FROM testData
-- !query schema
-struct<regr_avgx(y, x):double,regr_avgy(y, x):double>
+struct<mode(a):int,mode(b):int>
-- !query output
-22.666666666666668 20.0
+3 1
-- !query
-SELECT regr_avgx(y, x), regr_avgy(y, x) FROM testRegression WHERE x IS NOT NULL AND y IS NOT NULL
+SELECT a, mode(b) FROM testData GROUP BY a ORDER BY a
-- !query schema
-struct<regr_avgx(y, x):double,regr_avgy(y, x):double>
+struct<a:int,mode(b):int>
-- !query output
-22.666666666666668 20.0
-
-
--- !query
-SELECT k, avg(x), avg(y), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k
--- !query schema
-struct<k:int,avg(x):double,avg(y):double,regr_avgx(y, x):double,regr_avgy(y, x):double>
--- !query output
-1 NULL 10.0 NULL NULL
-2 22.666666666666668 21.25 22.666666666666668 20.0
-
-
--- !query
-SELECT k, avg(x) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), avg(y) FILTER (WHERE x IS NOT NULL AND y IS NOT NULL), regr_avgx(y, x), regr_avgy(y, x) FROM testRegression GROUP BY k
--- !query schema
-struct<k:int,avg(x) FILTER (WHERE ((x IS NOT NULL) AND (y IS NOT NULL))):double,avg(y) FILTER (WHERE ((x IS NOT NULL) AND (y IS NOT NULL))):double,regr_avgx(y, x):double,regr_avgy(y, x):double>
--- !query output
-1 NULL NULL NULL NULL
-2 22.666666666666668 20.0 22.666666666666668 20.0
-
-
--- !query
-SELECT
- percentile_cont(0.25) WITHIN GROUP (ORDER BY v),
- percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC)
-FROM aggr
--- !query schema
-struct<percentile_cont(0.25) WITHIN GROUP (ORDER BY v):double,percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC):double>
--- !query output
-10.0 30.0
-
-
--- !query
-SELECT
- k,
- percentile_cont(0.25) WITHIN GROUP (ORDER BY v),
- percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC)
-FROM aggr
-GROUP BY k
-ORDER BY k
--- !query schema
-struct<k:int,percentile_cont(0.25) WITHIN GROUP (ORDER BY v):double,percentile_cont(0.25) WITHIN GROUP (ORDER BY v DESC):double>
--- !query output
-0 10.0 30.0
-1 12.5 17.5
-2 17.5 26.25
-3 60.0 60.0
-4 NULL NULL
-
-
--- !query
-SELECT
- percentile_disc(0.25) WITHIN GROUP (ORDER BY v),
- percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC)
-FROM aggr
--- !query schema
-struct<percentile_disc(0.25) WITHIN GROUP (ORDER BY v):double,percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC):double>
--- !query output
-10.0 30.0
-
-
--- !query
-SELECT
- k,
- percentile_disc(0.25) WITHIN GROUP (ORDER BY v),
- percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC)
-FROM aggr
-GROUP BY k
-ORDER BY k
--- !query schema
-struct<k:int,percentile_disc(0.25) WITHIN GROUP (ORDER BY v):double,percentile_disc(0.25) WITHIN GROUP (ORDER BY v DESC):double>
--- !query output
-0 10.0 30.0
-1 10.0 20.0
-2 10.0 30.0
-3 60.0 60.0
-4 NULL NULL
+NULL 1
+1 1
+2 1
+3 1
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 20
-
-
-- !query
CREATE TEMPORARY VIEW grouping AS SELECT * FROM VALUES
("1", "2", "3", 1),
@@ -137,12 +134,14 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Syntax error at or near 'ROLLUP': extra input 'ROLLUP'(line 1, pos 53)
-
-== SQL ==
-SELECT a, b, c, count(d) FROM grouping GROUP BY WITH ROLLUP
------------------------------------------------------^^^
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'ROLLUP'",
+ "hint" : ": extra input 'ROLLUP'"
+ }
+}
-- !query
@@ -151,12 +150,14 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Syntax error at or near 'CUBE': extra input 'CUBE'(line 1, pos 53)
-
-== SQL ==
-SELECT a, b, c, count(d) FROM grouping GROUP BY WITH CUBE
------------------------------------------------------^^^
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'CUBE'",
+ "hint" : ": extra input 'CUBE'"
+ }
+}
-- !query
@@ -165,7 +166,14 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-expression 't.c1' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.
+{
+ "errorClass" : "MISSING_AGGREGATION",
+ "sqlState" : "42803",
+ "messageParameters" : {
+ "expression" : "\"c1\"",
+ "expressionAnyValue" : "\"any_value(c1)\""
+ }
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/having.sql.out sql/core/src/test/resources/sql-tests/results/having.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/having.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/having.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 13
-
-
-- !query
create temporary view hav as select * from values
("one", 1),
@@ -33,6 +30,30 @@
-- !query
+SELECT count(k) FROM hav GROUP BY v HAVING v = array(1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INT\"",
+ "right" : "\"ARRAY<INT>\"",
+ "sqlExpr" : "\"(v = array(1))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 44,
+ "stopIndex" : 55,
+ "fragment" : "v = array(1)"
+ } ]
+}
+
+
+-- !query
SELECT MIN(t.v) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(COUNT(1) > 0)
-- !query schema
struct<min(v):int>
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 30
-
-
-- !query
create or replace temporary view nested as values
(1, array(32, 97), array(array(12, 99), array(123, 42), array(1))),
@@ -20,7 +17,19 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-A lambda function should only be used in a higher order function. However, its class is org.apache.spark.sql.catalyst.expressions.Upper, which is not a higher order function.; line 1 pos 7
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_2306",
+ "messageParameters" : {
+ "class" : "org.apache.spark.sql.catalyst.expressions.Upper"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 20,
+ "fragment" : "upper(x -> x)"
+ } ]
+}
-- !query
@@ -148,6 +157,44 @@
-- !query
+select reduce(ys, 0, (y, a) -> y + a + x) as v from nested
+-- !query schema
+struct<v:int>
+-- !query output
+131
+15
+5
+
+
+-- !query
+select reduce(ys, (0 as sum, 0 as n), (acc, x) -> (acc.sum + x, acc.n + 1), acc -> acc.sum / acc.n) as v from nested
+-- !query schema
+struct<v:double>
+-- !query output
+0.5
+12.0
+64.5
+
+
+-- !query
+select transform(zs, z -> reduce(z, 1, (acc, val) -> acc * val * size(z))) as v from nested
+-- !query schema
+struct<v:array<int>>
+-- !query output
+[1010880,8]
+[17]
+[4752,20664,1]
+
+
+-- !query
+select reduce(cast(null as array<int>), 0, (a, y) -> a + y + 1, a -> a + 2) as v
+-- !query schema
+struct<v:int>
+-- !query output
+NULL
+
+
+-- !query
select exists(ys, y -> y > 30) as v from nested
-- !query schema
struct<v:boolean>
@@ -277,4 +324,4 @@
-- !query schema
struct<aggregate(split(abcdefgh, , -1), array(array()), lambdafunction(array(array(namedlambdavariable())), namedlambdavariable(), namedlambdavariable()), lambdafunction(namedlambdavariable(), namedlambdavariable())):array<array<string>>>
-- !query output
-[[""]]
+[["h"]]
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ilike-all.sql.out sql/core/src/test/resources/sql-tests/results/ilike-all.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ilike-all.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ilike-all.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 14
-
-
-- !query
CREATE OR REPLACE TEMPORARY VIEW ilike_all_table AS SELECT * FROM (VALUES
('gOOgle', '%oo%'),
@@ -132,9 +129,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Expected something between '(' and ')'.(line 1, pos 50)
-
-== SQL ==
-SELECT company FROM ilike_any_table WHERE company ILIKE ALL ()
---------------------------------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0064",
+ "messageParameters" : {
+ "msg" : "Expected something between '(' and ')'."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 51,
+ "stopIndex" : 62,
+ "fragment" : "ILIKE ALL ()"
+ } ]
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ilike-any.sql.out sql/core/src/test/resources/sql-tests/results/ilike-any.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ilike-any.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ilike-any.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 14
-
-
-- !query
CREATE OR REPLACE TEMPORARY VIEW ilike_any_table AS SELECT * FROM (VALUES
('Google', '%Oo%'),
@@ -138,9 +135,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Expected something between '(' and ')'.(line 1, pos 50)
-
-== SQL ==
-SELECT company FROM ilike_any_table WHERE company ILIKE ANY ()
---------------------------------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0064",
+ "messageParameters" : {
+ "msg" : "Expected something between '(' and ')'."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 51,
+ "stopIndex" : 62,
+ "fragment" : "ILIKE ANY ()"
+ } ]
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ilike.sql.out sql/core/src/test/resources/sql-tests/results/ilike.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/ilike.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/ilike.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 26
-
-
-- !query
select null ilike 'a'
-- !query schema
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out sql/core/src/test/resources/sql-tests/results/inline-table.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/inline-table.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 19
-
-
-- !query
select * from values ("one", 1)
-- !query schema
@@ -113,7 +110,19 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot evaluate expression rand(5) in inline table definition; line 1 pos 29
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_2304",
+ "messageParameters" : {
+ "sqlExpr" : "rand(5)"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 30,
+ "stopIndex" : 36,
+ "fragment" : "rand(5)"
+ } ]
+}
-- !query
@@ -122,7 +131,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-expected 2 columns but found 1 columns in row 1; line 1 pos 14
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_2305",
+ "messageParameters" : {
+ "numCols" : "2",
+ "ri" : "1",
+ "rowSize" : "1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 56,
+ "fragment" : "values (\"one\", 2.0), (\"two\") as data(a, b)"
+ } ]
+}
-- !query
@@ -131,7 +154,19 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-incompatible types found in column b for inline table; line 1 pos 14
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_2303",
+ "messageParameters" : {
+ "name" : "b"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 78,
+ "fragment" : "values (\"one\", array(0, 1)), (\"two\", struct(1, 2)) as data(a, b)"
+ } ]
+}
-- !query
@@ -140,7 +175,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-expected 2 columns but found 1 columns in row 0; line 1 pos 14
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_2305",
+ "messageParameters" : {
+ "numCols" : "2",
+ "ri" : "0",
+ "rowSize" : "1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 51,
+ "fragment" : "values (\"one\"), (\"two\") as data(a, b)"
+ } ]
+}
-- !query
@@ -149,7 +198,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Undefined function: random_not_exist_func. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.random_not_exist_func.; line 1 pos 29
+{
+ "errorClass" : "UNRESOLVED_ROUTINE",
+ "sqlState" : "42883",
+ "messageParameters" : {
+ "routineName" : "`random_not_exist_func`",
+ "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 30,
+ "stopIndex" : 53,
+ "fragment" : "random_not_exist_func(1)"
+ } ]
+}
-- !query
@@ -158,7 +221,19 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot evaluate expression count(1) in inline table definition; line 1 pos 29
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_2304",
+ "messageParameters" : {
+ "sqlExpr" : "count(1)"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 30,
+ "stopIndex" : 37,
+ "fragment" : "count(1)"
+ } ]
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/inner-join.sql.out sql/core/src/test/resources/sql-tests/results/inner-join.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/inner-join.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/inner-join.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 7
-
-
-- !query
CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a)
-- !query schema
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/intersect-all.sql.out sql/core/src/test/resources/sql-tests/results/intersect-all.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/intersect-all.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/intersect-all.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 26
-
-
-- !query
CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES
(1, 2),
@@ -98,7 +95,25 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-IntersectAll can only be performed on tables with the compatible column types. The first column of the second table is array<int> type which is not compatible with int at same column of first table
+{
+ "errorClass" : "INCOMPATIBLE_COLUMN_TYPE",
+ "sqlState" : "42825",
+ "messageParameters" : {
+ "columnOrdinalNumber" : "first",
+ "dataType1" : "\"ARRAY<INT>\"",
+ "dataType2" : "\"INT\"",
+ "hint" : "",
+ "operator" : "INTERSECT ALL",
+ "tableOrdinalNumber" : "second"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 51,
+ "fragment" : "SELECT * FROM tab1\nINTERSECT ALL\nSELECT array(1), 2"
+ } ]
+}
-- !query
@@ -109,7 +124,23 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-IntersectAll can only be performed on tables with the same number of columns, but the first table has 1 columns and the second table has 2 columns
+{
+ "errorClass" : "NUM_COLUMNS_MISMATCH",
+ "sqlState" : "42826",
+ "messageParameters" : {
+ "firstNumColumns" : "1",
+ "invalidNumColumns" : "2",
+ "invalidOrdinalNum" : "second",
+ "operator" : "INTERSECT ALL"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 1,
+ "stopIndex" : 54,
+ "fragment" : "SELECT k FROM tab1\nINTERSECT ALL\nSELECT k, v FROM tab2"
+ } ]
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/interval.sql.out sql/core/src/test/resources/sql-tests/results/interval.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/interval.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/interval.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 285
-
-
-- !query
select 3 * (timestamp'2019-10-15 10:11:12.001002' - date'2019-10-15')
-- !query schema
@@ -16,12 +13,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot mix year-month and day-time fields: interval 4 month 2 weeks 3 microseconds(line 1, pos 7)
-
-== SQL ==
-select interval 4 month 2 weeks 3 microseconds * 1.5
--------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0029",
+ "messageParameters" : {
+ "literal" : "interval 4 month 2 weeks 3 microseconds"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 46,
+ "fragment" : "interval 4 month 2 weeks 3 microseconds"
+ } ]
+}
-- !query
@@ -186,7 +190,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(CAST('2' AS DOUBLE) / INTERVAL '02' SECOND)' due to data type mismatch: differing types in '(CAST('2' AS DOUBLE) / INTERVAL '02' SECOND)' (double and interval second).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"DOUBLE\"",
+ "right" : "\"INTERVAL SECOND\"",
+ "sqlExpr" : "\"(2 / INTERVAL '02' SECOND)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "'2' / interval 2 second"
+ } ]
+}
-- !query
@@ -195,7 +214,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(CAST('2' AS DOUBLE) / INTERVAL '2' YEAR)' due to data type mismatch: differing types in '(CAST('2' AS DOUBLE) / INTERVAL '2' YEAR)' (double and interval year).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"DOUBLE\"",
+ "right" : "\"INTERVAL YEAR\"",
+ "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "'2' / interval 2 year"
+ } ]
+}
-- !query
@@ -204,10 +238,17 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error.
-== SQL(line 1, position 8) ==
-select interval '2 seconds' / 0
- ^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "INTERVAL_DIVIDED_BY_ZERO",
+ "sqlState" : "22012",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 31,
+ "fragment" : "interval '2 seconds' / 0"
+ } ]
+}
-- !query
@@ -240,10 +281,17 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" (except for ANSI interval type) to bypass this error.
-== SQL(line 1, position 8) ==
-select interval '2' year / 0
- ^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "INTERVAL_DIVIDED_BY_ZERO",
+ "sqlState" : "22012",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "interval '2' year / 0"
+ } ]
+}
-- !query
@@ -276,7 +324,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(2 / INTERVAL '2' YEAR)' due to data type mismatch: differing types in '(2 / INTERVAL '2' YEAR)' (int and interval year).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INT\"",
+ "right" : "\"INTERVAL YEAR\"",
+ "sqlExpr" : "\"(2 / INTERVAL '2' YEAR)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "2 / interval '2' year"
+ } ]
+}
-- !query
@@ -285,7 +348,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(2 / INTERVAL '02' HOUR)' due to data type mismatch: differing types in '(2 / INTERVAL '02' HOUR)' (int and interval hour).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INT\"",
+ "right" : "\"INTERVAL HOUR\"",
+ "sqlExpr" : "\"(2 / INTERVAL '02' HOUR)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "2 / interval '2' hour"
+ } ]
+}
-- !query
@@ -294,7 +372,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(NULL / INTERVAL '2' YEAR)' due to data type mismatch: differing types in '(NULL / INTERVAL '2' YEAR)' (void and interval year).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"VOID\"",
+ "right" : "\"INTERVAL YEAR\"",
+ "sqlExpr" : "\"(NULL / INTERVAL '2' YEAR)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 31,
+ "fragment" : "null / interval '2' year"
+ } ]
+}
-- !query
@@ -303,7 +396,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(NULL / INTERVAL '02' HOUR)' due to data type mismatch: differing types in '(NULL / INTERVAL '02' HOUR)' (void and interval hour).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"VOID\"",
+ "right" : "\"INTERVAL HOUR\"",
+ "sqlExpr" : "\"(NULL / INTERVAL '02' HOUR)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 31,
+ "fragment" : "null / interval '2' hour"
+ } ]
+}
-- !query
@@ -312,12 +420,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot mix year-month and day-time fields: interval '-1 month 1 day -1 second'(line 1, pos 8)
-
-== SQL ==
-select -interval '-1 month 1 day -1 second'
---------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0029",
+ "messageParameters" : {
+ "literal" : "interval '-1 month 1 day -1 second'"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 9,
+ "stopIndex" : 43,
+ "fragment" : "interval '-1 month 1 day -1 second'"
+ } ]
+}
-- !query
@@ -342,12 +457,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot mix year-month and day-time fields: interval -1 month 1 day -1 second(line 1, pos 8)
-
-== SQL ==
-select -interval -1 month 1 day -1 second
---------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0029",
+ "messageParameters" : {
+ "literal" : "interval -1 month 1 day -1 second"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 9,
+ "stopIndex" : 41,
+ "fragment" : "interval -1 month 1 day -1 second"
+ } ]
+}
-- !query
@@ -372,12 +494,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot mix year-month and day-time fields: interval '-1 month 1 day -1 second'(line 1, pos 8)
-
-== SQL ==
-select +interval '-1 month 1 day -1 second'
---------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0029",
+ "messageParameters" : {
+ "literal" : "interval '-1 month 1 day -1 second'"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 9,
+ "stopIndex" : 43,
+ "fragment" : "interval '-1 month 1 day -1 second'"
+ } ]
+}
-- !query
@@ -402,12 +531,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot mix year-month and day-time fields: interval -1 month 1 day -1 second(line 1, pos 8)
-
-== SQL ==
-select +interval -1 month 1 day -1 second
---------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0029",
+ "messageParameters" : {
+ "literal" : "interval -1 month 1 day -1 second"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 9,
+ "stopIndex" : 41,
+ "fragment" : "interval -1 month 1 day -1 second"
+ } ]
+}
-- !query
@@ -819,12 +955,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot mix year-month and day-time fields: interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond(line 1, pos 7)
-
-== SQL ==
-select interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond
--------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0029",
+ "messageParameters" : {
+ "literal" : "interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 97,
+ "fragment" : "interval 1 year 2 month 3 week 4 day 5 hour 6 minute 7 seconds 8 millisecond 9 microsecond"
+ } ]
+}
-- !query
@@ -857,12 +1000,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot mix year-month and day-time fields: interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second(line 1, pos 7)
-
-== SQL ==
-select interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second
--------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0029",
+ "messageParameters" : {
+ "literal" : "interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 92,
+ "fragment" : "interval '30' year '25' month '-100' day '40' hour '80' minute '299.889987299' second"
+ } ]
+}
-- !query
@@ -991,12 +1141,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Interval string does not match day-time format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval day to hour: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16)
-
-== SQL ==
-select interval '20 15:40:32.99899999' day to hour
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "Interval string does not match day-time format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval day to hour: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 50,
+ "fragment" : "'20 15:40:32.99899999' day to hour"
+ } ]
+}
-- !query
@@ -1005,12 +1162,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Interval string does not match day-time format of `[+|-]d h:m`, `INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE` when cast to interval day to minute: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16)
-
-== SQL ==
-select interval '20 15:40:32.99899999' day to minute
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "Interval string does not match day-time format of `[+|-]d h:m`, `INTERVAL [+|-]'[+|-]d h:m' DAY TO MINUTE` when cast to interval day to minute: 20 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 52,
+ "fragment" : "'20 15:40:32.99899999' day to minute"
+ } ]
+}
-- !query
@@ -1019,12 +1183,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Interval string does not match day-time format of `[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE` when cast to interval hour to minute: 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16)
-
-== SQL ==
-select interval '15:40:32.99899999' hour to minute
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "Interval string does not match day-time format of `[+|-]h:m`, `INTERVAL [+|-]'[+|-]h:m' HOUR TO MINUTE` when cast to interval hour to minute: 15:40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 50,
+ "fragment" : "'15:40:32.99899999' hour to minute"
+ } ]
+}
-- !query
@@ -1033,12 +1204,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16)
-
-== SQL ==
-select interval '15:40.99899999' hour to second
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 47,
+ "fragment" : "'15:40.99899999' hour to second"
+ } ]
+}
-- !query
@@ -1047,12 +1225,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16)
-
-== SQL ==
-select interval '15:40' hour to second
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "Interval string does not match day-time format of `[+|-]h:m:s.n`, `INTERVAL [+|-]'[+|-]h:m:s.n' HOUR TO SECOND` when cast to interval hour to second: 15:40, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 38,
+ "fragment" : "'15:40' hour to second"
+ } ]
+}
-- !query
@@ -1061,12 +1246,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Interval string does not match day-time format of `[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND` when cast to interval minute to second: 20 40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16)
-
-== SQL ==
-select interval '20 40:32.99899999' minute to second
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "Interval string does not match day-time format of `[+|-]m:s.n`, `INTERVAL [+|-]'[+|-]m:s.n' MINUTE TO SECOND` when cast to interval minute to second: 20 40:32.99899999, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 52,
+ "fragment" : "'20 40:32.99899999' minute to second"
+ } ]
+}
-- !query
@@ -1075,12 +1267,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Error parsing ' 10 nanoseconds' to interval, invalid unit 'nanoseconds'(line 1, pos 16)
-
-== SQL ==
-select interval 10 nanoseconds
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0062",
+ "messageParameters" : {
+ "msg" : "Error parsing ' 10 nanoseconds' to interval, invalid unit 'nanoseconds'"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 30,
+ "fragment" : "10 nanoseconds"
+ } ]
+}
-- !query
@@ -1169,12 +1368,14 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Error parsing ' 1 fake_unit' to interval, invalid unit 'fake_unit'(line 1, pos 16)
-
-== SQL ==
-select interval 1 fake_unit
-----------------^^^
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'fake_unit'",
+ "hint" : ""
+ }
+}
-- !query
@@ -1183,12 +1384,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-The value of from-to unit must be a string(line 1, pos 16)
-
-== SQL ==
-select interval 1 year to month
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0027",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 31,
+ "fragment" : "1 year to month"
+ } ]
+}
-- !query
@@ -1197,12 +1402,20 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Intervals FROM year TO second are not supported.(line 1, pos 16)
-
-== SQL ==
-select interval '1' year to second
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0028",
+ "messageParameters" : {
+ "from" : "year",
+ "to" : "second"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 34,
+ "fragment" : "'1' year to second"
+ } ]
+}
-- !query
@@ -1211,12 +1424,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Can only have a single from-to unit in the interval literal syntax(line 1, pos 37)
-
-== SQL ==
-select interval '10-9' year to month '2-1' year to month
--------------------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0024",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 56,
+ "fragment" : "interval '10-9' year to month '2-1' year to month"
+ } ]
+}
-- !query
@@ -1225,12 +1442,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Can only have a single from-to unit in the interval literal syntax(line 1, pos 37)
-
-== SQL ==
-select interval '10-9' year to month '12:11:10' hour to second
--------------------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0024",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 62,
+ "fragment" : "interval '10-9' year to month '12:11:10' hour to second"
+ } ]
+}
-- !query
@@ -1239,12 +1460,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Can only have a single from-to unit in the interval literal syntax(line 1, pos 40)
-
-== SQL ==
-select interval '1 15:11' day to minute '12:11:10' hour to second
-----------------------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0024",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 65,
+ "fragment" : "interval '1 15:11' day to minute '12:11:10' hour to second"
+ } ]
+}
-- !query
@@ -1253,12 +1478,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Can only have a single from-to unit in the interval literal syntax(line 1, pos 23)
-
-== SQL ==
-select interval 1 year '2-1' year to month
------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0024",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 42,
+ "fragment" : "interval 1 year '2-1' year to month"
+ } ]
+}
-- !query
@@ -1267,12 +1496,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Can only have a single from-to unit in the interval literal syntax(line 1, pos 23)
-
-== SQL ==
-select interval 1 year '12:11:10' hour to second
------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0024",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 48,
+ "fragment" : "interval 1 year '12:11:10' hour to second"
+ } ]
+}
-- !query
@@ -1281,12 +1514,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Can only have a single from-to unit in the interval literal syntax(line 1, pos 37)
-
-== SQL ==
-select interval '10-9' year to month '1' year
--------------------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0024",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 45,
+ "fragment" : "interval '10-9' year to month '1' year"
+ } ]
+}
-- !query
@@ -1295,12 +1532,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Can only have a single from-to unit in the interval literal syntax(line 1, pos 42)
-
-== SQL ==
-select interval '12:11:10' hour to second '1' year
-------------------------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0024",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 50,
+ "fragment" : "interval '12:11:10' hour to second '1' year"
+ } ]
+}
-- !query
@@ -1309,7 +1550,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Undefined function: interval. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.interval.; line 1 pos 7
+{
+ "errorClass" : "UNRESOLVED_ROUTINE",
+ "sqlState" : "42883",
+ "messageParameters" : {
+ "routineName" : "`interval`",
+ "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 21,
+ "fragment" : "interval (-30)"
+ } ]
+}
-- !query
@@ -1318,7 +1573,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Undefined function: interval. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.interval.; line 1 pos 7
+{
+ "errorClass" : "UNRESOLVED_ROUTINE",
+ "sqlState" : "42883",
+ "messageParameters" : {
+ "routineName" : "`interval`",
+ "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 23,
+ "fragment" : "interval (a + 1)"
+ } ]
+}
-- !query
@@ -1327,12 +1596,14 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Syntax error at or near 'day': extra input 'day'(line 1, pos 27)
-
-== SQL ==
-select interval 30 day day day
----------------------------^^^
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'day'",
+ "hint" : ": extra input 'day'"
+ }
+}
-- !query
@@ -1341,7 +1612,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Undefined function: interval. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.interval.; line 1 pos 7
+{
+ "errorClass" : "UNRESOLVED_ROUTINE",
+ "sqlState" : "42883",
+ "messageParameters" : {
+ "routineName" : "`interval`",
+ "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 21,
+ "fragment" : "interval (-30)"
+ } ]
+}
-- !query
@@ -1350,7 +1635,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Undefined function: interval. This function is neither a built-in/temporary function, nor a persistent function that is qualified as spark_catalog.default.interval.; line 1 pos 7
+{
+ "errorClass" : "UNRESOLVED_ROUTINE",
+ "sqlState" : "42883",
+ "messageParameters" : {
+ "routineName" : "`interval`",
+ "searchPath" : "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 23,
+ "fragment" : "interval (a + 1)"
+ } ]
+}
-- !query
@@ -1359,12 +1658,14 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Syntax error at or near 'days': extra input 'days'(line 1, pos 29)
-
-== SQL ==
-select interval 30 days days days
------------------------------^^^
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'days'",
+ "hint" : ": extra input 'days'"
+ }
+}
-- !query
@@ -1381,12 +1682,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Error parsing interval year-month string: integer overflow(line 1, pos 16)
-
-== SQL ==
-SELECT INTERVAL '178956970-8' YEAR TO MONTH
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "Error parsing interval year-month string: integer overflow"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 43,
+ "fragment" : "'178956970-8' YEAR TO MONTH"
+ } ]
+}
-- !query
@@ -1431,7 +1739,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '2' YEAR + CAST('3-3 year to month' AS DOUBLE))' due to data type mismatch: differing types in '(INTERVAL '2' YEAR + CAST('3-3 year to month' AS DOUBLE))' (interval year and double).; line 2 pos 2
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL YEAR\"",
+ "right" : "\"DOUBLE\"",
+ "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3 year to month)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 10,
+ "stopIndex" : 48,
+ "fragment" : "interval '2' year + '3-3 year to month'"
+ } ]
+}
-- !query
@@ -1456,7 +1779,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '2' YEAR + CAST('3-3' AS DOUBLE))' due to data type mismatch: differing types in '(INTERVAL '2' YEAR + CAST('3-3' AS DOUBLE))' (interval year and double).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL YEAR\"",
+ "right" : "\"DOUBLE\"",
+ "sqlExpr" : "\"(INTERVAL '2' YEAR + 3-3)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 32,
+ "fragment" : "interval '2' year + '3-3'"
+ } ]
+}
-- !query
@@ -1465,7 +1803,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '2' YEAR - CAST('4' AS DOUBLE))' due to data type mismatch: differing types in '(INTERVAL '2' YEAR - CAST('4' AS DOUBLE))' (interval year and double).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL YEAR\"",
+ "right" : "\"DOUBLE\"",
+ "sqlExpr" : "\"(INTERVAL '2' YEAR - 4)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "interval '2' year - '4'"
+ } ]
+}
-- !query
@@ -1498,7 +1851,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '2' YEAR + CAST(interval_view.str AS DOUBLE))' due to data type mismatch: differing types in '(INTERVAL '2' YEAR + CAST(interval_view.str AS DOUBLE))' (interval year and double).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL YEAR\"",
+ "right" : "\"DOUBLE\"",
+ "sqlExpr" : "\"(INTERVAL '2' YEAR + str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "interval '2' year + str"
+ } ]
+}
-- !query
@@ -1507,7 +1875,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '2' YEAR - CAST(interval_view.str AS DOUBLE))' due to data type mismatch: differing types in '(INTERVAL '2' YEAR - CAST(interval_view.str AS DOUBLE))' (interval year and double).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL YEAR\"",
+ "right" : "\"DOUBLE\"",
+ "sqlExpr" : "\"(INTERVAL '2' YEAR - str)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "interval '2' year - str"
+ } ]
+}
-- !query
@@ -1532,7 +1915,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' DAY' due to data type mismatch: argument 1 requires (timestamp or timestamp without time zone) type, however, 'INTERVAL '2-2' YEAR TO MONTH' is of interval year to month type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"",
+ "inputType" : "\"INTERVAL YEAR TO MONTH\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' DAY\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 54,
+ "fragment" : "interval '2-2' year to month + interval '3' day"
+ } ]
+}
-- !query
@@ -1541,7 +1941,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' DAY' due to data type mismatch: argument 1 requires (timestamp or timestamp without time zone) type, however, 'INTERVAL '2-2' YEAR TO MONTH' is of interval year to month type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"",
+ "inputType" : "\"INTERVAL YEAR TO MONTH\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + INTERVAL '3' DAY\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 54,
+ "fragment" : "interval '3' day + interval '2-2' year to month"
+ } ]
+}
-- !query
@@ -1550,7 +1967,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'INTERVAL '2-2' YEAR TO MONTH + (- INTERVAL '3' DAY)' due to data type mismatch: argument 1 requires (timestamp or timestamp without time zone) type, however, 'INTERVAL '2-2' YEAR TO MONTH' is of interval year to month type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"INTERVAL '2-2' YEAR TO MONTH\"",
+ "inputType" : "\"INTERVAL YEAR TO MONTH\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"INTERVAL '2-2' YEAR TO MONTH + (- INTERVAL '3' DAY)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 54,
+ "fragment" : "interval '2-2' year to month - interval '3' day"
+ } ]
+}
-- !query
@@ -1559,7 +1993,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '3' DAY - INTERVAL '2-2' YEAR TO MONTH)' due to data type mismatch: differing types in '(INTERVAL '3' DAY - INTERVAL '2-2' YEAR TO MONTH)' (interval day and interval year to month).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL DAY\"",
+ "right" : "\"INTERVAL YEAR TO MONTH\"",
+ "sqlExpr" : "\"(INTERVAL '3' DAY - INTERVAL '2-2' YEAR TO MONTH)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 54,
+ "fragment" : "interval '3' day - interval '2-2' year to month"
+ } ]
+}
-- !query
@@ -1568,7 +2017,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '1 + (- INTERVAL '02' SECOND)' due to data type mismatch: argument 1 requires (timestamp or timestamp without time zone) type, however, '1' is of int type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"1 + (- INTERVAL '02' SECOND)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "1 - interval '2' second"
+ } ]
+}
-- !query
@@ -1577,7 +2043,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(1 + INTERVAL '2' MONTH)' due to data type mismatch: differing types in '(1 + INTERVAL '2' MONTH)' (int and interval month).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INT\"",
+ "right" : "\"INTERVAL MONTH\"",
+ "sqlExpr" : "\"(1 + INTERVAL '2' MONTH)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "1 + interval '2' month"
+ } ]
+}
-- !query
@@ -1586,7 +2067,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '1 + INTERVAL '02' SECOND' due to data type mismatch: argument 1 requires (timestamp or timestamp without time zone) type, however, '1' is of int type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"1\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"(TIMESTAMP OR TIMESTAMP WITHOUT TIME ZONE)\"",
+ "sqlExpr" : "\"1 + INTERVAL '02' SECOND\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "interval '2' second + 1"
+ } ]
+}
-- !query
@@ -1595,7 +2093,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '2' MONTH - 1)' due to data type mismatch: differing types in '(INTERVAL '2' MONTH - 1)' (interval month and int).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL MONTH\"",
+ "right" : "\"INT\"",
+ "sqlExpr" : "\"(INTERVAL '2' MONTH - 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "interval '2' month - 1"
+ } ]
+}
-- !query
@@ -1652,12 +2165,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Interval string does not match year-month format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval year to month: - 2-2 (line 1, pos 16)
-
-== SQL ==
-select interval '-\t2-2\t' year to month
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "Interval string does not match year-month format of `[+|-]d h`, `INTERVAL [+|-]'[+|-]d h' DAY TO HOUR` when cast to interval year to month: -\t2-2\t"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 40,
+ "fragment" : "'-\\t2-2\\t' year to month"
+ } ]
+}
-- !query
@@ -1674,13 +2194,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Interval string does not match day-time format of `[+|-]d h:m:s.n`, `INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND` when cast to interval day to second:
-- 10 12:34:46.789 , set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0.(line 1, pos 16)
-
-== SQL ==
-select interval '\n-\t10\t 12:34:46.789\t' day to second
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "Interval string does not match day-time format of `[+|-]d h:m:s.n`, `INTERVAL [+|-]'[+|-]d h:m:s.n' DAY TO SECOND` when cast to interval day to second: \n-\t10\t 12:34:46.789\t, set spark.sql.legacy.fromDayTimeString.enabled to true to restore the behavior before Spark 3.0."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 56,
+ "fragment" : "'\\n-\\t10\\t 12:34:46.789\\t' day to second"
+ } ]
+}
-- !query
@@ -1689,12 +2215,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: 中文 interval 1 day(line 1, pos 7)
-
-== SQL ==
-select interval '中文 interval 1 day'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'中文 interval 1 day'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 35,
+ "fragment" : "interval '中文 interval 1 day'"
+ } ]
+}
-- !query
@@ -1703,12 +2238,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: interval中文 1 day(line 1, pos 7)
-
-== SQL ==
-select interval 'interval中文 1 day'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'interval中文 1 day'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 34,
+ "fragment" : "interval 'interval中文 1 day'"
+ } ]
+}
-- !query
@@ -1717,12 +2261,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: interval 1中文day(line 1, pos 7)
-
-== SQL ==
-select interval 'interval 1中文day'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'interval 1中文day'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 33,
+ "fragment" : "interval 'interval 1中文day'"
+ } ]
+}
-- !query
@@ -1731,7 +2284,14 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error.
+{
+ "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW",
+ "sqlState" : "22015",
+ "messageParameters" : {
+ "alternative" : "",
+ "message" : "integer overflow"
+ }
+}
-- !query
@@ -1740,7 +2300,14 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error.
+{
+ "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW",
+ "sqlState" : "22015",
+ "messageParameters" : {
+ "alternative" : " Use 'try_subtract' to tolerate overflow and return NULL instead.",
+ "message" : "integer overflow"
+ }
+}
-- !query
@@ -1749,7 +2316,14 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-integer overflow. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error.
+{
+ "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW",
+ "sqlState" : "22015",
+ "messageParameters" : {
+ "alternative" : " Use 'try_add' to tolerate overflow and return NULL instead.",
+ "message" : "integer overflow"
+ }
+}
-- !query
@@ -1828,12 +2402,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: +(line 1, pos 7)
-
-== SQL ==
-select interval '+'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'+'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 19,
+ "fragment" : "interval '+'"
+ } ]
+}
-- !query
@@ -1842,12 +2425,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: +.(line 1, pos 7)
-
-== SQL ==
-select interval '+.'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'+.'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 20,
+ "fragment" : "interval '+.'"
+ } ]
+}
-- !query
@@ -1856,12 +2448,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: 1(line 1, pos 7)
-
-== SQL ==
-select interval '1'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'1'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 19,
+ "fragment" : "interval '1'"
+ } ]
+}
-- !query
@@ -1870,12 +2471,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: 1.2(line 1, pos 7)
-
-== SQL ==
-select interval '1.2'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'1.2'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 21,
+ "fragment" : "interval '1.2'"
+ } ]
+}
-- !query
@@ -1884,12 +2494,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: - 2(line 1, pos 7)
-
-== SQL ==
-select interval '- 2'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'- 2'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 21,
+ "fragment" : "interval '- 2'"
+ } ]
+}
-- !query
@@ -1898,12 +2517,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: 1 day -(line 1, pos 7)
-
-== SQL ==
-select interval '1 day -'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'1 day -'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 25,
+ "fragment" : "interval '1 day -'"
+ } ]
+}
-- !query
@@ -1912,12 +2540,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Cannot parse the INTERVAL value: 1 day 1(line 1, pos 7)
-
-== SQL ==
-select interval '1 day 1'
--------^^^
+{
+ "errorClass" : "INVALID_TYPED_LITERAL",
+ "sqlState" : "42604",
+ "messageParameters" : {
+ "value" : "'1 day 1'",
+ "valueType" : "\"INTERVAL\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 25,
+ "fragment" : "interval '1 day 1'"
+ } ]
+}
-- !query
@@ -1926,12 +2563,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Can only use numbers in the interval value part for multiple unit value pairs interval form, but got invalid value: 1 day 2(line 1, pos 16)
-
-== SQL ==
-select interval '1 day 2' day
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0026",
+ "messageParameters" : {
+ "value" : "1 day 2"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 29,
+ "fragment" : "'1 day 2' day"
+ } ]
+}
-- !query
@@ -1940,12 +2584,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Can only use numbers in the interval value part for multiple unit value pairs interval form, but got invalid value: interval 1(line 1, pos 16)
-
-== SQL ==
-select interval 'interval 1' day
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0026",
+ "messageParameters" : {
+ "value" : "interval 1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 32,
+ "fragment" : "'interval 1' day"
+ } ]
+}
-- !query
@@ -1978,10 +2629,21 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW",
+ "sqlState" : "22015",
+ "messageParameters" : {
+ "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.",
+ "message" : "Interval value overflows after being divided by -1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 51,
+ "fragment" : "(INTERVAL '-178956970-8' YEAR TO MONTH) / -1"
+ } ]
+}
-- !query
@@ -1990,10 +2652,21 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT (INTERVAL '-178956970-8' YEAR TO MONTH) / -1L
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW",
+ "sqlState" : "22015",
+ "messageParameters" : {
+ "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.",
+ "message" : "Interval value overflows after being divided by -1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 52,
+ "fragment" : "(INTERVAL '-178956970-8' YEAR TO MONTH) / -1L"
+ } ]
+}
-- !query
@@ -2036,10 +2709,21 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW",
+ "sqlState" : "22015",
+ "messageParameters" : {
+ "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.",
+ "message" : "Interval value overflows after being divided by -1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 65,
+ "fragment" : "(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1"
+ } ]
+}
-- !query
@@ -2048,10 +2732,21 @@
struct<>
-- !query output
org.apache.spark.SparkArithmeticException
-Overflow in integral divide. Use 'try_divide' to tolerate overflow and return NULL instead. If necessary set spark.sql.ansi.enabled to "false" (except for ANSI interval type) to bypass this error.
-== SQL(line 1, position 8) ==
-SELECT (INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1L
- ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
+{
+ "errorClass" : "INTERVAL_ARITHMETIC_OVERFLOW",
+ "sqlState" : "22015",
+ "messageParameters" : {
+ "alternative" : " Use 'try_divide' to tolerate overflow and return NULL instead.",
+ "message" : "Interval value overflows after being divided by -1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 66,
+ "fragment" : "(INTERVAL '-106751991 04:00:54.775808' DAY TO SECOND) / -1L"
+ } ]
+}
-- !query
@@ -2174,12 +2869,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-requirement failed: day 106751992 outside range [0, 106751991](line 1, pos 16)
-
-== SQL ==
-SELECT INTERVAL '106751992 04' DAY TO HOUR
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "requirement failed: day 106751992 outside range [0, 106751991]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 42,
+ "fragment" : "'106751992 04' DAY TO HOUR"
+ } ]
+}
-- !query
@@ -2188,12 +2890,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-requirement failed: day 106751992 outside range [0, 106751991](line 1, pos 16)
-
-== SQL ==
-SELECT INTERVAL '-106751992 04' DAY TO HOUR
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "requirement failed: day 106751992 outside range [0, 106751991]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 43,
+ "fragment" : "'-106751992 04' DAY TO HOUR"
+ } ]
+}
-- !query
@@ -2202,12 +2911,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-requirement failed: hour 2562047789 outside range [0, 2562047788](line 1, pos 16)
-
-== SQL ==
-SELECT INTERVAL '2562047789:00' HOUR TO MINUTE
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "requirement failed: hour 2562047789 outside range [0, 2562047788]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 46,
+ "fragment" : "'2562047789:00' HOUR TO MINUTE"
+ } ]
+}
-- !query
@@ -2216,12 +2932,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-requirement failed: hour 2562047789 outside range [0, 2562047788](line 1, pos 16)
-
-== SQL ==
-SELECT INTERVAL '-2562047789:00' HOUR TO MINUTE
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "requirement failed: hour 2562047789 outside range [0, 2562047788]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 47,
+ "fragment" : "'-2562047789:00' HOUR TO MINUTE"
+ } ]
+}
-- !query
@@ -2230,12 +2953,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-requirement failed: minute 153722867281 outside range [0, 153722867280](line 1, pos 16)
-
-== SQL ==
-SELECT INTERVAL '153722867281:54.775808' MINUTE TO SECOND
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "requirement failed: minute 153722867281 outside range [0, 153722867280]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 57,
+ "fragment" : "'153722867281:54.775808' MINUTE TO SECOND"
+ } ]
+}
-- !query
@@ -2244,12 +2974,19 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-requirement failed: minute 153722867281 outside range [0, 153722867280](line 1, pos 16)
-
-== SQL ==
-SELECT INTERVAL '-153722867281:54.775808' MINUTE TO SECOND
-----------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0063",
+ "messageParameters" : {
+ "msg" : "requirement failed: minute 153722867281 outside range [0, 153722867280]"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 17,
+ "stopIndex" : 58,
+ "fragment" : "'-153722867281:54.775808' MINUTE TO SECOND"
+ } ]
+}
-- !query
@@ -2386,7 +3123,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '1' MONTH > INTERVAL '20' DAY)' due to data type mismatch: differing types in '(INTERVAL '1' MONTH > INTERVAL '20' DAY)' (interval month and interval day).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL MONTH\"",
+ "right" : "\"INTERVAL DAY\"",
+ "sqlExpr" : "\"(INTERVAL '1' MONTH > INTERVAL '20' DAY)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 42,
+ "fragment" : "INTERVAL 1 MONTH > INTERVAL 20 DAYS"
+ } ]
+}
-- !query
@@ -2395,7 +3147,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '1' DAY < '1')' due to data type mismatch: differing types in '(INTERVAL '1' DAY < '1')' (interval day and string).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL DAY\"",
+ "right" : "\"STRING\"",
+ "sqlExpr" : "\"(INTERVAL '1' DAY < 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "INTERVAL '1' DAY < '1'"
+ } ]
+}
-- !query
@@ -2404,7 +3171,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '1' DAY = '1')' due to data type mismatch: differing types in '(INTERVAL '1' DAY = '1')' (interval day and string).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL DAY\"",
+ "right" : "\"STRING\"",
+ "sqlExpr" : "\"(INTERVAL '1' DAY = 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "INTERVAL '1' DAY = '1'"
+ } ]
+}
-- !query
@@ -2413,7 +3195,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '1' DAY > '1')' due to data type mismatch: differing types in '(INTERVAL '1' DAY > '1')' (interval day and string).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL DAY\"",
+ "right" : "\"STRING\"",
+ "sqlExpr" : "\"(INTERVAL '1' DAY > 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "INTERVAL '1' DAY > '1'"
+ } ]
+}
-- !query
@@ -2422,7 +3219,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '('1' < INTERVAL '1' DAY)' due to data type mismatch: differing types in '('1' < INTERVAL '1' DAY)' (string and interval day).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"STRING\"",
+ "right" : "\"INTERVAL DAY\"",
+ "sqlExpr" : "\"(1 < INTERVAL '1' DAY)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "'1' < INTERVAL '1' DAY"
+ } ]
+}
-- !query
@@ -2431,7 +3243,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '('1' = INTERVAL '1' DAY)' due to data type mismatch: differing types in '('1' = INTERVAL '1' DAY)' (string and interval day).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"STRING\"",
+ "right" : "\"INTERVAL DAY\"",
+ "sqlExpr" : "\"(1 = INTERVAL '1' DAY)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "'1' = INTERVAL '1' DAY"
+ } ]
+}
-- !query
@@ -2440,7 +3267,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '('1' > INTERVAL '1' DAY)' due to data type mismatch: differing types in '('1' > INTERVAL '1' DAY)' (string and interval day).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"STRING\"",
+ "right" : "\"INTERVAL DAY\"",
+ "sqlExpr" : "\"(1 > INTERVAL '1' DAY)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 29,
+ "fragment" : "'1' > INTERVAL '1' DAY"
+ } ]
+}
-- !query
@@ -2449,7 +3291,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '1' YEAR < '1')' due to data type mismatch: differing types in '(INTERVAL '1' YEAR < '1')' (interval year and string).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL YEAR\"",
+ "right" : "\"STRING\"",
+ "sqlExpr" : "\"(INTERVAL '1' YEAR < 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "INTERVAL '1' YEAR < '1'"
+ } ]
+}
-- !query
@@ -2458,7 +3315,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '1' YEAR = '1')' due to data type mismatch: differing types in '(INTERVAL '1' YEAR = '1')' (interval year and string).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL YEAR\"",
+ "right" : "\"STRING\"",
+ "sqlExpr" : "\"(INTERVAL '1' YEAR = 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "INTERVAL '1' YEAR = '1'"
+ } ]
+}
-- !query
@@ -2467,7 +3339,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '1' YEAR > '1')' due to data type mismatch: differing types in '(INTERVAL '1' YEAR > '1')' (interval year and string).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL YEAR\"",
+ "right" : "\"STRING\"",
+ "sqlExpr" : "\"(INTERVAL '1' YEAR > 1)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "INTERVAL '1' YEAR > '1'"
+ } ]
+}
-- !query
@@ -2476,7 +3363,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '('1' < INTERVAL '1' YEAR)' due to data type mismatch: differing types in '('1' < INTERVAL '1' YEAR)' (string and interval year).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"STRING\"",
+ "right" : "\"INTERVAL YEAR\"",
+ "sqlExpr" : "\"(1 < INTERVAL '1' YEAR)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "'1' < INTERVAL '1' YEAR"
+ } ]
+}
-- !query
@@ -2485,7 +3387,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '('1' = INTERVAL '1' YEAR)' due to data type mismatch: differing types in '('1' = INTERVAL '1' YEAR)' (string and interval year).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"STRING\"",
+ "right" : "\"INTERVAL YEAR\"",
+ "sqlExpr" : "\"(1 = INTERVAL '1' YEAR)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "'1' = INTERVAL '1' YEAR"
+ } ]
+}
-- !query
@@ -2494,7 +3411,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '('1' > INTERVAL '1' YEAR)' due to data type mismatch: differing types in '('1' > INTERVAL '1' YEAR)' (string and interval year).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"STRING\"",
+ "right" : "\"INTERVAL YEAR\"",
+ "sqlExpr" : "\"(1 > INTERVAL '1' YEAR)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "'1' > INTERVAL '1' YEAR"
+ } ]
+}
-- !query
@@ -2519,7 +3451,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'array(INTERVAL '1' MONTH, INTERVAL '20' DAY)' due to data type mismatch: input to function array should all be the same type, but it's [interval month, interval day]; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "dataType" : "(\"INTERVAL MONTH\" or \"INTERVAL DAY\")",
+ "functionName" : "`array`",
+ "sqlExpr" : "\"array(INTERVAL '1' MONTH, INTERVAL '20' DAY)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 48,
+ "fragment" : "array(INTERVAL 1 MONTH, INTERVAL 20 DAYS)"
+ } ]
+}
-- !query
@@ -2544,7 +3491,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'coalesce(INTERVAL '1' MONTH, INTERVAL '20' DAY)' due to data type mismatch: input to function coalesce should all be the same type, but it's [interval month, interval day]; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.DATA_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "dataType" : "(\"INTERVAL MONTH\" or \"INTERVAL DAY\")",
+ "functionName" : "`coalesce`",
+ "sqlExpr" : "\"coalesce(INTERVAL '1' MONTH, INTERVAL '20' DAY)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 51,
+ "fragment" : "coalesce(INTERVAL 1 MONTH, INTERVAL 20 DAYS)"
+ } ]
+}
-- !query
@@ -2601,7 +3563,22 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve '(INTERVAL '1' MONTH div INTERVAL '-1' DAY)' due to data type mismatch: differing types in '(INTERVAL '1' MONTH div INTERVAL '-1' DAY)' (interval month and interval day).; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.BINARY_OP_DIFF_TYPES",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "left" : "\"INTERVAL MONTH\"",
+ "right" : "\"INTERVAL DAY\"",
+ "sqlExpr" : "\"(INTERVAL '1' MONTH div INTERVAL '-1' DAY)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 49,
+ "fragment" : "div(INTERVAL '1' MONTH, INTERVAL '-1' DAY)"
+ } ]
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/join-empty-relation.sql.out sql/core/src/test/resources/sql-tests/results/join-empty-relation.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/join-empty-relation.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/join-empty-relation.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 24
-
-
-- !query
CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a)
-- !query schema
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 71
-
-
-- !query
CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)
-- !query schema
@@ -19,6 +16,22 @@
-- !query
+CREATE VIEW t3(c1, c2) AS VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE VIEW t4(c1, c2) AS VALUES (0, 1), (0, 2), (1, 1), (1, 3)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
SELECT * FROM t1, LATERAL (SELECT c1)
-- !query schema
struct<c1:int,c2:int,c1:int>
@@ -117,7 +130,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 't1.*' given input columns 'c1, c2'; line 1 pos 70
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_1051",
+ "messageParameters" : {
+ "columns" : "c1, c2",
+ "targetString" : "t1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 71,
+ "stopIndex" : 74,
+ "fragment" : "t1.*"
+ } ]
+}
-- !query
@@ -152,12 +178,21 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-The feature is not supported: LATERAL join with NATURAL join.(line 1, pos 14)
-
-== SQL ==
-SELECT * FROM t1 NATURAL JOIN LATERAL (SELECT c1 + c2 AS c2)
---------------^^^
+{
+ "errorClass" : "INCOMPATIBLE_JOIN_TYPES",
+ "sqlState" : "42613",
+ "messageParameters" : {
+ "joinType1" : "LATERAL",
+ "joinType2" : "NATURAL"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 18,
+ "stopIndex" : 60,
+ "fragment" : "NATURAL JOIN LATERAL (SELECT c1 + c2 AS c2)"
+ } ]
+}
-- !query
@@ -166,12 +201,17 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-The feature is not supported: LATERAL join with USING join.(line 1, pos 14)
-
-== SQL ==
-SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c2) USING (c2)
---------------^^^
+{
+ "errorClass" : "UNSUPPORTED_FEATURE.LATERAL_JOIN_USING",
+ "sqlState" : "0A000",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 18,
+ "stopIndex" : 63,
+ "fragment" : "JOIN LATERAL (SELECT c1 + c2 AS c2) USING (c2)"
+ } ]
+}
-- !query
@@ -253,6 +293,15 @@
-- !query
+SELECT * FROM t1, LATERAL (SELECT max(c2) AS m FROM t2 WHERE t1.c2 < t2.c2)
+-- !query schema
+struct<c1:int,c2:int,m:int>
+-- !query output
+0 1 3
+1 2 3
+
+
+-- !query
SELECT * FROM t1 JOIN t2 JOIN LATERAL (SELECT t1.c2 + t2.c2)
-- !query schema
struct<c1:int,c2:int,c1:int,c2:int,(outer(spark_catalog.default.t1.c2) + outer(spark_catalog.default.t2.c2)):int>
@@ -269,7 +318,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Column 't2.c1' does not exist. Did you mean one of the following? []; line 1 pos 50
+{
+ "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION",
+ "sqlState" : "42703",
+ "messageParameters" : {
+ "objectName" : "`t2`.`c1`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 51,
+ "stopIndex" : 55,
+ "fragment" : "t2.c1"
+ } ]
+}
-- !query
@@ -294,11 +356,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Non-deterministic lateral subqueries are not supported when joining with outer relations that produce more than one row
-SubqueryAlias __auto_generated_subquery_name
-+- Project [(cast((outer(c1#x) + outer(c2#x)) as double) + rand(0)) AS c3#x]
- +- OneRowRelation
-; line 1 pos 9
+{
+ "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.NON_DETERMINISTIC_LATERAL_SUBQUERIES",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "treeNode" : "LateralJoin lateral-subquery#x [c1#x && c2#x], Inner\n: +- SubqueryAlias __auto_generated_subquery_name\n: +- Project [(cast((outer(c1#x) + outer(c2#x)) as double) + rand(0)) AS c3#x]\n: +- OneRowRelation\n+- SubqueryAlias spark_catalog.default.t1\n +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x])\n +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n +- LocalRelation [col1#x, col2#x]\n"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 10,
+ "stopIndex" : 58,
+ "fragment" : "FROM t1, LATERAL (SELECT c1 + c2 + rand(0) AS c3)"
+ } ]
+}
-- !query
@@ -307,14 +378,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Non-deterministic lateral subqueries are not supported when joining with outer relations that produce more than one row
-SubqueryAlias __auto_generated_subquery_name
-+- Project [rand(0) AS rand(0)#x]
- +- SubqueryAlias spark_catalog.default.t2
- +- View (`default`.`t2`, [c1#x,c2#x])
- +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]
- +- LocalRelation [col1#x, col2#x]
-; line 1 pos 9
+{
+ "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.NON_DETERMINISTIC_LATERAL_SUBQUERIES",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "treeNode" : "LateralJoin lateral-subquery#x [], Inner\n: +- SubqueryAlias __auto_generated_subquery_name\n: +- Project [rand(0) AS rand(0)#x]\n: +- SubqueryAlias spark_catalog.default.t2\n: +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x])\n: +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n: +- LocalRelation [col1#x, col2#x]\n+- SubqueryAlias spark_catalog.default.t1\n +- View (`spark_catalog`.`default`.`t1`, [c1#x,c2#x])\n +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n +- LocalRelation [col1#x, col2#x]\n"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 10,
+ "stopIndex" : 50,
+ "fragment" : "FROM t1, LATERAL (SELECT rand(0) FROM t2)"
+ } ]
+}
-- !query
@@ -323,7 +400,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Lateral join condition cannot be non-deterministic: ((CAST(spark_catalog.default.t1.c1 AS DOUBLE) + rand(0)) = CAST(s.c1 AS DOUBLE)); line 1 pos 17
+{
+ "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.LATERAL_JOIN_CONDITION_NON_DETERMINISTIC",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "condition" : "((CAST(spark_catalog.default.t1.c1 AS DOUBLE) + rand(0)) = CAST(s.c1 AS DOUBLE))"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 55,
+ "stopIndex" : 76,
+ "fragment" : "t1.c1 + rand(0) = s.c1"
+ } ]
+}
-- !query
@@ -387,7 +477,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Column 't1.c1' does not exist. Did you mean one of the following? []; line 1 pos 61
+{
+ "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION",
+ "sqlState" : "42703",
+ "messageParameters" : {
+ "objectName" : "`t1`.`c1`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 62,
+ "stopIndex" : 66,
+ "fragment" : "t1.c1"
+ } ]
+}
-- !query
@@ -396,7 +499,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Column 'c2' does not exist. Did you mean one of the following? []; line 1 pos 70
+{
+ "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION",
+ "sqlState" : "42703",
+ "messageParameters" : {
+ "objectName" : "`c2`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 71,
+ "stopIndex" : 72,
+ "fragment" : "c2"
+ } ]
+}
-- !query
@@ -423,7 +539,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Column 't1.c1' does not exist. Did you mean one of the following? [spark_catalog.default.t2.c1, spark_catalog.default.t2.c2]; line 1 pos 73
+{
+ "errorClass" : "UNRESOLVED_COLUMN.WITH_SUGGESTION",
+ "sqlState" : "42703",
+ "messageParameters" : {
+ "objectName" : "`t1`.`c1`",
+ "proposal" : "`spark_catalog`.`default`.`t2`.`c1`, `spark_catalog`.`default`.`t2`.`c2`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 74,
+ "stopIndex" : 78,
+ "fragment" : "t1.c1"
+ } ]
+}
-- !query
@@ -681,6 +811,734 @@
-- !query
+SELECT * FROM t3 JOIN LATERAL (SELECT EXPLODE(c2))
+-- !query schema
+struct<c1:int,c2:array<int>,col:int>
+-- !query output
+0 [0,1] 0
+0 [0,1] 1
+1 [2] 2
+NULL [4] 4
+
+
+-- !query
+SELECT * FROM t3 JOIN LATERAL (SELECT EXPLODE_OUTER(c2))
+-- !query schema
+struct<c1:int,c2:array<int>,col:int>
+-- !query output
+0 [0,1] 0
+0 [0,1] 1
+1 [2] 2
+2 [] NULL
+NULL [4] 4
+
+
+-- !query
+SELECT * FROM t3 JOIN LATERAL (SELECT EXPLODE(c2)) t(c3) ON c1 = c3
+-- !query schema
+struct<c1:int,c2:array<int>,c3:int>
+-- !query output
+0 [0,1] 0
+
+
+-- !query
+SELECT * FROM t3 LEFT JOIN LATERAL (SELECT EXPLODE(c2)) t(c3) ON c1 = c3
+-- !query schema
+struct<c1:int,c2:array<int>,c3:int>
+-- !query output
+0 [0,1] 0
+1 [2] NULL
+2 [] NULL
+NULL [4] NULL
+
+
+-- !query
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT sum(t2.c2) over (order by t2.c1)
+ FROM t2
+ WHERE t2.c1 >= t1.c1)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "treeNode" : "Filter (c1#x >= outer(c1#x))\n+- SubqueryAlias spark_catalog.default.t2\n +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x])\n +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n +- LocalRelation [col1#x, col2#x]\n"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 34,
+ "stopIndex" : 108,
+ "fragment" : "SELECT sum(t2.c2) over (order by t2.c1)\n FROM t2\n WHERE t2.c1 >= t1.c1"
+ } ]
+}
+
+
+-- !query
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT t2.c2
+ FROM t2
+ WHERE t2.c1 = t1.c1
+ UNION ALL
+ SELECT t4.c2
+ FROM t4
+ WHERE t4.c1 = t1.c1)
+-- !query schema
+struct<c1:int,c2:int,c2:int>
+-- !query output
+0 1 1
+0 1 2
+0 1 2
+0 1 3
+1 2 1
+1 2 3
+
+
+-- !query
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT t2.c2
+ FROM t2
+ WHERE t2.c1 = t1.c1
+ UNION DISTINCT
+ SELECT t4.c2
+ FROM t4
+ WHERE t4.c1 > t1.c2)
+-- !query schema
+struct<c1:int,c2:int,c2:int>
+-- !query output
+0 1 2
+0 1 3
+
+
+-- !query
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT COUNT(t2.c2)
+ FROM t2
+ WHERE t2.c1 = t1.c1
+ UNION DISTINCT
+ SELECT COUNT(t4.c2)
+ FROM t4
+ WHERE t4.c1 > t1.c2)
+-- !query schema
+struct<c1:int,c2:int,count(c2):bigint>
+-- !query output
+0 1 0
+0 1 2
+1 2 0
+
+
+-- !query
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT t2.c1, t2.c2
+ FROM t2
+ WHERE t2.c1 = t1.c1
+ UNION ALL
+ SELECT t4.c2, t4.c1
+ FROM t4
+ WHERE t4.c1 = t1.c1)
+-- !query schema
+struct<c1:int,c2:int,c1:int,c2:int>
+-- !query output
+0 1 0 2
+0 1 0 3
+0 1 1 0
+0 1 2 0
+1 2 1 1
+1 2 3 1
+
+
+-- !query
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT t2.c2
+ FROM t2
+ WHERE t2.c1 = t1.c1 and t2.c2 >= t1.c2
+ UNION ALL
+ SELECT t4.c2
+ FROM t4)
+-- !query schema
+struct<c1:int,c2:int,c2:int>
+-- !query output
+0 1 1
+0 1 1
+0 1 2
+0 1 2
+0 1 3
+0 1 3
+1 2 1
+1 2 1
+1 2 2
+1 2 3
+
+
+-- !query
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT t2.c2
+ FROM t2
+ WHERE t2.c1 = t1.c1
+ UNION ALL
+ SELECT t4.c2
+ FROM t4)
+-- !query schema
+struct<c1:int,c2:int,c2:int>
+-- !query output
+0 1 1
+0 1 1
+0 1 2
+0 1 2
+0 1 3
+0 1 3
+1 2 1
+1 2 1
+1 2 2
+1 2 3
+
+
+-- !query
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT t2.c2
+ FROM t2
+ WHERE t2.c1 = t1.c1
+ GROUP BY t2.c2
+ UNION ALL
+ SELECT t4.c2
+ FROM t4
+ WHERE t4.c1 > t1.c2
+ GROUP BY t4.c2)
+-- !query schema
+struct<c1:int,c2:int,c2:int>
+-- !query output
+0 1 2
+0 1 3
+
+
+-- !query
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT t2.c1 - t1.c1
+ FROM t2
+ GROUP BY t2.c1 - t1.c1
+ UNION ALL
+ SELECT t4.c2
+ FROM t4
+ WHERE t4.c1 > t1.c2
+ GROUP BY t4.c2)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.CORRELATED_REFERENCE",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "sqlExprs" : "\"(c1 - c1)\",\"(c1 - c1) AS `(c1 - outer(spark_catalog.default.t1.c1))`\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 69,
+ "stopIndex" : 90,
+ "fragment" : "GROUP BY t2.c1 - t1.c1"
+ } ]
+}
+
+
+-- !query
+SELECT * FROM t1 JOIN LATERAL
+ (SELECT sum(t2.c2) over (order by t2.c1)
+ FROM t2
+ WHERE t2.c1 >= t1.c1
+ UNION ALL
+ SELECT t4.c2
+ FROM t4)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY.ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED",
+ "sqlState" : "0A000",
+ "messageParameters" : {
+ "treeNode" : "Filter (c1#x >= outer(c1#x))\n+- SubqueryAlias spark_catalog.default.t2\n +- View (`spark_catalog`.`default`.`t2`, [c1#x,c2#x])\n +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x]\n +- LocalRelation [col1#x, col2#x]\n"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 34,
+ "stopIndex" : 108,
+ "fragment" : "SELECT sum(t2.c2) over (order by t2.c1)\n FROM t2\n WHERE t2.c1 >= t1.c1"
+ } ]
+}
+
+
+-- !query
+(SELECT * FROM t1 JOIN LATERAL (SELECT * FROM t2 WHERE t2.c1 = t1.c1))
+UNION ALL
+(SELECT * FROM t1 JOIN t4)
+-- !query schema
+struct<c1:int,c2:int,c1:int,c2:int>
+-- !query output
+0 1 0 1
+0 1 0 2
+0 1 0 2
+0 1 0 3
+0 1 1 1
+0 1 1 3
+1 2 0 1
+1 2 0 2
+1 2 1 1
+1 2 1 3
+
+
+-- !query
+(SELECT * FROM t1 JOIN LATERAL
+ (SELECT t2.c2
+ FROM t2
+ WHERE t2.c1 = t1.c1
+ UNION ALL
+ SELECT t4.c2
+ FROM t4
+ WHERE t4.c1 = t1.c1))
+UNION ALL
+(SELECT * FROM t2 JOIN LATERAL
+ (SELECT t1.c2
+ FROM t1
+ WHERE t2.c1 <= t1.c1
+ UNION ALL
+ SELECT t4.c2
+ FROM t4
+ WHERE t4.c1 < t2.c1))
+-- !query schema
+struct<c1:int,c2:int,c2:int>
+-- !query output
+0 1 1
+0 1 2
+0 1 2
+0 1 3
+0 2 1
+0 2 2
+0 3 1
+0 3 2
+1 2 1
+1 2 3
+
+
+-- !query
+SELECT * FROM LATERAL EXPLODE(ARRAY(1, 2))
+-- !query schema
+struct<col:int>
+-- !query output
+1
+2
+
+
+-- !query
+SELECT * FROM t1, LATERAL RANGE(3)
+-- !query schema
+struct<c1:int,c2:int,id:bigint>
+-- !query output
+0 1 0
+0 1 1
+0 1 2
+1 2 0
+1 2 1
+1 2 2
+
+
+-- !query
+SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3)
+-- !query schema
+struct<c1:int,c2:int,c3:int>
+-- !query output
+0 1 0
+0 1 1
+1 2 1
+1 2 2
+
+
+-- !query
+SELECT * FROM t3, LATERAL EXPLODE(c2) t2(v)
+-- !query schema
+struct<c1:int,c2:array<int>,v:int>
+-- !query output
+0 [0,1] 0
+0 [0,1] 1
+1 [2] 2
+NULL [4] 4
+
+
+-- !query
+SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v)
+-- !query schema
+struct<c1:int,c2:array<int>,v:int>
+-- !query output
+0 [0,1] 0
+0 [0,1] 1
+1 [2] 2
+2 [] NULL
+NULL [4] 4
+
+
+-- !query
+SELECT * FROM EXPLODE(ARRAY(1, 2)) t(v), LATERAL (SELECT v + 1)
+-- !query schema
+struct<v:int,(outer(t.v) + 1):int>
+-- !query output
+1 2
+2 3
+
+
+-- !query
+SELECT * FROM t1 JOIN LATERAL EXPLODE(ARRAY(c1, c2)) t(c3) ON t1.c1 = c3
+-- !query schema
+struct<c1:int,c2:int,c3:int>
+-- !query output
+0 1 0
+1 2 1
+
+
+-- !query
+SELECT * FROM t3 JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3
+-- !query schema
+struct<c1:int,c2:array<int>,c3:int>
+-- !query output
+0 [0,1] 0
+
+
+-- !query
+SELECT * FROM t3 LEFT JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3
+-- !query schema
+struct<c1:int,c2:array<int>,c3:int>
+-- !query output
+0 [0,1] 0
+1 [2] NULL
+2 [] NULL
+NULL [4] NULL
+
+
+-- !query
+SELECT * FROM t1, LATERAL (SELECT * FROM EXPLODE(ARRAY(c1, c2)))
+-- !query schema
+struct<c1:int,c2:int,col:int>
+-- !query output
+0 1 0
+0 1 1
+1 2 1
+1 2 2
+
+
+-- !query
+SELECT * FROM t1, LATERAL (SELECT t1.c1 + c3 FROM EXPLODE(ARRAY(c1, c2)) t(c3))
+-- !query schema
+struct<c1:int,c2:int,(outer(spark_catalog.default.t1.c1) + c3):int>
+-- !query output
+0 1 0
+0 1 1
+1 2 2
+1 2 3
+
+
+-- !query
+SELECT * FROM t1, LATERAL (SELECT t1.c1 + c3 FROM EXPLODE(ARRAY(c1, c2)) t(c3) WHERE t1.c2 > 1)
+-- !query schema
+struct<c1:int,c2:int,(outer(spark_catalog.default.t1.c1) + c3):int>
+-- !query output
+1 2 2
+1 2 3
+
+
+-- !query
+SELECT * FROM t1, LATERAL (SELECT * FROM EXPLODE(ARRAY(c1, c2)) l(x) JOIN EXPLODE(ARRAY(c2, c1)) r(y) ON x = y)
+-- !query schema
+struct<c1:int,c2:int,x:int,y:int>
+-- !query output
+0 1 0 0
+0 1 1 1
+1 2 1 1
+1 2 2 2
+
+
+-- !query
+CREATE OR REPLACE TEMPORARY VIEW array_struct(id, arr) AS VALUES
+ (1, ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))),
+ (2, ARRAY()),
+ (3, ARRAY(STRUCT(3, 'c')))
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM t1, LATERAL INLINE(ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b')))
+-- !query schema
+struct<c1:int,c2:int,col1:int,col2:string>
+-- !query output
+0 1 1 a
+0 1 2 b
+1 2 1 a
+1 2 2 b
+
+
+-- !query
+SELECT c1, t.* FROM t1, LATERAL INLINE(ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))) t(x, y)
+-- !query schema
+struct<c1:int,x:int,y:string>
+-- !query output
+0 1 a
+0 2 b
+1 1 a
+1 2 b
+
+
+-- !query
+SELECT * FROM array_struct JOIN LATERAL INLINE(arr)
+-- !query schema
+struct<id:int,arr:array<struct<col1:int,col2:string>>,col1:int,col2:string>
+-- !query output
+1 [{"col1":1,"col2":"a"},{"col1":2,"col2":"b"}] 1 a
+1 [{"col1":1,"col2":"a"},{"col1":2,"col2":"b"}] 2 b
+3 [{"col1":3,"col2":"c"}] 3 c
+
+
+-- !query
+SELECT * FROM array_struct LEFT JOIN LATERAL INLINE(arr) t(k, v) ON id = k
+-- !query schema
+struct<id:int,arr:array<struct<col1:int,col2:string>>,k:int,v:string>
+-- !query output
+1 [{"col1":1,"col2":"a"},{"col1":2,"col2":"b"}] 1 a
+2 [] NULL NULL
+3 [{"col1":3,"col2":"c"}] 3 c
+
+
+-- !query
+SELECT * FROM array_struct JOIN LATERAL INLINE_OUTER(arr)
+-- !query schema
+struct<id:int,arr:array<struct<col1:int,col2:string>>,col1:int,col2:string>
+-- !query output
+1 [{"col1":1,"col2":"a"},{"col1":2,"col2":"b"}] 1 a
+1 [{"col1":1,"col2":"a"},{"col1":2,"col2":"b"}] 2 b
+2 [] NULL NULL
+3 [{"col1":3,"col2":"c"}] 3 c
+
+
+-- !query
+DROP VIEW array_struct
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * FROM LATERAL posexplode(ARRAY(1, 2))
+-- !query schema
+struct<pos:int,col:int>
+-- !query output
+0 1
+1 2
+
+
+-- !query
+SELECT * FROM t1, LATERAL posexplode(ARRAY(c1, c2)) t2(pos, c3)
+-- !query schema
+struct<c1:int,c2:int,pos:int,c3:int>
+-- !query output
+0 1 0 0
+0 1 1 1
+1 2 0 1
+1 2 1 2
+
+
+-- !query
+SELECT * FROM t1 JOIN LATERAL posexplode(ARRAY(c1, c2)) t(pos, c3) ON t1.c1 = c3
+-- !query schema
+struct<c1:int,c2:int,pos:int,c3:int>
+-- !query output
+0 1 0 0
+1 2 0 1
+
+
+-- !query
+SELECT * FROM t3, LATERAL posexplode(c2) t2(pos, v)
+-- !query schema
+struct<c1:int,c2:array<int>,pos:int,v:int>
+-- !query output
+0 [0,1] 0 0
+0 [0,1] 1 1
+1 [2] 0 2
+NULL [4] 0 4
+
+
+-- !query
+SELECT * FROM t3 JOIN LATERAL posexplode(c2) t(pos, c3) ON t3.c1 = c3
+-- !query schema
+struct<c1:int,c2:array<int>,pos:int,c3:int>
+-- !query output
+0 [0,1] 0 0
+
+
+-- !query
+SELECT * FROM t3, LATERAL posexplode_outer(c2) t2(pos, v)
+-- !query schema
+struct<c1:int,c2:array<int>,pos:int,v:int>
+-- !query output
+0 [0,1] 0 0
+0 [0,1] 1 1
+1 [2] 0 2
+2 [] NULL NULL
+NULL [4] 0 4
+
+
+-- !query
+SELECT * FROM t3 LEFT JOIN LATERAL posexplode(c2) t(pos, c3) ON t3.c1 = c3
+-- !query schema
+struct<c1:int,c2:array<int>,pos:int,c3:int>
+-- !query output
+0 [0,1] 0 0
+1 [2] NULL NULL
+2 [] NULL NULL
+NULL [4] NULL NULL
+
+
+-- !query
+SELECT * FROM t3 LEFT JOIN LATERAL posexplode_outer(c2) t(pos, c3) ON t3.c1 = c3
+-- !query schema
+struct<c1:int,c2:array<int>,pos:int,c3:int>
+-- !query output
+0 [0,1] 0 0
+1 [2] NULL NULL
+2 [] NULL NULL
+NULL [4] NULL NULL
+
+
+-- !query
+CREATE OR REPLACE TEMP VIEW json_table(key, jstring) AS VALUES
+ ('1', '{"f1": "1", "f2": "2", "f3": 3, "f5": 5.23}'),
+ ('2', '{"f1": "1", "f3": "3", "f2": 2, "f4": 4.01}'),
+ ('3', '{"f1": 3, "f4": "4", "f3": "3", "f2": 2, "f5": 5.01}'),
+ ('4', cast(null as string)),
+ ('5', '{"f1": null, "f5": ""}'),
+ ('6', '[invalid JSON string]')
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT t1.key, t2.* FROM json_table t1, LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2
+-- !query schema
+struct<key:string,c0:string,c1:string,c2:string,c3:string,c4:string>
+-- !query output
+1 1 2 3 NULL 5.23
+2 1 2 3 4.01 NULL
+3 3 2 3 4 5.01
+4 NULL NULL NULL NULL NULL
+5 NULL NULL NULL NULL
+6 NULL NULL NULL NULL NULL
+
+
+-- !query
+SELECT t1.key, t2.* FROM json_table t1, LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2 WHERE t2.c0 IS NOT NULL
+-- !query schema
+struct<key:string,c0:string,c1:string,c2:string,c3:string,c4:string>
+-- !query output
+1 1 2 3 NULL 5.23
+2 1 2 3 4.01 NULL
+3 3 2 3 4 5.01
+
+
+-- !query
+SELECT t1.key, t2.* FROM json_table t1
+ JOIN LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2(f1, f2, f3, f4, f5)
+ ON t1.key = t2.f1
+-- !query schema
+struct<key:string,f1:string,f2:string,f3:string,f4:string,f5:string>
+-- !query output
+1 1 2 3 NULL 5.23
+3 3 2 3 4 5.01
+
+
+-- !query
+SELECT t1.key, t2.* FROM json_table t1
+ LEFT JOIN LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2(f1, f2, f3, f4, f5)
+ ON t1.key = t2.f1
+-- !query schema
+struct<key:string,f1:string,f2:string,f3:string,f4:string,f5:string>
+-- !query output
+1 1 2 3 NULL 5.23
+2 NULL NULL NULL NULL NULL
+3 3 2 3 4 5.01
+4 NULL NULL NULL NULL NULL
+5 NULL NULL NULL NULL NULL
+6 NULL NULL NULL NULL NULL
+
+
+-- !query
+DROP VIEW json_table
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT t.* FROM t1, LATERAL stack(2, 'Key', c1, 'Value', c2) t
+-- !query schema
+struct<col0:string,col1:int>
+-- !query output
+Key 0
+Key 1
+Value 1
+Value 2
+
+
+-- !query
+SELECT t.* FROM t1 JOIN LATERAL stack(1, c1, c2) t(x, y)
+-- !query schema
+struct<x:int,y:int>
+-- !query output
+0 1
+1 2
+
+
+-- !query
+SELECT t.* FROM t1 JOIN t3 ON t1.c1 = t3.c1 JOIN LATERAL stack(1, t1.c2, t3.c2) t
+-- !query schema
+struct<col0:int,col1:array<int>>
+-- !query output
+1 [0,1]
+2 [2]
+
+
+-- !query
+SELECT t.* FROM t1, LATERAL stack(c1, c2)
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputExpr" : "\"outer(spark_catalog.default.t1.c1)\"",
+ "inputName" : "n",
+ "inputType" : "\"INT\"",
+ "sqlExpr" : "\"stack(outer(spark_catalog.default.t1.c1), outer(spark_catalog.default.t1.c2))\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 29,
+ "stopIndex" : 41,
+ "fragment" : "stack(c1, c2)"
+ } ]
+}
+
+
+-- !query
DROP VIEW t1
-- !query schema
struct<>
@@ -693,4 +1551,20 @@
-- !query schema
struct<>
-- !query output
+
+
+
+-- !query
+DROP VIEW t3
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DROP VIEW t4
+-- !query schema
+struct<>
+-- !query output
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out sql/core/src/test/resources/sql-tests/results/json-functions.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/json-functions.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 73
-
-
-- !query
select to_json(named_struct('a', 1, 'b', 2))
-- !query schema
@@ -72,7 +69,17 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Must use a map() function for options; line 1 pos 7
+{
+ "errorClass" : "INVALID_OPTIONS.NON_MAP_FUNCTION",
+ "sqlState" : "42K06",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 80,
+ "fragment" : "to_json(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE'))"
+ } ]
+}
-- !query
@@ -81,7 +88,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-A type of keys and values in map() must be string, but got map<string,int>; line 1 pos 7
+{
+ "errorClass" : "INVALID_OPTIONS.NON_STRING_TYPE",
+ "sqlState" : "42K06",
+ "messageParameters" : {
+ "mapType" : "\"MAP<STRING, INT>\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 60,
+ "fragment" : "to_json(named_struct('a', 1, 'b', 2), map('mode', 1))"
+ } ]
+}
-- !query
@@ -90,7 +110,23 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function to_json. Expected: one of 1 and 2; Found: 0; line 1 pos 7
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "0",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "[1, 2]",
+ "functionName" : "`to_json`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 16,
+ "fragment" : "to_json()"
+ } ]
+}
-- !query
@@ -115,7 +151,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-The expression '1' is not a valid schema string.; line 1 pos 7
+{
+ "errorClass" : "INVALID_SCHEMA.NON_STRING_LITERAL",
+ "sqlState" : "42K07",
+ "messageParameters" : {
+ "inputSchema" : "\"1\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 30,
+ "fragment" : "from_json('{\"a\":1}', 1)"
+ } ]
+}
-- !query
@@ -124,20 +173,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Cannot parse the data type:
-Syntax error at or near 'InvalidType': extra input 'InvalidType'(line 1, pos 2)
-
-== SQL ==
-a InvalidType
---^^^
-
-Failed fallback parsing:
-DataType invalidtype is not supported.(line 1, pos 2)
-
-== SQL ==
-a InvalidType
---^^^
-; line 1 pos 7
+{
+ "errorClass" : "PARSE_SYNTAX_ERROR",
+ "sqlState" : "42601",
+ "messageParameters" : {
+ "error" : "'InvalidType'",
+ "hint" : ": extra input 'InvalidType'"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 44,
+ "fragment" : "from_json('{\"a\":1}', 'a InvalidType')"
+ } ]
+}
-- !query
@@ -146,7 +196,17 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Must use a map() function for options; line 1 pos 7
+{
+ "errorClass" : "INVALID_OPTIONS.NON_MAP_FUNCTION",
+ "sqlState" : "42K06",
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 72,
+ "fragment" : "from_json('{\"a\":1}', 'a INT', named_struct('mode', 'PERMISSIVE'))"
+ } ]
+}
-- !query
@@ -155,7 +215,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-A type of keys and values in map() must be string, but got map<string,int>; line 1 pos 7
+{
+ "errorClass" : "INVALID_OPTIONS.NON_STRING_TYPE",
+ "sqlState" : "42K06",
+ "messageParameters" : {
+ "mapType" : "\"MAP<STRING, INT>\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 52,
+ "fragment" : "from_json('{\"a\":1}', 'a INT', map('mode', 1))"
+ } ]
+}
-- !query
@@ -164,7 +237,23 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function from_json. Expected: one of 2 and 3; Found: 0; line 1 pos 7
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "0",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "[2, 3]",
+ "functionName" : "`from_json`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 18,
+ "fragment" : "from_json()"
+ } ]
+}
-- !query
@@ -339,7 +428,14 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '02-29' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "datetime" : "'02-29'"
+ }
+}
-- !query
@@ -351,7 +447,14 @@
struct<>
-- !query output
org.apache.spark.SparkUpgradeException
-You may get a different result due to the upgrading to Spark >= 3.0: Fail to parse '02-29' in the new parser. You can set spark.sql.legacy.timeParserPolicy to LEGACY to restore the behavior before Spark 3.0, or set to CORRECTED and treat it as an invalid datetime string.
+{
+ "errorClass" : "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER",
+ "sqlState" : "42K0B",
+ "messageParameters" : {
+ "config" : "\"spark.sql.legacy.timeParserPolicy\"",
+ "datetime" : "'02-29'"
+ }
+}
-- !query
@@ -392,7 +495,21 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'schema_of_json(NULL)' due to data type mismatch: The input json should be a foldable string expression and not null; however, got NULL.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_NULL",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "exprName" : "json",
+ "sqlExpr" : "\"schema_of_json(NULL)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 27,
+ "fragment" : "schema_of_json(null)"
+ } ]
+}
-- !query
@@ -409,7 +526,23 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'schema_of_json(jsontable.jsonField)' due to data type mismatch: The input json should be a foldable string expression and not null; however, got jsontable.jsonField.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputExpr" : "\"jsonField\"",
+ "inputName" : "json",
+ "inputType" : "\"STRING\"",
+ "sqlExpr" : "\"schema_of_json(jsonField)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 32,
+ "fragment" : "schema_of_json(jsonField)"
+ } ]
+}
-- !query
@@ -426,7 +559,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'json_array_length(2)' due to data type mismatch: argument 1 requires string type, however, '2' is of int type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"2\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"STRING\"",
+ "sqlExpr" : "\"json_array_length(2)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 27,
+ "fragment" : "json_array_length(2)"
+ } ]
+}
-- !query
@@ -435,7 +585,23 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function json_array_length. Expected: 1; Found: 0; line 1 pos 7
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "0",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "1",
+ "functionName" : "`json_array_length`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 26,
+ "fragment" : "json_array_length()"
+ } ]
+}
-- !query
@@ -508,7 +674,23 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function json_object_keys. Expected: 1; Found: 0; line 1 pos 7
+{
+ "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION",
+ "sqlState" : "42605",
+ "messageParameters" : {
+ "actualNum" : "0",
+ "docroot" : "https://spark.apache.org/docs/latest",
+ "expectedNum" : "1",
+ "functionName" : "`json_object_keys`"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 25,
+ "fragment" : "json_object_keys()"
+ } ]
+}
-- !query
@@ -525,7 +707,24 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-cannot resolve 'json_object_keys(200)' due to data type mismatch: argument 1 requires string type, however, '200' is of int type.; line 1 pos 7
+{
+ "errorClass" : "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE",
+ "sqlState" : "42K09",
+ "messageParameters" : {
+ "inputSql" : "\"200\"",
+ "inputType" : "\"INT\"",
+ "paramIndex" : "1",
+ "requiredType" : "\"STRING\"",
+ "sqlExpr" : "\"json_object_keys(200)\""
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 8,
+ "stopIndex" : 28,
+ "fragment" : "json_object_keys(200)"
+ } ]
+}
-- !query
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/like-all.sql.out sql/core/src/test/resources/sql-tests/results/like-all.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/like-all.sql.out 2023-09-21 10:45:41.959461262 +0800
+++ sql/core/src/test/resources/sql-tests/results/like-all.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 14
-
-
-- !query
CREATE OR REPLACE TEMPORARY VIEW like_all_table AS SELECT * FROM (VALUES
('google', '%oo%'),
@@ -132,9 +129,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Expected something between '(' and ')'.(line 1, pos 49)
-
-== SQL ==
-SELECT company FROM like_all_table WHERE company LIKE ALL ()
--------------------------------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0064",
+ "messageParameters" : {
+ "msg" : "Expected something between '(' and ')'."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 50,
+ "stopIndex" : 60,
+ "fragment" : "LIKE ALL ()"
+ } ]
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/like-any.sql.out sql/core/src/test/resources/sql-tests/results/like-any.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/like-any.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/like-any.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 14
-
-
-- !query
CREATE OR REPLACE TEMPORARY VIEW like_any_table AS SELECT * FROM (VALUES
('google', '%oo%'),
@@ -138,9 +135,16 @@
struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
-
-Expected something between '(' and ')'.(line 1, pos 49)
-
-== SQL ==
-SELECT company FROM like_any_table WHERE company LIKE ANY ()
--------------------------------------------------^^^
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_0064",
+ "messageParameters" : {
+ "msg" : "Expected something between '(' and ')'."
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 50,
+ "stopIndex" : 60,
+ "fragment" : "LIKE ANY ()"
+ } ]
+}
diff -urN /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/limit.sql.out sql/core/src/test/resources/sql-tests/results/limit.sql.out
--- /mnt/nvme1/git/spark/sql/core/src/test/resources/sql-tests/results/limit.sql.out 2023-09-15 11:42:05.970317944 +0800
+++ sql/core/src/test/resources/sql-tests/results/limit.sql.out 2023-09-18 15:00:43.014708418 +0800
@@ -1,7 +1,4 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 14
-
-
-- !query
SELECT * FROM testdata LIMIT 2
-- !query schema
@@ -53,7 +50,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-The limit expression must be equal to or greater than 0, but got -1
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_2403",
+ "messageParameters" : {
+ "name" : "limit",
+ "v" : "-1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 30,
+ "stopIndex" : 31,
+ "fragment" : "-1"
+ } ]
+}
-- !query
@@ -62,7 +72,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-The limit expression must be equal to or greater than 0, but got -1
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_2403",
+ "messageParameters" : {
+ "name" : "limit",
+ "v" : "-1"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 37,
+ "stopIndex" : 38,
+ "fragment" : "-1"
+ } ]
+}
-- !query
@@ -79,7 +102,20 @@
struct<>
-- !query output
org.apache.spark.sql.AnalysisException
-The evaluated limit expression must not be null, but got CAST(NULL AS INT)
+{
+ "errorClass" : "_LEGACY_ERROR_TEMP_2402",
+ "messageParameters" :
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment