diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 2b459e4c73bbb..14d93a498fc59 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -264,20 +264,20 @@ jobs: with: distribution: zulu java-version: ${{ matrix.java }} - - name: Install Python 3.9 + - name: Install Python 3.11 uses: actions/setup-python@v5 # We should install one Python that is higher than 3+ for SQL and Yarn because: # - SQL component also has Python related tests, for example, IntegratedUDFTestUtils. # - Yarn has a Python specific test too, for example, YarnClusterSuite. if: contains(matrix.modules, 'yarn') || (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) || contains(matrix.modules, 'connect') with: - python-version: '3.9' + python-version: '3.11' architecture: x64 - - name: Install Python packages (Python 3.9) + - name: Install Python packages (Python 3.11) if: (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-')) || contains(matrix.modules, 'connect') run: | - python3.9 -m pip install 'numpy>=1.20.0' pyarrow pandas scipy unittest-xml-reporting 'lxml==4.9.4' 'grpcio==1.62.0' 'grpcio-status==1.62.0' 'protobuf==4.25.1' - python3.9 -m pip list + python3.11 -m pip install 'numpy>=1.20.0' pyarrow pandas scipy unittest-xml-reporting 'lxml==4.9.4' 'grpcio==1.62.0' 'grpcio-status==1.62.0' 'protobuf==4.25.1' + python3.11 -m pip list # Run the tests. - name: Run tests env: ${{ fromJSON(inputs.envs) }} @@ -608,14 +608,14 @@ jobs: with: input: sql/connect/common/src/main against: 'https://github.com/apache/spark.git#branch=branch-3.5,subdir=connector/connect/common/src/main' - - name: Install Python 3.9 + - name: Install Python 3.11 uses: actions/setup-python@v5 with: - python-version: '3.9' + python-version: '3.11' - name: Install dependencies for Python CodeGen check run: | - python3.9 -m pip install 'black==23.9.1' 'protobuf==4.25.1' 'mypy==1.8.0' 'mypy-protobuf==3.3.0' - python3.9 -m pip list + python3.11 -m pip install 'black==23.9.1' 'protobuf==4.25.1' 'mypy==1.8.0' 'mypy-protobuf==3.3.0' + python3.11 -m pip list - name: Python CodeGen check run: ./dev/connect-check-protos.py @@ -1112,6 +1112,10 @@ jobs: with: distribution: zulu java-version: ${{ inputs.java }} + - name: Install R + run: | + sudo apt update + sudo apt-get install r-base - name: Start Minikube uses: medyagh/setup-minikube@v0.0.18 with: diff --git a/assembly/pom.xml b/assembly/pom.xml index 01bd324efc118..17bb81fa023ba 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -117,6 +117,12 @@ org.apache.spark spark-connect-client-jvm_${scala.binary.version} ${project.version} + + + org.apache.spark + spark-connect-shims_${scala.binary.version} + + provided diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationAwareUTF8String.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationAwareUTF8String.java index fb610a5d96f17..d67697eaea38b 100644 --- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationAwareUTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationAwareUTF8String.java @@ -1363,9 +1363,9 @@ public static UTF8String trimRight( public static UTF8String[] splitSQL(final UTF8String input, final UTF8String delim, final int limit, final int collationId) { - if (CollationFactory.fetchCollation(collationId).supportsBinaryEquality) { + if (CollationFactory.fetchCollation(collationId).isUtf8BinaryType) { return input.split(delim, limit); - } else if (CollationFactory.fetchCollation(collationId).supportsLowercaseEquality) { + } else if (CollationFactory.fetchCollation(collationId).isUtf8LcaseType) { return lowercaseSplitSQL(input, delim, limit); } else { return icuSplitSQL(input, delim, limit, collationId); diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java index 113c5f866fd88..50bb93465921e 100644 --- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationFactory.java @@ -154,6 +154,24 @@ public static class Collation { */ public final boolean supportsLowercaseEquality; + /** + * Support for Space Trimming implies that that based on specifier (for now only right trim) + * leading, trailing or both spaces are removed from the input string before comparison. + */ + public final boolean supportsSpaceTrimming; + + /** + * Is Utf8 binary type as indicator if collation base type is UTF8 binary. Note currently only + * collations Utf8_Binary and Utf8_Binary_RTRIM are considered as Utf8 binary type. + */ + public final boolean isUtf8BinaryType; + + /** + * Is Utf8 lcase type as indicator if collation base type is UTF8 lcase. Note currently only + * collations Utf8_Lcase and Utf8_Lcase_RTRIM are considered as Utf8 Lcase type. + */ + public final boolean isUtf8LcaseType; + public Collation( String collationName, String provider, @@ -161,31 +179,27 @@ public Collation( Comparator comparator, String version, ToLongFunction hashFunction, - boolean supportsBinaryEquality, - boolean supportsBinaryOrdering, - boolean supportsLowercaseEquality) { + BiFunction equalsFunction, + boolean isUtf8BinaryType, + boolean isUtf8LcaseType, + boolean supportsSpaceTrimming) { this.collationName = collationName; this.provider = provider; this.collator = collator; this.comparator = comparator; this.version = version; this.hashFunction = hashFunction; - this.supportsBinaryEquality = supportsBinaryEquality; - this.supportsBinaryOrdering = supportsBinaryOrdering; - this.supportsLowercaseEquality = supportsLowercaseEquality; - - // De Morgan's Law to check supportsBinaryOrdering => supportsBinaryEquality - assert(!supportsBinaryOrdering || supportsBinaryEquality); + this.isUtf8BinaryType = isUtf8BinaryType; + this.isUtf8LcaseType = isUtf8LcaseType; + this.equalsFunction = equalsFunction; + this.supportsSpaceTrimming = supportsSpaceTrimming; + this.supportsBinaryEquality = !supportsSpaceTrimming && isUtf8BinaryType; + this.supportsBinaryOrdering = !supportsSpaceTrimming && isUtf8BinaryType; + this.supportsLowercaseEquality = !supportsSpaceTrimming && isUtf8LcaseType; // No Collation can simultaneously support binary equality and lowercase equality assert(!supportsBinaryEquality || !supportsLowercaseEquality); assert(SUPPORTED_PROVIDERS.contains(provider)); - - if (supportsBinaryEquality) { - this.equalsFunction = UTF8String::equals; - } else { - this.equalsFunction = (s1, s2) -> this.comparator.compare(s1, s2) == 0; - } } /** @@ -538,27 +552,61 @@ private static boolean isValidCollationId(int collationId) { @Override protected Collation buildCollation() { if (caseSensitivity == CaseSensitivity.UNSPECIFIED) { + Comparator comparator; + ToLongFunction hashFunction; + BiFunction equalsFunction; + boolean supportsSpaceTrimming = spaceTrimming != SpaceTrimming.NONE; + + if (spaceTrimming == SpaceTrimming.NONE) { + comparator = UTF8String::binaryCompare; + hashFunction = s -> (long) s.hashCode(); + equalsFunction = UTF8String::equals; + } else { + comparator = (s1, s2) -> applyTrimmingPolicy(s1, spaceTrimming).binaryCompare( + applyTrimmingPolicy(s2, spaceTrimming)); + hashFunction = s -> (long) applyTrimmingPolicy(s, spaceTrimming).hashCode(); + equalsFunction = (s1, s2) -> applyTrimmingPolicy(s1, spaceTrimming).equals( + applyTrimmingPolicy(s2, spaceTrimming)); + } + return new Collation( normalizedCollationName(), PROVIDER_SPARK, null, - UTF8String::binaryCompare, + comparator, "1.0", - s -> (long) s.hashCode(), - /* supportsBinaryEquality = */ true, - /* supportsBinaryOrdering = */ true, - /* supportsLowercaseEquality = */ false); + hashFunction, + equalsFunction, + /* isUtf8BinaryType = */ true, + /* isUtf8LcaseType = */ false, + spaceTrimming != SpaceTrimming.NONE); } else { + Comparator comparator; + ToLongFunction hashFunction; + + if (spaceTrimming == SpaceTrimming.NONE) { + comparator = CollationAwareUTF8String::compareLowerCase; + hashFunction = s -> + (long) CollationAwareUTF8String.lowerCaseCodePoints(s).hashCode(); + } else { + comparator = (s1, s2) -> CollationAwareUTF8String.compareLowerCase( + applyTrimmingPolicy(s1, spaceTrimming), + applyTrimmingPolicy(s2, spaceTrimming)); + hashFunction = s -> (long) CollationAwareUTF8String.lowerCaseCodePoints( + applyTrimmingPolicy(s, spaceTrimming)).hashCode(); + } + return new Collation( normalizedCollationName(), PROVIDER_SPARK, null, - CollationAwareUTF8String::compareLowerCase, + comparator, "1.0", - s -> (long) CollationAwareUTF8String.lowerCaseCodePoints(s).hashCode(), - /* supportsBinaryEquality = */ false, - /* supportsBinaryOrdering = */ false, - /* supportsLowercaseEquality = */ true); + hashFunction, + (s1, s2) -> comparator.compare(s1, s2) == 0, + /* isUtf8BinaryType = */ false, + /* isUtf8LcaseType = */ true, + spaceTrimming != SpaceTrimming.NONE); } } @@ -917,16 +965,34 @@ protected Collation buildCollation() { Collator collator = Collator.getInstance(resultLocale); // Freeze ICU collator to ensure thread safety. collator.freeze(); + + Comparator comparator; + ToLongFunction hashFunction; + + if (spaceTrimming == SpaceTrimming.NONE) { + hashFunction = s -> (long) collator.getCollationKey( + s.toValidString()).hashCode(); + comparator = (s1, s2) -> + collator.compare(s1.toValidString(), s2.toValidString()); + } else { + comparator = (s1, s2) -> collator.compare( + applyTrimmingPolicy(s1, spaceTrimming).toValidString(), + applyTrimmingPolicy(s2, spaceTrimming).toValidString()); + hashFunction = s -> (long) collator.getCollationKey( + applyTrimmingPolicy(s, spaceTrimming).toValidString()).hashCode(); + } + return new Collation( normalizedCollationName(), PROVIDER_ICU, collator, - (s1, s2) -> collator.compare(s1.toValidString(), s2.toValidString()), + comparator, ICU_COLLATOR_VERSION, - s -> (long) collator.getCollationKey(s.toValidString()).hashCode(), - /* supportsBinaryEquality = */ false, - /* supportsBinaryOrdering = */ false, - /* supportsLowercaseEquality = */ false); + hashFunction, + (s1, s2) -> comparator.compare(s1, s2) == 0, + /* isUtf8BinaryType = */ false, + /* isUtf8LcaseType = */ false, + spaceTrimming != SpaceTrimming.NONE); } @Override @@ -1103,14 +1169,6 @@ public static boolean isCaseSensitiveAndAccentInsensitive(int collationId) { Collation.CollationSpecICU.AccentSensitivity.AI; } - /** - * Returns whether the collation uses trim collation for the given collation id. - */ - public static boolean usesTrimCollation(int collationId) { - return Collation.CollationSpec.getSpaceTrimming(collationId) != - Collation.CollationSpec.SpaceTrimming.NONE; - } - public static void assertValidProvider(String provider) throws SparkException { if (!SUPPORTED_PROVIDERS.contains(provider.toLowerCase())) { Map params = Map.of( @@ -1137,12 +1195,12 @@ public static String[] getICULocaleNames() { public static UTF8String getCollationKey(UTF8String input, int collationId) { Collation collation = fetchCollation(collationId); - if (usesTrimCollation(collationId)) { + if (collation.supportsSpaceTrimming) { input = Collation.CollationSpec.applyTrimmingPolicy(input, collationId); } - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return input; - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return CollationAwareUTF8String.lowerCaseCodePoints(input); } else { CollationKey collationKey = collation.collator.getCollationKey( @@ -1153,12 +1211,12 @@ public static UTF8String getCollationKey(UTF8String input, int collationId) { public static byte[] getCollationKeyBytes(UTF8String input, int collationId) { Collation collation = fetchCollation(collationId); - if (usesTrimCollation(collationId)) { + if (collation.supportsSpaceTrimming) { input = Collation.CollationSpec.applyTrimmingPolicy(input, collationId); } - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return input.getBytes(); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return CollationAwareUTF8String.lowerCaseCodePoints(input).getBytes(); } else { return collation.collator.getCollationKey( diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationSupport.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationSupport.java index f05d9e512568f..978b663cc25c9 100644 --- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationSupport.java +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationSupport.java @@ -37,9 +37,9 @@ public final class CollationSupport { public static class StringSplitSQL { public static UTF8String[] exec(final UTF8String s, final UTF8String d, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return execBinary(s, d); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return execLowercase(s, d); } else { return execICU(s, d, collationId); @@ -48,9 +48,9 @@ public static UTF8String[] exec(final UTF8String s, final UTF8String d, final in public static String genCode(final String s, final String d, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); String expr = "CollationSupport.StringSplitSQL.exec"; - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return String.format(expr + "Binary(%s, %s)", s, d); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return String.format(expr + "Lowercase(%s, %s)", s, d); } else { return String.format(expr + "ICU(%s, %s, %d)", s, d, collationId); @@ -71,9 +71,9 @@ public static UTF8String[] execICU(final UTF8String string, final UTF8String del public static class Contains { public static boolean exec(final UTF8String l, final UTF8String r, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return execBinary(l, r); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return execLowercase(l, r); } else { return execICU(l, r, collationId); @@ -82,9 +82,9 @@ public static boolean exec(final UTF8String l, final UTF8String r, final int col public static String genCode(final String l, final String r, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); String expr = "CollationSupport.Contains.exec"; - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return String.format(expr + "Binary(%s, %s)", l, r); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return String.format(expr + "Lowercase(%s, %s)", l, r); } else { return String.format(expr + "ICU(%s, %s, %d)", l, r, collationId); @@ -109,9 +109,9 @@ public static class StartsWith { public static boolean exec(final UTF8String l, final UTF8String r, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return execBinary(l, r); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return execLowercase(l, r); } else { return execICU(l, r, collationId); @@ -120,9 +120,9 @@ public static boolean exec(final UTF8String l, final UTF8String r, public static String genCode(final String l, final String r, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); String expr = "CollationSupport.StartsWith.exec"; - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return String.format(expr + "Binary(%s, %s)", l, r); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return String.format(expr + "Lowercase(%s, %s)", l, r); } else { return String.format(expr + "ICU(%s, %s, %d)", l, r, collationId); @@ -146,9 +146,9 @@ public static boolean execICU(final UTF8String l, final UTF8String r, public static class EndsWith { public static boolean exec(final UTF8String l, final UTF8String r, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return execBinary(l, r); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return execLowercase(l, r); } else { return execICU(l, r, collationId); @@ -157,9 +157,9 @@ public static boolean exec(final UTF8String l, final UTF8String r, final int col public static String genCode(final String l, final String r, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); String expr = "CollationSupport.EndsWith.exec"; - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return String.format(expr + "Binary(%s, %s)", l, r); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return String.format(expr + "Lowercase(%s, %s)", l, r); } else { return String.format(expr + "ICU(%s, %s, %d)", l, r, collationId); @@ -184,9 +184,9 @@ public static boolean execICU(final UTF8String l, final UTF8String r, public static class Upper { public static UTF8String exec(final UTF8String v, final int collationId, boolean useICU) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return useICU ? execBinaryICU(v) : execBinary(v); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return execLowercase(v); } else { return execICU(v, collationId); @@ -195,10 +195,10 @@ public static UTF8String exec(final UTF8String v, final int collationId, boolean public static String genCode(final String v, final int collationId, boolean useICU) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); String expr = "CollationSupport.Upper.exec"; - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { String funcName = useICU ? "BinaryICU" : "Binary"; return String.format(expr + "%s(%s)", funcName, v); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return String.format(expr + "Lowercase(%s)", v); } else { return String.format(expr + "ICU(%s, %d)", v, collationId); @@ -221,9 +221,9 @@ public static UTF8String execICU(final UTF8String v, final int collationId) { public static class Lower { public static UTF8String exec(final UTF8String v, final int collationId, boolean useICU) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return useICU ? execBinaryICU(v) : execBinary(v); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return execLowercase(v); } else { return execICU(v, collationId); @@ -232,10 +232,10 @@ public static UTF8String exec(final UTF8String v, final int collationId, boolean public static String genCode(final String v, final int collationId, boolean useICU) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); String expr = "CollationSupport.Lower.exec"; - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { String funcName = useICU ? "BinaryICU" : "Binary"; return String.format(expr + "%s(%s)", funcName, v); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return String.format(expr + "Lowercase(%s)", v); } else { return String.format(expr + "ICU(%s, %d)", v, collationId); @@ -258,9 +258,9 @@ public static UTF8String execICU(final UTF8String v, final int collationId) { public static class InitCap { public static UTF8String exec(final UTF8String v, final int collationId, boolean useICU) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return useICU ? execBinaryICU(v) : execBinary(v); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return execLowercase(v); } else { return execICU(v, collationId); @@ -270,10 +270,10 @@ public static UTF8String exec(final UTF8String v, final int collationId, boolean public static String genCode(final String v, final int collationId, boolean useICU) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); String expr = "CollationSupport.InitCap.exec"; - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { String funcName = useICU ? "BinaryICU" : "Binary"; return String.format(expr + "%s(%s)", funcName, v); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return String.format(expr + "Lowercase(%s)", v); } else { return String.format(expr + "ICU(%s, %d)", v, collationId); @@ -296,7 +296,7 @@ public static UTF8String execICU(final UTF8String v, final int collationId) { public static class FindInSet { public static int exec(final UTF8String word, final UTF8String set, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return execBinary(word, set); } else { return execCollationAware(word, set, collationId); @@ -305,7 +305,7 @@ public static int exec(final UTF8String word, final UTF8String set, final int co public static String genCode(final String word, final String set, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); String expr = "CollationSupport.FindInSet.exec"; - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return String.format(expr + "Binary(%s, %s)", word, set); } else { return String.format(expr + "CollationAware(%s, %s, %d)", word, set, collationId); @@ -324,9 +324,9 @@ public static class StringInstr { public static int exec(final UTF8String string, final UTF8String substring, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return execBinary(string, substring); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return execLowercase(string, substring); } else { return execICU(string, substring, collationId); @@ -336,9 +336,9 @@ public static String genCode(final String string, final String substring, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); String expr = "CollationSupport.StringInstr.exec"; - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return String.format(expr + "Binary(%s, %s)", string, substring); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return String.format(expr + "Lowercase(%s, %s)", string, substring); } else { return String.format(expr + "ICU(%s, %s, %d)", string, substring, collationId); @@ -360,9 +360,9 @@ public static class StringReplace { public static UTF8String exec(final UTF8String src, final UTF8String search, final UTF8String replace, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return execBinary(src, search, replace); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return execLowercase(src, search, replace); } else { return execICU(src, search, replace, collationId); @@ -372,9 +372,9 @@ public static String genCode(final String src, final String search, final String final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); String expr = "CollationSupport.StringReplace.exec"; - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return String.format(expr + "Binary(%s, %s, %s)", src, search, replace); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return String.format(expr + "Lowercase(%s, %s, %s)", src, search, replace); } else { return String.format(expr + "ICU(%s, %s, %s, %d)", src, search, replace, collationId); @@ -398,9 +398,9 @@ public static class StringLocate { public static int exec(final UTF8String string, final UTF8String substring, final int start, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return execBinary(string, substring, start); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return execLowercase(string, substring, start); } else { return execICU(string, substring, start, collationId); @@ -410,9 +410,9 @@ public static String genCode(final String string, final String substring, final final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); String expr = "CollationSupport.StringLocate.exec"; - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return String.format(expr + "Binary(%s, %s, %d)", string, substring, start); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return String.format(expr + "Lowercase(%s, %s, %d)", string, substring, start); } else { return String.format(expr + "ICU(%s, %s, %d, %d)", string, substring, start, collationId); @@ -436,9 +436,9 @@ public static class SubstringIndex { public static UTF8String exec(final UTF8String string, final UTF8String delimiter, final int count, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return execBinary(string, delimiter, count); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return execLowercase(string, delimiter, count); } else { return execICU(string, delimiter, count, collationId); @@ -448,9 +448,9 @@ public static String genCode(final String string, final String delimiter, final String count, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); String expr = "CollationSupport.SubstringIndex.exec"; - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return String.format(expr + "Binary(%s, %s, %s)", string, delimiter, count); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return String.format(expr + "Lowercase(%s, %s, %s)", string, delimiter, count); } else { return String.format(expr + "ICU(%s, %s, %s, %d)", string, delimiter, count, collationId); @@ -474,9 +474,9 @@ public static class StringTranslate { public static UTF8String exec(final UTF8String source, Map dict, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return execBinary(source, dict); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return execLowercase(source, dict); } else { return execICU(source, dict, collationId); @@ -503,9 +503,9 @@ public static UTF8String exec( final UTF8String trimString, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return execBinary(srcString, trimString); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return execLowercase(srcString, trimString); } else { return execICU(srcString, trimString, collationId); @@ -520,9 +520,9 @@ public static String genCode( final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); String expr = "CollationSupport.StringTrim.exec"; - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return String.format(expr + "Binary(%s, %s)", srcString, trimString); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return String.format(expr + "Lowercase(%s, %s)", srcString, trimString); } else { return String.format(expr + "ICU(%s, %s, %d)", srcString, trimString, collationId); @@ -559,9 +559,9 @@ public static UTF8String exec( final UTF8String trimString, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return execBinary(srcString, trimString); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return execLowercase(srcString, trimString); } else { return execICU(srcString, trimString, collationId); @@ -576,9 +576,9 @@ public static String genCode( final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); String expr = "CollationSupport.StringTrimLeft.exec"; - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return String.format(expr + "Binary(%s, %s)", srcString, trimString); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return String.format(expr + "Lowercase(%s, %s)", srcString, trimString); } else { return String.format(expr + "ICU(%s, %s, %d)", srcString, trimString, collationId); @@ -614,9 +614,9 @@ public static UTF8String exec( final UTF8String trimString, final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return execBinary(srcString, trimString); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return execLowercase(srcString, trimString); } else { return execICU(srcString, trimString, collationId); @@ -631,9 +631,9 @@ public static String genCode( final int collationId) { CollationFactory.Collation collation = CollationFactory.fetchCollation(collationId); String expr = "CollationSupport.StringTrimRight.exec"; - if (collation.supportsBinaryEquality) { + if (collation.isUtf8BinaryType) { return String.format(expr + "Binary(%s, %s)", srcString, trimString); - } else if (collation.supportsLowercaseEquality) { + } else if (collation.isUtf8LcaseType) { return String.format(expr + "Lowercase(%s, %s)", srcString, trimString); } else { return String.format(expr + "ICU(%s, %s, %d)", srcString, trimString, collationId); @@ -669,7 +669,7 @@ public static UTF8String execICU( public static boolean supportsLowercaseRegex(final int collationId) { // for regex, only Unicode case-insensitive matching is possible, // so UTF8_LCASE is treated as UNICODE_CI in this context - return CollationFactory.fetchCollation(collationId).supportsLowercaseEquality; + return CollationFactory.fetchCollation(collationId).isUtf8LcaseType; } static final int lowercaseRegexFlags = Pattern.UNICODE_CASE | Pattern.CASE_INSENSITIVE; diff --git a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala index ff40f16e5a052..df9af1579d4f1 100644 --- a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala +++ b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala @@ -38,22 +38,22 @@ class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ig assert(UTF8_BINARY_COLLATION_ID == 0) val utf8Binary = fetchCollation(UTF8_BINARY_COLLATION_ID) assert(utf8Binary.collationName == "UTF8_BINARY") - assert(utf8Binary.supportsBinaryEquality) + assert(utf8Binary.isUtf8BinaryType) assert(UTF8_LCASE_COLLATION_ID == 1) val utf8Lcase = fetchCollation(UTF8_LCASE_COLLATION_ID) assert(utf8Lcase.collationName == "UTF8_LCASE") - assert(!utf8Lcase.supportsBinaryEquality) + assert(!utf8Lcase.isUtf8BinaryType) assert(UNICODE_COLLATION_ID == (1 << 29)) val unicode = fetchCollation(UNICODE_COLLATION_ID) assert(unicode.collationName == "UNICODE") - assert(!unicode.supportsBinaryEquality) + assert(!unicode.isUtf8BinaryType) assert(UNICODE_CI_COLLATION_ID == ((1 << 29) | (1 << 17))) val unicodeCi = fetchCollation(UNICODE_CI_COLLATION_ID) assert(unicodeCi.collationName == "UNICODE_CI") - assert(!unicodeCi.supportsBinaryEquality) + assert(!unicodeCi.isUtf8BinaryType) } test("UTF8_BINARY and ICU root locale collation names") { @@ -127,6 +127,11 @@ class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ig CollationTestCase("UTF8_BINARY", "aaa", "AAA", false), CollationTestCase("UTF8_BINARY", "aaa", "bbb", false), CollationTestCase("UTF8_BINARY", "å", "a\u030A", false), + CollationTestCase("UTF8_BINARY_RTRIM", "aaa", "aaa", true), + CollationTestCase("UTF8_BINARY_RTRIM", "aaa", "aaa ", true), + CollationTestCase("UTF8_BINARY_RTRIM", "aaa ", "aaa ", true), + CollationTestCase("UTF8_BINARY_RTRIM", "aaa", " aaa ", false), + CollationTestCase("UTF8_BINARY_RTRIM", " ", " ", true), CollationTestCase("UTF8_LCASE", "aaa", "aaa", true), CollationTestCase("UTF8_LCASE", "aaa", "AAA", true), CollationTestCase("UTF8_LCASE", "aaa", "AaA", true), @@ -134,15 +139,30 @@ class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ig CollationTestCase("UTF8_LCASE", "aaa", "aa", false), CollationTestCase("UTF8_LCASE", "aaa", "bbb", false), CollationTestCase("UTF8_LCASE", "å", "a\u030A", false), + CollationTestCase("UTF8_LCASE_RTRIM", "aaa", "AaA", true), + CollationTestCase("UTF8_LCASE_RTRIM", "aaa", "AaA ", true), + CollationTestCase("UTF8_LCASE_RTRIM", "aaa ", "AaA ", true), + CollationTestCase("UTF8_LCASE_RTRIM", "aaa", " AaA ", false), + CollationTestCase("UTF8_LCASE_RTRIM", " ", " ", true), CollationTestCase("UNICODE", "aaa", "aaa", true), CollationTestCase("UNICODE", "aaa", "AAA", false), CollationTestCase("UNICODE", "aaa", "bbb", false), CollationTestCase("UNICODE", "å", "a\u030A", true), + CollationTestCase("UNICODE_RTRIM", "aaa", "aaa", true), + CollationTestCase("UNICODE_RTRIM", "aaa", "aaa ", true), + CollationTestCase("UNICODE_RTRIM", "aaa ", "aaa ", true), + CollationTestCase("UNICODE_RTRIM", "aaa", " aaa ", false), + CollationTestCase("UNICODE_RTRIM", " ", " ", true), CollationTestCase("UNICODE_CI", "aaa", "aaa", true), CollationTestCase("UNICODE_CI", "aaa", "AAA", true), CollationTestCase("UNICODE_CI", "aaa", "bbb", false), CollationTestCase("UNICODE_CI", "å", "a\u030A", true), - CollationTestCase("UNICODE_CI", "Å", "a\u030A", true) + CollationTestCase("UNICODE_CI", "Å", "a\u030A", true), + CollationTestCase("UNICODE_CI_RTRIM", "aaa", "AaA", true), + CollationTestCase("UNICODE_CI_RTRIM", "aaa", "AaA ", true), + CollationTestCase("UNICODE_CI_RTRIM", "aaa ", "AaA ", true), + CollationTestCase("UNICODE_CI_RTRIM", "aaa", " AaA ", false), + CollationTestCase("UNICODE_RTRIM", " ", " ", true) ) checks.foreach(testCase => { @@ -162,19 +182,48 @@ class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ig CollationTestCase("UTF8_BINARY", "aaa", "AAA", 1), CollationTestCase("UTF8_BINARY", "aaa", "bbb", -1), CollationTestCase("UTF8_BINARY", "aaa", "BBB", 1), + CollationTestCase("UTF8_BINARY_RTRIM", "aaa ", "aaa", 0), + CollationTestCase("UTF8_BINARY_RTRIM", "aaa ", "aaa ", 0), + CollationTestCase("UTF8_BINARY_RTRIM", "aaa ", "bbb", -1), + CollationTestCase("UTF8_BINARY_RTRIM", "aaa ", "bbb ", -1), + CollationTestCase("UTF8_BINARY_RTRIM", "aaa", "BBB" , 1), + CollationTestCase("UTF8_BINARY_RTRIM", "aaa ", "BBB " , 1), + CollationTestCase("UTF8_BINARY_RTRIM", " ", " " , 0), CollationTestCase("UTF8_LCASE", "aaa", "aaa", 0), CollationTestCase("UTF8_LCASE", "aaa", "AAA", 0), CollationTestCase("UTF8_LCASE", "aaa", "AaA", 0), CollationTestCase("UTF8_LCASE", "aaa", "AaA", 0), CollationTestCase("UTF8_LCASE", "aaa", "aa", 1), CollationTestCase("UTF8_LCASE", "aaa", "bbb", -1), + CollationTestCase("UTF8_LCASE_RTRIM", "aaa ", "AAA", 0), + CollationTestCase("UTF8_LCASE_RTRIM", "aaa ", "AAA ", 0), + CollationTestCase("UTF8_LCASE_RTRIM", "aaa", "bbb ", -1), + CollationTestCase("UTF8_LCASE_RTRIM", "aaa ", "bbb ", -1), + CollationTestCase("UTF8_LCASE_RTRIM", "aaa ", "aa", 1), + CollationTestCase("UTF8_LCASE_RTRIM", "aaa ", "aa ", 1), + CollationTestCase("UTF8_LCASE_RTRIM", " ", " ", 0), CollationTestCase("UNICODE", "aaa", "aaa", 0), CollationTestCase("UNICODE", "aaa", "AAA", -1), CollationTestCase("UNICODE", "aaa", "bbb", -1), CollationTestCase("UNICODE", "aaa", "BBB", -1), + CollationTestCase("UNICODE_RTRIM", "aaa ", "aaa", 0), + CollationTestCase("UNICODE_RTRIM", "aaa ", "aaa ", 0), + CollationTestCase("UNICODE_RTRIM", "aaa ", "bbb", -1), + CollationTestCase("UNICODE_RTRIM", "aaa ", "bbb ", -1), + CollationTestCase("UNICODE_RTRIM", "aaa", "BBB" , -1), + CollationTestCase("UNICODE_RTRIM", "aaa ", "BBB " , -1), + CollationTestCase("UNICODE_RTRIM", " ", " ", 0), CollationTestCase("UNICODE_CI", "aaa", "aaa", 0), CollationTestCase("UNICODE_CI", "aaa", "AAA", 0), - CollationTestCase("UNICODE_CI", "aaa", "bbb", -1)) + CollationTestCase("UNICODE_CI", "aaa", "bbb", -1), + CollationTestCase("UNICODE_CI_RTRIM", "aaa ", "AAA", 0), + CollationTestCase("UNICODE_CI_RTRIM", "aaa ", "AAA ", 0), + CollationTestCase("UNICODE_CI_RTRIM", "aaa", "bbb ", -1), + CollationTestCase("UNICODE_CI_RTRIM", "aaa ", "bbb ", -1), + CollationTestCase("UNICODE_CI_RTRIM", "aaa ", "aa", 1), + CollationTestCase("UNICODE_CI_RTRIM", "aaa ", "aa ", 1), + CollationTestCase("UNICODE_CI_RTRIM", " ", " ", 0) + ) checks.foreach(testCase => { val collation = fetchCollation(testCase.collationName) @@ -456,7 +505,7 @@ class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ig val e = intercept[SparkException] { fetchCollation(collationName) } - assert(e.getErrorClass === "COLLATION_INVALID_NAME") + assert(e.getCondition === "COLLATION_INVALID_NAME") assert(e.getMessageParameters.asScala === Map( "collationName" -> collationName, "proposals" -> proposals)) } diff --git a/common/utils/src/main/java/org/apache/spark/SparkThrowable.java b/common/utils/src/main/java/org/apache/spark/SparkThrowable.java index e1235b2982ba0..39808f58b08ae 100644 --- a/common/utils/src/main/java/org/apache/spark/SparkThrowable.java +++ b/common/utils/src/main/java/org/apache/spark/SparkThrowable.java @@ -35,19 +35,29 @@ */ @Evolving public interface SparkThrowable { - // Succinct, human-readable, unique, and consistent representation of the error category - // If null, error class is not set - String getErrorClass(); + /** + * Succinct, human-readable, unique, and consistent representation of the error condition. + * If null, error condition is not set. + */ + String getCondition(); + + /** + * Succinct, human-readable, unique, and consistent representation of the error category. + * If null, error class is not set. + * @deprecated Use {@link #getCondition()} instead. + */ + @Deprecated + default String getErrorClass() { return getCondition(); } // Portable error identifier across SQL engines // If null, error class or SQLSTATE is not set default String getSqlState() { - return SparkThrowableHelper.getSqlState(this.getErrorClass()); + return SparkThrowableHelper.getSqlState(this.getCondition()); } // True if this error is an internal error. default boolean isInternalError() { - return SparkThrowableHelper.isInternalError(this.getErrorClass()); + return SparkThrowableHelper.isInternalError(this.getCondition()); } default Map getMessageParameters() { diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index f6317d731c77b..502558c21faa9 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -376,6 +376,12 @@ ], "sqlState" : "429BB" }, + "CANNOT_REMOVE_RESERVED_PROPERTY" : { + "message" : [ + "Cannot remove reserved property: ." + ], + "sqlState" : "42000" + }, "CANNOT_RENAME_ACROSS_SCHEMA" : { "message" : [ "Renaming a across schemas is not allowed." @@ -606,6 +612,12 @@ ], "sqlState" : "42711" }, + "COLUMN_ARRAY_ELEMENT_TYPE_MISMATCH" : { + "message" : [ + "Some values in field are incompatible with the column array type. Expected type ." + ], + "sqlState" : "0A000" + }, "COLUMN_NOT_DEFINED_IN_TABLE" : { "message" : [ " column is not defined in table , defined table columns are: ." @@ -1212,6 +1224,12 @@ ], "sqlState" : "42604" }, + "EMPTY_SCHEMA_NOT_SUPPORTED_FOR_DATASOURCE" : { + "message" : [ + "The datasource does not support writing empty or nested empty schemas. Please make sure the data schema has at least one or more column(s)." + ], + "sqlState" : "0A000" + }, "ENCODER_NOT_FOUND" : { "message" : [ "Not found an encoder of the type to Spark SQL internal representation.", @@ -2537,6 +2555,11 @@ "Interval string does not match second-nano format of ss.nnnnnnnnn." ] }, + "TIMEZONE_INTERVAL_OUT_OF_RANGE" : { + "message" : [ + "The interval value must be in the range of [-18, +18] hours with second precision." + ] + }, "UNKNOWN_PARSING_ERROR" : { "message" : [ "Unknown error when parsing ." @@ -2588,6 +2611,13 @@ }, "sqlState" : "42K0K" }, + "INVALID_JAVA_IDENTIFIER_AS_FIELD_NAME" : { + "message" : [ + " is not a valid identifier of Java and cannot be used as field name", + "." + ], + "sqlState" : "46121" + }, "INVALID_JOIN_TYPE_FOR_JOINWITH" : { "message" : [ "Invalid join type in joinWith: ." @@ -3824,6 +3854,12 @@ ], "sqlState" : "42000" }, + "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY" : { + "message" : [ + "The expression must be inside 'partitionedBy'." + ], + "sqlState" : "42S23" + }, "PATH_ALREADY_EXISTS" : { "message" : [ "Path already exists. Set mode as \"overwrite\" to overwrite the existing path." @@ -3996,6 +4032,18 @@ ], "sqlState" : "22023" }, + "SCALAR_FUNCTION_NOT_COMPATIBLE" : { + "message" : [ + "ScalarFunction not overrides method 'produceResult(InternalRow)' with custom implementation." + ], + "sqlState" : "42K0O" + }, + "SCALAR_FUNCTION_NOT_FULLY_IMPLEMENTED" : { + "message" : [ + "ScalarFunction not implements or overrides method 'produceResult(InternalRow)'." + ], + "sqlState" : "42K0P" + }, "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION" : { "message" : [ "The correlated scalar subquery '' is neither present in GROUP BY, nor in an aggregate function.", @@ -5157,6 +5205,11 @@ "message" : [ "TRIM specifier in the collation." ] + }, + "UPDATE_COLUMN_NULLABILITY" : { + "message" : [ + "Update column nullability for MySQL and MS SQL Server." + ] } }, "sqlState" : "0A000" @@ -5685,11 +5738,6 @@ "Expected format is 'RESET' or 'RESET key'. If you want to include special characters in key, please use quotes, e.g., RESET `key`." ] }, - "_LEGACY_ERROR_TEMP_0044" : { - "message" : [ - "The interval value must be in the range of [-18, +18] hours with second precision." - ] - }, "_LEGACY_ERROR_TEMP_0045" : { "message" : [ "Invalid time zone displacement value." @@ -6107,11 +6155,6 @@ " is not a valid Spark SQL Data Source." ] }, - "_LEGACY_ERROR_TEMP_1136" : { - "message" : [ - "Cannot save interval data type into external storage." - ] - }, "_LEGACY_ERROR_TEMP_1137" : { "message" : [ "Unable to resolve given []." @@ -6137,11 +6180,6 @@ "Multiple sources found for (), please specify the fully qualified class name." ] }, - "_LEGACY_ERROR_TEMP_1142" : { - "message" : [ - "Datasource does not support writing empty or nested empty schemas. Please make sure the data schema has at least one or more column(s)." - ] - }, "_LEGACY_ERROR_TEMP_1143" : { "message" : [ "The data to be inserted needs to have the same number of columns as the target table: target table has column(s) but the inserted data has column(s), which contain partition column(s) having assigned constant values." @@ -6923,11 +6961,6 @@ "Missing database location." ] }, - "_LEGACY_ERROR_TEMP_2069" : { - "message" : [ - "Cannot remove reserved property: ." - ] - }, "_LEGACY_ERROR_TEMP_2070" : { "message" : [ "Writing job failed." @@ -7045,7 +7078,7 @@ }, "_LEGACY_ERROR_TEMP_2097" : { "message" : [ - "Could not execute broadcast in secs. You can increase the timeout for broadcasts via or disable broadcast join by setting to -1." + "Could not execute broadcast in secs. You can increase the timeout for broadcasts via or disable broadcast join by setting to -1 or remove the broadcast hint if it exists in your code." ] }, "_LEGACY_ERROR_TEMP_2098" : { @@ -7183,12 +7216,6 @@ "cannot have circular references in class, but got the circular reference of class ." ] }, - "_LEGACY_ERROR_TEMP_2140" : { - "message" : [ - "`` is not a valid identifier of Java and cannot be used as field name", - "." - ] - }, "_LEGACY_ERROR_TEMP_2144" : { "message" : [ "Unable to find constructor for . This could happen if is an interface, or a trait without companion object constructor." @@ -7677,11 +7704,6 @@ "comment on table is not supported." ] }, - "_LEGACY_ERROR_TEMP_2271" : { - "message" : [ - "UpdateColumnNullability is not supported." - ] - }, "_LEGACY_ERROR_TEMP_2272" : { "message" : [ "Rename column is only supported for MySQL version 8.0 and above." @@ -7946,11 +7968,6 @@ " is not currently supported" ] }, - "_LEGACY_ERROR_TEMP_3055" : { - "message" : [ - "ScalarFunction neither implement magic method nor override 'produceResult'" - ] - }, "_LEGACY_ERROR_TEMP_3056" : { "message" : [ "Unexpected row-level read relations (allow multiple = ): " @@ -8309,11 +8326,6 @@ "Partitions truncate is not supported" ] }, - "_LEGACY_ERROR_TEMP_3146" : { - "message" : [ - "Cannot find a compatible ScalarFunction#produceResult" - ] - }, "_LEGACY_ERROR_TEMP_3147" : { "message" : [ ": Batch scan are not supported" @@ -8699,6 +8711,21 @@ "Doesn't support month or year interval: " ] }, + "_LEGACY_ERROR_TEMP_3300" : { + "message" : [ + "error while calling spill() on : " + ] + }, + "_LEGACY_ERROR_TEMP_3301" : { + "message" : [ + "Not enough memory to grow pointer array" + ] + }, + "_LEGACY_ERROR_TEMP_3302" : { + "message" : [ + "No enough memory for aggregation" + ] + }, "_LEGACY_ERROR_USER_RAISED_EXCEPTION" : { "message" : [ "" diff --git a/common/utils/src/main/resources/error/error-states.json b/common/utils/src/main/resources/error/error-states.json index 87811fef9836e..fb899e4eb207e 100644 --- a/common/utils/src/main/resources/error/error-states.json +++ b/common/utils/src/main/resources/error/error-states.json @@ -4631,6 +4631,18 @@ "standard": "N", "usedBy": ["Spark"] }, + "42K0O": { + "description": "ScalarFunction not overrides method 'produceResult(InternalRow)' with custom implementation.", + "origin": "Spark", + "standard": "N", + "usedBy": ["Spark"] + }, + "42K0P": { + "description": "ScalarFunction not implements or overrides method 'produceResult(InternalRow)'.", + "origin": "Spark", + "standard": "N", + "usedBy": ["Spark"] + }, "42KD0": { "description": "Ambiguous name reference.", "origin": "Databricks", @@ -4901,6 +4913,12 @@ "standard": "N", "usedBy": ["SQL Server"] }, + "42S23": { + "description": "Partition transform expression not in 'partitionedBy'", + "origin": "Spark", + "standard": "N", + "usedBy": ["Spark"] + }, "44000": { "description": "with check option violation", "origin": "SQL/Foundation", diff --git a/common/utils/src/main/scala/org/apache/spark/SparkException.scala b/common/utils/src/main/scala/org/apache/spark/SparkException.scala index fcaee787fd8d3..0c0a1902ee2a1 100644 --- a/common/utils/src/main/scala/org/apache/spark/SparkException.scala +++ b/common/utils/src/main/scala/org/apache/spark/SparkException.scala @@ -69,7 +69,7 @@ class SparkException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull override def getQueryContext: Array[QueryContext] = context } @@ -179,7 +179,7 @@ private[spark] class SparkUpgradeException private( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull } /** @@ -212,7 +212,7 @@ private[spark] class SparkArithmeticException private( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull override def getQueryContext: Array[QueryContext] = context } @@ -250,7 +250,7 @@ private[spark] class SparkUnsupportedOperationException private( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull } private[spark] object SparkUnsupportedOperationException { @@ -280,7 +280,7 @@ private[spark] class SparkClassNotFoundException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass } /** @@ -296,7 +296,7 @@ private[spark] class SparkConcurrentModificationException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass } /** @@ -346,7 +346,7 @@ private[spark] class SparkDateTimeException private( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull override def getQueryContext: Array[QueryContext] = context } @@ -362,7 +362,7 @@ private[spark] class SparkFileNotFoundException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass } /** @@ -396,7 +396,7 @@ private[spark] class SparkNumberFormatException private( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull override def getQueryContext: Array[QueryContext] = context } @@ -448,7 +448,7 @@ private[spark] class SparkIllegalArgumentException private( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull override def getQueryContext: Array[QueryContext] = context } @@ -477,7 +477,7 @@ private[spark] class SparkRuntimeException private( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull override def getQueryContext: Array[QueryContext] = context } @@ -506,7 +506,7 @@ private[spark] class SparkPythonException private( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull override def getQueryContext: Array[QueryContext] = context } @@ -524,7 +524,7 @@ private[spark] class SparkNoSuchElementException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass override def getQueryContext: Array[QueryContext] = context } @@ -541,7 +541,7 @@ private[spark] class SparkSecurityException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass } /** @@ -575,7 +575,7 @@ private[spark] class SparkArrayIndexOutOfBoundsException private( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull override def getQueryContext: Array[QueryContext] = context } @@ -591,7 +591,7 @@ private[spark] class SparkSQLException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass } /** @@ -606,5 +606,5 @@ private[spark] class SparkSQLFeatureNotSupportedException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass } diff --git a/common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala b/common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala index 428c9d2a49351..b6c2b176de62b 100644 --- a/common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala +++ b/common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala @@ -81,7 +81,7 @@ private[spark] object SparkThrowableHelper { import ErrorMessageFormat._ format match { case PRETTY => e.getMessage - case MINIMAL | STANDARD if e.getErrorClass == null => + case MINIMAL | STANDARD if e.getCondition == null => toJsonString { generator => val g = generator.useDefaultPrettyPrinter() g.writeStartObject() @@ -92,7 +92,7 @@ private[spark] object SparkThrowableHelper { g.writeEndObject() } case MINIMAL | STANDARD => - val errorClass = e.getErrorClass + val errorClass = e.getCondition toJsonString { generator => val g = generator.useDefaultPrettyPrinter() g.writeStartObject() diff --git a/common/utils/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala b/common/utils/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala index 259f4330224c9..1972ef05d8759 100644 --- a/common/utils/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala +++ b/common/utils/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala @@ -84,7 +84,7 @@ class StreamingQueryException private[sql]( s"""${classOf[StreamingQueryException].getName}: ${cause.getMessage} |$queryDebugString""".stripMargin - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava } diff --git a/common/utils/src/main/scala/org/apache/spark/util/JsonUtils.scala b/common/utils/src/main/scala/org/apache/spark/util/JsonUtils.scala index 4d729adfbb7eb..f88f267727c11 100644 --- a/common/utils/src/main/scala/org/apache/spark/util/JsonUtils.scala +++ b/common/utils/src/main/scala/org/apache/spark/util/JsonUtils.scala @@ -24,6 +24,7 @@ import com.fasterxml.jackson.core.{JsonEncoding, JsonGenerator} import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} import com.fasterxml.jackson.module.scala.DefaultScalaModule +import org.apache.spark.util.SparkErrorUtils.tryWithResource private[spark] trait JsonUtils { @@ -31,12 +32,12 @@ private[spark] trait JsonUtils { .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) def toJsonString(block: JsonGenerator => Unit): String = { - val baos = new ByteArrayOutputStream() - val generator = mapper.createGenerator(baos, JsonEncoding.UTF8) - block(generator) - generator.close() - baos.close() - new String(baos.toByteArray, StandardCharsets.UTF_8) + tryWithResource(new ByteArrayOutputStream()) { baos => + tryWithResource(mapper.createGenerator(baos, JsonEncoding.UTF8)) { generator => + block(generator) + } + new String(baos.toByteArray, StandardCharsets.UTF_8) + } } } diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala index 751ac275e048a..bb0858decdf8f 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala @@ -436,7 +436,7 @@ abstract class AvroLogicalTypeSuite extends QueryTest with SharedSparkSession { val ex = intercept[SparkException] { spark.read.format("avro").load(s"$dir.avro").collect() } - assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(ex.getCondition.startsWith("FAILED_READ_FILE")) checkError( exception = ex.getCause.asInstanceOf[SparkArithmeticException], condition = "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index be887bd5237b0..0df6a7c4bc90e 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -891,7 +891,7 @@ abstract class AvroSuite val ex = intercept[SparkException] { spark.read.schema("a DECIMAL(4, 3)").format("avro").load(path.toString).collect() } - assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(ex.getCondition.startsWith("FAILED_READ_FILE")) checkError( exception = ex.getCause.asInstanceOf[AnalysisException], condition = "AVRO_INCOMPATIBLE_READ_TYPE", @@ -969,7 +969,7 @@ abstract class AvroSuite val ex = intercept[SparkException] { spark.read.schema(s"a $sqlType").format("avro").load(path.toString).collect() } - assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(ex.getCondition.startsWith("FAILED_READ_FILE")) checkError( exception = ex.getCause.asInstanceOf[AnalysisException], condition = "AVRO_INCOMPATIBLE_READ_TYPE", @@ -1006,7 +1006,7 @@ abstract class AvroSuite val ex = intercept[SparkException] { spark.read.schema(s"a $sqlType").format("avro").load(path.toString).collect() } - assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(ex.getCondition.startsWith("FAILED_READ_FILE")) checkError( exception = ex.getCause.asInstanceOf[AnalysisException], condition = "AVRO_INCOMPATIBLE_READ_TYPE", @@ -1515,7 +1515,7 @@ abstract class AvroSuite .write.format("avro").option("avroSchema", avroSchema) .save(s"$tempDir/${UUID.randomUUID()}") } - assert(ex.getErrorClass == "TASK_WRITE_FAILED") + assert(ex.getCondition == "TASK_WRITE_FAILED") assert(ex.getCause.isInstanceOf[java.lang.NullPointerException]) assert(ex.getCause.getMessage.contains( "null value for (non-nullable) string at test_schema.Name")) @@ -1673,8 +1673,12 @@ abstract class AvroSuite exception = intercept[AnalysisException] { sql("select interval 1 days").write.format("avro").mode("overwrite").save(tempDir) }, - condition = "_LEGACY_ERROR_TEMP_1136", - parameters = Map.empty + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + parameters = Map( + "format" -> "Avro", + "columnName" -> "`INTERVAL '1 days'`", + "columnType" -> "\"INTERVAL\"" + ) ) checkError( exception = intercept[AnalysisException] { @@ -2629,7 +2633,7 @@ abstract class AvroSuite val e = intercept[SparkException] { df.write.format("avro").option("avroSchema", avroSchema).save(path3_x) } - assert(e.getErrorClass == "TASK_WRITE_FAILED") + assert(e.getCondition == "TASK_WRITE_FAILED") assert(e.getCause.isInstanceOf[SparkUpgradeException]) } checkDefaultLegacyRead(oldPath) @@ -2884,7 +2888,7 @@ abstract class AvroSuite val e = intercept[SparkException] { df.write.format("avro").option("avroSchema", avroSchema).save(dir.getCanonicalPath) } - assert(e.getErrorClass == "TASK_WRITE_FAILED") + assert(e.getCondition == "TASK_WRITE_FAILED") val errMsg = e.getCause.asInstanceOf[SparkUpgradeException].getMessage assert(errMsg.contains("You may get a different result due to the upgrading")) } @@ -2895,7 +2899,7 @@ abstract class AvroSuite val e = intercept[SparkException] { df.write.format("avro").save(dir.getCanonicalPath) } - assert(e.getErrorClass == "TASK_WRITE_FAILED") + assert(e.getCondition == "TASK_WRITE_FAILED") val errMsg = e.getCause.asInstanceOf[SparkUpgradeException].getMessage assert(errMsg.contains("You may get a different result due to the upgrading")) } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CatalogSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CatalogSuite.scala index 0e3a683d2701d..ce552bdd4f0f0 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CatalogSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CatalogSuite.scala @@ -69,7 +69,7 @@ class CatalogSuite extends ConnectFunSuite with RemoteSparkSession with SQLHelpe val exception = intercept[SparkException] { spark.catalog.setCurrentCatalog("notExists") } - assert(exception.getErrorClass == "CATALOG_NOT_FOUND") + assert(exception.getCondition == "CATALOG_NOT_FOUND") spark.catalog.setCurrentCatalog("testcat") assert(spark.catalog.currentCatalog().equals("testcat")) val catalogsAfterChange = spark.catalog.listCatalogs().collect() diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientDataFrameStatSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientDataFrameStatSuite.scala index 88281352f2479..84ed624a95214 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientDataFrameStatSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientDataFrameStatSuite.scala @@ -251,16 +251,16 @@ class ClientDataFrameStatSuite extends ConnectFunSuite with RemoteSparkSession { val error1 = intercept[AnalysisException] { df.stat.bloomFilter("id", -1000, 100) } - assert(error1.getErrorClass === "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE") + assert(error1.getCondition === "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE") val error2 = intercept[AnalysisException] { df.stat.bloomFilter("id", 1000, -100) } - assert(error2.getErrorClass === "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE") + assert(error2.getCondition === "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE") val error3 = intercept[AnalysisException] { df.stat.bloomFilter("id", 1000, -1.0) } - assert(error3.getErrorClass === "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE") + assert(error3.getCondition === "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE") } } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala index b47231948dc98..0371981b728d1 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala @@ -95,7 +95,7 @@ class ClientE2ETestSuite .collect() } assert( - ex.getErrorClass === + ex.getCondition === "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER") assert( ex.getMessageParameters.asScala == Map( @@ -122,12 +122,12 @@ class ClientE2ETestSuite Seq("1").toDS().withColumn("udf_val", throwException($"value")).collect() } - assert(ex.getErrorClass != null) + assert(ex.getCondition != null) assert(!ex.getMessageParameters.isEmpty) assert(ex.getCause.isInstanceOf[SparkException]) val cause = ex.getCause.asInstanceOf[SparkException] - assert(cause.getErrorClass == null) + assert(cause.getCondition == null) assert(cause.getMessageParameters.isEmpty) assert(cause.getMessage.contains("test" * 10000)) } @@ -141,7 +141,7 @@ class ClientE2ETestSuite val ex = intercept[AnalysisException] { spark.sql("select x").collect() } - assert(ex.getErrorClass != null) + assert(ex.getCondition != null) assert(!ex.messageParameters.isEmpty) assert(ex.getSqlState != null) assert(!ex.isInternalError) @@ -169,14 +169,14 @@ class ClientE2ETestSuite val ex = intercept[NoSuchNamespaceException] { spark.sql("use database123") } - assert(ex.getErrorClass != null) + assert(ex.getCondition != null) } test("table not found for spark.catalog.getTable") { val ex = intercept[AnalysisException] { spark.catalog.getTable("test_table") } - assert(ex.getErrorClass != null) + assert(ex.getCondition != null) } test("throw NamespaceAlreadyExistsException") { @@ -185,7 +185,7 @@ class ClientE2ETestSuite val ex = intercept[NamespaceAlreadyExistsException] { spark.sql("create database test_db") } - assert(ex.getErrorClass != null) + assert(ex.getCondition != null) } finally { spark.sql("drop database test_db") } @@ -197,7 +197,7 @@ class ClientE2ETestSuite val ex = intercept[TempTableAlreadyExistsException] { spark.sql("create temporary view test_view as select 1") } - assert(ex.getErrorClass != null) + assert(ex.getCondition != null) } finally { spark.sql("drop view test_view") } @@ -209,7 +209,7 @@ class ClientE2ETestSuite val ex = intercept[TableAlreadyExistsException] { spark.sql(s"create table testcat.test_table (id int)") } - assert(ex.getErrorClass != null) + assert(ex.getCondition != null) } } @@ -217,7 +217,7 @@ class ClientE2ETestSuite val ex = intercept[ParseException] { spark.sql("selet 1").collect() } - assert(ex.getErrorClass != null) + assert(ex.getCondition != null) assert(!ex.messageParameters.isEmpty) assert(ex.getSqlState != null) assert(!ex.isInternalError) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala index 46aeaeff43d2f..ac56600392aa3 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala @@ -224,7 +224,7 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach { val error = constructor(testParams).asInstanceOf[Throwable with SparkThrowable] assert(error.getMessage.contains(testParams.message)) assert(error.getCause == null) - assert(error.getErrorClass == testParams.errorClass.get) + assert(error.getCondition == testParams.errorClass.get) assert(error.getMessageParameters.asScala == testParams.messageParameters) assert(error.getQueryContext.isEmpty) } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala index 7176c582d0bbc..10e4c11c406fe 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala @@ -783,7 +783,7 @@ class ArrowEncoderSuite extends ConnectFunSuite with BeforeAndAfterAll { Iterator.tabulate(10)(i => (i, "itr_" + i)) } } - assert(e.getErrorClass == "CANNOT_USE_KRYO") + assert(e.getCondition == "CANNOT_USE_KRYO") } test("transforming encoder") { diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/ClientStreamingQuerySuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/ClientStreamingQuerySuite.scala index 27b1ee014a719..b1a7d81916e92 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/ClientStreamingQuerySuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/ClientStreamingQuerySuite.scala @@ -331,7 +331,7 @@ class ClientStreamingQuerySuite extends QueryTest with RemoteSparkSession with L query.awaitTermination() } - assert(exception.getErrorClass != null) + assert(exception.getCondition != null) assert(exception.getMessageParameters().get("id") == query.id.toString) assert(exception.getMessageParameters().get("runId") == query.runId.toString) assert(exception.getCause.isInstanceOf[SparkException]) @@ -369,7 +369,7 @@ class ClientStreamingQuerySuite extends QueryTest with RemoteSparkSession with L spark.streams.awaitAnyTermination() } - assert(exception.getErrorClass != null) + assert(exception.getCondition != null) assert(exception.getMessageParameters().get("id") == query.id.toString) assert(exception.getMessageParameters().get("runId") == query.runId.toString) assert(exception.getCause.isInstanceOf[SparkException]) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MsSqlServerIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MsSqlServerIntegrationSuite.scala index e5fd453cb057c..aaaaa28558342 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MsSqlServerIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MsSqlServerIntegrationSuite.scala @@ -115,7 +115,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JD exception = intercept[SparkSQLFeatureNotSupportedException] { sql(s"ALTER TABLE $tbl ALTER COLUMN ID DROP NOT NULL") }, - condition = "_LEGACY_ERROR_TEMP_2271") + condition = "UNSUPPORTED_FEATURE.UPDATE_COLUMN_NULLABILITY") } test("SPARK-47440: SQLServer does not support boolean expression in binary comparison") { diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLIntegrationSuite.scala index 700c05b54a256..a895739254373 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLIntegrationSuite.scala @@ -142,7 +142,7 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest exception = intercept[SparkSQLFeatureNotSupportedException] { sql(s"ALTER TABLE $tbl ALTER COLUMN ID DROP NOT NULL") }, - condition = "_LEGACY_ERROR_TEMP_2271") + condition = "UNSUPPORTED_FEATURE.UPDATE_COLUMN_NULLABILITY") } override def testCreateTableWithProperty(tbl: String): Unit = { diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala index 6bb415a928837..f70b500f974a4 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.jdbc.v2 import java.sql.Connection -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkSQLException} import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCTableCatalog @@ -65,6 +65,112 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCT |) """.stripMargin ).executeUpdate() + + connection.prepareStatement("CREATE TABLE array_test_table (int_array int[]," + + "float_array FLOAT8[], timestamp_array TIMESTAMP[], string_array TEXT[]," + + "datetime_array TIMESTAMPTZ[], array_of_int_arrays INT[][])").executeUpdate() + + val query = + """ + INSERT INTO array_test_table + (int_array, float_array, timestamp_array, string_array, + datetime_array, array_of_int_arrays) + VALUES + ( + ARRAY[1, 2, 3], -- Array of integers + ARRAY[1.1, 2.2, 3.3], -- Array of floats + ARRAY['2023-01-01 12:00'::timestamp, '2023-06-01 08:30'::timestamp], + ARRAY['hello', 'world'], -- Array of strings + ARRAY['2023-10-04 12:00:00+00'::timestamptz, + '2023-12-01 14:15:00+00'::timestamptz], + ARRAY[ARRAY[1, 2]] -- Array of arrays of integers + ), + ( + ARRAY[10, 20, 30], -- Another set of data + ARRAY[10.5, 20.5, 30.5], + ARRAY['2022-01-01 09:15'::timestamp, '2022-03-15 07:45'::timestamp], + ARRAY['postgres', 'arrays'], + ARRAY['2022-11-22 09:00:00+00'::timestamptz, + '2022-12-31 23:59:59+00'::timestamptz], + ARRAY[ARRAY[10, 20]] + ); + """ + connection.prepareStatement(query).executeUpdate() + + connection.prepareStatement("CREATE TABLE array_int (col int[])").executeUpdate() + connection.prepareStatement("CREATE TABLE array_bigint(col bigint[])").executeUpdate() + connection.prepareStatement("CREATE TABLE array_smallint (col smallint[])").executeUpdate() + connection.prepareStatement("CREATE TABLE array_boolean (col boolean[])").executeUpdate() + connection.prepareStatement("CREATE TABLE array_float (col real[])").executeUpdate() + connection.prepareStatement("CREATE TABLE array_double (col float8[])").executeUpdate() + connection.prepareStatement("CREATE TABLE array_timestamp (col timestamp[])").executeUpdate() + connection.prepareStatement("CREATE TABLE array_timestamptz (col timestamptz[])") + .executeUpdate() + + connection.prepareStatement("INSERT INTO array_int VALUES (array[array[10]])").executeUpdate() + connection.prepareStatement("INSERT INTO array_bigint VALUES (array[array[10]])") + .executeUpdate() + connection.prepareStatement("INSERT INTO array_smallint VALUES (array[array[10]])") + .executeUpdate() + connection.prepareStatement("INSERT INTO array_boolean VALUES (array[array[true]])") + .executeUpdate() + connection.prepareStatement("INSERT INTO array_float VALUES (array[array[10.5]])") + .executeUpdate() + connection.prepareStatement("INSERT INTO array_double VALUES (array[array[10.1]])") + .executeUpdate() + connection.prepareStatement("INSERT INTO array_timestamp VALUES (" + + "array[array['2022-01-01 09:15'::timestamp]])").executeUpdate() + connection.prepareStatement("INSERT INTO array_timestamptz VALUES " + + "(array[array['2022-01-01 09:15'::timestamptz]])").executeUpdate() + connection.prepareStatement( + "CREATE TABLE datetime (name VARCHAR(32), date1 DATE, time1 TIMESTAMP)") + .executeUpdate() + } + + test("Test multi-dimensional column types") { + // This test is used to verify that the multi-dimensional + // column types are supported by the JDBC V2 data source. + // We do not verify any result output + // + val df = spark.read.format("jdbc") + .option("url", jdbcUrl) + .option("dbtable", "array_test_table") + .load() + df.collect() + + val array_tables = Array( + ("array_int", "\"ARRAY\""), + ("array_bigint", "\"ARRAY\""), + ("array_smallint", "\"ARRAY\""), + ("array_boolean", "\"ARRAY\""), + ("array_float", "\"ARRAY\""), + ("array_double", "\"ARRAY\""), + ("array_timestamp", "\"ARRAY\""), + ("array_timestamptz", "\"ARRAY\"") + ) + + array_tables.foreach { case (dbtable, arrayType) => + checkError( + exception = intercept[SparkSQLException] { + val df = spark.read.format("jdbc") + .option("url", jdbcUrl) + .option("dbtable", dbtable) + .load() + df.collect() + }, + condition = "COLUMN_ARRAY_ELEMENT_TYPE_MISMATCH", + parameters = Map("pos" -> "0", "type" -> arrayType), + sqlState = Some("0A000") + ) + } + } + + override def dataPreparation(connection: Connection): Unit = { + super.dataPreparation(connection) + connection.prepareStatement("INSERT INTO datetime VALUES " + + "('amy', '2022-05-19', '2022-05-19 00:00:00')").executeUpdate() + connection.prepareStatement("INSERT INTO datetime VALUES " + + "('alex', '2022-05-18', '2022-05-18 00:00:00')").executeUpdate() } override def testUpdateColumnType(tbl: String): Unit = { @@ -123,4 +229,77 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCT ) } } + + override def testDatetime(tbl: String): Unit = { + val df1 = sql(s"SELECT name FROM $tbl WHERE " + + "dayofyear(date1) > 100 AND dayofmonth(date1) > 10 ") + checkFilterPushed(df1) + val rows1 = df1.collect() + assert(rows1.length === 2) + assert(rows1(0).getString(0) === "amy") + assert(rows1(1).getString(0) === "alex") + + val df2 = sql(s"SELECT name FROM $tbl WHERE year(date1) = 2022 AND quarter(date1) = 2") + checkFilterPushed(df2) + val rows2 = df2.collect() + assert(rows2.length === 2) + assert(rows2(0).getString(0) === "amy") + assert(rows2(1).getString(0) === "alex") + + val df3 = sql(s"SELECT name FROM $tbl WHERE second(time1) = 0 AND month(date1) = 5") + checkFilterPushed(df3) + val rows3 = df3.collect() + assert(rows3.length === 2) + assert(rows3(0).getString(0) === "amy") + assert(rows3(1).getString(0) === "alex") + + val df4 = sql(s"SELECT name FROM $tbl WHERE hour(time1) = 0 AND minute(time1) = 0") + checkFilterPushed(df4) + val rows4 = df4.collect() + assert(rows4.length === 2) + assert(rows4(0).getString(0) === "amy") + assert(rows4(1).getString(0) === "alex") + + val df5 = sql(s"SELECT name FROM $tbl WHERE " + + "extract(WEEk from date1) > 10 AND extract(YEAROFWEEK from date1) = 2022") + checkFilterPushed(df5) + val rows5 = df5.collect() + assert(rows5.length === 2) + assert(rows5(0).getString(0) === "amy") + assert(rows5(1).getString(0) === "alex") + + val df6 = sql(s"SELECT name FROM $tbl WHERE date_add(date1, 1) = date'2022-05-20' " + + "AND datediff(date1, '2022-05-10') > 0") + checkFilterPushed(df6, false) + val rows6 = df6.collect() + assert(rows6.length === 1) + assert(rows6(0).getString(0) === "amy") + + val df7 = sql(s"SELECT name FROM $tbl WHERE weekday(date1) = 2") + checkFilterPushed(df7) + val rows7 = df7.collect() + assert(rows7.length === 1) + assert(rows7(0).getString(0) === "alex") + + val df8 = sql(s"SELECT name FROM $tbl WHERE dayofweek(date1) = 4") + checkFilterPushed(df8) + val rows8 = df8.collect() + assert(rows8.length === 1) + assert(rows8(0).getString(0) === "alex") + + val df9 = sql(s"SELECT name FROM $tbl WHERE " + + "dayofyear(date1) > 100 order by dayofyear(date1) limit 1") + checkFilterPushed(df9) + val rows9 = df9.collect() + assert(rows9.length === 1) + assert(rows9(0).getString(0) === "alex") + + // Postgres does not support + val df10 = sql(s"SELECT name FROM $tbl WHERE trunc(date1, 'week') = date'2022-05-16'") + checkFilterPushed(df10, false) + val rows10 = df10.collect() + assert(rows10.length === 2) + assert(rows10(0).getString(0) === "amy") + assert(rows10(1).getString(0) === "alex") + } } diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaExceptions.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaExceptions.scala index 13a68e72269f0..c4adb6b3f26e1 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaExceptions.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaExceptions.scala @@ -184,5 +184,5 @@ private[kafka010] class KafkaIllegalStateException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass } diff --git a/core/src/main/java/org/apache/spark/memory/SparkOutOfMemoryError.java b/core/src/main/java/org/apache/spark/memory/SparkOutOfMemoryError.java index 8ec5c2221b6e9..0e35ebecfd270 100644 --- a/core/src/main/java/org/apache/spark/memory/SparkOutOfMemoryError.java +++ b/core/src/main/java/org/apache/spark/memory/SparkOutOfMemoryError.java @@ -32,14 +32,6 @@ public final class SparkOutOfMemoryError extends OutOfMemoryError implements Spa String errorClass; Map messageParameters; - public SparkOutOfMemoryError(String s) { - super(s); - } - - public SparkOutOfMemoryError(OutOfMemoryError e) { - super(e.getMessage()); - } - public SparkOutOfMemoryError(String errorClass, Map messageParameters) { super(SparkThrowableHelper.getMessage(errorClass, messageParameters)); this.errorClass = errorClass; @@ -52,7 +44,7 @@ public Map getMessageParameters() { } @Override - public String getErrorClass() { + public String getCondition() { return errorClass; } } diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index df224bc902bff..bd9f58bf7415f 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -21,13 +21,7 @@ import java.io.InterruptedIOException; import java.io.IOException; import java.nio.channels.ClosedByInterruptException; -import java.util.Arrays; -import java.util.ArrayList; -import java.util.BitSet; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; +import java.util.*; import com.google.common.annotations.VisibleForTesting; @@ -291,8 +285,12 @@ private long trySpillAndAcquire( logger.error("error while calling spill() on {}", e, MDC.of(LogKeys.MEMORY_CONSUMER$.MODULE$, consumerToSpill)); // checkstyle.off: RegexpSinglelineJava - throw new SparkOutOfMemoryError("error while calling spill() on " + consumerToSpill + " : " - + e.getMessage()); + throw new SparkOutOfMemoryError( + "_LEGACY_ERROR_TEMP_3300", + new HashMap() {{ + put("consumerToSpill", consumerToSpill.toString()); + put("message", e.getMessage()); + }}); // checkstyle.on: RegexpSinglelineJava } } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index 7579c0aefb250..761ced66f78cf 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -18,6 +18,7 @@ package org.apache.spark.util.collection.unsafe.sort; import java.util.Comparator; +import java.util.HashMap; import java.util.LinkedList; import javax.annotation.Nullable; @@ -215,7 +216,7 @@ public void expandPointerArray(LongArray newArray) { if (array != null) { if (newArray.size() < array.size()) { // checkstyle.off: RegexpSinglelineJava - throw new SparkOutOfMemoryError("Not enough memory to grow pointer array"); + throw new SparkOutOfMemoryError("_LEGACY_ERROR_TEMP_3301", new HashMap()); // checkstyle.on: RegexpSinglelineJava } Platform.copyMemory( diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 3b7c7778e26ce..573608c4327e0 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -173,7 +173,7 @@ class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( } private def canShuffleMergeBeEnabled(): Boolean = { - val isPushShuffleEnabled = Utils.isPushBasedShuffleEnabled(rdd.sparkContext.getConf, + val isPushShuffleEnabled = Utils.isPushBasedShuffleEnabled(rdd.sparkContext.conf, // invoked at driver isDriver = true) if (isPushShuffleEnabled && rdd.isBarrier()) { diff --git a/core/src/main/scala/org/apache/spark/SparkFileAlreadyExistsException.scala b/core/src/main/scala/org/apache/spark/SparkFileAlreadyExistsException.scala index 0e578f045452e..82a0261f32ae7 100644 --- a/core/src/main/scala/org/apache/spark/SparkFileAlreadyExistsException.scala +++ b/core/src/main/scala/org/apache/spark/SparkFileAlreadyExistsException.scala @@ -33,5 +33,5 @@ private[spark] class SparkFileAlreadyExistsException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass } diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala index f0d6cba6ae734..3c3017a9a64c1 100644 --- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala +++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala @@ -45,8 +45,8 @@ private[spark] abstract class StreamFileInputFormat[T] * which is set through setMaxSplitSize */ def setMinPartitions(sc: SparkContext, context: JobContext, minPartitions: Int): Unit = { - val defaultMaxSplitBytes = sc.getConf.get(config.FILES_MAX_PARTITION_BYTES) - val openCostInBytes = sc.getConf.get(config.FILES_OPEN_COST_IN_BYTES) + val defaultMaxSplitBytes = sc.conf.get(config.FILES_MAX_PARTITION_BYTES) + val openCostInBytes = sc.conf.get(config.FILES_OPEN_COST_IN_BYTES) val defaultParallelism = Math.max(sc.defaultParallelism, minPartitions) val files = listStatus(context).asScala val totalBytes = files.filterNot(_.isDirectory).map(_.getLen + openCostInBytes).sum diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 2c89fe7885d08..4f7338f74e298 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -174,7 +174,7 @@ private[spark] class DAGScheduler( // `NUM_CANCELLED_JOB_GROUPS_TO_TRACK` stored. On a new job submission, if its job group is in // this set, the job will be immediately cancelled. private[scheduler] val cancelledJobGroups = - new LimitedSizeFIFOSet[String](sc.getConf.get(config.NUM_CANCELLED_JOB_GROUPS_TO_TRACK)) + new LimitedSizeFIFOSet[String](sc.conf.get(config.NUM_CANCELLED_JOB_GROUPS_TO_TRACK)) /** * Contains the locations that each RDD's partitions are cached on. This map's keys are RDD ids @@ -224,9 +224,9 @@ private[spark] class DAGScheduler( private val closureSerializer = SparkEnv.get.closureSerializer.newInstance() /** If enabled, FetchFailed will not cause stage retry, in order to surface the problem. */ - private val disallowStageRetryForTest = sc.getConf.get(TEST_NO_STAGE_RETRY) + private val disallowStageRetryForTest = sc.conf.get(TEST_NO_STAGE_RETRY) - private val shouldMergeResourceProfiles = sc.getConf.get(config.RESOURCE_PROFILE_MERGE_CONFLICTS) + private val shouldMergeResourceProfiles = sc.conf.get(config.RESOURCE_PROFILE_MERGE_CONFLICTS) /** * Whether to unregister all the outputs on the host in condition that we receive a FetchFailure, @@ -234,19 +234,19 @@ private[spark] class DAGScheduler( * executor(instead of the host) on a FetchFailure. */ private[scheduler] val unRegisterOutputOnHostOnFetchFailure = - sc.getConf.get(config.UNREGISTER_OUTPUT_ON_HOST_ON_FETCH_FAILURE) + sc.conf.get(config.UNREGISTER_OUTPUT_ON_HOST_ON_FETCH_FAILURE) /** * Number of consecutive stage attempts allowed before a stage is aborted. */ private[scheduler] val maxConsecutiveStageAttempts = - sc.getConf.get(config.STAGE_MAX_CONSECUTIVE_ATTEMPTS) + sc.conf.get(config.STAGE_MAX_CONSECUTIVE_ATTEMPTS) /** * Max stage attempts allowed before a stage is aborted. */ private[scheduler] val maxStageAttempts: Int = { - Math.max(maxConsecutiveStageAttempts, sc.getConf.get(config.STAGE_MAX_ATTEMPTS)) + Math.max(maxConsecutiveStageAttempts, sc.conf.get(config.STAGE_MAX_ATTEMPTS)) } /** @@ -254,7 +254,7 @@ private[spark] class DAGScheduler( * count spark.stage.maxConsecutiveAttempts */ private[scheduler] val ignoreDecommissionFetchFailure = - sc.getConf.get(config.STAGE_IGNORE_DECOMMISSION_FETCH_FAILURE) + sc.conf.get(config.STAGE_IGNORE_DECOMMISSION_FETCH_FAILURE) /** * Number of max concurrent tasks check failures for each barrier job. @@ -264,14 +264,14 @@ private[spark] class DAGScheduler( /** * Time in seconds to wait between a max concurrent tasks check failure and the next check. */ - private val timeIntervalNumTasksCheck = sc.getConf + private val timeIntervalNumTasksCheck = sc.conf .get(config.BARRIER_MAX_CONCURRENT_TASKS_CHECK_INTERVAL) /** * Max number of max concurrent tasks check failures allowed for a job before fail the job * submission. */ - private val maxFailureNumTasksCheck = sc.getConf + private val maxFailureNumTasksCheck = sc.conf .get(config.BARRIER_MAX_CONCURRENT_TASKS_CHECK_MAX_FAILURES) private val messageScheduler = @@ -286,26 +286,26 @@ private[spark] class DAGScheduler( taskScheduler.setDAGScheduler(this) - private val pushBasedShuffleEnabled = Utils.isPushBasedShuffleEnabled(sc.getConf, isDriver = true) + private val pushBasedShuffleEnabled = Utils.isPushBasedShuffleEnabled(sc.conf, isDriver = true) private val blockManagerMasterDriverHeartbeatTimeout = - sc.getConf.get(config.STORAGE_BLOCKMANAGER_MASTER_DRIVER_HEARTBEAT_TIMEOUT).millis + sc.conf.get(config.STORAGE_BLOCKMANAGER_MASTER_DRIVER_HEARTBEAT_TIMEOUT).millis private val shuffleMergeResultsTimeoutSec = - sc.getConf.get(config.PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT) + sc.conf.get(config.PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT) private val shuffleMergeFinalizeWaitSec = - sc.getConf.get(config.PUSH_BASED_SHUFFLE_MERGE_FINALIZE_TIMEOUT) + sc.conf.get(config.PUSH_BASED_SHUFFLE_MERGE_FINALIZE_TIMEOUT) private val shuffleMergeWaitMinSizeThreshold = - sc.getConf.get(config.PUSH_BASED_SHUFFLE_SIZE_MIN_SHUFFLE_SIZE_TO_WAIT) + sc.conf.get(config.PUSH_BASED_SHUFFLE_SIZE_MIN_SHUFFLE_SIZE_TO_WAIT) - private val shufflePushMinRatio = sc.getConf.get(config.PUSH_BASED_SHUFFLE_MIN_PUSH_RATIO) + private val shufflePushMinRatio = sc.conf.get(config.PUSH_BASED_SHUFFLE_MIN_PUSH_RATIO) private val shuffleMergeFinalizeNumThreads = - sc.getConf.get(config.PUSH_BASED_SHUFFLE_MERGE_FINALIZE_THREADS) + sc.conf.get(config.PUSH_BASED_SHUFFLE_MERGE_FINALIZE_THREADS) - private val shuffleFinalizeRpcThreads = sc.getConf.get(config.PUSH_SHUFFLE_FINALIZE_RPC_THREADS) + private val shuffleFinalizeRpcThreads = sc.conf.get(config.PUSH_SHUFFLE_FINALIZE_RPC_THREADS) // Since SparkEnv gets initialized after DAGScheduler, externalShuffleClient needs to be // initialized lazily @@ -328,11 +328,10 @@ private[spark] class DAGScheduler( ThreadUtils.newDaemonFixedThreadPool(shuffleFinalizeRpcThreads, "shuffle-merge-finalize-rpc") /** Whether rdd cache visibility tracking is enabled. */ - private val trackingCacheVisibility: Boolean = - sc.getConf.get(RDD_CACHE_VISIBILITY_TRACKING_ENABLED) + private val trackingCacheVisibility: Boolean = sc.conf.get(RDD_CACHE_VISIBILITY_TRACKING_ENABLED) /** Whether to abort a stage after canceling all of its tasks. */ - private val legacyAbortStageAfterKillTasks = sc.getConf.get(LEGACY_ABORT_STAGE_AFTER_KILL_TASKS) + private val legacyAbortStageAfterKillTasks = sc.conf.get(LEGACY_ABORT_STAGE_AFTER_KILL_TASKS) /** * Called by the TaskSetManager to report task's starting. @@ -557,7 +556,7 @@ private[spark] class DAGScheduler( * TODO SPARK-24942 Improve cluster resource management with jobs containing barrier stage */ private def checkBarrierStageWithDynamicAllocation(rdd: RDD[_]): Unit = { - if (rdd.isBarrier() && Utils.isDynamicAllocationEnabled(sc.getConf)) { + if (rdd.isBarrier() && Utils.isDynamicAllocationEnabled(sc.conf)) { throw SparkCoreErrors.barrierStageWithDynamicAllocationError() } } @@ -2163,7 +2162,7 @@ private[spark] class DAGScheduler( case mapStage: ShuffleMapStage => val numMissingPartitions = mapStage.findMissingPartitions().length if (numMissingPartitions < mapStage.numTasks) { - if (sc.getConf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) { + if (sc.conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) { val reason = "A shuffle map stage with indeterminate output was failed " + "and retried. However, Spark can only do this while using the new " + "shuffle block fetching protocol. Please check the config " + @@ -2893,8 +2892,8 @@ private[spark] class DAGScheduler( val finalException = exception.collect { // If the error is user-facing (defines error class and is not internal error), we don't // wrap it with "Job aborted" and expose this error to the end users directly. - case st: Exception with SparkThrowable if st.getErrorClass != null && - !SparkThrowableHelper.isInternalError(st.getErrorClass) => + case st: Exception with SparkThrowable if st.getCondition != null && + !SparkThrowableHelper.isInternalError(st.getCondition) => st }.getOrElse { new SparkException(s"Job aborted due to stage failure: $reason", cause = exception.orNull) diff --git a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala index 7a2b7d9caec42..fc7a4675429aa 100644 --- a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala +++ b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala @@ -35,7 +35,7 @@ private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging { // Carriage return private val CR = '\r' // Update period of progress bar, in milliseconds - private val updatePeriodMSec = sc.getConf.get(UI_CONSOLE_PROGRESS_UPDATE_INTERVAL) + private val updatePeriodMSec = sc.conf.get(UI_CONSOLE_PROGRESS_UPDATE_INTERVAL) // Delay to show up a progress bar, in milliseconds private val firstDelayMSec = 500L diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 380231ce97c0b..ca51e61f5ed44 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -288,7 +288,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft sem.acquire(1) sc.cancelJobGroupAndFutureJobs(s"job-group-$idx") ThreadUtils.awaitReady(job, Duration.Inf).failed.foreach { case e: SparkException => - assert(e.getErrorClass == "SPARK_JOB_CANCELLED") + assert(e.getCondition == "SPARK_JOB_CANCELLED") } } // submit a job with the 0 job group that was evicted from cancelledJobGroups set, it should run diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index 9f310c06ac5ae..e38efc27b78f9 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -343,7 +343,7 @@ abstract class SparkFunSuite parameters: Map[String, String] = Map.empty, matchPVals: Boolean = false, queryContext: Array[ExpectedContext] = Array.empty): Unit = { - assert(exception.getErrorClass === condition) + assert(exception.getCondition === condition) sqlState.foreach(state => assert(exception.getSqlState === state)) val expectedParameters = exception.getMessageParameters.asScala if (matchPVals) { diff --git a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala index 946ea75686e32..ea845c0f93a4b 100644 --- a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala @@ -47,7 +47,7 @@ class SparkThrowableSuite extends SparkFunSuite { }}} */ private val regenerateCommand = "SPARK_GENERATE_GOLDEN_FILES=1 build/sbt " + - "\"core/testOnly *SparkThrowableSuite -- -t \\\"Error classes match with document\\\"\"" + "\"core/testOnly *SparkThrowableSuite -- -t \\\"Error conditions are correctly formatted\\\"\"" private val errorJsonFilePath = getWorkspaceFilePath( "common", "utils", "src", "main", "resources", "error", "error-conditions.json") @@ -199,7 +199,7 @@ class SparkThrowableSuite extends SparkFunSuite { val e = intercept[SparkException] { getMessage("UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", Map.empty[String, String]) } - assert(e.getErrorClass === "INTERNAL_ERROR") + assert(e.getCondition === "INTERNAL_ERROR") assert(e.getMessageParameters().get("message").contains("Undefined error message parameter")) } @@ -245,7 +245,7 @@ class SparkThrowableSuite extends SparkFunSuite { throw new SparkException("Arbitrary legacy message") } catch { case e: SparkThrowable => - assert(e.getErrorClass == null) + assert(e.getCondition == null) assert(!e.isInternalError) assert(e.getSqlState == null) case _: Throwable => @@ -262,7 +262,7 @@ class SparkThrowableSuite extends SparkFunSuite { cause = null) } catch { case e: SparkThrowable => - assert(e.getErrorClass == "CANNOT_PARSE_DECIMAL") + assert(e.getCondition == "CANNOT_PARSE_DECIMAL") assert(!e.isInternalError) assert(e.getSqlState == "22018") case _: Throwable => @@ -357,7 +357,7 @@ class SparkThrowableSuite extends SparkFunSuite { |}""".stripMargin) // Legacy mode when an exception does not have any error class class LegacyException extends Throwable with SparkThrowable { - override def getErrorClass: String = null + override def getCondition: String = null override def getMessage: String = "Test message" } val e3 = new LegacyException @@ -452,7 +452,7 @@ class SparkThrowableSuite extends SparkFunSuite { val e = intercept[SparkException] { new ErrorClassesJsonReader(Seq(errorJsonFilePath.toUri.toURL, json.toURI.toURL)) } - assert(e.getErrorClass === "INTERNAL_ERROR") + assert(e.getCondition === "INTERNAL_ERROR") assert(e.getMessage.contains("DIVIDE.BY_ZERO")) } @@ -478,7 +478,7 @@ class SparkThrowableSuite extends SparkFunSuite { val e = intercept[SparkException] { new ErrorClassesJsonReader(Seq(errorJsonFilePath.toUri.toURL, json.toURI.toURL)) } - assert(e.getErrorClass === "INTERNAL_ERROR") + assert(e.getCondition === "INTERNAL_ERROR") assert(e.getMessage.contains("BY.ZERO")) } } diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index 1efef3383b821..b0f36b9744fa8 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -317,13 +317,13 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio // Instead, crash the driver by directly accessing the broadcast value. val e1 = intercept[SparkException] { broadcast.value } assert(e1.isInternalError) - assert(e1.getErrorClass == "INTERNAL_ERROR_BROADCAST") + assert(e1.getCondition == "INTERNAL_ERROR_BROADCAST") val e2 = intercept[SparkException] { broadcast.unpersist(blocking = true) } assert(e2.isInternalError) - assert(e2.getErrorClass == "INTERNAL_ERROR_BROADCAST") + assert(e2.getCondition == "INTERNAL_ERROR_BROADCAST") val e3 = intercept[SparkException] { broadcast.destroy(blocking = true) } assert(e3.isInternalError) - assert(e3.getErrorClass == "INTERNAL_ERROR_BROADCAST") + assert(e3.getCondition == "INTERNAL_ERROR_BROADCAST") } else { val results = sc.parallelize(1 to partitions, partitions).map(x => (x, broadcast.value.sum)) assert(results.collect().toSet === (1 to partitions).map(x => (x, list.sum)).toSet) @@ -339,7 +339,7 @@ package object testPackage extends Assertions { val thrown = intercept[SparkException] { broadcast.value } assert(thrown.getMessage.contains("BroadcastSuite.scala")) assert(thrown.isInternalError) - assert(thrown.getErrorClass == "INTERNAL_ERROR_BROADCAST") + assert(thrown.getCondition == "INTERNAL_ERROR_BROADCAST") } } diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 805e7ca467497..fa13092dc47aa 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -21,7 +21,7 @@ import java.io.{Externalizable, ObjectInput, ObjectOutput} import java.lang.Thread.UncaughtExceptionHandler import java.net.URL import java.nio.ByteBuffer -import java.util.Properties +import java.util.{HashMap, Properties} import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.concurrent.atomic.AtomicBoolean @@ -522,7 +522,13 @@ class ExecutorSuite extends SparkFunSuite testThrowable(new OutOfMemoryError(), depthToCheck, isFatal = true) testThrowable(new InterruptedException(), depthToCheck, isFatal = false) testThrowable(new RuntimeException("test"), depthToCheck, isFatal = false) - testThrowable(new SparkOutOfMemoryError("test"), depthToCheck, isFatal = false) + testThrowable( + new SparkOutOfMemoryError( + "_LEGACY_ERROR_USER_RAISED_EXCEPTION", + new HashMap[String, String]() { + put("errorMessage", "test") + }), + depthToCheck, isFatal = false) } // Verify we can handle the cycle in the exception chain diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 8ba2f6c414cb9..91e84b0780798 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -274,7 +274,7 @@ tink/1.15.0//tink-1.15.0.jar transaction-api/1.1//transaction-api-1.1.jar univocity-parsers/2.9.1//univocity-parsers-2.9.1.jar wildfly-openssl/1.1.3.Final//wildfly-openssl-1.1.3.Final.jar -xbean-asm9-shaded/4.25//xbean-asm9-shaded-4.25.jar +xbean-asm9-shaded/4.26//xbean-asm9-shaded-4.26.jar xmlschema-core/2.3.1//xmlschema-core-2.3.1.jar xz/1.10//xz-1.10.jar zjsonpatch/0.3.0//zjsonpatch-0.3.0.jar diff --git a/dev/infra/Dockerfile b/dev/infra/Dockerfile index 1619b009e9364..10a39497c8ed9 100644 --- a/dev/infra/Dockerfile +++ b/dev/infra/Dockerfile @@ -24,7 +24,7 @@ LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image" # Overwrite this label to avoid exposing the underlying Ubuntu OS version label LABEL org.opencontainers.image.version="" -ENV FULL_REFRESH_DATE 20241002 +ENV FULL_REFRESH_DATE 20241007 ENV DEBIAN_FRONTEND noninteractive ENV DEBCONF_NONINTERACTIVE_SEEN true @@ -91,10 +91,10 @@ RUN mkdir -p /usr/local/pypy/pypy3.9 && \ ln -sf /usr/local/pypy/pypy3.9/bin/pypy /usr/local/bin/pypy3.9 && \ ln -sf /usr/local/pypy/pypy3.9/bin/pypy /usr/local/bin/pypy3 RUN curl -sS https://bootstrap.pypa.io/get-pip.py | pypy3 -RUN pypy3 -m pip install 'numpy==1.26.4' 'six==1.16.0' 'pandas==2.2.3' scipy coverage matplotlib lxml +RUN pypy3 -m pip install numpy 'six==1.16.0' 'pandas==2.2.3' scipy coverage matplotlib lxml -ARG BASIC_PIP_PKGS="numpy==1.26.4 pyarrow>=15.0.0 six==1.16.0 pandas==2.2.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" +ARG BASIC_PIP_PKGS="numpy pyarrow>=15.0.0 six==1.16.0 pandas==2.2.3 scipy plotly>=4.8 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect ARG CONNECT_PIP_PKGS="grpcio==1.62.0 grpcio-status==1.62.0 protobuf==4.25.1 googleapis-common-protos==1.56.4 graphviz==0.20.3" diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala index 30f3e4c4af021..5486c39034fd3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -204,7 +204,7 @@ final class Binarizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) val inputType = try { SchemaUtils.getSchemaFieldType(schema, inputColName) } catch { - case e: SparkIllegalArgumentException if e.getErrorClass == "FIELD_NOT_FOUND" => + case e: SparkIllegalArgumentException if e.getCondition == "FIELD_NOT_FOUND" => throw new SparkException(s"Input column $inputColName does not exist.") case e: Exception => throw e diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala index 8e64f60427d90..20b03edf23c4a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -127,7 +127,7 @@ private[feature] trait StringIndexerBase extends Params with HasHandleInvalid wi validateAndTransformField(schema, inputColName, dtype, outputColName) ) } catch { - case e: SparkIllegalArgumentException if e.getErrorClass == "FIELD_NOT_FOUND" => + case e: SparkIllegalArgumentException if e.getCondition == "FIELD_NOT_FOUND" => if (skipNonExistsCol) { None } else { diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala index 3b306eff99689..ff132e2a29a89 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala @@ -17,12 +17,13 @@ package org.apache.spark.ml.util +import org.apache.spark.SparkIllegalArgumentException import org.apache.spark.ml.attribute._ import org.apache.spark.ml.linalg.VectorUDT -import org.apache.spark.sql.catalyst.util.AttributeNameParser +import org.apache.spark.sql.catalyst.util.{AttributeNameParser, QuotingUtils} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ - /** * Utils for handling schemas. */ @@ -206,6 +207,10 @@ private[spark] object SchemaUtils { checkColumnTypes(schema, colName, typeCandidates) } + def toSQLId(parts: String): String = { + AttributeNameParser.parseAttributeName(parts).map(QuotingUtils.quoteIdentifier).mkString(".") + } + /** * Get schema field. * @param schema input schema @@ -213,11 +218,16 @@ private[spark] object SchemaUtils { */ def getSchemaField(schema: StructType, colName: String): StructField = { val colSplits = AttributeNameParser.parseAttributeName(colName) - var field = schema(colSplits(0)) - for (colSplit <- colSplits.slice(1, colSplits.length)) { - field = field.dataType.asInstanceOf[StructType](colSplit) + val fieldOpt = schema.findNestedField(colSplits, resolver = SQLConf.get.resolver) + if (fieldOpt.isEmpty) { + throw new SparkIllegalArgumentException( + errorClass = "FIELD_NOT_FOUND", + messageParameters = Map( + "fieldName" -> toSQLId(colName), + "fields" -> schema.fields.map(f => toSQLId(f.name)).mkString(", ")) + ) } - field + fieldOpt.get._2 } /** diff --git a/pom.xml b/pom.xml index bfaee1be609c0..cab7f7f595434 100644 --- a/pom.xml +++ b/pom.xml @@ -119,7 +119,7 @@ 3.9.9 3.2.0 spark - 9.7 + 9.7.1 2.0.16 2.24.1 @@ -491,7 +491,7 @@ org.apache.xbean xbean-asm9-shaded - 4.25 + 4.26 -Wconf:cat=deprecation&msg=it will become a keyword in Scala 3:e + + -Wconf:cat=deprecation&msg=method getErrorClass in trait SparkThrowable is deprecated:e -Xss128m diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 3ccb0bddfb0eb..f31a29788aafe 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -195,6 +195,9 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.SparkSession.setDefaultSession"), ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.api.SparkSessionCompanion.clearActiveSession"), ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.api.SparkSessionCompanion.clearDefaultSession"), + + // SPARK-49748: Add getCondition and deprecate getErrorClass in SparkThrowable + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.SparkThrowable.getCondition"), ) ++ loggingExcludes("org.apache.spark.sql.DataFrameReader") ++ loggingExcludes("org.apache.spark.sql.streaming.DataStreamReader") ++ loggingExcludes("org.apache.spark.sql.SparkSession#Builder") diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5882fcbf336b0..a87e0af0b542f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -254,7 +254,9 @@ object SparkBuild extends PomBuild { // reduce the cost of migration in subsequent versions. "-Wconf:cat=deprecation&msg=it will become a keyword in Scala 3:e", // SPARK-46938 to prevent enum scan on pmml-model, under spark-mllib module. - "-Wconf:cat=other&site=org.dmg.pmml.*:w" + "-Wconf:cat=other&site=org.dmg.pmml.*:w", + // SPARK-49937 ban call the method `SparkThrowable#getErrorClass` + "-Wconf:cat=deprecation&msg=method getErrorClass in trait SparkThrowable is deprecated:e" ) } ) @@ -1070,20 +1072,9 @@ object DependencyOverrides { object ExcludedDependencies { lazy val settings = Seq( libraryDependencies ~= { libs => libs.filterNot(_.name == "groovy-all") }, - // SPARK-33705: Due to sbt compiler issues, it brings exclusions defined in maven pom back to - // the classpath directly and assemble test scope artifacts to assembly/target/scala-xx/jars, - // which is also will be added to the classpath of some unit tests that will build a subprocess - // to run `spark-submit`, e.g. HiveThriftServer2Test. - // - // These artifacts are for the jersey-1 API but Spark use jersey-2 ones, so it cause test - // flakiness w/ jar conflicts issues. - // - // Also jersey-1 is only used by yarn module(see resource-managers/yarn/pom.xml) for testing - // purpose only. Here we exclude them from the whole project scope and add them w/ yarn only. excludeDependencies ++= Seq( - ExclusionRule(organization = "com.sun.jersey"), ExclusionRule(organization = "ch.qos.logback"), - ExclusionRule("javax.ws.rs", "jsr311-api")) + ExclusionRule("javax.servlet", "javax.servlet-api")) ) } @@ -1227,10 +1218,6 @@ object YARN { val hadoopProvidedProp = "spark.yarn.isHadoopProvided" lazy val settings = Seq( - excludeDependencies --= Seq( - ExclusionRule(organization = "com.sun.jersey"), - ExclusionRule("javax.servlet", "javax.servlet-api"), - ExclusionRule("javax.ws.rs", "jsr311-api")), Compile / unmanagedResources := (Compile / unmanagedResources).value.filter(!_.getName.endsWith(s"$propFileName")), genConfigProperties := { diff --git a/project/plugins.sbt b/project/plugins.sbt index 151af24440c05..b2d0177e6a411 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -24,7 +24,7 @@ libraryDependencies += "com.puppycrawl.tools" % "checkstyle" % "10.17.0" // checkstyle uses guava 33.1.0-jre. libraryDependencies += "com.google.guava" % "guava" % "33.1.0-jre" -addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "2.2.0") +addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "2.3.0") addSbtPlugin("com.github.sbt" % "sbt-eclipse" % "6.2.0") @@ -36,9 +36,9 @@ addSbtPlugin("com.github.sbt" % "sbt-unidoc" % "0.5.0") addSbtPlugin("io.spray" % "sbt-revolver" % "0.10.0") -libraryDependencies += "org.ow2.asm" % "asm" % "9.7" +libraryDependencies += "org.ow2.asm" % "asm" % "9.7.1" -libraryDependencies += "org.ow2.asm" % "asm-commons" % "9.7" +libraryDependencies += "org.ow2.asm" % "asm-commons" % "9.7.1" addSbtPlugin("com.simplytyped" % "sbt-antlr4" % "0.8.3") diff --git a/python/pyspark/errors/error-conditions.json b/python/pyspark/errors/error-conditions.json index ed62ea117d369..ab01d386645b2 100644 --- a/python/pyspark/errors/error-conditions.json +++ b/python/pyspark/errors/error-conditions.json @@ -94,9 +94,9 @@ "Could not get batch id from ." ] }, - "CANNOT_INFER_ARRAY_TYPE": { + "CANNOT_INFER_ARRAY_ELEMENT_TYPE": { "message": [ - "Can not infer Array Type from a list with None as the first element." + "Can not infer the element data type, an non-empty list starting with an non-None value is required." ] }, "CANNOT_INFER_EMPTY_SCHEMA": { @@ -1103,6 +1103,11 @@ "`` is not supported, it should be one of the values from " ] }, + "UNSUPPORTED_PLOT_BACKEND_PARAM": { + "message": [ + "`` does not support `` set to , it should be one of the values from " + ] + }, "UNSUPPORTED_SIGNATURE": { "message": [ "Unsupported signature: ." diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 937753b50bb13..b89755d9c18a5 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -699,7 +699,7 @@ class LinearSVC( >>> model_path = temp_path + "/svm_model" >>> model.save(model_path) >>> model2 = LinearSVCModel.load(model_path) - >>> model.coefficients[0] == model2.coefficients[0] + >>> bool(model.coefficients[0] == model2.coefficients[0]) True >>> model.intercept == model2.intercept True @@ -1210,7 +1210,7 @@ class LogisticRegression( >>> model_path = temp_path + "/lr_model" >>> blorModel.save(model_path) >>> model2 = LogisticRegressionModel.load(model_path) - >>> blorModel.coefficients[0] == model2.coefficients[0] + >>> bool(blorModel.coefficients[0] == model2.coefficients[0]) True >>> blorModel.intercept == model2.intercept True @@ -2038,9 +2038,9 @@ class RandomForestClassifier( >>> result = model.transform(test0).head() >>> result.prediction 0.0 - >>> numpy.argmax(result.probability) + >>> int(numpy.argmax(result.probability)) 0 - >>> numpy.argmax(result.newRawPrediction) + >>> int(numpy.argmax(result.newRawPrediction)) 0 >>> result.leafId DenseVector([0.0, 0.0, 0.0]) diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index d08e241b41d23..d7cc27e274279 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -266,7 +266,7 @@ class LinearRegression( True >>> abs(model.transform(test0).head().newPrediction - (-1.0)) < 0.001 True - >>> abs(model.coefficients[0] - 1.0) < 0.001 + >>> bool(abs(model.coefficients[0] - 1.0) < 0.001) True >>> abs(model.intercept - 0.0) < 0.001 True @@ -283,11 +283,11 @@ class LinearRegression( >>> model_path = temp_path + "/lr_model" >>> model.save(model_path) >>> model2 = LinearRegressionModel.load(model_path) - >>> model.coefficients[0] == model2.coefficients[0] + >>> bool(model.coefficients[0] == model2.coefficients[0]) True - >>> model.intercept == model2.intercept + >>> bool(model.intercept == model2.intercept) True - >>> model.transform(test0).take(1) == model2.transform(test0).take(1) + >>> bool(model.transform(test0).take(1) == model2.transform(test0).take(1)) True >>> model.numFeatures 1 @@ -2542,7 +2542,7 @@ class GeneralizedLinearRegression( >>> model2 = GeneralizedLinearRegressionModel.load(model_path) >>> model.intercept == model2.intercept True - >>> model.coefficients[0] == model2.coefficients[0] + >>> bool(model.coefficients[0] == model2.coefficients[0]) True >>> model.transform(df).take(1) == model2.transform(df).take(1) True diff --git a/python/pyspark/ml/tests/test_functions.py b/python/pyspark/ml/tests/test_functions.py index 7df0a26394140..e67e46ded67bd 100644 --- a/python/pyspark/ml/tests/test_functions.py +++ b/python/pyspark/ml/tests/test_functions.py @@ -18,6 +18,7 @@ import numpy as np +from pyspark.loose_version import LooseVersion from pyspark.ml.functions import predict_batch_udf from pyspark.sql.functions import array, struct, col from pyspark.sql.types import ArrayType, DoubleType, IntegerType, StructType, StructField, FloatType @@ -193,6 +194,10 @@ def predict(inputs): batch_sizes = preds["preds"].to_numpy() self.assertTrue(all(batch_sizes <= batch_size)) + # TODO(SPARK-49793): enable the test below + @unittest.skipIf( + LooseVersion(np.__version__) >= LooseVersion("2"), "Caching does not work with numpy 2" + ) def test_caching(self): def make_predict_fn(): # emulate loading a model, this should only be invoked once (per worker process) diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index e8713d81c4d62..888beff663523 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -706,7 +706,7 @@ class CrossValidator( >>> cvModel = cv.fit(dataset) >>> cvModel.getNumFolds() 3 - >>> cvModel.avgMetrics[0] + >>> float(cvModel.avgMetrics[0]) 0.5 >>> path = tempfile.mkdtemp() >>> model_path = path + "/model" diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 1e1795d9fb3d4..bf8fd04dc2837 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -172,9 +172,9 @@ class LogisticRegressionModel(LinearClassificationModel): >>> path = tempfile.mkdtemp() >>> lrm.save(sc, path) >>> sameModel = LogisticRegressionModel.load(sc, path) - >>> sameModel.predict(numpy.array([0.0, 1.0])) + >>> int(sameModel.predict(numpy.array([0.0, 1.0]))) 1 - >>> sameModel.predict(SparseVector(2, {0: 1.0})) + >>> int(sameModel.predict(SparseVector(2, {0: 1.0}))) 0 >>> from shutil import rmtree >>> try: @@ -555,7 +555,7 @@ class SVMModel(LinearClassificationModel): >>> svm.predict(sc.parallelize([[1.0]])).collect() [1] >>> svm.clearThreshold() - >>> svm.predict(numpy.array([1.0])) + >>> float(svm.predict(numpy.array([1.0]))) 1.44... >>> sparse_data = [ @@ -573,9 +573,9 @@ class SVMModel(LinearClassificationModel): >>> path = tempfile.mkdtemp() >>> svm.save(sc, path) >>> sameModel = SVMModel.load(sc, path) - >>> sameModel.predict(SparseVector(2, {1: 1.0})) + >>> int(sameModel.predict(SparseVector(2, {1: 1.0}))) 1 - >>> sameModel.predict(SparseVector(2, {0: -1.0})) + >>> int(sameModel.predict(SparseVector(2, {0: -1.0}))) 0 >>> from shutil import rmtree >>> try: @@ -756,11 +756,11 @@ class NaiveBayesModel(Saveable, Loader["NaiveBayesModel"]): ... LabeledPoint(1.0, [1.0, 0.0]), ... ] >>> model = NaiveBayes.train(sc.parallelize(data)) - >>> model.predict(numpy.array([0.0, 1.0])) + >>> float(model.predict(numpy.array([0.0, 1.0]))) 0.0 - >>> model.predict(numpy.array([1.0, 0.0])) + >>> float(model.predict(numpy.array([1.0, 0.0]))) 1.0 - >>> model.predict(sc.parallelize([[1.0, 0.0]])).collect() + >>> list(map(float, model.predict(sc.parallelize([[1.0, 0.0]])).collect())) [1.0] >>> sparse_data = [ ... LabeledPoint(0.0, SparseVector(2, {1: 0.0})), @@ -768,15 +768,18 @@ class NaiveBayesModel(Saveable, Loader["NaiveBayesModel"]): ... LabeledPoint(1.0, SparseVector(2, {0: 1.0})) ... ] >>> model = NaiveBayes.train(sc.parallelize(sparse_data)) - >>> model.predict(SparseVector(2, {1: 1.0})) + >>> float(model.predict(SparseVector(2, {1: 1.0}))) 0.0 - >>> model.predict(SparseVector(2, {0: 1.0})) + >>> float(model.predict(SparseVector(2, {0: 1.0}))) 1.0 >>> import os, tempfile >>> path = tempfile.mkdtemp() >>> model.save(sc, path) >>> sameModel = NaiveBayesModel.load(sc, path) - >>> sameModel.predict(SparseVector(2, {0: 1.0})) == model.predict(SparseVector(2, {0: 1.0})) + >>> bool(( + ... sameModel.predict(SparseVector(2, {0: 1.0})) == + ... model.predict(SparseVector(2, {0: 1.0})) + ... )) True >>> from shutil import rmtree >>> try: diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 24884f4853371..915a55595cb53 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -554,9 +554,9 @@ class PCA: ... Vectors.dense([4.0, 0.0, 0.0, 6.0, 7.0])] >>> model = PCA(2).fit(sc.parallelize(data)) >>> pcArray = model.transform(Vectors.sparse(5, [(1, 1.0), (3, 7.0)])).toArray() - >>> pcArray[0] + >>> float(pcArray[0]) 1.648... - >>> pcArray[1] + >>> float(pcArray[1]) -4.013... """ diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index 80bbd717071dc..dbe1048a64b36 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -134,9 +134,9 @@ def normalRDD( >>> stats = x.stats() >>> stats.count() 1000 - >>> abs(stats.mean() - 0.0) < 0.1 + >>> bool(abs(stats.mean() - 0.0) < 0.1) True - >>> abs(stats.stdev() - 1.0) < 0.1 + >>> bool(abs(stats.stdev() - 1.0) < 0.1) True """ return callMLlibFunc("normalRDD", sc._jsc, size, numPartitions, seed) @@ -186,10 +186,10 @@ def logNormalRDD( >>> stats = x.stats() >>> stats.count() 1000 - >>> abs(stats.mean() - expMean) < 0.5 + >>> bool(abs(stats.mean() - expMean) < 0.5) True >>> from math import sqrt - >>> abs(stats.stdev() - expStd) < 0.5 + >>> bool(abs(stats.stdev() - expStd) < 0.5) True """ return callMLlibFunc( @@ -238,7 +238,7 @@ def poissonRDD( >>> abs(stats.mean() - mean) < 0.5 True >>> from math import sqrt - >>> abs(stats.stdev() - sqrt(mean)) < 0.5 + >>> bool(abs(stats.stdev() - sqrt(mean)) < 0.5) True """ return callMLlibFunc("poissonRDD", sc._jsc, float(mean), size, numPartitions, seed) @@ -285,7 +285,7 @@ def exponentialRDD( >>> abs(stats.mean() - mean) < 0.5 True >>> from math import sqrt - >>> abs(stats.stdev() - sqrt(mean)) < 0.5 + >>> bool(abs(stats.stdev() - sqrt(mean)) < 0.5) True """ return callMLlibFunc("exponentialRDD", sc._jsc, float(mean), size, numPartitions, seed) @@ -336,9 +336,9 @@ def gammaRDD( >>> stats = x.stats() >>> stats.count() 1000 - >>> abs(stats.mean() - expMean) < 0.5 + >>> bool(abs(stats.mean() - expMean) < 0.5) True - >>> abs(stats.stdev() - expStd) < 0.5 + >>> bool(abs(stats.stdev() - expStd) < 0.5) True """ return callMLlibFunc( @@ -384,7 +384,7 @@ def uniformVectorRDD( >>> mat = np.matrix(RandomRDDs.uniformVectorRDD(sc, 10, 10).collect()) >>> mat.shape (10, 10) - >>> mat.max() <= 1.0 and mat.min() >= 0.0 + >>> bool(mat.max() <= 1.0 and mat.min() >= 0.0) True >>> RandomRDDs.uniformVectorRDD(sc, 10, 10, 4).getNumPartitions() 4 @@ -430,9 +430,9 @@ def normalVectorRDD( >>> mat = np.matrix(RandomRDDs.normalVectorRDD(sc, 100, 100, seed=1).collect()) >>> mat.shape (100, 100) - >>> abs(mat.mean() - 0.0) < 0.1 + >>> bool(abs(mat.mean() - 0.0) < 0.1) True - >>> abs(mat.std() - 1.0) < 0.1 + >>> bool(abs(mat.std() - 1.0) < 0.1) True """ return callMLlibFunc("normalVectorRDD", sc._jsc, numRows, numCols, numPartitions, seed) @@ -488,9 +488,9 @@ def logNormalVectorRDD( >>> mat = np.matrix(m) >>> mat.shape (100, 100) - >>> abs(mat.mean() - expMean) < 0.1 + >>> bool(abs(mat.mean() - expMean) < 0.1) True - >>> abs(mat.std() - expStd) < 0.1 + >>> bool(abs(mat.std() - expStd) < 0.1) True """ return callMLlibFunc( @@ -545,13 +545,13 @@ def poissonVectorRDD( >>> import numpy as np >>> mean = 100.0 >>> rdd = RandomRDDs.poissonVectorRDD(sc, mean, 100, 100, seed=1) - >>> mat = np.mat(rdd.collect()) + >>> mat = np.asmatrix(rdd.collect()) >>> mat.shape (100, 100) - >>> abs(mat.mean() - mean) < 0.5 + >>> bool(abs(mat.mean() - mean) < 0.5) True >>> from math import sqrt - >>> abs(mat.std() - sqrt(mean)) < 0.5 + >>> bool(abs(mat.std() - sqrt(mean)) < 0.5) True """ return callMLlibFunc( @@ -599,13 +599,13 @@ def exponentialVectorRDD( >>> import numpy as np >>> mean = 0.5 >>> rdd = RandomRDDs.exponentialVectorRDD(sc, mean, 100, 100, seed=1) - >>> mat = np.mat(rdd.collect()) + >>> mat = np.asmatrix(rdd.collect()) >>> mat.shape (100, 100) - >>> abs(mat.mean() - mean) < 0.5 + >>> bool(abs(mat.mean() - mean) < 0.5) True >>> from math import sqrt - >>> abs(mat.std() - sqrt(mean)) < 0.5 + >>> bool(abs(mat.std() - sqrt(mean)) < 0.5) True """ return callMLlibFunc( @@ -662,9 +662,9 @@ def gammaVectorRDD( >>> mat = np.matrix(RandomRDDs.gammaVectorRDD(sc, shape, scale, 100, 100, seed=1).collect()) >>> mat.shape (100, 100) - >>> abs(mat.mean() - expMean) < 0.1 + >>> bool(abs(mat.mean() - expMean) < 0.1) True - >>> abs(mat.std() - expStd) < 0.1 + >>> bool(abs(mat.std() - expStd) < 0.1) True """ return callMLlibFunc( diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index f1003327912d0..87f05bc0979b8 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -144,9 +144,9 @@ class LinearRegressionModelBase(LinearModel): -------- >>> from pyspark.mllib.linalg import SparseVector >>> lrmb = LinearRegressionModelBase(np.array([1.0, 2.0]), 0.1) - >>> abs(lrmb.predict(np.array([-1.03, 7.777])) - 14.624) < 1e-6 + >>> bool(abs(lrmb.predict(np.array([-1.03, 7.777])) - 14.624) < 1e-6) True - >>> abs(lrmb.predict(SparseVector(2, {0: -1.03, 1: 7.777})) - 14.624) < 1e-6 + >>> bool(abs(lrmb.predict(SparseVector(2, {0: -1.03, 1: 7.777})) - 14.624) < 1e-6) True """ @@ -190,23 +190,23 @@ class LinearRegressionModel(LinearRegressionModelBase): ... ] >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, ... initialWeights=np.array([1.0])) - >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 + >>> bool(abs(lrm.predict(np.array([0.0])) - 0) < 0.5) True - >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 + >>> bool(abs(lrm.predict(np.array([1.0])) - 1) < 0.5) True - >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + >>> bool(abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5) True - >>> abs(lrm.predict(sc.parallelize([[1.0]])).collect()[0] - 1) < 0.5 + >>> bool(abs(lrm.predict(sc.parallelize([[1.0]])).collect()[0] - 1) < 0.5) True >>> import os, tempfile >>> path = tempfile.mkdtemp() >>> lrm.save(sc, path) >>> sameModel = LinearRegressionModel.load(sc, path) - >>> abs(sameModel.predict(np.array([0.0])) - 0) < 0.5 + >>> bool(abs(sameModel.predict(np.array([0.0])) - 0) < 0.5) True - >>> abs(sameModel.predict(np.array([1.0])) - 1) < 0.5 + >>> bool(abs(sameModel.predict(np.array([1.0])) - 1) < 0.5) True - >>> abs(sameModel.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + >>> bool(abs(sameModel.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5) True >>> from shutil import rmtree >>> try: @@ -221,16 +221,16 @@ class LinearRegressionModel(LinearRegressionModelBase): ... ] >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, ... initialWeights=np.array([1.0])) - >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 + >>> bool(abs(lrm.predict(np.array([0.0])) - 0) < 0.5) True - >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + >>> bool(abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5) True >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, step=1.0, ... miniBatchFraction=1.0, initialWeights=np.array([1.0]), regParam=0.1, regType="l2", ... intercept=True, validateData=True) - >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 + >>> bool(abs(lrm.predict(np.array([0.0])) - 0) < 0.5) True - >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + >>> bool(abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5) True """ @@ -402,23 +402,23 @@ class LassoModel(LinearRegressionModelBase): ... ] >>> lrm = LassoWithSGD.train( ... sc.parallelize(data), iterations=10, initialWeights=np.array([1.0])) - >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 + >>> bool(abs(lrm.predict(np.array([0.0])) - 0) < 0.5) True - >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 + >>> bool(abs(lrm.predict(np.array([1.0])) - 1) < 0.5) True - >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + >>> bool(abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5) True - >>> abs(lrm.predict(sc.parallelize([[1.0]])).collect()[0] - 1) < 0.5 + >>> bool(abs(lrm.predict(sc.parallelize([[1.0]])).collect()[0] - 1) < 0.5) True >>> import os, tempfile >>> path = tempfile.mkdtemp() >>> lrm.save(sc, path) >>> sameModel = LassoModel.load(sc, path) - >>> abs(sameModel.predict(np.array([0.0])) - 0) < 0.5 + >>> bool(abs(sameModel.predict(np.array([0.0])) - 0) < 0.5) True - >>> abs(sameModel.predict(np.array([1.0])) - 1) < 0.5 + >>> bool(abs(sameModel.predict(np.array([1.0])) - 1) < 0.5) True - >>> abs(sameModel.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + >>> bool(abs(sameModel.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5) True >>> from shutil import rmtree >>> try: @@ -433,16 +433,16 @@ class LassoModel(LinearRegressionModelBase): ... ] >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, ... initialWeights=np.array([1.0])) - >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 + >>> bool(abs(lrm.predict(np.array([0.0])) - 0) < 0.5) True - >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + >>> bool(abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5) True >>> lrm = LassoWithSGD.train(sc.parallelize(data), iterations=10, step=1.0, ... regParam=0.01, miniBatchFraction=1.0, initialWeights=np.array([1.0]), intercept=True, ... validateData=True) - >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 + >>> bool(abs(lrm.predict(np.array([0.0])) - 0) < 0.5) True - >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + >>> bool(abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5) True """ @@ -580,23 +580,23 @@ class RidgeRegressionModel(LinearRegressionModelBase): ... ] >>> lrm = RidgeRegressionWithSGD.train(sc.parallelize(data), iterations=10, ... initialWeights=np.array([1.0])) - >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 + >>> bool(abs(lrm.predict(np.array([0.0])) - 0) < 0.5) True - >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 + >>> bool(abs(lrm.predict(np.array([1.0])) - 1) < 0.5) True - >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + >>> bool(abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5) True - >>> abs(lrm.predict(sc.parallelize([[1.0]])).collect()[0] - 1) < 0.5 + >>> bool(abs(lrm.predict(sc.parallelize([[1.0]])).collect()[0] - 1) < 0.5) True >>> import os, tempfile >>> path = tempfile.mkdtemp() >>> lrm.save(sc, path) >>> sameModel = RidgeRegressionModel.load(sc, path) - >>> abs(sameModel.predict(np.array([0.0])) - 0) < 0.5 + >>> bool(abs(sameModel.predict(np.array([0.0])) - 0) < 0.5) True - >>> abs(sameModel.predict(np.array([1.0])) - 1) < 0.5 + >>> bool(abs(sameModel.predict(np.array([1.0])) - 1) < 0.5) True - >>> abs(sameModel.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + >>> bool(abs(sameModel.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5) True >>> from shutil import rmtree >>> try: @@ -611,16 +611,16 @@ class RidgeRegressionModel(LinearRegressionModelBase): ... ] >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), iterations=10, ... initialWeights=np.array([1.0])) - >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 + >>> bool(abs(lrm.predict(np.array([0.0])) - 0) < 0.5) True - >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + >>> bool(abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5) True >>> lrm = RidgeRegressionWithSGD.train(sc.parallelize(data), iterations=10, step=1.0, ... regParam=0.01, miniBatchFraction=1.0, initialWeights=np.array([1.0]), intercept=True, ... validateData=True) - >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 + >>> bool(abs(lrm.predict(np.array([0.0])) - 0) < 0.5) True - >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 + >>> bool(abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5) True """ @@ -764,19 +764,19 @@ class IsotonicRegressionModel(Saveable, Loader["IsotonicRegressionModel"]): -------- >>> data = [(1, 0, 1), (2, 1, 1), (3, 2, 1), (1, 3, 1), (6, 4, 1), (17, 5, 1), (16, 6, 1)] >>> irm = IsotonicRegression.train(sc.parallelize(data)) - >>> irm.predict(3) + >>> float(irm.predict(3)) 2.0 - >>> irm.predict(5) + >>> float(irm.predict(5)) 16.5 - >>> irm.predict(sc.parallelize([3, 5])).collect() + >>> list(map(float, irm.predict(sc.parallelize([3, 5])).collect())) [2.0, 16.5] >>> import os, tempfile >>> path = tempfile.mkdtemp() >>> irm.save(sc, path) >>> sameModel = IsotonicRegressionModel.load(sc, path) - >>> sameModel.predict(3) + >>> float(sameModel.predict(3)) 2.0 - >>> sameModel.predict(5) + >>> float(sameModel.predict(5)) 16.5 >>> from shutil import rmtree >>> try: diff --git a/python/pyspark/pandas/data_type_ops/datetime_ops.py b/python/pyspark/pandas/data_type_ops/datetime_ops.py index 9b4cc72fa2e45..dc2f68232e730 100644 --- a/python/pyspark/pandas/data_type_ops/datetime_ops.py +++ b/python/pyspark/pandas/data_type_ops/datetime_ops.py @@ -34,6 +34,7 @@ ) from pyspark.sql.utils import pyspark_column_op from pyspark.pandas._typing import Dtype, IndexOpsLike, SeriesOrIndex +from pyspark.pandas.spark import functions as SF from pyspark.pandas.base import IndexOpsMixin from pyspark.pandas.data_type_ops.base import ( DataTypeOps, @@ -150,10 +151,7 @@ class DatetimeNTZOps(DatetimeOps): """ def _cast_spark_column_timestamp_to_long(self, scol: Column) -> Column: - from pyspark import SparkContext - - jvm = SparkContext._active_spark_context._jvm - return Column(jvm.PythonSQLUtils.castTimestampNTZToLong(scol._jc)) + return SF.timestamp_ntz_to_long(scol) def astype(self, index_ops: IndexOpsLike, dtype: Union[str, type, Dtype]) -> IndexOpsLike: dtype, spark_type = pandas_on_spark_type(dtype) diff --git a/python/pyspark/pandas/generic.py b/python/pyspark/pandas/generic.py index 6e63cff1d37b9..55f15fd2eb1a2 100644 --- a/python/pyspark/pandas/generic.py +++ b/python/pyspark/pandas/generic.py @@ -2631,7 +2631,7 @@ def first_valid_index(self) -> Optional[Union[Scalar, Tuple[Scalar, ...]]]: 500 5.0 dtype: float64 - >>> s.first_valid_index() + >>> int(s.first_valid_index()) 300 Support for MultiIndex @@ -2950,7 +2950,7 @@ def get(self, key: Any, default: Optional[Any] = None) -> Any: 20 1 b 20 2 b - >>> df.x.get(10) + >>> int(df.x.get(10)) 0 >>> df.x.get(20) @@ -3008,7 +3008,7 @@ def squeeze(self, axis: Optional[Axis] = None) -> Union[Scalar, "DataFrame", "Se 0 2 dtype: int64 - >>> even_primes.squeeze() + >>> int(even_primes.squeeze()) 2 Squeezing objects with more than one value in every axis does nothing: @@ -3066,7 +3066,7 @@ def squeeze(self, axis: Optional[Axis] = None) -> Union[Scalar, "DataFrame", "Se Squeezing all axes will project directly into a scalar: - >>> df_1a.squeeze() + >>> int(df_1a.squeeze()) 3 """ if axis is not None: diff --git a/python/pyspark/pandas/indexing.py b/python/pyspark/pandas/indexing.py index b5bf65a4907b7..c93366a31e315 100644 --- a/python/pyspark/pandas/indexing.py +++ b/python/pyspark/pandas/indexing.py @@ -122,7 +122,7 @@ class AtIndexer(IndexerLike): Get value at specified row/column pair - >>> psdf.at[4, 'B'] + >>> int(psdf.at[4, 'B']) 2 Get array if an index occurs multiple times @@ -202,7 +202,7 @@ class iAtIndexer(IndexerLike): Get value at specified row/column pair - >>> df.iat[1, 2] + >>> int(df.iat[1, 2]) 1 Get value within a series @@ -214,7 +214,7 @@ class iAtIndexer(IndexerLike): 30 3 dtype: int64 - >>> psser.iat[1] + >>> int(psser.iat[1]) 2 """ @@ -853,7 +853,7 @@ class LocIndexer(LocIndexerLike): Single label for column. - >>> df.loc['cobra', 'shield'] + >>> int(df.loc['cobra', 'shield']) 2 List of labels for row. diff --git a/python/pyspark/pandas/internal.py b/python/pyspark/pandas/internal.py index 4be345201ba65..6063641e22e3b 100644 --- a/python/pyspark/pandas/internal.py +++ b/python/pyspark/pandas/internal.py @@ -902,11 +902,10 @@ def attach_default_index( @staticmethod def attach_sequence_column(sdf: PySparkDataFrame, column_name: str) -> PySparkDataFrame: - scols = [scol_for(sdf, column) for column in sdf.columns] sequential_index = ( F.row_number().over(Window.orderBy(F.monotonically_increasing_id())).cast("long") - 1 ) - return sdf.select(sequential_index.alias(column_name), *scols) + return sdf.select(sequential_index.alias(column_name), "*") @staticmethod def attach_distributed_column(sdf: PySparkDataFrame, column_name: str) -> PySparkDataFrame: diff --git a/python/pyspark/pandas/plot/core.py b/python/pyspark/pandas/plot/core.py index 7333fae1ad432..12c17a06f153b 100644 --- a/python/pyspark/pandas/plot/core.py +++ b/python/pyspark/pandas/plot/core.py @@ -841,7 +841,7 @@ def barh(self, x=None, y=None, **kwargs): elif isinstance(self.data, DataFrame): return self(kind="barh", x=x, y=y, **kwargs) - def box(self, **kwds): + def box(self, precision=0.01, **kwds): """ Make a box plot of the DataFrame columns. @@ -857,14 +857,13 @@ def box(self, **kwds): Parameters ---------- - **kwds : optional - Additional keyword arguments are documented in - :meth:`pyspark.pandas.Series.plot`. - precision: scalar, default = 0.01 This argument is used by pandas-on-Spark to compute approximate statistics for building a boxplot. Use *smaller* values to get more precise - statistics (matplotlib-only). + statistics. + **kwds : optional + Additional keyword arguments are documented in + :meth:`pyspark.pandas.Series.plot`. Returns ------- @@ -902,7 +901,7 @@ def box(self, **kwds): from pyspark.pandas import DataFrame, Series if isinstance(self.data, (Series, DataFrame)): - return self(kind="box", **kwds) + return self(kind="box", precision=precision, **kwds) def hist(self, bins=10, **kwds): """ diff --git a/python/pyspark/pandas/series.py b/python/pyspark/pandas/series.py index ff941b692f95f..7e276860fbab1 100644 --- a/python/pyspark/pandas/series.py +++ b/python/pyspark/pandas/series.py @@ -4558,7 +4558,7 @@ def pop(self, item: Name) -> Union["Series", Scalar]: C 2 dtype: int64 - >>> s.pop('A') + >>> int(s.pop('A')) 0 >>> s @@ -5821,7 +5821,7 @@ def asof(self, where: Union[Any, List]) -> Union[Scalar, "Series"]: A scalar `where`. - >>> s.asof(20) + >>> float(s.asof(20)) 2.0 For a sequence `where`, a Series is returned. The first value is @@ -5836,12 +5836,12 @@ def asof(self, where: Union[Any, List]) -> Union[Scalar, "Series"]: Missing values are not considered. The following is ``2.0``, not NaN, even though NaN is at the index location for ``30``. - >>> s.asof(30) + >>> float(s.asof(30)) 2.0 >>> s = ps.Series([1, 2, np.nan, 4], index=[10, 30, 20, 40]) >>> with ps.option_context("compute.eager_check", False): - ... s.asof(20) + ... float(s.asof(20)) ... 1.0 """ diff --git a/python/pyspark/pandas/spark/functions.py b/python/pyspark/pandas/spark/functions.py index 4d95466a98e12..bdd11559df3b6 100644 --- a/python/pyspark/pandas/spark/functions.py +++ b/python/pyspark/pandas/spark/functions.py @@ -39,6 +39,10 @@ def _invoke_internal_function_over_columns(name: str, *cols: "ColumnOrName") -> return Column(sc._jvm.PythonSQLUtils.internalFn(name, _to_seq(sc, cols, _to_java_column))) +def timestamp_ntz_to_long(col: Column) -> Column: + return _invoke_internal_function_over_columns("timestamp_ntz_to_long", col) + + def product(col: Column, dropna: bool) -> Column: return _invoke_internal_function_over_columns("pandas_product", col, F.lit(dropna)) diff --git a/python/pyspark/sql/connect/expressions.py b/python/pyspark/sql/connect/expressions.py index 85f1b3565c696..203b6ce371a5c 100644 --- a/python/pyspark/sql/connect/expressions.py +++ b/python/pyspark/sql/connect/expressions.py @@ -301,7 +301,7 @@ def _infer_type(cls, value: Any) -> DataType: return NullType() elif isinstance(value, (bytes, bytearray)): return BinaryType() - elif isinstance(value, bool): + elif isinstance(value, (bool, np.bool_)): return BooleanType() elif isinstance(value, int): if JVM_INT_MIN <= value <= JVM_INT_MAX: @@ -323,10 +323,8 @@ def _infer_type(cls, value: Any) -> DataType: return StringType() elif isinstance(value, decimal.Decimal): return DecimalType() - elif isinstance(value, datetime.datetime) and is_timestamp_ntz_preferred(): - return TimestampNTZType() elif isinstance(value, datetime.datetime): - return TimestampType() + return TimestampNTZType() if is_timestamp_ntz_preferred() else TimestampType() elif isinstance(value, datetime.date): return DateType() elif isinstance(value, datetime.timedelta): @@ -335,23 +333,15 @@ def _infer_type(cls, value: Any) -> DataType: dt = _from_numpy_type(value.dtype) if dt is not None: return dt - elif isinstance(value, np.bool_): - return BooleanType() elif isinstance(value, list): # follow the 'infer_array_from_first_element' strategy in 'sql.types._infer_type' # right now, it's dedicated for pyspark.ml params like array<...>, array> - if len(value) == 0: - raise PySparkValueError( - errorClass="CANNOT_BE_EMPTY", - messageParameters={"item": "value"}, - ) - first = value[0] - if first is None: + if len(value) == 0 or value[0] is None: raise PySparkTypeError( - errorClass="CANNOT_INFER_ARRAY_TYPE", + errorClass="CANNOT_INFER_ARRAY_ELEMENT_TYPE", messageParameters={}, ) - return ArrayType(LiteralExpression._infer_type(first), True) + return ArrayType(LiteralExpression._infer_type(value[0]), True) raise PySparkTypeError( errorClass="UNSUPPORTED_DATA_TYPE", diff --git a/python/pyspark/sql/plot/core.py b/python/pyspark/sql/plot/core.py index f9667ee2c0d69..4bf75474d92c3 100644 --- a/python/pyspark/sql/plot/core.py +++ b/python/pyspark/sql/plot/core.py @@ -15,15 +15,17 @@ # limitations under the License. # -from typing import Any, TYPE_CHECKING, Optional, Union +from typing import Any, TYPE_CHECKING, List, Optional, Union from types import ModuleType from pyspark.errors import PySparkRuntimeError, PySparkTypeError, PySparkValueError +from pyspark.sql import Column, functions as F from pyspark.sql.types import NumericType -from pyspark.sql.utils import require_minimum_plotly_version +from pyspark.sql.utils import is_remote, require_minimum_plotly_version if TYPE_CHECKING: - from pyspark.sql import DataFrame + from pyspark.sql import DataFrame, Row + from pyspark.sql._typing import ColumnOrName import pandas as pd from plotly.graph_objs import Figure @@ -338,3 +340,148 @@ def pie(self, x: str, y: str, **kwargs: Any) -> "Figure": }, ) return self(kind="pie", x=x, y=y, **kwargs) + + def box( + self, column: Union[str, List[str]], precision: float = 0.01, **kwargs: Any + ) -> "Figure": + """ + Make a box plot of the DataFrame columns. + + Make a box-and-whisker plot from DataFrame columns, optionally grouped by some + other columns. A box plot is a method for graphically depicting groups of numerical + data through their quartiles. The box extends from the Q1 to Q3 quartile values of + the data, with a line at the median (Q2). The whiskers extend from the edges of box + to show the range of the data. By default, they extend no more than + 1.5 * IQR (IQR = Q3 - Q1) from the edges of the box, ending at the farthest data point + within that interval. Outliers are plotted as separate dots. + + Parameters + ---------- + column: str or list of str + Column name or list of names to be used for creating the boxplot. + precision: float, default = 0.01 + This argument is used by pyspark to compute approximate statistics + for building a boxplot. + **kwargs + Additional keyword arguments. + + Returns + ------- + :class:`plotly.graph_objs.Figure` + + Examples + -------- + >>> data = [ + ... ("A", 50, 55), + ... ("B", 55, 60), + ... ("C", 60, 65), + ... ("D", 65, 70), + ... ("E", 70, 75), + ... ("F", 10, 15), + ... ("G", 85, 90), + ... ("H", 5, 150), + ... ] + >>> columns = ["student", "math_score", "english_score"] + >>> df = spark.createDataFrame(data, columns) + >>> df.plot.box(column="math_score") # doctest: +SKIP + >>> df.plot.box(column=["math_score", "english_score"]) # doctest: +SKIP + """ + return self(kind="box", column=column, precision=precision, **kwargs) + + +class PySparkBoxPlotBase: + @staticmethod + def compute_box( + sdf: "DataFrame", colnames: List[str], whis: float, precision: float, showfliers: bool + ) -> Optional["Row"]: + assert len(colnames) > 0 + formatted_colnames = ["`{}`".format(colname) for colname in colnames] + + stats_scols = [] + for i, colname in enumerate(formatted_colnames): + percentiles = F.percentile_approx(colname, [0.25, 0.50, 0.75], int(1.0 / precision)) + q1 = F.get(percentiles, 0) + med = F.get(percentiles, 1) + q3 = F.get(percentiles, 2) + iqr = q3 - q1 + lfence = q1 - F.lit(whis) * iqr + ufence = q3 + F.lit(whis) * iqr + + stats_scols.append( + F.struct( + F.mean(colname).alias("mean"), + med.alias("med"), + q1.alias("q1"), + q3.alias("q3"), + lfence.alias("lfence"), + ufence.alias("ufence"), + ).alias(f"_box_plot_stats_{i}") + ) + + sdf_stats = sdf.select(*stats_scols) + + result_scols = [] + for i, colname in enumerate(formatted_colnames): + value = F.col(colname) + + lfence = F.col(f"_box_plot_stats_{i}.lfence") + ufence = F.col(f"_box_plot_stats_{i}.ufence") + mean = F.col(f"_box_plot_stats_{i}.mean") + med = F.col(f"_box_plot_stats_{i}.med") + q1 = F.col(f"_box_plot_stats_{i}.q1") + q3 = F.col(f"_box_plot_stats_{i}.q3") + + outlier = ~value.between(lfence, ufence) + + # Computes min and max values of non-outliers - the whiskers + upper_whisker = F.max(F.when(~outlier, value).otherwise(F.lit(None))) + lower_whisker = F.min(F.when(~outlier, value).otherwise(F.lit(None))) + + # If it shows fliers, take the top 1k with the highest absolute values + # Here we normalize the values by subtracting the median. + if showfliers: + pair = F.when( + outlier, + F.struct(F.abs(value - med), value.alias("val")), + ).otherwise(F.lit(None)) + topk = collect_top_k(pair, 1001, False) + fliers = F.when(F.size(topk) > 0, topk["val"]).otherwise(F.lit(None)) + else: + fliers = F.lit(None) + + result_scols.append( + F.struct( + F.first(mean).alias("mean"), + F.first(med).alias("med"), + F.first(q1).alias("q1"), + F.first(q3).alias("q3"), + upper_whisker.alias("upper_whisker"), + lower_whisker.alias("lower_whisker"), + fliers.alias("fliers"), + ).alias(f"_box_plot_results_{i}") + ) + + sdf_result = sdf.join(sdf_stats.hint("broadcast")).select(*result_scols) + return sdf_result.first() + + +def _invoke_internal_function_over_columns(name: str, *cols: "ColumnOrName") -> Column: + if is_remote(): + from pyspark.sql.connect.functions.builtin import _invoke_function_over_columns + + return _invoke_function_over_columns(name, *cols) + + else: + from pyspark.sql.classic.column import _to_seq, _to_java_column + from pyspark import SparkContext + + sc = SparkContext._active_spark_context + return Column( + sc._jvm.PythonSQLUtils.internalFn( # type: ignore + name, _to_seq(sc, cols, _to_java_column) # type: ignore + ) + ) + + +def collect_top_k(col: Column, num: int, reverse: bool) -> Column: + return _invoke_internal_function_over_columns("collect_top_k", col, F.lit(num), F.lit(reverse)) diff --git a/python/pyspark/sql/plot/plotly.py b/python/pyspark/sql/plot/plotly.py index 91f5363464717..71d40720e874d 100644 --- a/python/pyspark/sql/plot/plotly.py +++ b/python/pyspark/sql/plot/plotly.py @@ -17,7 +17,8 @@ from typing import TYPE_CHECKING, Any -from pyspark.sql.plot import PySparkPlotAccessor +from pyspark.errors import PySparkValueError +from pyspark.sql.plot import PySparkPlotAccessor, PySparkBoxPlotBase if TYPE_CHECKING: from pyspark.sql import DataFrame @@ -29,6 +30,8 @@ def plot_pyspark(data: "DataFrame", kind: str, **kwargs: Any) -> "Figure": if kind == "pie": return plot_pie(data, **kwargs) + if kind == "box": + return plot_box(data, **kwargs) return plotly.plot(PySparkPlotAccessor.plot_data_map[kind](data), kind, **kwargs) @@ -43,3 +46,75 @@ def plot_pie(data: "DataFrame", **kwargs: Any) -> "Figure": fig = express.pie(pdf, values=y, names=x, **kwargs) return fig + + +def plot_box(data: "DataFrame", **kwargs: Any) -> "Figure": + import plotly.graph_objs as go + + # 'whis' isn't actually an argument in plotly (but in matplotlib). But seems like + # plotly doesn't expose the reach of the whiskers to the beyond the first and + # third quartiles (?). Looks they use default 1.5. + whis = kwargs.pop("whis", 1.5) + # 'precision' is pyspark specific to control precision for approx_percentile + precision = kwargs.pop("precision", 0.01) + colnames = kwargs.pop("column", None) + if isinstance(colnames, str): + colnames = [colnames] + + # Plotly options + boxpoints = kwargs.pop("boxpoints", "suspectedoutliers") + notched = kwargs.pop("notched", False) + if boxpoints not in ["suspectedoutliers", False]: + raise PySparkValueError( + errorClass="UNSUPPORTED_PLOT_BACKEND_PARAM", + messageParameters={ + "backend": "plotly", + "param": "boxpoints", + "value": str(boxpoints), + "supported_values": ", ".join(["suspectedoutliers", "False"]), + }, + ) + if notched: + raise PySparkValueError( + errorClass="UNSUPPORTED_PLOT_BACKEND_PARAM", + messageParameters={ + "backend": "plotly", + "param": "notched", + "value": str(notched), + "supported_values": ", ".join(["False"]), + }, + ) + + fig = go.Figure() + + results = PySparkBoxPlotBase.compute_box( + data, + colnames, + whis, + precision, + boxpoints is not None, + ) + assert len(results) == len(colnames) # type: ignore + + for i, colname in enumerate(colnames): + result = results[i] # type: ignore + + fig.add_trace( + go.Box( + x=[i], + name=colname, + q1=[result["q1"]], + median=[result["med"]], + q3=[result["q3"]], + mean=[result["mean"]], + lowerfence=[result["lower_whisker"]], + upperfence=[result["upper_whisker"]], + y=[result["fliers"]] if result["fliers"] else None, + boxpoints=boxpoints, + notched=notched, + **kwargs, + ) + ) + + fig["layout"]["yaxis"]["title"] = "value" + return fig diff --git a/python/pyspark/sql/streaming/python_streaming_source_runner.py b/python/pyspark/sql/streaming/python_streaming_source_runner.py index c50bd3915784b..a7349779dc626 100644 --- a/python/pyspark/sql/streaming/python_streaming_source_runner.py +++ b/python/pyspark/sql/streaming/python_streaming_source_runner.py @@ -193,6 +193,8 @@ def main(infile: IO, outfile: IO) -> None: reader.stop() except BaseException as e: handle_worker_exception(e, outfile) + # ensure that the updates to the socket are flushed + outfile.flush() sys.exit(-1) send_accumulator_updates(outfile) diff --git a/python/pyspark/sql/tests/connect/test_parity_readwriter.py b/python/pyspark/sql/tests/connect/test_parity_readwriter.py index 46333b555c351..f83f3edbfa787 100644 --- a/python/pyspark/sql/tests/connect/test_parity_readwriter.py +++ b/python/pyspark/sql/tests/connect/test_parity_readwriter.py @@ -33,6 +33,7 @@ def test_api(self): def test_partitioning_functions(self): self.check_partitioning_functions(DataFrameWriterV2) + self.partitioning_functions_user_error() if __name__ == "__main__": diff --git a/python/pyspark/sql/tests/plot/test_frame_plot_plotly.py b/python/pyspark/sql/tests/plot/test_frame_plot_plotly.py index 70a1b336f734a..d870cdbf9959b 100644 --- a/python/pyspark/sql/tests/plot/test_frame_plot_plotly.py +++ b/python/pyspark/sql/tests/plot/test_frame_plot_plotly.py @@ -19,7 +19,7 @@ from datetime import datetime import pyspark.sql.plot # noqa: F401 -from pyspark.errors import PySparkTypeError +from pyspark.errors import PySparkTypeError, PySparkValueError from pyspark.testing.sqlutils import ReusedSQLTestCase, have_plotly, plotly_requirement_message @@ -48,79 +48,190 @@ def sdf3(self): columns = ["sales", "signups", "visits", "date"] return self.spark.createDataFrame(data, columns) - def _check_fig_data(self, kind, fig_data, expected_x, expected_y, expected_name=""): - if kind == "line": - self.assertEqual(fig_data["mode"], "lines") - self.assertEqual(fig_data["type"], "scatter") - elif kind == "bar": - self.assertEqual(fig_data["type"], "bar") - elif kind == "barh": - self.assertEqual(fig_data["type"], "bar") - self.assertEqual(fig_data["orientation"], "h") - elif kind == "scatter": - self.assertEqual(fig_data["type"], "scatter") - self.assertEqual(fig_data["orientation"], "v") - self.assertEqual(fig_data["mode"], "markers") - elif kind == "area": - self.assertEqual(fig_data["type"], "scatter") - self.assertEqual(fig_data["orientation"], "v") - self.assertEqual(fig_data["mode"], "lines") - elif kind == "pie": - self.assertEqual(fig_data["type"], "pie") - self.assertEqual(list(fig_data["labels"]), expected_x) - self.assertEqual(list(fig_data["values"]), expected_y) - return - - self.assertEqual(fig_data["xaxis"], "x") - self.assertEqual(list(fig_data["x"]), expected_x) - self.assertEqual(fig_data["yaxis"], "y") - self.assertEqual(list(fig_data["y"]), expected_y) - self.assertEqual(fig_data["name"], expected_name) + @property + def sdf4(self): + data = [ + ("A", 50, 55), + ("B", 55, 60), + ("C", 60, 65), + ("D", 65, 70), + ("E", 70, 75), + # outliers + ("F", 10, 15), + ("G", 85, 90), + ("H", 5, 150), + ] + columns = ["student", "math_score", "english_score"] + return self.spark.createDataFrame(data, columns) + + def _check_fig_data(self, fig_data, **kwargs): + for key, expected_value in kwargs.items(): + if key in ["x", "y", "labels", "values"]: + converted_values = [v.item() if hasattr(v, "item") else v for v in fig_data[key]] + self.assertEqual(converted_values, expected_value) + else: + self.assertEqual(fig_data[key], expected_value) def test_line_plot(self): # single column as vertical axis fig = self.sdf.plot(kind="line", x="category", y="int_val") - self._check_fig_data("line", fig["data"][0], ["A", "B", "C"], [10, 30, 20]) + expected_fig_data = { + "mode": "lines", + "name": "", + "orientation": "v", + "x": ["A", "B", "C"], + "xaxis": "x", + "y": [10, 30, 20], + "yaxis": "y", + "type": "scatter", + } + self._check_fig_data(fig["data"][0], **expected_fig_data) # multiple columns as vertical axis fig = self.sdf.plot.line(x="category", y=["int_val", "float_val"]) - self._check_fig_data("line", fig["data"][0], ["A", "B", "C"], [10, 30, 20], "int_val") - self._check_fig_data("line", fig["data"][1], ["A", "B", "C"], [1.5, 2.5, 3.5], "float_val") + expected_fig_data = { + "mode": "lines", + "name": "int_val", + "orientation": "v", + "x": ["A", "B", "C"], + "xaxis": "x", + "y": [10, 30, 20], + "yaxis": "y", + "type": "scatter", + } + self._check_fig_data(fig["data"][0], **expected_fig_data) + expected_fig_data = { + "mode": "lines", + "name": "float_val", + "orientation": "v", + "x": ["A", "B", "C"], + "xaxis": "x", + "y": [1.5, 2.5, 3.5], + "yaxis": "y", + "type": "scatter", + } + self._check_fig_data(fig["data"][1], **expected_fig_data) def test_bar_plot(self): # single column as vertical axis fig = self.sdf.plot(kind="bar", x="category", y="int_val") - self._check_fig_data("bar", fig["data"][0], ["A", "B", "C"], [10, 30, 20]) + expected_fig_data = { + "name": "", + "orientation": "v", + "x": ["A", "B", "C"], + "xaxis": "x", + "y": [10, 30, 20], + "yaxis": "y", + "type": "bar", + } + self._check_fig_data(fig["data"][0], **expected_fig_data) # multiple columns as vertical axis fig = self.sdf.plot.bar(x="category", y=["int_val", "float_val"]) - self._check_fig_data("bar", fig["data"][0], ["A", "B", "C"], [10, 30, 20], "int_val") - self._check_fig_data("bar", fig["data"][1], ["A", "B", "C"], [1.5, 2.5, 3.5], "float_val") + expected_fig_data = { + "name": "int_val", + "orientation": "v", + "x": ["A", "B", "C"], + "xaxis": "x", + "y": [10, 30, 20], + "yaxis": "y", + "type": "bar", + } + self._check_fig_data(fig["data"][0], **expected_fig_data) + expected_fig_data = { + "name": "float_val", + "orientation": "v", + "x": ["A", "B", "C"], + "xaxis": "x", + "y": [1.5, 2.5, 3.5], + "yaxis": "y", + "type": "bar", + } + self._check_fig_data(fig["data"][1], **expected_fig_data) def test_barh_plot(self): # single column as vertical axis fig = self.sdf.plot(kind="barh", x="category", y="int_val") - self._check_fig_data("barh", fig["data"][0], ["A", "B", "C"], [10, 30, 20]) + expected_fig_data = { + "name": "", + "orientation": "h", + "x": ["A", "B", "C"], + "xaxis": "x", + "y": [10, 30, 20], + "yaxis": "y", + "type": "bar", + } + self._check_fig_data(fig["data"][0], **expected_fig_data) # multiple columns as vertical axis fig = self.sdf.plot.barh(x="category", y=["int_val", "float_val"]) - self._check_fig_data("barh", fig["data"][0], ["A", "B", "C"], [10, 30, 20], "int_val") - self._check_fig_data("barh", fig["data"][1], ["A", "B", "C"], [1.5, 2.5, 3.5], "float_val") + expected_fig_data = { + "name": "int_val", + "orientation": "h", + "x": ["A", "B", "C"], + "xaxis": "x", + "y": [10, 30, 20], + "yaxis": "y", + "type": "bar", + } + self._check_fig_data(fig["data"][0], **expected_fig_data) + expected_fig_data = { + "name": "float_val", + "orientation": "h", + "x": ["A", "B", "C"], + "xaxis": "x", + "y": [1.5, 2.5, 3.5], + "yaxis": "y", + "type": "bar", + } + self._check_fig_data(fig["data"][1], **expected_fig_data) # multiple columns as horizontal axis fig = self.sdf.plot.barh(x=["int_val", "float_val"], y="category") - self._check_fig_data("barh", fig["data"][0], [10, 30, 20], ["A", "B", "C"], "int_val") - self._check_fig_data("barh", fig["data"][1], [1.5, 2.5, 3.5], ["A", "B", "C"], "float_val") + expected_fig_data = { + "name": "int_val", + "orientation": "h", + "y": ["A", "B", "C"], + "xaxis": "x", + "x": [10, 30, 20], + "yaxis": "y", + "type": "bar", + } + self._check_fig_data(fig["data"][0], **expected_fig_data) + expected_fig_data = { + "name": "float_val", + "orientation": "h", + "y": ["A", "B", "C"], + "xaxis": "x", + "x": [1.5, 2.5, 3.5], + "yaxis": "y", + "type": "bar", + } + self._check_fig_data(fig["data"][1], **expected_fig_data) def test_scatter_plot(self): fig = self.sdf2.plot(kind="scatter", x="length", y="width") - self._check_fig_data( - "scatter", fig["data"][0], [5.1, 4.9, 7.0, 6.4, 5.9], [3.5, 3.0, 3.2, 3.2, 3.0] - ) + expected_fig_data = { + "name": "", + "orientation": "v", + "x": [5.1, 4.9, 7.0, 6.4, 5.9], + "xaxis": "x", + "y": [3.5, 3.0, 3.2, 3.2, 3.0], + "yaxis": "y", + "type": "scatter", + } + self._check_fig_data(fig["data"][0], **expected_fig_data) + expected_fig_data = { + "name": "", + "orientation": "v", + "y": [5.1, 4.9, 7.0, 6.4, 5.9], + "xaxis": "x", + "x": [3.5, 3.0, 3.2, 3.2, 3.0], + "yaxis": "y", + "type": "scatter", + } fig = self.sdf2.plot.scatter(x="width", y="length") - self._check_fig_data( - "scatter", fig["data"][0], [3.5, 3.0, 3.2, 3.2, 3.0], [5.1, 4.9, 7.0, 6.4, 5.9] - ) + self._check_fig_data(fig["data"][0], **expected_fig_data) def test_area_plot(self): # single column as vertical axis @@ -131,13 +242,53 @@ def test_area_plot(self): datetime(2018, 3, 31, 0, 0), datetime(2018, 4, 30, 0, 0), ] - self._check_fig_data("area", fig["data"][0], expected_x, [3, 2, 3, 9]) + expected_fig_data = { + "name": "", + "orientation": "v", + "x": expected_x, + "xaxis": "x", + "y": [3, 2, 3, 9], + "yaxis": "y", + "mode": "lines", + "type": "scatter", + } + self._check_fig_data(fig["data"][0], **expected_fig_data) # multiple columns as vertical axis fig = self.sdf3.plot.area(x="date", y=["sales", "signups", "visits"]) - self._check_fig_data("area", fig["data"][0], expected_x, [3, 2, 3, 9], "sales") - self._check_fig_data("area", fig["data"][1], expected_x, [5, 5, 6, 12], "signups") - self._check_fig_data("area", fig["data"][2], expected_x, [20, 42, 28, 62], "visits") + expected_fig_data = { + "name": "sales", + "orientation": "v", + "x": expected_x, + "xaxis": "x", + "y": [3, 2, 3, 9], + "yaxis": "y", + "mode": "lines", + "type": "scatter", + } + self._check_fig_data(fig["data"][0], **expected_fig_data) + expected_fig_data = { + "name": "signups", + "orientation": "v", + "x": expected_x, + "xaxis": "x", + "y": [5, 5, 6, 12], + "yaxis": "y", + "mode": "lines", + "type": "scatter", + } + self._check_fig_data(fig["data"][1], **expected_fig_data) + expected_fig_data = { + "name": "visits", + "orientation": "v", + "x": expected_x, + "xaxis": "x", + "y": [20, 42, 28, 62], + "yaxis": "y", + "mode": "lines", + "type": "scatter", + } + self._check_fig_data(fig["data"][2], **expected_fig_data) def test_pie_plot(self): fig = self.sdf3.plot(kind="pie", x="date", y="sales") @@ -147,17 +298,83 @@ def test_pie_plot(self): datetime(2018, 3, 31, 0, 0), datetime(2018, 4, 30, 0, 0), ] - self._check_fig_data("pie", fig["data"][0], expected_x, [3, 2, 3, 9]) + expected_fig_data = { + "name": "", + "labels": expected_x, + "values": [3, 2, 3, 9], + "type": "pie", + } + self._check_fig_data(fig["data"][0], **expected_fig_data) # y is not a numerical column with self.assertRaises(PySparkTypeError) as pe: self.sdf.plot.pie(x="int_val", y="category") + self.check_error( exception=pe.exception, errorClass="PLOT_NOT_NUMERIC_COLUMN", messageParameters={"arg_name": "y", "arg_type": "StringType()"}, ) + def test_box_plot(self): + fig = self.sdf4.plot.box(column="math_score") + expected_fig_data = { + "boxpoints": "suspectedoutliers", + "lowerfence": (5,), + "mean": (50.0,), + "median": (55,), + "name": "math_score", + "notched": False, + "q1": (10,), + "q3": (65,), + "upperfence": (85,), + "x": [0], + "type": "box", + } + self._check_fig_data(fig["data"][0], **expected_fig_data) + + fig = self.sdf4.plot(kind="box", column=["math_score", "english_score"]) + self._check_fig_data(fig["data"][0], **expected_fig_data) + expected_fig_data = { + "boxpoints": "suspectedoutliers", + "lowerfence": (55,), + "mean": (72.5,), + "median": (65,), + "name": "english_score", + "notched": False, + "q1": (55,), + "q3": (75,), + "upperfence": (90,), + "x": [1], + "y": [[150, 15]], + "type": "box", + } + self._check_fig_data(fig["data"][1], **expected_fig_data) + with self.assertRaises(PySparkValueError) as pe: + self.sdf4.plot.box(column="math_score", boxpoints=True) + self.check_error( + exception=pe.exception, + errorClass="UNSUPPORTED_PLOT_BACKEND_PARAM", + messageParameters={ + "backend": "plotly", + "param": "boxpoints", + "value": "True", + "supported_values": ", ".join(["suspectedoutliers", "False"]), + }, + ) + with self.assertRaises(PySparkValueError) as pe: + self.sdf4.plot.box(column="math_score", notched=True) + self.check_error( + exception=pe.exception, + errorClass="UNSUPPORTED_PLOT_BACKEND_PARAM", + messageParameters={ + "backend": "plotly", + "param": "notched", + "value": "True", + "supported_values": ", ".join(["False"]), + }, + ) + class DataFramePlotPlotlyTests(DataFramePlotPlotlyTestsMixin, ReusedSQLTestCase): pass diff --git a/python/pyspark/sql/tests/streaming/test_streaming_listener.py b/python/pyspark/sql/tests/streaming/test_streaming_listener.py index c3ae62e64cc30..51f62f56a7c54 100644 --- a/python/pyspark/sql/tests/streaming/test_streaming_listener.py +++ b/python/pyspark/sql/tests/streaming/test_streaming_listener.py @@ -381,7 +381,12 @@ def verify(test_listener): .start() ) self.assertTrue(q.isActive) - q.awaitTermination(10) + wait_count = 0 + while progress_event is None or progress_event.progress.batchId == 0: + q.awaitTermination(0.5) + wait_count = wait_count + 1 + if wait_count > 100: + self.fail("Not getting progress event after 50 seconds") q.stop() # Make sure all events are empty diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index cc43804949e84..2f53ca38743c1 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -15,6 +15,8 @@ # limitations under the License. # +import glob +import os import pydoc import shutil import tempfile @@ -47,6 +49,7 @@ pandas_requirement_message, pyarrow_requirement_message, ) +from pyspark.testing.utils import SPARK_HOME class DataFrameTestsMixin: @@ -506,14 +509,16 @@ def test_toDF_with_schema_string(self): # number of fields must match. self.assertRaisesRegex( - Exception, "FIELD_STRUCT_LENGTH_MISMATCH", lambda: rdd.toDF("key: int").collect() + Exception, + "FIELD_STRUCT_LENGTH_MISMATCH", + lambda: rdd.coalesce(1).toDF("key: int").collect(), ) # field types mismatch will cause exception at runtime. self.assertRaisesRegex( Exception, "FIELD_DATA_TYPE_UNACCEPTABLE", - lambda: rdd.toDF("key: float, value: string").collect(), + lambda: rdd.coalesce(1).toDF("key: float, value: string").collect(), ) # flat schema values will be wrapped into row. @@ -777,6 +782,16 @@ def test_df_show(self): ) def test_df_merge_into(self): + filename_pattern = ( + "sql/catalyst/target/scala-*/test-classes/org/apache/spark/sql/connector/catalog/" + "InMemoryRowLevelOperationTableCatalog.class" + ) + if not bool(glob.glob(os.path.join(SPARK_HOME, filename_pattern))): + raise unittest.SkipTest( + "org.apache.spark.sql.connector.catalog.InMemoryRowLevelOperationTableCatalog' " + "is not available. Will skip the related tests" + ) + try: # InMemoryRowLevelOperationTableCatalog is a test catalog that is included in the # catalyst-test package. If Spark complains that it can't find this class, make sure diff --git a/python/pyspark/sql/tests/test_readwriter.py b/python/pyspark/sql/tests/test_readwriter.py index f4f32dea9060a..2fca6b57decf9 100644 --- a/python/pyspark/sql/tests/test_readwriter.py +++ b/python/pyspark/sql/tests/test_readwriter.py @@ -255,6 +255,7 @@ def check_api(self, tpe): def test_partitioning_functions(self): self.check_partitioning_functions(DataFrameWriterV2) + self.partitioning_functions_user_error() def check_partitioning_functions(self, tpe): import datetime @@ -274,6 +275,35 @@ def check_partitioning_functions(self, tpe): self.assertIsInstance(writer.partitionedBy(bucket(11, col("id"))), tpe) self.assertIsInstance(writer.partitionedBy(bucket(3, "id"), hours(col("ts"))), tpe) + def partitioning_functions_user_error(self): + import datetime + from pyspark.sql.functions.partitioning import years, months, days, hours, bucket + + df = self.spark.createDataFrame( + [(1, datetime.datetime(2000, 1, 1), "foo")], ("id", "ts", "value") + ) + + with self.assertRaisesRegex( + Exception, "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY" + ): + df.select(years("ts")).collect() + with self.assertRaisesRegex( + Exception, "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY" + ): + df.select(months("ts")).collect() + with self.assertRaisesRegex( + Exception, "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY" + ): + df.select(days("ts")).collect() + with self.assertRaisesRegex( + Exception, "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY" + ): + df.select(hours("ts")).collect() + with self.assertRaisesRegex( + Exception, "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY" + ): + df.select(bucket(2, "ts")).collect() + def test_create(self): df = self.df with self.table("test_table"): diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 770a550030f51..5a10aa797c1b1 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -29,43 +29,8 @@ Spark Project YARN yarn - 1.19 - - - hadoop-3 - - true - - - - org.apache.hadoop - hadoop-client-runtime - ${hadoop.version} - ${hadoop.deps.scope} - - - org.apache.hadoop - hadoop-client-minicluster - ${hadoop.version} - test - - - - org.bouncycastle - bcprov-jdk18on - test - - - org.bouncycastle - bcpkix-jdk18on - test - - - - - org.apache.spark @@ -102,6 +67,35 @@ org.apache.hadoop hadoop-client-api ${hadoop.version} + ${hadoop.deps.scope} + + + org.apache.hadoop + hadoop-client-runtime + ${hadoop.version} + ${hadoop.deps.scope} + + + org.apache.hadoop + hadoop-client-minicluster + ${hadoop.version} + test + + + + javax.xml.bind + jaxb-api + test + + + org.bouncycastle + bcprov-jdk18on + test + + + org.bouncycastle + bcpkix-jdk18on + test @@ -135,22 +129,6 @@ - - - org.eclipse.jetty.orbit - javax.servlet.jsp - 2.2.0.v201112011158 - test - - - org.eclipse.jetty.orbit - javax.servlet.jsp.jstl - 1.2.0.v201105211821 - test - - org.mockito mockito-core @@ -166,65 +144,6 @@ byte-buddy-agent test - - - - com.sun.jersey - jersey-core - test - ${jersey-1.version} - - - com.sun.jersey - jersey-json - test - ${jersey-1.version} - - - com.sun.jersey - jersey-server - test - ${jersey-1.version} - - - com.sun.jersey.contribs - jersey-guice - test - ${jersey-1.version} - - - com.sun.jersey - jersey-servlet - test - ${jersey-1.version} - - - - - ${hive.group} - hive-exec - ${hive.classifier} - provided - - - ${hive.group} - hive-metastore - provided - - - org.apache.thrift - libthrift - provided - - - org.apache.thrift - libfb303 - provided - diff --git a/sql/api/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/api/src/main/scala/org/apache/spark/sql/AnalysisException.scala index a2c1f2cc41f8f..51825ee1a5bed 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -139,7 +139,7 @@ class AnalysisException protected ( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull override def getQueryContext: Array[QueryContext] = context diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala index 10da24567545b..f2c7dd533af3a 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala @@ -100,7 +100,7 @@ abstract class AbstractParser extends DataTypeParserInterface with Logging { command = Option(command), start = e.origin, stop = e.origin, - errorClass = e.getErrorClass, + errorClass = e.getCondition, messageParameters = e.getMessageParameters.asScala.toMap, queryContext = e.getQueryContext) } @@ -275,7 +275,7 @@ class ParseException private ( } def withCommand(cmd: String): ParseException = { - val cl = getErrorClass + val cl = getCondition val (newCl, params) = if (cl == "PARSE_SYNTAX_ERROR" && cmd.trim().isEmpty) { // PARSE_EMPTY_STATEMENT error class overrides the PARSE_SYNTAX_ERROR when cmd is empty ("PARSE_EMPTY_STATEMENT", Map.empty[String, String]) @@ -287,7 +287,7 @@ class ParseException private ( override def getQueryContext: Array[QueryContext] = queryContext - override def getErrorClass: String = errorClass.getOrElse { + override def getCondition: String = errorClass.getOrElse { throw SparkException.internalError("ParseException shall have an error class.") } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala index 3527a10496862..907c46f583cf1 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/ExecutionErrors.scala @@ -195,9 +195,9 @@ private[sql] trait ExecutionErrors extends DataTypeErrorsBase { fieldName: String, walkedTypePath: WalkedTypePath): SparkUnsupportedOperationException = { new SparkUnsupportedOperationException( - errorClass = "_LEGACY_ERROR_TEMP_2140", + errorClass = "INVALID_JAVA_IDENTIFIER_AS_FIELD_NAME", messageParameters = - Map("fieldName" -> fieldName, "walkedTypePath" -> walkedTypePath.toString)) + Map("fieldName" -> toSQLId(fieldName), "walkedTypePath" -> walkedTypePath.toString)) } def primaryConstructorNotFoundError(cls: Class[_]): SparkRuntimeException = { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala index b0743d6de4772..53cbf086c96e3 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala @@ -516,8 +516,11 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0043", ctx) } - def intervalValueOutOfRangeError(ctx: IntervalContext): Throwable = { - new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0044", ctx) + def intervalValueOutOfRangeError(input: String, ctx: IntervalContext): Throwable = { + new ParseException( + errorClass = "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + messageParameters = Map("input" -> input), + ctx) } def invalidTimeZoneDisplacementValueError(ctx: SetTimeZoneContext): Throwable = { diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala index 29d48e3d1f47f..1c93c2ad550e9 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StringType.scala @@ -48,7 +48,7 @@ class StringType private (val collationId: Int) extends AtomicType with Serializ !CollationFactory.isCaseSensitiveAndAccentInsensitive(collationId) private[sql] def usesTrimCollation: Boolean = - CollationFactory.usesTrimCollation(collationId) + CollationFactory.fetchCollation(collationId).supportsSpaceTrimming private[sql] def isUTF8BinaryCollation: Boolean = collationId == CollationFactory.UTF8_BINARY_COLLATION_ID diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala index 4ef1cf400b80e..07f6b50bd4a7a 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -321,7 +321,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru * * If includeCollections is true, this will return fields that are nested in maps and arrays. */ - private[sql] def findNestedField( + private[spark] def findNestedField( fieldNames: Seq[String], includeCollections: Boolean = false, resolver: SqlApiAnalysis.Resolver = _ == _, diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionUtils.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionUtils.java new file mode 100644 index 0000000000000..07e13610aa950 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionUtils.java @@ -0,0 +1,89 @@ +/* + * 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.catalyst.expressions.json; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonToken; + +import org.apache.spark.sql.catalyst.expressions.SharedFactory; +import org.apache.spark.sql.catalyst.json.CreateJacksonParser; +import org.apache.spark.sql.catalyst.util.GenericArrayData; +import org.apache.spark.unsafe.types.UTF8String; + +public class JsonExpressionUtils { + + public static Integer lengthOfJsonArray(UTF8String json) { + // return null for null input + if (json == null) { + return null; + } + try (JsonParser jsonParser = + CreateJacksonParser.utf8String(SharedFactory.jsonFactory(), json)) { + if (jsonParser.nextToken() == null) { + return null; + } + // Only JSON array are supported for this function. + if (jsonParser.currentToken() != JsonToken.START_ARRAY) { + return null; + } + // Parse the array to compute its length. + int length = 0; + // Keep traversing until the end of JSON array + while (jsonParser.nextToken() != JsonToken.END_ARRAY) { + length += 1; + // skip all the child of inner object or array + jsonParser.skipChildren(); + } + return length; + } catch (IOException e) { + return null; + } + } + + public static GenericArrayData jsonObjectKeys(UTF8String json) { + // return null for `NULL` input + if (json == null) { + return null; + } + try (JsonParser jsonParser = + CreateJacksonParser.utf8String(SharedFactory.jsonFactory(), json)) { + // return null if an empty string or any other valid JSON string is encountered + if (jsonParser.nextToken() == null || jsonParser.currentToken() != JsonToken.START_OBJECT) { + return null; + } + // Parse the JSON string to get all the keys of outermost JSON object + List arrayBufferOfKeys = new ArrayList<>(); + + // traverse until the end of input and ensure it returns valid key + while (jsonParser.nextValue() != null && jsonParser.currentName() != null) { + // add current fieldName to the ArrayBuffer + arrayBufferOfKeys.add(UTF8String.fromString(jsonParser.currentName())); + + // skip all the children of inner object or array + jsonParser.skipChildren(); + } + return new GenericArrayData(arrayBufferOfKeys.toArray()); + } catch (IOException e) { + return null; + } + } +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java index ca4ea5114c26b..c0078872bd843 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java @@ -20,8 +20,11 @@ import org.apache.spark.SparkUnsupportedOperationException; import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.util.QuotingUtils; import org.apache.spark.sql.types.DataType; +import java.util.Map; + /** * Interface for a function that produces a result value for each input row. *

@@ -149,7 +152,10 @@ public interface ScalarFunction extends BoundFunction { * @return a result value */ default R produceResult(InternalRow input) { - throw new SparkUnsupportedOperationException("_LEGACY_ERROR_TEMP_3146"); + throw new SparkUnsupportedOperationException( + "SCALAR_FUNCTION_NOT_COMPATIBLE", + Map.of("scalarFunc", QuotingUtils.quoteIdentifier(name())) + ); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java index 721e6a60befe2..12a2879794b10 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java @@ -58,7 +58,7 @@ public int numElements() { private UnsafeArrayData setNullBits(UnsafeArrayData arrayData) { if (data.hasNull()) { for (int i = 0; i < length; i++) { - if (data.isNullAt(i)) { + if (data.isNullAt(offset + i)) { arrayData.setNullAt(i); } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 5d41c07b47842..49f3092390536 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -39,7 +39,6 @@ import org.apache.spark.sql.catalyst.optimizer.OptimizeUpdateFields import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2 import org.apache.spark.sql.catalyst.trees.AlwaysProcess import org.apache.spark.sql.catalyst.trees.CurrentOrigin.withOrigin import org.apache.spark.sql.catalyst.trees.TreePattern._ @@ -203,6 +202,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor with CheckAnalysis with SQLConfHelper with ColumnResolutionHelper { private val v1SessionCatalog: SessionCatalog = catalogManager.v1SessionCatalog + private val relationResolution = new RelationResolution(catalogManager) override protected def validatePlanChanges( previousPlan: LogicalPlan, @@ -972,30 +972,6 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor } } - private def isResolvingView: Boolean = AnalysisContext.get.catalogAndNamespace.nonEmpty - private def isReferredTempViewName(nameParts: Seq[String]): Boolean = { - AnalysisContext.get.referredTempViewNames.exists { n => - (n.length == nameParts.length) && n.zip(nameParts).forall { - case (a, b) => resolver(a, b) - } - } - } - - // If we are resolving database objects (relations, functions, etc.) insides views, we may need to - // expand single or multi-part identifiers with the current catalog and namespace of when the - // view was created. - private def expandIdentifier(nameParts: Seq[String]): Seq[String] = { - if (!isResolvingView || isReferredTempViewName(nameParts)) return nameParts - - if (nameParts.length == 1) { - AnalysisContext.get.catalogAndNamespace :+ nameParts.head - } else if (catalogManager.isCatalogRegistered(nameParts.head)) { - nameParts - } else { - AnalysisContext.get.catalogAndNamespace.head +: nameParts - } - } - /** * Adds metadata columns to output for child relations when nodes are missing resolved attributes. * @@ -1122,7 +1098,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor case i @ InsertIntoStatement(table, _, _, _, _, _, _) => val relation = table match { case u: UnresolvedRelation if !u.isStreaming => - resolveRelation(u).getOrElse(u) + relationResolution.resolveRelation(u).getOrElse(u) case other => other } @@ -1139,7 +1115,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor case write: V2WriteCommand => write.table match { case u: UnresolvedRelation if !u.isStreaming => - resolveRelation(u).map(unwrapRelationPlan).map { + relationResolution.resolveRelation(u).map(unwrapRelationPlan).map { case v: View => throw QueryCompilationErrors.writeIntoViewNotAllowedError( v.desc.identifier, write) case r: DataSourceV2Relation => write.withNewTable(r) @@ -1154,12 +1130,12 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor } case u: UnresolvedRelation => - resolveRelation(u).map(resolveViews).getOrElse(u) + relationResolution.resolveRelation(u).map(resolveViews).getOrElse(u) case r @ RelationTimeTravel(u: UnresolvedRelation, timestamp, version) if timestamp.forall(ts => ts.resolved && !SubqueryExpression.hasSubquery(ts)) => val timeTravelSpec = TimeTravelSpec.create(timestamp, version, conf.sessionLocalTimeZone) - resolveRelation(u, timeTravelSpec).getOrElse(r) + relationResolution.resolveRelation(u, timeTravelSpec).getOrElse(r) case u @ UnresolvedTable(identifier, cmd, suggestAlternative) => lookupTableOrView(identifier).map { @@ -1194,29 +1170,6 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor }.getOrElse(u) } - private def lookupTempView(identifier: Seq[String]): Option[TemporaryViewRelation] = { - // We are resolving a view and this name is not a temp view when that view was created. We - // return None earlier here. - if (isResolvingView && !isReferredTempViewName(identifier)) return None - v1SessionCatalog.getRawLocalOrGlobalTempView(identifier) - } - - private def resolveTempView( - identifier: Seq[String], - isStreaming: Boolean = false, - isTimeTravel: Boolean = false): Option[LogicalPlan] = { - lookupTempView(identifier).map { v => - val tempViewPlan = v1SessionCatalog.getTempViewRelation(v) - if (isStreaming && !tempViewPlan.isStreaming) { - throw QueryCompilationErrors.readNonStreamingTempViewError(identifier.quoted) - } - if (isTimeTravel) { - throw QueryCompilationErrors.timeTravelUnsupportedError(toSQLId(identifier)) - } - tempViewPlan - } - } - /** * Resolves relations to `ResolvedTable` or `Resolved[Temp/Persistent]View`. This is * for resolving DDL and misc commands. @@ -1224,10 +1177,10 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor private def lookupTableOrView( identifier: Seq[String], viewOnly: Boolean = false): Option[LogicalPlan] = { - lookupTempView(identifier).map { tempView => + relationResolution.lookupTempView(identifier).map { tempView => ResolvedTempView(identifier.asIdentifier, tempView.tableMeta) }.orElse { - expandIdentifier(identifier) match { + relationResolution.expandIdentifier(identifier) match { case CatalogAndIdentifier(catalog, ident) => if (viewOnly && !CatalogV2Util.isSessionCatalog(catalog)) { throw QueryCompilationErrors.catalogOperationNotSupported(catalog, "views") @@ -1246,113 +1199,6 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor } } } - - private def createRelation( - catalog: CatalogPlugin, - ident: Identifier, - table: Option[Table], - options: CaseInsensitiveStringMap, - isStreaming: Boolean): Option[LogicalPlan] = { - table.map { - // To utilize this code path to execute V1 commands, e.g. INSERT, - // either it must be session catalog, or tracksPartitionsInCatalog - // must be false so it does not require use catalog to manage partitions. - // Obviously we cannot execute V1Table by V1 code path if the table - // is not from session catalog and the table still requires its catalog - // to manage partitions. - case v1Table: V1Table if CatalogV2Util.isSessionCatalog(catalog) - || !v1Table.catalogTable.tracksPartitionsInCatalog => - if (isStreaming) { - if (v1Table.v1Table.tableType == CatalogTableType.VIEW) { - throw QueryCompilationErrors.permanentViewNotSupportedByStreamingReadingAPIError( - ident.quoted) - } - SubqueryAlias( - catalog.name +: ident.asMultipartIdentifier, - UnresolvedCatalogRelation(v1Table.v1Table, options, isStreaming = true)) - } else { - v1SessionCatalog.getRelation(v1Table.v1Table, options) - } - - case table => - if (isStreaming) { - val v1Fallback = table match { - case withFallback: V2TableWithV1Fallback => - Some(UnresolvedCatalogRelation(withFallback.v1Table, isStreaming = true)) - case _ => None - } - SubqueryAlias( - catalog.name +: ident.asMultipartIdentifier, - StreamingRelationV2(None, table.name, table, options, table.columns.toAttributes, - Some(catalog), Some(ident), v1Fallback)) - } else { - SubqueryAlias( - catalog.name +: ident.asMultipartIdentifier, - DataSourceV2Relation.create(table, Some(catalog), Some(ident), options)) - } - } - } - - /** - * Resolves relations to v1 relation if it's a v1 table from the session catalog, or to v2 - * relation. This is for resolving DML commands and SELECT queries. - */ - private def resolveRelation( - u: UnresolvedRelation, - timeTravelSpec: Option[TimeTravelSpec] = None): Option[LogicalPlan] = { - val timeTravelSpecFromOptions = TimeTravelSpec.fromOptions( - u.options, - conf.getConf(SQLConf.TIME_TRAVEL_TIMESTAMP_KEY), - conf.getConf(SQLConf.TIME_TRAVEL_VERSION_KEY), - conf.sessionLocalTimeZone - ) - if (timeTravelSpec.nonEmpty && timeTravelSpecFromOptions.nonEmpty) { - throw new AnalysisException("MULTIPLE_TIME_TRAVEL_SPEC", Map.empty[String, String]) - } - val finalTimeTravelSpec = timeTravelSpec.orElse(timeTravelSpecFromOptions) - resolveTempView(u.multipartIdentifier, u.isStreaming, finalTimeTravelSpec.isDefined).orElse { - expandIdentifier(u.multipartIdentifier) match { - case CatalogAndIdentifier(catalog, ident) => - val key = - ((catalog.name +: ident.namespace :+ ident.name).toImmutableArraySeq, - finalTimeTravelSpec) - AnalysisContext.get.relationCache.get(key).map { cache => - val cachedRelation = cache.transform { - case multi: MultiInstanceRelation => - val newRelation = multi.newInstance() - newRelation.copyTagsFrom(multi) - newRelation - } - u.getTagValue(LogicalPlan.PLAN_ID_TAG).map { planId => - val cachedConnectRelation = cachedRelation.clone() - cachedConnectRelation.setTagValue(LogicalPlan.PLAN_ID_TAG, planId) - cachedConnectRelation - }.getOrElse(cachedRelation) - }.orElse { - val writePrivilegesString = - Option(u.options.get(UnresolvedRelation.REQUIRED_WRITE_PRIVILEGES)) - val table = CatalogV2Util.loadTable( - catalog, ident, finalTimeTravelSpec, writePrivilegesString) - val loaded = createRelation( - catalog, ident, table, u.clearWritePrivileges.options, u.isStreaming) - loaded.foreach(AnalysisContext.get.relationCache.update(key, _)) - u.getTagValue(LogicalPlan.PLAN_ID_TAG).map { planId => - loaded.map { loadedRelation => - val loadedConnectRelation = loadedRelation.clone() - loadedConnectRelation.setTagValue(LogicalPlan.PLAN_ID_TAG, planId) - loadedConnectRelation - } - }.getOrElse(loaded) - } - case _ => None - } - } - } - - /** Consumes an unresolved relation and resolves it to a v1 or v2 relation or temporary view. */ - def resolveRelationOrTempView(u: UnresolvedRelation): LogicalPlan = { - EliminateSubqueryAliases(resolveRelation(u).getOrElse(u)) - } } /** Handle INSERT INTO for DSv2 */ @@ -2135,7 +1981,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor if (ResolveFunctions.lookupBuiltinOrTempFunction(nameParts, Some(f)).isDefined) { f } else { - val CatalogAndIdentifier(catalog, ident) = expandIdentifier(nameParts) + val CatalogAndIdentifier(catalog, ident) = + relationResolution.expandIdentifier(nameParts) val fullName = normalizeFuncName((catalog.name +: ident.namespace :+ ident.name).toImmutableArraySeq) if (externalFunctionNameSet.contains(fullName)) { @@ -2186,7 +2033,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor ResolvedNonPersistentFunc(nameParts.head, V1Function(info)) } }.getOrElse { - val CatalogAndIdentifier(catalog, ident) = expandIdentifier(nameParts) + val CatalogAndIdentifier(catalog, ident) = + relationResolution.expandIdentifier(nameParts) val fullName = catalog.name +: ident.namespace :+ ident.name CatalogV2Util.loadFunction(catalog, ident).map { func => ResolvedPersistentFunc(catalog.asFunctionCatalog, ident, func) @@ -2198,7 +2046,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor withPosition(u) { try { val resolvedFunc = resolveBuiltinOrTempTableFunction(u.name, u.functionArgs).getOrElse { - val CatalogAndIdentifier(catalog, ident) = expandIdentifier(u.name) + val CatalogAndIdentifier(catalog, ident) = + relationResolution.expandIdentifier(u.name) if (CatalogV2Util.isSessionCatalog(catalog)) { v1SessionCatalog.resolvePersistentTableFunction( ident.asFunctionIdentifier, u.functionArgs) @@ -2355,7 +2204,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor private[analysis] def resolveFunction(u: UnresolvedFunction): Expression = { withPosition(u) { resolveBuiltinOrTempFunction(u.nameParts, u.arguments, u).getOrElse { - val CatalogAndIdentifier(catalog, ident) = expandIdentifier(u.nameParts) + val CatalogAndIdentifier(catalog, ident) = + relationResolution.expandIdentifier(u.nameParts) if (CatalogV2Util.isSessionCatalog(catalog)) { resolveV1Function(ident.asFunctionIdentifier, u.arguments, u) } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index a4f424ba4b421..4720b9dcdfa13 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -1621,7 +1621,7 @@ class PreemptedError() { // errors have the lowest priority. def set(error: Exception with SparkThrowable, priority: Option[Int] = None): Unit = { val calculatedPriority = priority.getOrElse { - error.getErrorClass match { + error.getCondition match { case c if c.startsWith("INTERNAL_ERROR") => 1 case _ => 2 } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index e22a4b941b30c..8181078c519fc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -24,20 +24,12 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern._ -/** - * A helper class used to detect duplicate relations fast in `DeduplicateRelations`. Two relations - * are duplicated if: - * 1. they are the same class. - * 2. they have the same output attribute IDs. - * - * The first condition is necessary because the CTE relation definition node and reference node have - * the same output attribute IDs but they are not duplicated. - */ -case class RelationWrapper(cls: Class[_], outputAttrIds: Seq[Long]) - object DeduplicateRelations extends Rule[LogicalPlan] { + + type ExprIdMap = mutable.HashMap[Class[_], mutable.HashSet[Long]] + override def apply(plan: LogicalPlan): LogicalPlan = { - val newPlan = renewDuplicatedRelations(mutable.HashSet.empty, plan)._1 + val newPlan = renewDuplicatedRelations(mutable.HashMap.empty, plan)._1 // Wait for `ResolveMissingReferences` to resolve missing attributes first def noMissingInput(p: LogicalPlan) = !p.exists(_.missingInput.nonEmpty) @@ -86,10 +78,10 @@ object DeduplicateRelations extends Rule[LogicalPlan] { } private def existDuplicatedExprId( - existingRelations: mutable.HashSet[RelationWrapper], - plan: RelationWrapper): Boolean = { - existingRelations.filter(_.cls == plan.cls) - .exists(_.outputAttrIds.intersect(plan.outputAttrIds).nonEmpty) + existingRelations: ExprIdMap, + planClass: Class[_], exprIds: Seq[Long]): Boolean = { + val attrSet = existingRelations.getOrElse(planClass, mutable.HashSet.empty) + exprIds.exists(attrSet.contains) } /** @@ -100,20 +92,16 @@ object DeduplicateRelations extends Rule[LogicalPlan] { * whether the plan is changed or not) */ private def renewDuplicatedRelations( - existingRelations: mutable.HashSet[RelationWrapper], + existingRelations: ExprIdMap, plan: LogicalPlan): (LogicalPlan, Boolean) = plan match { case p: LogicalPlan if p.isStreaming => (plan, false) case m: MultiInstanceRelation => - val planWrapper = RelationWrapper(m.getClass, m.output.map(_.exprId.id)) - if (existingRelations.contains(planWrapper)) { - val newNode = m.newInstance() - newNode.copyTagsFrom(m) - (newNode, true) - } else { - existingRelations.add(planWrapper) - (m, false) - } + deduplicateAndRenew[LogicalPlan with MultiInstanceRelation]( + existingRelations, + m, + _.output.map(_.exprId.id), + node => node.newInstance().asInstanceOf[LogicalPlan with MultiInstanceRelation]) case p: Project => deduplicateAndRenew[Project]( @@ -207,7 +195,7 @@ object DeduplicateRelations extends Rule[LogicalPlan] { } private def deduplicate( - existingRelations: mutable.HashSet[RelationWrapper], + existingRelations: ExprIdMap, plan: LogicalPlan): (LogicalPlan, Boolean) = { var planChanged = false val newPlan = if (plan.children.nonEmpty) { @@ -291,20 +279,21 @@ object DeduplicateRelations extends Rule[LogicalPlan] { } private def deduplicateAndRenew[T <: LogicalPlan]( - existingRelations: mutable.HashSet[RelationWrapper], plan: T, + existingRelations: ExprIdMap, plan: T, getExprIds: T => Seq[Long], copyNewPlan: T => T): (LogicalPlan, Boolean) = { var (newPlan, planChanged) = deduplicate(existingRelations, plan) if (newPlan.resolved) { val exprIds = getExprIds(newPlan.asInstanceOf[T]) if (exprIds.nonEmpty) { - val planWrapper = RelationWrapper(newPlan.getClass, exprIds) - if (existDuplicatedExprId(existingRelations, planWrapper)) { + if (existDuplicatedExprId(existingRelations, newPlan.getClass, exprIds)) { newPlan = copyNewPlan(newPlan.asInstanceOf[T]) newPlan.copyTagsFrom(plan) (newPlan, true) } else { - existingRelations.add(planWrapper) + val attrSet = existingRelations.getOrElseUpdate(newPlan.getClass, mutable.HashSet.empty) + exprIds.foreach(attrSet.add) + existingRelations.put(newPlan.getClass, attrSet) (newPlan, planChanged) } } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala new file mode 100644 index 0000000000000..08be456f090e2 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala @@ -0,0 +1,245 @@ +/* + * 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.catalyst.analysis + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.catalyst.catalog.{ + CatalogTableType, + TemporaryViewRelation, + UnresolvedCatalogRelation +} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} +import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2 +import org.apache.spark.sql.connector.catalog.{ + CatalogManager, + CatalogPlugin, + CatalogV2Util, + Identifier, + LookupCatalog, + Table, + V1Table, + V2TableWithV1Fallback +} +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ +import org.apache.spark.sql.errors.{DataTypeErrorsBase, QueryCompilationErrors} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.ArrayImplicits._ + +class RelationResolution(override val catalogManager: CatalogManager) + extends DataTypeErrorsBase + with Logging + with LookupCatalog + with SQLConfHelper { + val v1SessionCatalog = catalogManager.v1SessionCatalog + + /** + * If we are resolving database objects (relations, functions, etc.) inside views, we may need to + * expand single or multi-part identifiers with the current catalog and namespace of when the + * view was created. + */ + def expandIdentifier(nameParts: Seq[String]): Seq[String] = { + if (!isResolvingView || isReferredTempViewName(nameParts)) { + return nameParts + } + + if (nameParts.length == 1) { + AnalysisContext.get.catalogAndNamespace :+ nameParts.head + } else if (catalogManager.isCatalogRegistered(nameParts.head)) { + nameParts + } else { + AnalysisContext.get.catalogAndNamespace.head +: nameParts + } + } + + /** + * Lookup temporary view by `identifier`. Returns `None` if the view wasn't found. + */ + def lookupTempView(identifier: Seq[String]): Option[TemporaryViewRelation] = { + // We are resolving a view and this name is not a temp view when that view was created. We + // return None earlier here. + if (isResolvingView && !isReferredTempViewName(identifier)) { + return None + } + + v1SessionCatalog.getRawLocalOrGlobalTempView(identifier) + } + + /** + * Resolve relation `u` to v1 relation if it's a v1 table from the session catalog, or to v2 + * relation. This is for resolving DML commands and SELECT queries. + */ + def resolveRelation( + u: UnresolvedRelation, + timeTravelSpec: Option[TimeTravelSpec] = None): Option[LogicalPlan] = { + val timeTravelSpecFromOptions = TimeTravelSpec.fromOptions( + u.options, + conf.getConf(SQLConf.TIME_TRAVEL_TIMESTAMP_KEY), + conf.getConf(SQLConf.TIME_TRAVEL_VERSION_KEY), + conf.sessionLocalTimeZone + ) + if (timeTravelSpec.nonEmpty && timeTravelSpecFromOptions.nonEmpty) { + throw new AnalysisException("MULTIPLE_TIME_TRAVEL_SPEC", Map.empty[String, String]) + } + val finalTimeTravelSpec = timeTravelSpec.orElse(timeTravelSpecFromOptions) + resolveTempView( + u.multipartIdentifier, + u.isStreaming, + finalTimeTravelSpec.isDefined + ).orElse { + expandIdentifier(u.multipartIdentifier) match { + case CatalogAndIdentifier(catalog, ident) => + val key = + ( + (catalog.name +: ident.namespace :+ ident.name).toImmutableArraySeq, + finalTimeTravelSpec + ) + AnalysisContext.get.relationCache + .get(key) + .map { cache => + val cachedRelation = cache.transform { + case multi: MultiInstanceRelation => + val newRelation = multi.newInstance() + newRelation.copyTagsFrom(multi) + newRelation + } + u.getTagValue(LogicalPlan.PLAN_ID_TAG) + .map { planId => + val cachedConnectRelation = cachedRelation.clone() + cachedConnectRelation.setTagValue(LogicalPlan.PLAN_ID_TAG, planId) + cachedConnectRelation + } + .getOrElse(cachedRelation) + } + .orElse { + val writePrivilegesString = + Option(u.options.get(UnresolvedRelation.REQUIRED_WRITE_PRIVILEGES)) + val table = + CatalogV2Util.loadTable(catalog, ident, finalTimeTravelSpec, writePrivilegesString) + val loaded = createRelation( + catalog, + ident, + table, + u.clearWritePrivileges.options, + u.isStreaming + ) + loaded.foreach(AnalysisContext.get.relationCache.update(key, _)) + u.getTagValue(LogicalPlan.PLAN_ID_TAG) + .map { planId => + loaded.map { loadedRelation => + val loadedConnectRelation = loadedRelation.clone() + loadedConnectRelation.setTagValue(LogicalPlan.PLAN_ID_TAG, planId) + loadedConnectRelation + } + } + .getOrElse(loaded) + } + case _ => None + } + } + } + + private def createRelation( + catalog: CatalogPlugin, + ident: Identifier, + table: Option[Table], + options: CaseInsensitiveStringMap, + isStreaming: Boolean): Option[LogicalPlan] = { + table.map { + // To utilize this code path to execute V1 commands, e.g. INSERT, + // either it must be session catalog, or tracksPartitionsInCatalog + // must be false so it does not require use catalog to manage partitions. + // Obviously we cannot execute V1Table by V1 code path if the table + // is not from session catalog and the table still requires its catalog + // to manage partitions. + case v1Table: V1Table + if CatalogV2Util.isSessionCatalog(catalog) + || !v1Table.catalogTable.tracksPartitionsInCatalog => + if (isStreaming) { + if (v1Table.v1Table.tableType == CatalogTableType.VIEW) { + throw QueryCompilationErrors.permanentViewNotSupportedByStreamingReadingAPIError( + ident.quoted + ) + } + SubqueryAlias( + catalog.name +: ident.asMultipartIdentifier, + UnresolvedCatalogRelation(v1Table.v1Table, options, isStreaming = true) + ) + } else { + v1SessionCatalog.getRelation(v1Table.v1Table, options) + } + + case table => + if (isStreaming) { + val v1Fallback = table match { + case withFallback: V2TableWithV1Fallback => + Some(UnresolvedCatalogRelation(withFallback.v1Table, isStreaming = true)) + case _ => None + } + SubqueryAlias( + catalog.name +: ident.asMultipartIdentifier, + StreamingRelationV2( + None, + table.name, + table, + options, + table.columns.toAttributes, + Some(catalog), + Some(ident), + v1Fallback + ) + ) + } else { + SubqueryAlias( + catalog.name +: ident.asMultipartIdentifier, + DataSourceV2Relation.create(table, Some(catalog), Some(ident), options) + ) + } + } + } + + private def resolveTempView( + identifier: Seq[String], + isStreaming: Boolean = false, + isTimeTravel: Boolean = false): Option[LogicalPlan] = { + lookupTempView(identifier).map { v => + val tempViewPlan = v1SessionCatalog.getTempViewRelation(v) + if (isStreaming && !tempViewPlan.isStreaming) { + throw QueryCompilationErrors.readNonStreamingTempViewError(identifier.quoted) + } + if (isTimeTravel) { + throw QueryCompilationErrors.timeTravelUnsupportedError(toSQLId(identifier)) + } + tempViewPlan + } + } + + private def isResolvingView: Boolean = AnalysisContext.get.catalogAndNamespace.nonEmpty + + private def isReferredTempViewName(nameParts: Seq[String]): Boolean = { + val resolver = conf.resolver + AnalysisContext.get.referredTempViewNames.exists { n => + (n.length == nameParts.length) && n.zip(nameParts).forall { + case (a, b) => resolver(a, b) + } + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala index 2642b4a1c5daa..0f9b93cc2986d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala @@ -36,7 +36,7 @@ class ResolveDataFrameDropColumns(val catalogManager: CatalogManager) // df.drop(col("non-existing-column")) val dropped = d.dropList.map { case u: UnresolvedAttribute => - resolveExpressionByPlanChildren(u, d.child) + resolveExpressionByPlanChildren(u, d) case e => e } val remaining = d.child.output.filterNot(attr => dropped.exists(_.semanticEquals(attr))) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index d7d53230470d9..f2f86a90d5172 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -87,7 +87,7 @@ object ExpressionEncoder { } constructProjection(row).get(0, anyObjectType).asInstanceOf[T] } catch { - case e: SparkRuntimeException if e.getErrorClass == "NOT_NULL_ASSERT_VIOLATION" => + case e: SparkRuntimeException if e.getCondition == "NOT_NULL_ASSERT_VIOLATION" => throw e case e: Exception => throw QueryExecutionErrors.expressionDecodingError(e, expressions) @@ -115,7 +115,7 @@ object ExpressionEncoder { inputRow(0) = t extractProjection(inputRow) } catch { - case e: SparkRuntimeException if e.getErrorClass == "NOT_NULL_ASSERT_VIOLATION" => + case e: SparkRuntimeException if e.getCondition == "NOT_NULL_ASSERT_VIOLATION" => throw e case e: Exception => throw QueryExecutionErrors.expressionEncodingError(e, expressions) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala index 08cb03edb78b6..38b927f5bbf38 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala @@ -32,11 +32,11 @@ import org.apache.spark.sql.internal.types.{AbstractMapType, StringTypeWithCaseA import org.apache.spark.sql.types.{DataType, MapType, StringType, StructType, VariantType} import org.apache.spark.unsafe.types.UTF8String -object ExprUtils extends QueryErrorsBase { +object ExprUtils extends EvalHelper with QueryErrorsBase { def evalTypeExpr(exp: Expression): DataType = { if (exp.foldable) { - exp.eval() match { + prepareForEval(exp).eval() match { case s: UTF8String if s != null => val dataType = DataType.parseTypeWithFallback( s.toString, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index de15ec43c4f31..6a57ba2aaa569 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -383,10 +383,10 @@ abstract class Expression extends TreeNode[Expression] { trait FoldableUnevaluable extends Expression { override def foldable: Boolean = true - final override def eval(input: InternalRow = null): Any = + override def eval(input: InternalRow = null): Any = throw QueryExecutionErrors.cannotEvaluateExpressionError(this) - final override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = throw QueryExecutionErrors.cannotGenerateCodeForExpressionError(this) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala index 433f8500fab1f..04d31b5797819 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala @@ -17,7 +17,11 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.SparkException +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.trees.UnaryLike +import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLExpr import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.types.{DataType, IntegerType} @@ -37,8 +41,21 @@ import org.apache.spark.sql.types.{DataType, IntegerType} abstract class PartitionTransformExpression extends Expression with Unevaluable with UnaryLike[Expression] { override def nullable: Boolean = true -} + override def eval(input: InternalRow): Any = + throw new SparkException( + errorClass = "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY", + messageParameters = Map("expression" -> toSQLExpr(this)), + cause = null + ) + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = + throw new SparkException( + errorClass = "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY", + messageParameters = Map("expression" -> toSQLExpr(this)), + cause = null + ) +} /** * Expression for the v2 partition transform years. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala index 220920a5a3198..d14c8cb675387 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.connector.catalog.{FunctionCatalog, Identifier} import org.apache.spark.sql.connector.catalog.functions._ import org.apache.spark.sql.connector.catalog.functions.ScalarFunction.MAGIC_METHOD_NAME import org.apache.spark.sql.connector.expressions.{BucketTransform, Expression => V2Expression, FieldReference, IdentityTransform, Literal => V2Literal, NamedReference, NamedTransform, NullOrdering => V2NullOrdering, SortDirection => V2SortDirection, SortOrder => V2SortOrder, SortValue, Transform} +import org.apache.spark.sql.errors.DataTypeErrors.toSQLId import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.types._ import org.apache.spark.util.ArrayImplicits._ @@ -182,8 +183,8 @@ object V2ExpressionUtils extends SQLConfHelper with Logging { ApplyFunctionExpression(scalarFunc, arguments) case _ => throw new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_3055", - messageParameters = Map("scalarFunc" -> scalarFunc.name())) + errorClass = "SCALAR_FUNCTION_NOT_FULLY_IMPLEMENTED", + messageParameters = Map("scalarFunc" -> toSQLId(scalarFunc.name()))) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala index 0a4882bfada17..3270c6e87e2cd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.trees.UnaryLike -import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData, TypeUtils} +import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData, TypeUtils, UnsafeRowUtils} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase} import org.apache.spark.sql.types._ import org.apache.spark.util.BoundedPriorityQueue @@ -145,6 +145,7 @@ case class CollectList( """, group = "agg_funcs", since = "2.0.0") +// TODO: Make CollectSet collation aware case class CollectSet( child: Expression, mutableAggBufferOffset: Int = 0, @@ -178,14 +179,15 @@ case class CollectSet( } override def checkInputDataTypes(): TypeCheckResult = { - if (!child.dataType.existsRecursively(_.isInstanceOf[MapType])) { + if (!child.dataType.existsRecursively(_.isInstanceOf[MapType]) && + UnsafeRowUtils.isBinaryStable(child.dataType)) { TypeCheckResult.TypeCheckSuccess } else { DataTypeMismatch( errorSubClass = "UNSUPPORTED_INPUT_TYPE", messageParameters = Map( "functionName" -> toSQLId(prettyName), - "dataType" -> toSQLType(MapType) + "dataType" -> (s"${toSQLType(MapType)} " + "or \"COLLATED STRING\"") ) ) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index d8ba1fe840bd0..497fdc0936267 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -298,7 +298,7 @@ abstract class BinaryArithmetic extends BinaryOperator case "+" => "try_add" case "-" => "try_subtract" case "*" => "try_multiply" - case _ => "" + case _ => "unknown_function" } val overflowCheck = if (failOnError) { val javaType = CodeGenerator.boxedType(dataType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index c091d51fc177f..bb54749126860 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -27,6 +27,7 @@ import org.apache.spark.SparkException.internalError import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion, UnresolvedAttribute, UnresolvedSeed} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch +import org.apache.spark.sql.catalyst.expressions.KnownNotContainsNull import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke @@ -5330,15 +5331,12 @@ case class ArrayCompact(child: Expression) child.dataType.asInstanceOf[ArrayType].elementType, true) lazy val lambda = LambdaFunction(isNotNull(lv), Seq(lv)) - override lazy val replacement: Expression = ArrayFilter(child, lambda) + override lazy val replacement: Expression = KnownNotContainsNull(ArrayFilter(child, lambda)) override def inputTypes: Seq[AbstractDataType] = Seq(ArrayType) override def prettyName: String = "array_compact" - override def dataType: ArrayType = - child.dataType.asInstanceOf[ArrayType].copy(containsNull = false) - override protected def withNewChildInternal(newChild: Expression): ArrayCompact = copy(child = newChild) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraintExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraintExpressions.scala index 75d912633a0fc..f05db0b090c90 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraintExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraintExpressions.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, FalseLiteral} -import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types.{ArrayType, DataType} trait TaggingExpression extends UnaryExpression { override def nullable: Boolean = child.nullable @@ -52,6 +52,17 @@ case class KnownNotNull(child: Expression) extends TaggingExpression { copy(child = newChild) } +case class KnownNotContainsNull(child: Expression) extends TaggingExpression { + override def dataType: DataType = + child.dataType.asInstanceOf[ArrayType].copy(containsNull = false) + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = + child.genCode(ctx) + + override protected def withNewChildInternal(newChild: Expression): KnownNotContainsNull = + copy(child = newChild) +} + case class KnownFloatingPointNormalized(child: Expression) extends TaggingExpression { override protected def withNewChildInternal(newChild: Expression): KnownFloatingPointNormalized = copy(child = newChild) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala index 3a667f370428e..7128190902550 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala @@ -415,7 +415,7 @@ abstract class HashExpression[E] extends Expression { protected def genHashString( ctx: CodegenContext, stringType: StringType, input: String, result: String): String = { - if (stringType.supportsBinaryEquality) { + if (stringType.supportsBinaryEquality && !stringType.usesTrimCollation) { val baseObject = s"$input.getBaseObject()" val baseOffset = s"$input.getBaseOffset()" val numBytes = s"$input.numBytes()" @@ -566,7 +566,7 @@ abstract class InterpretedHashFunction { hashUnsafeBytes(a, Platform.BYTE_ARRAY_OFFSET, a.length, seed) case s: UTF8String => val st = dataType.asInstanceOf[StringType] - if (st.supportsBinaryEquality) { + if (st.supportsBinaryEquality && !st.usesTrimCollation) { hashUnsafeBytes(s.getBaseObject, s.getBaseOffset, s.numBytes(), seed) } else { val stringHash = CollationFactory @@ -817,7 +817,7 @@ case class HiveHash(children: Seq[Expression]) extends HashExpression[Int] { override protected def genHashString( ctx: CodegenContext, stringType: StringType, input: String, result: String): String = { - if (stringType.supportsBinaryEquality) { + if (stringType.supportsBinaryEquality && !stringType.usesTrimCollation) { val baseObject = s"$input.getBaseObject()" val baseOffset = s"$input.getBaseOffset()" val numBytes = s"$input.numBytes()" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala new file mode 100644 index 0000000000000..6291e62304a38 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala @@ -0,0 +1,113 @@ +/* + * 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.catalyst.expressions.json + +import com.fasterxml.jackson.core.JsonFactory + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.ExprUtils +import org.apache.spark.sql.catalyst.expressions.variant.VariantExpressionEvalUtils +import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JsonInferSchema, JSONOptions} +import org.apache.spark.sql.catalyst.util.{FailFastMode, FailureSafeParser, PermissiveMode} +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType, VariantType} +import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.util.Utils + +object JsonExpressionEvalUtils { + + def schemaOfJson( + jsonFactory: JsonFactory, + jsonOptions: JSONOptions, + jsonInferSchema: JsonInferSchema, + json: UTF8String): UTF8String = { + val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, json)) { parser => + parser.nextToken() + // To match with schema inference from JSON datasource. + jsonInferSchema.inferField(parser) match { + case st: StructType => + jsonInferSchema.canonicalizeType(st, jsonOptions).getOrElse(StructType(Nil)) + case at: ArrayType if at.elementType.isInstanceOf[StructType] => + jsonInferSchema + .canonicalizeType(at.elementType, jsonOptions) + .map(ArrayType(_, containsNull = at.containsNull)) + .getOrElse(ArrayType(StructType(Nil), containsNull = at.containsNull)) + case other: DataType => + jsonInferSchema.canonicalizeType(other, jsonOptions).getOrElse( + SQLConf.get.defaultStringType) + } + } + + UTF8String.fromString(dt.sql) + } +} + +class JsonToStructsEvaluator( + options: Map[String, String], + nullableSchema: DataType, + nameOfCorruptRecord: String, + timeZoneId: Option[String], + variantAllowDuplicateKeys: Boolean) extends Serializable { + + // This converts parsed rows to the desired output by the given schema. + @transient + private lazy val converter = nullableSchema match { + case _: StructType => + (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next() else null + case _: ArrayType => + (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next().getArray(0) else null + case _: MapType => + (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next().getMap(0) else null + } + + @transient + private lazy val parser = { + val parsedOptions = new JSONOptions(options, timeZoneId.get, nameOfCorruptRecord) + val mode = parsedOptions.parseMode + if (mode != PermissiveMode && mode != FailFastMode) { + throw QueryCompilationErrors.parseModeUnsupportedError("from_json", mode) + } + val (parserSchema, actualSchema) = nullableSchema match { + case s: StructType => + ExprUtils.verifyColumnNameOfCorruptRecord(s, parsedOptions.columnNameOfCorruptRecord) + (s, StructType(s.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord))) + case other => + (StructType(Array(StructField("value", other))), other) + } + + val rawParser = new JacksonParser(actualSchema, parsedOptions, allowArrayAsStructs = false) + val createParser = CreateJacksonParser.utf8String _ + + new FailureSafeParser[UTF8String]( + input => rawParser.parse(input, createParser, identity[UTF8String]), + mode, + parserSchema, + parsedOptions.columnNameOfCorruptRecord) + } + + final def evaluate(json: UTF8String): Any = { + if (json == null) return null + nullableSchema match { + case _: VariantType => + VariantExpressionEvalUtils.parseJson(json, + allowDuplicateKeys = variantAllowDuplicateKeys) + case _ => + converter(parser.parse(json)) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index bdcf3f0c1eeab..6eef3d6f9d7df 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.expressions import java.io._ -import scala.collection.mutable.ArrayBuffer import scala.util.parsing.combinator.RegexParsers import com.fasterxml.jackson.core._ @@ -31,7 +30,8 @@ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, CodegenFallback, ExprCode} import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper -import org.apache.spark.sql.catalyst.expressions.variant.VariantExpressionEvalUtils +import org.apache.spark.sql.catalyst.expressions.json.{JsonExpressionEvalUtils, JsonExpressionUtils, JsonToStructsEvaluator} +import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke import org.apache.spark.sql.catalyst.json._ import org.apache.spark.sql.catalyst.trees.TreePattern.{JSON_TO_STRUCT, TreePattern} import org.apache.spark.sql.catalyst.util._ @@ -638,7 +638,6 @@ case class JsonToStructs( variantAllowDuplicateKeys: Boolean = SQLConf.get.getConf(SQLConf.VARIANT_ALLOW_DUPLICATE_KEYS)) extends UnaryExpression with TimeZoneAwareExpression - with CodegenFallback with ExpectsInputTypes with NullIntolerant with QueryErrorsBase { @@ -646,7 +645,7 @@ case class JsonToStructs( // The JSON input data might be missing certain fields. We force the nullability // of the user-provided schema to avoid data corruptions. In particular, the parquet-mr encoder // can generate incorrect files if values are missing in columns declared as non-nullable. - val nullableSchema = schema.asNullable + private val nullableSchema: DataType = schema.asNullable override def nullable: Boolean = true @@ -679,53 +678,35 @@ case class JsonToStructs( messageParameters = Map("schema" -> toSQLType(nullableSchema))) } - // This converts parsed rows to the desired output by the given schema. - @transient - lazy val converter = nullableSchema match { - case _: StructType => - (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next() else null - case _: ArrayType => - (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next().getArray(0) else null - case _: MapType => - (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next().getMap(0) else null - } - - val nameOfCorruptRecord = SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD) - @transient lazy val parser = { - val parsedOptions = new JSONOptions(options, timeZoneId.get, nameOfCorruptRecord) - val mode = parsedOptions.parseMode - if (mode != PermissiveMode && mode != FailFastMode) { - throw QueryCompilationErrors.parseModeUnsupportedError("from_json", mode) - } - val (parserSchema, actualSchema) = nullableSchema match { - case s: StructType => - ExprUtils.verifyColumnNameOfCorruptRecord(s, parsedOptions.columnNameOfCorruptRecord) - (s, StructType(s.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord))) - case other => - (StructType(Array(StructField("value", other))), other) - } - - val rawParser = new JacksonParser(actualSchema, parsedOptions, allowArrayAsStructs = false) - val createParser = CreateJacksonParser.utf8String _ - - new FailureSafeParser[UTF8String]( - input => rawParser.parse(input, createParser, identity[UTF8String]), - mode, - parserSchema, - parsedOptions.columnNameOfCorruptRecord) - } - override def dataType: DataType = nullableSchema override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = copy(timeZoneId = Option(timeZoneId)) - override def nullSafeEval(json: Any): Any = nullableSchema match { - case _: VariantType => - VariantExpressionEvalUtils.parseJson(json.asInstanceOf[UTF8String], - allowDuplicateKeys = variantAllowDuplicateKeys) - case _ => - converter(parser.parse(json.asInstanceOf[UTF8String])) + @transient + private val nameOfCorruptRecord = SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD) + + @transient + private lazy val evaluator = new JsonToStructsEvaluator( + options, nullableSchema, nameOfCorruptRecord, timeZoneId, variantAllowDuplicateKeys) + + override def nullSafeEval(json: Any): Any = evaluator.evaluate(json.asInstanceOf[UTF8String]) + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val refEvaluator = ctx.addReferenceObj("evaluator", evaluator) + val eval = child.genCode(ctx) + val resultType = CodeGenerator.boxedType(dataType) + val resultTerm = ctx.freshName("result") + ev.copy(code = + code""" + |${eval.code} + |$resultType $resultTerm = ($resultType) $refEvaluator.evaluate(${eval.value}); + |boolean ${ev.isNull} = $resultTerm == null; + |${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; + |if (!${ev.isNull}) { + | ${ev.value} = $resultTerm; + |} + |""".stripMargin) } override def inputTypes: Seq[AbstractDataType] = StringTypeWithCaseAccentSensitivity :: Nil @@ -877,7 +858,9 @@ case class StructsToJson( case class SchemaOfJson( child: Expression, options: Map[String, String]) - extends UnaryExpression with CodegenFallback with QueryErrorsBase { + extends UnaryExpression + with RuntimeReplaceable + with QueryErrorsBase { def this(child: Expression) = this(child, Map.empty[String, String]) @@ -918,26 +901,20 @@ case class SchemaOfJson( } } - override def eval(v: InternalRow): Any = { - val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, json)) { parser => - parser.nextToken() - // To match with schema inference from JSON datasource. - jsonInferSchema.inferField(parser) match { - case st: StructType => - jsonInferSchema.canonicalizeType(st, jsonOptions).getOrElse(StructType(Nil)) - case at: ArrayType if at.elementType.isInstanceOf[StructType] => - jsonInferSchema - .canonicalizeType(at.elementType, jsonOptions) - .map(ArrayType(_, containsNull = at.containsNull)) - .getOrElse(ArrayType(StructType(Nil), containsNull = at.containsNull)) - case other: DataType => - jsonInferSchema.canonicalizeType(other, jsonOptions).getOrElse( - SQLConf.get.defaultStringType) - } - } - - UTF8String.fromString(dt.sql) - } + @transient private lazy val jsonFactoryObjectType = ObjectType(classOf[JsonFactory]) + @transient private lazy val jsonOptionsObjectType = ObjectType(classOf[JSONOptions]) + @transient private lazy val jsonInferSchemaObjectType = ObjectType(classOf[JsonInferSchema]) + + override def replacement: Expression = StaticInvoke( + JsonExpressionEvalUtils.getClass, + dataType, + "schemaOfJson", + Seq(Literal(jsonFactory, jsonFactoryObjectType), + Literal(jsonOptions, jsonOptionsObjectType), + Literal(jsonInferSchema, jsonInferSchemaObjectType), + child), + Seq(jsonFactoryObjectType, jsonOptionsObjectType, jsonInferSchemaObjectType, child.dataType) + ) override def prettyName: String = "schema_of_json" @@ -967,54 +944,26 @@ case class SchemaOfJson( group = "json_funcs", since = "3.1.0" ) -case class LengthOfJsonArray(child: Expression) extends UnaryExpression - with CodegenFallback with ExpectsInputTypes { +case class LengthOfJsonArray(child: Expression) + extends UnaryExpression + with ExpectsInputTypes + with RuntimeReplaceable { override def inputTypes: Seq[AbstractDataType] = Seq(StringTypeWithCaseAccentSensitivity) override def dataType: DataType = IntegerType override def nullable: Boolean = true override def prettyName: String = "json_array_length" - override def eval(input: InternalRow): Any = { - val json = child.eval(input).asInstanceOf[UTF8String] - // return null for null input - if (json == null) { - return null - } - - try { - Utils.tryWithResource(CreateJacksonParser.utf8String(SharedFactory.jsonFactory, json)) { - parser => { - // return null if null array is encountered. - if (parser.nextToken() == null) { - return null - } - // Parse the array to compute its length. - parseCounter(parser, input) - } - } - } catch { - case _: JsonProcessingException | _: IOException => null - } - } - - private def parseCounter(parser: JsonParser, input: InternalRow): Any = { - var length = 0 - // Only JSON array are supported for this function. - if (parser.currentToken != JsonToken.START_ARRAY) { - return null - } - // Keep traversing until the end of JSON array - while(parser.nextToken() != JsonToken.END_ARRAY) { - length += 1 - // skip all the child of inner object or array - parser.skipChildren() - } - length - } - override protected def withNewChildInternal(newChild: Expression): LengthOfJsonArray = copy(child = newChild) + + override def replacement: Expression = StaticInvoke( + classOf[JsonExpressionUtils], + dataType, + "lengthOfJsonArray", + Seq(child), + inputTypes + ) } /** @@ -1040,50 +989,23 @@ case class LengthOfJsonArray(child: Expression) extends UnaryExpression group = "json_funcs", since = "3.1.0" ) -case class JsonObjectKeys(child: Expression) extends UnaryExpression with CodegenFallback - with ExpectsInputTypes { +case class JsonObjectKeys(child: Expression) + extends UnaryExpression + with ExpectsInputTypes + with RuntimeReplaceable { override def inputTypes: Seq[AbstractDataType] = Seq(StringTypeWithCaseAccentSensitivity) override def dataType: DataType = ArrayType(SQLConf.get.defaultStringType) override def nullable: Boolean = true override def prettyName: String = "json_object_keys" - override def eval(input: InternalRow): Any = { - val json = child.eval(input).asInstanceOf[UTF8String] - // return null for `NULL` input - if(json == null) { - return null - } - - try { - Utils.tryWithResource(CreateJacksonParser.utf8String(SharedFactory.jsonFactory, json)) { - parser => { - // return null if an empty string or any other valid JSON string is encountered - if (parser.nextToken() == null || parser.currentToken() != JsonToken.START_OBJECT) { - return null - } - // Parse the JSON string to get all the keys of outermost JSON object - getJsonKeys(parser, input) - } - } - } catch { - case _: JsonProcessingException | _: IOException => null - } - } - - private def getJsonKeys(parser: JsonParser, input: InternalRow): GenericArrayData = { - val arrayBufferOfKeys = ArrayBuffer.empty[UTF8String] - - // traverse until the end of input and ensure it returns valid key - while(parser.nextValue() != null && parser.currentName() != null) { - // add current fieldName to the ArrayBuffer - arrayBufferOfKeys += UTF8String.fromString(parser.currentName) - - // skip all the children of inner object or array - parser.skipChildren() - } - new GenericArrayData(arrayBufferOfKeys.toArray[Any]) - } + override def replacement: Expression = StaticInvoke( + classOf[JsonExpressionUtils], + dataType, + "jsonObjectKeys", + Seq(child), + inputTypes + ) override protected def withNewChildInternal(newChild: Expression): JsonObjectKeys = copy(child = newChild) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala index 3cec83facd01d..16bdaa1f7f708 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala @@ -21,7 +21,7 @@ import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult, UnresolvedSeed} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch -import org.apache.spark.sql.catalyst.expressions.ExpectsInputTypes.{ordinalNumber, toSQLExpr, toSQLType} +import org.apache.spark.sql.catalyst.expressions.ExpectsInputTypes.{ordinalNumber, toSQLExpr, toSQLId, toSQLType} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, FalseLiteral} import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike, UnaryLike} @@ -263,7 +263,7 @@ case class Uniform(min: Expression, max: Expression, seedExpression: Expression, result = DataTypeMismatch( errorSubClass = "NON_FOLDABLE_INPUT", messageParameters = Map( - "inputName" -> name, + "inputName" -> toSQLId(name), "inputType" -> requiredType, "inputExpr" -> toSQLExpr(expr))) } else expr.dataType match { @@ -374,14 +374,14 @@ case class RandStr( var result: TypeCheckResult = TypeCheckResult.TypeCheckSuccess def requiredType = "INT or SMALLINT" Seq((length, "length", 0), - (seedExpression, "seedExpression", 1)).foreach { + (seedExpression, "seed", 1)).foreach { case (expr: Expression, name: String, index: Int) => if (result == TypeCheckResult.TypeCheckSuccess) { if (!expr.foldable) { result = DataTypeMismatch( errorSubClass = "NON_FOLDABLE_INPUT", messageParameters = Map( - "inputName" -> name, + "inputName" -> toSQLId(name), "inputType" -> requiredType, "inputExpr" -> toSQLExpr(expr))) } else expr.dataType match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala index d982e1f19da0c..9c291634401ee 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala @@ -81,7 +81,8 @@ class JsonInferSchema(options: JSONOptions) extends Serializable with Logging { */ def infer[T]( json: RDD[T], - createParser: (JsonFactory, T) => JsonParser): StructType = { + createParser: (JsonFactory, T) => JsonParser, + isReadFile: Boolean = false): StructType = { val parseMode = options.parseMode val columnNameOfCorruptRecord = options.columnNameOfCorruptRecord @@ -96,6 +97,9 @@ class JsonInferSchema(options: JSONOptions) extends Serializable with Logging { Some(inferField(parser)) } } catch { + // If we are not reading from files but hit `RuntimeException`, it means corrupted record. + case e: RuntimeException if !isReadFile => + handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, e) case e @ (_: JsonProcessingException | _: MalformedInputException) => handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, e) case e: CharConversionException if options.encoding.isEmpty => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 1601d798283c9..c0cd976b9e9b2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -260,19 +260,32 @@ object ReorderAssociativeOperator extends Rule[LogicalPlan] { q.transformExpressionsDownWithPruning(_.containsPattern(BINARY_ARITHMETIC)) { case a @ Add(_, _, f) if a.deterministic && a.dataType.isInstanceOf[IntegralType] => val (foldables, others) = flattenAdd(a, groupingExpressionSet).partition(_.foldable) - if (foldables.size > 1) { + if (foldables.nonEmpty) { val foldableExpr = foldables.reduce((x, y) => Add(x, y, f)) - val c = Literal.create(foldableExpr.eval(EmptyRow), a.dataType) - if (others.isEmpty) c else Add(others.reduce((x, y) => Add(x, y, f)), c, f) + val foldableValue = foldableExpr.eval(EmptyRow) + if (others.isEmpty) { + Literal.create(foldableValue, a.dataType) + } else if (foldableValue == 0) { + others.reduce((x, y) => Add(x, y, f)) + } else { + Add(others.reduce((x, y) => Add(x, y, f)), Literal.create(foldableValue, a.dataType), f) + } } else { a } case m @ Multiply(_, _, f) if m.deterministic && m.dataType.isInstanceOf[IntegralType] => val (foldables, others) = flattenMultiply(m, groupingExpressionSet).partition(_.foldable) - if (foldables.size > 1) { + if (foldables.nonEmpty) { val foldableExpr = foldables.reduce((x, y) => Multiply(x, y, f)) - val c = Literal.create(foldableExpr.eval(EmptyRow), m.dataType) - if (others.isEmpty) c else Multiply(others.reduce((x, y) => Multiply(x, y, f)), c, f) + val foldableValue = foldableExpr.eval(EmptyRow) + if (others.isEmpty || (foldableValue == 0 && !m.nullable)) { + Literal.create(foldableValue, m.dataType) + } else if (foldableValue == 1) { + others.reduce((x, y) => Multiply(x, y, f)) + } else { + Multiply(others.reduce((x, y) => Multiply(x, y, f)), + Literal.create(foldableValue, m.dataType), f) + } } else { m } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index c9150b8a26100..3ecb680cf6427 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3256,7 +3256,7 @@ class AstBuilder extends DataTypeAstBuilder } catch { case e: SparkArithmeticException => throw new ParseException( - errorClass = e.getErrorClass, + errorClass = e.getCondition, messageParameters = e.getMessageParameters.asScala.toMap, ctx) } @@ -3552,7 +3552,7 @@ class AstBuilder extends DataTypeAstBuilder // Keep error class of SparkIllegalArgumentExceptions and enrich it with query context case se: SparkIllegalArgumentException => val pe = new ParseException( - errorClass = se.getErrorClass, + errorClass = se.getCondition, messageParameters = se.getMessageParameters.asScala.toMap, ctx) pe.setStackTrace(se.getStackTrace) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala index 46f14876be363..8d88b05546ed2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala @@ -127,7 +127,7 @@ object GeneratedColumn { } catch { case ex: AnalysisException => // Improve error message if possible - if (ex.getErrorClass == "UNRESOLVED_COLUMN.WITH_SUGGESTION") { + if (ex.getCondition == "UNRESOLVED_COLUMN.WITH_SUGGESTION") { ex.messageParameters.get("objectName").foreach { unresolvedCol => val resolver = SQLConf.get.resolver // Whether `col` = `unresolvedCol` taking into account case-sensitivity @@ -144,7 +144,7 @@ object GeneratedColumn { } } } - if (ex.getErrorClass == "UNRESOLVED_ROUTINE") { + if (ex.getCondition == "UNRESOLVED_ROUTINE") { // Cannot resolve function using built-in catalog ex.messageParameters.get("routineName").foreach { fnName => throw unsupportedExpressionError(s"failed to resolve $fnName to a built-in function") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala index e296b5be6134b..118dd92c3ed54 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala @@ -205,7 +205,9 @@ object UnsafeRowUtils { * can lead to rows being semantically equal even though their binary representations differ). */ def isBinaryStable(dataType: DataType): Boolean = !dataType.existsRecursively { - case st: StringType => !CollationFactory.fetchCollation(st.collationId).supportsBinaryEquality + case st: StringType => + val collation = CollationFactory.fetchCollation(st.collationId) + (!collation.supportsBinaryEquality) case _ => false } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 0e02e4249addd..9dc15c4a1b78d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -1691,12 +1691,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat messageParameters = Map("className" -> className)) } - def cannotSaveIntervalIntoExternalStorageError(): Throwable = { - new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1136", - messageParameters = Map.empty) - } - def cannotResolveAttributeError(name: String, outputStr: String): Throwable = { new AnalysisException( errorClass = "_LEGACY_ERROR_TEMP_1137", @@ -1729,10 +1723,10 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat "sourceNames" -> sourceNames.mkString(", "))) } - def writeEmptySchemasUnsupportedByDataSourceError(): Throwable = { + def writeEmptySchemasUnsupportedByDataSourceError(format: String): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1142", - messageParameters = Map.empty) + errorClass = "EMPTY_SCHEMA_NOT_SUPPORTED_FOR_DATASOURCE", + messageParameters = Map("format" -> format)) } def insertMismatchedColumnNumberError( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala index 301880f1bfc61..ebcc98a3af27a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala @@ -870,7 +870,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE def cannotRemoveReservedPropertyError(property: String): SparkUnsupportedOperationException = { new SparkUnsupportedOperationException( - errorClass = "_LEGACY_ERROR_TEMP_2069", + errorClass = "CANNOT_REMOVE_RESERVED_PROPERTY", messageParameters = Map("property" -> property)) } @@ -1112,7 +1112,8 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE def cannotAcquireMemoryToBuildUnsafeHashedRelationError(): Throwable = { new SparkOutOfMemoryError( - "_LEGACY_ERROR_TEMP_2107") + "_LEGACY_ERROR_TEMP_2107", + new java.util.HashMap[String, String]()) } def rowLargerThan256MUnsupportedError(): SparkUnsupportedOperationException = { @@ -1257,6 +1258,12 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE "dataType" -> toSQLType(dataType))) } + def wrongDatatypeInSomeRows(pos: Int, dataType: DataType): SparkSQLException = { + new SparkSQLException( + errorClass = "COLUMN_ARRAY_ELEMENT_TYPE_MISMATCH", + messageParameters = Map("pos" -> pos.toString(), "type" -> toSQLType(dataType))) + } + def rootConverterReturnNullError(): SparkRuntimeException = { new SparkRuntimeException( errorClass = "INVALID_JSON_ROOT_FIELD", @@ -2275,7 +2282,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE def unsupportedUpdateColumnNullabilityError(): SparkSQLFeatureNotSupportedException = { new SparkSQLFeatureNotSupportedException( - errorClass = "_LEGACY_ERROR_TEMP_2271", + errorClass = "UNSUPPORTED_FEATURE.UPDATE_COLUMN_NULLABILITY", messageParameters = Map.empty) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/exceptions/SqlScriptingException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/exceptions/SqlScriptingException.scala index f0c28c95046eb..7602366c71a65 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/exceptions/SqlScriptingException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/exceptions/SqlScriptingException.scala @@ -33,7 +33,7 @@ class SqlScriptingException ( cause) with SparkThrowable { - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 969eee4d912e4..08002887135ce 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -782,7 +782,7 @@ object SQLConf { CollationFactory.fetchCollation(collationName) true } catch { - case e: SparkException if e.getErrorClass == "COLLATION_INVALID_NAME" => false + case e: SparkException if e.getCondition == "COLLATION_INVALID_NAME" => false } }, "DEFAULT_COLLATION", diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala index 1c860e61973c6..ccd9ed209f92a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala @@ -55,7 +55,12 @@ private[sql] object ByteExactNumeric extends ByteIsIntegral with Ordering.ByteOr private[sql] object ShortExactNumeric extends ShortIsIntegral with Ordering.ShortOrdering { - private def checkOverflow(res: Int, x: Short, y: Short, op: String, hint: String): Unit = { + private def checkOverflow( + res: Int, + x: Short, + y: Short, + op: String, + hint: String = "unknown_function"): Unit = { if (res > Short.MaxValue || res < Short.MinValue) { throw QueryExecutionErrors.binaryArithmeticCauseOverflowError(x, op, y, hint) } diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java index 0db155e88aea5..339f16407ae60 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java @@ -80,7 +80,7 @@ public void testLoadWithoutConfig() { SparkException exc = Assertions.assertThrows(CatalogNotFoundException.class, () -> Catalogs.load("missing", conf)); - Assertions.assertEquals(exc.getErrorClass(), "CATALOG_NOT_FOUND"); + Assertions.assertEquals(exc.getCondition(), "CATALOG_NOT_FOUND"); Assertions.assertEquals(exc.getMessageParameters().get("catalogName"), "`missing`"); } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index e23a753dafe8c..8409f454bfb88 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -1832,4 +1832,14 @@ class AnalysisSuite extends AnalysisTest with Matchers { preemptedError.clear() assert(preemptedError.getErrorOpt().isEmpty) } + + test("SPARK-49782: ResolveDataFrameDropColumns rule resolves complex UnresolvedAttribute") { + val function = UnresolvedFunction("trim", Seq(UnresolvedAttribute("i")), isDistinct = false) + val addColumnF = Project(Seq(UnresolvedAttribute("i"), Alias(function, "f")()), testRelation5) + // Drop column "f" via ResolveDataFrameDropColumns rule. + val inputPlan = DataFrameDropColumns(Seq(UnresolvedAttribute("f")), addColumnF) + // The expected Project (root node) should only have column "i". + val expectedPlan = Project(Seq(UnresolvedAttribute("i")), addColumnF).analyze + checkAnalysis(inputPlan, expectedPlan) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index 33b9fb488c94f..71744f4d15105 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -205,7 +205,7 @@ trait AnalysisTest extends PlanTest { assert(e.message.contains(message)) } if (condition.isDefined) { - assert(e.getErrorClass == condition.get) + assert(e.getCondition == condition.get) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index 3e9a93dc743df..6ee19bab5180a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -1133,7 +1133,7 @@ class UnsupportedOperationsSuite extends SparkFunSuite with SQLHelper { } } if (!condition.isEmpty) { - assert(e.getErrorClass == condition) + assert(e.getCondition == condition) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala index e8239c7523948..f3817e4dd1a8b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala @@ -106,7 +106,7 @@ class CSVExprUtilsSuite extends SparkFunSuite { } catch { case e: SparkIllegalArgumentException => assert(separatorStr.isEmpty) - assert(e.getErrorClass === expectedErrorClass.get) + assert(e.getCondition === expectedErrorClass.get) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala index 35a27f41da80a..6bd5b457ea24e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala @@ -173,7 +173,7 @@ class EncoderResolutionSuite extends PlanTest { val exception = intercept[SparkRuntimeException] { fromRow(InternalRow(new GenericArrayData(Array(1, null)))) } - assert(exception.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(exception.getCondition == "NOT_NULL_ASSERT_VIOLATION") } test("the real number of fields doesn't match encoder schema: tuple encoder") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index f73911d344d96..79c6d07d6d218 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -279,7 +279,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { // Check the error class only since the parameters may change depending on how we are running // this test case. val exception = intercept[SparkRuntimeException](toRow(encoder, null)) - assert(exception.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(exception.getCondition == "NOT_NULL_ASSERT_VIOLATION") } test("RowEncoder should validate external type") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala index 6f3890cafd2ac..92ef24bb8ec63 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala @@ -636,7 +636,7 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(murmur3Hash1, interpretedHash1) checkEvaluation(murmur3Hash2, interpretedHash2) - if (CollationFactory.fetchCollation(collation).supportsBinaryEquality) { + if (CollationFactory.fetchCollation(collation).isUtf8BinaryType) { assert(interpretedHash1 != interpretedHash2) } else { assert(interpretedHash1 == interpretedHash2) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala index 3aeb0c882ac3c..891e2d048b7a8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala @@ -64,7 +64,7 @@ object BufferHolderSparkSubmitSuite extends Assertions { val e1 = intercept[SparkIllegalArgumentException] { holder.grow(-1) } - assert(e1.getErrorClass === "_LEGACY_ERROR_TEMP_3198") + assert(e1.getCondition === "_LEGACY_ERROR_TEMP_3198") // while to reuse a buffer may happen, this test checks whether the buffer can be grown holder.grow(ARRAY_MAX / 2) @@ -82,6 +82,6 @@ object BufferHolderSparkSubmitSuite extends Assertions { val e2 = intercept[SparkIllegalArgumentException] { holder.grow(ARRAY_MAX + 1 - holder.totalSize()) } - assert(e2.getErrorClass === "_LEGACY_ERROR_TEMP_3199") + assert(e2.getCondition === "_LEGACY_ERROR_TEMP_3199") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerSuite.scala index 48cdbbe7be539..70a2ae94109fc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerSuite.scala @@ -21,13 +21,13 @@ import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{Add, Alias, AttributeReference, IntegerLiteral, Literal, Multiply, NamedExpression, Remainder} +import org.apache.spark.sql.catalyst.expressions.{Add, Alias, ArrayCompact, AttributeReference, CreateArray, CreateStruct, IntegerLiteral, Literal, MapFromEntries, Multiply, NamedExpression, Remainder} import org.apache.spark.sql.catalyst.expressions.aggregate.Sum import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LocalRelation, LogicalPlan, OneRowRelation, Project} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.IntegerType +import org.apache.spark.sql.types.{ArrayType, IntegerType, MapType, StructField, StructType} /** * A dummy optimizer rule for testing that decrements integer literals until 0. @@ -313,4 +313,25 @@ class OptimizerSuite extends PlanTest { assert(message1.contains("not a valid aggregate expression")) } } + + test("SPARK-49924: Keep containsNull after ArrayCompact replacement") { + val optimizer = new SimpleTestOptimizer() { + override def defaultBatches: Seq[Batch] = + Batch("test", fixedPoint, + ReplaceExpressions) :: Nil + } + + val array1 = ArrayCompact(CreateArray(Literal(1) :: Literal.apply(null) :: Nil, false)) + val plan1 = Project(Alias(array1, "arr")() :: Nil, OneRowRelation()).analyze + val optimized1 = optimizer.execute(plan1) + assert(optimized1.schema === + StructType(StructField("arr", ArrayType(IntegerType, false), false) :: Nil)) + + val struct = CreateStruct(Literal(1) :: Literal(2) :: Nil) + val array2 = ArrayCompact(CreateArray(struct :: Literal.apply(null) :: Nil, false)) + val plan2 = Project(Alias(MapFromEntries(array2), "map")() :: Nil, OneRowRelation()).analyze + val optimized2 = optimizer.execute(plan2) + assert(optimized2.schema === + StructType(StructField("map", MapType(IntegerType, IntegerType, false), false) :: Nil)) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala index f4b2fce74dc49..9090e0c7fc104 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.Count import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor @@ -74,4 +75,35 @@ class ReorderAssociativeOperatorSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + + test("SPARK-49915: Handle zero and one in associative operators") { + val originalQuery = + testRelation.select( + $"a" + 0, + Literal(-3) + $"a" + 3, + $"b" * 0 * 1 * 2 * 3, + Count($"b") * 0, + $"b" * 1 * 1, + ($"b" + 0) * 1 * 2 * 3 * 4, + $"a" + 0 + $"b" + 0 + $"c" + 0, + $"a" + 0 + $"b" * 1 + $"c" + 0 + ) + + val optimized = Optimize.execute(originalQuery.analyze) + + val correctAnswer = + testRelation + .select( + $"a".as("(a + 0)"), + $"a".as("((-3 + a) + 3)"), + ($"b" * 0).as("((((b * 0) * 1) * 2) * 3)"), + Literal(0L).as("(count(b) * 0)"), + $"b".as("((b * 1) * 1)"), + ($"b" * 24).as("(((((b + 0) * 1) * 2) * 3) * 4)"), + ($"a" + $"b" + $"c").as("""(((((a + 0) + b) + 0) + c) + 0)"""), + ($"a" + $"b" + $"c").as("((((a + 0) + (b * 1)) + c) + 0)") + ).analyze + + comparePlans(optimized, correctAnswer) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index b7e2490b552cc..926beacc592a5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -3065,7 +3065,7 @@ class DDLParserSuite extends AnalysisTest { s"(id BIGINT GENERATED ALWAYS AS IDENTITY $identitySpecStr, val INT) USING foo" ) } - assert(exception.getErrorClass === "IDENTITY_COLUMNS_DUPLICATED_SEQUENCE_GENERATOR_OPTION") + assert(exception.getCondition === "IDENTITY_COLUMNS_DUPLICATED_SEQUENCE_GENERATOR_OPTION") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala index 2972ba2db21de..2e702e5642a92 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala @@ -50,7 +50,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { val e = intercept[ParseException] { parseScript(sqlScriptText) } - assert(e.getErrorClass === "PARSE_SYNTAX_ERROR") + assert(e.getCondition === "PARSE_SYNTAX_ERROR") assert(e.getMessage.contains("Syntax error")) assert(e.getMessage.contains("SELECT")) } @@ -90,7 +90,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { val e = intercept[ParseException] { parseScript(sqlScriptText) } - assert(e.getErrorClass === "PARSE_SYNTAX_ERROR") + assert(e.getCondition === "PARSE_SYNTAX_ERROR") assert(e.getMessage.contains("Syntax error")) assert(e.getMessage.contains("at or near ';'")) } @@ -105,7 +105,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { val e = intercept[ParseException] { parseScript(sqlScriptText) } - assert(e.getErrorClass === "PARSE_SYNTAX_ERROR") + assert(e.getCondition === "PARSE_SYNTAX_ERROR") assert(e.getMessage.contains("Syntax error")) assert(e.getMessage.contains("at or near end of input")) } @@ -367,7 +367,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { val e = intercept[ParseException] { parseScript(sqlScriptText) } - assert(e.getErrorClass === "PARSE_SYNTAX_ERROR") + assert(e.getCondition === "PARSE_SYNTAX_ERROR") assert(e.getMessage.contains("Syntax error")) } diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_array_compact.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_array_compact.explain index a78195c4ae295..d42d0fd0a46ee 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_array_compact.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_array_compact.explain @@ -1,2 +1,2 @@ -Project [filter(e#0, lambdafunction(isnotnull(lambda arg#0), lambda arg#0, false)) AS array_compact(e)#0] +Project [knownnotcontainsnull(filter(e#0, lambdafunction(isnotnull(lambda arg#0), lambda arg#0, false))) AS array_compact(e)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_json_array_length.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_json_array_length.explain index 50ab91560e64a..d70e2eb60aba5 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_json_array_length.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_json_array_length.explain @@ -1,2 +1,2 @@ -Project [json_array_length(g#0) AS json_array_length(g)#0] +Project [static_invoke(JsonExpressionUtils.lengthOfJsonArray(g#0)) AS json_array_length(g)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_json_object_keys.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_json_object_keys.explain index 30153bb192e55..8a2ea5336c160 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_json_object_keys.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_json_object_keys.explain @@ -1,2 +1,2 @@ -Project [json_object_keys(g#0) AS json_object_keys(g)#0] +Project [static_invoke(JsonExpressionUtils.jsonObjectKeys(g#0)) AS json_object_keys(g)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json.explain index 8ec799bc58084..b400aeeca5af2 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json.explain @@ -1,2 +1,2 @@ -Project [schema_of_json([{"col":01}]) AS schema_of_json([{"col":01}])#0] +Project [static_invoke(JsonExpressionEvalUtils.schemaOfJson(com.fasterxml.jackson.core.JsonFactory, org.apache.spark.sql.catalyst.json.JSONOptions, org.apache.spark.sql.catalyst.json.JsonInferSchema, [{"col":01}])) AS schema_of_json([{"col":01}])#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json_with_options.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json_with_options.explain index 13867949177a4..b400aeeca5af2 100644 --- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json_with_options.explain +++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json_with_options.explain @@ -1,2 +1,2 @@ -Project [schema_of_json([{"col":01}], (allowNumericLeadingZeros,true)) AS schema_of_json([{"col":01}])#0] +Project [static_invoke(JsonExpressionEvalUtils.schemaOfJson(com.fasterxml.jackson.core.JsonFactory, org.apache.spark.sql.catalyst.json.JSONOptions, org.apache.spark.sql.catalyst.json.JsonInferSchema, [{"col":01}])) AS schema_of_json([{"col":01}])#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 25fd7d13b7d48..4e6994f9c2f8b 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -3118,7 +3118,7 @@ class SparkConnectPlanner( .newBuilder() exception_builder .setExceptionMessage(e.toString()) - .setErrorClass(e.getErrorClass) + .setErrorClass(e.getCondition) val stackTrace = Option(ExceptionUtils.getStackTrace(e)) stackTrace.foreach { s => diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala index 0468a55e23027..e62c19b66c8e5 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala @@ -345,7 +345,7 @@ object SparkConnectService extends Logging { val kvStore = sc.statusStore.store.asInstanceOf[ElementTrackingStore] listener = new SparkConnectServerListener(kvStore, sc.conf) sc.listenerBus.addToStatusQueue(listener) - uiTab = if (sc.getConf.get(UI_ENABLED)) { + uiTab = if (sc.conf.get(UI_ENABLED)) { Some( new SparkConnectServerTab( new SparkConnectServerAppStatusStore(kvStore), diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala index f1636ed1ef092..837d4a4d3ee78 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala @@ -114,8 +114,8 @@ private[connect] object ErrorUtils extends Logging { case sparkThrowable: SparkThrowable => val sparkThrowableBuilder = FetchErrorDetailsResponse.SparkThrowable .newBuilder() - if (sparkThrowable.getErrorClass != null) { - sparkThrowableBuilder.setErrorClass(sparkThrowable.getErrorClass) + if (sparkThrowable.getCondition != null) { + sparkThrowableBuilder.setErrorClass(sparkThrowable.getCondition) } for (queryCtx <- sparkThrowable.getQueryContext) { val builder = FetchErrorDetailsResponse.QueryContext @@ -193,7 +193,7 @@ private[connect] object ErrorUtils extends Logging { if (state != null && state.nonEmpty) { errorInfo.putMetadata("sqlState", state) } - val errorClass = e.getErrorClass + val errorClass = e.getCondition if (errorClass != null && errorClass.nonEmpty) { val messageParameters = JsonMethods.compact( JsonMethods.render(map2jvalue(e.getMessageParameters.asScala.toMap))) diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala index 42bb93de05e26..1f522ea28b761 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala @@ -37,7 +37,7 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA val exGetOrCreate = intercept[SparkSQLException] { SparkConnectService.sessionManager.getOrCreateIsolatedSession(key, None) } - assert(exGetOrCreate.getErrorClass == "INVALID_HANDLE.FORMAT") + assert(exGetOrCreate.getCondition == "INVALID_HANDLE.FORMAT") } test( @@ -72,7 +72,7 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA key, Some(sessionHolder.session.sessionUUID + "invalid")) } - assert(exGet.getErrorClass == "INVALID_HANDLE.SESSION_CHANGED") + assert(exGet.getCondition == "INVALID_HANDLE.SESSION_CHANGED") } test( @@ -85,12 +85,12 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA val exGetOrCreate = intercept[SparkSQLException] { SparkConnectService.sessionManager.getOrCreateIsolatedSession(key, None) } - assert(exGetOrCreate.getErrorClass == "INVALID_HANDLE.SESSION_CLOSED") + assert(exGetOrCreate.getCondition == "INVALID_HANDLE.SESSION_CLOSED") val exGet = intercept[SparkSQLException] { SparkConnectService.sessionManager.getIsolatedSession(key, None) } - assert(exGet.getErrorClass == "INVALID_HANDLE.SESSION_CLOSED") + assert(exGet.getCondition == "INVALID_HANDLE.SESSION_CLOSED") val sessionGetIfPresent = SparkConnectService.sessionManager.getIsolatedSessionIfPresent(key) assert(sessionGetIfPresent.isEmpty) @@ -102,7 +102,7 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA val exGet = intercept[SparkSQLException] { SparkConnectService.sessionManager.getIsolatedSession(key, None) } - assert(exGet.getErrorClass == "INVALID_HANDLE.SESSION_NOT_FOUND") + assert(exGet.getCondition == "INVALID_HANDLE.SESSION_NOT_FOUND") val sessionGetIfPresent = SparkConnectService.sessionManager.getIsolatedSessionIfPresent(key) assert(sessionGetIfPresent.isEmpty) diff --git a/sql/connect/shims/pom.xml b/sql/connect/shims/pom.xml index 6bb12a927738c..d177b4a9971f5 100644 --- a/sql/connect/shims/pom.xml +++ b/sql/connect/shims/pom.xml @@ -34,6 +34,13 @@ connect-shims + + + org.scala-lang + scala-library + + + target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 972cf76d27535..16236940fe072 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -79,6 +79,12 @@ ${project.version} test-jar test + + + org.apache.spark + spark-connect-shims_${scala.binary.version} + + org.apache.spark diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataSourceRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataSourceRegistration.scala index 8ffdbb952b082..3b64cb97e10b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataSourceRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataSourceRegistration.scala @@ -68,7 +68,7 @@ class DataSourceRegistration private[sql] (dataSourceManager: DataSourceManager) DataSource.lookupDataSource(name, SQLConf.get) throw QueryCompilationErrors.dataSourceAlreadyExists(name) } catch { - case e: SparkClassNotFoundException if e.getErrorClass == "DATA_SOURCE_NOT_FOUND" => // OK + case e: SparkClassNotFoundException if e.getCondition == "DATA_SOURCE_NOT_FOUND" => // OK case _: Throwable => // If there are other errors when resolving the data source, it's unclear whether // it's safe to proceed. To prevent potential lookup errors, treat it as an existing diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 55525380aee55..99ab3ca69fb20 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -1100,7 +1100,7 @@ object SparkSession extends api.BaseSparkSessionCompanion with Logging { private def applyExtensions( sparkContext: SparkContext, extensions: SparkSessionExtensions): SparkSessionExtensions = { - val extensionConfClassNames = sparkContext.getConf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS) + val extensionConfClassNames = sparkContext.conf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS) .getOrElse(Seq.empty) extensionConfClassNames.foreach { extensionConfClassName => try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala index 3504f6e76f79d..08395ef4c347c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala @@ -143,9 +143,6 @@ private[sql] object PythonSQLUtils extends Logging { } } - def castTimestampNTZToLong(c: Column): Column = - Column.internalFn("timestamp_ntz_to_long", c) - def unresolvedNamedLambdaVariable(name: String): Column = Column(internal.UnresolvedNamedLambdaVariable.apply(name)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 8fc860c503c96..9fbe400a555fc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -189,10 +189,29 @@ class SparkSqlAstBuilder extends AstBuilder { val key = SQLConf.SESSION_LOCAL_TIMEZONE.key if (ctx.interval != null) { val interval = parseIntervalLiteral(ctx.interval) - if (interval.months != 0 || interval.days != 0 || - math.abs(interval.microseconds) > 18 * DateTimeConstants.MICROS_PER_HOUR || - interval.microseconds % DateTimeConstants.MICROS_PER_SECOND != 0) { - throw QueryParsingErrors.intervalValueOutOfRangeError(ctx.interval()) + if (interval.months != 0) { + throw QueryParsingErrors.intervalValueOutOfRangeError( + toSQLValue(interval.months), + ctx.interval() + ) + } + else if (interval.days != 0) { + throw QueryParsingErrors.intervalValueOutOfRangeError( + toSQLValue(interval.days), + ctx.interval() + ) + } + else if (math.abs(interval.microseconds) > 18 * DateTimeConstants.MICROS_PER_HOUR) { + throw QueryParsingErrors.intervalValueOutOfRangeError( + toSQLValue((math.abs(interval.microseconds) / DateTimeConstants.MICROS_PER_HOUR).toInt), + ctx.interval() + ) + } + else if (interval.microseconds % DateTimeConstants.MICROS_PER_SECOND != 0) { + throw QueryParsingErrors.intervalValueOutOfRangeError( + toSQLValue((interval.microseconds / DateTimeConstants.MICROS_PER_SECOND).toInt), + ctx.interval() + ) } else { val seconds = (interval.microseconds / DateTimeConstants.MICROS_PER_SECOND).toInt SetCommand(Some(key -> Some(ZoneOffset.ofTotalSeconds(seconds).toString))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 8f2b7ca5cba25..750b74aab384f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -682,7 +682,7 @@ case class HashAggregateExec( | $unsafeRowKeys, $unsafeRowKeyHash); | if ($unsafeRowBuffer == null) { | // failed to allocate the first page - | throw new $oomeClassName("No enough memory for aggregation"); + | throw new $oomeClassName("_LEGACY_ERROR_TEMP_3302", new java.util.HashMap()); | } |} """.stripMargin diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala index 45a71b4da7287..19a36483abe6d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala @@ -173,7 +173,8 @@ abstract class HashMapGenerator( ${hashBytes(bytes)} """ } - case st: StringType if st.supportsBinaryEquality => hashBytes(s"$input.getBytes()") + case st: StringType if st.supportsBinaryEquality => + hashBytes(s"$input.getBytes()") case st: StringType if !st.supportsBinaryEquality => hashLong(s"CollationFactory.fetchCollation(${st.collationId})" + s".hashFunction.applyAsLong($input)") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala index 1ebf0d143bd1f..2f1cda9d0f9be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.aggregate +import java.util + import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.memory.SparkOutOfMemoryError @@ -210,7 +212,7 @@ class TungstenAggregationIterator( if (buffer == null) { // failed to allocate the first page // scalastyle:off throwerror - throw new SparkOutOfMemoryError("No enough memory for aggregation") + throw new SparkOutOfMemoryError("_LEGACY_ERROR_TEMP_3302", new util.HashMap()) // scalastyle:on throwerror } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 968c204841e46..3698dc2f0808e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -514,7 +514,8 @@ case class DataSource( dataSource.createRelation( sparkSession.sqlContext, mode, caseInsensitiveOptions, Dataset.ofRows(sparkSession, data)) case format: FileFormat => - disallowWritingIntervals(outputColumns.map(_.dataType), forbidAnsiIntervals = false) + disallowWritingIntervals( + outputColumns.toStructType.asNullable, format.toString, forbidAnsiIntervals = false) val cmd = planForWritingFileFormat(format, mode, data) val qe = sparkSession.sessionState.executePlan(cmd) qe.assertCommandExecuted() @@ -539,8 +540,8 @@ case class DataSource( } SaveIntoDataSourceCommand(data, dataSource, caseInsensitiveOptions, mode) case format: FileFormat => - disallowWritingIntervals(data.schema.map(_.dataType), forbidAnsiIntervals = false) - DataSource.validateSchema(data.schema, sparkSession.sessionState.conf) + disallowWritingIntervals(data.schema, format.toString, forbidAnsiIntervals = false) + DataSource.validateSchema(format.toString, data.schema, sparkSession.sessionState.conf) planForWritingFileFormat(format, mode, data) case _ => throw SparkException.internalError( s"${providingClass.getCanonicalName} does not allow create table as select.") @@ -566,12 +567,15 @@ case class DataSource( } private def disallowWritingIntervals( - dataTypes: Seq[DataType], + outputColumns: Seq[StructField], + format: String, forbidAnsiIntervals: Boolean): Unit = { - dataTypes.foreach( - TypeUtils.invokeOnceForInterval(_, forbidAnsiIntervals) { - throw QueryCompilationErrors.cannotSaveIntervalIntoExternalStorageError() - }) + outputColumns.foreach { field => + TypeUtils.invokeOnceForInterval(field.dataType, forbidAnsiIntervals) { + throw QueryCompilationErrors.dataTypeUnsupportedByDataSourceError( + format, field + )} + } } } @@ -838,7 +842,7 @@ object DataSource extends Logging { * @param schema * @param conf */ - def validateSchema(schema: StructType, conf: SQLConf): Unit = { + def validateSchema(formatName: String, schema: StructType, conf: SQLConf): Unit = { val shouldAllowEmptySchema = conf.getConf(SQLConf.ALLOW_EMPTY_SCHEMAS_FOR_WRITES) def hasEmptySchema(schema: StructType): Boolean = { schema.size == 0 || schema.exists { @@ -849,7 +853,7 @@ object DataSource extends Logging { if (!shouldAllowEmptySchema && hasEmptySchema(schema)) { - throw QueryCompilationErrors.writeEmptySchemasUnsupportedByDataSourceError() + throw QueryCompilationErrors.writeEmptySchemasUnsupportedByDataSourceError(formatName) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 7946068b9452e..6e79a2f2a3267 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -585,14 +585,26 @@ object JdbcUtils extends Logging with SQLConfHelper { arr => new GenericArrayData(elementConversion(et0)(arr)) } + case IntegerType => arrayConverter[Int]((i: Int) => i) + case FloatType => arrayConverter[Float]((f: Float) => f) + case DoubleType => arrayConverter[Double]((d: Double) => d) + case ShortType => arrayConverter[Short]((s: Short) => s) + case BooleanType => arrayConverter[Boolean]((b: Boolean) => b) + case LongType => arrayConverter[Long]((l: Long) => l) + case _ => (array: Object) => array.asInstanceOf[Array[Any]] } (rs: ResultSet, row: InternalRow, pos: Int) => - val array = nullSafeConvert[java.sql.Array]( - input = rs.getArray(pos + 1), - array => new GenericArrayData(elementConversion(et)(array.getArray))) - row.update(pos, array) + try { + val array = nullSafeConvert[java.sql.Array]( + input = rs.getArray(pos + 1), + array => new GenericArrayData(elementConversion(et)(array.getArray()))) + row.update(pos, array) + } catch { + case e: java.lang.ClassCastException => + throw QueryExecutionErrors.wrongDatatypeInSomeRows(pos, dt) + } case NullType => (_: ResultSet, row: InternalRow, pos: Int) => row.update(pos, null) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index 7c98c31bba220..cb4c4f5290880 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -164,7 +164,8 @@ object MultiLineJsonDataSource extends JsonDataSource { .getOrElse(createParser(_: JsonFactory, _: PortableDataStream)) SQLExecution.withSQLConfPropagated(sparkSession) { - new JsonInferSchema(parsedOptions).infer[PortableDataStream](sampled, parser) + new JsonInferSchema(parsedOptions) + .infer[PortableDataStream](sampled, parser, isReadFile = true) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 29385904a7525..cbbf9f88f89d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -89,9 +89,9 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { LogicalRelation(ds.resolveRelation()) } catch { case _: ClassNotFoundException => u - case e: SparkIllegalArgumentException if e.getErrorClass != null => + case e: SparkIllegalArgumentException if e.getCondition != null => u.failAnalysis( - errorClass = e.getErrorClass, + errorClass = e.getCondition, messageParameters = e.getMessageParameters.asScala.toMap, cause = e) case e: Exception if !e.isInstanceOf[AnalysisException] => @@ -469,8 +469,8 @@ object PreprocessTableInsertion extends ResolveInsertionBase { supportColDefaultValue = true) } catch { case e: AnalysisException if staticPartCols.nonEmpty && - (e.getErrorClass == "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS" || - e.getErrorClass == "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS") => + (e.getCondition == "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS" || + e.getCondition == "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS") => val newException = e.copy( errorClass = Some("INSERT_PARTITION_COLUMN_ARITY_MISMATCH"), messageParameters = e.messageParameters ++ Map( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala index 168aea5b041f8..4242fc5d8510a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala @@ -131,7 +131,7 @@ object FileDataSourceV2 { // The error is already FAILED_READ_FILE, throw it directly. To be consistent, schema // inference code path throws `FAILED_READ_FILE`, but the file reading code path can reach // that code path as well and we should not double-wrap the error. - case e: SparkException if e.getErrorClass == "FAILED_READ_FILE.CANNOT_READ_FILE_FOOTER" => + case e: SparkException if e.getCondition == "FAILED_READ_FILE.CANNOT_READ_FILE_FOOTER" => throw e case e: SchemaColumnConvertNotSupportedException => throw QueryExecutionErrors.parquetColumnDataTypeMismatchError( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala index d890107277d6c..5c0f8c0a4afd9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala @@ -164,7 +164,7 @@ trait FileScan extends Scan if (splitFiles.length == 1) { val path = splitFiles(0).toPath if (!isSplitable(path) && splitFiles(0).length > - sparkSession.sparkContext.getConf.get(IO_WARNING_LARGEFILETHRESHOLD)) { + sparkSession.sparkContext.conf.get(IO_WARNING_LARGEFILETHRESHOLD)) { logWarning(log"Loading one large unsplittable file ${MDC(PATH, path.toString)} with only " + log"one partition, the reason is: ${MDC(REASON, getFileUnSplittableReason(path))}") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWrite.scala index cdcf6f21fd008..f4cabcb69d08c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWrite.scala @@ -96,7 +96,7 @@ trait FileWrite extends Write { SchemaUtils.checkColumnNameDuplication( schema.fields.map(_.name).toImmutableArraySeq, caseSensitiveAnalysis) } - DataSource.validateSchema(schema, sqlConf) + DataSource.validateSchema(formatName, schema, sqlConf) // TODO: [SPARK-36340] Unify check schema filed of DataSource V2 Insert. schema.foreach { field => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index e669165f4f2f8..8ec903f8e61da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -70,7 +70,16 @@ case class EnsureRequirements( case (child, distribution) => val numPartitions = distribution.requiredNumPartitions .getOrElse(conf.numShufflePartitions) - ShuffleExchangeExec(distribution.createPartitioning(numPartitions), child, shuffleOrigin) + distribution match { + case _: StatefulOpClusteredDistribution => + ShuffleExchangeExec( + distribution.createPartitioning(numPartitions), child, + REQUIRED_BY_STATEFUL_OPERATOR) + + case _ => + ShuffleExchangeExec( + distribution.createPartitioning(numPartitions), child, shuffleOrigin) + } } // Get the indexes of children which have specified distribution requirements and need to be diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index ae11229cd516e..31a3f53eb7191 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -177,6 +177,11 @@ case object REBALANCE_PARTITIONS_BY_NONE extends ShuffleOrigin // the output needs to be partitioned by the given columns. case object REBALANCE_PARTITIONS_BY_COL extends ShuffleOrigin +// Indicates that the shuffle operator was added by the internal `EnsureRequirements` rule, but +// was required by a stateful operator. The physical partitioning is static and Spark shouldn't +// change it. +case object REQUIRED_BY_STATEFUL_OPERATOR extends ShuffleOrigin + /** * Performs a shuffle that will result in the desired partitioning. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 8f030884ad33b..14adf951f07e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -374,7 +374,7 @@ abstract class StreamExecution( "message" -> message)) errorClassOpt = e match { - case t: SparkThrowable => Option(t.getErrorClass) + case t: SparkThrowable => Option(t.getCondition) case _ => None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index 884b8aa3853cb..3df63c41dbf97 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -282,7 +282,7 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with newMap } catch { - case e: SparkException if e.getErrorClass.contains("CANNOT_LOAD_STATE_STORE") => + case e: SparkException if e.getCondition.contains("CANNOT_LOAD_STATE_STORE") => throw e case e: OutOfMemoryError => throw QueryExecutionErrors.notEnoughMemoryToLoadStore( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index 6ab634668bc2a..870ed79ec1747 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -389,7 +389,7 @@ private[sql] class RocksDBStateStoreProvider new RocksDBStateStore(version) } catch { - case e: SparkException if e.getErrorClass.contains("CANNOT_LOAD_STATE_STORE") => + case e: SparkException if e.getCondition.contains("CANNOT_LOAD_STATE_STORE") => throw e case e: OutOfMemoryError => throw QueryExecutionErrors.notEnoughMemoryToLoadStore( @@ -409,7 +409,7 @@ private[sql] class RocksDBStateStoreProvider new RocksDBStateStore(version) } catch { - case e: SparkException if e.getErrorClass.contains("CANNOT_LOAD_STATE_STORE") => + case e: SparkException if e.getCondition.contains("CANNOT_LOAD_STATE_STORE") => throw e case e: OutOfMemoryError => throw QueryExecutionErrors.notEnoughMemoryToLoadStore( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 52b8d35e2fbf8..64689e75e2e5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -177,7 +177,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { try { Some(makeTable(catalogName +: ns :+ tableName)) } catch { - case e: AnalysisException if e.getErrorClass == "UNSUPPORTED_FEATURE.HIVE_TABLE_TYPE" => + case e: AnalysisException if e.getCondition == "UNSUPPORTED_FEATURE.HIVE_TABLE_TYPE" => Some(new Table( name = tableName, catalog = catalogName, @@ -189,7 +189,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { } } } catch { - case e: AnalysisException if e.getErrorClass == "TABLE_OR_VIEW_NOT_FOUND" => None + case e: AnalysisException if e.getCondition == "TABLE_OR_VIEW_NOT_FOUND" => None } } @@ -203,7 +203,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { case _ => false } } catch { - case e: AnalysisException if e.getErrorClass == "TABLE_OR_VIEW_NOT_FOUND" => false + case e: AnalysisException if e.getCondition == "TABLE_OR_VIEW_NOT_FOUND" => false } } @@ -323,7 +323,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { case _ => false } } catch { - case e: AnalysisException if e.getErrorClass == "UNRESOLVED_ROUTINE" => false + case e: AnalysisException if e.getCondition == "UNRESOLVED_ROUTINE" => false } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala index 60258ecbb0d61..8341063e09890 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala @@ -23,6 +23,7 @@ import java.util import java.util.Locale import scala.util.Using +import scala.util.control.NonFatal import org.apache.spark.SparkThrowable import org.apache.spark.internal.LogKeys.COLUMN_NAME @@ -30,7 +31,7 @@ import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.SQLConfHelper import org.apache.spark.sql.catalyst.analysis.{IndexAlreadyExistsException, NonEmptyNamespaceException, NoSuchIndexException} import org.apache.spark.sql.connector.catalog.Identifier -import org.apache.spark.sql.connector.expressions.NamedReference +import org.apache.spark.sql.connector.expressions.{Expression, NamedReference} import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} import org.apache.spark.sql.execution.datasources.v2.TableSampleInfo @@ -300,6 +301,28 @@ private case class PostgresDialect() } } + class PostgresSQLBuilder extends JDBCSQLBuilder { + override def visitExtract(field: String, source: String): String = { + field match { + case "DAY_OF_YEAR" => s"EXTRACT(DOY FROM $source)" + case "YEAR_OF_WEEK" => s"EXTRACT(YEAR FROM $source)" + case "DAY_OF_WEEK" => s"EXTRACT(DOW FROM $source)" + case _ => super.visitExtract(field, source) + } + } + } + + override def compileExpression(expr: Expression): Option[String] = { + val postgresSQLBuilder = new PostgresSQLBuilder() + try { + Some(postgresSQLBuilder.build(expr)) + } catch { + case NonFatal(e) => + logWarning("Error occurs while compiling V2 expression", e) + None + } + } + override def supportsLimit: Boolean = true override def supportsOffset: Boolean = true diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java index 9fbd1919a2668..9988d04220f0f 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java @@ -85,7 +85,7 @@ public void isInCollectionCheckExceptionMessage() { Dataset df = spark.createDataFrame(rows, schema); AnalysisException e = Assertions.assertThrows(AnalysisException.class, () -> df.filter(df.col("a").isInCollection(Arrays.asList(new Column("b"))))); - Assertions.assertTrue(e.getErrorClass().equals("DATATYPE_MISMATCH.DATA_DIFF_TYPES")); + Assertions.assertTrue(e.getCondition().equals("DATATYPE_MISMATCH.DATA_DIFF_TYPES")); Map messageParameters = new HashMap<>(); messageParameters.put("functionName", "`in`"); messageParameters.put("dataType", "[\"INT\", \"ARRAY\"]"); diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out index 26e9394932a17..37d84f6c5fc00 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out @@ -69,6 +69,24 @@ Project [a#x, (b#x + c#x) AS (b + c)#x] +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet +-- !query +select b + 0 from t1 where a = 5 +-- !query analysis +Project [(b#x + 0) AS (b + 0)#x] ++- Filter (a#x = 5) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + +-- !query +select -100 + b + 100 from t1 where a = 5 +-- !query analysis +Project [((-100 + b#x) + 100) AS ((-100 + b) + 100)#x] ++- Filter (a#x = 5) + +- SubqueryAlias spark_catalog.default.t1 + +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet + + -- !query select a+10, b*0 from t1 -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out index 133cd6a60a4fb..31919381c99b6 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out @@ -188,7 +188,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42K09", "messageParameters" : { "inputExpr" : "\"col\"", - "inputName" : "seed", + "inputName" : "`seed`", "inputType" : "integer or floating-point", "sqlExpr" : "\"uniform(10, 20, col)\"" }, @@ -211,7 +211,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42K09", "messageParameters" : { "inputExpr" : "\"col\"", - "inputName" : "min", + "inputName" : "`min`", "inputType" : "integer or floating-point", "sqlExpr" : "\"uniform(col, 10, 0)\"" }, @@ -436,7 +436,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42K09", "messageParameters" : { "inputExpr" : "\"col\"", - "inputName" : "length", + "inputName" : "`length`", "inputType" : "INT or SMALLINT", "sqlExpr" : "\"randstr(col, 0)\"" }, @@ -459,7 +459,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42K09", "messageParameters" : { "inputExpr" : "\"col\"", - "inputName" : "seedExpression", + "inputName" : "`seed`", "inputType" : "INT or SMALLINT", "sqlExpr" : "\"randstr(10, col)\"" }, diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/timezone.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/timezone.sql.out index 9059f37f3607b..5b55a0c218934 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/timezone.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timezone.sql.out @@ -64,7 +64,11 @@ SET TIME ZONE INTERVAL 3 DAYS -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "3" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", @@ -80,7 +84,11 @@ SET TIME ZONE INTERVAL 24 HOURS -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "24" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", @@ -96,7 +104,11 @@ SET TIME ZONE INTERVAL '19:40:32' HOUR TO SECOND -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "19" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", @@ -128,7 +140,11 @@ SET TIME ZONE INTERVAL 10 HOURS 1 MILLISECOND -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "36000" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", diff --git a/sql/core/src/test/resources/sql-tests/inputs/null-handling.sql b/sql/core/src/test/resources/sql-tests/inputs/null-handling.sql index 040be00503442..dcdf241df73d9 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/null-handling.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/null-handling.sql @@ -10,6 +10,8 @@ insert into t1 values(7,null,null); -- Adding anything to null gives null select a, b+c from t1; +select b + 0 from t1 where a = 5; +select -100 + b + 100 from t1 where a = 5; -- Multiplying null by zero gives null select a+10, b*0 from t1; diff --git a/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out b/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out index ece6dbef1605d..fb96be8317a5b 100644 --- a/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out @@ -77,6 +77,22 @@ struct 7 NULL +-- !query +select b + 0 from t1 where a = 5 +-- !query schema +struct<(b + 0):int> +-- !query output +NULL + + +-- !query +select -100 + b + 100 from t1 where a = 5 +-- !query schema +struct<((-100 + b) + 100):int> +-- !query output +NULL + + -- !query select a+10, b*0 from t1 -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/random.sql.out b/sql/core/src/test/resources/sql-tests/results/random.sql.out index 0b4e5e078ee15..01638abdcec6e 100644 --- a/sql/core/src/test/resources/sql-tests/results/random.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/random.sql.out @@ -240,7 +240,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42K09", "messageParameters" : { "inputExpr" : "\"col\"", - "inputName" : "seed", + "inputName" : "`seed`", "inputType" : "integer or floating-point", "sqlExpr" : "\"uniform(10, 20, col)\"" }, @@ -265,7 +265,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42K09", "messageParameters" : { "inputExpr" : "\"col\"", - "inputName" : "min", + "inputName" : "`min`", "inputType" : "integer or floating-point", "sqlExpr" : "\"uniform(col, 10, 0)\"" }, @@ -520,7 +520,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42K09", "messageParameters" : { "inputExpr" : "\"col\"", - "inputName" : "length", + "inputName" : "`length`", "inputType" : "INT or SMALLINT", "sqlExpr" : "\"randstr(col, 0)\"" }, @@ -545,7 +545,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException "sqlState" : "42K09", "messageParameters" : { "inputExpr" : "\"col\"", - "inputName" : "seedExpression", + "inputName" : "`seed`", "inputType" : "INT or SMALLINT", "sqlExpr" : "\"randstr(10, col)\"" }, diff --git a/sql/core/src/test/resources/sql-tests/results/timezone.sql.out b/sql/core/src/test/resources/sql-tests/results/timezone.sql.out index d34599a49c5ff..5f0fdef50e3db 100644 --- a/sql/core/src/test/resources/sql-tests/results/timezone.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timezone.sql.out @@ -80,7 +80,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "3" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", @@ -98,7 +102,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "24" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", @@ -116,7 +124,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "19" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", @@ -152,7 +164,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "36000" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt index 96bed479d2e06..4bf7de791b279 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt @@ -175,125 +175,125 @@ Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#21, cou Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] Functions [1]: [avg(qoh#18)] Aggregate Attributes [1]: [avg(qoh#18)#23] -Results [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, avg(qoh#18)#23 AS qoh#24] +Results [5]: [i_product_name#12 AS i_product_name#24, i_brand#9 AS i_brand#25, i_class#10 AS i_class#26, i_category#11 AS i_category#27, avg(qoh#18)#23 AS qoh#28] (27) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_product_name#25, i_brand#26, i_class#27, i_category#28, sum#29, count#30] +Output [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] (28) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#25, i_brand#26, i_class#27, i_category#28, sum#29, count#30] -Keys [4]: [i_product_name#25, i_brand#26, i_class#27, i_category#28] -Functions [1]: [avg(inv_quantity_on_hand#31)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#31)#17] -Results [4]: [i_product_name#25, i_brand#26, i_class#27, avg(inv_quantity_on_hand#31)#17 AS qoh#32] +Input [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] +Keys [4]: [i_product_name#29, i_brand#30, i_class#31, i_category#32] +Functions [1]: [avg(inv_quantity_on_hand#35)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#17] +Results [4]: [i_product_name#29, i_brand#30, i_class#31, avg(inv_quantity_on_hand#35)#17 AS qoh#36] (29) HashAggregate [codegen id : 16] -Input [4]: [i_product_name#25, i_brand#26, i_class#27, qoh#32] -Keys [3]: [i_product_name#25, i_brand#26, i_class#27] -Functions [1]: [partial_avg(qoh#32)] -Aggregate Attributes [2]: [sum#33, count#34] -Results [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36] +Input [4]: [i_product_name#29, i_brand#30, i_class#31, qoh#36] +Keys [3]: [i_product_name#29, i_brand#30, i_class#31] +Functions [1]: [partial_avg(qoh#36)] +Aggregate Attributes [2]: [sum#37, count#38] +Results [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] (30) Exchange -Input [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36] -Arguments: hashpartitioning(i_product_name#25, i_brand#26, i_class#27, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] +Arguments: hashpartitioning(i_product_name#29, i_brand#30, i_class#31, 5), ENSURE_REQUIREMENTS, [plan_id=5] (31) HashAggregate [codegen id : 17] -Input [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36] -Keys [3]: [i_product_name#25, i_brand#26, i_class#27] -Functions [1]: [avg(qoh#32)] -Aggregate Attributes [1]: [avg(qoh#32)#37] -Results [5]: [i_product_name#25, i_brand#26, i_class#27, null AS i_category#38, avg(qoh#32)#37 AS qoh#39] +Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] +Keys [3]: [i_product_name#29, i_brand#30, i_class#31] +Functions [1]: [avg(qoh#36)] +Aggregate Attributes [1]: [avg(qoh#36)#41] +Results [5]: [i_product_name#29, i_brand#30, i_class#31, null AS i_category#42, avg(qoh#36)#41 AS qoh#43] (32) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_product_name#40, i_brand#41, i_class#42, i_category#43, sum#44, count#45] +Output [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49] (33) HashAggregate [codegen id : 25] -Input [6]: [i_product_name#40, i_brand#41, i_class#42, i_category#43, sum#44, count#45] -Keys [4]: [i_product_name#40, i_brand#41, i_class#42, i_category#43] -Functions [1]: [avg(inv_quantity_on_hand#46)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#46)#17] -Results [3]: [i_product_name#40, i_brand#41, avg(inv_quantity_on_hand#46)#17 AS qoh#47] +Input [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49] +Keys [4]: [i_product_name#44, i_brand#45, i_class#46, i_category#47] +Functions [1]: [avg(inv_quantity_on_hand#50)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#50)#17] +Results [3]: [i_product_name#44, i_brand#45, avg(inv_quantity_on_hand#50)#17 AS qoh#51] (34) HashAggregate [codegen id : 25] -Input [3]: [i_product_name#40, i_brand#41, qoh#47] -Keys [2]: [i_product_name#40, i_brand#41] -Functions [1]: [partial_avg(qoh#47)] -Aggregate Attributes [2]: [sum#48, count#49] -Results [4]: [i_product_name#40, i_brand#41, sum#50, count#51] +Input [3]: [i_product_name#44, i_brand#45, qoh#51] +Keys [2]: [i_product_name#44, i_brand#45] +Functions [1]: [partial_avg(qoh#51)] +Aggregate Attributes [2]: [sum#52, count#53] +Results [4]: [i_product_name#44, i_brand#45, sum#54, count#55] (35) Exchange -Input [4]: [i_product_name#40, i_brand#41, sum#50, count#51] -Arguments: hashpartitioning(i_product_name#40, i_brand#41, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55] +Arguments: hashpartitioning(i_product_name#44, i_brand#45, 5), ENSURE_REQUIREMENTS, [plan_id=6] (36) HashAggregate [codegen id : 26] -Input [4]: [i_product_name#40, i_brand#41, sum#50, count#51] -Keys [2]: [i_product_name#40, i_brand#41] -Functions [1]: [avg(qoh#47)] -Aggregate Attributes [1]: [avg(qoh#47)#52] -Results [5]: [i_product_name#40, i_brand#41, null AS i_class#53, null AS i_category#54, avg(qoh#47)#52 AS qoh#55] +Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55] +Keys [2]: [i_product_name#44, i_brand#45] +Functions [1]: [avg(qoh#51)] +Aggregate Attributes [1]: [avg(qoh#51)#56] +Results [5]: [i_product_name#44, i_brand#45, null AS i_class#57, null AS i_category#58, avg(qoh#51)#56 AS qoh#59] (37) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_product_name#56, i_brand#57, i_class#58, i_category#59, sum#60, count#61] +Output [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65] (38) HashAggregate [codegen id : 34] -Input [6]: [i_product_name#56, i_brand#57, i_class#58, i_category#59, sum#60, count#61] -Keys [4]: [i_product_name#56, i_brand#57, i_class#58, i_category#59] -Functions [1]: [avg(inv_quantity_on_hand#62)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#62)#17] -Results [2]: [i_product_name#56, avg(inv_quantity_on_hand#62)#17 AS qoh#63] +Input [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65] +Keys [4]: [i_product_name#60, i_brand#61, i_class#62, i_category#63] +Functions [1]: [avg(inv_quantity_on_hand#66)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#66)#17] +Results [2]: [i_product_name#60, avg(inv_quantity_on_hand#66)#17 AS qoh#67] (39) HashAggregate [codegen id : 34] -Input [2]: [i_product_name#56, qoh#63] -Keys [1]: [i_product_name#56] -Functions [1]: [partial_avg(qoh#63)] -Aggregate Attributes [2]: [sum#64, count#65] -Results [3]: [i_product_name#56, sum#66, count#67] +Input [2]: [i_product_name#60, qoh#67] +Keys [1]: [i_product_name#60] +Functions [1]: [partial_avg(qoh#67)] +Aggregate Attributes [2]: [sum#68, count#69] +Results [3]: [i_product_name#60, sum#70, count#71] (40) Exchange -Input [3]: [i_product_name#56, sum#66, count#67] -Arguments: hashpartitioning(i_product_name#56, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [3]: [i_product_name#60, sum#70, count#71] +Arguments: hashpartitioning(i_product_name#60, 5), ENSURE_REQUIREMENTS, [plan_id=7] (41) HashAggregate [codegen id : 35] -Input [3]: [i_product_name#56, sum#66, count#67] -Keys [1]: [i_product_name#56] -Functions [1]: [avg(qoh#63)] -Aggregate Attributes [1]: [avg(qoh#63)#68] -Results [5]: [i_product_name#56, null AS i_brand#69, null AS i_class#70, null AS i_category#71, avg(qoh#63)#68 AS qoh#72] +Input [3]: [i_product_name#60, sum#70, count#71] +Keys [1]: [i_product_name#60] +Functions [1]: [avg(qoh#67)] +Aggregate Attributes [1]: [avg(qoh#67)#72] +Results [5]: [i_product_name#60, null AS i_brand#73, null AS i_class#74, null AS i_category#75, avg(qoh#67)#72 AS qoh#76] (42) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_product_name#73, i_brand#74, i_class#75, i_category#76, sum#77, count#78] +Output [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82] (43) HashAggregate [codegen id : 43] -Input [6]: [i_product_name#73, i_brand#74, i_class#75, i_category#76, sum#77, count#78] -Keys [4]: [i_product_name#73, i_brand#74, i_class#75, i_category#76] -Functions [1]: [avg(inv_quantity_on_hand#79)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#79)#17] -Results [1]: [avg(inv_quantity_on_hand#79)#17 AS qoh#80] +Input [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82] +Keys [4]: [i_product_name#77, i_brand#78, i_class#79, i_category#80] +Functions [1]: [avg(inv_quantity_on_hand#83)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#83)#17] +Results [1]: [avg(inv_quantity_on_hand#83)#17 AS qoh#84] (44) HashAggregate [codegen id : 43] -Input [1]: [qoh#80] +Input [1]: [qoh#84] Keys: [] -Functions [1]: [partial_avg(qoh#80)] -Aggregate Attributes [2]: [sum#81, count#82] -Results [2]: [sum#83, count#84] +Functions [1]: [partial_avg(qoh#84)] +Aggregate Attributes [2]: [sum#85, count#86] +Results [2]: [sum#87, count#88] (45) Exchange -Input [2]: [sum#83, count#84] +Input [2]: [sum#87, count#88] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] (46) HashAggregate [codegen id : 44] -Input [2]: [sum#83, count#84] +Input [2]: [sum#87, count#88] Keys: [] -Functions [1]: [avg(qoh#80)] -Aggregate Attributes [1]: [avg(qoh#80)#85] -Results [5]: [null AS i_product_name#86, null AS i_brand#87, null AS i_class#88, null AS i_category#89, avg(qoh#80)#85 AS qoh#90] +Functions [1]: [avg(qoh#84)] +Aggregate Attributes [1]: [avg(qoh#84)#89] +Results [5]: [null AS i_product_name#90, null AS i_brand#91, null AS i_class#92, null AS i_category#93, avg(qoh#84)#89 AS qoh#94] (47) Union (48) TakeOrderedAndProject -Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#24] -Arguments: 100, [qoh#24 ASC NULLS FIRST, i_product_name#12 ASC NULLS FIRST, i_brand#9 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#24] +Input [5]: [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28] +Arguments: 100, [qoh#28 ASC NULLS FIRST, i_product_name#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_class#26 ASC NULLS FIRST, i_category#27 ASC NULLS FIRST], [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28] ===== Subqueries ===== @@ -306,22 +306,22 @@ BroadcastExchange (53) (49) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_month_seq#91] +Output [2]: [d_date_sk#7, d_month_seq#95] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#7, d_month_seq#91] +Input [2]: [d_date_sk#7, d_month_seq#95] (51) Filter [codegen id : 1] -Input [2]: [d_date_sk#7, d_month_seq#91] -Condition : (((isnotnull(d_month_seq#91) AND (d_month_seq#91 >= 1212)) AND (d_month_seq#91 <= 1223)) AND isnotnull(d_date_sk#7)) +Input [2]: [d_date_sk#7, d_month_seq#95] +Condition : (((isnotnull(d_month_seq#95) AND (d_month_seq#95 >= 1212)) AND (d_month_seq#95 <= 1223)) AND isnotnull(d_date_sk#7)) (52) Project [codegen id : 1] Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_month_seq#91] +Input [2]: [d_date_sk#7, d_month_seq#95] (53) BroadcastExchange Input [1]: [d_date_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt index 0c4267b3ca513..042f946b8fca4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Union WholeStageCodegen (8) - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt index 4b8993f370f4d..8aab8e91acfc8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt @@ -160,125 +160,125 @@ Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#21, coun Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(qoh#18)] Aggregate Attributes [1]: [avg(qoh#18)#23] -Results [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, avg(qoh#18)#23 AS qoh#24] +Results [5]: [i_product_name#11 AS i_product_name#24, i_brand#8 AS i_brand#25, i_class#9 AS i_class#26, i_category#10 AS i_category#27, avg(qoh#18)#23 AS qoh#28] (24) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#25, i_brand#26, i_class#27, i_category#28, sum#29, count#30] +Output [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] (25) HashAggregate [codegen id : 10] -Input [6]: [i_product_name#25, i_brand#26, i_class#27, i_category#28, sum#29, count#30] -Keys [4]: [i_product_name#25, i_brand#26, i_class#27, i_category#28] -Functions [1]: [avg(inv_quantity_on_hand#31)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#31)#17] -Results [4]: [i_product_name#25, i_brand#26, i_class#27, avg(inv_quantity_on_hand#31)#17 AS qoh#32] +Input [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] +Keys [4]: [i_product_name#29, i_brand#30, i_class#31, i_category#32] +Functions [1]: [avg(inv_quantity_on_hand#35)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#17] +Results [4]: [i_product_name#29, i_brand#30, i_class#31, avg(inv_quantity_on_hand#35)#17 AS qoh#36] (26) HashAggregate [codegen id : 10] -Input [4]: [i_product_name#25, i_brand#26, i_class#27, qoh#32] -Keys [3]: [i_product_name#25, i_brand#26, i_class#27] -Functions [1]: [partial_avg(qoh#32)] -Aggregate Attributes [2]: [sum#33, count#34] -Results [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36] +Input [4]: [i_product_name#29, i_brand#30, i_class#31, qoh#36] +Keys [3]: [i_product_name#29, i_brand#30, i_class#31] +Functions [1]: [partial_avg(qoh#36)] +Aggregate Attributes [2]: [sum#37, count#38] +Results [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] (27) Exchange -Input [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36] -Arguments: hashpartitioning(i_product_name#25, i_brand#26, i_class#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] +Arguments: hashpartitioning(i_product_name#29, i_brand#30, i_class#31, 5), ENSURE_REQUIREMENTS, [plan_id=4] (28) HashAggregate [codegen id : 11] -Input [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36] -Keys [3]: [i_product_name#25, i_brand#26, i_class#27] -Functions [1]: [avg(qoh#32)] -Aggregate Attributes [1]: [avg(qoh#32)#37] -Results [5]: [i_product_name#25, i_brand#26, i_class#27, null AS i_category#38, avg(qoh#32)#37 AS qoh#39] +Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] +Keys [3]: [i_product_name#29, i_brand#30, i_class#31] +Functions [1]: [avg(qoh#36)] +Aggregate Attributes [1]: [avg(qoh#36)#41] +Results [5]: [i_product_name#29, i_brand#30, i_class#31, null AS i_category#42, avg(qoh#36)#41 AS qoh#43] (29) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#40, i_brand#41, i_class#42, i_category#43, sum#44, count#45] +Output [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49] (30) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#40, i_brand#41, i_class#42, i_category#43, sum#44, count#45] -Keys [4]: [i_product_name#40, i_brand#41, i_class#42, i_category#43] -Functions [1]: [avg(inv_quantity_on_hand#46)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#46)#17] -Results [3]: [i_product_name#40, i_brand#41, avg(inv_quantity_on_hand#46)#17 AS qoh#47] +Input [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49] +Keys [4]: [i_product_name#44, i_brand#45, i_class#46, i_category#47] +Functions [1]: [avg(inv_quantity_on_hand#50)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#50)#17] +Results [3]: [i_product_name#44, i_brand#45, avg(inv_quantity_on_hand#50)#17 AS qoh#51] (31) HashAggregate [codegen id : 16] -Input [3]: [i_product_name#40, i_brand#41, qoh#47] -Keys [2]: [i_product_name#40, i_brand#41] -Functions [1]: [partial_avg(qoh#47)] -Aggregate Attributes [2]: [sum#48, count#49] -Results [4]: [i_product_name#40, i_brand#41, sum#50, count#51] +Input [3]: [i_product_name#44, i_brand#45, qoh#51] +Keys [2]: [i_product_name#44, i_brand#45] +Functions [1]: [partial_avg(qoh#51)] +Aggregate Attributes [2]: [sum#52, count#53] +Results [4]: [i_product_name#44, i_brand#45, sum#54, count#55] (32) Exchange -Input [4]: [i_product_name#40, i_brand#41, sum#50, count#51] -Arguments: hashpartitioning(i_product_name#40, i_brand#41, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55] +Arguments: hashpartitioning(i_product_name#44, i_brand#45, 5), ENSURE_REQUIREMENTS, [plan_id=5] (33) HashAggregate [codegen id : 17] -Input [4]: [i_product_name#40, i_brand#41, sum#50, count#51] -Keys [2]: [i_product_name#40, i_brand#41] -Functions [1]: [avg(qoh#47)] -Aggregate Attributes [1]: [avg(qoh#47)#52] -Results [5]: [i_product_name#40, i_brand#41, null AS i_class#53, null AS i_category#54, avg(qoh#47)#52 AS qoh#55] +Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55] +Keys [2]: [i_product_name#44, i_brand#45] +Functions [1]: [avg(qoh#51)] +Aggregate Attributes [1]: [avg(qoh#51)#56] +Results [5]: [i_product_name#44, i_brand#45, null AS i_class#57, null AS i_category#58, avg(qoh#51)#56 AS qoh#59] (34) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#56, i_brand#57, i_class#58, i_category#59, sum#60, count#61] +Output [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65] (35) HashAggregate [codegen id : 22] -Input [6]: [i_product_name#56, i_brand#57, i_class#58, i_category#59, sum#60, count#61] -Keys [4]: [i_product_name#56, i_brand#57, i_class#58, i_category#59] -Functions [1]: [avg(inv_quantity_on_hand#62)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#62)#17] -Results [2]: [i_product_name#56, avg(inv_quantity_on_hand#62)#17 AS qoh#63] +Input [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65] +Keys [4]: [i_product_name#60, i_brand#61, i_class#62, i_category#63] +Functions [1]: [avg(inv_quantity_on_hand#66)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#66)#17] +Results [2]: [i_product_name#60, avg(inv_quantity_on_hand#66)#17 AS qoh#67] (36) HashAggregate [codegen id : 22] -Input [2]: [i_product_name#56, qoh#63] -Keys [1]: [i_product_name#56] -Functions [1]: [partial_avg(qoh#63)] -Aggregate Attributes [2]: [sum#64, count#65] -Results [3]: [i_product_name#56, sum#66, count#67] +Input [2]: [i_product_name#60, qoh#67] +Keys [1]: [i_product_name#60] +Functions [1]: [partial_avg(qoh#67)] +Aggregate Attributes [2]: [sum#68, count#69] +Results [3]: [i_product_name#60, sum#70, count#71] (37) Exchange -Input [3]: [i_product_name#56, sum#66, count#67] -Arguments: hashpartitioning(i_product_name#56, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [3]: [i_product_name#60, sum#70, count#71] +Arguments: hashpartitioning(i_product_name#60, 5), ENSURE_REQUIREMENTS, [plan_id=6] (38) HashAggregate [codegen id : 23] -Input [3]: [i_product_name#56, sum#66, count#67] -Keys [1]: [i_product_name#56] -Functions [1]: [avg(qoh#63)] -Aggregate Attributes [1]: [avg(qoh#63)#68] -Results [5]: [i_product_name#56, null AS i_brand#69, null AS i_class#70, null AS i_category#71, avg(qoh#63)#68 AS qoh#72] +Input [3]: [i_product_name#60, sum#70, count#71] +Keys [1]: [i_product_name#60] +Functions [1]: [avg(qoh#67)] +Aggregate Attributes [1]: [avg(qoh#67)#72] +Results [5]: [i_product_name#60, null AS i_brand#73, null AS i_class#74, null AS i_category#75, avg(qoh#67)#72 AS qoh#76] (39) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#73, i_brand#74, i_class#75, i_category#76, sum#77, count#78] +Output [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82] (40) HashAggregate [codegen id : 28] -Input [6]: [i_product_name#73, i_brand#74, i_class#75, i_category#76, sum#77, count#78] -Keys [4]: [i_product_name#73, i_brand#74, i_class#75, i_category#76] -Functions [1]: [avg(inv_quantity_on_hand#79)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#79)#17] -Results [1]: [avg(inv_quantity_on_hand#79)#17 AS qoh#80] +Input [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82] +Keys [4]: [i_product_name#77, i_brand#78, i_class#79, i_category#80] +Functions [1]: [avg(inv_quantity_on_hand#83)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#83)#17] +Results [1]: [avg(inv_quantity_on_hand#83)#17 AS qoh#84] (41) HashAggregate [codegen id : 28] -Input [1]: [qoh#80] +Input [1]: [qoh#84] Keys: [] -Functions [1]: [partial_avg(qoh#80)] -Aggregate Attributes [2]: [sum#81, count#82] -Results [2]: [sum#83, count#84] +Functions [1]: [partial_avg(qoh#84)] +Aggregate Attributes [2]: [sum#85, count#86] +Results [2]: [sum#87, count#88] (42) Exchange -Input [2]: [sum#83, count#84] +Input [2]: [sum#87, count#88] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] (43) HashAggregate [codegen id : 29] -Input [2]: [sum#83, count#84] +Input [2]: [sum#87, count#88] Keys: [] -Functions [1]: [avg(qoh#80)] -Aggregate Attributes [1]: [avg(qoh#80)#85] -Results [5]: [null AS i_product_name#86, null AS i_brand#87, null AS i_class#88, null AS i_category#89, avg(qoh#80)#85 AS qoh#90] +Functions [1]: [avg(qoh#84)] +Aggregate Attributes [1]: [avg(qoh#84)#89] +Results [5]: [null AS i_product_name#90, null AS i_brand#91, null AS i_class#92, null AS i_category#93, avg(qoh#84)#89 AS qoh#94] (44) Union (45) TakeOrderedAndProject -Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#24] -Arguments: 100, [qoh#24 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#8 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#24] +Input [5]: [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28] +Arguments: 100, [qoh#28 ASC NULLS FIRST, i_product_name#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_class#26 ASC NULLS FIRST, i_category#27 ASC NULLS FIRST], [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28] ===== Subqueries ===== @@ -291,22 +291,22 @@ BroadcastExchange (50) (46) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#91] +Output [2]: [d_date_sk#6, d_month_seq#95] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_month_seq#91] +Input [2]: [d_date_sk#6, d_month_seq#95] (48) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_month_seq#91] -Condition : (((isnotnull(d_month_seq#91) AND (d_month_seq#91 >= 1212)) AND (d_month_seq#91 <= 1223)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#6, d_month_seq#95] +Condition : (((isnotnull(d_month_seq#95) AND (d_month_seq#95 >= 1212)) AND (d_month_seq#95 <= 1223)) AND isnotnull(d_date_sk#6)) (49) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_month_seq#91] +Input [2]: [d_date_sk#6, d_month_seq#95] (50) BroadcastExchange Input [1]: [d_date_sk#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt index 22f73cc9b9db5..d747066f5945b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Union WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt index 9c28ff9f351d8..a4c009f8219b4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt @@ -186,265 +186,265 @@ Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#23] +Results [9]: [i_category#16 AS i_category#23, i_class#15 AS i_class#24, i_brand#14 AS i_brand#25, i_product_name#17 AS i_product_name#26, d_year#8 AS d_year#27, d_qoy#10 AS d_qoy#28, d_moy#9 AS d_moy#29, s_store_id#12 AS s_store_id#30, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#31] (25) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sum#32, isEmpty#33] +Output [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41] (26) HashAggregate [codegen id : 16] -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sum#32, isEmpty#33] -Keys [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31] -Functions [1]: [sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))#22] -Results [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))#22 AS sumsales#36] +Input [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41] +Keys [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39] +Functions [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22] +Results [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22 AS sumsales#44] (27) HashAggregate [codegen id : 16] -Input [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sumsales#36] -Keys [7]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30] -Functions [1]: [partial_sum(sumsales#36)] -Aggregate Attributes [2]: [sum#37, isEmpty#38] -Results [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40] +Input [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sumsales#44] +Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38] +Functions [1]: [partial_sum(sumsales#44)] +Aggregate Attributes [2]: [sum#45, isEmpty#46] +Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] (28) Exchange -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40] -Arguments: hashpartitioning(i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, 5), ENSURE_REQUIREMENTS, [plan_id=5] (29) HashAggregate [codegen id : 17] -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40] -Keys [7]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30] -Functions [1]: [sum(sumsales#36)] -Aggregate Attributes [1]: [sum(sumsales#36)#41] -Results [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, null AS s_store_id#42, sum(sumsales#36)#41 AS sumsales#43] +Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] +Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38] +Functions [1]: [sum(sumsales#44)] +Aggregate Attributes [1]: [sum(sumsales#44)#49] +Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, null AS s_store_id#50, sum(sumsales#44)#49 AS sumsales#51] (30) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51, sum#52, isEmpty#53] +Output [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] (31) HashAggregate [codegen id : 25] -Input [10]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51, sum#52, isEmpty#53] -Keys [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51] -Functions [1]: [sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))#22] -Results [7]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))#22 AS sumsales#56] +Input [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] +Keys [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59] +Functions [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22] +Results [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22 AS sumsales#64] (32) HashAggregate [codegen id : 25] -Input [7]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sumsales#56] -Keys [6]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49] -Functions [1]: [partial_sum(sumsales#56)] -Aggregate Attributes [2]: [sum#57, isEmpty#58] -Results [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60] +Input [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sumsales#64] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [partial_sum(sumsales#64)] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] (33) Exchange -Input [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60] -Arguments: hashpartitioning(i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Arguments: hashpartitioning(i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, 5), ENSURE_REQUIREMENTS, [plan_id=6] (34) HashAggregate [codegen id : 26] -Input [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60] -Keys [6]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49] -Functions [1]: [sum(sumsales#56)] -Aggregate Attributes [1]: [sum(sumsales#56)#61] -Results [9]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, null AS d_moy#62, null AS s_store_id#63, sum(sumsales#56)#61 AS sumsales#64] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [sum(sumsales#64)] +Aggregate Attributes [1]: [sum(sumsales#64)#69] +Results [9]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, null AS d_moy#70, null AS s_store_id#71, sum(sumsales#64)#69 AS sumsales#72] (35) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72, sum#73, isEmpty#74] +Output [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] (36) HashAggregate [codegen id : 34] -Input [10]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72, sum#73, isEmpty#74] -Keys [8]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72] -Functions [1]: [sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))#22] -Results [6]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))#22 AS sumsales#77] +Input [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] +Keys [8]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80] +Functions [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22] +Results [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22 AS sumsales#85] (37) HashAggregate [codegen id : 34] -Input [6]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sumsales#77] -Keys [5]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69] -Functions [1]: [partial_sum(sumsales#77)] -Aggregate Attributes [2]: [sum#78, isEmpty#79] -Results [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81] +Input [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sumsales#85] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [partial_sum(sumsales#85)] +Aggregate Attributes [2]: [sum#86, isEmpty#87] +Results [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] (38) Exchange -Input [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81] -Arguments: hashpartitioning(i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Arguments: hashpartitioning(i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, 5), ENSURE_REQUIREMENTS, [plan_id=7] (39) HashAggregate [codegen id : 35] -Input [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81] -Keys [5]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69] -Functions [1]: [sum(sumsales#77)] -Aggregate Attributes [1]: [sum(sumsales#77)#82] -Results [9]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, null AS d_qoy#83, null AS d_moy#84, null AS s_store_id#85, sum(sumsales#77)#82 AS sumsales#86] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [sum(sumsales#85)] +Aggregate Attributes [1]: [sum(sumsales#85)#90] +Results [9]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, null AS d_qoy#91, null AS d_moy#92, null AS s_store_id#93, sum(sumsales#85)#90 AS sumsales#94] (40) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94, sum#95, isEmpty#96] +Output [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104] (41) HashAggregate [codegen id : 43] -Input [10]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94, sum#95, isEmpty#96] -Keys [8]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94] -Functions [1]: [sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))#22] -Results [5]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))#22 AS sumsales#99] +Input [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104] +Keys [8]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102] +Functions [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22] +Results [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22 AS sumsales#107] (42) HashAggregate [codegen id : 43] -Input [5]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sumsales#99] -Keys [4]: [i_category#87, i_class#88, i_brand#89, i_product_name#90] -Functions [1]: [partial_sum(sumsales#99)] -Aggregate Attributes [2]: [sum#100, isEmpty#101] -Results [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103] +Input [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sumsales#107] +Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98] +Functions [1]: [partial_sum(sumsales#107)] +Aggregate Attributes [2]: [sum#108, isEmpty#109] +Results [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] (43) Exchange -Input [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103] -Arguments: hashpartitioning(i_category#87, i_class#88, i_brand#89, i_product_name#90, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] +Arguments: hashpartitioning(i_category#95, i_class#96, i_brand#97, i_product_name#98, 5), ENSURE_REQUIREMENTS, [plan_id=8] (44) HashAggregate [codegen id : 44] -Input [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103] -Keys [4]: [i_category#87, i_class#88, i_brand#89, i_product_name#90] -Functions [1]: [sum(sumsales#99)] -Aggregate Attributes [1]: [sum(sumsales#99)#104] -Results [9]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, null AS d_year#105, null AS d_qoy#106, null AS d_moy#107, null AS s_store_id#108, sum(sumsales#99)#104 AS sumsales#109] +Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] +Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98] +Functions [1]: [sum(sumsales#107)] +Aggregate Attributes [1]: [sum(sumsales#107)#112] +Results [9]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, null AS d_year#113, null AS d_qoy#114, null AS d_moy#115, null AS s_store_id#116, sum(sumsales#107)#112 AS sumsales#117] (45) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117, sum#118, isEmpty#119] +Output [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127] (46) HashAggregate [codegen id : 52] -Input [10]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117, sum#118, isEmpty#119] -Keys [8]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117] -Functions [1]: [sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))#22] -Results [4]: [i_category#110, i_class#111, i_brand#112, sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))#22 AS sumsales#122] +Input [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127] +Keys [8]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125] +Functions [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22] +Results [4]: [i_category#118, i_class#119, i_brand#120, sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22 AS sumsales#130] (47) HashAggregate [codegen id : 52] -Input [4]: [i_category#110, i_class#111, i_brand#112, sumsales#122] -Keys [3]: [i_category#110, i_class#111, i_brand#112] -Functions [1]: [partial_sum(sumsales#122)] -Aggregate Attributes [2]: [sum#123, isEmpty#124] -Results [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126] +Input [4]: [i_category#118, i_class#119, i_brand#120, sumsales#130] +Keys [3]: [i_category#118, i_class#119, i_brand#120] +Functions [1]: [partial_sum(sumsales#130)] +Aggregate Attributes [2]: [sum#131, isEmpty#132] +Results [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] (48) Exchange -Input [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126] -Arguments: hashpartitioning(i_category#110, i_class#111, i_brand#112, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] +Arguments: hashpartitioning(i_category#118, i_class#119, i_brand#120, 5), ENSURE_REQUIREMENTS, [plan_id=9] (49) HashAggregate [codegen id : 53] -Input [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126] -Keys [3]: [i_category#110, i_class#111, i_brand#112] -Functions [1]: [sum(sumsales#122)] -Aggregate Attributes [1]: [sum(sumsales#122)#127] -Results [9]: [i_category#110, i_class#111, i_brand#112, null AS i_product_name#128, null AS d_year#129, null AS d_qoy#130, null AS d_moy#131, null AS s_store_id#132, sum(sumsales#122)#127 AS sumsales#133] +Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] +Keys [3]: [i_category#118, i_class#119, i_brand#120] +Functions [1]: [sum(sumsales#130)] +Aggregate Attributes [1]: [sum(sumsales#130)#135] +Results [9]: [i_category#118, i_class#119, i_brand#120, null AS i_product_name#136, null AS d_year#137, null AS d_qoy#138, null AS d_moy#139, null AS s_store_id#140, sum(sumsales#130)#135 AS sumsales#141] (50) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141, sum#142, isEmpty#143] +Output [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151] (51) HashAggregate [codegen id : 61] -Input [10]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141, sum#142, isEmpty#143] -Keys [8]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141] -Functions [1]: [sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))#22] -Results [3]: [i_category#134, i_class#135, sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))#22 AS sumsales#146] +Input [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151] +Keys [8]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149] +Functions [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22] +Results [3]: [i_category#142, i_class#143, sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22 AS sumsales#154] (52) HashAggregate [codegen id : 61] -Input [3]: [i_category#134, i_class#135, sumsales#146] -Keys [2]: [i_category#134, i_class#135] -Functions [1]: [partial_sum(sumsales#146)] -Aggregate Attributes [2]: [sum#147, isEmpty#148] -Results [4]: [i_category#134, i_class#135, sum#149, isEmpty#150] +Input [3]: [i_category#142, i_class#143, sumsales#154] +Keys [2]: [i_category#142, i_class#143] +Functions [1]: [partial_sum(sumsales#154)] +Aggregate Attributes [2]: [sum#155, isEmpty#156] +Results [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] (53) Exchange -Input [4]: [i_category#134, i_class#135, sum#149, isEmpty#150] -Arguments: hashpartitioning(i_category#134, i_class#135, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] +Arguments: hashpartitioning(i_category#142, i_class#143, 5), ENSURE_REQUIREMENTS, [plan_id=10] (54) HashAggregate [codegen id : 62] -Input [4]: [i_category#134, i_class#135, sum#149, isEmpty#150] -Keys [2]: [i_category#134, i_class#135] -Functions [1]: [sum(sumsales#146)] -Aggregate Attributes [1]: [sum(sumsales#146)#151] -Results [9]: [i_category#134, i_class#135, null AS i_brand#152, null AS i_product_name#153, null AS d_year#154, null AS d_qoy#155, null AS d_moy#156, null AS s_store_id#157, sum(sumsales#146)#151 AS sumsales#158] +Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] +Keys [2]: [i_category#142, i_class#143] +Functions [1]: [sum(sumsales#154)] +Aggregate Attributes [1]: [sum(sumsales#154)#159] +Results [9]: [i_category#142, i_class#143, null AS i_brand#160, null AS i_product_name#161, null AS d_year#162, null AS d_qoy#163, null AS d_moy#164, null AS s_store_id#165, sum(sumsales#154)#159 AS sumsales#166] (55) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166, sum#167, isEmpty#168] +Output [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176] (56) HashAggregate [codegen id : 70] -Input [10]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166, sum#167, isEmpty#168] -Keys [8]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166] -Functions [1]: [sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))#22] -Results [2]: [i_category#159, sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))#22 AS sumsales#171] +Input [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176] +Keys [8]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174] +Functions [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22] +Results [2]: [i_category#167, sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22 AS sumsales#179] (57) HashAggregate [codegen id : 70] -Input [2]: [i_category#159, sumsales#171] -Keys [1]: [i_category#159] -Functions [1]: [partial_sum(sumsales#171)] -Aggregate Attributes [2]: [sum#172, isEmpty#173] -Results [3]: [i_category#159, sum#174, isEmpty#175] +Input [2]: [i_category#167, sumsales#179] +Keys [1]: [i_category#167] +Functions [1]: [partial_sum(sumsales#179)] +Aggregate Attributes [2]: [sum#180, isEmpty#181] +Results [3]: [i_category#167, sum#182, isEmpty#183] (58) Exchange -Input [3]: [i_category#159, sum#174, isEmpty#175] -Arguments: hashpartitioning(i_category#159, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Input [3]: [i_category#167, sum#182, isEmpty#183] +Arguments: hashpartitioning(i_category#167, 5), ENSURE_REQUIREMENTS, [plan_id=11] (59) HashAggregate [codegen id : 71] -Input [3]: [i_category#159, sum#174, isEmpty#175] -Keys [1]: [i_category#159] -Functions [1]: [sum(sumsales#171)] -Aggregate Attributes [1]: [sum(sumsales#171)#176] -Results [9]: [i_category#159, null AS i_class#177, null AS i_brand#178, null AS i_product_name#179, null AS d_year#180, null AS d_qoy#181, null AS d_moy#182, null AS s_store_id#183, sum(sumsales#171)#176 AS sumsales#184] +Input [3]: [i_category#167, sum#182, isEmpty#183] +Keys [1]: [i_category#167] +Functions [1]: [sum(sumsales#179)] +Aggregate Attributes [1]: [sum(sumsales#179)#184] +Results [9]: [i_category#167, null AS i_class#185, null AS i_brand#186, null AS i_product_name#187, null AS d_year#188, null AS d_qoy#189, null AS d_moy#190, null AS s_store_id#191, sum(sumsales#179)#184 AS sumsales#192] (60) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192, sum#193, isEmpty#194] +Output [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202] (61) HashAggregate [codegen id : 79] -Input [10]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192, sum#193, isEmpty#194] -Keys [8]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192] -Functions [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))#22] -Results [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))#22 AS sumsales#197] +Input [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202] +Keys [8]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200] +Functions [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22] +Results [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22 AS sumsales#205] (62) HashAggregate [codegen id : 79] -Input [1]: [sumsales#197] +Input [1]: [sumsales#205] Keys: [] -Functions [1]: [partial_sum(sumsales#197)] -Aggregate Attributes [2]: [sum#198, isEmpty#199] -Results [2]: [sum#200, isEmpty#201] +Functions [1]: [partial_sum(sumsales#205)] +Aggregate Attributes [2]: [sum#206, isEmpty#207] +Results [2]: [sum#208, isEmpty#209] (63) Exchange -Input [2]: [sum#200, isEmpty#201] +Input [2]: [sum#208, isEmpty#209] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] (64) HashAggregate [codegen id : 80] -Input [2]: [sum#200, isEmpty#201] +Input [2]: [sum#208, isEmpty#209] Keys: [] -Functions [1]: [sum(sumsales#197)] -Aggregate Attributes [1]: [sum(sumsales#197)#202] -Results [9]: [null AS i_category#203, null AS i_class#204, null AS i_brand#205, null AS i_product_name#206, null AS d_year#207, null AS d_qoy#208, null AS d_moy#209, null AS s_store_id#210, sum(sumsales#197)#202 AS sumsales#211] +Functions [1]: [sum(sumsales#205)] +Aggregate Attributes [1]: [sum(sumsales#205)#210] +Results [9]: [null AS i_category#211, null AS i_class#212, null AS i_brand#213, null AS i_product_name#214, null AS d_year#215, null AS d_qoy#216, null AS d_moy#217, null AS s_store_id#218, sum(sumsales#205)#210 AS sumsales#219] (65) Union (66) Sort [codegen id : 81] -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0 +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 (67) WindowGroupLimit -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Partial +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Partial (68) Exchange -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: hashpartitioning(i_category#23, 5), ENSURE_REQUIREMENTS, [plan_id=13] (69) Sort [codegen id : 82] -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0 +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 (70) WindowGroupLimit -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Final +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Final (71) Window -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [rank(sumsales#23) windowspecdefinition(i_category#16, sumsales#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#212], [i_category#16], [sumsales#23 DESC NULLS LAST] +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [rank(sumsales#31) windowspecdefinition(i_category#23, sumsales#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#220], [i_category#23], [sumsales#31 DESC NULLS LAST] (72) Filter [codegen id : 83] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212] -Condition : (rk#212 <= 100) +Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] +Condition : (rk#220 <= 100) (73) TakeOrderedAndProject -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212] -Arguments: 100, [i_category#16 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#23 ASC NULLS FIRST, rk#212 ASC NULLS FIRST], [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212] +Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] +Arguments: 100, [i_category#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_product_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_qoy#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, sumsales#31 ASC NULLS FIRST, rk#220 ASC NULLS FIRST], [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] ===== Subqueries ===== @@ -457,22 +457,22 @@ BroadcastExchange (78) (74) Scan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] +Output [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 1] -Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] (76) Filter [codegen id : 1] -Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#213) AND (d_month_seq#213 >= 1212)) AND (d_month_seq#213 <= 1223)) AND isnotnull(d_date_sk#7)) +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] +Condition : (((isnotnull(d_month_seq#221) AND (d_month_seq#221 >= 1212)) AND (d_month_seq#221 <= 1223)) AND isnotnull(d_date_sk#7)) (77) Project [codegen id : 1] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] (78) BroadcastExchange Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt index 795fa297b9bad..b6a4358c4d43b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ InputAdapter Union WholeStageCodegen (8) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 WholeStageCodegen (7) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt index 75d526da4ba71..417af4fe924ee 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt @@ -171,265 +171,265 @@ Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#23] +Results [9]: [i_category#16 AS i_category#23, i_class#15 AS i_class#24, i_brand#14 AS i_brand#25, i_product_name#17 AS i_product_name#26, d_year#8 AS d_year#27, d_qoy#10 AS d_qoy#28, d_moy#9 AS d_moy#29, s_store_id#12 AS s_store_id#30, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#31] (22) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sum#32, isEmpty#33] +Output [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41] (23) HashAggregate [codegen id : 10] -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sum#32, isEmpty#33] -Keys [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31] -Functions [1]: [sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))#22] -Results [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))#22 AS sumsales#36] +Input [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41] +Keys [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39] +Functions [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22] +Results [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22 AS sumsales#44] (24) HashAggregate [codegen id : 10] -Input [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sumsales#36] -Keys [7]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30] -Functions [1]: [partial_sum(sumsales#36)] -Aggregate Attributes [2]: [sum#37, isEmpty#38] -Results [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40] +Input [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sumsales#44] +Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38] +Functions [1]: [partial_sum(sumsales#44)] +Aggregate Attributes [2]: [sum#45, isEmpty#46] +Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] (25) Exchange -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40] -Arguments: hashpartitioning(i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, 5), ENSURE_REQUIREMENTS, [plan_id=4] (26) HashAggregate [codegen id : 11] -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40] -Keys [7]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30] -Functions [1]: [sum(sumsales#36)] -Aggregate Attributes [1]: [sum(sumsales#36)#41] -Results [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, null AS s_store_id#42, sum(sumsales#36)#41 AS sumsales#43] +Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] +Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38] +Functions [1]: [sum(sumsales#44)] +Aggregate Attributes [1]: [sum(sumsales#44)#49] +Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, null AS s_store_id#50, sum(sumsales#44)#49 AS sumsales#51] (27) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51, sum#52, isEmpty#53] +Output [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] (28) HashAggregate [codegen id : 16] -Input [10]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51, sum#52, isEmpty#53] -Keys [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51] -Functions [1]: [sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))#22] -Results [7]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))#22 AS sumsales#56] +Input [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] +Keys [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59] +Functions [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22] +Results [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22 AS sumsales#64] (29) HashAggregate [codegen id : 16] -Input [7]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sumsales#56] -Keys [6]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49] -Functions [1]: [partial_sum(sumsales#56)] -Aggregate Attributes [2]: [sum#57, isEmpty#58] -Results [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60] +Input [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sumsales#64] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [partial_sum(sumsales#64)] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] (30) Exchange -Input [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60] -Arguments: hashpartitioning(i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Arguments: hashpartitioning(i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, 5), ENSURE_REQUIREMENTS, [plan_id=5] (31) HashAggregate [codegen id : 17] -Input [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60] -Keys [6]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49] -Functions [1]: [sum(sumsales#56)] -Aggregate Attributes [1]: [sum(sumsales#56)#61] -Results [9]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, null AS d_moy#62, null AS s_store_id#63, sum(sumsales#56)#61 AS sumsales#64] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [sum(sumsales#64)] +Aggregate Attributes [1]: [sum(sumsales#64)#69] +Results [9]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, null AS d_moy#70, null AS s_store_id#71, sum(sumsales#64)#69 AS sumsales#72] (32) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72, sum#73, isEmpty#74] +Output [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] (33) HashAggregate [codegen id : 22] -Input [10]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72, sum#73, isEmpty#74] -Keys [8]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72] -Functions [1]: [sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))#22] -Results [6]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))#22 AS sumsales#77] +Input [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] +Keys [8]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80] +Functions [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22] +Results [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22 AS sumsales#85] (34) HashAggregate [codegen id : 22] -Input [6]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sumsales#77] -Keys [5]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69] -Functions [1]: [partial_sum(sumsales#77)] -Aggregate Attributes [2]: [sum#78, isEmpty#79] -Results [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81] +Input [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sumsales#85] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [partial_sum(sumsales#85)] +Aggregate Attributes [2]: [sum#86, isEmpty#87] +Results [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] (35) Exchange -Input [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81] -Arguments: hashpartitioning(i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Arguments: hashpartitioning(i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, 5), ENSURE_REQUIREMENTS, [plan_id=6] (36) HashAggregate [codegen id : 23] -Input [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81] -Keys [5]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69] -Functions [1]: [sum(sumsales#77)] -Aggregate Attributes [1]: [sum(sumsales#77)#82] -Results [9]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, null AS d_qoy#83, null AS d_moy#84, null AS s_store_id#85, sum(sumsales#77)#82 AS sumsales#86] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [sum(sumsales#85)] +Aggregate Attributes [1]: [sum(sumsales#85)#90] +Results [9]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, null AS d_qoy#91, null AS d_moy#92, null AS s_store_id#93, sum(sumsales#85)#90 AS sumsales#94] (37) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94, sum#95, isEmpty#96] +Output [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104] (38) HashAggregate [codegen id : 28] -Input [10]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94, sum#95, isEmpty#96] -Keys [8]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94] -Functions [1]: [sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))#22] -Results [5]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))#22 AS sumsales#99] +Input [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104] +Keys [8]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102] +Functions [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22] +Results [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22 AS sumsales#107] (39) HashAggregate [codegen id : 28] -Input [5]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sumsales#99] -Keys [4]: [i_category#87, i_class#88, i_brand#89, i_product_name#90] -Functions [1]: [partial_sum(sumsales#99)] -Aggregate Attributes [2]: [sum#100, isEmpty#101] -Results [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103] +Input [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sumsales#107] +Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98] +Functions [1]: [partial_sum(sumsales#107)] +Aggregate Attributes [2]: [sum#108, isEmpty#109] +Results [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] (40) Exchange -Input [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103] -Arguments: hashpartitioning(i_category#87, i_class#88, i_brand#89, i_product_name#90, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] +Arguments: hashpartitioning(i_category#95, i_class#96, i_brand#97, i_product_name#98, 5), ENSURE_REQUIREMENTS, [plan_id=7] (41) HashAggregate [codegen id : 29] -Input [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103] -Keys [4]: [i_category#87, i_class#88, i_brand#89, i_product_name#90] -Functions [1]: [sum(sumsales#99)] -Aggregate Attributes [1]: [sum(sumsales#99)#104] -Results [9]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, null AS d_year#105, null AS d_qoy#106, null AS d_moy#107, null AS s_store_id#108, sum(sumsales#99)#104 AS sumsales#109] +Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] +Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98] +Functions [1]: [sum(sumsales#107)] +Aggregate Attributes [1]: [sum(sumsales#107)#112] +Results [9]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, null AS d_year#113, null AS d_qoy#114, null AS d_moy#115, null AS s_store_id#116, sum(sumsales#107)#112 AS sumsales#117] (42) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117, sum#118, isEmpty#119] +Output [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127] (43) HashAggregate [codegen id : 34] -Input [10]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117, sum#118, isEmpty#119] -Keys [8]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117] -Functions [1]: [sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))#22] -Results [4]: [i_category#110, i_class#111, i_brand#112, sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))#22 AS sumsales#122] +Input [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127] +Keys [8]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125] +Functions [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22] +Results [4]: [i_category#118, i_class#119, i_brand#120, sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22 AS sumsales#130] (44) HashAggregate [codegen id : 34] -Input [4]: [i_category#110, i_class#111, i_brand#112, sumsales#122] -Keys [3]: [i_category#110, i_class#111, i_brand#112] -Functions [1]: [partial_sum(sumsales#122)] -Aggregate Attributes [2]: [sum#123, isEmpty#124] -Results [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126] +Input [4]: [i_category#118, i_class#119, i_brand#120, sumsales#130] +Keys [3]: [i_category#118, i_class#119, i_brand#120] +Functions [1]: [partial_sum(sumsales#130)] +Aggregate Attributes [2]: [sum#131, isEmpty#132] +Results [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] (45) Exchange -Input [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126] -Arguments: hashpartitioning(i_category#110, i_class#111, i_brand#112, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] +Arguments: hashpartitioning(i_category#118, i_class#119, i_brand#120, 5), ENSURE_REQUIREMENTS, [plan_id=8] (46) HashAggregate [codegen id : 35] -Input [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126] -Keys [3]: [i_category#110, i_class#111, i_brand#112] -Functions [1]: [sum(sumsales#122)] -Aggregate Attributes [1]: [sum(sumsales#122)#127] -Results [9]: [i_category#110, i_class#111, i_brand#112, null AS i_product_name#128, null AS d_year#129, null AS d_qoy#130, null AS d_moy#131, null AS s_store_id#132, sum(sumsales#122)#127 AS sumsales#133] +Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] +Keys [3]: [i_category#118, i_class#119, i_brand#120] +Functions [1]: [sum(sumsales#130)] +Aggregate Attributes [1]: [sum(sumsales#130)#135] +Results [9]: [i_category#118, i_class#119, i_brand#120, null AS i_product_name#136, null AS d_year#137, null AS d_qoy#138, null AS d_moy#139, null AS s_store_id#140, sum(sumsales#130)#135 AS sumsales#141] (47) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141, sum#142, isEmpty#143] +Output [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151] (48) HashAggregate [codegen id : 40] -Input [10]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141, sum#142, isEmpty#143] -Keys [8]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141] -Functions [1]: [sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))#22] -Results [3]: [i_category#134, i_class#135, sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))#22 AS sumsales#146] +Input [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151] +Keys [8]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149] +Functions [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22] +Results [3]: [i_category#142, i_class#143, sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22 AS sumsales#154] (49) HashAggregate [codegen id : 40] -Input [3]: [i_category#134, i_class#135, sumsales#146] -Keys [2]: [i_category#134, i_class#135] -Functions [1]: [partial_sum(sumsales#146)] -Aggregate Attributes [2]: [sum#147, isEmpty#148] -Results [4]: [i_category#134, i_class#135, sum#149, isEmpty#150] +Input [3]: [i_category#142, i_class#143, sumsales#154] +Keys [2]: [i_category#142, i_class#143] +Functions [1]: [partial_sum(sumsales#154)] +Aggregate Attributes [2]: [sum#155, isEmpty#156] +Results [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] (50) Exchange -Input [4]: [i_category#134, i_class#135, sum#149, isEmpty#150] -Arguments: hashpartitioning(i_category#134, i_class#135, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] +Arguments: hashpartitioning(i_category#142, i_class#143, 5), ENSURE_REQUIREMENTS, [plan_id=9] (51) HashAggregate [codegen id : 41] -Input [4]: [i_category#134, i_class#135, sum#149, isEmpty#150] -Keys [2]: [i_category#134, i_class#135] -Functions [1]: [sum(sumsales#146)] -Aggregate Attributes [1]: [sum(sumsales#146)#151] -Results [9]: [i_category#134, i_class#135, null AS i_brand#152, null AS i_product_name#153, null AS d_year#154, null AS d_qoy#155, null AS d_moy#156, null AS s_store_id#157, sum(sumsales#146)#151 AS sumsales#158] +Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] +Keys [2]: [i_category#142, i_class#143] +Functions [1]: [sum(sumsales#154)] +Aggregate Attributes [1]: [sum(sumsales#154)#159] +Results [9]: [i_category#142, i_class#143, null AS i_brand#160, null AS i_product_name#161, null AS d_year#162, null AS d_qoy#163, null AS d_moy#164, null AS s_store_id#165, sum(sumsales#154)#159 AS sumsales#166] (52) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166, sum#167, isEmpty#168] +Output [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176] (53) HashAggregate [codegen id : 46] -Input [10]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166, sum#167, isEmpty#168] -Keys [8]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166] -Functions [1]: [sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))#22] -Results [2]: [i_category#159, sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))#22 AS sumsales#171] +Input [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176] +Keys [8]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174] +Functions [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22] +Results [2]: [i_category#167, sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22 AS sumsales#179] (54) HashAggregate [codegen id : 46] -Input [2]: [i_category#159, sumsales#171] -Keys [1]: [i_category#159] -Functions [1]: [partial_sum(sumsales#171)] -Aggregate Attributes [2]: [sum#172, isEmpty#173] -Results [3]: [i_category#159, sum#174, isEmpty#175] +Input [2]: [i_category#167, sumsales#179] +Keys [1]: [i_category#167] +Functions [1]: [partial_sum(sumsales#179)] +Aggregate Attributes [2]: [sum#180, isEmpty#181] +Results [3]: [i_category#167, sum#182, isEmpty#183] (55) Exchange -Input [3]: [i_category#159, sum#174, isEmpty#175] -Arguments: hashpartitioning(i_category#159, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [3]: [i_category#167, sum#182, isEmpty#183] +Arguments: hashpartitioning(i_category#167, 5), ENSURE_REQUIREMENTS, [plan_id=10] (56) HashAggregate [codegen id : 47] -Input [3]: [i_category#159, sum#174, isEmpty#175] -Keys [1]: [i_category#159] -Functions [1]: [sum(sumsales#171)] -Aggregate Attributes [1]: [sum(sumsales#171)#176] -Results [9]: [i_category#159, null AS i_class#177, null AS i_brand#178, null AS i_product_name#179, null AS d_year#180, null AS d_qoy#181, null AS d_moy#182, null AS s_store_id#183, sum(sumsales#171)#176 AS sumsales#184] +Input [3]: [i_category#167, sum#182, isEmpty#183] +Keys [1]: [i_category#167] +Functions [1]: [sum(sumsales#179)] +Aggregate Attributes [1]: [sum(sumsales#179)#184] +Results [9]: [i_category#167, null AS i_class#185, null AS i_brand#186, null AS i_product_name#187, null AS d_year#188, null AS d_qoy#189, null AS d_moy#190, null AS s_store_id#191, sum(sumsales#179)#184 AS sumsales#192] (57) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192, sum#193, isEmpty#194] +Output [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202] (58) HashAggregate [codegen id : 52] -Input [10]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192, sum#193, isEmpty#194] -Keys [8]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192] -Functions [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))#22] -Results [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))#22 AS sumsales#197] +Input [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202] +Keys [8]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200] +Functions [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22] +Results [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22 AS sumsales#205] (59) HashAggregate [codegen id : 52] -Input [1]: [sumsales#197] +Input [1]: [sumsales#205] Keys: [] -Functions [1]: [partial_sum(sumsales#197)] -Aggregate Attributes [2]: [sum#198, isEmpty#199] -Results [2]: [sum#200, isEmpty#201] +Functions [1]: [partial_sum(sumsales#205)] +Aggregate Attributes [2]: [sum#206, isEmpty#207] +Results [2]: [sum#208, isEmpty#209] (60) Exchange -Input [2]: [sum#200, isEmpty#201] +Input [2]: [sum#208, isEmpty#209] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] (61) HashAggregate [codegen id : 53] -Input [2]: [sum#200, isEmpty#201] +Input [2]: [sum#208, isEmpty#209] Keys: [] -Functions [1]: [sum(sumsales#197)] -Aggregate Attributes [1]: [sum(sumsales#197)#202] -Results [9]: [null AS i_category#203, null AS i_class#204, null AS i_brand#205, null AS i_product_name#206, null AS d_year#207, null AS d_qoy#208, null AS d_moy#209, null AS s_store_id#210, sum(sumsales#197)#202 AS sumsales#211] +Functions [1]: [sum(sumsales#205)] +Aggregate Attributes [1]: [sum(sumsales#205)#210] +Results [9]: [null AS i_category#211, null AS i_class#212, null AS i_brand#213, null AS i_product_name#214, null AS d_year#215, null AS d_qoy#216, null AS d_moy#217, null AS s_store_id#218, sum(sumsales#205)#210 AS sumsales#219] (62) Union (63) Sort [codegen id : 54] -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0 +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 (64) WindowGroupLimit -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Partial +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Partial (65) Exchange -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: hashpartitioning(i_category#23, 5), ENSURE_REQUIREMENTS, [plan_id=12] (66) Sort [codegen id : 55] -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0 +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 (67) WindowGroupLimit -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Final +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Final (68) Window -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [rank(sumsales#23) windowspecdefinition(i_category#16, sumsales#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#212], [i_category#16], [sumsales#23 DESC NULLS LAST] +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [rank(sumsales#31) windowspecdefinition(i_category#23, sumsales#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#220], [i_category#23], [sumsales#31 DESC NULLS LAST] (69) Filter [codegen id : 56] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212] -Condition : (rk#212 <= 100) +Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] +Condition : (rk#220 <= 100) (70) TakeOrderedAndProject -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212] -Arguments: 100, [i_category#16 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#23 ASC NULLS FIRST, rk#212 ASC NULLS FIRST], [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212] +Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] +Arguments: 100, [i_category#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_product_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_qoy#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, sumsales#31 ASC NULLS FIRST, rk#220 ASC NULLS FIRST], [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] ===== Subqueries ===== @@ -442,22 +442,22 @@ BroadcastExchange (75) (71) Scan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] +Output [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct (72) ColumnarToRow [codegen id : 1] -Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] (73) Filter [codegen id : 1] -Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#213) AND (d_month_seq#213 >= 1212)) AND (d_month_seq#213 <= 1223)) AND isnotnull(d_date_sk#7)) +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] +Condition : (((isnotnull(d_month_seq#221) AND (d_month_seq#221 >= 1212)) AND (d_month_seq#221 <= 1223)) AND isnotnull(d_date_sk#7)) (74) Project [codegen id : 1] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] (75) BroadcastExchange Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt index 89393f265a49f..5a43dced056bd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ InputAdapter Union WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 WholeStageCodegen (4) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala index 879c0c480943d..8600ec4f8787f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala @@ -741,7 +741,7 @@ class CollationExpressionWalkerSuite extends SparkFunSuite with SharedSparkSessi assert(resultUTF8.collect() === resultUTF8Lcase.collect()) } } catch { - case e: SparkRuntimeException => assert(e.getErrorClass == "USER_RAISED_EXCEPTION") + case e: SparkRuntimeException => assert(e.getCondition == "USER_RAISED_EXCEPTION") case other: Throwable => throw other } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala index 4c3cd93873bd4..d568cd77050fd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala @@ -49,9 +49,13 @@ class CollationSQLExpressionsSuite val testCases = Seq( Md5TestCase("Spark", "UTF8_BINARY", "8cde774d6f7333752ed72cacddb05126"), + Md5TestCase("Spark", "UTF8_BINARY_RTRIM", "8cde774d6f7333752ed72cacddb05126"), Md5TestCase("Spark", "UTF8_LCASE", "8cde774d6f7333752ed72cacddb05126"), + Md5TestCase("Spark", "UTF8_LCASE_RTRIM", "8cde774d6f7333752ed72cacddb05126"), Md5TestCase("SQL", "UNICODE", "9778840a0100cb30c982876741b0b5a2"), - Md5TestCase("SQL", "UNICODE_CI", "9778840a0100cb30c982876741b0b5a2") + Md5TestCase("SQL", "UNICODE_RTRIM", "9778840a0100cb30c982876741b0b5a2"), + Md5TestCase("SQL", "UNICODE_CI", "9778840a0100cb30c982876741b0b5a2"), + Md5TestCase("SQL", "UNICODE_CI_RTRIM", "9778840a0100cb30c982876741b0b5a2") ) // Supported collations @@ -81,11 +85,19 @@ class CollationSQLExpressionsSuite val testCases = Seq( Sha2TestCase("Spark", "UTF8_BINARY", 256, "529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b"), + Sha2TestCase("Spark", "UTF8_BINARY_RTRIM", 256, + "529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b"), Sha2TestCase("Spark", "UTF8_LCASE", 256, "529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b"), + Sha2TestCase("Spark", "UTF8_LCASE_RTRIM", 256, + "529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b"), Sha2TestCase("SQL", "UNICODE", 256, "a7056a455639d1c7deec82ee787db24a0c1878e2792b4597709f0facf7cc7b35"), + Sha2TestCase("SQL", "UNICODE_RTRIM", 256, + "a7056a455639d1c7deec82ee787db24a0c1878e2792b4597709f0facf7cc7b35"), Sha2TestCase("SQL", "UNICODE_CI", 256, + "a7056a455639d1c7deec82ee787db24a0c1878e2792b4597709f0facf7cc7b35"), + Sha2TestCase("SQL", "UNICODE_CI_RTRIM", 256, "a7056a455639d1c7deec82ee787db24a0c1878e2792b4597709f0facf7cc7b35") ) @@ -114,9 +126,13 @@ class CollationSQLExpressionsSuite val testCases = Seq( Sha1TestCase("Spark", "UTF8_BINARY", "85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c"), + Sha1TestCase("Spark", "UTF8_BINARY_RTRIM", "85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c"), Sha1TestCase("Spark", "UTF8_LCASE", "85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c"), + Sha1TestCase("Spark", "UTF8_LCASE_RTRIM", "85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c"), Sha1TestCase("SQL", "UNICODE", "2064cb643caa8d9e1de12eea7f3e143ca9f8680d"), - Sha1TestCase("SQL", "UNICODE_CI", "2064cb643caa8d9e1de12eea7f3e143ca9f8680d") + Sha1TestCase("SQL", "UNICODE_RTRIM", "2064cb643caa8d9e1de12eea7f3e143ca9f8680d"), + Sha1TestCase("SQL", "UNICODE_CI", "2064cb643caa8d9e1de12eea7f3e143ca9f8680d"), + Sha1TestCase("SQL", "UNICODE_CI_RTRIM", "2064cb643caa8d9e1de12eea7f3e143ca9f8680d") ) // Supported collations @@ -144,9 +160,13 @@ class CollationSQLExpressionsSuite val testCases = Seq( Crc321TestCase("Spark", "UTF8_BINARY", 1557323817), + Crc321TestCase("Spark", "UTF8_BINARY_RTRIM", 1557323817), Crc321TestCase("Spark", "UTF8_LCASE", 1557323817), + Crc321TestCase("Spark", "UTF8_LCASE_RTRIM", 1557323817), Crc321TestCase("SQL", "UNICODE", 1299261525), - Crc321TestCase("SQL", "UNICODE_CI", 1299261525) + Crc321TestCase("SQL", "UNICODE_RTRIM", 1299261525), + Crc321TestCase("SQL", "UNICODE_CI", 1299261525), + Crc321TestCase("SQL", "UNICODE_CI_RTRIM", 1299261525) ) // Supported collations @@ -172,9 +192,13 @@ class CollationSQLExpressionsSuite val testCases = Seq( Murmur3HashTestCase("Spark", "UTF8_BINARY", 228093765), + Murmur3HashTestCase("Spark ", "UTF8_BINARY_RTRIM", 1779328737), Murmur3HashTestCase("Spark", "UTF8_LCASE", -1928694360), + Murmur3HashTestCase("Spark ", "UTF8_LCASE_RTRIM", -1928694360), Murmur3HashTestCase("SQL", "UNICODE", -1923567940), - Murmur3HashTestCase("SQL", "UNICODE_CI", 1029527950) + Murmur3HashTestCase("SQL ", "UNICODE_RTRIM", -1923567940), + Murmur3HashTestCase("SQL", "UNICODE_CI", 1029527950), + Murmur3HashTestCase("SQL ", "UNICODE_CI_RTRIM", 1029527950) ) // Supported collations @@ -200,9 +224,13 @@ class CollationSQLExpressionsSuite val testCases = Seq( XxHash64TestCase("Spark", "UTF8_BINARY", -4294468057691064905L), + XxHash64TestCase("Spark ", "UTF8_BINARY_RTRIM", 6480371823304753502L), XxHash64TestCase("Spark", "UTF8_LCASE", -3142112654825786434L), + XxHash64TestCase("Spark ", "UTF8_LCASE_RTRIM", -3142112654825786434L), XxHash64TestCase("SQL", "UNICODE", 5964849564945649886L), - XxHash64TestCase("SQL", "UNICODE_CI", 3732497619779520590L) + XxHash64TestCase("SQL ", "UNICODE_RTRIM", 5964849564945649886L), + XxHash64TestCase("SQL", "UNICODE_CI", 3732497619779520590L), + XxHash64TestCase("SQL ", "UNICODE_CI_RTRIM", 3732497619779520590L) ) // Supported collations @@ -2791,16 +2819,24 @@ class CollationSQLExpressionsSuite } } - test("collect_set supports collation") { + test("collect_set does not support collation") { val collation = "UNICODE" val query = s"SELECT collect_set(col) FROM VALUES ('a'), ('b'), ('a') AS tab(col);" withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collation) { - val result = sql(query).collect().head.getSeq[String](0).toSet - val expected = Set("a", "b") - assert(result == expected) - // check result row data type - val dataType = ArrayType(StringType(collation), false) - assert(sql(query).schema.head.dataType == dataType) + checkError( + exception = intercept[AnalysisException] { + sql(query) + }, + condition = "DATATYPE_MISMATCH.UNSUPPORTED_INPUT_TYPE", + sqlState = Some("42K09"), + parameters = Map( + "functionName" -> "`collect_set`", + "dataType" -> "\"MAP\" or \"COLLATED STRING\"", + "sqlExpr" -> "\"collect_set(col)\""), + context = ExpectedContext( + fragment = "collect_set(col)", + start = 7, + stop = 22)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala index ef01f71c68bf9..b6da0b169f050 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala @@ -101,8 +101,12 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("collate function syntax") { assert(sql(s"select collate('aaa', 'utf8_binary')").schema(0).dataType == StringType("UTF8_BINARY")) + assert(sql(s"select collate('aaa', 'utf8_binary_rtrim')").schema(0).dataType == + StringType("UTF8_BINARY_RTRIM")) assert(sql(s"select collate('aaa', 'utf8_lcase')").schema(0).dataType == StringType("UTF8_LCASE")) + assert(sql(s"select collate('aaa', 'utf8_lcase_rtrim')").schema(0).dataType == + StringType("UTF8_LCASE_RTRIM")) } test("collate function syntax with default collation set") { @@ -260,14 +264,23 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { Seq( ("utf8_binary", "aaa", "AAA", false), ("utf8_binary", "aaa", "aaa", true), + ("utf8_binary_rtrim", "aaa", "AAA", false), + ("utf8_binary_rtrim", "aaa", "aaa ", true), ("utf8_lcase", "aaa", "aaa", true), ("utf8_lcase", "aaa", "AAA", true), ("utf8_lcase", "aaa", "bbb", false), + ("utf8_lcase_rtrim", "aaa", "AAA ", true), + ("utf8_lcase_rtrim", "aaa", "bbb", false), ("unicode", "aaa", "aaa", true), ("unicode", "aaa", "AAA", false), + ("unicode_rtrim", "aaa ", "aaa ", true), + ("unicode_rtrim", "aaa", "AAA", false), ("unicode_CI", "aaa", "aaa", true), ("unicode_CI", "aaa", "AAA", true), - ("unicode_CI", "aaa", "bbb", false) + ("unicode_CI", "aaa", "bbb", false), + ("unicode_CI_rtrim", "aaa", "aaa", true), + ("unicode_CI_rtrim", "aaa ", "AAA ", true), + ("unicode_CI_rtrim", "aaa", "bbb", false) ).foreach { case (collationName, left, right, expected) => checkAnswer( @@ -284,15 +297,19 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { ("utf8_binary", "AAA", "aaa", true), ("utf8_binary", "aaa", "aaa", false), ("utf8_binary", "aaa", "BBB", false), + ("utf8_binary_rtrim", "aaa ", "aaa ", false), ("utf8_lcase", "aaa", "aaa", false), ("utf8_lcase", "AAA", "aaa", false), ("utf8_lcase", "aaa", "bbb", true), + ("utf8_lcase_rtrim", "AAA ", "aaa", false), ("unicode", "aaa", "aaa", false), ("unicode", "aaa", "AAA", true), ("unicode", "aaa", "BBB", true), + ("unicode_rtrim", "aaa ", "aaa", false), ("unicode_CI", "aaa", "aaa", false), ("unicode_CI", "aaa", "AAA", false), - ("unicode_CI", "aaa", "bbb", true) + ("unicode_CI", "aaa", "bbb", true), + ("unicode_CI_rtrim", "aaa ", "aaa", false) ).foreach { case (collationName, left, right, expected) => checkAnswer( @@ -355,18 +372,22 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("aggregates count respects collation") { Seq( + ("utf8_binary_rtrim", Seq("aaa", "aaa "), Seq(Row(2, "aaa"))), ("utf8_binary", Seq("AAA", "aaa"), Seq(Row(1, "AAA"), Row(1, "aaa"))), ("utf8_binary", Seq("aaa", "aaa"), Seq(Row(2, "aaa"))), ("utf8_binary", Seq("aaa", "bbb"), Seq(Row(1, "aaa"), Row(1, "bbb"))), ("utf8_lcase", Seq("aaa", "aaa"), Seq(Row(2, "aaa"))), ("utf8_lcase", Seq("AAA", "aaa"), Seq(Row(2, "AAA"))), ("utf8_lcase", Seq("aaa", "bbb"), Seq(Row(1, "aaa"), Row(1, "bbb"))), + ("utf8_lcase_rtrim", Seq("aaa", "AAA "), Seq(Row(2, "aaa"))), ("unicode", Seq("AAA", "aaa"), Seq(Row(1, "AAA"), Row(1, "aaa"))), ("unicode", Seq("aaa", "aaa"), Seq(Row(2, "aaa"))), ("unicode", Seq("aaa", "bbb"), Seq(Row(1, "aaa"), Row(1, "bbb"))), + ("unicode_rtrim", Seq("aaa", "aaa "), Seq(Row(2, "aaa"))), ("unicode_CI", Seq("aaa", "aaa"), Seq(Row(2, "aaa"))), ("unicode_CI", Seq("AAA", "aaa"), Seq(Row(2, "AAA"))), - ("unicode_CI", Seq("aaa", "bbb"), Seq(Row(1, "aaa"), Row(1, "bbb"))) + ("unicode_CI", Seq("aaa", "bbb"), Seq(Row(1, "aaa"), Row(1, "bbb"))), + ("unicode_CI_rtrim", Seq("aaa", "AAA "), Seq(Row(2, "aaa"))) ).foreach { case (collationName: String, input: Seq[String], expected: Seq[Row]) => checkAnswer(sql( @@ -1101,12 +1122,218 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } } + test("Check order by on table with collated string column") { + val tableName = "t" + Seq( + // (collationName, data, expResult) + ( + "", // non-collated + Seq((5, "bbb"), (3, "a"), (1, "A"), (4, "aaaa"), (6, "cc"), (2, "BbB")), + Seq(1, 2, 3, 4, 5, 6) + ), + ( + "UTF8_BINARY", + Seq((5, "bbb"), (3, "a"), (1, "A"), (4, "aaaa"), (6, "cc"), (2, "BbB")), + Seq(1, 2, 3, 4, 5, 6) + ), + ( + "UTF8_LCASE", + Seq((2, "bbb"), (1, "a"), (1, "A"), (1, "aaaa"), (3, "cc"), (2, "BbB")), + Seq(1, 1, 1, 2, 2, 3) + ), + ( + "UNICODE", + Seq((4, "bbb"), (1, "a"), (2, "A"), (3, "aaaa"), (6, "cc"), (5, "BbB")), + Seq(1, 2, 3, 4, 5, 6) + ), + ( + "UNICODE_CI", + Seq((2, "bbb"), (1, "a"), (1, "A"), (1, "aaaa"), (3, "cc"), (2, "BbB")), + Seq(1, 1, 1, 2, 2, 3) + ) + ).foreach { + case (collationName, data, expResult) => + val collationSetup = if (collationName.isEmpty) "" else "collate " + collationName + withTable(tableName) { + sql(s"create table $tableName (c1 integer, c2 string $collationSetup)") + data.foreach { + case (c1, c2) => + sql(s"insert into $tableName values ($c1, '$c2')") + } + checkAnswer(sql(s"select c1 from $tableName order by c2"), expResult.map(Row(_))) + } + } + } + + test("Check order by on StructType") { + Seq( + // (collationName, data, expResult) + ( + "", // non-collated + Seq((5, "b", "A"), (3, "aa", "A"), (6, "b", "B"), (2, "A", "c"), (1, "A", "D"), + (4, "aa", "B")), + Seq(1, 2, 3, 4, 5, 6) + ), + ( + "UTF8_BINARY", + Seq((5, "b", "A"), (3, "aa", "A"), (6, "b", "B"), (2, "A", "c"), (1, "A", "D"), + (4, "aa", "B")), + Seq(1, 2, 3, 4, 5, 6) + ), + ( + "UTF8_LCASE", + Seq((3, "A", "C"), (2, "A", "b"), (2, "a", "b"), (4, "B", "c"), (1, "a", "a"), + (5, "b", "d")), + Seq(1, 2, 2, 3, 4, 5) + ), + ( + "UNICODE", + Seq((4, "A", "C"), (3, "A", "b"), (2, "a", "b"), (5, "b", "c"), (1, "a", "a"), + (6, "b", "d")), + Seq(1, 2, 3, 4, 5, 6) + ), + ( + "UNICODE_CI", + Seq((3, "A", "C"), (2, "A", "b"), (2, "a", "b"), (4, "B", "c"), (1, "a", "a"), + (5, "b", "d")), + Seq(1, 2, 2, 3, 4, 5) + ) + ).foreach { + case (collationName, data, expResult) => + val collationSetup = if (collationName.isEmpty) "" else "collate " + collationName + val tableName = "t" + withTable(tableName) { + sql(s"create table $tableName (c1 integer, c2 struct<" + + s"s1: string $collationSetup," + + s"s2: string $collationSetup>)") + data.foreach { + case (c1, s1, s2) => + sql(s"insert into $tableName values ($c1, struct('$s1', '$s2'))") + } + checkAnswer(sql(s"select c1 from $tableName order by c2"), expResult.map(Row(_))) + } + } + } + + test("Check order by on StructType with few collated fields") { + val data = Seq( + (2, "b", "a", "a", "a", "a"), + (4, "b", "b", "B", "a", "a"), + (1, "a", "a", "a", "a", "a"), + (6, "b", "b", "b", "B", "B"), + (3, "b", "b", "a", "a", "a"), + (5, "b", "b", "b", "B", "a")) + val tableName = "t" + withTable(tableName) { + sql(s"create table $tableName (c1 integer, c2 struct<" + + s"s1: string, " + + s"s2: string collate UTF8_BINARY, " + + s"s3: string collate UTF8_LCASE, " + + s"s4: string collate UNICODE, " + + s"s5: string collate UNICODE_CI>)") + data.foreach { + case (order, s1, s2, s3, s4, s5) => + sql(s"insert into $tableName values ($order, struct('$s1', '$s2', '$s3', '$s4', '$s5'))") + } + val expResult = Seq(1, 2, 3, 4, 5, 6) + checkAnswer(sql(s"select c1 from $tableName order by c2"), expResult.map(Row(_))) + } + } + + test("Check order by on ArrayType with collated strings") { + Seq( + // (collationName, order, data) + ( + "", + Seq((3, Seq("b", "Aa", "c")), (2, Seq("A", "b")), (1, Seq("A")), (2, Seq("A", "b"))), + Seq(1, 2, 2, 3) + ), + ( + "UTF8_BINARY", + Seq((3, Seq("b", "Aa", "c")), (2, Seq("A", "b")), (1, Seq("A")), (2, Seq("A", "b"))), + Seq(1, 2, 2, 3) + ), + ( + "UTF8_LCASE", + Seq((4, Seq("B", "a")), (4, Seq("b", "A")), (2, Seq("aa")), (1, Seq("A")), + (5, Seq("b", "e")), (3, Seq("b"))), + Seq(1, 2, 3, 4, 4, 5) + ), + ( + "UNICODE", + Seq((5, Seq("b", "C")), (4, Seq("b", "AA")), (1, Seq("a")), (4, Seq("b", "AA")), + (3, Seq("b")), (2, Seq("A", "a"))), + Seq(1, 2, 3, 4, 4, 5) + ), + ( + "UNICODE_CI", + Seq((4, Seq("B", "a")), (4, Seq("b", "A")), (2, Seq("aa")), (1, Seq("A")), + (5, Seq("b", "e")), (3, Seq("b"))), + Seq(1, 2, 3, 4, 4, 5) + ) + ).foreach { + case (collationName, dataWithOrder, expResult) => + val collationSetup = if (collationName.isEmpty) "" else "collate " + collationName + val tableName1 = "t1" + val tableName2 = "t2" + withTable(tableName1, tableName2) { + sql(s"create table $tableName1 (c1 integer, c2 array)") + sql(s"create table $tableName2 (c1 integer," + + s" c2 struct>)") + dataWithOrder.foreach { + case (order, data) => + val arrayData = data.map(d => s"'$d'").mkString(", ") + sql(s"insert into $tableName1 values ($order, array($arrayData))") + sql(s"insert into $tableName2 values ($order, struct(array($arrayData)))") + } + checkAnswer(sql(s"select c1 from $tableName1 order by c2"), expResult.map(Row(_))) + checkAnswer(sql(s"select c1 from $tableName2 order by c2"), expResult.map(Row(_))) + } + } + } + + test("Check order by on StructType with different types containing collated strings") { + val data = Seq( + (5, ("b", Seq(("b", "B", "a"), ("a", "a", "a")), "a")), + (2, ("b", Seq(("a", "a", "a")), "a")), + (2, ("b", Seq(("a", "a", "a")), "a")), + (4, ("b", Seq(("b", "a", "a")), "a")), + (3, ("b", Seq(("a", "a", "a"), ("a", "a", "a")), "a")), + (5, ("b", Seq(("b", "B", "a")), "a")), + (4, ("b", Seq(("b", "a", "a")), "a")), + (6, ("b", Seq(("b", "b", "B")), "A")), + (5, ("b", Seq(("b", "b", "a")), "a")), + (1, ("a", Seq(("a", "a", "a")), "a")), + (7, ("b", Seq(("b", "b", "B")), "b")), + (6, ("b", Seq(("b", "b", "B")), "a")), + (5, ("b", Seq(("b", "b", "a")), "a")) + ) + val tableName = "t" + withTable(tableName) { + sql(s"create table $tableName " + + s"(c1 integer," + + s"c2 string," + + s"c3 array>," + + s"c4 string collate UNICODE_CI)") + data.foreach { + case (c1, (c2, c3, c4)) => + val c3String = c3.map { case (f1, f2, f3) => s"struct('$f1', '$f2', '$f3')"} + .mkString(", ") + sql(s"insert into $tableName values ($c1, '$c2', array($c3String), '$c4')") + } + val expResult = Seq(1, 2, 2, 3, 4, 4, 5, 5, 5, 5, 6, 6, 7) + checkAnswer(sql(s"select c1 from $tableName order by c2, c3, c4"), expResult.map(Row(_))) + } + } + for (collation <- Seq("UTF8_BINARY", "UTF8_LCASE", "UNICODE", "UNICODE_CI", "UNICODE_CI_RTRIM", "")) { for (codeGen <- Seq("NO_CODEGEN", "CODEGEN_ONLY")) { val collationSetup = if (collation.isEmpty) "" else " COLLATE " + collation val supportsBinaryEquality = collation.isEmpty || collation == "UNICODE" || - CollationFactory.fetchCollation(collation).supportsBinaryEquality + CollationFactory.fetchCollation(collation).isUtf8BinaryType test(s"Group by on map containing$collationSetup strings ($codeGen)") { val tableName = "t" @@ -1331,7 +1558,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { ) // Only if collation doesn't support binary equality, collation key should be injected. - if (!CollationFactory.fetchCollation(t.collation).supportsBinaryEquality) { + if (!CollationFactory.fetchCollation(t.collation).isUtf8BinaryType) { assert(collectFirst(queryPlan) { case b: HashJoin => b.leftKeys.head }.head.isInstanceOf[CollationKey]) @@ -1388,7 +1615,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { ) // Only if collation doesn't support binary equality, collation key should be injected. - if (!CollationFactory.fetchCollation(t.collation).supportsBinaryEquality) { + if (!CollationFactory.fetchCollation(t.collation).isUtf8BinaryType) { assert(collectFirst(queryPlan) { case b: BroadcastHashJoinExec => b.leftKeys.head }.head.asInstanceOf[ArrayTransform].function.asInstanceOf[LambdaFunction]. @@ -1449,7 +1676,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { ) // Only if collation doesn't support binary equality, collation key should be injected. - if (!CollationFactory.fetchCollation(t.collation).supportsBinaryEquality) { + if (!CollationFactory.fetchCollation(t.collation).isUtf8BinaryType) { assert(collectFirst(queryPlan) { case b: BroadcastHashJoinExec => b.leftKeys.head }.head.asInstanceOf[ArrayTransform].function. @@ -1508,7 +1735,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { ) // Only if collation doesn't support binary equality, collation key should be injected. - if (!CollationFactory.fetchCollation(t.collation).supportsBinaryEquality) { + if (!CollationFactory.fetchCollation(t.collation).isUtf8BinaryType) { assert(queryPlan.toString().contains("collationkey")) } else { assert(!queryPlan.toString().contains("collationkey")) @@ -1567,7 +1794,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { ) // Only if collation doesn't support binary equality, collation key should be injected. - if (!CollationFactory.fetchCollation(t.collation).supportsBinaryEquality) { + if (!CollationFactory.fetchCollation(t.collation).isUtf8BinaryType) { assert(queryPlan.toString().contains("collationkey")) } else { assert(!queryPlan.toString().contains("collationkey")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index e80c3b23a7db3..25f4d9f62354a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -648,7 +648,7 @@ class DataFrameAggregateSuite extends QueryTest condition = "DATATYPE_MISMATCH.UNSUPPORTED_INPUT_TYPE", parameters = Map( "functionName" -> "`collect_set`", - "dataType" -> "\"MAP\"", + "dataType" -> "\"MAP\" or \"COLLATED STRING\"", "sqlExpr" -> "\"collect_set(b)\"" ), context = ExpectedContext( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala index 8c1cc6c3bea1d..48ea0e01a4372 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala @@ -39,6 +39,15 @@ import org.apache.spark.unsafe.types.CalendarInterval class DataFrameComplexTypeSuite extends QueryTest with SharedSparkSession { import testImplicits._ + test("ArrayTransform with scan input") { + withTempPath { f => + spark.sql("select array(array(1, null, 3), array(4, 5, null), array(null, 8, 9)) as a") + .write.parquet(f.getAbsolutePath) + val df = spark.read.parquet(f.getAbsolutePath).selectExpr("transform(a, (x, i) -> x)") + checkAnswer(df, Row(Seq(Seq(1, null, 3), Seq(4, 5, null), Seq(null, 8, 9)))) + } + } + test("UDF on struct") { val f = udf((a: String) => a) val df = sparkContext.parallelize(Seq((1, 1))).toDF("a", "b") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 47691e1ccd40f..39c839ae5a518 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -478,7 +478,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { intercept[AnalysisException](df.select(expr)), condition = "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", parameters = Map( - "inputName" -> "length", + "inputName" -> "`length`", "inputType" -> "INT or SMALLINT", "inputExpr" -> "\"a\"", "sqlExpr" -> "\"randstr(a, 10)\""), @@ -530,7 +530,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { intercept[AnalysisException](df.select(expr)), condition = "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT", parameters = Map( - "inputName" -> "min", + "inputName" -> "`min`", "inputType" -> "integer or floating-point", "inputExpr" -> "\"a\"", "sqlExpr" -> "\"uniform(a, 10)\""), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 85f296665b6e0..45c34d9c73367 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -1273,7 +1273,7 @@ class DatasetSuite extends QueryTest // Just check the error class here to avoid flakiness due to different parameters. assert(intercept[SparkRuntimeException] { buildDataset(Row(Row("hello", null))).collect() - }.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + }.getCondition == "NOT_NULL_ASSERT_VIOLATION") } test("SPARK-12478: top level null field") { @@ -1416,7 +1416,7 @@ class DatasetSuite extends QueryTest val ex = intercept[SparkRuntimeException] { spark.createDataFrame(rdd, schema).collect() } - assert(ex.getErrorClass == "EXPRESSION_ENCODING_FAILED") + assert(ex.getCondition == "EXPRESSION_ENCODING_FAILED") assert(ex.getCause.getMessage.contains("The 1th field 'b' of input row cannot be null")) } @@ -1612,7 +1612,7 @@ class DatasetSuite extends QueryTest test("Dataset should throw RuntimeException if top-level product input object is null") { val e = intercept[SparkRuntimeException](Seq(ClassData("a", 1), null).toDS()) - assert(e.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(e.getCondition == "NOT_NULL_ASSERT_VIOLATION") } test("dropDuplicates") { @@ -2121,7 +2121,7 @@ class DatasetSuite extends QueryTest test("SPARK-23835: null primitive data type should throw NullPointerException") { val ds = Seq[(Option[Int], Option[Int])]((Some(1), None)).toDS() val exception = intercept[SparkRuntimeException](ds.as[(Int, Int)].collect()) - assert(exception.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(exception.getCondition == "NOT_NULL_ASSERT_VIOLATION") } test("SPARK-24569: Option of primitive types are mistakenly mapped to struct type") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index e44bd5de4f4c4..9c529d1422119 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -128,13 +128,20 @@ class FileBasedDataSourceSuite extends QueryTest allFileBasedDataSources.foreach { format => test(s"SPARK-23372 error while writing empty schema files using $format") { + val formatMapping = Map( + "csv" -> "CSV", + "json" -> "JSON", + "parquet" -> "Parquet", + "orc" -> "ORC", + "text" -> "Text" + ) withTempPath { outputPath => checkError( exception = intercept[AnalysisException] { spark.emptyDataFrame.write.format(format).save(outputPath.toString) }, - condition = "_LEGACY_ERROR_TEMP_1142", - parameters = Map.empty + condition = "EMPTY_SCHEMA_NOT_SUPPORTED_FOR_DATASOURCE", + parameters = Map("format" -> formatMapping(format)) ) } @@ -150,8 +157,8 @@ class FileBasedDataSourceSuite extends QueryTest exception = intercept[AnalysisException] { df.write.format(format).save(outputPath.toString) }, - condition = "_LEGACY_ERROR_TEMP_1142", - parameters = Map.empty + condition = "EMPTY_SCHEMA_NOT_SUPPORTED_FOR_DATASOURCE", + parameters = Map("format" -> formatMapping(format)) ) } } @@ -506,14 +513,23 @@ class FileBasedDataSourceSuite extends QueryTest withSQLConf( SQLConf.USE_V1_SOURCE_LIST.key -> useV1List, SQLConf.LEGACY_INTERVAL_ENABLED.key -> "true") { + val formatMapping = Map( + "csv" -> "CSV", + "json" -> "JSON", + "parquet" -> "Parquet", + "orc" -> "ORC" + ) // write path Seq("csv", "json", "parquet", "orc").foreach { format => checkError( exception = intercept[AnalysisException] { sql("select interval 1 days").write.format(format).mode("overwrite").save(tempDir) }, - condition = "_LEGACY_ERROR_TEMP_1136", - parameters = Map.empty + condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", + parameters = Map( + "format" -> formatMapping(format), + "columnName" -> "`INTERVAL '1 days'`", + "columnType" -> "\"INTERVAL\"") ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala index a892cd4db02b0..3f921618297d0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala @@ -205,7 +205,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { withLCAOn { checkAnswer(sql(query), expectedAnswerLCAOn) } withLCAOff { assert(intercept[AnalysisException]{ sql(query) } - .getErrorClass == "UNRESOLVED_COLUMN.WITH_SUGGESTION") + .getCondition == "UNRESOLVED_COLUMN.WITH_SUGGESTION") } } @@ -216,8 +216,8 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { errorParams: Map[String, String]): Unit = { val e1 = intercept[AnalysisException] { sql(q1) } val e2 = intercept[AnalysisException] { sql(q2) } - assert(e1.getErrorClass == condition) - assert(e2.getErrorClass == condition) + assert(e1.getCondition == condition) + assert(e2.getCondition == condition) errorParams.foreach { case (k, v) => assert(e1.messageParameters.get(k).exists(_ == v)) assert(e2.messageParameters.get(k).exists(_ == v)) @@ -1187,7 +1187,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { "sum_avg * 1.0 as sum_avg1, sum_avg1 + dept " + s"from $testTable group by dept, properties.joinYear $havingSuffix" ).foreach { query => - assert(intercept[AnalysisException](sql(query)).getErrorClass == + assert(intercept[AnalysisException](sql(query)).getCondition == "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_AGGREGATE_WITH_WINDOW_AND_HAVING") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RuntimeNullChecksV2Writes.scala b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeNullChecksV2Writes.scala index 754c46cc5cd3e..b48ff7121c767 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RuntimeNullChecksV2Writes.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeNullChecksV2Writes.scala @@ -64,7 +64,7 @@ class RuntimeNullChecksV2Writes extends QueryTest with SQLTestUtils with SharedS sql("INSERT INTO t VALUES ('txt', null)") } } - assert(e.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(e.getCondition == "NOT_NULL_ASSERT_VIOLATION") } } @@ -404,7 +404,7 @@ class RuntimeNullChecksV2Writes extends QueryTest with SQLTestUtils with SharedS private def assertNotNullException(e: SparkRuntimeException, colPath: Seq[String]): Unit = { e.getCause match { - case _ if e.getErrorClass == "NOT_NULL_ASSERT_VIOLATION" => + case _ if e.getCondition == "NOT_NULL_ASSERT_VIOLATION" => case other => fail(s"Unexpected exception cause: $other") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala index 38e004e0b7209..4bd20bc245613 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala @@ -148,7 +148,7 @@ trait SQLQueryTestHelper extends Logging { try { result } catch { - case e: SparkThrowable with Throwable if e.getErrorClass != null => + case e: SparkThrowable with Throwable if e.getCondition != null => (emptySchema, Seq(e.getClass.getName, getMessage(e, format))) case a: AnalysisException => // Do not output the logical plan tree which contains expression IDs. @@ -160,7 +160,7 @@ trait SQLQueryTestHelper extends Logging { // information of stage, task ID, etc. // To make result matching simpler, here we match the cause of the exception if it exists. s.getCause match { - case e: SparkThrowable with Throwable if e.getErrorClass != null => + case e: SparkThrowable with Throwable if e.getCondition != null => (emptySchema, Seq(e.getClass.getName, getMessage(e, format))) case cause => (emptySchema, Seq(cause.getClass.getName, cause.getMessage)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 16118526f2fe4..76919d6583106 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -163,9 +163,9 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSparkSession exception = intercept[SparkUnsupportedOperationException] { Seq(InvalidInJava(1)).toDS() }, - condition = "_LEGACY_ERROR_TEMP_2140", + condition = "INVALID_JAVA_IDENTIFIER_AS_FIELD_NAME", parameters = Map( - "fieldName" -> "abstract", + "fieldName" -> "`abstract`", "walkedTypePath" -> "- root class: \"org.apache.spark.sql.InvalidInJava\"")) } @@ -174,9 +174,9 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSparkSession exception = intercept[SparkUnsupportedOperationException] { Seq(InvalidInJava2(1)).toDS() }, - condition = "_LEGACY_ERROR_TEMP_2140", + condition = "INVALID_JAVA_IDENTIFIER_AS_FIELD_NAME", parameters = Map( - "fieldName" -> "0", + "fieldName" -> "`0`", "walkedTypePath" -> "- root class: \"org.apache.spark.sql.ScalaReflectionRelationSuite.InvalidInJava2\"")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index f17cf25565145..f8f7fd246832f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -925,12 +925,12 @@ class SubquerySuite extends QueryTest withSQLConf(SQLConf.DECORRELATE_INNER_QUERY_ENABLED.key -> "false") { val error = intercept[AnalysisException] { sql(query) } - assert(error.getErrorClass == "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + + assert(error.getCondition == "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED") } withSQLConf(SQLConf.DECORRELATE_SET_OPS_ENABLED.key -> "false") { val error = intercept[AnalysisException] { sql(query) } - assert(error.getErrorClass == "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + + assert(error.getCondition == "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED") } @@ -1004,12 +1004,12 @@ class SubquerySuite extends QueryTest withSQLConf(SQLConf.DECORRELATE_INNER_QUERY_ENABLED.key -> "false") { val error = intercept[AnalysisException] { sql(query) } - assert(error.getErrorClass == "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + + assert(error.getCondition == "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED") } withSQLConf(SQLConf.DECORRELATE_SET_OPS_ENABLED.key -> "false") { val error = intercept[AnalysisException] { sql(query) } - assert(error.getErrorClass == "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + + assert(error.getCondition == "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 2e072e5afc926..d550d0f94f236 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -821,14 +821,14 @@ class UDFSuite extends QueryTest with SharedSparkSession { val e1 = intercept[SparkException] { Seq("20").toDF("col").select(udf(f1).apply(Column("col"))).collect() } - assert(e1.getErrorClass == "FAILED_EXECUTE_UDF") + assert(e1.getCondition == "FAILED_EXECUTE_UDF") assert(e1.getCause.getStackTrace.head.toString.contains( "UDFSuite$MalformedClassObject$MalformedNonPrimitiveFunction")) val e2 = intercept[SparkException] { Seq(20).toDF("col").select(udf(f2).apply(Column("col"))).collect() } - assert(e2.getErrorClass == "FAILED_EXECUTE_UDF") + assert(e2.getCondition == "FAILED_EXECUTE_UDF") assert(e2.getCause.getStackTrace.head.toString.contains( "UDFSuite$MalformedClassObject$MalformedPrimitiveFunction")) } @@ -938,7 +938,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { val e = intercept[SparkException] { input.select(overflowFunc($"dateTime")).collect() } - assert(e.getErrorClass == "FAILED_EXECUTE_UDF") + assert(e.getCondition == "FAILED_EXECUTE_UDF") assert(e.getCause.isInstanceOf[java.lang.ArithmeticException]) } @@ -1053,7 +1053,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { val e = intercept[SparkException] { input.select(overflowFunc($"d")).collect() } - assert(e.getErrorClass == "FAILED_EXECUTE_UDF") + assert(e.getCondition == "FAILED_EXECUTE_UDF") assert(e.getCause.isInstanceOf[java.lang.ArithmeticException]) } @@ -1101,7 +1101,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { val e = intercept[SparkException] { input.select(overflowFunc($"p")).collect() } - assert(e.getErrorClass == "FAILED_EXECUTE_UDF") + assert(e.getCondition == "FAILED_EXECUTE_UDF") assert(e.getCause.isInstanceOf[java.lang.ArithmeticException]) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/VariantEndToEndSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/VariantEndToEndSuite.scala index 19d4ac23709b6..fe5c6ef004920 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/VariantEndToEndSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantEndToEndSuite.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.sql -import org.apache.spark.SparkThrowable +import org.apache.spark.{SparkException, SparkRuntimeException} import org.apache.spark.sql.QueryTest.sameRows import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} @@ -359,16 +359,24 @@ class VariantEndToEndSuite extends QueryTest with SharedSparkSession { val expectedMetadata: Array[Byte] = Array(VERSION, 3, 0, 1, 2, 3, 'a', 'b', 'c') assert(actual === new VariantVal(expectedValue, expectedMetadata)) } - withSQLConf(SQLConf.VARIANT_ALLOW_DUPLICATE_KEYS.key -> "false") { - val df = Seq(json).toDF("j") - .selectExpr("from_json(j,'variant')") - checkError( - exception = intercept[SparkThrowable] { + // Check whether the parse_json and from_json expressions throw the correct exception. + Seq("from_json(j, 'variant')", "parse_json(j)").foreach { expr => + withSQLConf(SQLConf.VARIANT_ALLOW_DUPLICATE_KEYS.key -> "false") { + val df = Seq(json).toDF("j").selectExpr(expr) + val exception = intercept[SparkException] { df.collect() - }, - condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", - parameters = Map("badRecord" -> json, "failFastMode" -> "FAILFAST") - ) + } + checkError( + exception = exception, + condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION", + parameters = Map("badRecord" -> json, "failFastMode" -> "FAILFAST") + ) + checkError( + exception = exception.getCause.asInstanceOf[SparkRuntimeException], + condition = "VARIANT_DUPLICATE_KEY", + parameters = Map("key" -> "a") + ) + } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala index d6599debd3b11..6b0fd6084099c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala @@ -414,8 +414,8 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { new JavaStrLen(new JavaStrLenNoImpl)) checkError( exception = intercept[AnalysisException](sql("SELECT testcat.ns.strlen('abc')").collect()), - condition = "_LEGACY_ERROR_TEMP_3055", - parameters = Map("scalarFunc" -> "strlen"), + condition = "SCALAR_FUNCTION_NOT_FULLY_IMPLEMENTED", + parameters = Map("scalarFunc" -> "`strlen`"), context = ExpectedContext( fragment = "testcat.ns.strlen('abc')", start = 7, @@ -448,8 +448,8 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { addFunction(Identifier.of(Array("ns"), "add"), new JavaLongAdd(new JavaLongAddMismatchMagic)) checkError( exception = intercept[AnalysisException](sql("SELECT testcat.ns.add(1L, 2L)").collect()), - condition = "_LEGACY_ERROR_TEMP_3055", - parameters = Map("scalarFunc" -> "long_add_mismatch_magic"), + condition = "SCALAR_FUNCTION_NOT_FULLY_IMPLEMENTED", + parameters = Map("scalarFunc" -> "`long_add_mismatch_magic`"), context = ExpectedContext( fragment = "testcat.ns.add(1L, 2L)", start = 7, @@ -458,6 +458,23 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { ) } + test("SPARK-49549: scalar function w/ mismatch a compatible ScalarFunction#produceResult") { + case object CharLength extends ScalarFunction[Int] { + override def inputTypes(): Array[DataType] = Array(StringType) + override def resultType(): DataType = IntegerType + override def name(): String = "CHAR_LENGTH" + } + + catalog("testcat").asInstanceOf[SupportsNamespaces].createNamespace(Array("ns"), emptyProps) + addFunction(Identifier.of(Array("ns"), "my_strlen"), StrLen(CharLength)) + checkError( + exception = intercept[SparkUnsupportedOperationException] + (sql("SELECT testcat.ns.my_strlen('abc')").collect()), + condition = "SCALAR_FUNCTION_NOT_COMPATIBLE", + parameters = Map("scalarFunc" -> "`CHAR_LENGTH`") + ) + } + test("SPARK-35390: scalar function w/ type coercion") { catalog("testcat").asInstanceOf[SupportsNamespaces].createNamespace(Array("ns"), emptyProps) addFunction(Identifier.of(Array("ns"), "add"), new JavaLongAdd(new JavaLongAddDefault(false))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 6b58d23e92603..52ae1bf5d9d3b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -840,7 +840,7 @@ class DataSourceV2SQLSuiteV1Filter val exception = intercept[SparkRuntimeException] { insertNullValueAndCheck() } - assert(exception.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(exception.getCondition == "NOT_NULL_ASSERT_VIOLATION") } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala index 9d4e4fc016722..053616c88d638 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala @@ -1326,7 +1326,7 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase { | UPDATE SET s = named_struct('n_i', null, 'n_l', -1L) |""".stripMargin) } - assert(e1.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(e1.getCondition == "NOT_NULL_ASSERT_VIOLATION") val e2 = intercept[SparkRuntimeException] { sql( @@ -1337,7 +1337,7 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase { | UPDATE SET s = named_struct('n_i', null, 'n_l', -1L) |""".stripMargin) } - assert(e2.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(e2.getCondition == "NOT_NULL_ASSERT_VIOLATION") val e3 = intercept[SparkRuntimeException] { sql( @@ -1348,7 +1348,7 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase { | INSERT (pk, s, dep) VALUES (s.pk, named_struct('n_i', null, 'n_l', -1L), 'invalid') |""".stripMargin) } - assert(e3.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(e3.getCondition == "NOT_NULL_ASSERT_VIOLATION") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala index b4fdf50447458..92c175fe2f94a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala @@ -1003,7 +1003,7 @@ class QueryCompilationErrorsSuite val exception = intercept[AnalysisException] { sql(queryWithTrailingComma) } - assert(exception.getErrorClass === "TRAILING_COMMA_IN_SELECT") + assert(exception.getCondition === "TRAILING_COMMA_IN_SELECT") } val unresolvedColumnErrors = Seq( @@ -1017,7 +1017,7 @@ class QueryCompilationErrorsSuite val exception = intercept[AnalysisException] { sql(query) } - assert(exception.getErrorClass === "UNRESOLVED_COLUMN.WITH_SUGGESTION") + assert(exception.getCondition === "UNRESOLVED_COLUMN.WITH_SUGGESTION") } // sanity checks diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index 9d1448d0ac09d..1adb1fdf05032 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -35,11 +35,12 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, Encoder, Kry import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{NamedParameter, UnresolvedGenerator} import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Concat, CreateArray, EmptyRow, Expression, Flatten, Grouping, Literal, RowNumber, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Concat, CreateArray, EmptyRow, Expression, Flatten, Grouping, Literal, RowNumber, UnaryExpression, Years} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.expressions.objects.InitializeJavaBean import org.apache.spark.sql.catalyst.rules.RuleIdCollection +import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLExpr import org.apache.spark.sql.execution.datasources.jdbc.{DriverRegistry, JDBCOptions} import org.apache.spark.sql.execution.datasources.jdbc.connection.ConnectionProvider import org.apache.spark.sql.execution.datasources.orc.OrcTest @@ -292,7 +293,7 @@ class QueryExecutionErrorsSuite val e = intercept[SparkException] { df.write.parquet(dir.getCanonicalPath) } - assert(e.getErrorClass == "TASK_WRITE_FAILED") + assert(e.getCondition == "TASK_WRITE_FAILED") val format = "Parquet" val config = "\"" + SQLConf.PARQUET_REBASE_MODE_IN_WRITE.key + "\"" @@ -311,7 +312,7 @@ class QueryExecutionErrorsSuite val ex = intercept[SparkException] { spark.read.schema("time timestamp_ntz").orc(file.getCanonicalPath).collect() } - assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(ex.getCondition.startsWith("FAILED_READ_FILE")) checkError( exception = ex.getCause.asInstanceOf[SparkUnsupportedOperationException], condition = "UNSUPPORTED_FEATURE.ORC_TYPE_CAST", @@ -333,7 +334,7 @@ class QueryExecutionErrorsSuite val ex = intercept[SparkException] { spark.read.schema("time timestamp_ltz").orc(file.getCanonicalPath).collect() } - assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(ex.getCondition.startsWith("FAILED_READ_FILE")) checkError( exception = ex.getCause.asInstanceOf[SparkUnsupportedOperationException], condition = "UNSUPPORTED_FEATURE.ORC_TYPE_CAST", @@ -381,7 +382,7 @@ class QueryExecutionErrorsSuite } val e2 = e1.getCause.asInstanceOf[SparkException] - assert(e2.getErrorClass == "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION") + assert(e2.getCondition == "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION") checkError( exception = e2.getCause.asInstanceOf[SparkRuntimeException], @@ -920,7 +921,7 @@ class QueryExecutionErrorsSuite val e = intercept[StreamingQueryException] { query.awaitTermination() } - assert(e.getErrorClass === "STREAM_FAILED") + assert(e.getCondition === "STREAM_FAILED") assert(e.getCause.isInstanceOf[NullPointerException]) } @@ -1006,6 +1007,17 @@ class QueryExecutionErrorsSuite sqlState = "XX000") } + test("PartitionTransformExpression error on eval") { + val expr = Years(Literal("foo")) + val e = intercept[SparkException] { + expr.eval() + } + checkError( + exception = e, + condition = "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY", + parameters = Map("expression" -> toSQLExpr(expr))) + } + test("INTERNAL_ERROR: Calling doGenCode on unresolved") { val e = intercept[SparkException] { val ctx = new CodegenContext diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 75f016d050de9..c5e64c96b2c8a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -904,7 +904,7 @@ class AdaptiveQueryExecSuite val error = intercept[SparkException] { aggregated.count() } - assert(error.getErrorClass === "INVALID_BUCKET_FILE") + assert(error.getCondition === "INVALID_BUCKET_FILE") assert(error.getMessage contains "Invalid bucket file") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala index deb62eb3ac234..387a2baa256bf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala @@ -368,7 +368,7 @@ class BinaryFileFormatSuite extends QueryTest with SharedSparkSession { checkAnswer(readContent(), expected) } } - assert(caught.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(caught.getCondition.startsWith("FAILED_READ_FILE")) assert(caught.getCause.getMessage.contains("exceeds the max length allowed")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVParsingOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVParsingOptionsSuite.scala new file mode 100644 index 0000000000000..8c8304503cef8 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVParsingOptionsSuite.scala @@ -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.execution.datasources.csv + +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.test.SharedSparkSession + +class CSVParsingOptionsSuite extends QueryTest with SharedSparkSession { + import testImplicits._ + + test("SPARK-49955: null string value does not mean corrupted file") { + val str = "abc" + val stringDataset = Seq(str, null).toDS() + val df = spark.read.csv(stringDataset) + // `spark.read.csv(rdd)` removes all null values at the beginning. + checkAnswer(df, Seq(Row("abc"))) + val df2 = spark.read.option("mode", "failfast").csv(stringDataset) + checkAnswer(df2, Seq(Row("abc"))) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 023f401516dc3..422ae02a18322 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -392,7 +392,7 @@ abstract class CSVSuite condition = "FAILED_READ_FILE.NO_HINT", parameters = Map("path" -> s".*$carsFile.*")) val e2 = e1.getCause.asInstanceOf[SparkException] - assert(e2.getErrorClass == "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION") + assert(e2.getCondition == "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION") checkError( exception = e2.getCause.asInstanceOf[SparkRuntimeException], condition = "MALFORMED_CSV_RECORD", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala index 703085dca66f1..11cc0b99bbde7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.datasources.json +import org.apache.spark.SparkException import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{StringType, StructType} @@ -185,4 +186,14 @@ class JsonParsingOptionsSuite extends QueryTest with SharedSparkSession { assert(df.first().getString(0) == "Cazen Lee") assert(df.first().getString(1) == "$10") } + + test("SPARK-49955: null string value does not mean corrupted file") { + val str = "{\"name\": \"someone\"}" + val stringDataset = Seq(str, null).toDS() + val df = spark.read.json(stringDataset) + checkAnswer(df, Seq(Row(null, "someone"), Row(null, null))) + + val e = intercept[SparkException](spark.read.option("mode", "failfast").json(stringDataset)) + assert(e.getCause.isInstanceOf[NullPointerException]) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index f13d66b76838f..500c0647bcb2a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -708,7 +708,7 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { val ex = intercept[SparkException] { sql(s"select A from $tableName where A < 0").collect() } - assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(ex.getCondition.startsWith("FAILED_READ_FILE")) assert(ex.getCause.isInstanceOf[SparkRuntimeException]) assert(ex.getCause.getMessage.contains( """Found duplicate field(s) "A": [A, a] in case-insensitive mode""")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala index 2e6413d998d12..ab0d4d9bc53b8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala @@ -604,14 +604,14 @@ abstract class OrcQueryTest extends OrcTest { val e1 = intercept[SparkException] { testIgnoreCorruptFiles() } - assert(e1.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(e1.getCondition.startsWith("FAILED_READ_FILE")) assert(e1.getCause.getMessage.contains("Malformed ORC file") || // Hive ORC table scan uses a different code path and has one more error stack e1.getCause.getCause.getMessage.contains("Malformed ORC file")) val e2 = intercept[SparkException] { testIgnoreCorruptFilesWithoutSchemaInfer() } - assert(e2.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(e2.getCondition.startsWith("FAILED_READ_FILE")) assert(e2.getCause.getMessage.contains("Malformed ORC file") || // Hive ORC table scan uses a different code path and has one more error stack e2.getCause.getCause.getMessage.contains("Malformed ORC file")) @@ -625,7 +625,7 @@ abstract class OrcQueryTest extends OrcTest { val e4 = intercept[SparkException] { testAllCorruptFilesWithoutSchemaInfer() } - assert(e4.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(e4.getCondition.startsWith("FAILED_READ_FILE")) assert(e4.getCause.getMessage.contains("Malformed ORC file") || // Hive ORC table scan uses a different code path and has one more error stack e4.getCause.getCause.getMessage.contains("Malformed ORC file")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala index 9348d10711b35..040999476ece1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala @@ -450,8 +450,8 @@ abstract class OrcSuite val ex = intercept[SparkException] { spark.read.orc(basePath).columns.length } - assert(ex.getErrorClass == "CANNOT_MERGE_SCHEMAS") - assert(ex.getCause.asInstanceOf[SparkException].getErrorClass === + assert(ex.getCondition == "CANNOT_MERGE_SCHEMAS") + assert(ex.getCause.asInstanceOf[SparkException].getCondition === "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 5c382b1858716..903dda7f41c0d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -1958,7 +1958,7 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared val ex = intercept[SparkException] { sql(s"select a from $tableName where b > 0").collect() } - assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(ex.getCondition.startsWith("FAILED_READ_FILE")) assert(ex.getCause.isInstanceOf[SparkRuntimeException]) assert(ex.getCause.getMessage.contains( """Found duplicate field(s) "B": [B, b] in case-insensitive mode""")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index 0afa545595c77..95fb178154929 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -1223,7 +1223,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession val m1 = intercept[SparkException] { spark.range(1).coalesce(1).write.options(extraOptions).parquet(dir.getCanonicalPath) } - assert(m1.getErrorClass == "TASK_WRITE_FAILED") + assert(m1.getCondition == "TASK_WRITE_FAILED") assert(m1.getCause.getMessage.contains("Intentional exception for testing purposes")) } @@ -1233,8 +1233,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession .coalesce(1) df.write.partitionBy("a").options(extraOptions).parquet(dir.getCanonicalPath) } - if (m2.getErrorClass != null) { - assert(m2.getErrorClass == "TASK_WRITE_FAILED") + if (m2.getCondition != null) { + assert(m2.getCondition == "TASK_WRITE_FAILED") assert(m2.getCause.getMessage.contains("Intentional exception for testing purposes")) } else { assert(m2.getMessage.contains("TASK_WRITE_FAILED")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index eb4618834504c..87a2843f34de1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -1181,7 +1181,7 @@ abstract class ParquetPartitionDiscoverySuite spark.read.parquet(dir.toString) } val msg = exception.getMessage - assert(exception.getErrorClass === "CONFLICTING_PARTITION_COLUMN_NAMES") + assert(exception.getCondition === "CONFLICTING_PARTITION_COLUMN_NAMES") // Partitions inside the error message can be presented in any order assert("Partition column name list #[0-1]: col1".r.findFirstIn(msg).isDefined) assert("Partition column name list #[0-1]: col1, col2".r.findFirstIn(msg).isDefined) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 4d413efe50430..22a02447e720f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -1075,7 +1075,7 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS val e = intercept[SparkException] { readParquet("d DECIMAL(3, 2)", path).collect() } - assert(e.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(e.getCondition.startsWith("FAILED_READ_FILE")) assert(e.getCause.getMessage.contains("Please read this column/field as Spark BINARY type")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala index 6d9092391a98e..30503af0fab6f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala @@ -414,7 +414,7 @@ abstract class ParquetRebaseDatetimeSuite val e = intercept[SparkException] { df.write.parquet(dir.getCanonicalPath) } - assert(e.getErrorClass == "TASK_WRITE_FAILED") + assert(e.getCondition == "TASK_WRITE_FAILED") val errMsg = e.getCause.asInstanceOf[SparkUpgradeException].getMessage assert(errMsg.contains("You may get a different result due to the upgrading")) } @@ -431,7 +431,7 @@ abstract class ParquetRebaseDatetimeSuite val e = intercept[SparkException] { df.write.parquet(dir.getCanonicalPath) } - assert(e.getErrorClass == "TASK_WRITE_FAILED") + assert(e.getCondition == "TASK_WRITE_FAILED") val errMsg = e.getCause.asInstanceOf[SparkUpgradeException].getMessage assert(errMsg.contains("You may get a different result due to the upgrading")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala index 95378d9467478..08fd8a9ecb53e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala @@ -319,7 +319,7 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { .load(path.getAbsolutePath) val exception = intercept[SparkException](dfRead.collect()) - assert(exception.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(exception.getCondition.startsWith("FAILED_READ_FILE")) assert(exception.getCause.getMessage.contains( ParquetFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala index c88f51a6b7d06..8091d6e64fdc1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala @@ -1173,7 +1173,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { exception = intercept[SparkUnsupportedOperationException] { catalog.alterNamespace(testNs, NamespaceChange.removeProperty(p)) }, - condition = "_LEGACY_ERROR_TEMP_2069", + condition = "CANNOT_REMOVE_RESERVED_PROPERTY", parameters = Map("property" -> p)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceChangeDataReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceChangeDataReadSuite.scala index 4833b8630134c..59c0af8afd198 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceChangeDataReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceChangeDataReadSuite.scala @@ -90,7 +90,7 @@ abstract class StateDataSourceChangeDataReaderSuite extends StateDataSourceTestB .option(StateSourceOptions.CHANGE_END_BATCH_ID, 2) .load(tempDir.getAbsolutePath) } - assert(exc.getErrorClass === "STDS_INVALID_OPTION_VALUE.WITH_MESSAGE") + assert(exc.getCondition === "STDS_INVALID_OPTION_VALUE.WITH_MESSAGE") } } @@ -103,7 +103,7 @@ abstract class StateDataSourceChangeDataReaderSuite extends StateDataSourceTestB .option(StateSourceOptions.CHANGE_END_BATCH_ID, 0) .load(tempDir.getAbsolutePath) } - assert(exc.getErrorClass === "STDS_INVALID_OPTION_VALUE.IS_NEGATIVE") + assert(exc.getCondition === "STDS_INVALID_OPTION_VALUE.IS_NEGATIVE") } } @@ -116,7 +116,7 @@ abstract class StateDataSourceChangeDataReaderSuite extends StateDataSourceTestB .option(StateSourceOptions.CHANGE_END_BATCH_ID, 0) .load(tempDir.getAbsolutePath) } - assert(exc.getErrorClass === "STDS_INVALID_OPTION_VALUE.WITH_MESSAGE") + assert(exc.getCondition === "STDS_INVALID_OPTION_VALUE.WITH_MESSAGE") } } @@ -130,7 +130,7 @@ abstract class StateDataSourceChangeDataReaderSuite extends StateDataSourceTestB .option(StateSourceOptions.CHANGE_END_BATCH_ID, 0) .load(tempDir.getAbsolutePath) } - assert(exc.getErrorClass === "STDS_CONFLICT_OPTIONS") + assert(exc.getCondition === "STDS_CONFLICT_OPTIONS") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala index 5f55848d540df..300da03f73e1f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala @@ -1137,7 +1137,7 @@ abstract class StateDataSourceReadSuite extends StateDataSourceTestBase with Ass val exc = intercept[StateStoreSnapshotPartitionNotFound] { stateDfError.show() } - assert(exc.getErrorClass === "CANNOT_LOAD_STATE_STORE.SNAPSHOT_PARTITION_ID_NOT_FOUND") + assert(exc.getCondition === "CANNOT_LOAD_STATE_STORE.SNAPSHOT_PARTITION_ID_NOT_FOUND") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala index dcebece29037f..1f2be12058eb7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala @@ -330,7 +330,7 @@ class PythonDataSourceSuite extends PythonDataSourceSuiteBase { val err = intercept[AnalysisException] { spark.read.format(dataSourceName).schema(schema).load().collect() } - assert(err.getErrorClass == "PYTHON_DATA_SOURCE_ERROR") + assert(err.getCondition == "PYTHON_DATA_SOURCE_ERROR") assert(err.getMessage.contains("PySparkNotImplementedError")) } @@ -350,7 +350,7 @@ class PythonDataSourceSuite extends PythonDataSourceSuiteBase { val err = intercept[AnalysisException] { spark.read.format(dataSourceName).schema(schema).load().collect() } - assert(err.getErrorClass == "PYTHON_DATA_SOURCE_ERROR") + assert(err.getCondition == "PYTHON_DATA_SOURCE_ERROR") assert(err.getMessage.contains("error creating reader")) } @@ -369,7 +369,7 @@ class PythonDataSourceSuite extends PythonDataSourceSuiteBase { val err = intercept[AnalysisException] { spark.read.format(dataSourceName).schema(schema).load().collect() } - assert(err.getErrorClass == "PYTHON_DATA_SOURCE_ERROR") + assert(err.getCondition == "PYTHON_DATA_SOURCE_ERROR") assert(err.getMessage.contains("DATA_SOURCE_TYPE_MISMATCH")) assert(err.getMessage.contains("PySparkAssertionError")) } @@ -480,7 +480,7 @@ class PythonDataSourceSuite extends PythonDataSourceSuiteBase { spark.dataSource.registerPython(dataSourceName, dataSource) val err = intercept[AnalysisException]( spark.read.format(dataSourceName).load().collect()) - assert(err.getErrorClass == "PYTHON_DATA_SOURCE_ERROR") + assert(err.getCondition == "PYTHON_DATA_SOURCE_ERROR") assert(err.getMessage.contains("partitions")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala index 8d0e1c5f578fa..3d91a045907fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala @@ -574,7 +574,7 @@ class PythonStreamingDataSourceSuite extends PythonDataSourceSuiteBase { val q = spark.readStream.format(dataSourceName).load().writeStream.format("console").start() q.awaitTermination() } - assert(err.getErrorClass == "STREAM_FAILED") + assert(err.getCondition == "STREAM_FAILED") assert(err.getMessage.contains("error creating stream reader")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala index 38533825ece90..99483bc0ee8dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala @@ -423,14 +423,14 @@ class StateSchemaCompatibilityCheckerSuite extends SharedSparkSession { // collation checks are also performed in this path. so we need to check for them explicitly. if (keyCollationChecks) { assert(ex.getMessage.contains("Binary inequality column is not supported")) - assert(ex.getErrorClass === "STATE_STORE_UNSUPPORTED_OPERATION_BINARY_INEQUALITY") + assert(ex.getCondition === "STATE_STORE_UNSUPPORTED_OPERATION_BINARY_INEQUALITY") } else { if (ignoreValueSchema) { // if value schema is ignored, the mismatch has to be on the key schema - assert(ex.getErrorClass === "STATE_STORE_KEY_SCHEMA_NOT_COMPATIBLE") + assert(ex.getCondition === "STATE_STORE_KEY_SCHEMA_NOT_COMPATIBLE") } else { - assert(ex.getErrorClass === "STATE_STORE_KEY_SCHEMA_NOT_COMPATIBLE" || - ex.getErrorClass === "STATE_STORE_VALUE_SCHEMA_NOT_COMPATIBLE") + assert(ex.getCondition === "STATE_STORE_KEY_SCHEMA_NOT_COMPATIBLE" || + ex.getCondition === "STATE_STORE_VALUE_SCHEMA_NOT_COMPATIBLE") } assert(ex.getMessage.contains("does not match existing")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index 8bbc7a31760d9..2a9944a81cb2a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -1373,7 +1373,7 @@ abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider] put(store, "a", 0, 0) val e = intercept[SparkException](quietly { store.commit() } ) - assert(e.getErrorClass == "CANNOT_WRITE_STATE_STORE.CANNOT_COMMIT") + assert(e.getCondition == "CANNOT_WRITE_STATE_STORE.CANNOT_COMMIT") if (store.getClass.getName contains ROCKSDB_STATE_STORE) { assert(e.getMessage contains "RocksDBStateStore[id=(op=0,part=0)") } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala index aca968745d198..0cc4f7bf2548e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala @@ -504,6 +504,12 @@ class ColumnVectorSuite extends SparkFunSuite with SQLHelper { val arr = new ColumnarArray(testVector, 0, testVector.capacity) assert(arr.toSeq(testVector.dataType) == expected) assert(arr.copy().toSeq(testVector.dataType) == expected) + + if (expected.nonEmpty) { + val withOffset = new ColumnarArray(testVector, 1, testVector.capacity - 1) + assert(withOffset.toSeq(testVector.dataType) == expected.tail) + assert(withOffset.copy().toSeq(testVector.dataType) == expected.tail) + } } testVectors("getInts with dictionary and nulls", 3, IntegerType) { testVector => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 1a6cdd1258cc3..2b58440baf852 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -492,8 +492,8 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { val sqlText = "set time zone interval 19 hours" checkError( exception = intercept[ParseException](sql(sqlText)), - condition = "_LEGACY_ERROR_TEMP_0044", - parameters = Map.empty, + condition = "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + parameters = Map("input" -> "19"), context = ExpectedContext(sqlText, 0, 30)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala index 054c7e644ff55..0550fae3805d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala @@ -2688,7 +2688,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel val df = sql("SELECT SUM(2147483647 + DEPT) FROM h2.test.employee") checkAggregateRemoved(df, ansiMode) val expectedPlanFragment = if (ansiMode) { - "PushedAggregates: [SUM(2147483647 + DEPT)], " + + "PushedAggregates: [SUM(DEPT + 2147483647)], " + "PushedFilters: [], " + "PushedGroupByExpressions: []" } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 41447d8af5740..baf99798965da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -956,7 +956,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { val msg = intercept[SparkRuntimeException] { sql("INSERT INTO TABLE test_table SELECT 2, null") } - assert(msg.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(msg.getCondition == "NOT_NULL_ASSERT_VIOLATION") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index d9ce8002d285b..a0eea14e54eed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -296,7 +296,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { val exception = SparkException.internalError("testpurpose") testSerialization( new QueryTerminatedEvent(UUID.randomUUID, UUID.randomUUID, - Some(exception.getMessage), Some(exception.getErrorClass))) + Some(exception.getMessage), Some(exception.getCondition))) } test("only one progress event per interval when no data") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 8471995cb1e50..c12846d7512d9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -43,7 +43,7 @@ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Complete import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.connector.read.InputPartition import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, ReadLimit} -import org.apache.spark.sql.execution.exchange.ReusedExchangeExec +import org.apache.spark.sql.execution.exchange.{REQUIRED_BY_STATEFUL_OPERATOR, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.{MemorySink, TestForeachWriter} import org.apache.spark.sql.functions._ @@ -1448,6 +1448,28 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi } } + test("SPARK-49905 shuffle added by stateful operator should use the shuffle origin " + + "`REQUIRED_BY_STATEFUL_OPERATOR`") { + val inputData = MemoryStream[Int] + + // Use the streaming aggregation as an example - all stateful operators are using the same + // distribution, named `StatefulOpClusteredDistribution`. + val df = inputData.toDF().groupBy("value").count() + + testStream(df, OutputMode.Update())( + AddData(inputData, 1, 2, 3, 1, 2, 3), + CheckAnswer((1, 2), (2, 2), (3, 2)), + Execute { qe => + val shuffleOpt = qe.lastExecution.executedPlan.collect { + case s: ShuffleExchangeExec => s + } + + assert(shuffleOpt.nonEmpty, "No shuffle exchange found in the query plan") + assert(shuffleOpt.head.shuffleOrigin === REQUIRED_BY_STATEFUL_OPERATOR) + } + ) + } + private def checkAppendOutputModeException(df: DataFrame): Unit = { withTempDir { outputDir => withTempDir { checkpointDir => diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServerErrors.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServerErrors.scala index 8a8bdd4d38ee3..59d1b61f2f8e7 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServerErrors.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServerErrors.scala @@ -38,7 +38,7 @@ object HiveThriftServerErrors { def runningQueryError(e: Throwable, format: ErrorMessageFormat.Value): Throwable = e match { case st: SparkThrowable if format == ErrorMessageFormat.PRETTY => - val errorClassPrefix = Option(st.getErrorClass).map(e => s"[$e] ").getOrElse("") + val errorClassPrefix = Option(st.getCondition).map(e => s"[$e] ").getOrElse("") new HiveSQLException( s"Error running query: $errorClassPrefix${st.toString}", st.getSqlState, st) case st: SparkThrowable with Throwable =>