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 5d06c91dd092b..934572cd0d678 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 @@ -44,14 +44,14 @@ public class CollationAwareUTF8String { /** * Returns whether the target string starts with the specified prefix, starting from the * specified position (0-based index referring to character position in UTF8String), with respect - * to the UTF8_BINARY_LCASE collation. The method assumes that the prefix is already lowercased + * to the UTF8_LCASE collation. The method assumes that the prefix is already lowercased * prior to method call to avoid the overhead of calling .toLowerCase() multiple times on the * same prefix string. * * @param target the string to be searched in * @param lowercasePattern the string to be searched for * @param startPos the start position for searching (in the target string) - * @return whether the target string starts with the specified prefix in UTF8_BINARY_LCASE + * @return whether the target string starts with the specified prefix in UTF8_LCASE */ public static boolean lowercaseMatchFrom( final UTF8String target, @@ -63,7 +63,7 @@ public static boolean lowercaseMatchFrom( /** * Returns the length of the substring of the target string that starts with the specified * prefix, starting from the specified position (0-based index referring to character position - * in UTF8String), with respect to the UTF8_BINARY_LCASE collation. The method assumes that the + * in UTF8String), with respect to the UTF8_LCASE collation. The method assumes that the * prefix is already lowercased. The method only considers the part of target string that * starts from the specified (inclusive) position (that is, the method does not look at UTF8 * characters of the target string at or after position `endPos`). If the prefix is not found, @@ -90,7 +90,7 @@ private static int lowercaseMatchLengthFrom( /** * Returns the position of the first occurrence of the pattern string in the target string, * starting from the specified position (0-based index referring to character position in - * UTF8String), with respect to the UTF8_BINARY_LCASE collation. The method assumes that the + * UTF8String), with respect to the UTF8_LCASE collation. The method assumes that the * pattern string is already lowercased prior to call. If the pattern is not found, * MATCH_NOT_FOUND is returned. * @@ -115,7 +115,7 @@ private static int lowercaseFind( /** * Returns whether the target string ends with the specified suffix, ending at the specified * position (0-based index referring to character position in UTF8String), with respect to the - * UTF8_BINARY_LCASE collation. The method assumes that the suffix is already lowercased prior + * UTF8_LCASE collation. The method assumes that the suffix is already lowercased prior * to method call to avoid the overhead of calling .toLowerCase() multiple times on the same * suffix string. * @@ -134,7 +134,7 @@ public static boolean lowercaseMatchUntil( /** * Returns the length of the substring of the target string that ends with the specified * suffix, ending at the specified position (0-based index referring to character position in - * UTF8String), with respect to the UTF8_BINARY_LCASE collation. The method assumes that the + * UTF8String), with respect to the UTF8_LCASE collation. The method assumes that the * suffix is already lowercased. The method only considers the part of target string that ends * at the specified (non-inclusive) position (that is, the method does not look at UTF8 * characters of the target string at or after position `endPos`). If the suffix is not found, @@ -161,7 +161,7 @@ private static int lowercaseMatchLengthUntil( /** * Returns the position of the last occurrence of the pattern string in the target string, * ending at the specified position (0-based index referring to character position in - * UTF8String), with respect to the UTF8_BINARY_LCASE collation. The method assumes that the + * UTF8String), with respect to the UTF8_LCASE collation. The method assumes that the * pattern string is already lowercased prior to call. If the pattern is not found, * MATCH_NOT_FOUND is returned. * @@ -184,7 +184,7 @@ private static int lowercaseRFind( } /** - * Lowercase UTF8String comparison used for UTF8_BINARY_LCASE collation. While the default + * Lowercase UTF8String comparison used for UTF8_LCASE collation. While the default * UTF8String comparison is equivalent to a.toLowerCase().binaryCompare(b.toLowerCase()), this * method uses code points to compare the strings in a case-insensitive manner using ICU rules, * as well as handling special rules for one-to-many case mappings (see: lowerCaseCodePoints). @@ -489,7 +489,7 @@ public static int findInSet(final UTF8String match, final UTF8String set, int co /** * Returns the position of the first occurrence of the pattern string in the target string, * starting from the specified position (0-based index referring to character position in - * UTF8String), with respect to the UTF8_BINARY_LCASE collation. If the pattern is not found, + * UTF8String), with respect to the UTF8_LCASE collation. If the pattern is not found, * MATCH_NOT_FOUND is returned. * * @param target the string to be searched in 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 3c9240678467e..61ec6f7da2158 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 @@ -134,7 +134,7 @@ public static class Collation { /** * Support for Lowercase Equality implies that it is possible to check equality on * byte by byte level, but only after calling "UTF8String.toLowerCase" on both arguments. - * This allows custom collation support for UTF8_BINARY_LCASE collation in various Spark + * This allows custom collation support for UTF8_LCASE collation in various Spark * expressions, as this particular collation is not supported by the external ICU library. */ public final boolean supportsLowercaseEquality; @@ -220,7 +220,7 @@ public Collation( * --- * Some illustrative examples of collation name to ID mapping: * - UTF8_BINARY -> 0 - * - UTF8_BINARY_LCASE -> 1 + * - UTF8_LCASE -> 1 * - UNICODE -> 0x20000000 * - UNICODE_AI -> 0x20010000 * - UNICODE_CI -> 0x20020000 @@ -326,7 +326,7 @@ protected static SparkException collationInvalidNameException(String collationNa private static int collationNameToId(String collationName) throws SparkException { // Collation names provided by user are treated as case-insensitive. String collationNameUpper = collationName.toUpperCase(); - if (collationNameUpper.startsWith("UTF8_BINARY")) { + if (collationNameUpper.startsWith("UTF8_")) { return CollationSpecUTF8Binary.collationNameToId(collationName, collationNameUpper); } else { return CollationSpecICU.collationNameToId(collationName, collationNameUpper); @@ -339,7 +339,7 @@ private static int collationNameToId(String collationName) throws SparkException private static class CollationSpecUTF8Binary extends CollationSpec { /** - * Bit 0 in collation ID having value 0 for plain UTF8_BINARY and 1 for UTF8_BINARY_LCASE + * Bit 0 in collation ID having value 0 for plain UTF8_BINARY and 1 for UTF8_LCASE * collation. */ private enum CaseSensitivity { @@ -358,11 +358,11 @@ private enum CaseSensitivity { private static final int UTF8_BINARY_COLLATION_ID = new CollationSpecUTF8Binary(CaseSensitivity.UNSPECIFIED).collationId; - private static final int UTF8_BINARY_LCASE_COLLATION_ID = + private static final int UTF8_LCASE_COLLATION_ID = new CollationSpecUTF8Binary(CaseSensitivity.LCASE).collationId; protected static Collation UTF8_BINARY_COLLATION = new CollationSpecUTF8Binary(CaseSensitivity.UNSPECIFIED).buildCollation(); - protected static Collation UTF8_BINARY_LCASE_COLLATION = + protected static Collation UTF8_LCASE_COLLATION = new CollationSpecUTF8Binary(CaseSensitivity.LCASE).buildCollation(); private final int collationId; @@ -376,8 +376,8 @@ private static int collationNameToId(String originalName, String collationName) throws SparkException { if (UTF8_BINARY_COLLATION.collationName.equals(collationName)) { return UTF8_BINARY_COLLATION_ID; - } else if (UTF8_BINARY_LCASE_COLLATION.collationName.equals(collationName)) { - return UTF8_BINARY_LCASE_COLLATION_ID; + } else if (UTF8_LCASE_COLLATION.collationName.equals(collationName)) { + return UTF8_LCASE_COLLATION_ID; } else { // Throw exception with original (before case conversion) collation name. throw collationInvalidNameException(originalName); @@ -409,7 +409,7 @@ protected Collation buildCollation() { /* supportsLowercaseEquality = */ false); } else { return new Collation( - "UTF8_BINARY_LCASE", + "UTF8_LCASE", PROVIDER_SPARK, null, CollationAwareUTF8String::compareLowerCase, @@ -633,7 +633,7 @@ private static CollationSpecICU fromCollationId(int collationId) { // Locale ID remains after removing all other specifiers. int localeId = collationId; // Verify locale ID is valid against `ICULocaleNames` array. - assert (localeId < ICULocaleNames.length); + assert(localeId >= 0 && localeId < ICULocaleNames.length); CaseSensitivity caseSensitivity = CaseSensitivity.values()[caseSensitivityOrdinal]; AccentSensitivity accentSensitivity = AccentSensitivity.values()[accentSensitivityOrdinal]; String locale = ICULocaleNames[localeId]; @@ -728,8 +728,8 @@ public CollationIdentifier identifier() { public static final int UTF8_BINARY_COLLATION_ID = Collation.CollationSpecUTF8Binary.UTF8_BINARY_COLLATION_ID; - public static final int UTF8_BINARY_LCASE_COLLATION_ID = - Collation.CollationSpecUTF8Binary.UTF8_BINARY_LCASE_COLLATION_ID; + public static final int UTF8_LCASE_COLLATION_ID = + Collation.CollationSpecUTF8Binary.UTF8_LCASE_COLLATION_ID; public static final int UNICODE_COLLATION_ID = Collation.CollationSpecICU.UNICODE_COLLATION_ID; public static final int UNICODE_CI_COLLATION_ID = @@ -766,7 +766,7 @@ public static StringSearch getStringSearch( /** * Returns a collation-unaware StringSearch object for the given pattern and target strings. * While this object does not respect collation, it can be used to find occurrences of the pattern - * in the target string for UTF8_BINARY or UTF8_BINARY_LCASE (if arguments are lowercased). + * in the target string for UTF8_BINARY or UTF8_LCASE (if arguments are lowercased). */ public static StringSearch getStringSearch( final UTF8String targetUTF8String, 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 e415275360dc0..5eebec7f1301b 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 @@ -743,7 +743,7 @@ public static UTF8String execLowercase( public static boolean supportsLowercaseRegex(final int collationId) { // for regex, only Unicode case-insensitive matching is possible, - // so UTF8_BINARY_LCASE is treated as UNICODE_CI in this context + // so UTF8_LCASE is treated as UNICODE_CI in this context return CollationFactory.fetchCollation(collationId).supportsLowercaseEquality; } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CollationSupportSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CollationSupportSuite.java index b9d20be264cfc..58826005fc467 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CollationSupportSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CollationSupportSuite.java @@ -34,7 +34,7 @@ public class CollationSupportSuite { * the specified collations (as often seen in some pass-through Spark expressions). */ private final String[] testSupportedCollations = - {"UTF8_BINARY", "UTF8_BINARY_LCASE", "UNICODE", "UNICODE_CI"}; + {"UTF8_BINARY", "UTF8_LCASE", "UNICODE", "UNICODE_CI"}; /** * Collation-aware UTF8String comparison. @@ -83,35 +83,35 @@ public void testCompare() throws SparkException { // Non-ASCII strings assertStringCompare("äü", "bü", "UTF8_BINARY", 1); assertStringCompare("bxx", "bü", "UTF8_BINARY", -1); - assertStringCompare("äü", "bü", "UTF8_BINARY_LCASE", 1); - assertStringCompare("bxx", "bü", "UTF8_BINARY_LCASE", -1); + assertStringCompare("äü", "bü", "UTF8_LCASE", 1); + assertStringCompare("bxx", "bü", "UTF8_LCASE", -1); assertStringCompare("äü", "bü", "UNICODE", -1); assertStringCompare("bxx", "bü", "UNICODE", 1); assertStringCompare("äü", "bü", "UNICODE_CI", -1); assertStringCompare("bxx", "bü", "UNICODE_CI", 1); // Case variation assertStringCompare("AbCd", "aBcD", "UTF8_BINARY", -1); - assertStringCompare("ABCD", "abcd", "UTF8_BINARY_LCASE", 0); + assertStringCompare("ABCD", "abcd", "UTF8_LCASE", 0); assertStringCompare("AbcD", "aBCd", "UNICODE", 1); assertStringCompare("abcd", "ABCD", "UNICODE_CI", 0); // Accent variation assertStringCompare("aBćD", "ABĆD", "UTF8_BINARY", 1); - assertStringCompare("AbCδ", "ABCΔ", "UTF8_BINARY_LCASE", 0); + assertStringCompare("AbCδ", "ABCΔ", "UTF8_LCASE", 0); assertStringCompare("äBCd", "ÄBCD", "UNICODE", -1); assertStringCompare("Ab́cD", "AB́CD", "UNICODE_CI", 0); // Case-variable character length assertStringCompare("i\u0307", "İ", "UTF8_BINARY", -1); assertStringCompare("İ", "i\u0307", "UTF8_BINARY", 1); - assertStringCompare("i\u0307", "İ", "UTF8_BINARY_LCASE", 0); - assertStringCompare("İ", "i\u0307", "UTF8_BINARY_LCASE", 0); + assertStringCompare("i\u0307", "İ", "UTF8_LCASE", 0); + assertStringCompare("İ", "i\u0307", "UTF8_LCASE", 0); assertStringCompare("i\u0307", "İ", "UNICODE", -1); assertStringCompare("İ", "i\u0307", "UNICODE", 1); assertStringCompare("i\u0307", "İ", "UNICODE_CI", 0); assertStringCompare("İ", "i\u0307", "UNICODE_CI", 0); - assertStringCompare("i\u0307İ", "i\u0307İ", "UTF8_BINARY_LCASE", 0); - assertStringCompare("i\u0307İ", "İi\u0307", "UTF8_BINARY_LCASE", 0); - assertStringCompare("İi\u0307", "i\u0307İ", "UTF8_BINARY_LCASE", 0); - assertStringCompare("İi\u0307", "İi\u0307", "UTF8_BINARY_LCASE", 0); + assertStringCompare("i\u0307İ", "i\u0307İ", "UTF8_LCASE", 0); + assertStringCompare("i\u0307İ", "İi\u0307", "UTF8_LCASE", 0); + assertStringCompare("İi\u0307", "i\u0307İ", "UTF8_LCASE", 0); + assertStringCompare("İi\u0307", "İi\u0307", "UTF8_LCASE", 0); assertStringCompare("i\u0307İ", "i\u0307İ", "UNICODE_CI", 0); assertStringCompare("i\u0307İ", "İi\u0307", "UNICODE_CI", 0); assertStringCompare("İi\u0307", "i\u0307İ", "UNICODE_CI", 0); @@ -120,9 +120,9 @@ public void testCompare() throws SparkException { assertStringCompare("ς", "σ", "UTF8_BINARY", -1); assertStringCompare("ς", "Σ", "UTF8_BINARY", 1); assertStringCompare("σ", "Σ", "UTF8_BINARY", 1); - assertStringCompare("ς", "σ", "UTF8_BINARY_LCASE", 0); - assertStringCompare("ς", "Σ", "UTF8_BINARY_LCASE", 0); - assertStringCompare("σ", "Σ", "UTF8_BINARY_LCASE", 0); + assertStringCompare("ς", "σ", "UTF8_LCASE", 0); + assertStringCompare("ς", "Σ", "UTF8_LCASE", 0); + assertStringCompare("σ", "Σ", "UTF8_LCASE", 0); assertStringCompare("ς", "σ", "UNICODE", 1); assertStringCompare("ς", "Σ", "UNICODE", 1); assertStringCompare("σ", "Σ", "UNICODE", -1); @@ -198,9 +198,9 @@ public void testContains() throws SparkException { assertContains("", "", "UNICODE", true); assertContains("c", "", "UNICODE", true); assertContains("", "c", "UNICODE", false); - assertContains("", "", "UTF8_BINARY_LCASE", true); - assertContains("c", "", "UTF8_BINARY_LCASE", true); - assertContains("", "c", "UTF8_BINARY_LCASE", false); + assertContains("", "", "UTF8_LCASE", true); + assertContains("c", "", "UTF8_LCASE", true); + assertContains("", "c", "UTF8_LCASE", false); assertContains("", "", "UNICODE_CI", true); assertContains("c", "", "UNICODE_CI", true); assertContains("", "c", "UNICODE_CI", false); @@ -211,9 +211,9 @@ public void testContains() throws SparkException { assertContains("abcde", "abcde", "UNICODE", true); assertContains("abcde", "aBcDe", "UNICODE", false); assertContains("abcde", "fghij", "UNICODE", false); - assertContains("abcde", "C", "UTF8_BINARY_LCASE", true); - assertContains("abcde", "AbCdE", "UTF8_BINARY_LCASE", true); - assertContains("abcde", "X", "UTF8_BINARY_LCASE", false); + assertContains("abcde", "C", "UTF8_LCASE", true); + assertContains("abcde", "AbCdE", "UTF8_LCASE", true); + assertContains("abcde", "X", "UTF8_LCASE", false); assertContains("abcde", "c", "UNICODE_CI", true); assertContains("abcde", "bCD", "UNICODE_CI", true); assertContains("abcde", "123", "UNICODE_CI", false); @@ -222,8 +222,8 @@ public void testContains() throws SparkException { assertContains("aBcDe", "BcD", "UTF8_BINARY", true); assertContains("aBcDe", "abcde", "UNICODE", false); assertContains("aBcDe", "aBcDe", "UNICODE", true); - assertContains("aBcDe", "bcd", "UTF8_BINARY_LCASE", true); - assertContains("aBcDe", "BCD", "UTF8_BINARY_LCASE", true); + assertContains("aBcDe", "bcd", "UTF8_LCASE", true); + assertContains("aBcDe", "BCD", "UTF8_LCASE", true); assertContains("aBcDe", "abcde", "UNICODE_CI", true); assertContains("aBcDe", "AbCdE", "UNICODE_CI", true); // Accent variation @@ -231,8 +231,8 @@ public void testContains() throws SparkException { assertContains("aBcDe", "BćD", "UTF8_BINARY", false); assertContains("aBcDe", "abćde", "UNICODE", false); assertContains("aBcDe", "aBćDe", "UNICODE", false); - assertContains("aBcDe", "bćd", "UTF8_BINARY_LCASE", false); - assertContains("aBcDe", "BĆD", "UTF8_BINARY_LCASE", false); + assertContains("aBcDe", "bćd", "UTF8_LCASE", false); + assertContains("aBcDe", "BĆD", "UTF8_LCASE", false); assertContains("aBcDe", "abćde", "UNICODE_CI", false); assertContains("aBcDe", "AbĆdE", "UNICODE_CI", false); // Variable byte length characters @@ -244,21 +244,21 @@ public void testContains() throws SparkException { assertContains("ab世De", "AB世dE", "UNICODE", false); assertContains("äbćδe", "äbćδe", "UNICODE", true); assertContains("äbćδe", "ÄBcΔÉ", "UNICODE", false); - assertContains("ab世De", "b世D", "UTF8_BINARY_LCASE", true); - assertContains("ab世De", "B世d", "UTF8_BINARY_LCASE", true); - assertContains("äbćδe", "bćδ", "UTF8_BINARY_LCASE", true); - assertContains("äbćδe", "BcΔ", "UTF8_BINARY_LCASE", false); + assertContains("ab世De", "b世D", "UTF8_LCASE", true); + assertContains("ab世De", "B世d", "UTF8_LCASE", true); + assertContains("äbćδe", "bćδ", "UTF8_LCASE", true); + assertContains("äbćδe", "BcΔ", "UTF8_LCASE", false); assertContains("ab世De", "ab世De", "UNICODE_CI", true); assertContains("ab世De", "AB世dE", "UNICODE_CI", true); assertContains("äbćδe", "ÄbćδE", "UNICODE_CI", true); assertContains("äbćδe", "ÄBcΔÉ", "UNICODE_CI", false); // Characters with the same binary lowercase representation - assertContains("The Kelvin.", "Kelvin", "UTF8_BINARY_LCASE", true); - assertContains("The Kelvin.", "Kelvin", "UTF8_BINARY_LCASE", true); - assertContains("The KKelvin.", "KKelvin", "UTF8_BINARY_LCASE", true); - assertContains("2 Kelvin.", "2 Kelvin", "UTF8_BINARY_LCASE", true); - assertContains("2 Kelvin.", "2 Kelvin", "UTF8_BINARY_LCASE", true); - assertContains("The KKelvin.", "KKelvin,", "UTF8_BINARY_LCASE", false); + assertContains("The Kelvin.", "Kelvin", "UTF8_LCASE", true); + assertContains("The Kelvin.", "Kelvin", "UTF8_LCASE", true); + assertContains("The KKelvin.", "KKelvin", "UTF8_LCASE", true); + assertContains("2 Kelvin.", "2 Kelvin", "UTF8_LCASE", true); + assertContains("2 Kelvin.", "2 Kelvin", "UTF8_LCASE", true); + assertContains("The KKelvin.", "KKelvin,", "UTF8_LCASE", false); // Case-variable character length assertContains("i̇", "i", "UNICODE_CI", false); assertContains("i̇", "\u0307", "UNICODE_CI", false); @@ -272,20 +272,20 @@ public void testContains() throws SparkException { assertContains("adİos", "Io", "UNICODE_CI", false); assertContains("adİos", "i̇o", "UNICODE_CI", true); assertContains("adİos", "İo", "UNICODE_CI", true); - assertContains("i̇", "i", "UTF8_BINARY_LCASE", true); // != UNICODE_CI - assertContains("İ", "\u0307", "UTF8_BINARY_LCASE", false); - assertContains("İ", "i", "UTF8_BINARY_LCASE", false); - assertContains("i̇", "\u0307", "UTF8_BINARY_LCASE", true); // != UNICODE_CI - assertContains("i̇", "İ", "UTF8_BINARY_LCASE", true); - assertContains("İ", "i", "UTF8_BINARY_LCASE", false); - assertContains("adi̇os", "io", "UTF8_BINARY_LCASE", false); - assertContains("adi̇os", "Io", "UTF8_BINARY_LCASE", false); - assertContains("adi̇os", "i̇o", "UTF8_BINARY_LCASE", true); - assertContains("adi̇os", "İo", "UTF8_BINARY_LCASE", true); - assertContains("adİos", "io", "UTF8_BINARY_LCASE", false); - assertContains("adİos", "Io", "UTF8_BINARY_LCASE", false); - assertContains("adİos", "i̇o", "UTF8_BINARY_LCASE", true); - assertContains("adİos", "İo", "UTF8_BINARY_LCASE", true); + assertContains("i̇", "i", "UTF8_LCASE", true); // != UNICODE_CI + assertContains("İ", "\u0307", "UTF8_LCASE", false); + assertContains("İ", "i", "UTF8_LCASE", false); + assertContains("i̇", "\u0307", "UTF8_LCASE", true); // != UNICODE_CI + assertContains("i̇", "İ", "UTF8_LCASE", true); + assertContains("İ", "i", "UTF8_LCASE", false); + assertContains("adi̇os", "io", "UTF8_LCASE", false); + assertContains("adi̇os", "Io", "UTF8_LCASE", false); + assertContains("adi̇os", "i̇o", "UTF8_LCASE", true); + assertContains("adi̇os", "İo", "UTF8_LCASE", true); + assertContains("adİos", "io", "UTF8_LCASE", false); + assertContains("adİos", "Io", "UTF8_LCASE", false); + assertContains("adİos", "i̇o", "UTF8_LCASE", true); + assertContains("adİos", "İo", "UTF8_LCASE", true); } private void assertStartsWith( @@ -306,9 +306,9 @@ public void testStartsWith() throws SparkException { assertStartsWith("", "", "UNICODE", true); assertStartsWith("c", "", "UNICODE", true); assertStartsWith("", "c", "UNICODE", false); - assertStartsWith("", "", "UTF8_BINARY_LCASE", true); - assertStartsWith("c", "", "UTF8_BINARY_LCASE", true); - assertStartsWith("", "c", "UTF8_BINARY_LCASE", false); + assertStartsWith("", "", "UTF8_LCASE", true); + assertStartsWith("c", "", "UTF8_LCASE", true); + assertStartsWith("", "c", "UTF8_LCASE", false); assertStartsWith("", "", "UNICODE_CI", true); assertStartsWith("c", "", "UNICODE_CI", true); assertStartsWith("", "c", "UNICODE_CI", false); @@ -319,9 +319,9 @@ public void testStartsWith() throws SparkException { assertStartsWith("abcde", "abcde", "UNICODE", true); assertStartsWith("abcde", "aBcDe", "UNICODE", false); assertStartsWith("abcde", "fghij", "UNICODE", false); - assertStartsWith("abcde", "A", "UTF8_BINARY_LCASE", true); - assertStartsWith("abcde", "AbCdE", "UTF8_BINARY_LCASE", true); - assertStartsWith("abcde", "X", "UTF8_BINARY_LCASE", false); + assertStartsWith("abcde", "A", "UTF8_LCASE", true); + assertStartsWith("abcde", "AbCdE", "UTF8_LCASE", true); + assertStartsWith("abcde", "X", "UTF8_LCASE", false); assertStartsWith("abcde", "a", "UNICODE_CI", true); assertStartsWith("abcde", "aBC", "UNICODE_CI", true); assertStartsWith("abcde", "bcd", "UNICODE_CI", false); @@ -331,8 +331,8 @@ public void testStartsWith() throws SparkException { assertStartsWith("aBcDe", "aBc", "UTF8_BINARY", true); assertStartsWith("aBcDe", "abcde", "UNICODE", false); assertStartsWith("aBcDe", "aBcDe", "UNICODE", true); - assertStartsWith("aBcDe", "abc", "UTF8_BINARY_LCASE", true); - assertStartsWith("aBcDe", "ABC", "UTF8_BINARY_LCASE", true); + assertStartsWith("aBcDe", "abc", "UTF8_LCASE", true); + assertStartsWith("aBcDe", "ABC", "UTF8_LCASE", true); assertStartsWith("aBcDe", "abcde", "UNICODE_CI", true); assertStartsWith("aBcDe", "AbCdE", "UNICODE_CI", true); // Accent variation @@ -340,8 +340,8 @@ public void testStartsWith() throws SparkException { assertStartsWith("aBcDe", "aBć", "UTF8_BINARY", false); assertStartsWith("aBcDe", "abćde", "UNICODE", false); assertStartsWith("aBcDe", "aBćDe", "UNICODE", false); - assertStartsWith("aBcDe", "abć", "UTF8_BINARY_LCASE", false); - assertStartsWith("aBcDe", "ABĆ", "UTF8_BINARY_LCASE", false); + assertStartsWith("aBcDe", "abć", "UTF8_LCASE", false); + assertStartsWith("aBcDe", "ABĆ", "UTF8_LCASE", false); assertStartsWith("aBcDe", "abćde", "UNICODE_CI", false); assertStartsWith("aBcDe", "AbĆdE", "UNICODE_CI", false); // Variable byte length characters @@ -353,21 +353,21 @@ public void testStartsWith() throws SparkException { assertStartsWith("ab世De", "AB世dE", "UNICODE", false); assertStartsWith("äbćδe", "äbćδe", "UNICODE", true); assertStartsWith("äbćδe", "ÄBcΔÉ", "UNICODE", false); - assertStartsWith("ab世De", "ab世", "UTF8_BINARY_LCASE", true); - assertStartsWith("ab世De", "aB世", "UTF8_BINARY_LCASE", true); - assertStartsWith("äbćδe", "äbć", "UTF8_BINARY_LCASE", true); - assertStartsWith("äbćδe", "äBc", "UTF8_BINARY_LCASE", false); + assertStartsWith("ab世De", "ab世", "UTF8_LCASE", true); + assertStartsWith("ab世De", "aB世", "UTF8_LCASE", true); + assertStartsWith("äbćδe", "äbć", "UTF8_LCASE", true); + assertStartsWith("äbćδe", "äBc", "UTF8_LCASE", false); assertStartsWith("ab世De", "ab世De", "UNICODE_CI", true); assertStartsWith("ab世De", "AB世dE", "UNICODE_CI", true); assertStartsWith("äbćδe", "ÄbćδE", "UNICODE_CI", true); assertStartsWith("äbćδe", "ÄBcΔÉ", "UNICODE_CI", false); // Characters with the same binary lowercase representation - assertStartsWith("Kelvin.", "Kelvin", "UTF8_BINARY_LCASE", true); - assertStartsWith("Kelvin.", "Kelvin", "UTF8_BINARY_LCASE", true); - assertStartsWith("KKelvin.", "KKelvin", "UTF8_BINARY_LCASE", true); - assertStartsWith("2 Kelvin.", "2 Kelvin", "UTF8_BINARY_LCASE", true); - assertStartsWith("2 Kelvin.", "2 Kelvin", "UTF8_BINARY_LCASE", true); - assertStartsWith("KKelvin.", "KKelvin,", "UTF8_BINARY_LCASE", false); + assertStartsWith("Kelvin.", "Kelvin", "UTF8_LCASE", true); + assertStartsWith("Kelvin.", "Kelvin", "UTF8_LCASE", true); + assertStartsWith("KKelvin.", "KKelvin", "UTF8_LCASE", true); + assertStartsWith("2 Kelvin.", "2 Kelvin", "UTF8_LCASE", true); + assertStartsWith("2 Kelvin.", "2 Kelvin", "UTF8_LCASE", true); + assertStartsWith("KKelvin.", "KKelvin,", "UTF8_LCASE", false); // Case-variable character length assertStartsWith("i̇", "i", "UNICODE_CI", false); assertStartsWith("i̇", "İ", "UNICODE_CI", true); @@ -384,21 +384,21 @@ public void testStartsWith() throws SparkException { assertStartsWith("İonic", "Io", "UNICODE_CI", false); assertStartsWith("İonic", "i̇o", "UNICODE_CI", true); assertStartsWith("İonic", "İo", "UNICODE_CI", true); - assertStartsWith("i̇", "i", "UTF8_BINARY_LCASE", true); // != UNICODE_CI - assertStartsWith("i̇", "İ", "UTF8_BINARY_LCASE", true); - assertStartsWith("İ", "i", "UTF8_BINARY_LCASE", false); - assertStartsWith("İİİ", "i̇i̇", "UTF8_BINARY_LCASE", true); - assertStartsWith("İİİ", "i̇i", "UTF8_BINARY_LCASE", false); - assertStartsWith("İi̇İ", "i̇İ", "UTF8_BINARY_LCASE", true); - assertStartsWith("i̇İi̇i̇", "İi̇İi", "UTF8_BINARY_LCASE", true); // != UNICODE_CI - assertStartsWith("i̇onic", "io", "UTF8_BINARY_LCASE", false); - assertStartsWith("i̇onic", "Io", "UTF8_BINARY_LCASE", false); - assertStartsWith("i̇onic", "i̇o", "UTF8_BINARY_LCASE", true); - assertStartsWith("i̇onic", "İo", "UTF8_BINARY_LCASE", true); - assertStartsWith("İonic", "io", "UTF8_BINARY_LCASE", false); - assertStartsWith("İonic", "Io", "UTF8_BINARY_LCASE", false); - assertStartsWith("İonic", "i̇o", "UTF8_BINARY_LCASE", true); - assertStartsWith("İonic", "İo", "UTF8_BINARY_LCASE", true); + assertStartsWith("i̇", "i", "UTF8_LCASE", true); // != UNICODE_CI + assertStartsWith("i̇", "İ", "UTF8_LCASE", true); + assertStartsWith("İ", "i", "UTF8_LCASE", false); + assertStartsWith("İİİ", "i̇i̇", "UTF8_LCASE", true); + assertStartsWith("İİİ", "i̇i", "UTF8_LCASE", false); + assertStartsWith("İi̇İ", "i̇İ", "UTF8_LCASE", true); + assertStartsWith("i̇İi̇i̇", "İi̇İi", "UTF8_LCASE", true); // != UNICODE_CI + assertStartsWith("i̇onic", "io", "UTF8_LCASE", false); + assertStartsWith("i̇onic", "Io", "UTF8_LCASE", false); + assertStartsWith("i̇onic", "i̇o", "UTF8_LCASE", true); + assertStartsWith("i̇onic", "İo", "UTF8_LCASE", true); + assertStartsWith("İonic", "io", "UTF8_LCASE", false); + assertStartsWith("İonic", "Io", "UTF8_LCASE", false); + assertStartsWith("İonic", "i̇o", "UTF8_LCASE", true); + assertStartsWith("İonic", "İo", "UTF8_LCASE", true); } private void assertEndsWith(String pattern, String suffix, String collationName, boolean expected) @@ -418,9 +418,9 @@ public void testEndsWith() throws SparkException { assertEndsWith("", "", "UNICODE", true); assertEndsWith("c", "", "UNICODE", true); assertEndsWith("", "c", "UNICODE", false); - assertEndsWith("", "", "UTF8_BINARY_LCASE", true); - assertEndsWith("c", "", "UTF8_BINARY_LCASE", true); - assertEndsWith("", "c", "UTF8_BINARY_LCASE", false); + assertEndsWith("", "", "UTF8_LCASE", true); + assertEndsWith("c", "", "UTF8_LCASE", true); + assertEndsWith("", "c", "UTF8_LCASE", false); assertEndsWith("", "", "UNICODE_CI", true); assertEndsWith("c", "", "UNICODE_CI", true); assertEndsWith("", "c", "UNICODE_CI", false); @@ -431,9 +431,9 @@ public void testEndsWith() throws SparkException { assertEndsWith("abcde", "abcde", "UNICODE", true); assertEndsWith("abcde", "aBcDe", "UNICODE", false); assertEndsWith("abcde", "fghij", "UNICODE", false); - assertEndsWith("abcde", "E", "UTF8_BINARY_LCASE", true); - assertEndsWith("abcde", "AbCdE", "UTF8_BINARY_LCASE", true); - assertEndsWith("abcde", "X", "UTF8_BINARY_LCASE", false); + assertEndsWith("abcde", "E", "UTF8_LCASE", true); + assertEndsWith("abcde", "AbCdE", "UTF8_LCASE", true); + assertEndsWith("abcde", "X", "UTF8_LCASE", false); assertEndsWith("abcde", "e", "UNICODE_CI", true); assertEndsWith("abcde", "CDe", "UNICODE_CI", true); assertEndsWith("abcde", "bcd", "UNICODE_CI", false); @@ -443,8 +443,8 @@ public void testEndsWith() throws SparkException { assertEndsWith("aBcDe", "cDe", "UTF8_BINARY", true); assertEndsWith("aBcDe", "abcde", "UNICODE", false); assertEndsWith("aBcDe", "aBcDe", "UNICODE", true); - assertEndsWith("aBcDe", "cde", "UTF8_BINARY_LCASE", true); - assertEndsWith("aBcDe", "CDE", "UTF8_BINARY_LCASE", true); + assertEndsWith("aBcDe", "cde", "UTF8_LCASE", true); + assertEndsWith("aBcDe", "CDE", "UTF8_LCASE", true); assertEndsWith("aBcDe", "abcde", "UNICODE_CI", true); assertEndsWith("aBcDe", "AbCdE", "UNICODE_CI", true); // Accent variation @@ -452,8 +452,8 @@ public void testEndsWith() throws SparkException { assertEndsWith("aBcDe", "ćDe", "UTF8_BINARY", false); assertEndsWith("aBcDe", "abćde", "UNICODE", false); assertEndsWith("aBcDe", "aBćDe", "UNICODE", false); - assertEndsWith("aBcDe", "ćde", "UTF8_BINARY_LCASE", false); - assertEndsWith("aBcDe", "ĆDE", "UTF8_BINARY_LCASE", false); + assertEndsWith("aBcDe", "ćde", "UTF8_LCASE", false); + assertEndsWith("aBcDe", "ĆDE", "UTF8_LCASE", false); assertEndsWith("aBcDe", "abćde", "UNICODE_CI", false); assertEndsWith("aBcDe", "AbĆdE", "UNICODE_CI", false); // Variable byte length characters @@ -465,21 +465,21 @@ public void testEndsWith() throws SparkException { assertEndsWith("ab世De", "AB世dE", "UNICODE", false); assertEndsWith("äbćδe", "äbćδe", "UNICODE", true); assertEndsWith("äbćδe", "ÄBcΔÉ", "UNICODE", false); - assertEndsWith("ab世De", "世De", "UTF8_BINARY_LCASE", true); - assertEndsWith("ab世De", "世dE", "UTF8_BINARY_LCASE", true); - assertEndsWith("äbćδe", "ćδe", "UTF8_BINARY_LCASE", true); - assertEndsWith("äbćδe", "cδE", "UTF8_BINARY_LCASE", false); + assertEndsWith("ab世De", "世De", "UTF8_LCASE", true); + assertEndsWith("ab世De", "世dE", "UTF8_LCASE", true); + assertEndsWith("äbćδe", "ćδe", "UTF8_LCASE", true); + assertEndsWith("äbćδe", "cδE", "UTF8_LCASE", false); assertEndsWith("ab世De", "ab世De", "UNICODE_CI", true); assertEndsWith("ab世De", "AB世dE", "UNICODE_CI", true); assertEndsWith("äbćδe", "ÄbćδE", "UNICODE_CI", true); assertEndsWith("äbćδe", "ÄBcΔÉ", "UNICODE_CI", false); // Characters with the same binary lowercase representation - assertEndsWith("The Kelvin", "Kelvin", "UTF8_BINARY_LCASE", true); - assertEndsWith("The Kelvin", "Kelvin", "UTF8_BINARY_LCASE", true); - assertEndsWith("The KKelvin", "KKelvin", "UTF8_BINARY_LCASE", true); - assertEndsWith("The 2 Kelvin", "2 Kelvin", "UTF8_BINARY_LCASE", true); - assertEndsWith("The 2 Kelvin", "2 Kelvin", "UTF8_BINARY_LCASE", true); - assertEndsWith("The KKelvin", "KKelvin,", "UTF8_BINARY_LCASE", false); + assertEndsWith("The Kelvin", "Kelvin", "UTF8_LCASE", true); + assertEndsWith("The Kelvin", "Kelvin", "UTF8_LCASE", true); + assertEndsWith("The KKelvin", "KKelvin", "UTF8_LCASE", true); + assertEndsWith("The 2 Kelvin", "2 Kelvin", "UTF8_LCASE", true); + assertEndsWith("The 2 Kelvin", "2 Kelvin", "UTF8_LCASE", true); + assertEndsWith("The KKelvin", "KKelvin,", "UTF8_LCASE", false); // Case-variable character length assertEndsWith("i̇", "\u0307", "UNICODE_CI", false); assertEndsWith("i̇", "İ", "UNICODE_CI", true); @@ -496,22 +496,22 @@ public void testEndsWith() throws SparkException { assertEndsWith("the İo", "Io", "UNICODE_CI", false); assertEndsWith("the İo", "i̇o", "UNICODE_CI", true); assertEndsWith("the İo", "İo", "UNICODE_CI", true); - assertEndsWith("i̇", "\u0307", "UTF8_BINARY_LCASE", true); // != UNICODE_CI - assertEndsWith("i̇", "İ", "UTF8_BINARY_LCASE", true); - assertEndsWith("İ", "\u0307", "UTF8_BINARY_LCASE", false); - assertEndsWith("İİİ", "i̇i̇", "UTF8_BINARY_LCASE", true); - assertEndsWith("İİİ", "ii̇", "UTF8_BINARY_LCASE", false); - assertEndsWith("İi̇İ", "İi̇", "UTF8_BINARY_LCASE", true); - assertEndsWith("i̇İi̇i̇", "\u0307İi̇İ", "UTF8_BINARY_LCASE", true); // != UNICODE_CI - assertEndsWith("i̇İi̇i̇", "\u0307İİ", "UTF8_BINARY_LCASE", false); - assertEndsWith("the i̇o", "io", "UTF8_BINARY_LCASE", false); - assertEndsWith("the i̇o", "Io", "UTF8_BINARY_LCASE", false); - assertEndsWith("the i̇o", "i̇o", "UTF8_BINARY_LCASE", true); - assertEndsWith("the i̇o", "İo", "UTF8_BINARY_LCASE", true); - assertEndsWith("the İo", "io", "UTF8_BINARY_LCASE", false); - assertEndsWith("the İo", "Io", "UTF8_BINARY_LCASE", false); - assertEndsWith("the İo", "i̇o", "UTF8_BINARY_LCASE", true); - assertEndsWith("the İo", "İo", "UTF8_BINARY_LCASE", true); + assertEndsWith("i̇", "\u0307", "UTF8_LCASE", true); // != UNICODE_CI + assertEndsWith("i̇", "İ", "UTF8_LCASE", true); + assertEndsWith("İ", "\u0307", "UTF8_LCASE", false); + assertEndsWith("İİİ", "i̇i̇", "UTF8_LCASE", true); + assertEndsWith("İİİ", "ii̇", "UTF8_LCASE", false); + assertEndsWith("İi̇İ", "İi̇", "UTF8_LCASE", true); + assertEndsWith("i̇İi̇i̇", "\u0307İi̇İ", "UTF8_LCASE", true); // != UNICODE_CI + assertEndsWith("i̇İi̇i̇", "\u0307İİ", "UTF8_LCASE", false); + assertEndsWith("the i̇o", "io", "UTF8_LCASE", false); + assertEndsWith("the i̇o", "Io", "UTF8_LCASE", false); + assertEndsWith("the i̇o", "i̇o", "UTF8_LCASE", true); + assertEndsWith("the i̇o", "İo", "UTF8_LCASE", true); + assertEndsWith("the İo", "io", "UTF8_LCASE", false); + assertEndsWith("the İo", "Io", "UTF8_LCASE", false); + assertEndsWith("the İo", "i̇o", "UTF8_LCASE", true); + assertEndsWith("the İo", "İo", "UTF8_LCASE", true); } private void assertStringSplitSQL(String str, String delimiter, String collationName, @@ -544,9 +544,9 @@ public void testStringSplitSQL() throws SparkException { assertStringSplitSQL("", "", "UNICODE", empty_match); assertStringSplitSQL("abc", "", "UNICODE", array_abc); assertStringSplitSQL("", "abc", "UNICODE", empty_match); - assertStringSplitSQL("", "", "UTF8_BINARY_LCASE", empty_match); - assertStringSplitSQL("abc", "", "UTF8_BINARY_LCASE", array_abc); - assertStringSplitSQL("", "abc", "UTF8_BINARY_LCASE", empty_match); + assertStringSplitSQL("", "", "UTF8_LCASE", empty_match); + assertStringSplitSQL("abc", "", "UTF8_LCASE", array_abc); + assertStringSplitSQL("", "abc", "UTF8_LCASE", empty_match); assertStringSplitSQL("", "", "UNICODE_CI", empty_match); assertStringSplitSQL("abc", "", "UNICODE_CI", array_abc); assertStringSplitSQL("", "abc", "UNICODE_CI", empty_match); @@ -557,9 +557,9 @@ public void testStringSplitSQL() throws SparkException { assertStringSplitSQL("1a2", "1a2", "UNICODE", full_match); assertStringSplitSQL("1a2", "1A2", "UNICODE", array_1a2); assertStringSplitSQL("1a2", "3b4", "UNICODE", array_1a2); - assertStringSplitSQL("1a2", "A", "UTF8_BINARY_LCASE", array_1_2); - assertStringSplitSQL("1a2", "1A2", "UTF8_BINARY_LCASE", full_match); - assertStringSplitSQL("1a2", "X", "UTF8_BINARY_LCASE", array_1a2); + assertStringSplitSQL("1a2", "A", "UTF8_LCASE", array_1_2); + assertStringSplitSQL("1a2", "1A2", "UTF8_LCASE", full_match); + assertStringSplitSQL("1a2", "X", "UTF8_LCASE", array_1a2); assertStringSplitSQL("1a2", "a", "UNICODE_CI", array_1_2); assertStringSplitSQL("1a2", "A", "UNICODE_CI", array_1_2); assertStringSplitSQL("1a2", "1A2", "UNICODE_CI", full_match); @@ -569,8 +569,8 @@ public void testStringSplitSQL() throws SparkException { assertStringSplitSQL("AaXbB", "X", "UTF8_BINARY", array_Aa_bB); assertStringSplitSQL("AaXbB", "axb", "UNICODE", array_AaXbB); assertStringSplitSQL("AaXbB", "aXb", "UNICODE", array_A_B); - assertStringSplitSQL("AaXbB", "axb", "UTF8_BINARY_LCASE", array_A_B); - assertStringSplitSQL("AaXbB", "AXB", "UTF8_BINARY_LCASE", array_A_B); + assertStringSplitSQL("AaXbB", "axb", "UTF8_LCASE", array_A_B); + assertStringSplitSQL("AaXbB", "AXB", "UTF8_LCASE", array_A_B); assertStringSplitSQL("AaXbB", "axb", "UNICODE_CI", array_A_B); assertStringSplitSQL("AaXbB", "AxB", "UNICODE_CI", array_A_B); // Accent variation @@ -578,8 +578,8 @@ public void testStringSplitSQL() throws SparkException { assertStringSplitSQL("aBcDe", "BćD", "UTF8_BINARY", array_aBcDe); assertStringSplitSQL("aBcDe", "abćde", "UNICODE", array_aBcDe); assertStringSplitSQL("aBcDe", "aBćDe", "UNICODE", array_aBcDe); - assertStringSplitSQL("aBcDe", "bćd", "UTF8_BINARY_LCASE", array_aBcDe); - assertStringSplitSQL("aBcDe", "BĆD", "UTF8_BINARY_LCASE", array_aBcDe); + assertStringSplitSQL("aBcDe", "bćd", "UTF8_LCASE", array_aBcDe); + assertStringSplitSQL("aBcDe", "BĆD", "UTF8_LCASE", array_aBcDe); assertStringSplitSQL("aBcDe", "abćde", "UNICODE_CI", array_aBcDe); assertStringSplitSQL("aBcDe", "AbĆdE", "UNICODE_CI", array_aBcDe); // Variable byte length characters @@ -591,10 +591,10 @@ public void testStringSplitSQL() throws SparkException { assertStringSplitSQL("äb世De", "äB世de", "UNICODE", array_special); assertStringSplitSQL("äbćδe", "äbćδe", "UNICODE", full_match); assertStringSplitSQL("äbćδe", "ÄBcΔÉ", "UNICODE", array_abcde); - assertStringSplitSQL("äb世De", "b世D", "UTF8_BINARY_LCASE", array_a_e); - assertStringSplitSQL("äb世De", "B世d", "UTF8_BINARY_LCASE", array_a_e); - assertStringSplitSQL("äbćδe", "bćδ", "UTF8_BINARY_LCASE", array_a_e); - assertStringSplitSQL("äbćδe", "BcΔ", "UTF8_BINARY_LCASE", array_abcde); + assertStringSplitSQL("äb世De", "b世D", "UTF8_LCASE", array_a_e); + assertStringSplitSQL("äb世De", "B世d", "UTF8_LCASE", array_a_e); + assertStringSplitSQL("äbćδe", "bćδ", "UTF8_LCASE", array_a_e); + assertStringSplitSQL("äbćδe", "BcΔ", "UTF8_LCASE", array_abcde); assertStringSplitSQL("äb世De", "ab世De", "UNICODE_CI", array_special); assertStringSplitSQL("äb世De", "AB世dE", "UNICODE_CI", array_special); assertStringSplitSQL("äbćδe", "ÄbćδE", "UNICODE_CI", full_match); @@ -613,44 +613,44 @@ private void assertUpper(String target, String collationName, String expected) public void testUpper() throws SparkException { // Edge cases assertUpper("", "UTF8_BINARY", ""); - assertUpper("", "UTF8_BINARY_LCASE", ""); + assertUpper("", "UTF8_LCASE", ""); assertUpper("", "UNICODE", ""); assertUpper("", "UNICODE_CI", ""); // Basic tests assertUpper("abcde", "UTF8_BINARY", "ABCDE"); - assertUpper("abcde", "UTF8_BINARY_LCASE", "ABCDE"); + assertUpper("abcde", "UTF8_LCASE", "ABCDE"); assertUpper("abcde", "UNICODE", "ABCDE"); assertUpper("abcde", "UNICODE_CI", "ABCDE"); // Uppercase present assertUpper("AbCdE", "UTF8_BINARY", "ABCDE"); assertUpper("aBcDe", "UTF8_BINARY", "ABCDE"); - assertUpper("AbCdE", "UTF8_BINARY_LCASE", "ABCDE"); - assertUpper("aBcDe", "UTF8_BINARY_LCASE", "ABCDE"); + assertUpper("AbCdE", "UTF8_LCASE", "ABCDE"); + assertUpper("aBcDe", "UTF8_LCASE", "ABCDE"); assertUpper("AbCdE", "UNICODE", "ABCDE"); assertUpper("aBcDe", "UNICODE", "ABCDE"); assertUpper("AbCdE", "UNICODE_CI", "ABCDE"); assertUpper("aBcDe", "UNICODE_CI", "ABCDE"); // Accent letters assertUpper("aBćDe","UTF8_BINARY", "ABĆDE"); - assertUpper("aBćDe","UTF8_BINARY_LCASE", "ABĆDE"); + assertUpper("aBćDe","UTF8_LCASE", "ABĆDE"); assertUpper("aBćDe","UNICODE", "ABĆDE"); assertUpper("aBćDe","UNICODE_CI", "ABĆDE"); // Variable byte length characters assertUpper("ab世De", "UTF8_BINARY", "AB世DE"); assertUpper("äbćδe", "UTF8_BINARY", "ÄBĆΔE"); - assertUpper("ab世De", "UTF8_BINARY_LCASE", "AB世DE"); - assertUpper("äbćδe", "UTF8_BINARY_LCASE", "ÄBĆΔE"); + assertUpper("ab世De", "UTF8_LCASE", "AB世DE"); + assertUpper("äbćδe", "UTF8_LCASE", "ÄBĆΔE"); assertUpper("ab世De", "UNICODE", "AB世DE"); assertUpper("äbćδe", "UNICODE", "ÄBĆΔE"); assertUpper("ab世De", "UNICODE_CI", "AB世DE"); assertUpper("äbćδe", "UNICODE_CI", "ÄBĆΔE"); // Case-variable character length assertUpper("i\u0307o", "UTF8_BINARY","I\u0307O"); - assertUpper("i\u0307o", "UTF8_BINARY_LCASE","I\u0307O"); + assertUpper("i\u0307o", "UTF8_LCASE","I\u0307O"); assertUpper("i\u0307o", "UNICODE","I\u0307O"); assertUpper("i\u0307o", "UNICODE_CI","I\u0307O"); assertUpper("ß fi ffi ff st ῗ", "UTF8_BINARY","SS FI FFI FF ST \u0399\u0308\u0342"); - assertUpper("ß fi ffi ff st ῗ", "UTF8_BINARY_LCASE","SS FI FFI FF ST \u0399\u0308\u0342"); + assertUpper("ß fi ffi ff st ῗ", "UTF8_LCASE","SS FI FFI FF ST \u0399\u0308\u0342"); assertUpper("ß fi ffi ff st ῗ", "UNICODE","SS FI FFI FF ST \u0399\u0308\u0342"); assertUpper("ß fi ffi ff st ῗ", "UNICODE","SS FI FFI FF ST \u0399\u0308\u0342"); } @@ -667,40 +667,40 @@ private void assertLower(String target, String collationName, String expected) public void testLower() throws SparkException { // Edge cases assertLower("", "UTF8_BINARY", ""); - assertLower("", "UTF8_BINARY_LCASE", ""); + assertLower("", "UTF8_LCASE", ""); assertLower("", "UNICODE", ""); assertLower("", "UNICODE_CI", ""); // Basic tests assertLower("ABCDE", "UTF8_BINARY", "abcde"); - assertLower("ABCDE", "UTF8_BINARY_LCASE", "abcde"); + assertLower("ABCDE", "UTF8_LCASE", "abcde"); assertLower("ABCDE", "UNICODE", "abcde"); assertLower("ABCDE", "UNICODE_CI", "abcde"); // Uppercase present assertLower("AbCdE", "UTF8_BINARY", "abcde"); assertLower("aBcDe", "UTF8_BINARY", "abcde"); - assertLower("AbCdE", "UTF8_BINARY_LCASE", "abcde"); - assertLower("aBcDe", "UTF8_BINARY_LCASE", "abcde"); + assertLower("AbCdE", "UTF8_LCASE", "abcde"); + assertLower("aBcDe", "UTF8_LCASE", "abcde"); assertLower("AbCdE", "UNICODE", "abcde"); assertLower("aBcDe", "UNICODE", "abcde"); assertLower("AbCdE", "UNICODE_CI", "abcde"); assertLower("aBcDe", "UNICODE_CI", "abcde"); // Accent letters assertLower("AbĆdE","UTF8_BINARY", "abćde"); - assertLower("AbĆdE","UTF8_BINARY_LCASE", "abćde"); + assertLower("AbĆdE","UTF8_LCASE", "abćde"); assertLower("AbĆdE","UNICODE", "abćde"); assertLower("AbĆdE","UNICODE_CI", "abćde"); // Variable byte length characters assertLower("aB世De", "UTF8_BINARY", "ab世de"); assertLower("ÄBĆΔE", "UTF8_BINARY", "äbćδe"); - assertLower("aB世De", "UTF8_BINARY_LCASE", "ab世de"); - assertLower("ÄBĆΔE", "UTF8_BINARY_LCASE", "äbćδe"); + assertLower("aB世De", "UTF8_LCASE", "ab世de"); + assertLower("ÄBĆΔE", "UTF8_LCASE", "äbćδe"); assertLower("aB世De", "UNICODE", "ab世de"); assertLower("ÄBĆΔE", "UNICODE", "äbćδe"); assertLower("aB世De", "UNICODE_CI", "ab世de"); assertLower("ÄBĆΔE", "UNICODE_CI", "äbćδe"); // Case-variable character length assertLower("İo", "UTF8_BINARY","i\u0307o"); - assertLower("İo", "UTF8_BINARY_LCASE","i\u0307o"); + assertLower("İo", "UTF8_LCASE","i\u0307o"); assertLower("İo", "UNICODE","i\u0307o"); assertLower("İo", "UNICODE_CI","i\u0307o"); } @@ -717,75 +717,75 @@ private void assertInitCap(String target, String collationName, String expected) public void testInitCap() throws SparkException { // Edge cases assertInitCap("", "UTF8_BINARY", ""); - assertInitCap("", "UTF8_BINARY_LCASE", ""); + assertInitCap("", "UTF8_LCASE", ""); assertInitCap("", "UNICODE", ""); assertInitCap("", "UNICODE_CI", ""); // Basic tests assertInitCap("ABCDE", "UTF8_BINARY", "Abcde"); - assertInitCap("ABCDE", "UTF8_BINARY_LCASE", "Abcde"); + assertInitCap("ABCDE", "UTF8_LCASE", "Abcde"); assertInitCap("ABCDE", "UNICODE", "Abcde"); assertInitCap("ABCDE", "UNICODE_CI", "Abcde"); // Uppercase present assertInitCap("AbCdE", "UTF8_BINARY", "Abcde"); assertInitCap("aBcDe", "UTF8_BINARY", "Abcde"); - assertInitCap("AbCdE", "UTF8_BINARY_LCASE", "Abcde"); - assertInitCap("aBcDe", "UTF8_BINARY_LCASE", "Abcde"); + assertInitCap("AbCdE", "UTF8_LCASE", "Abcde"); + assertInitCap("aBcDe", "UTF8_LCASE", "Abcde"); assertInitCap("AbCdE", "UNICODE", "Abcde"); assertInitCap("aBcDe", "UNICODE", "Abcde"); assertInitCap("AbCdE", "UNICODE_CI", "Abcde"); assertInitCap("aBcDe", "UNICODE_CI", "Abcde"); // Accent letters assertInitCap("AbĆdE", "UTF8_BINARY", "Abćde"); - assertInitCap("AbĆdE", "UTF8_BINARY_LCASE", "Abćde"); + assertInitCap("AbĆdE", "UTF8_LCASE", "Abćde"); assertInitCap("AbĆdE", "UNICODE", "Abćde"); assertInitCap("AbĆdE", "UNICODE_CI", "Abćde"); // Variable byte length characters assertInitCap("aB 世 De", "UTF8_BINARY", "Ab 世 De"); assertInitCap("ÄBĆΔE", "UTF8_BINARY", "Äbćδe"); - assertInitCap("aB 世 De", "UTF8_BINARY_LCASE", "Ab 世 De"); - assertInitCap("ÄBĆΔE", "UTF8_BINARY_LCASE", "Äbćδe"); + assertInitCap("aB 世 De", "UTF8_LCASE", "Ab 世 De"); + assertInitCap("ÄBĆΔE", "UTF8_LCASE", "Äbćδe"); assertInitCap("aB 世 De", "UNICODE", "Ab 世 De"); assertInitCap("ÄBĆΔE", "UNICODE", "Äbćδe"); assertInitCap("aB 世 de", "UNICODE_CI", "Ab 世 De"); assertInitCap("ÄBĆΔE", "UNICODE_CI", "Äbćδe"); // Case-variable character length assertInitCap("İo", "UTF8_BINARY", "I\u0307o"); - assertInitCap("İo", "UTF8_BINARY_LCASE", "İo"); + assertInitCap("İo", "UTF8_LCASE", "İo"); assertInitCap("İo", "UNICODE", "İo"); assertInitCap("İo", "UNICODE_CI", "İo"); assertInitCap("i\u0307o", "UTF8_BINARY", "I\u0307o"); - assertInitCap("i\u0307o", "UTF8_BINARY_LCASE", "I\u0307o"); + assertInitCap("i\u0307o", "UTF8_LCASE", "I\u0307o"); assertInitCap("i\u0307o", "UNICODE", "I\u0307o"); assertInitCap("i\u0307o", "UNICODE_CI", "I\u0307o"); // Different possible word boundaries assertInitCap("a b c", "UTF8_BINARY", "A B C"); assertInitCap("a b c", "UNICODE", "A B C"); - assertInitCap("a b c", "UTF8_BINARY_LCASE", "A B C"); + assertInitCap("a b c", "UTF8_LCASE", "A B C"); assertInitCap("a b c", "UNICODE_CI", "A B C"); assertInitCap("a.b,c", "UTF8_BINARY", "A.b,c"); assertInitCap("a.b,c", "UNICODE", "A.b,C"); - assertInitCap("a.b,c", "UTF8_BINARY_LCASE", "A.b,C"); + assertInitCap("a.b,c", "UTF8_LCASE", "A.b,C"); assertInitCap("a.b,c", "UNICODE_CI", "A.b,C"); assertInitCap("a. b-c", "UTF8_BINARY", "A. B-c"); assertInitCap("a. b-c", "UNICODE", "A. B-C"); - assertInitCap("a. b-c", "UTF8_BINARY_LCASE", "A. B-C"); + assertInitCap("a. b-c", "UTF8_LCASE", "A. B-C"); assertInitCap("a. b-c", "UNICODE_CI", "A. B-C"); assertInitCap("a?b世c", "UTF8_BINARY", "A?b世c"); assertInitCap("a?b世c", "UNICODE", "A?B世C"); - assertInitCap("a?b世c", "UTF8_BINARY_LCASE", "A?B世C"); + assertInitCap("a?b世c", "UTF8_LCASE", "A?B世C"); assertInitCap("a?b世c", "UNICODE_CI", "A?B世C"); // Titlecase characters that are different from uppercase characters assertInitCap("dzDZDz", "UTF8_BINARY", "Dzdzdz"); assertInitCap("dzDZDz", "UNICODE", "Dzdzdz"); - assertInitCap("dzDZDz", "UTF8_BINARY_LCASE", "Dzdzdz"); + assertInitCap("dzDZDz", "UTF8_LCASE", "Dzdzdz"); assertInitCap("dzDZDz", "UNICODE_CI", "Dzdzdz"); assertInitCap("džaba Ljubav NJegova", "UTF8_BINARY", "Džaba Ljubav Njegova"); assertInitCap("džaba Ljubav NJegova", "UNICODE", "Džaba Ljubav Njegova"); - assertInitCap("džaba Ljubav NJegova", "UTF8_BINARY_LCASE", "Džaba Ljubav Njegova"); + assertInitCap("džaba Ljubav NJegova", "UTF8_LCASE", "Džaba Ljubav Njegova"); assertInitCap("džaba Ljubav NJegova", "UNICODE_CI", "Džaba Ljubav Njegova"); assertInitCap("ß fi ffi ff st ΣΗΜΕΡΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", "UTF8_BINARY", "ß fi ffi ff st Σημερινος Ασημενιος I\u0307ota"); - assertInitCap("ß fi ffi ff st ΣΗΜΕΡΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", "UTF8_BINARY_LCASE", + assertInitCap("ß fi ffi ff st ΣΗΜΕΡΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", "UTF8_LCASE", "Ss Fi Ffi Ff St Σημερινος Ασημενιος İota"); assertInitCap("ß fi ffi ff st ΣΗΜΕΡΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", "UNICODE", "Ss Fi Ffi Ff St Σημερινος Ασημενιος İota"); @@ -810,13 +810,13 @@ public void testStringInstr() throws SparkException { assertStringInstr("", "xxxx", "UTF8_BINARY", 0); assertStringInstr("test大千世界X大千世界", "大千", "UTF8_BINARY", 5); assertStringInstr("test大千世界X大千世界", "界X", "UTF8_BINARY", 8); - assertStringInstr("aaads", "Aa", "UTF8_BINARY_LCASE", 1); - assertStringInstr("aaaDs", "de", "UTF8_BINARY_LCASE", 0); - assertStringInstr("aaaDs", "ds", "UTF8_BINARY_LCASE", 4); - assertStringInstr("xxxx", "", "UTF8_BINARY_LCASE", 1); - assertStringInstr("", "xxxx", "UTF8_BINARY_LCASE", 0); - assertStringInstr("test大千世界X大千世界", "大千", "UTF8_BINARY_LCASE", 5); - assertStringInstr("test大千世界X大千世界", "界x", "UTF8_BINARY_LCASE", 8); + assertStringInstr("aaads", "Aa", "UTF8_LCASE", 1); + assertStringInstr("aaaDs", "de", "UTF8_LCASE", 0); + assertStringInstr("aaaDs", "ds", "UTF8_LCASE", 4); + assertStringInstr("xxxx", "", "UTF8_LCASE", 1); + assertStringInstr("", "xxxx", "UTF8_LCASE", 0); + assertStringInstr("test大千世界X大千世界", "大千", "UTF8_LCASE", 5); + assertStringInstr("test大千世界X大千世界", "界x", "UTF8_LCASE", 8); assertStringInstr("aaads", "Aa", "UNICODE", 0); assertStringInstr("aaads", "aa", "UNICODE", 1); assertStringInstr("aaads", "de", "UNICODE", 0); @@ -840,18 +840,18 @@ public void testStringInstr() throws SparkException { assertStringInstr("abi̇oİo", "İo", "UNICODE_CI", 3); assertStringInstr("ai̇oxXİo", "Xx", "UNICODE_CI", 5); assertStringInstr("aİoi̇oxx", "XX", "UNICODE_CI", 7); - assertStringInstr("i̇", "i", "UTF8_BINARY_LCASE", 1); // != UNICODE_CI - assertStringInstr("i̇", "\u0307", "UTF8_BINARY_LCASE", 2); // != UNICODE_CI - assertStringInstr("i̇", "İ", "UTF8_BINARY_LCASE", 1); - assertStringInstr("İ", "i", "UTF8_BINARY_LCASE", 0); - assertStringInstr("İoi̇o12", "i̇o", "UTF8_BINARY_LCASE", 1); - assertStringInstr("i̇oİo12", "İo", "UTF8_BINARY_LCASE", 1); - assertStringInstr("abİoi̇o", "i̇o", "UTF8_BINARY_LCASE", 3); - assertStringInstr("abi̇oİo", "İo", "UTF8_BINARY_LCASE", 3); - assertStringInstr("abI\u0307oi̇o", "İo", "UTF8_BINARY_LCASE", 3); - assertStringInstr("ai̇oxXİo", "Xx", "UTF8_BINARY_LCASE", 5); - assertStringInstr("abİoi̇o", "\u0307o", "UTF8_BINARY_LCASE", 6); - assertStringInstr("aİoi̇oxx", "XX", "UTF8_BINARY_LCASE", 7); + assertStringInstr("i̇", "i", "UTF8_LCASE", 1); // != UNICODE_CI + assertStringInstr("i̇", "\u0307", "UTF8_LCASE", 2); // != UNICODE_CI + assertStringInstr("i̇", "İ", "UTF8_LCASE", 1); + assertStringInstr("İ", "i", "UTF8_LCASE", 0); + assertStringInstr("İoi̇o12", "i̇o", "UTF8_LCASE", 1); + assertStringInstr("i̇oİo12", "İo", "UTF8_LCASE", 1); + assertStringInstr("abİoi̇o", "i̇o", "UTF8_LCASE", 3); + assertStringInstr("abi̇oİo", "İo", "UTF8_LCASE", 3); + assertStringInstr("abI\u0307oi̇o", "İo", "UTF8_LCASE", 3); + assertStringInstr("ai̇oxXİo", "Xx", "UTF8_LCASE", 5); + assertStringInstr("abİoi̇o", "\u0307o", "UTF8_LCASE", 6); + assertStringInstr("aİoi̇oxx", "XX", "UTF8_LCASE", 7); } private void assertFindInSet(String word, String set, String collationName, @@ -869,15 +869,15 @@ public void testFindInSet() throws SparkException { assertFindInSet("def", "abc,b,ab,c,def", "UTF8_BINARY", 5); assertFindInSet("d,ef", "abc,b,ab,c,def", "UTF8_BINARY", 0); assertFindInSet("", "abc,b,ab,c,def", "UTF8_BINARY", 0); - assertFindInSet("a", "abc,b,ab,c,def", "UTF8_BINARY_LCASE", 0); - assertFindInSet("c", "abc,b,ab,c,def", "UTF8_BINARY_LCASE", 4); - assertFindInSet("AB", "abc,b,ab,c,def", "UTF8_BINARY_LCASE", 3); - assertFindInSet("AbC", "abc,b,ab,c,def", "UTF8_BINARY_LCASE", 1); - assertFindInSet("abcd", "abc,b,ab,c,def", "UTF8_BINARY_LCASE", 0); - assertFindInSet("d,ef", "abc,b,ab,c,def", "UTF8_BINARY_LCASE", 0); - assertFindInSet("XX", "xx", "UTF8_BINARY_LCASE", 1); - assertFindInSet("", "abc,b,ab,c,def", "UTF8_BINARY_LCASE", 0); - assertFindInSet("界x", "test,大千,世,界X,大,千,世界", "UTF8_BINARY_LCASE", 4); + assertFindInSet("a", "abc,b,ab,c,def", "UTF8_LCASE", 0); + assertFindInSet("c", "abc,b,ab,c,def", "UTF8_LCASE", 4); + assertFindInSet("AB", "abc,b,ab,c,def", "UTF8_LCASE", 3); + assertFindInSet("AbC", "abc,b,ab,c,def", "UTF8_LCASE", 1); + assertFindInSet("abcd", "abc,b,ab,c,def", "UTF8_LCASE", 0); + assertFindInSet("d,ef", "abc,b,ab,c,def", "UTF8_LCASE", 0); + assertFindInSet("XX", "xx", "UTF8_LCASE", 1); + assertFindInSet("", "abc,b,ab,c,def", "UTF8_LCASE", 0); + assertFindInSet("界x", "test,大千,世,界X,大,千,世界", "UTF8_LCASE", 4); assertFindInSet("a", "abc,b,ab,c,def", "UNICODE", 0); assertFindInSet("ab", "abc,b,ab,c,def", "UNICODE", 3); assertFindInSet("Ab", "abc,b,ab,c,def", "UNICODE", 0); @@ -915,12 +915,12 @@ public void testReplace() throws SparkException { assertReplace("replace", "", "123", "UTF8_BINARY", "replace"); assertReplace("abcabc", "b", "12", "UTF8_BINARY", "a12ca12c"); assertReplace("abcdabcd", "bc", "", "UTF8_BINARY", "adad"); - assertReplace("r世eplace", "pl", "xx", "UTF8_BINARY_LCASE", "r世exxace"); - assertReplace("repl世ace", "PL", "AB", "UTF8_BINARY_LCASE", "reAB世ace"); - assertReplace("Replace", "", "123", "UTF8_BINARY_LCASE", "Replace"); - assertReplace("re世place", "世", "x", "UTF8_BINARY_LCASE", "rexplace"); - assertReplace("abcaBc", "B", "12", "UTF8_BINARY_LCASE", "a12ca12c"); - assertReplace("AbcdabCd", "Bc", "", "UTF8_BINARY_LCASE", "Adad"); + assertReplace("r世eplace", "pl", "xx", "UTF8_LCASE", "r世exxace"); + assertReplace("repl世ace", "PL", "AB", "UTF8_LCASE", "reAB世ace"); + assertReplace("Replace", "", "123", "UTF8_LCASE", "Replace"); + assertReplace("re世place", "世", "x", "UTF8_LCASE", "rexplace"); + assertReplace("abcaBc", "B", "12", "UTF8_LCASE", "a12ca12c"); + assertReplace("AbcdabCd", "Bc", "", "UTF8_LCASE", "Adad"); assertReplace("re世place", "plx", "123", "UNICODE", "re世place"); assertReplace("世Replace", "re", "", "UNICODE", "世Replace"); assertReplace("replace世", "", "123", "UNICODE", "replace世"); @@ -956,18 +956,18 @@ public void testLocate() throws SparkException { assertLocate("界x", "test大千世界X大千世界", 1, "UTF8_BINARY", 0); assertLocate("界X", "test大千世界X大千世界", 1, "UTF8_BINARY", 8); assertLocate("界", "test大千世界X大千世界", 13, "UTF8_BINARY", 13); - assertLocate("AA", "aaads", 1, "UTF8_BINARY_LCASE", 1); - assertLocate("aa", "aAads", 2, "UTF8_BINARY_LCASE", 2); - assertLocate("aa", "aaAds", 3, "UTF8_BINARY_LCASE", 0); - assertLocate("abC", "abcabc", 1, "UTF8_BINARY_LCASE", 1); - assertLocate("abC", "abCabc", 2, "UTF8_BINARY_LCASE", 4); - assertLocate("abc", "abcabc", 4, "UTF8_BINARY_LCASE", 4); - assertLocate("界x", "test大千世界X大千世界", 1, "UTF8_BINARY_LCASE", 8); - assertLocate("界X", "test大千世界Xtest大千世界", 1, "UTF8_BINARY_LCASE", 8); - assertLocate("界", "test大千世界X大千世界", 13, "UTF8_BINARY_LCASE", 13); - assertLocate("大千", "test大千世界大千世界", 1, "UTF8_BINARY_LCASE", 5); - assertLocate("大千", "test大千世界大千世界", 9, "UTF8_BINARY_LCASE", 9); - assertLocate("大千", "大千世界大千世界", 1, "UTF8_BINARY_LCASE", 1); + assertLocate("AA", "aaads", 1, "UTF8_LCASE", 1); + assertLocate("aa", "aAads", 2, "UTF8_LCASE", 2); + assertLocate("aa", "aaAds", 3, "UTF8_LCASE", 0); + assertLocate("abC", "abcabc", 1, "UTF8_LCASE", 1); + assertLocate("abC", "abCabc", 2, "UTF8_LCASE", 4); + assertLocate("abc", "abcabc", 4, "UTF8_LCASE", 4); + assertLocate("界x", "test大千世界X大千世界", 1, "UTF8_LCASE", 8); + assertLocate("界X", "test大千世界Xtest大千世界", 1, "UTF8_LCASE", 8); + assertLocate("界", "test大千世界X大千世界", 13, "UTF8_LCASE", 13); + assertLocate("大千", "test大千世界大千世界", 1, "UTF8_LCASE", 5); + assertLocate("大千", "test大千世界大千世界", 9, "UTF8_LCASE", 9); + assertLocate("大千", "大千世界大千世界", 1, "UTF8_LCASE", 1); assertLocate("aa", "Aaads", 1, "UNICODE", 2); assertLocate("AA", "aaads", 1, "UNICODE", 0); assertLocate("aa", "aAads", 2, "UNICODE", 0); @@ -993,17 +993,17 @@ public void testLocate() throws SparkException { assertLocate("大千", "大千世界大千世界", 1, "UNICODE_CI", 1); // Case-variable character length assertLocate("\u0307", "i̇", 1, "UTF8_BINARY", 2); - assertLocate("\u0307", "İ", 1, "UTF8_BINARY_LCASE", 0); // != UTF8_BINARY + assertLocate("\u0307", "İ", 1, "UTF8_LCASE", 0); // != UTF8_BINARY assertLocate("i", "i̇", 1, "UNICODE_CI", 0); assertLocate("\u0307", "i̇", 1, "UNICODE_CI", 0); assertLocate("i̇", "i", 1, "UNICODE_CI", 0); assertLocate("İ", "i̇", 1, "UNICODE_CI", 1); assertLocate("İ", "i", 1, "UNICODE_CI", 0); - assertLocate("i", "i̇", 1, "UTF8_BINARY_LCASE", 1); // != UNICODE_CI - assertLocate("\u0307", "i̇", 1, "UTF8_BINARY_LCASE", 2); // != UNICODE_CI - assertLocate("i̇", "i", 1, "UTF8_BINARY_LCASE", 0); - assertLocate("İ", "i̇", 1, "UTF8_BINARY_LCASE", 1); - assertLocate("İ", "i", 1, "UTF8_BINARY_LCASE", 0); + assertLocate("i", "i̇", 1, "UTF8_LCASE", 1); // != UNICODE_CI + assertLocate("\u0307", "i̇", 1, "UTF8_LCASE", 2); // != UNICODE_CI + assertLocate("i̇", "i", 1, "UTF8_LCASE", 0); + assertLocate("İ", "i̇", 1, "UTF8_LCASE", 1); + assertLocate("İ", "i", 1, "UTF8_LCASE", 0); assertLocate("i̇o", "İo世界大千世界", 1, "UNICODE_CI", 1); assertLocate("i̇o", "大千İo世界大千世界", 1, "UNICODE_CI", 3); assertLocate("i̇o", "世界İo大千世界大千İo", 4, "UNICODE_CI", 11); @@ -1026,25 +1026,25 @@ public void testSubstringIndex() throws SparkException { assertSubstringIndex("wwwgapachegorg", "g", -3, "UTF8_BINARY", "apachegorg"); assertSubstringIndex("www||apache||org", "||", 2, "UTF8_BINARY", "www||apache"); assertSubstringIndex("aaaaaaaaaa", "aa", 2, "UTF8_BINARY", "a"); - assertSubstringIndex("AaAaAaAaAa", "aa", 2, "UTF8_BINARY_LCASE", "A"); - assertSubstringIndex("www.apache.org", ".", 3, "UTF8_BINARY_LCASE", "www.apache.org"); - assertSubstringIndex("wwwXapacheXorg", "x", 2, "UTF8_BINARY_LCASE", "wwwXapache"); - assertSubstringIndex("wwwxapachexorg", "X", 1, "UTF8_BINARY_LCASE", "www"); - assertSubstringIndex("www.apache.org", ".", 0, "UTF8_BINARY_LCASE", ""); - assertSubstringIndex("www.apache.ORG", ".", -3, "UTF8_BINARY_LCASE", "www.apache.ORG"); - assertSubstringIndex("wwwGapacheGorg", "g", 1, "UTF8_BINARY_LCASE", "www"); - assertSubstringIndex("wwwGapacheGorg", "g", 3, "UTF8_BINARY_LCASE", "wwwGapacheGor"); - assertSubstringIndex("gwwwGapacheGorg", "g", 3, "UTF8_BINARY_LCASE", "gwwwGapache"); - assertSubstringIndex("wwwGapacheGorg", "g", -3, "UTF8_BINARY_LCASE", "apacheGorg"); - assertSubstringIndex("wwwmapacheMorg", "M", -2, "UTF8_BINARY_LCASE", "apacheMorg"); - assertSubstringIndex("www.apache.org", ".", -1, "UTF8_BINARY_LCASE", "org"); - assertSubstringIndex("www.apache.org.", ".", -1, "UTF8_BINARY_LCASE", ""); - assertSubstringIndex("", ".", -2, "UTF8_BINARY_LCASE", ""); - assertSubstringIndex("test大千世界X大千世界", "x", -1, "UTF8_BINARY_LCASE", "大千世界"); - assertSubstringIndex("test大千世界X大千世界", "X", 1, "UTF8_BINARY_LCASE", "test大千世界"); - assertSubstringIndex("test大千世界大千世界", "千", 2, "UTF8_BINARY_LCASE", "test大千世界大"); - assertSubstringIndex("www||APACHE||org", "||", 2, "UTF8_BINARY_LCASE", "www||APACHE"); - assertSubstringIndex("www||APACHE||org", "||", -1, "UTF8_BINARY_LCASE", "org"); + assertSubstringIndex("AaAaAaAaAa", "aa", 2, "UTF8_LCASE", "A"); + assertSubstringIndex("www.apache.org", ".", 3, "UTF8_LCASE", "www.apache.org"); + assertSubstringIndex("wwwXapacheXorg", "x", 2, "UTF8_LCASE", "wwwXapache"); + assertSubstringIndex("wwwxapachexorg", "X", 1, "UTF8_LCASE", "www"); + assertSubstringIndex("www.apache.org", ".", 0, "UTF8_LCASE", ""); + assertSubstringIndex("www.apache.ORG", ".", -3, "UTF8_LCASE", "www.apache.ORG"); + assertSubstringIndex("wwwGapacheGorg", "g", 1, "UTF8_LCASE", "www"); + assertSubstringIndex("wwwGapacheGorg", "g", 3, "UTF8_LCASE", "wwwGapacheGor"); + assertSubstringIndex("gwwwGapacheGorg", "g", 3, "UTF8_LCASE", "gwwwGapache"); + assertSubstringIndex("wwwGapacheGorg", "g", -3, "UTF8_LCASE", "apacheGorg"); + assertSubstringIndex("wwwmapacheMorg", "M", -2, "UTF8_LCASE", "apacheMorg"); + assertSubstringIndex("www.apache.org", ".", -1, "UTF8_LCASE", "org"); + assertSubstringIndex("www.apache.org.", ".", -1, "UTF8_LCASE", ""); + assertSubstringIndex("", ".", -2, "UTF8_LCASE", ""); + assertSubstringIndex("test大千世界X大千世界", "x", -1, "UTF8_LCASE", "大千世界"); + assertSubstringIndex("test大千世界X大千世界", "X", 1, "UTF8_LCASE", "test大千世界"); + assertSubstringIndex("test大千世界大千世界", "千", 2, "UTF8_LCASE", "test大千世界大"); + assertSubstringIndex("www||APACHE||org", "||", 2, "UTF8_LCASE", "www||APACHE"); + assertSubstringIndex("www||APACHE||org", "||", -1, "UTF8_LCASE", "org"); assertSubstringIndex("AaAaAaAaAa", "Aa", 2, "UNICODE", "Aa"); assertSubstringIndex("wwwYapacheyorg", "y", 3, "UNICODE", "wwwYapacheyorg"); assertSubstringIndex("www.apache.org", ".", 2, "UNICODE", "www.apache"); @@ -1105,20 +1105,20 @@ public void testSubstringIndex() throws SparkException { assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i̇o", 3, "UNICODE_CI", "ai̇bi̇oİo12"); assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", 3, "UNICODE_CI", "ai̇bİoi̇o12"); assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i̇o", 3, "UNICODE_CI", "ai̇bİoi̇o12"); - assertSubstringIndex("abi̇12", "i", 1, "UTF8_BINARY_LCASE", "ab"); // != UNICODE_CI - assertSubstringIndex("abi̇12", "\u0307", 1, "UTF8_BINARY_LCASE", "abi"); // != UNICODE_CI - assertSubstringIndex("abi̇12", "İ", 1, "UTF8_BINARY_LCASE", "ab"); - assertSubstringIndex("abİ12", "i", 1, "UTF8_BINARY_LCASE", "abİ12"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", -4, "UTF8_BINARY_LCASE", "İo12İoi̇o"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i̇o", -4, "UTF8_BINARY_LCASE", "İo12İoi̇o"); - assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", -4, "UTF8_BINARY_LCASE", "i̇o12i̇oİo"); - assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i̇o", -4, "UTF8_BINARY_LCASE", "i̇o12i̇oİo"); - assertSubstringIndex("bİoi̇o12i̇o", "\u0307oi", 1, "UTF8_BINARY_LCASE", "bİoi̇o12i̇o"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", 3, "UTF8_BINARY_LCASE", "ai̇bi̇oİo12"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i̇o", 3, "UTF8_BINARY_LCASE", "ai̇bi̇oİo12"); - assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", 3, "UTF8_BINARY_LCASE", "ai̇bİoi̇o12"); - assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i̇o", 3, "UTF8_BINARY_LCASE", "ai̇bİoi̇o12"); - assertSubstringIndex("bİoi̇o12i̇o", "\u0307oi", 1, "UTF8_BINARY_LCASE", "bİoi̇o12i̇o"); + assertSubstringIndex("abi̇12", "i", 1, "UTF8_LCASE", "ab"); // != UNICODE_CI + assertSubstringIndex("abi̇12", "\u0307", 1, "UTF8_LCASE", "abi"); // != UNICODE_CI + assertSubstringIndex("abi̇12", "İ", 1, "UTF8_LCASE", "ab"); + assertSubstringIndex("abİ12", "i", 1, "UTF8_LCASE", "abİ12"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", -4, "UTF8_LCASE", "İo12İoi̇o"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i̇o", -4, "UTF8_LCASE", "İo12İoi̇o"); + assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", -4, "UTF8_LCASE", "i̇o12i̇oİo"); + assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i̇o", -4, "UTF8_LCASE", "i̇o12i̇oİo"); + assertSubstringIndex("bİoi̇o12i̇o", "\u0307oi", 1, "UTF8_LCASE", "bİoi̇o12i̇o"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", 3, "UTF8_LCASE", "ai̇bi̇oİo12"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i̇o", 3, "UTF8_LCASE", "ai̇bi̇oİo12"); + assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", 3, "UTF8_LCASE", "ai̇bİoi̇o12"); + assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i̇o", 3, "UTF8_LCASE", "ai̇bİoi̇o12"); + assertSubstringIndex("bİoi̇o12i̇o", "\u0307oi", 1, "UTF8_LCASE", "bİoi̇o12i̇o"); } private void assertStringTrim( @@ -1207,77 +1207,77 @@ public void testStringTrim() throws SparkException { assertStringTrimRight("UTF8_BINARY", "xxasdxx", "x", "xxasd"); assertStringTrimRight("UTF8_BINARY", "xa世ax", "x", "xa世a"); - assertStringTrim("UTF8_BINARY_LCASE", "asd", null, "asd"); - assertStringTrim("UTF8_BINARY_LCASE", " asd ", null, "asd"); - assertStringTrim("UTF8_BINARY_LCASE", " a世a ", null, "a世a"); - assertStringTrim("UTF8_BINARY_LCASE", "asd", "x", "asd"); - assertStringTrim("UTF8_BINARY_LCASE", "xxasdxx", "x", "asd"); - assertStringTrim("UTF8_BINARY_LCASE", "xa世ax", "x", "a世a"); - - assertStringTrimLeft("UTF8_BINARY_LCASE", "asd", null, "asd"); - assertStringTrimLeft("UTF8_BINARY_LCASE", " asd ", null, "asd "); - assertStringTrimLeft("UTF8_BINARY_LCASE", " a世a ", null, "a世a "); - assertStringTrimLeft("UTF8_BINARY_LCASE", "asd", "x", "asd"); - assertStringTrimLeft("UTF8_BINARY_LCASE", "xxasdxx", "x", "asdxx"); - assertStringTrimLeft("UTF8_BINARY_LCASE", "xa世ax", "x", "a世ax"); - - assertStringTrimRight("UTF8_BINARY_LCASE", "asd", null, "asd"); - assertStringTrimRight("UTF8_BINARY_LCASE", " asd ", null, " asd"); - assertStringTrimRight("UTF8_BINARY_LCASE", " a世a ", null, " a世a"); - assertStringTrimRight("UTF8_BINARY_LCASE", "asd", "x", "asd"); - assertStringTrimRight("UTF8_BINARY_LCASE", "xxasdxx", "x", "xxasd"); - assertStringTrimRight("UTF8_BINARY_LCASE", "xa世ax", "x", "xa世a"); - - assertStringTrim("UTF8_BINARY_LCASE", "asd", null, "asd"); - assertStringTrim("UTF8_BINARY_LCASE", " asd ", null, "asd"); - assertStringTrim("UTF8_BINARY_LCASE", " a世a ", null, "a世a"); - assertStringTrim("UTF8_BINARY_LCASE", "asd", "x", "asd"); - assertStringTrim("UTF8_BINARY_LCASE", "xxasdxx", "x", "asd"); - assertStringTrim("UTF8_BINARY_LCASE", "xa世ax", "x", "a世a"); + assertStringTrim("UTF8_LCASE", "asd", null, "asd"); + assertStringTrim("UTF8_LCASE", " asd ", null, "asd"); + assertStringTrim("UTF8_LCASE", " a世a ", null, "a世a"); + assertStringTrim("UTF8_LCASE", "asd", "x", "asd"); + assertStringTrim("UTF8_LCASE", "xxasdxx", "x", "asd"); + assertStringTrim("UTF8_LCASE", "xa世ax", "x", "a世a"); + + assertStringTrimLeft("UTF8_LCASE", "asd", null, "asd"); + assertStringTrimLeft("UTF8_LCASE", " asd ", null, "asd "); + assertStringTrimLeft("UTF8_LCASE", " a世a ", null, "a世a "); + assertStringTrimLeft("UTF8_LCASE", "asd", "x", "asd"); + assertStringTrimLeft("UTF8_LCASE", "xxasdxx", "x", "asdxx"); + assertStringTrimLeft("UTF8_LCASE", "xa世ax", "x", "a世ax"); + + assertStringTrimRight("UTF8_LCASE", "asd", null, "asd"); + assertStringTrimRight("UTF8_LCASE", " asd ", null, " asd"); + assertStringTrimRight("UTF8_LCASE", " a世a ", null, " a世a"); + assertStringTrimRight("UTF8_LCASE", "asd", "x", "asd"); + assertStringTrimRight("UTF8_LCASE", "xxasdxx", "x", "xxasd"); + assertStringTrimRight("UTF8_LCASE", "xa世ax", "x", "xa世a"); + + assertStringTrim("UTF8_LCASE", "asd", null, "asd"); + assertStringTrim("UTF8_LCASE", " asd ", null, "asd"); + assertStringTrim("UTF8_LCASE", " a世a ", null, "a世a"); + assertStringTrim("UTF8_LCASE", "asd", "x", "asd"); + assertStringTrim("UTF8_LCASE", "xxasdxx", "x", "asd"); + assertStringTrim("UTF8_LCASE", "xa世ax", "x", "a世a"); // Test cases where trimString has more than one character assertStringTrim("UTF8_BINARY", "ddsXXXaa", "asd", "XXX"); assertStringTrimLeft("UTF8_BINARY", "ddsXXXaa", "asd", "XXXaa"); assertStringTrimRight("UTF8_BINARY", "ddsXXXaa", "asd", "ddsXXX"); - assertStringTrim("UTF8_BINARY_LCASE", "ddsXXXaa", "asd", "XXX"); - assertStringTrimLeft("UTF8_BINARY_LCASE", "ddsXXXaa", "asd", "XXXaa"); - assertStringTrimRight("UTF8_BINARY_LCASE", "ddsXXXaa", "asd", "ddsXXX"); + assertStringTrim("UTF8_LCASE", "ddsXXXaa", "asd", "XXX"); + assertStringTrimLeft("UTF8_LCASE", "ddsXXXaa", "asd", "XXXaa"); + assertStringTrimRight("UTF8_LCASE", "ddsXXXaa", "asd", "ddsXXX"); // Test cases specific to collation type // uppercase trim, lowercase src assertStringTrim("UTF8_BINARY", "asd", "A", "asd"); - assertStringTrim("UTF8_BINARY_LCASE", "asd", "A", "sd"); + assertStringTrim("UTF8_LCASE", "asd", "A", "sd"); // lowercase trim, uppercase src assertStringTrim("UTF8_BINARY", "ASD", "a", "ASD"); - assertStringTrim("UTF8_BINARY_LCASE", "ASD", "a", "SD"); + assertStringTrim("UTF8_LCASE", "ASD", "a", "SD"); // uppercase and lowercase chars of different byte-length (utf8) assertStringTrim("UTF8_BINARY", "ẞaaaẞ", "ß", "ẞaaaẞ"); assertStringTrimLeft("UTF8_BINARY", "ẞaaaẞ", "ß", "ẞaaaẞ"); assertStringTrimRight("UTF8_BINARY", "ẞaaaẞ", "ß", "ẞaaaẞ"); - assertStringTrim("UTF8_BINARY_LCASE", "ẞaaaẞ", "ß", "aaa"); - assertStringTrimLeft("UTF8_BINARY_LCASE", "ẞaaaẞ", "ß", "aaaẞ"); - assertStringTrimRight("UTF8_BINARY_LCASE", "ẞaaaẞ", "ß", "ẞaaa"); + assertStringTrim("UTF8_LCASE", "ẞaaaẞ", "ß", "aaa"); + assertStringTrimLeft("UTF8_LCASE", "ẞaaaẞ", "ß", "aaaẞ"); + assertStringTrimRight("UTF8_LCASE", "ẞaaaẞ", "ß", "ẞaaa"); assertStringTrim("UTF8_BINARY", "ßaaaß", "ẞ", "ßaaaß"); assertStringTrimLeft("UTF8_BINARY", "ßaaaß", "ẞ", "ßaaaß"); assertStringTrimRight("UTF8_BINARY", "ßaaaß", "ẞ", "ßaaaß"); - assertStringTrim("UTF8_BINARY_LCASE", "ßaaaß", "ẞ", "aaa"); - assertStringTrimLeft("UTF8_BINARY_LCASE", "ßaaaß", "ẞ", "aaaß"); - assertStringTrimRight("UTF8_BINARY_LCASE", "ßaaaß", "ẞ", "ßaaa"); + assertStringTrim("UTF8_LCASE", "ßaaaß", "ẞ", "aaa"); + assertStringTrimLeft("UTF8_LCASE", "ßaaaß", "ẞ", "aaaß"); + assertStringTrimRight("UTF8_LCASE", "ßaaaß", "ẞ", "ßaaa"); // different byte-length (utf8) chars trimmed assertStringTrim("UTF8_BINARY", "Ëaaaẞ", "Ëẞ", "aaa"); assertStringTrimLeft("UTF8_BINARY", "Ëaaaẞ", "Ëẞ", "aaaẞ"); assertStringTrimRight("UTF8_BINARY", "Ëaaaẞ", "Ëẞ", "Ëaaa"); - assertStringTrim("UTF8_BINARY_LCASE", "Ëaaaẞ", "Ëẞ", "aaa"); - assertStringTrimLeft("UTF8_BINARY_LCASE", "Ëaaaẞ", "Ëẞ", "aaaẞ"); - assertStringTrimRight("UTF8_BINARY_LCASE", "Ëaaaẞ", "Ëẞ", "Ëaaa"); + assertStringTrim("UTF8_LCASE", "Ëaaaẞ", "Ëẞ", "aaa"); + assertStringTrimLeft("UTF8_LCASE", "Ëaaaẞ", "Ëẞ", "aaaẞ"); + assertStringTrimRight("UTF8_LCASE", "Ëaaaẞ", "Ëẞ", "Ëaaa"); } // TODO: Test more collation-aware string expressions. 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 35a40ba9f398c..c539e859b5509 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 @@ -40,9 +40,9 @@ class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ig assert(utf8Binary.collationName == "UTF8_BINARY") assert(utf8Binary.supportsBinaryEquality) - assert(UTF8_BINARY_LCASE_COLLATION_ID == 1) - val utf8BinaryLcase = fetchCollation(UTF8_BINARY_LCASE_COLLATION_ID) - assert(utf8BinaryLcase.collationName == "UTF8_BINARY_LCASE") + assert(UTF8_LCASE_COLLATION_ID == 1) + val utf8BinaryLcase = fetchCollation(UTF8_LCASE_COLLATION_ID) + assert(utf8BinaryLcase.collationName == "UTF8_LCASE") assert(!utf8BinaryLcase.supportsBinaryEquality) assert(UNICODE_COLLATION_ID == (1 << 29)) @@ -60,7 +60,7 @@ class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ig // Collation name already normalized. Seq( "UTF8_BINARY", - "UTF8_BINARY_LCASE", + "UTF8_LCASE", "UNICODE", "UNICODE_CI", "UNICODE_AI", @@ -79,7 +79,7 @@ class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ig ("UNICODE_AI_CI", "UNICODE_CI_AI"), // Randomized case collation names. ("utf8_binary", "UTF8_BINARY"), - ("UtF8_binARy_LcasE", "UTF8_BINARY_LCASE"), + ("UtF8_LcasE", "UTF8_LCASE"), ("unicode", "UNICODE"), ("UnICoDe_cs_aI", "UNICODE_AI") ).foreach{ @@ -132,13 +132,13 @@ 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_LCASE", "aaa", "aaa", true), - CollationTestCase("UTF8_BINARY_LCASE", "aaa", "AAA", true), - CollationTestCase("UTF8_BINARY_LCASE", "aaa", "AaA", true), - CollationTestCase("UTF8_BINARY_LCASE", "aaa", "AaA", true), - CollationTestCase("UTF8_BINARY_LCASE", "aaa", "aa", false), - CollationTestCase("UTF8_BINARY_LCASE", "aaa", "bbb", false), - CollationTestCase("UTF8_BINARY_LCASE", "å", "a\u030A", false), + CollationTestCase("UTF8_LCASE", "aaa", "aaa", true), + CollationTestCase("UTF8_LCASE", "aaa", "AAA", true), + CollationTestCase("UTF8_LCASE", "aaa", "AaA", true), + CollationTestCase("UTF8_LCASE", "aaa", "AaA", true), + CollationTestCase("UTF8_LCASE", "aaa", "aa", false), + CollationTestCase("UTF8_LCASE", "aaa", "bbb", false), + CollationTestCase("UTF8_LCASE", "å", "a\u030A", false), CollationTestCase("UNICODE", "aaa", "aaa", true), CollationTestCase("UNICODE", "aaa", "AAA", false), CollationTestCase("UNICODE", "aaa", "bbb", false), @@ -167,12 +167,12 @@ 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_LCASE", "aaa", "aaa", 0), - CollationTestCase("UTF8_BINARY_LCASE", "aaa", "AAA", 0), - CollationTestCase("UTF8_BINARY_LCASE", "aaa", "AaA", 0), - CollationTestCase("UTF8_BINARY_LCASE", "aaa", "AaA", 0), - CollationTestCase("UTF8_BINARY_LCASE", "aaa", "aa", 1), - CollationTestCase("UTF8_BINARY_LCASE", "aaa", "bbb", -1), + 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("UNICODE", "aaa", "aaa", 0), CollationTestCase("UNICODE", "aaa", "AAA", -1), CollationTestCase("UNICODE", "aaa", "bbb", -1), @@ -229,7 +229,7 @@ class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ig test("test collation caching") { Seq( "UTF8_BINARY", - "UTF8_BINARY_LCASE", + "UTF8_LCASE", "UNICODE", "UNICODE_CI", "UNICODE_AI", @@ -409,7 +409,7 @@ class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ig test("repeated and/or incompatible specifiers in collation name") { Seq( - "UTF8_BINARY_LCASE_LCASE", + "UTF8_LCASE_LCASE", "UNICODE_CS_CS", "UNICODE_CI_CI", "UNICODE_CI_CS", diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala index e0ad8f7078caf..aee287f4bbb31 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala @@ -700,8 +700,8 @@ class PlanGenerationTestSuite } test("select collated string") { - val schema = StructType( - StructField("s", StringType(CollationFactory.UTF8_BINARY_LCASE_COLLATION_ID)) :: Nil) + val schema = + StructType(StructField("s", StringType(CollationFactory.UTF8_LCASE_COLLATION_ID)) :: Nil) createLocalRelation(schema.catalogString).select("s") } diff --git a/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json b/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json index 86595d46654c0..2ccad0345af62 100644 --- a/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json +++ b/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.json @@ -8,7 +8,7 @@ "planId": "0" }, "localRelation": { - "schema": "struct\u003cs:string collate UTF8_BINARY_LCASE\u003e" + "schema": "struct\u003cs:string collate UTF8_LCASE\u003e" } }, "expressions": [{ diff --git a/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.proto.bin b/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.proto.bin index 30d816526ccea..3708878a2de2d 100644 Binary files a/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.proto.bin and b/connector/connect/common/src/test/resources/query-tests/queries/select_collated_string.proto.bin differ diff --git a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala index 632e2308fc76b..7e862bcfc533f 100644 --- a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala +++ b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala @@ -1036,7 +1036,7 @@ class SparkConnectProtoSuite extends PlanTest with SparkConnectPlanTest { } test("SPARK-47144: Collated string") { - Seq("UTF8_BINARY", "UTF8_BINARY_LCASE", "UNICODE", "UNICODE_CI").map(collationName => + Seq("UTF8_BINARY", "UTF8_LCASE", "UNICODE", "UNICODE_CI").map(collationName => Seq( s"select 'abc' collate $collationName", s"select collation('abc' collate $collationName)").map(query => diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index 1882c1fd1f6ad..13c64b4bdc286 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -1840,7 +1840,7 @@ def test_repr(self): NullType(), StringType(), StringType("UTF8_BINARY"), - StringType("UTF8_BINARY_LCASE"), + StringType("UTF8_LCASE"), StringType("UNICODE"), StringType("UNICODE_CI"), CharType(10), @@ -2217,18 +2217,18 @@ def test_from_ddl(self): def test_collated_string(self): dfs = [ - self.spark.sql("SELECT 'abc' collate UTF8_BINARY_LCASE"), + self.spark.sql("SELECT 'abc' collate UTF8_LCASE"), self.spark.createDataFrame( - [], StructType([StructField("id", StringType("UTF8_BINARY_LCASE"))]) + [], StructType([StructField("id", StringType("UTF8_LCASE"))]) ), ] for df in dfs: # performs both datatype -> proto & proto -> datatype conversions self.assertEqual( - df.to(StructType([StructField("new", StringType("UTF8_BINARY_LCASE"))])) + df.to(StructType([StructField("new", StringType("UTF8_LCASE"))])) .schema[0] .dataType, - StringType("UTF8_BINARY_LCASE"), + StringType("UTF8_LCASE"), ) def test_infer_array_element_type_with_struct(self): @@ -2378,7 +2378,7 @@ def __init__(self, **kwargs): (1.0, StringType()), ([], StringType()), ({}, StringType()), - ("", StringType("UTF8_BINARY_LCASE")), + ("", StringType("UTF8_LCASE")), # Char ("", CharType(10)), (1, CharType(10)), @@ -2447,7 +2447,7 @@ def __init__(self, **kwargs): failure_spec = [ # String (match anything but None) (None, StringType(), ValueError), - (None, StringType("UTF8_BINARY_LCASE"), ValueError), + (None, StringType("UTF8_LCASE"), ValueError), # CharType (match anything but None) (None, CharType(10), ValueError), # VarcharType (match anything but None) 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 b8dadbc9e1dca..6ec55db008c75 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 @@ -43,7 +43,7 @@ class StringType private(val collationId: Int) extends AtomicType with Serializa collationId == CollationFactory.UTF8_BINARY_COLLATION_ID def isUTF8BinaryLcaseCollation: Boolean = - collationId == CollationFactory.UTF8_BINARY_LCASE_COLLATION_ID + collationId == CollationFactory.UTF8_LCASE_COLLATION_ID /** * Support for Binary Ordering implies that strings are considered equal only diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala index 7c02475a60adb..c528b523c5e7f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala @@ -38,8 +38,8 @@ import org.apache.spark.sql.types._ Examples: > SET spark.sql.collation.enabled=true; spark.sql.collation.enabled true - > SELECT COLLATION('Spark SQL' _FUNC_ UTF8_BINARY_LCASE); - UTF8_BINARY_LCASE + > SELECT COLLATION('Spark SQL' _FUNC_ UTF8_LCASE); + UTF8_LCASE > SET spark.sql.collation.enabled=false; spark.sql.collation.enabled false """, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationExpressionSuite.scala index cf6f29f9df097..d20dab8eb8ac2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationExpressionSuite.scala @@ -33,8 +33,8 @@ class CollationExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { } test("collate against literal") { - val collateExpr = Collate(Literal("abc"), "UTF8_BINARY_LCASE") - val collationId = CollationFactory.collationNameToId("UTF8_BINARY_LCASE") + val collateExpr = Collate(Literal("abc"), "UTF8_LCASE") + val collationId = CollationFactory.collationNameToId("UTF8_LCASE") assert(collateExpr.dataType == StringType(collationId)) checkEvaluation(collateExpr, "abc") } @@ -73,11 +73,11 @@ class CollationExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { test("collation on explicitly collated string") { checkEvaluation( Collation(Literal.create("abc", - StringType(CollationFactory.UTF8_BINARY_LCASE_COLLATION_ID))).replacement, - "UTF8_BINARY_LCASE") + StringType(CollationFactory.UTF8_LCASE_COLLATION_ID))).replacement, + "UTF8_LCASE") checkEvaluation( - Collation(Collate(Literal("abc"), "UTF8_BINARY_LCASE")).replacement, - "UTF8_BINARY_LCASE") + Collation(Collate(Literal("abc"), "UTF8_LCASE")).replacement, + "UTF8_LCASE") } test("Array operations on arrays of collated strings") { @@ -92,8 +92,8 @@ class CollationExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { (Seq("a"), Seq("a"), true, "UTF8_BINARY"), (Seq("a"), Seq("b"), false, "UTF8_BINARY"), (Seq("a"), Seq("A"), false, "UTF8_BINARY"), - (Seq("a"), Seq("A"), true, "UTF8_BINARY_LCASE"), - (Seq("a", "B"), Seq("A", "b"), true, "UTF8_BINARY_LCASE"), + (Seq("a"), Seq("A"), true, "UTF8_LCASE"), + (Seq("a", "B"), Seq("A", "b"), true, "UTF8_LCASE"), (Seq("a"), Seq("A"), false, "UNICODE"), (Seq("a", "B"), Seq("A", "b"), true, "UNICODE_CI") ) @@ -108,8 +108,8 @@ class CollationExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { (Seq("a", "b", "c"), Seq("a", "b", "c"), "UTF8_BINARY"), (Seq("a", "a", "a"), Seq("a"), "UTF8_BINARY"), (Seq("aaa", "AAA", "Aaa", "aAa"), Seq("aaa", "AAA", "Aaa", "aAa"), "UTF8_BINARY"), - (Seq("aaa", "AAA", "Aaa", "aAa"), Seq("aaa"), "UTF8_BINARY_LCASE"), - (Seq("aaa", "AAA", "Aaa", "aAa", "b"), Seq("aaa", "b"), "UTF8_BINARY_LCASE"), + (Seq("aaa", "AAA", "Aaa", "aAa"), Seq("aaa"), "UTF8_LCASE"), + (Seq("aaa", "AAA", "Aaa", "aAa", "b"), Seq("aaa", "b"), "UTF8_LCASE"), (Seq("aaa", "AAA", "Aaa", "aAa"), Seq("aaa"), "UNICODE_CI") ) for ((in, out, collName) <- distinct) @@ -120,8 +120,8 @@ class CollationExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { (Seq("a"), Seq("a"), Seq("a"), "UTF8_BINARY"), (Seq("a"), Seq("b"), Seq("a", "b"), "UTF8_BINARY"), (Seq("a"), Seq("A"), Seq("a", "A"), "UTF8_BINARY"), - (Seq("a"), Seq("A"), Seq("a"), "UTF8_BINARY_LCASE"), - (Seq("a", "B"), Seq("A", "b"), Seq("a", "B"), "UTF8_BINARY_LCASE"), + (Seq("a"), Seq("A"), Seq("a"), "UTF8_LCASE"), + (Seq("a", "B"), Seq("A", "b"), Seq("a", "B"), "UTF8_LCASE"), (Seq("a"), Seq("A"), Seq("a", "A"), "UNICODE"), (Seq("a", "B"), Seq("A", "b"), Seq("a", "B"), "UNICODE_CI") ) @@ -136,8 +136,8 @@ class CollationExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { (Seq("a"), Seq("a"), Seq("a"), "UTF8_BINARY"), (Seq("a"), Seq("b"), Seq.empty, "UTF8_BINARY"), (Seq("a"), Seq("A"), Seq.empty, "UTF8_BINARY"), - (Seq("a"), Seq("A"), Seq("a"), "UTF8_BINARY_LCASE"), - (Seq("a", "B"), Seq("A", "b"), Seq("a", "B"), "UTF8_BINARY_LCASE"), + (Seq("a"), Seq("A"), Seq("a"), "UTF8_LCASE"), + (Seq("a", "B"), Seq("A", "b"), Seq("a", "B"), "UTF8_LCASE"), (Seq("a"), Seq("A"), Seq.empty, "UNICODE"), (Seq("a", "B"), Seq("A", "b"), Seq("a", "B"), "UNICODE_CI") ) @@ -152,8 +152,8 @@ class CollationExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { (Seq("a"), Seq("a"), Seq.empty, "UTF8_BINARY"), (Seq("a"), Seq("b"), Seq("a"), "UTF8_BINARY"), (Seq("a"), Seq("A"), Seq("a"), "UTF8_BINARY"), - (Seq("a"), Seq("A"), Seq.empty, "UTF8_BINARY_LCASE"), - (Seq("a", "B"), Seq("A", "b"), Seq.empty, "UTF8_BINARY_LCASE"), + (Seq("a"), Seq("A"), Seq.empty, "UTF8_LCASE"), + (Seq("a", "B"), Seq("A", "b"), Seq.empty, "UTF8_LCASE"), (Seq("a"), Seq("A"), Seq("a"), "UNICODE"), (Seq("a", "B"), Seq("A", "b"), Seq.empty, "UNICODE_CI") ) @@ -170,10 +170,10 @@ class CollationExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { ("aa", "UTF8_BINARY", UTF8String.fromString("aa").getBytes), ("AA", "UTF8_BINARY", UTF8String.fromString("AA").getBytes), ("aA", "UTF8_BINARY", UTF8String.fromString("aA").getBytes), - ("", "UTF8_BINARY_LCASE", UTF8String.fromString("").getBytes), - ("aa", "UTF8_BINARY_LCASE", UTF8String.fromString("aa").getBytes), - ("AA", "UTF8_BINARY_LCASE", UTF8String.fromString("aa").getBytes), - ("aA", "UTF8_BINARY_LCASE", UTF8String.fromString("aa").getBytes), + ("", "UTF8_LCASE", UTF8String.fromString("").getBytes), + ("aa", "UTF8_LCASE", UTF8String.fromString("aa").getBytes), + ("AA", "UTF8_LCASE", UTF8String.fromString("aa").getBytes), + ("aA", "UTF8_LCASE", UTF8String.fromString("aa").getBytes), ("", "UNICODE", Array[Byte](1, 1, 0)), ("aa", "UNICODE", Array[Byte](42, 42, 1, 6, 1, 6, 0)), ("AA", "UNICODE", Array[Byte](42, 42, 1, 6, 1, -36, -36, 0)), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationRegexpExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationRegexpExpressionsSuite.scala index 4085022e7ab8c..6f0d0c13b32a3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationRegexpExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollationRegexpExpressionsSuite.scala @@ -30,10 +30,10 @@ class CollationRegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalH val testCases = Seq( LikeTestCase("AbC", "%AbC%", ".b.", "UTF8_BINARY", true, true, true), LikeTestCase("AbC", "%ABC%", ".B.", "UTF8_BINARY", false, true, false), - LikeTestCase("AbC", "%abc%", ".b.", "UTF8_BINARY_LCASE", true, true, true), - LikeTestCase("", "", "", "UTF8_BINARY_LCASE", true, true, true), - LikeTestCase("Foo", "", "", "UTF8_BINARY_LCASE", false, false, true), - LikeTestCase("", "%foo%", ".o.", "UTF8_BINARY_LCASE", false, false, false), + LikeTestCase("AbC", "%abc%", ".b.", "UTF8_LCASE", true, true, true), + LikeTestCase("", "", "", "UTF8_LCASE", true, true, true), + LikeTestCase("Foo", "", "", "UTF8_LCASE", false, false, true), + LikeTestCase("", "%foo%", ".o.", "UTF8_LCASE", false, false, false), LikeTestCase("AbC", "%ABC%", ".B.", "UTF8_BINARY", false, true, false), LikeTestCase(null, "%foo%", ".o.", "UTF8_BINARY", null, null, null), LikeTestCase("Foo", null, null, "UTF8_BINARY", null, null, null), @@ -60,8 +60,8 @@ class CollationRegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalH val testCases = Seq( StringSplitTestCase("1A2B3C", "[ABC]", "UTF8_BINARY", Seq("1", "2", "3", "")), StringSplitTestCase("1A2B3C", "[abc]", "UTF8_BINARY", Seq("1A2B3C")), - StringSplitTestCase("1A2B3C", "[ABC]", "UTF8_BINARY_LCASE", Seq("1", "2", "3", "")), - StringSplitTestCase("1A2B3C", "[abc]", "UTF8_BINARY_LCASE", Seq("1", "2", "3", "")), + StringSplitTestCase("1A2B3C", "[ABC]", "UTF8_LCASE", Seq("1", "2", "3", "")), + StringSplitTestCase("1A2B3C", "[abc]", "UTF8_LCASE", Seq("1", "2", "3", "")), StringSplitTestCase("1A2B3C", "[1-9]+", "UTF8_BINARY", Seq("", "A", "B", "C")), StringSplitTestCase("", "", "UTF8_BINARY", Seq("")), StringSplitTestCase("1A2B3C", "", "UTF8_BINARY", Seq("1", "A", "2", "B", "3", "C")), @@ -84,11 +84,11 @@ class CollationRegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalH val testCases = Seq( RegexpTestCase("AbC-aBc", ".b.", "UTF8_BINARY", "AbC", Seq("AbC"), 1), RegexpTestCase("AbC-abc", ".b.", "UTF8_BINARY", "AbC", Seq("AbC", "abc"), 2), - RegexpTestCase("AbC-aBc", ".b.", "UTF8_BINARY_LCASE", "AbC", Seq("AbC", "aBc"), 2), - RegexpTestCase("ABC-abc", ".b.", "UTF8_BINARY_LCASE", "ABC", Seq("ABC", "abc"), 2), - RegexpTestCase("", "", "UTF8_BINARY_LCASE", "", Seq(""), 1), - RegexpTestCase("Foo", "", "UTF8_BINARY_LCASE", "", Seq("", "", "", ""), 4), - RegexpTestCase("", ".o.", "UTF8_BINARY_LCASE", "", Seq(), 0), + RegexpTestCase("AbC-aBc", ".b.", "UTF8_LCASE", "AbC", Seq("AbC", "aBc"), 2), + RegexpTestCase("ABC-abc", ".b.", "UTF8_LCASE", "ABC", Seq("ABC", "abc"), 2), + RegexpTestCase("", "", "UTF8_LCASE", "", Seq(""), 1), + RegexpTestCase("Foo", "", "UTF8_LCASE", "", Seq("", "", "", ""), 4), + RegexpTestCase("", ".o.", "UTF8_LCASE", "", Seq(), 0), RegexpTestCase("Foo", ".O.", "UTF8_BINARY", "", Seq(), 0), RegexpTestCase(null, ".O.", "UTF8_BINARY", null, null, null), RegexpTestCase("Foo", null, "UTF8_BINARY", null, null, null), @@ -123,7 +123,7 @@ class CollationRegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalH val nullStr = Literal.create(null, StringType) // Supported collations (StringTypeBinaryLcase) val binaryCollation = StringType(CollationFactory.collationNameToId("UTF8_BINARY")) - val lowercaseCollation = StringType(CollationFactory.collationNameToId("UTF8_BINARY_LCASE")) + val lowercaseCollation = StringType(CollationFactory.collationNameToId("UTF8_LCASE")) // LikeAll checkEvaluation(Literal.create("foo", binaryCollation).likeAll("%foo%", "%oo"), true) checkEvaluation(Literal.create("foo", binaryCollation).likeAll("%foo%", "%bar%"), false) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala index b640344658d40..b3444b0b43077 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/MergeScalarSubqueriesSuite.scala @@ -38,7 +38,7 @@ class MergeScalarSubqueriesSuite extends PlanTest { val testRelation = LocalRelation(Symbol("a").int, Symbol("b").int, Symbol("c").string) val testRelationWithNonBinaryCollation = LocalRelation( Symbol("utf8_binary").string("UTF8_BINARY"), - Symbol("utf8_binary_lcase").string("UTF8_BINARY_LCASE")) + Symbol("utf8_lcase").string("UTF8_LCASE")) private def definitionNode(plan: LogicalPlan, cteIndex: Int) = { CTERelationDef(plan, cteIndex, underSubquery = true) @@ -204,7 +204,7 @@ class MergeScalarSubqueriesSuite extends PlanTest { val subquery1 = ScalarSubquery(testRelationWithNonBinaryCollation.groupBy( Symbol("utf8_binary"))(max(Symbol("utf8_binary")).as("max_utf8_binary"))) val subquery2 = ScalarSubquery(testRelationWithNonBinaryCollation.groupBy( - Symbol("utf8_binary_lcase"))(max(Symbol("utf8_binary_lcase")).as("utf8_binary_lcase"))) + Symbol("utf8_lcase"))(max(Symbol("utf8_lcase")).as("utf8_lcase"))) val originalQuery = testRelationWithNonBinaryCollation.select(subquery1, subquery2) Optimize.execute(originalQuery.analyze).collect { case WithCTE(_, _) => fail("Should not have merged") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtilsSuite.scala index b6e87c456de0c..0b3f1f1bdb79d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtilsSuite.scala @@ -93,7 +93,7 @@ class UnsafeRowUtilsSuite extends SparkFunSuite { } test("isBinaryStable on complex types containing collated strings") { - val nonBinaryStringType = StringType(CollationFactory.collationNameToId("UTF8_BINARY_LCASE")) + val nonBinaryStringType = StringType(CollationFactory.collationNameToId("UTF8_LCASE")) // simple checks assert(UnsafeRowUtils.isBinaryStable(IntegerType)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala index cfda19ed67a03..3a88b900430da 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeSuite.scala @@ -711,69 +711,69 @@ class DataTypeSuite extends SparkFunSuite { checkEqualsIgnoreCompatibleCollation(StringType, IntegerType, expected = false) checkEqualsIgnoreCompatibleCollation(IntegerType, StringType, expected = false) // Collated `StringType`. - checkEqualsIgnoreCompatibleCollation(StringType, StringType("UTF8_BINARY_LCASE"), + checkEqualsIgnoreCompatibleCollation(StringType, StringType("UTF8_LCASE"), expected = true) checkEqualsIgnoreCompatibleCollation( - StringType("UTF8_BINARY"), StringType("UTF8_BINARY_LCASE"), expected = true) + StringType("UTF8_BINARY"), StringType("UTF8_LCASE"), expected = true) // Complex types. checkEqualsIgnoreCompatibleCollation( ArrayType(StringType), - ArrayType(StringType("UTF8_BINARY_LCASE")), + ArrayType(StringType("UTF8_LCASE")), expected = true ) checkEqualsIgnoreCompatibleCollation( ArrayType(StringType), - ArrayType(ArrayType(StringType("UTF8_BINARY_LCASE"))), + ArrayType(ArrayType(StringType("UTF8_LCASE"))), expected = false ) checkEqualsIgnoreCompatibleCollation( ArrayType(ArrayType(StringType)), - ArrayType(ArrayType(StringType("UTF8_BINARY_LCASE"))), + ArrayType(ArrayType(StringType("UTF8_LCASE"))), expected = true ) checkEqualsIgnoreCompatibleCollation( MapType(StringType, StringType), - MapType(StringType, StringType("UTF8_BINARY_LCASE")), + MapType(StringType, StringType("UTF8_LCASE")), expected = true ) checkEqualsIgnoreCompatibleCollation( - MapType(StringType("UTF8_BINARY_LCASE"), StringType), + MapType(StringType("UTF8_LCASE"), StringType), MapType(StringType, StringType), expected = false ) checkEqualsIgnoreCompatibleCollation( - MapType(StringType("UTF8_BINARY_LCASE"), ArrayType(StringType)), - MapType(StringType("UTF8_BINARY_LCASE"), ArrayType(StringType("UTF8_BINARY_LCASE"))), + MapType(StringType("UTF8_LCASE"), ArrayType(StringType)), + MapType(StringType("UTF8_LCASE"), ArrayType(StringType("UTF8_LCASE"))), expected = true ) checkEqualsIgnoreCompatibleCollation( MapType(ArrayType(StringType), IntegerType), - MapType(ArrayType(StringType("UTF8_BINARY_LCASE")), IntegerType), + MapType(ArrayType(StringType("UTF8_LCASE")), IntegerType), expected = false ) checkEqualsIgnoreCompatibleCollation( - MapType(ArrayType(StringType("UTF8_BINARY_LCASE")), IntegerType), - MapType(ArrayType(StringType("UTF8_BINARY_LCASE")), IntegerType), + MapType(ArrayType(StringType("UTF8_LCASE")), IntegerType), + MapType(ArrayType(StringType("UTF8_LCASE")), IntegerType), expected = true ) checkEqualsIgnoreCompatibleCollation( StructType(StructField("a", StringType) :: Nil), - StructType(StructField("a", StringType("UTF8_BINARY_LCASE")) :: Nil), + StructType(StructField("a", StringType("UTF8_LCASE")) :: Nil), expected = true ) checkEqualsIgnoreCompatibleCollation( StructType(StructField("a", ArrayType(StringType)) :: Nil), - StructType(StructField("a", ArrayType(StringType("UTF8_BINARY_LCASE"))) :: Nil), + StructType(StructField("a", ArrayType(StringType("UTF8_LCASE"))) :: Nil), expected = true ) checkEqualsIgnoreCompatibleCollation( StructType(StructField("a", MapType(StringType, IntegerType)) :: Nil), - StructType(StructField("a", MapType(StringType("UTF8_BINARY_LCASE"), IntegerType)) :: Nil), + StructType(StructField("a", MapType(StringType("UTF8_LCASE"), IntegerType)) :: Nil), expected = false ) checkEqualsIgnoreCompatibleCollation( StructType(StructField("a", StringType) :: Nil), - StructType(StructField("b", StringType("UTF8_BINARY_LCASE")) :: Nil), + StructType(StructField("b", StringType("UTF8_LCASE")) :: Nil), expected = false ) // Null compatibility checks. @@ -784,7 +784,7 @@ class DataTypeSuite extends SparkFunSuite { ) checkEqualsIgnoreCompatibleCollation( ArrayType(StringType, containsNull = true), - ArrayType(StringType("UTF8_BINARY_LCASE"), containsNull = false), + ArrayType(StringType("UTF8_LCASE"), containsNull = false), expected = false ) checkEqualsIgnoreCompatibleCollation( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala index bd0685e10832f..562febe381130 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala @@ -39,7 +39,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { private val s = StructType.fromDDL("a INT, b STRING") private val UNICODE_COLLATION = "UNICODE" - private val UTF8_BINARY_LCASE_COLLATION = "UTF8_BINARY_LCASE" + private val UTF8_LCASE_COLLATION = "UTF8_LCASE" private val mapper = new ObjectMapper() test("lookup a single missing field should output existing fields") { @@ -642,7 +642,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { test("nested struct with collations to json") { val nestedStruct = StructType( StructField("nested", StructType( - StructField("c1", StringType(UTF8_BINARY_LCASE_COLLATION)) :: Nil)) :: Nil) + StructField("c1", StringType(UTF8_LCASE_COLLATION)) :: Nil)) :: Nil) val expectedJson = s""" @@ -660,7 +660,7 @@ class StructTypeSuite extends SparkFunSuite with SQLHelper { | "nullable": true, | "metadata": { | "${DataType.COLLATIONS_METADATA_KEY}": { - | "c1": "spark.$UTF8_BINARY_LCASE_COLLATION" + | "c1": "spark.$UTF8_LCASE_COLLATION" | } | } | } diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index 65b513264598b..27e56b24625bb 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -81,7 +81,7 @@ | org.apache.spark.sql.catalyst.expressions.Chr | char | SELECT char(65) | struct | | org.apache.spark.sql.catalyst.expressions.Chr | chr | SELECT chr(65) | struct | | org.apache.spark.sql.catalyst.expressions.Coalesce | coalesce | SELECT coalesce(NULL, 1, NULL) | struct | -| org.apache.spark.sql.catalyst.expressions.CollateExpressionBuilder | collate | SELECT COLLATION('Spark SQL' collate UTF8_BINARY_LCASE) | struct | +| org.apache.spark.sql.catalyst.expressions.CollateExpressionBuilder | collate | SELECT COLLATION('Spark SQL' collate UTF8_LCASE) | struct | | org.apache.spark.sql.catalyst.expressions.Collation | collation | SELECT collation('Spark SQL') | struct | | org.apache.spark.sql.catalyst.expressions.Concat | concat | SELECT concat('Spark', 'SQL') | struct | | org.apache.spark.sql.catalyst.expressions.ConcatWs | concat_ws | SELECT concat_ws(' ', 'Spark', 'SQL') | struct | diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out index 9a1f4ed1f8e57..e6409806bad7a 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out @@ -1,6 +1,6 @@ -- Automatically generated by SQLQueryTestSuite -- !query -create table t1(utf8_binary string collate utf8_binary, utf8_binary_lcase string collate utf8_binary_lcase) using parquet +create table t1(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet -- !query analysis CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false @@ -8,32 +8,32 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false -- !query insert into t1 values('aaa', 'aaa') -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_binary_lcase] -+- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_BINARY_LCASE) AS utf8_binary_lcase#x] +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x] +- LocalRelation [col1#x, col2#x] -- !query insert into t1 values('AAA', 'AAA') -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_binary_lcase] -+- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_BINARY_LCASE) AS utf8_binary_lcase#x] +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x] +- LocalRelation [col1#x, col2#x] -- !query insert into t1 values('bbb', 'bbb') -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_binary_lcase] -+- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_BINARY_LCASE) AS utf8_binary_lcase#x] +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x] +- LocalRelation [col1#x, col2#x] -- !query insert into t1 values('BBB', 'BBB') -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_binary_lcase] -+- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_BINARY_LCASE) AS utf8_binary_lcase#x] +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x] +- LocalRelation [col1#x, col2#x] @@ -48,68 +48,68 @@ select count(*) from t1 group by utf8_binary -- !query analysis Aggregate [utf8_binary#x], [count(1) AS count(1)#xL] +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_binary_lcase#x] parquet + +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet -- !query -select count(*) from t1 group by utf8_binary_lcase +select count(*) from t1 group by utf8_lcase -- !query analysis -Aggregate [utf8_binary_lcase#x], [count(1) AS count(1)#xL] +Aggregate [utf8_lcase#x], [count(1) AS count(1)#xL] +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_binary_lcase#x] parquet + +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet -- !query select * from t1 where utf8_binary = 'aaa' -- !query analysis -Project [utf8_binary#x, utf8_binary_lcase#x] +Project [utf8_binary#x, utf8_lcase#x] +- Filter (utf8_binary#x = aaa) +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_binary_lcase#x] parquet + +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet -- !query -select * from t1 where utf8_binary_lcase = 'aaa' collate utf8_binary_lcase +select * from t1 where utf8_lcase = 'aaa' collate utf8_lcase -- !query analysis -Project [utf8_binary#x, utf8_binary_lcase#x] -+- Filter (utf8_binary_lcase#x = collate(aaa, utf8_binary_lcase)) +Project [utf8_binary#x, utf8_lcase#x] ++- Filter (utf8_lcase#x = collate(aaa, utf8_lcase)) +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_binary_lcase#x] parquet + +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet -- !query select * from t1 where utf8_binary < 'bbb' -- !query analysis -Project [utf8_binary#x, utf8_binary_lcase#x] +Project [utf8_binary#x, utf8_lcase#x] +- Filter (utf8_binary#x < bbb) +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_binary_lcase#x] parquet + +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet -- !query -select * from t1 where utf8_binary_lcase < 'bbb' collate utf8_binary_lcase +select * from t1 where utf8_lcase < 'bbb' collate utf8_lcase -- !query analysis -Project [utf8_binary#x, utf8_binary_lcase#x] -+- Filter (utf8_binary_lcase#x < collate(bbb, utf8_binary_lcase)) +Project [utf8_binary#x, utf8_lcase#x] ++- Filter (utf8_lcase#x < collate(bbb, utf8_lcase)) +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_binary_lcase#x] parquet + +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet -- !query -select l.utf8_binary, r.utf8_binary_lcase from t1 l join t1 r on l.utf8_binary_lcase = r.utf8_binary_lcase +select l.utf8_binary, r.utf8_lcase from t1 l join t1 r on l.utf8_lcase = r.utf8_lcase -- !query analysis -Project [utf8_binary#x, utf8_binary_lcase#x] -+- Join Inner, (utf8_binary_lcase#x = utf8_binary_lcase#x) +Project [utf8_binary#x, utf8_lcase#x] ++- Join Inner, (utf8_lcase#x = utf8_lcase#x) :- SubqueryAlias l : +- SubqueryAlias spark_catalog.default.t1 - : +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_binary_lcase#x] parquet + : +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet +- SubqueryAlias r +- SubqueryAlias spark_catalog.default.t1 - +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_binary_lcase#x] parquet + +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet -- !query -create table t2(utf8_binary string collate utf8_binary, utf8_binary_lcase string collate utf8_binary_lcase) using parquet +create table t2(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet -- !query analysis CreateDataSourceTableCommand `spark_catalog`.`default`.`t2`, false @@ -117,28 +117,28 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`t2`, false -- !query insert into t2 values('aaa', 'aaa') -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [utf8_binary, utf8_binary_lcase] -+- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_BINARY_LCASE) AS utf8_binary_lcase#x] +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x] +- LocalRelation [col1#x, col2#x] -- !query insert into t2 values('bbb', 'bbb') -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [utf8_binary, utf8_binary_lcase] -+- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_BINARY_LCASE) AS utf8_binary_lcase#x] +InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t2, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t2], Append, `spark_catalog`.`default`.`t2`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t2), [utf8_binary, utf8_lcase] ++- Project [cast(col1#x as string) AS utf8_binary#x, cast(col2#x as string collate UTF8_LCASE) AS utf8_lcase#x] +- LocalRelation [col1#x, col2#x] -- !query -select * from t1 anti join t2 on t1.utf8_binary_lcase = t2.utf8_binary_lcase +select * from t1 anti join t2 on t1.utf8_lcase = t2.utf8_lcase -- !query analysis -Project [utf8_binary#x, utf8_binary_lcase#x] -+- Join LeftAnti, (utf8_binary_lcase#x = utf8_binary_lcase#x) +Project [utf8_binary#x, utf8_lcase#x] ++- Join LeftAnti, (utf8_lcase#x = utf8_lcase#x) :- SubqueryAlias spark_catalog.default.t1 - : +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_binary_lcase#x] parquet + : +- Relation spark_catalog.default.t1[utf8_binary#x,utf8_lcase#x] parquet +- SubqueryAlias spark_catalog.default.t2 - +- Relation spark_catalog.default.t2[utf8_binary#x,utf8_binary_lcase#x] parquet + +- Relation spark_catalog.default.t2[utf8_binary#x,utf8_lcase#x] parquet -- !query @@ -156,75 +156,75 @@ DropTable false, false -- !query -select col1 collate utf8_binary_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb') +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query analysis Except false -:- Project [collate(col1#x, utf8_binary_lcase) AS collate(col1)#x] +:- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] : +- LocalRelation [col1#x] -+- Project [collate(col1#x, utf8_binary_lcase) AS collate(col1)#x] ++- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] +- LocalRelation [col1#x] -- !query -select col1 collate utf8_binary_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb') +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query analysis Except All true -:- Project [collate(col1#x, utf8_binary_lcase) AS collate(col1)#x] +:- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] : +- LocalRelation [col1#x] -+- Project [collate(col1#x, utf8_binary_lcase) AS collate(col1)#x] ++- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] +- LocalRelation [col1#x] -- !query -select col1 collate utf8_binary_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb') +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query analysis Distinct +- Union false, false - :- Project [collate(col1#x, utf8_binary_lcase) AS collate(col1)#x] + :- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] : +- LocalRelation [col1#x] - +- Project [collate(col1#x, utf8_binary_lcase) AS collate(col1)#x] + +- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] +- LocalRelation [col1#x] -- !query -select col1 collate utf8_binary_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb') +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query analysis Union false, false -:- Project [collate(col1#x, utf8_binary_lcase) AS collate(col1)#x] +:- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] : +- LocalRelation [col1#x] -+- Project [collate(col1#x, utf8_binary_lcase) AS collate(col1)#x] ++- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] +- LocalRelation [col1#x] -- !query -select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb') +select col1 collate utf8_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query analysis Intersect false -:- Project [collate(col1#x, utf8_binary_lcase) AS collate(col1)#x] +:- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] : +- LocalRelation [col1#x] -+- Project [collate(col1#x, utf8_binary_lcase) AS collate(col1)#x] ++- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] +- LocalRelation [col1#x] -- !query -create table t1 (c1 struct) USING PARQUET +create table t1 (c1 struct) USING PARQUET -- !query analysis CreateDataSourceTableCommand `spark_catalog`.`default`.`t1`, false -- !query -insert into t1 values (named_struct('utf8_binary', 'aaa', 'utf8_binary_lcase', 'aaa')) +insert into t1 values (named_struct('utf8_binary', 'aaa', 'utf8_lcase', 'aaa')) -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [c1] -+- Project [named_struct(utf8_binary, col1#x.utf8_binary, utf8_binary_lcase, cast(col1#x.utf8_binary_lcase as string collate UTF8_BINARY_LCASE)) AS c1#x] ++- Project [named_struct(utf8_binary, col1#x.utf8_binary, utf8_lcase, cast(col1#x.utf8_lcase as string collate UTF8_LCASE)) AS c1#x] +- LocalRelation [col1#x] -- !query -insert into t1 values (named_struct('utf8_binary', 'AAA', 'utf8_binary_lcase', 'AAA')) +insert into t1 values (named_struct('utf8_binary', 'AAA', 'utf8_lcase', 'AAA')) -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/t1, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/t1], Append, `spark_catalog`.`default`.`t1`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/t1), [c1] -+- Project [named_struct(utf8_binary, col1#x.utf8_binary, utf8_binary_lcase, cast(col1#x.utf8_binary_lcase as string collate UTF8_BINARY_LCASE)) AS c1#x] ++- Project [named_struct(utf8_binary, col1#x.utf8_binary, utf8_lcase, cast(col1#x.utf8_lcase as string collate UTF8_LCASE)) AS c1#x] +- LocalRelation [col1#x] @@ -237,9 +237,9 @@ Aggregate [c1#x.utf8_binary], [count(1) AS count(1)#xL] -- !query -select count(*) from t1 group by c1.utf8_binary_lcase +select count(*) from t1 group by c1.utf8_lcase -- !query analysis -Aggregate [c1#x.utf8_binary_lcase], [count(1) AS count(1)#xL] +Aggregate [c1#x.utf8_lcase], [count(1) AS count(1)#xL] +- SubqueryAlias spark_catalog.default.t1 +- Relation spark_catalog.default.t1[c1#x] parquet @@ -252,65 +252,65 @@ DropTable false, false -- !query -select array_contains(ARRAY('aaa' collate utf8_binary_lcase),'AAA' collate utf8_binary_lcase) +select array_contains(ARRAY('aaa' collate utf8_lcase),'AAA' collate utf8_lcase) -- !query analysis -Project [array_contains(array(collate(aaa, utf8_binary_lcase)), collate(AAA, utf8_binary_lcase)) AS array_contains(array(collate(aaa)), collate(AAA))#x] +Project [array_contains(array(collate(aaa, utf8_lcase)), collate(AAA, utf8_lcase)) AS array_contains(array(collate(aaa)), collate(AAA))#x] +- OneRowRelation -- !query -select array_position(ARRAY('aaa' collate utf8_binary_lcase, 'bbb' collate utf8_binary_lcase),'BBB' collate utf8_binary_lcase) +select array_position(ARRAY('aaa' collate utf8_lcase, 'bbb' collate utf8_lcase),'BBB' collate utf8_lcase) -- !query analysis -Project [array_position(array(collate(aaa, utf8_binary_lcase), collate(bbb, utf8_binary_lcase)), collate(BBB, utf8_binary_lcase)) AS array_position(array(collate(aaa), collate(bbb)), collate(BBB))#xL] +Project [array_position(array(collate(aaa, utf8_lcase), collate(bbb, utf8_lcase)), collate(BBB, utf8_lcase)) AS array_position(array(collate(aaa), collate(bbb)), collate(BBB))#xL] +- OneRowRelation -- !query -select nullif('aaa' COLLATE utf8_binary_lcase, 'AAA' COLLATE utf8_binary_lcase) +select nullif('aaa' COLLATE utf8_lcase, 'AAA' COLLATE utf8_lcase) -- !query analysis -Project [nullif(collate(aaa, utf8_binary_lcase), collate(AAA, utf8_binary_lcase)) AS nullif(collate(aaa), collate(AAA))#x] +Project [nullif(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase)) AS nullif(collate(aaa), collate(AAA))#x] +- OneRowRelation -- !query -select least('aaa' COLLATE utf8_binary_lcase, 'AAA' collate utf8_binary_lcase, 'a' collate utf8_binary_lcase) +select least('aaa' COLLATE utf8_lcase, 'AAA' collate utf8_lcase, 'a' collate utf8_lcase) -- !query analysis -Project [least(collate(aaa, utf8_binary_lcase), collate(AAA, utf8_binary_lcase), collate(a, utf8_binary_lcase)) AS least(collate(aaa), collate(AAA), collate(a))#x] +Project [least(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase), collate(a, utf8_lcase)) AS least(collate(aaa), collate(AAA), collate(a))#x] +- OneRowRelation -- !query -select arrays_overlap(array('aaa' collate utf8_binary_lcase), array('AAA' collate utf8_binary_lcase)) +select arrays_overlap(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)) -- !query analysis -Project [arrays_overlap(array(collate(aaa, utf8_binary_lcase)), array(collate(AAA, utf8_binary_lcase))) AS arrays_overlap(array(collate(aaa)), array(collate(AAA)))#x] +Project [arrays_overlap(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS arrays_overlap(array(collate(aaa)), array(collate(AAA)))#x] +- OneRowRelation -- !query -select array_distinct(array('aaa' collate utf8_binary_lcase, 'AAA' collate utf8_binary_lcase)) +select array_distinct(array('aaa' collate utf8_lcase, 'AAA' collate utf8_lcase)) -- !query analysis -Project [array_distinct(array(collate(aaa, utf8_binary_lcase), collate(AAA, utf8_binary_lcase))) AS array_distinct(array(collate(aaa), collate(AAA)))#x] +Project [array_distinct(array(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase))) AS array_distinct(array(collate(aaa), collate(AAA)))#x] +- OneRowRelation -- !query -select array_union(array('aaa' collate utf8_binary_lcase), array('AAA' collate utf8_binary_lcase)) +select array_union(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)) -- !query analysis -Project [array_union(array(collate(aaa, utf8_binary_lcase)), array(collate(AAA, utf8_binary_lcase))) AS array_union(array(collate(aaa)), array(collate(AAA)))#x] +Project [array_union(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS array_union(array(collate(aaa)), array(collate(AAA)))#x] +- OneRowRelation -- !query -select array_intersect(array('aaa' collate utf8_binary_lcase), array('AAA' collate utf8_binary_lcase)) +select array_intersect(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)) -- !query analysis -Project [array_intersect(array(collate(aaa, utf8_binary_lcase)), array(collate(AAA, utf8_binary_lcase))) AS array_intersect(array(collate(aaa)), array(collate(AAA)))#x] +Project [array_intersect(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS array_intersect(array(collate(aaa)), array(collate(AAA)))#x] +- OneRowRelation -- !query -select array_except(array('aaa' collate utf8_binary_lcase), array('AAA' collate utf8_binary_lcase)) +select array_except(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)) -- !query analysis -Project [array_except(array(collate(aaa, utf8_binary_lcase)), array(collate(AAA, utf8_binary_lcase))) AS array_except(array(collate(aaa)), array(collate(AAA)))#x] +Project [array_except(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS array_except(array(collate(aaa)), array(collate(AAA)))#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/inputs/collations.sql b/sql/core/src/test/resources/sql-tests/inputs/collations.sql index 6bb0a0163443a..c0262a0f0ad14 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/collations.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/collations.sql @@ -1,7 +1,7 @@ -- test cases for collation support -- Create a test table with data -create table t1(utf8_binary string collate utf8_binary, utf8_binary_lcase string collate utf8_binary_lcase) using parquet; +create table t1(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet; insert into t1 values('aaa', 'aaa'); insert into t1 values('AAA', 'AAA'); insert into t1 values('bbb', 'bbb'); @@ -13,70 +13,70 @@ describe table t1; -- group by and count utf8_binary select count(*) from t1 group by utf8_binary; --- group by and count utf8_binary_lcase -select count(*) from t1 group by utf8_binary_lcase; +-- group by and count utf8_lcase +select count(*) from t1 group by utf8_lcase; -- filter equal utf8_binary select * from t1 where utf8_binary = 'aaa'; --- filter equal utf8_binary_lcase -select * from t1 where utf8_binary_lcase = 'aaa' collate utf8_binary_lcase; +-- filter equal utf8_lcase +select * from t1 where utf8_lcase = 'aaa' collate utf8_lcase; -- filter less then utf8_binary select * from t1 where utf8_binary < 'bbb'; --- filter less then utf8_binary_lcase -select * from t1 where utf8_binary_lcase < 'bbb' collate utf8_binary_lcase; +-- filter less then utf8_lcase +select * from t1 where utf8_lcase < 'bbb' collate utf8_lcase; -- inner join -select l.utf8_binary, r.utf8_binary_lcase from t1 l join t1 r on l.utf8_binary_lcase = r.utf8_binary_lcase; +select l.utf8_binary, r.utf8_lcase from t1 l join t1 r on l.utf8_lcase = r.utf8_lcase; -- create second table for anti-join -create table t2(utf8_binary string collate utf8_binary, utf8_binary_lcase string collate utf8_binary_lcase) using parquet; +create table t2(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet; insert into t2 values('aaa', 'aaa'); insert into t2 values('bbb', 'bbb'); -- anti-join on lcase -select * from t1 anti join t2 on t1.utf8_binary_lcase = t2.utf8_binary_lcase; +select * from t1 anti join t2 on t1.utf8_lcase = t2.utf8_lcase; drop table t2; drop table t1; -- set operations -select col1 collate utf8_binary_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb'); -select col1 collate utf8_binary_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb'); -select col1 collate utf8_binary_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb'); -select col1 collate utf8_binary_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb'); -select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb'); +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate utf8_lcase from values ('aaa'), ('bbb'); +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate utf8_lcase from values ('aaa'), ('bbb'); +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate utf8_lcase from values ('aaa'), ('bbb'); +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate utf8_lcase from values ('aaa'), ('bbb'); +select col1 collate utf8_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate utf8_lcase from values ('aaa'), ('bbb'); -- create table with struct field -create table t1 (c1 struct) USING PARQUET; +create table t1 (c1 struct) USING PARQUET; -insert into t1 values (named_struct('utf8_binary', 'aaa', 'utf8_binary_lcase', 'aaa')); -insert into t1 values (named_struct('utf8_binary', 'AAA', 'utf8_binary_lcase', 'AAA')); +insert into t1 values (named_struct('utf8_binary', 'aaa', 'utf8_lcase', 'aaa')); +insert into t1 values (named_struct('utf8_binary', 'AAA', 'utf8_lcase', 'AAA')); -- aggregate against nested field utf8_binary select count(*) from t1 group by c1.utf8_binary; --- aggregate against nested field utf8_binary_lcase -select count(*) from t1 group by c1.utf8_binary_lcase; +-- aggregate against nested field utf8_lcase +select count(*) from t1 group by c1.utf8_lcase; drop table t1; -- array function tests -select array_contains(ARRAY('aaa' collate utf8_binary_lcase),'AAA' collate utf8_binary_lcase); -select array_position(ARRAY('aaa' collate utf8_binary_lcase, 'bbb' collate utf8_binary_lcase),'BBB' collate utf8_binary_lcase); +select array_contains(ARRAY('aaa' collate utf8_lcase),'AAA' collate utf8_lcase); +select array_position(ARRAY('aaa' collate utf8_lcase, 'bbb' collate utf8_lcase),'BBB' collate utf8_lcase); -- utility -select nullif('aaa' COLLATE utf8_binary_lcase, 'AAA' COLLATE utf8_binary_lcase); -select least('aaa' COLLATE utf8_binary_lcase, 'AAA' collate utf8_binary_lcase, 'a' collate utf8_binary_lcase); +select nullif('aaa' COLLATE utf8_lcase, 'AAA' COLLATE utf8_lcase); +select least('aaa' COLLATE utf8_lcase, 'AAA' collate utf8_lcase, 'a' collate utf8_lcase); -- array operations -select arrays_overlap(array('aaa' collate utf8_binary_lcase), array('AAA' collate utf8_binary_lcase)); -select array_distinct(array('aaa' collate utf8_binary_lcase, 'AAA' collate utf8_binary_lcase)); -select array_union(array('aaa' collate utf8_binary_lcase), array('AAA' collate utf8_binary_lcase)); -select array_intersect(array('aaa' collate utf8_binary_lcase), array('AAA' collate utf8_binary_lcase)); -select array_except(array('aaa' collate utf8_binary_lcase), array('AAA' collate utf8_binary_lcase)); +select arrays_overlap(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)); +select array_distinct(array('aaa' collate utf8_lcase, 'AAA' collate utf8_lcase)); +select array_union(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)); +select array_intersect(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)); +select array_except(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)); -- ICU collations (all statements return true) select 'a' collate unicode < 'A'; diff --git a/sql/core/src/test/resources/sql-tests/results/collations.sql.out b/sql/core/src/test/resources/sql-tests/results/collations.sql.out index 96c875306d358..89e6665df9d04 100644 --- a/sql/core/src/test/resources/sql-tests/results/collations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/collations.sql.out @@ -1,6 +1,6 @@ -- Automatically generated by SQLQueryTestSuite -- !query -create table t1(utf8_binary string collate utf8_binary, utf8_binary_lcase string collate utf8_binary_lcase) using parquet +create table t1(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet -- !query schema struct<> -- !query output @@ -45,7 +45,7 @@ describe table t1 struct -- !query output utf8_binary string -utf8_binary_lcase string collate UTF8_BINARY_LCASE +utf8_lcase string collate UTF8_LCASE -- !query @@ -60,7 +60,7 @@ struct -- !query -select count(*) from t1 group by utf8_binary_lcase +select count(*) from t1 group by utf8_lcase -- !query schema struct -- !query output @@ -71,15 +71,15 @@ struct -- !query select * from t1 where utf8_binary = 'aaa' -- !query schema -struct +struct -- !query output aaa aaa -- !query -select * from t1 where utf8_binary_lcase = 'aaa' collate utf8_binary_lcase +select * from t1 where utf8_lcase = 'aaa' collate utf8_lcase -- !query schema -struct +struct -- !query output AAA AAA aaa aaa @@ -88,7 +88,7 @@ aaa aaa -- !query select * from t1 where utf8_binary < 'bbb' -- !query schema -struct +struct -- !query output AAA AAA BBB BBB @@ -96,18 +96,18 @@ aaa aaa -- !query -select * from t1 where utf8_binary_lcase < 'bbb' collate utf8_binary_lcase +select * from t1 where utf8_lcase < 'bbb' collate utf8_lcase -- !query schema -struct +struct -- !query output AAA AAA aaa aaa -- !query -select l.utf8_binary, r.utf8_binary_lcase from t1 l join t1 r on l.utf8_binary_lcase = r.utf8_binary_lcase +select l.utf8_binary, r.utf8_lcase from t1 l join t1 r on l.utf8_lcase = r.utf8_lcase -- !query schema -struct +struct -- !query output AAA AAA AAA aaa @@ -120,7 +120,7 @@ bbb bbb -- !query -create table t2(utf8_binary string collate utf8_binary, utf8_binary_lcase string collate utf8_binary_lcase) using parquet +create table t2(utf8_binary string collate utf8_binary, utf8_lcase string collate utf8_lcase) using parquet -- !query schema struct<> -- !query output @@ -144,9 +144,9 @@ struct<> -- !query -select * from t1 anti join t2 on t1.utf8_binary_lcase = t2.utf8_binary_lcase +select * from t1 anti join t2 on t1.utf8_lcase = t2.utf8_lcase -- !query schema -struct +struct -- !query output @@ -168,17 +168,17 @@ struct<> -- !query -select col1 collate utf8_binary_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb') +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query schema -struct +struct -- !query output zzz -- !query -select col1 collate utf8_binary_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb') +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query schema -struct +struct -- !query output aaa bbb @@ -187,9 +187,9 @@ zzz -- !query -select col1 collate utf8_binary_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb') +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query schema -struct +struct -- !query output aaa bbb @@ -197,9 +197,9 @@ zzz -- !query -select col1 collate utf8_binary_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb') +select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query schema -struct +struct -- !query output AAA BBB @@ -212,16 +212,16 @@ zzz -- !query -select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate utf8_binary_lcase from values ('aaa'), ('bbb') +select col1 collate utf8_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query schema -struct +struct -- !query output aaa bbb -- !query -create table t1 (c1 struct) USING PARQUET +create table t1 (c1 struct) USING PARQUET -- !query schema struct<> -- !query output @@ -229,7 +229,7 @@ struct<> -- !query -insert into t1 values (named_struct('utf8_binary', 'aaa', 'utf8_binary_lcase', 'aaa')) +insert into t1 values (named_struct('utf8_binary', 'aaa', 'utf8_lcase', 'aaa')) -- !query schema struct<> -- !query output @@ -237,7 +237,7 @@ struct<> -- !query -insert into t1 values (named_struct('utf8_binary', 'AAA', 'utf8_binary_lcase', 'AAA')) +insert into t1 values (named_struct('utf8_binary', 'AAA', 'utf8_lcase', 'AAA')) -- !query schema struct<> -- !query output @@ -254,7 +254,7 @@ struct -- !query -select count(*) from t1 group by c1.utf8_binary_lcase +select count(*) from t1 group by c1.utf8_lcase -- !query schema struct -- !query output @@ -270,7 +270,7 @@ struct<> -- !query -select array_contains(ARRAY('aaa' collate utf8_binary_lcase),'AAA' collate utf8_binary_lcase) +select array_contains(ARRAY('aaa' collate utf8_lcase),'AAA' collate utf8_lcase) -- !query schema struct -- !query output @@ -278,7 +278,7 @@ true -- !query -select array_position(ARRAY('aaa' collate utf8_binary_lcase, 'bbb' collate utf8_binary_lcase),'BBB' collate utf8_binary_lcase) +select array_position(ARRAY('aaa' collate utf8_lcase, 'bbb' collate utf8_lcase),'BBB' collate utf8_lcase) -- !query schema struct -- !query output @@ -286,23 +286,23 @@ struct -- !query -select nullif('aaa' COLLATE utf8_binary_lcase, 'AAA' COLLATE utf8_binary_lcase) +select nullif('aaa' COLLATE utf8_lcase, 'AAA' COLLATE utf8_lcase) -- !query schema -struct +struct -- !query output NULL -- !query -select least('aaa' COLLATE utf8_binary_lcase, 'AAA' collate utf8_binary_lcase, 'a' collate utf8_binary_lcase) +select least('aaa' COLLATE utf8_lcase, 'AAA' collate utf8_lcase, 'a' collate utf8_lcase) -- !query schema -struct +struct -- !query output a -- !query -select arrays_overlap(array('aaa' collate utf8_binary_lcase), array('AAA' collate utf8_binary_lcase)) +select arrays_overlap(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)) -- !query schema struct -- !query output @@ -310,33 +310,33 @@ true -- !query -select array_distinct(array('aaa' collate utf8_binary_lcase, 'AAA' collate utf8_binary_lcase)) +select array_distinct(array('aaa' collate utf8_lcase, 'AAA' collate utf8_lcase)) -- !query schema -struct> +struct> -- !query output ["aaa"] -- !query -select array_union(array('aaa' collate utf8_binary_lcase), array('AAA' collate utf8_binary_lcase)) +select array_union(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)) -- !query schema -struct> +struct> -- !query output ["aaa"] -- !query -select array_intersect(array('aaa' collate utf8_binary_lcase), array('AAA' collate utf8_binary_lcase)) +select array_intersect(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)) -- !query schema -struct> +struct> -- !query output ["aaa"] -- !query -select array_except(array('aaa' collate utf8_binary_lcase), array('AAA' collate utf8_binary_lcase)) +select array_except(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)) -- !query schema -struct> +struct> -- !query output [] 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 bcbd1331183c9..d34fd554e7bda 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 @@ -32,7 +32,7 @@ class CollationSQLExpressionsSuite extends QueryTest with SharedSparkSession { - private val testSuppCollations = Seq("UTF8_BINARY", "UTF8_BINARY_LCASE", "UNICODE", "UNICODE_CI") + private val testSuppCollations = Seq("UTF8_BINARY", "UTF8_LCASE", "UNICODE", "UNICODE_CI") test("Support Md5 hash expression with collation") { case class Md5TestCase( @@ -43,7 +43,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( Md5TestCase("Spark", "UTF8_BINARY", "8cde774d6f7333752ed72cacddb05126"), - Md5TestCase("Spark", "UTF8_BINARY_LCASE", "8cde774d6f7333752ed72cacddb05126"), + Md5TestCase("Spark", "UTF8_LCASE", "8cde774d6f7333752ed72cacddb05126"), Md5TestCase("SQL", "UNICODE", "9778840a0100cb30c982876741b0b5a2"), Md5TestCase("SQL", "UNICODE_CI", "9778840a0100cb30c982876741b0b5a2") ) @@ -75,7 +75,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( Sha2TestCase("Spark", "UTF8_BINARY", 256, "529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b"), - Sha2TestCase("Spark", "UTF8_BINARY_LCASE", 256, + Sha2TestCase("Spark", "UTF8_LCASE", 256, "529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b"), Sha2TestCase("SQL", "UNICODE", 256, "a7056a455639d1c7deec82ee787db24a0c1878e2792b4597709f0facf7cc7b35"), @@ -108,7 +108,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( Sha1TestCase("Spark", "UTF8_BINARY", "85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c"), - Sha1TestCase("Spark", "UTF8_BINARY_LCASE", "85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c"), + Sha1TestCase("Spark", "UTF8_LCASE", "85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c"), Sha1TestCase("SQL", "UNICODE", "2064cb643caa8d9e1de12eea7f3e143ca9f8680d"), Sha1TestCase("SQL", "UNICODE_CI", "2064cb643caa8d9e1de12eea7f3e143ca9f8680d") ) @@ -138,7 +138,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( Crc321TestCase("Spark", "UTF8_BINARY", 1557323817), - Crc321TestCase("Spark", "UTF8_BINARY_LCASE", 1557323817), + Crc321TestCase("Spark", "UTF8_LCASE", 1557323817), Crc321TestCase("SQL", "UNICODE", 1299261525), Crc321TestCase("SQL", "UNICODE_CI", 1299261525) ) @@ -166,7 +166,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( Murmur3HashTestCase("Spark", "UTF8_BINARY", 228093765), - Murmur3HashTestCase("Spark", "UTF8_BINARY_LCASE", 228093765), + Murmur3HashTestCase("Spark", "UTF8_LCASE", 228093765), Murmur3HashTestCase("SQL", "UNICODE", 17468742), Murmur3HashTestCase("SQL", "UNICODE_CI", 17468742) ) @@ -194,7 +194,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( XxHash64TestCase("Spark", "UTF8_BINARY", -4294468057691064905L), - XxHash64TestCase("Spark", "UTF8_BINARY_LCASE", -4294468057691064905L), + XxHash64TestCase("Spark", "UTF8_LCASE", -4294468057691064905L), XxHash64TestCase("SQL", "UNICODE", -2147923034195946097L), XxHash64TestCase("SQL", "UNICODE_CI", -2147923034195946097L) ) @@ -223,7 +223,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( UrlEncodeTestCase("https://spark.apache.org", "UTF8_BINARY", "https%3A%2F%2Fspark.apache.org"), - UrlEncodeTestCase("https://spark.apache.org", "UTF8_BINARY_LCASE", + UrlEncodeTestCase("https://spark.apache.org", "UTF8_LCASE", "https%3A%2F%2Fspark.apache.org"), UrlEncodeTestCase("https://spark.apache.org", "UNICODE", "https%3A%2F%2Fspark.apache.org"), @@ -257,7 +257,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UTF8_BINARY", "https://spark.apache.org"), - UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UTF8_BINARY_LCASE", + UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UTF8_LCASE", "https://spark.apache.org"), UrlDecodeTestCase("https%3A%2F%2Fspark.apache.org", "UNICODE", "https://spark.apache.org"), @@ -292,7 +292,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( ParseUrlTestCase("http://spark.apache.org/path?query=1", "UTF8_BINARY", "HOST", "spark.apache.org"), - ParseUrlTestCase("http://spark.apache.org/path?query=2", "UTF8_BINARY_LCASE", "PATH", + ParseUrlTestCase("http://spark.apache.org/path?query=2", "UTF8_LCASE", "PATH", "/path"), ParseUrlTestCase("http://spark.apache.org/path?query=3", "UNICODE", "QUERY", "query=3"), @@ -331,7 +331,7 @@ class CollationSQLExpressionsSuite Row(1), Seq( StructField("a", IntegerType, nullable = true) )), - CsvToStructsTestCase("true, 0.8", "UTF8_BINARY_LCASE", "'A BOOLEAN, B DOUBLE'", "", + CsvToStructsTestCase("true, 0.8", "UTF8_LCASE", "'A BOOLEAN, B DOUBLE'", "", Row(true, 0.8), Seq( StructField("A", BooleanType, nullable = true), StructField("B", DoubleType, nullable = true) @@ -374,7 +374,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( SchemaOfCsvTestCase("1", "UTF8_BINARY", "STRUCT<_c0: INT>"), - SchemaOfCsvTestCase("true,0.8", "UTF8_BINARY_LCASE", + SchemaOfCsvTestCase("true,0.8", "UTF8_LCASE", "STRUCT<_c0: BOOLEAN, _c1: DOUBLE>"), SchemaOfCsvTestCase("2015-08-26", "UNICODE", "STRUCT<_c0: DATE>"), SchemaOfCsvTestCase("abc", "UNICODE_CI", @@ -406,7 +406,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( StructsToCsvTestCase("named_struct('a', 1, 'b', 2)", "UTF8_BINARY", "1,2"), - StructsToCsvTestCase("named_struct('A', true, 'B', 2.0)", "UTF8_BINARY_LCASE", "true,2.0"), + StructsToCsvTestCase("named_struct('A', true, 'B', 2.0)", "UTF8_LCASE", "true,2.0"), StructsToCsvTestCase("named_struct()", "UNICODE", null), StructsToCsvTestCase("named_struct('time', to_timestamp('2015-08-26'))", "UNICODE_CI", "2015-08-26T00:00:00.000-07:00") @@ -439,7 +439,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( ConvTestCase("100", "2", "10", "UTF8_BINARY", "4"), - ConvTestCase("100", "2", "10", "UTF8_BINARY_LCASE", "4"), + ConvTestCase("100", "2", "10", "UTF8_LCASE", "4"), ConvTestCase("100", "2", "10", "UNICODE", "4"), ConvTestCase("100", "2", "10", "UNICODE_CI", "4") ) @@ -463,7 +463,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( BinTestCase("13", "UTF8_BINARY", "1101"), - BinTestCase("13", "UTF8_BINARY_LCASE", "1101"), + BinTestCase("13", "UTF8_LCASE", "1101"), BinTestCase("13", "UNICODE", "1101"), BinTestCase("13", "UNICODE_CI", "1101") ) @@ -488,7 +488,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( HexTestCase("13", "UTF8_BINARY", "D"), - HexTestCase("13", "UTF8_BINARY_LCASE", "D"), + HexTestCase("13", "UTF8_LCASE", "D"), HexTestCase("13", "UNICODE", "D"), HexTestCase("13", "UNICODE_CI", "D") ) @@ -513,7 +513,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( HexTestCase("Spark SQL", "UTF8_BINARY", "537061726B2053514C"), - HexTestCase("Spark SQL", "UTF8_BINARY_LCASE", "537061726B2053514C"), + HexTestCase("Spark SQL", "UTF8_LCASE", "537061726B2053514C"), HexTestCase("Spark SQL", "UNICODE", "537061726B2053514C"), HexTestCase("Spark SQL", "UNICODE_CI", "537061726B2053514C") ) @@ -536,7 +536,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( UnHexTestCase("537061726B2053514C", "UTF8_BINARY", "Spark SQL"), - UnHexTestCase("537061726B2053514C", "UTF8_BINARY_LCASE", "Spark SQL"), + UnHexTestCase("537061726B2053514C", "UTF8_LCASE", "Spark SQL"), UnHexTestCase("537061726B2053514C", "UNICODE", "Spark SQL"), UnHexTestCase("537061726B2053514C", "UNICODE_CI", "Spark SQL") ) @@ -567,9 +567,9 @@ class CollationSQLExpressionsSuite XPathTestCase("12", "sum(A/B)", "xpath_short", "UTF8_BINARY", 3, ShortType), XPathTestCase("34", "sum(a/b)", - "xpath_int", "UTF8_BINARY_LCASE", 7, IntegerType), + "xpath_int", "UTF8_LCASE", 7, IntegerType), XPathTestCase("56", "sum(A/B)", - "xpath_long", "UTF8_BINARY_LCASE", 11, LongType), + "xpath_long", "UTF8_LCASE", 11, LongType), XPathTestCase("78", "sum(a/b)", "xpath_float", "UNICODE", 15.0, FloatType), XPathTestCase("90", "sum(A/B)", @@ -604,7 +604,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( StringSpaceTestCase(1, "UTF8_BINARY", " "), - StringSpaceTestCase(2, "UTF8_BINARY_LCASE", " "), + StringSpaceTestCase(2, "UTF8_LCASE", " "), StringSpaceTestCase(3, "UNICODE", " "), StringSpaceTestCase(4, "UNICODE_CI", " ") ) @@ -636,7 +636,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( ToNumberTestCase("123", "UTF8_BINARY", "999", 123, DecimalType(3, 0)), - ToNumberTestCase("1", "UTF8_BINARY_LCASE", "0.00", 1.00, DecimalType(3, 2)), + ToNumberTestCase("1", "UTF8_LCASE", "0.00", 1.00, DecimalType(3, 2)), ToNumberTestCase("99,999", "UNICODE", "99,999", 99999, DecimalType(5, 0)), ToNumberTestCase("$14.99", "UNICODE_CI", "$99.99", 14.99, DecimalType(4, 2)) ) @@ -703,7 +703,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( ToCharTestCase(12, "UTF8_BINARY", "999", " 12"), - ToCharTestCase(34, "UTF8_BINARY_LCASE", "000D00", "034.00"), + ToCharTestCase(34, "UTF8_LCASE", "000D00", "034.00"), ToCharTestCase(56, "UNICODE", "$99.99", "$56.00"), ToCharTestCase(78, "UNICODE_CI", "99D9S", "78.0+") ) @@ -734,7 +734,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( GetJsonObjectTestCase("{\"a\":\"b\"}", "$.a", "UTF8_BINARY", "b"), - GetJsonObjectTestCase("{\"A\":\"1\"}", "$.A", "UTF8_BINARY_LCASE", "1"), + GetJsonObjectTestCase("{\"A\":\"1\"}", "$.A", "UTF8_LCASE", "1"), GetJsonObjectTestCase("{\"x\":true}", "$.x", "UNICODE", "true"), GetJsonObjectTestCase("{\"X\":1}", "$.X", "UNICODE_CI", "1") ) @@ -766,7 +766,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( JsonTupleTestCase("{\"a\":1, \"b\":2}", "'a', 'b'", "UTF8_BINARY", Row("1", "2")), - JsonTupleTestCase("{\"A\":\"3\", \"B\":\"4\"}", "'A', 'B'", "UTF8_BINARY_LCASE", + JsonTupleTestCase("{\"A\":\"3\", \"B\":\"4\"}", "'A', 'B'", "UTF8_LCASE", Row("3", "4")), JsonTupleTestCase("{\"x\":true, \"y\":false}", "'x', 'y'", "UNICODE", Row("true", "false")), @@ -801,8 +801,8 @@ class CollationSQLExpressionsSuite val testCases = Seq( JsonToStructsTestCase("{\"a\":1, \"b\":2.0}", "a INT, b DOUBLE", "UTF8_BINARY", Row(Row(1, 2.0))), - JsonToStructsTestCase("{\"A\":\"3\", \"B\":4}", "A STRING COLLATE UTF8_BINARY_LCASE, B INT", - "UTF8_BINARY_LCASE", Row(Row("3", 4))), + JsonToStructsTestCase("{\"A\":\"3\", \"B\":4}", "A STRING COLLATE UTF8_LCASE, B INT", + "UTF8_LCASE", Row(Row("3", 4))), JsonToStructsTestCase("{\"x\":true, \"y\":null}", "x BOOLEAN, y VOID", "UNICODE", Row(Row(true, null))), JsonToStructsTestCase("{\"X\":null, \"Y\":false}", "X VOID, Y BOOLEAN", @@ -836,7 +836,7 @@ class CollationSQLExpressionsSuite StructsToJsonTestCase("named_struct('a', 1, 'b', 2)", "UTF8_BINARY", Row("{\"a\":1,\"b\":2}")), StructsToJsonTestCase("array(named_struct('a', 1, 'b', 2))", - "UTF8_BINARY_LCASE", Row("[{\"a\":1,\"b\":2}]")), + "UTF8_LCASE", Row("[{\"a\":1,\"b\":2}]")), StructsToJsonTestCase("map('a', named_struct('b', 1))", "UNICODE", Row("{\"a\":{\"b\":1}}")), StructsToJsonTestCase("array(map('a', 1))", @@ -868,7 +868,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( LengthOfJsonArrayTestCase("'[1,2,3,4]'", "UTF8_BINARY", Row(4)), - LengthOfJsonArrayTestCase("'[1,2,3,{\"f1\":1,\"f2\":[5,6]},4]'", "UTF8_BINARY_LCASE", Row(5)), + LengthOfJsonArrayTestCase("'[1,2,3,{\"f1\":1,\"f2\":[5,6]},4]'", "UTF8_LCASE", Row(5)), LengthOfJsonArrayTestCase("'[1,2'", "UNICODE", Row(null)), LengthOfJsonArrayTestCase("'['", "UNICODE_CI", Row(null)) ) @@ -898,7 +898,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( JsonObjectKeysJsonArrayTestCase("{}", "UTF8_BINARY", Row(Seq())), - JsonObjectKeysJsonArrayTestCase("{\"k\":", "UTF8_BINARY_LCASE", + JsonObjectKeysJsonArrayTestCase("{\"k\":", "UTF8_LCASE", Row(null)), JsonObjectKeysJsonArrayTestCase("{\"k1\": \"v1\"}", "UNICODE", Row(Seq("k1"))), @@ -933,7 +933,7 @@ class CollationSQLExpressionsSuite SchemaOfJsonTestCase("'[{\"col\":0}]'", "UTF8_BINARY", Row("ARRAY>")), SchemaOfJsonTestCase("'[{\"col\":01}]', map('allowNumericLeadingZeros', 'true')", - "UTF8_BINARY_LCASE", Row("ARRAY>")), + "UTF8_LCASE", Row("ARRAY>")), SchemaOfJsonTestCase("'[]'", "UNICODE", Row("ARRAY")), SchemaOfJsonTestCase("''", @@ -962,7 +962,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( StringToMapTestCase("a:1,b:2,c:3", ",", ":", "UTF8_BINARY", Map("a" -> "1", "b" -> "2", "c" -> "3")), - StringToMapTestCase("A-1;B-2;C-3", ";", "-", "UTF8_BINARY_LCASE", + StringToMapTestCase("A-1;B-2;C-3", ";", "-", "UTF8_LCASE", Map("A" -> "1", "B" -> "2", "C" -> "3")), StringToMapTestCase("1:a,2:b,3:c", ",", ":", "UNICODE", Map("1" -> "a", "2" -> "b", "3" -> "c")), @@ -983,7 +983,7 @@ class CollationSQLExpressionsSuite case class RaiseErrorTestCase(errorMessage: String, collationName: String) val testCases = Seq( RaiseErrorTestCase("custom error message 1", "UTF8_BINARY"), - RaiseErrorTestCase("custom error message 2", "UTF8_BINARY_LCASE"), + RaiseErrorTestCase("custom error message 2", "UTF8_LCASE"), RaiseErrorTestCase("custom error message 3", "UNICODE"), RaiseErrorTestCase("custom error message 4", "UNICODE_CI") ) @@ -1002,7 +1002,7 @@ class CollationSQLExpressionsSuite test("Support CurrentDatabase/Catalog/User expressions with collation") { // Supported collations - Seq("UTF8_BINARY_LCASE", "UNICODE", "UNICODE_CI").foreach(collationName => + Seq("UTF8_LCASE", "UNICODE", "UNICODE_CI").foreach(collationName => withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collationName) { val queryDatabase = sql("SELECT current_schema()") val queryCatalog = sql("SELECT current_catalog()") @@ -1018,7 +1018,7 @@ class CollationSQLExpressionsSuite test("Support Uuid misc expression with collation") { // Supported collations - Seq("UTF8_BINARY_LCASE", "UNICODE", "UNICODE_CI").foreach(collationName => + Seq("UTF8_LCASE", "UNICODE", "UNICODE_CI").foreach(collationName => withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collationName) { val query = s"SELECT uuid()" // Result & data type @@ -1034,7 +1034,7 @@ class CollationSQLExpressionsSuite test("Support SparkVersion misc expression with collation") { // Supported collations - Seq("UTF8_BINARY", "UTF8_BINARY_LCASE", "UNICODE", "UNICODE_CI").foreach(collationName => + Seq("UTF8_BINARY", "UTF8_LCASE", "UNICODE", "UNICODE_CI").foreach(collationName => withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collationName) { val query = s"SELECT version()" // Result & data type @@ -1053,7 +1053,7 @@ class CollationSQLExpressionsSuite case class TypeOfTestCase(input: String, collationName: String, result: String) val testCases = Seq( TypeOfTestCase("1", "UTF8_BINARY", "int"), - TypeOfTestCase("\"A\"", "UTF8_BINARY_LCASE", "string collate UTF8_BINARY_LCASE"), + TypeOfTestCase("\"A\"", "UTF8_LCASE", "string collate UTF8_LCASE"), TypeOfTestCase("array(1)", "UNICODE", "array"), TypeOfTestCase("null", "UNICODE_CI", "void") ) @@ -1080,7 +1080,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( AesEncryptTestCase("Spark", "UTF8_BINARY", "'1234567890abcdef', 'ECB'", "8DE7DB79A23F3E8ED530994DDEA98913"), - AesEncryptTestCase("Spark", "UTF8_BINARY_LCASE", "'1234567890abcdef', 'ECB', 'DEFAULT', ''", + AesEncryptTestCase("Spark", "UTF8_LCASE", "'1234567890abcdef', 'ECB', 'DEFAULT', ''", "8DE7DB79A23F3E8ED530994DDEA98913"), AesEncryptTestCase("Spark", "UNICODE", "'1234567890abcdef', 'GCM', 'DEFAULT', " + "unhex('000000000000000000000000')", @@ -1113,7 +1113,7 @@ class CollationSQLExpressionsSuite AesDecryptTestCase("8DE7DB79A23F3E8ED530994DDEA98913", "UTF8_BINARY", "'1234567890abcdef', 'ECB'", "Spark"), AesDecryptTestCase("8DE7DB79A23F3E8ED530994DDEA98913", - "UTF8_BINARY_LCASE", "'1234567890abcdef', 'ECB', 'DEFAULT', ''", "Spark"), + "UTF8_LCASE", "'1234567890abcdef', 'ECB', 'DEFAULT', ''", "Spark"), AesDecryptTestCase("00000000000000000000000046596B2DE09C729FE48A0F81A00A4E7101DABEB61D", "UNICODE", "'1234567890abcdef', 'GCM', 'DEFAULT'", "Spark"), AesDecryptTestCase("000000000000000000000000000000008DE7DB79A23F3E8ED530994DDEA98913", @@ -1136,7 +1136,7 @@ class CollationSQLExpressionsSuite result: R) val testCases = Seq( MaskTestCase("ab-CD-12-@$", null, null, null, null, "UTF8_BINARY", "ab-CD-12-@$"), - MaskTestCase("ab-CD-12-@$", "X", null, null, null, "UTF8_BINARY_LCASE", "ab-XX-12-@$"), + MaskTestCase("ab-CD-12-@$", "X", null, null, null, "UTF8_LCASE", "ab-XX-12-@$"), MaskTestCase("ab-CD-12-@$", "X", "x", null, null, "UNICODE", "xx-XX-12-@$"), MaskTestCase("ab-CD-12-@$", "X", "x", "0", "#", "UNICODE_CI", "xx#XX#00###") ) @@ -1188,7 +1188,7 @@ class CollationSQLExpressionsSuite Row(1), Seq( StructField("a", IntegerType, nullable = true) )), - XmlToStructsTestCase("

true0.8

", "UTF8_BINARY_LCASE", + XmlToStructsTestCase("

true0.8

", "UTF8_LCASE", "'A BOOLEAN, B DOUBLE'", "", Row(true, 0.8), Seq( StructField("A", BooleanType, nullable = true), StructField("B", DoubleType, nullable = true) @@ -1230,7 +1230,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( SchemaOfXmlTestCase("

1

", "UTF8_BINARY", "STRUCT"), - SchemaOfXmlTestCase("

true0.8

", "UTF8_BINARY_LCASE", + SchemaOfXmlTestCase("

true0.8

", "UTF8_LCASE", "STRUCT"), SchemaOfXmlTestCase("

", "UNICODE", "STRUCT<>"), SchemaOfXmlTestCase("

123

", "UNICODE_CI", @@ -1266,7 +1266,7 @@ class CollationSQLExpressionsSuite | 1 | 2 |""".stripMargin), - StructsToXmlTestCase("named_struct('A', true, 'B', 2.0)", "UTF8_BINARY_LCASE", + StructsToXmlTestCase("named_struct('A', true, 'B', 2.0)", "UTF8_LCASE", s""" | true | 2.0 @@ -1304,7 +1304,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( ParseJsonTestCase("{\"a\":1,\"b\":2}", "UTF8_BINARY", "{\"a\":1,\"b\":2}"), - ParseJsonTestCase("{\"A\":3,\"B\":4}", "UTF8_BINARY_LCASE", "{\"A\":3,\"B\":4}"), + ParseJsonTestCase("{\"A\":3,\"B\":4}", "UTF8_LCASE", "{\"A\":3,\"B\":4}"), ParseJsonTestCase("{\"c\":5,\"d\":6}", "UNICODE", "{\"c\":5,\"d\":6}"), ParseJsonTestCase("{\"C\":7,\"D\":8}", "UNICODE_CI", "{\"C\":7,\"D\":8}") ) @@ -1373,7 +1373,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( IsVariantNullTestCase("'null'", "UTF8_BINARY", result = true), - IsVariantNullTestCase("'\"null\"'", "UTF8_BINARY_LCASE", result = false), + IsVariantNullTestCase("'\"null\"'", "UTF8_LCASE", result = false), IsVariantNullTestCase("'13'", "UNICODE", result = false), IsVariantNullTestCase("null", "UNICODE_CI", result = false) ) @@ -1404,7 +1404,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( VariantGetTestCase("{\"a\": 1}", "$.a", "int", "UTF8_BINARY", 1, IntegerType), - VariantGetTestCase("{\"a\": 1}", "$.b", "int", "UTF8_BINARY_LCASE", null, IntegerType), + VariantGetTestCase("{\"a\": 1}", "$.b", "int", "UTF8_LCASE", null, IntegerType), VariantGetTestCase("[1, \"2\"]", "$[1]", "string", "UNICODE", "2", StringType("UNICODE")), VariantGetTestCase("[1, \"2\"]", "$[2]", "string", "UNICODE_CI", null, StringType("UNICODE_CI")) @@ -1482,11 +1482,11 @@ class CollationSQLExpressionsSuite StructField("value", VariantType, nullable = false) ) ), - VariantExplodeTestCase("[\"Spark\", \"SQL\"]", "UTF8_BINARY_LCASE", + VariantExplodeTestCase("[\"Spark\", \"SQL\"]", "UTF8_LCASE", Row(0, "null", "\"Spark\"").toString() + Row(1, "null", "\"SQL\"").toString(), Seq[StructField]( StructField("pos", IntegerType, nullable = false), - StructField("key", StringType("UTF8_BINARY_LCASE")), + StructField("key", StringType("UTF8_LCASE")), StructField("value", VariantType, nullable = false) ) ), @@ -1533,7 +1533,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( SchemaOfVariantTestCase("null", "UTF8_BINARY", "VOID"), - SchemaOfVariantTestCase("[]", "UTF8_BINARY_LCASE", "ARRAY"), + SchemaOfVariantTestCase("[]", "UTF8_LCASE", "ARRAY"), SchemaOfVariantTestCase("[{\"a\":true,\"b\":0}]", "UNICODE", "ARRAY>"), SchemaOfVariantTestCase("[{\"A\":\"x\",\"B\":-1.00}]", "UNICODE_CI", @@ -1564,7 +1564,7 @@ class CollationSQLExpressionsSuite val testCases = Seq( SchemaOfVariantAggTestCase("('1'), ('2'), ('3')", "UTF8_BINARY", "BIGINT"), - SchemaOfVariantAggTestCase("('true'), ('false'), ('true')", "UTF8_BINARY_LCASE", "BOOLEAN"), + SchemaOfVariantAggTestCase("('true'), ('false'), ('true')", "UTF8_LCASE", "BOOLEAN"), SchemaOfVariantAggTestCase("('{\"a\": 1}'), ('{\"b\": true}'), ('{\"c\": 1.23}')", "UNICODE", "STRUCT"), SchemaOfVariantAggTestCase("('{\"A\": \"x\"}'), ('{\"B\": 9.99}'), ('{\"C\": 0}')", @@ -1588,7 +1588,7 @@ class CollationSQLExpressionsSuite test("Support InputFileName expression with collation") { // Supported collations - Seq("UTF8_BINARY", "UTF8_BINARY_LCASE", "UNICODE", "UNICODE_CI").foreach(collationName => { + Seq("UTF8_BINARY", "UTF8_LCASE", "UNICODE", "UNICODE_CI").foreach(collationName => { val query = s""" |select input_file_name() @@ -1607,7 +1607,7 @@ class CollationSQLExpressionsSuite case class DateFormatTestCase[R](date: String, format: String, collation: String, result: R) val testCases = Seq( DateFormatTestCase("2021-01-01", "yyyy-MM-dd", "UTF8_BINARY", "2021-01-01"), - DateFormatTestCase("2021-01-01", "yyyy-dd", "UTF8_BINARY_LCASE", "2021-01"), + DateFormatTestCase("2021-01-01", "yyyy-dd", "UTF8_LCASE", "2021-01"), DateFormatTestCase("2021-01-01", "yyyy-MM-dd", "UNICODE", "2021-01-01"), DateFormatTestCase("2021-01-01", "yyyy", "UNICODE_CI", "2021") ) @@ -1639,11 +1639,11 @@ class CollationSQLExpressionsSuite for { collateKey <- Seq(true, false) collateVal <- Seq(true, false) - defaultCollation <- Seq("UTF8_BINARY", "UTF8_BINARY_LCASE", "UNICODE") + defaultCollation <- Seq("UTF8_BINARY", "UTF8_LCASE", "UNICODE") } { - val mapKey = if (collateKey) "'a' collate utf8_binary_lcase" else "'a'" - val mapVal = if (collateVal) "'b' collate utf8_binary_lcase" else "'b'" - val collation = if (collateVal) "UTF8_BINARY_LCASE" else "UTF8_BINARY" + val mapKey = if (collateKey) "'a' collate utf8_lcase" else "'a'" + val mapVal = if (collateVal) "'b' collate utf8_lcase" else "'b'" + val collation = if (collateVal) "UTF8_LCASE" else "UTF8_BINARY" val queryExtractor = s"select collation(map($mapKey, $mapVal)[$mapKey])" val queryElementAt = s"select collation(element_at(map($mapKey, $mapVal), $mapKey))" @@ -1651,7 +1651,7 @@ class CollationSQLExpressionsSuite checkAnswer(sql(queryElementAt), Row(collation)) withSQLConf(SqlApiConf.DEFAULT_COLLATION -> defaultCollation) { - val res = if (collateVal) "UTF8_BINARY_LCASE" else defaultCollation + val res = if (collateVal) "UTF8_LCASE" else defaultCollation checkAnswer(sql(queryExtractor), Row(res)) checkAnswer(sql(queryElementAt), Row(res)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala index 7d894ac7eb4f7..7405830642796 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala @@ -32,7 +32,7 @@ class CollationSQLRegexpSuite case class LikeTestCase[R](l: String, r: String, c: String, result: R) val testCases = Seq( LikeTestCase("ABC", "%B%", "UTF8_BINARY", true), - LikeTestCase("AḂC", "%ḃ%", "UTF8_BINARY_LCASE", true), + LikeTestCase("AḂC", "%ḃ%", "UTF8_LCASE", true), LikeTestCase("ABC", "%b%", "UTF8_BINARY", false) ) testCases.foreach(t => { @@ -60,7 +60,7 @@ class CollationSQLRegexpSuite case class ILikeTestCase[R](l: String, r: String, c: String, result: R) val testCases = Seq( ILikeTestCase("ABC", "%b%", "UTF8_BINARY", true), - ILikeTestCase("AḂC", "%ḃ%", "UTF8_BINARY_LCASE", true), + ILikeTestCase("AḂC", "%ḃ%", "UTF8_LCASE", true), ILikeTestCase("ABC", "%b%", "UTF8_BINARY", true) ) testCases.foreach(t => { @@ -88,7 +88,7 @@ class CollationSQLRegexpSuite case class LikeAllTestCase[R](s: String, p: Seq[String], c: String, result: R) val testCases = Seq( LikeAllTestCase("foo", Seq("%foo%", "%oo"), "UTF8_BINARY", true), - LikeAllTestCase("Foo", Seq("%foo%", "%oo"), "UTF8_BINARY_LCASE", true), + LikeAllTestCase("Foo", Seq("%foo%", "%oo"), "UTF8_LCASE", true), LikeAllTestCase("foo", Seq("%foo%", "%bar%"), "UTF8_BINARY", false) ) testCases.foreach(t => { @@ -116,7 +116,7 @@ class CollationSQLRegexpSuite case class NotLikeAllTestCase[R](s: String, p: Seq[String], c: String, result: R) val testCases = Seq( NotLikeAllTestCase("foo", Seq("%foo%", "%oo"), "UTF8_BINARY", false), - NotLikeAllTestCase("Foo", Seq("%foo%", "%oo"), "UTF8_BINARY_LCASE", false), + NotLikeAllTestCase("Foo", Seq("%foo%", "%oo"), "UTF8_LCASE", false), NotLikeAllTestCase("foo", Seq("%goo%", "%bar%"), "UTF8_BINARY", true) ) testCases.foreach(t => { @@ -144,7 +144,7 @@ class CollationSQLRegexpSuite case class LikeAnyTestCase[R](s: String, p: Seq[String], c: String, result: R) val testCases = Seq( LikeAnyTestCase("foo", Seq("%foo%", "%bar"), "UTF8_BINARY", true), - LikeAnyTestCase("Foo", Seq("%foo%", "%bar"), "UTF8_BINARY_LCASE", true), + LikeAnyTestCase("Foo", Seq("%foo%", "%bar"), "UTF8_LCASE", true), LikeAnyTestCase("foo", Seq("%goo%", "%hoo%"), "UTF8_BINARY", false) ) testCases.foreach(t => { @@ -172,7 +172,7 @@ class CollationSQLRegexpSuite case class NotLikeAnyTestCase[R](s: String, p: Seq[String], c: String, result: R) val testCases = Seq( NotLikeAnyTestCase("foo", Seq("%foo%", "%hoo"), "UTF8_BINARY", true), - NotLikeAnyTestCase("Foo", Seq("%foo%", "%hoo"), "UTF8_BINARY_LCASE", true), + NotLikeAnyTestCase("Foo", Seq("%foo%", "%hoo"), "UTF8_LCASE", true), NotLikeAnyTestCase("foo", Seq("%foo%", "%oo%"), "UTF8_BINARY", false) ) testCases.foreach(t => { @@ -200,7 +200,7 @@ class CollationSQLRegexpSuite case class RLikeTestCase[R](l: String, r: String, c: String, result: R) val testCases = Seq( RLikeTestCase("ABC", ".B.", "UTF8_BINARY", true), - RLikeTestCase("AḂC", ".ḃ.", "UTF8_BINARY_LCASE", true), + RLikeTestCase("AḂC", ".ḃ.", "UTF8_LCASE", true), RLikeTestCase("ABC", ".b.", "UTF8_BINARY", false) ) testCases.foreach(t => { @@ -228,7 +228,7 @@ class CollationSQLRegexpSuite case class StringSplitTestCase[R](l: String, r: String, c: String, result: R) val testCases = Seq( StringSplitTestCase("ABC", "[B]", "UTF8_BINARY", Seq("A", "C")), - StringSplitTestCase("AḂC", "[ḃ]", "UTF8_BINARY_LCASE", Seq("A", "C")), + StringSplitTestCase("AḂC", "[ḃ]", "UTF8_LCASE", Seq("A", "C")), StringSplitTestCase("ABC", "[B]", "UTF8_BINARY", Seq("A", "C")) ) testCases.foreach(t => { @@ -256,7 +256,7 @@ class CollationSQLRegexpSuite case class RegExpReplaceTestCase[R](l: String, r: String, c: String, result: R) val testCases = Seq( RegExpReplaceTestCase("ABCDE", ".C.", "UTF8_BINARY", "AFFFE"), - RegExpReplaceTestCase("ABĆDE", ".ć.", "UTF8_BINARY_LCASE", "AFFFE"), + RegExpReplaceTestCase("ABĆDE", ".ć.", "UTF8_LCASE", "AFFFE"), RegExpReplaceTestCase("ABCDE", ".c.", "UTF8_BINARY", "ABCDE") ) testCases.foreach(t => { @@ -272,7 +272,7 @@ class CollationSQLRegexpSuite Row(t.result)) }) // Collation mismatch - val (c1, c2) = ("UTF8_BINARY", "UTF8_BINARY_LCASE") + val (c1, c2) = ("UTF8_BINARY", "UTF8_LCASE") val collationMismatch = intercept[AnalysisException] { sql(s"SELECT regexp_replace(collate('ABCDE','$c1'), '.c.', collate('FFF','$c2'))") } @@ -297,7 +297,7 @@ class CollationSQLRegexpSuite case class RegExpExtractTestCase[R](l: String, r: String, c: String, result: R) val testCases = Seq( RegExpExtractTestCase("ABCDE", ".C.", "UTF8_BINARY", "BCD"), - RegExpExtractTestCase("ABĆDE", ".ć.", "UTF8_BINARY_LCASE", "BĆD"), + RegExpExtractTestCase("ABĆDE", ".ć.", "UTF8_LCASE", "BĆD"), RegExpExtractTestCase("ABCDE", ".c.", "UTF8_BINARY", "") ) testCases.foreach(t => { @@ -327,7 +327,7 @@ class CollationSQLRegexpSuite case class RegExpExtractAllTestCase[R](l: String, r: String, c: String, result: R) val testCases = Seq( RegExpExtractAllTestCase("ABCDE", ".C.", "UTF8_BINARY", Seq("BCD")), - RegExpExtractAllTestCase("ABĆDE", ".ć.", "UTF8_BINARY_LCASE", Seq("BĆD")), + RegExpExtractAllTestCase("ABĆDE", ".ć.", "UTF8_LCASE", Seq("BĆD")), RegExpExtractAllTestCase("ABCDE", ".c.", "UTF8_BINARY", Seq()) ) testCases.foreach(t => { @@ -357,7 +357,7 @@ class CollationSQLRegexpSuite case class RegExpCountTestCase[R](l: String, r: String, c: String, result: R) val testCases = Seq( RegExpCountTestCase("ABCDE", ".C.", "UTF8_BINARY", 1), - RegExpCountTestCase("ABĆDE", ".ć.", "UTF8_BINARY_LCASE", 1), + RegExpCountTestCase("ABĆDE", ".ć.", "UTF8_LCASE", 1), RegExpCountTestCase("ABCDE", ".c.", "UTF8_BINARY", 0) ) testCases.foreach(t => { @@ -385,7 +385,7 @@ class CollationSQLRegexpSuite case class RegExpSubStrTestCase[R](l: String, r: String, c: String, result: R) val testCases = Seq( RegExpSubStrTestCase("ABCDE", ".C.", "UTF8_BINARY", "BCD"), - RegExpSubStrTestCase("ABĆDE", ".ć.", "UTF8_BINARY_LCASE", "BĆD"), + RegExpSubStrTestCase("ABĆDE", ".ć.", "UTF8_LCASE", "BĆD"), RegExpSubStrTestCase("ABCDE", ".c.", "UTF8_BINARY", null) ) testCases.foreach(t => { @@ -413,7 +413,7 @@ class CollationSQLRegexpSuite case class RegExpInStrTestCase[R](l: String, r: String, c: String, result: R) val testCases = Seq( RegExpInStrTestCase("ABCDE", ".C.", "UTF8_BINARY", 2), - RegExpInStrTestCase("ABĆDE", ".ć.", "UTF8_BINARY_LCASE", 2), + RegExpInStrTestCase("ABĆDE", ".ć.", "UTF8_LCASE", 2), RegExpInStrTestCase("ABCDE", ".c.", "UTF8_BINARY", 0) ) testCases.foreach(t => { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala index 31be149b9c9cb..556588ecf8d53 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala @@ -35,7 +35,7 @@ class CollationStringExpressionsSuite case class ConcatWsTestCase[R](s: String, a: Array[String], c: String, result: R) val testCases = Seq( ConcatWsTestCase(" ", Array("Spark", "SQL"), "UTF8_BINARY", "Spark SQL"), - ConcatWsTestCase(" ", Array("Spark", "SQL"), "UTF8_BINARY_LCASE", "Spark SQL"), + ConcatWsTestCase(" ", Array("Spark", "SQL"), "UTF8_LCASE", "Spark SQL"), ConcatWsTestCase(" ", Array("Spark", "SQL"), "UNICODE", "Spark SQL"), ConcatWsTestCase(" ", Array("Spark", "SQL"), "UNICODE_CI", "Spark SQL") ) @@ -56,7 +56,7 @@ class CollationStringExpressionsSuite }) // Collation mismatch val collationMismatch = intercept[AnalysisException] { - sql("SELECT concat_ws(' ',collate('Spark', 'UTF8_BINARY_LCASE'),collate('SQL', 'UNICODE'))") + sql("SELECT concat_ws(' ',collate('Spark', 'UTF8_LCASE'),collate('SQL', 'UNICODE'))") } assert(collationMismatch.getErrorClass === "COLLATION_MISMATCH.EXPLICIT") } @@ -66,7 +66,7 @@ class CollationStringExpressionsSuite case class EltTestCase[R](index: Int, inputs: Array[String], c: String, result: R) val testCases = Seq( EltTestCase(1, Array("Spark", "SQL"), "UTF8_BINARY", "Spark"), - EltTestCase(1, Array("Spark", "SQL"), "UTF8_BINARY_LCASE", "Spark"), + EltTestCase(1, Array("Spark", "SQL"), "UTF8_LCASE", "Spark"), EltTestCase(2, Array("Spark", "SQL"), "UNICODE", "SQL"), EltTestCase(2, Array("Spark", "SQL"), "UNICODE_CI", "SQL") ) @@ -86,7 +86,7 @@ class CollationStringExpressionsSuite }) // Collation mismatch val collationMismatch = intercept[AnalysisException] { - sql("SELECT elt(0 ,collate('Spark', 'UTF8_BINARY_LCASE'), collate('SQL', 'UNICODE'))") + sql("SELECT elt(0 ,collate('Spark', 'UTF8_LCASE'), collate('SQL', 'UNICODE'))") } assert(collationMismatch.getErrorClass === "COLLATION_MISMATCH.EXPLICIT") } @@ -97,7 +97,7 @@ class CollationStringExpressionsSuite val testCases = Seq( SplitPartTestCase("1a2", "a", 2, "UTF8_BINARY", "2"), SplitPartTestCase("1a2", "a", 2, "UNICODE", "2"), - SplitPartTestCase("1a2", "A", 2, "UTF8_BINARY_LCASE", "2"), + SplitPartTestCase("1a2", "A", 2, "UTF8_LCASE", "2"), SplitPartTestCase("1a2", "A", 2, "UNICODE_CI", "2") ) testCases.foreach(t => { @@ -114,7 +114,7 @@ class CollationStringExpressionsSuite val testCases = Seq( ContainsTestCase("", "", "UTF8_BINARY", true), ContainsTestCase("abcde", "C", "UNICODE", false), - ContainsTestCase("abcde", "FGH", "UTF8_BINARY_LCASE", false), + ContainsTestCase("abcde", "FGH", "UTF8_LCASE", false), ContainsTestCase("abcde", "BCD", "UNICODE_CI", true) ) testCases.foreach(t => { @@ -128,7 +128,7 @@ class CollationStringExpressionsSuite }) // Collation mismatch val collationMismatch = intercept[AnalysisException] { - sql("SELECT contains(collate('abcde','UTF8_BINARY_LCASE'),collate('C','UNICODE_CI'))") + sql("SELECT contains(collate('abcde','UTF8_LCASE'),collate('C','UNICODE_CI'))") } assert(collationMismatch.getErrorClass === "COLLATION_MISMATCH.EXPLICIT") } @@ -139,7 +139,7 @@ class CollationStringExpressionsSuite val testCases = Seq( SubstringIndexTestCase("wwwgapachegorg", "g", -3, "UTF8_BINARY", "apachegorg"), SubstringIndexTestCase("www||apache||org", "||", 2, "UTF8_BINARY", "www||apache"), - SubstringIndexTestCase("wwwXapacheXorg", "x", 2, "UTF8_BINARY_LCASE", "wwwXapache"), + SubstringIndexTestCase("wwwXapacheXorg", "x", 2, "UTF8_LCASE", "wwwXapache"), SubstringIndexTestCase("aaaaaaaaaa", "aa", 2, "UNICODE", "a"), SubstringIndexTestCase("wwwmapacheMorg", "M", -2, "UNICODE_CI", "apacheMorg") ) @@ -158,7 +158,7 @@ class CollationStringExpressionsSuite }) // Collation mismatch val collationMismatch = intercept[AnalysisException] { - sql("SELECT substring_index(collate('abcde','UTF8_BINARY_LCASE')," + + sql("SELECT substring_index(collate('abcde','UTF8_LCASE')," + "collate('C','UNICODE_CI'),1)") } assert(collationMismatch.getErrorClass === "COLLATION_MISMATCH.EXPLICIT") @@ -169,7 +169,7 @@ class CollationStringExpressionsSuite val testCases = Seq( // scalastyle:off StringInStrTestCase("test大千世界X大千世界", "大千", "UTF8_BINARY", 5), - StringInStrTestCase("test大千世界X大千世界", "界x", "UTF8_BINARY_LCASE", 8), + StringInStrTestCase("test大千世界X大千世界", "界x", "UTF8_LCASE", 8), StringInStrTestCase("test大千世界X大千世界", "界x", "UNICODE", 0), StringInStrTestCase("test大千世界X大千世界", "界y", "UNICODE_CI", 0), StringInStrTestCase("test大千世界X大千世界", "界x", "UNICODE_CI", 8), @@ -190,7 +190,7 @@ class CollationStringExpressionsSuite }) // Collation mismatch val collationMismatch = intercept[AnalysisException] { - sql(s"SELECT instr(collate('aaads','UTF8_BINARY'), collate('Aa','UTF8_BINARY_LCASE'))") + sql(s"SELECT instr(collate('aaads','UTF8_BINARY'), collate('Aa','UTF8_LCASE'))") } assert(collationMismatch.getErrorClass === "COLLATION_MISMATCH.EXPLICIT") } @@ -199,7 +199,7 @@ class CollationStringExpressionsSuite case class FindInSetTestCase[R](word: String, set: String, c: String, result: R) val testCases = Seq( FindInSetTestCase("AB", "abc,b,ab,c,def", "UTF8_BINARY", 0), - FindInSetTestCase("C", "abc,b,ab,c,def", "UTF8_BINARY_LCASE", 4), + FindInSetTestCase("C", "abc,b,ab,c,def", "UTF8_LCASE", 4), FindInSetTestCase("d,ef", "abc,b,ab,c,def", "UNICODE", 0), // scalastyle:off FindInSetTestCase("i̇o", "ab,İo,12", "UNICODE_CI", 2), @@ -221,7 +221,7 @@ class CollationStringExpressionsSuite // Collation mismatch val collationMismatch = intercept[AnalysisException] { sql(s"SELECT find_in_set(collate('AB','UTF8_BINARY')," + - s"collate('ab,xyz,fgh','UTF8_BINARY_LCASE'))") + s"collate('ab,xyz,fgh','UTF8_LCASE'))") } assert(collationMismatch.getErrorClass === "COLLATION_MISMATCH.EXPLICIT") } @@ -232,7 +232,7 @@ class CollationStringExpressionsSuite val testCases = Seq( StartsWithTestCase("", "", "UTF8_BINARY", true), StartsWithTestCase("abcde", "A", "UNICODE", false), - StartsWithTestCase("abcde", "FGH", "UTF8_BINARY_LCASE", false), + StartsWithTestCase("abcde", "FGH", "UTF8_LCASE", false), StartsWithTestCase("abcde", "ABC", "UNICODE_CI", true) ) testCases.foreach(t => { @@ -246,7 +246,7 @@ class CollationStringExpressionsSuite }) // Collation mismatch val collationMismatch = intercept[AnalysisException] { - sql("SELECT startswith(collate('abcde', 'UTF8_BINARY_LCASE'),collate('C', 'UNICODE_CI'))") + sql("SELECT startswith(collate('abcde', 'UTF8_LCASE'),collate('C', 'UNICODE_CI'))") } assert(collationMismatch.getErrorClass === "COLLATION_MISMATCH.EXPLICIT") } @@ -255,17 +255,17 @@ class CollationStringExpressionsSuite case class TranslateTestCase[R](input: String, matchExpression: String, replaceExpression: String, collation: String, result: R) val testCases = Seq( - TranslateTestCase("Translate", "Rnlt", "1234", "UTF8_BINARY_LCASE", "41a2s3a4e"), - TranslateTestCase("Translate", "Rnlt", "1234", "UTF8_BINARY_LCASE", "41a2s3a4e"), - TranslateTestCase("TRanslate", "rnlt", "XxXx", "UTF8_BINARY_LCASE", "xXaxsXaxe"), - TranslateTestCase("TRanslater", "Rrnlt", "xXxXx", "UTF8_BINARY_LCASE", "xxaxsXaxex"), - TranslateTestCase("TRanslater", "Rrnlt", "XxxXx", "UTF8_BINARY_LCASE", "xXaxsXaxeX"), + TranslateTestCase("Translate", "Rnlt", "1234", "UTF8_LCASE", "41a2s3a4e"), + TranslateTestCase("Translate", "Rnlt", "1234", "UTF8_LCASE", "41a2s3a4e"), + TranslateTestCase("TRanslate", "rnlt", "XxXx", "UTF8_LCASE", "xXaxsXaxe"), + TranslateTestCase("TRanslater", "Rrnlt", "xXxXx", "UTF8_LCASE", "xxaxsXaxex"), + TranslateTestCase("TRanslater", "Rrnlt", "XxxXx", "UTF8_LCASE", "xXaxsXaxeX"), // scalastyle:off - TranslateTestCase("test大千世界X大千世界", "界x", "AB", "UTF8_BINARY_LCASE", "test大千世AB大千世A"), - TranslateTestCase("大千世界test大千世界", "TEST", "abcd", "UTF8_BINARY_LCASE", "大千世界abca大千世界"), - TranslateTestCase("Test大千世界大千世界", "tT", "oO", "UTF8_BINARY_LCASE", "oeso大千世界大千世界"), - TranslateTestCase("大千世界大千世界tesT", "Tt", "Oo", "UTF8_BINARY_LCASE", "大千世界大千世界OesO"), - TranslateTestCase("大千世界大千世界tesT", "大千", "世世", "UTF8_BINARY_LCASE", "世世世界世世世界tesT"), + TranslateTestCase("test大千世界X大千世界", "界x", "AB", "UTF8_LCASE", "test大千世AB大千世A"), + TranslateTestCase("大千世界test大千世界", "TEST", "abcd", "UTF8_LCASE", "大千世界abca大千世界"), + TranslateTestCase("Test大千世界大千世界", "tT", "oO", "UTF8_LCASE", "oeso大千世界大千世界"), + TranslateTestCase("大千世界大千世界tesT", "Tt", "Oo", "UTF8_LCASE", "大千世界大千世界OesO"), + TranslateTestCase("大千世界大千世界tesT", "大千", "世世", "UTF8_LCASE", "世世世界世世世界tesT"), // scalastyle:on TranslateTestCase("Translate", "Rnlt", "1234", "UNICODE", "Tra2s3a4e"), TranslateTestCase("TRanslate", "rnlt", "XxXx", "UNICODE", "TRaxsXaxe"), @@ -287,16 +287,16 @@ class CollationStringExpressionsSuite TranslateTestCase("大千世界大千世界tesT", "Tt", "Oo", "UNICODE_CI", "大千世界大千世界OesO"), TranslateTestCase("大千世界大千世界tesT", "大千", "世世", "UNICODE_CI", "世世世界世世世界tesT"), // scalastyle:on - TranslateTestCase("Translate", "Rnlasdfjhgadt", "1234", "UTF8_BINARY_LCASE", "14234e"), + TranslateTestCase("Translate", "Rnlasdfjhgadt", "1234", "UTF8_LCASE", "14234e"), TranslateTestCase("Translate", "Rnlasdfjhgadt", "1234", "UNICODE_CI", "14234e"), TranslateTestCase("Translate", "Rnlasdfjhgadt", "1234", "UNICODE", "Tr4234e"), TranslateTestCase("Translate", "Rnlasdfjhgadt", "1234", "UTF8_BINARY", "Tr4234e"), - TranslateTestCase("Translate", "Rnlt", "123495834634", "UTF8_BINARY_LCASE", "41a2s3a4e"), + TranslateTestCase("Translate", "Rnlt", "123495834634", "UTF8_LCASE", "41a2s3a4e"), TranslateTestCase("Translate", "Rnlt", "123495834634", "UNICODE", "Tra2s3a4e"), TranslateTestCase("Translate", "Rnlt", "123495834634", "UNICODE_CI", "41a2s3a4e"), TranslateTestCase("Translate", "Rnlt", "123495834634", "UTF8_BINARY", "Tra2s3a4e"), TranslateTestCase("abcdef", "abcde", "123", "UTF8_BINARY", "123f"), - TranslateTestCase("abcdef", "abcde", "123", "UTF8_BINARY_LCASE", "123f"), + TranslateTestCase("abcdef", "abcde", "123", "UTF8_LCASE", "123f"), TranslateTestCase("abcdef", "abcde", "123", "UNICODE", "123f"), TranslateTestCase("abcdef", "abcde", "123", "UNICODE_CI", "123f") ) @@ -319,7 +319,7 @@ class CollationStringExpressionsSuite }) // Collation mismatch val collationMismatch = intercept[AnalysisException] { - sql(s"SELECT translate(collate('Translate', 'UTF8_BINARY_LCASE')," + + sql(s"SELECT translate(collate('Translate', 'UTF8_LCASE')," + s"collate('Rnlt', 'UNICODE'), '1234')") } assert(collationMismatch.getErrorClass === "COLLATION_MISMATCH.EXPLICIT") @@ -331,7 +331,7 @@ class CollationStringExpressionsSuite val testCases = Seq( // scalastyle:off ReplaceTestCase("r世eplace", "pl", "123", "UTF8_BINARY", "r世e123ace"), - ReplaceTestCase("repl世ace", "PL", "AB", "UTF8_BINARY_LCASE", "reAB世ace"), + ReplaceTestCase("repl世ace", "PL", "AB", "UTF8_LCASE", "reAB世ace"), ReplaceTestCase("abcdabcd", "bc", "", "UNICODE", "adad"), ReplaceTestCase("aBc世abc", "b", "12", "UNICODE_CI", "a12c世a12c"), ReplaceTestCase("abi̇o12i̇o", "İo", "yy", "UNICODE_CI", "abyy12yy"), @@ -355,7 +355,7 @@ class CollationStringExpressionsSuite }) // Collation mismatch val collationMismatch = intercept[AnalysisException] { - sql("SELECT startswith(collate('abcde', 'UTF8_BINARY_LCASE'),collate('C', 'UNICODE_CI'))") + sql("SELECT startswith(collate('abcde', 'UTF8_LCASE'),collate('C', 'UNICODE_CI'))") } assert(collationMismatch.getErrorClass === "COLLATION_MISMATCH.EXPLICIT") } @@ -366,7 +366,7 @@ class CollationStringExpressionsSuite val testCases = Seq( EndsWithTestCase("", "", "UTF8_BINARY", true), EndsWithTestCase("abcde", "E", "UNICODE", false), - EndsWithTestCase("abcde", "FGH", "UTF8_BINARY_LCASE", false), + EndsWithTestCase("abcde", "FGH", "UTF8_LCASE", false), EndsWithTestCase("abcde", "CDE", "UNICODE_CI", true) ) testCases.foreach(t => { @@ -380,7 +380,7 @@ class CollationStringExpressionsSuite }) // Collation mismatch val collationMismatch = intercept[AnalysisException] { - sql("SELECT endswith(collate('abcde', 'UTF8_BINARY_LCASE'),collate('C', 'UNICODE_CI'))") + sql("SELECT endswith(collate('abcde', 'UTF8_LCASE'),collate('C', 'UNICODE_CI'))") } assert(collationMismatch.getErrorClass === "COLLATION_MISMATCH.EXPLICIT") } @@ -391,7 +391,7 @@ class CollationStringExpressionsSuite val testCases = Seq( StringRepeatTestCase("", 1, "UTF8_BINARY", ""), StringRepeatTestCase("a", 0, "UNICODE", ""), - StringRepeatTestCase("XY", 3, "UTF8_BINARY_LCASE", "XYXYXY"), + StringRepeatTestCase("XY", 3, "UTF8_LCASE", "XYXYXY"), StringRepeatTestCase("123", 2, "UNICODE_CI", "123123") ) testCases.foreach(t => { @@ -406,16 +406,16 @@ class CollationStringExpressionsSuite case class AsciiUnBase64TestCase[R](q: String, dt: DataType, r: R) val testCases = Seq( AsciiUnBase64TestCase("select ascii('a' collate utf8_binary)", IntegerType, 97), - AsciiUnBase64TestCase("select ascii('B' collate utf8_binary_lcase)", IntegerType, 66), + AsciiUnBase64TestCase("select ascii('B' collate utf8_lcase)", IntegerType, 66), AsciiUnBase64TestCase("select ascii('#' collate unicode)", IntegerType, 35), AsciiUnBase64TestCase("select ascii('!' collate unicode_ci)", IntegerType, 33), AsciiUnBase64TestCase("select unbase64('QUJD' collate utf8_binary)", BinaryType, Seq(65, 66, 67)), - AsciiUnBase64TestCase("select unbase64('eHl6' collate utf8_binary_lcase)", BinaryType, + AsciiUnBase64TestCase("select unbase64('eHl6' collate utf8_lcase)", BinaryType, Seq(120, 121, 122)), AsciiUnBase64TestCase("select unbase64('IyMj' collate utf8_binary)", BinaryType, Seq(35, 35, 35)), - AsciiUnBase64TestCase("select unbase64('IQ==' collate utf8_binary_lcase)", BinaryType, + AsciiUnBase64TestCase("select unbase64('IQ==' collate utf8_lcase)", BinaryType, Seq(33)) ) testCases.foreach(t => { @@ -429,11 +429,11 @@ class CollationStringExpressionsSuite case class DefaultCollationTestCase[R](q: String, c: String, r: R) val testCases = Seq( DefaultCollationTestCase("select chr(97)", "UTF8_BINARY", "a"), - DefaultCollationTestCase("select chr(66)", "UTF8_BINARY_LCASE", "B"), + DefaultCollationTestCase("select chr(66)", "UTF8_LCASE", "B"), DefaultCollationTestCase("select base64('xyz')", "UNICODE", "eHl6"), DefaultCollationTestCase("select base64('!')", "UNICODE_CI", "IQ=="), DefaultCollationTestCase("select decode(encode('$', 'utf-8'), 'utf-8')", "UTF8_BINARY", "$"), - DefaultCollationTestCase("select decode(encode('X', 'utf-8'), 'utf-8')", "UTF8_BINARY_LCASE", + DefaultCollationTestCase("select decode(encode('X', 'utf-8'), 'utf-8')", "UTF8_LCASE", "X"), DefaultCollationTestCase("select format_number(123.123, '###.###')", "UNICODE", "123.123"), DefaultCollationTestCase("select format_number(99.99, '##.##')", "UNICODE_CI", "99.99") @@ -452,7 +452,7 @@ class CollationStringExpressionsSuite val testCases = Seq( EncodeToBinarySentencesTestCase("select encode('a' collate utf8_binary, 'utf-8')", BinaryType, Seq(97)), - EncodeToBinarySentencesTestCase("select encode('$' collate utf8_binary_lcase, 'utf-8')", + EncodeToBinarySentencesTestCase("select encode('$' collate utf8_lcase, 'utf-8')", BinaryType, Seq(36)), EncodeToBinarySentencesTestCase("select to_binary('B' collate unicode, 'utf-8')", BinaryType, Seq(66)), @@ -465,9 +465,9 @@ class CollationStringExpressionsSuite ArrayType(ArrayType(StringType)), Seq(Seq("Hello", "world"), Seq("Nice", "day"))), EncodeToBinarySentencesTestCase( """ - |select sentences('Something else. Nothing here.' collate utf8_binary_lcase) + |select sentences('Something else. Nothing here.' collate utf8_lcase) |""".stripMargin, - ArrayType(ArrayType(StringType("UTF8_BINARY_LCASE"))), + ArrayType(ArrayType(StringType("UTF8_LCASE"))), Seq(Seq("Something", "else"), Seq("Nothing", "here"))) ) testCases.foreach(t => { @@ -482,7 +482,7 @@ class CollationStringExpressionsSuite case class UpperTestCase[R](s: String, c: String, result: R) val testCases = Seq( UpperTestCase("aBc", "UTF8_BINARY", "ABC"), - UpperTestCase("aBc", "UTF8_BINARY_LCASE", "ABC"), + UpperTestCase("aBc", "UTF8_LCASE", "ABC"), UpperTestCase("aBc", "UNICODE", "ABC"), UpperTestCase("aBc", "UNICODE_CI", "ABC") ) @@ -499,7 +499,7 @@ class CollationStringExpressionsSuite case class LowerTestCase[R](s: String, c: String, result: R) val testCases = Seq( LowerTestCase("aBc", "UTF8_BINARY", "abc"), - LowerTestCase("aBc", "UTF8_BINARY_LCASE", "abc"), + LowerTestCase("aBc", "UTF8_LCASE", "abc"), LowerTestCase("aBc", "UNICODE", "abc"), LowerTestCase("aBc", "UNICODE_CI", "abc") ) @@ -516,7 +516,7 @@ class CollationStringExpressionsSuite case class InitCapTestCase[R](s: String, c: String, result: R) val testCases = Seq( InitCapTestCase("aBc ABc", "UTF8_BINARY", "Abc Abc"), - InitCapTestCase("aBc ABc", "UTF8_BINARY_LCASE", "Abc Abc"), + InitCapTestCase("aBc ABc", "UTF8_LCASE", "Abc Abc"), InitCapTestCase("aBc ABc", "UNICODE", "Abc Abc"), InitCapTestCase("aBc ABc", "UNICODE_CI", "Abc Abc") ) @@ -533,7 +533,7 @@ class CollationStringExpressionsSuite case class OverlayTestCase(l: String, r: String, pos: Int, c: String, result: String) val testCases = Seq( OverlayTestCase("hello", " world", 6, "UTF8_BINARY", "hello world"), - OverlayTestCase("nice", " day", 5, "UTF8_BINARY_LCASE", "nice day"), + OverlayTestCase("nice", " day", 5, "UTF8_LCASE", "nice day"), OverlayTestCase("A", "B", 1, "UNICODE", "B"), OverlayTestCase("!", "!!!", 1, "UNICODE_CI", "!!!") ) @@ -574,7 +574,7 @@ class CollationStringExpressionsSuite case class FormatStringTestCase(f: String, a: Seq[Any], c: String, r: String) val testCases = Seq( FormatStringTestCase("%s%s", Seq("'a'", "'b'"), "UTF8_BINARY", "ab"), - FormatStringTestCase("%d", Seq(123), "UTF8_BINARY_LCASE", "123"), + FormatStringTestCase("%d", Seq(123), "UTF8_LCASE", "123"), FormatStringTestCase("%s%d", Seq("'A'", 0), "UNICODE", "A0"), FormatStringTestCase("%s%s", Seq("'Hello'", "'!!!'"), "UNICODE_CI", "Hello!!!") ) @@ -594,7 +594,7 @@ class CollationStringExpressionsSuite case class SoundExTestCase(q: String, c: String, r: String) val testCases = Seq( SoundExTestCase("select soundex('A' collate utf8_binary)", "UTF8_BINARY", "A000"), - SoundExTestCase("select soundex('!' collate utf8_binary_lcase)", "UTF8_BINARY_LCASE", "!"), + SoundExTestCase("select soundex('!' collate utf8_lcase)", "UTF8_LCASE", "!"), SoundExTestCase("select soundex('$' collate unicode)", "UNICODE", "$"), SoundExTestCase("select soundex('X' collate unicode_ci)", "UNICODE_CI", "X000") ) @@ -612,11 +612,11 @@ class CollationStringExpressionsSuite case class LenTestCase(q: String, r: Int) val testCases = Seq( LenTestCase("select length('hello' collate utf8_binary)", 5), - LenTestCase("select length('world' collate utf8_binary_lcase)", 5), + LenTestCase("select length('world' collate utf8_lcase)", 5), LenTestCase("select length('ff' collate unicode)", 1), LenTestCase("select bit_length('hello' collate unicode_ci)", 40), LenTestCase("select bit_length('world' collate utf8_binary)", 40), - LenTestCase("select bit_length('ff' collate utf8_binary_lcase)", 24), + LenTestCase("select bit_length('ff' collate utf8_lcase)", 24), LenTestCase("select octet_length('hello' collate unicode)", 5), LenTestCase("select octet_length('world' collate unicode_ci)", 5), LenTestCase("select octet_length('ff' collate utf8_binary)", 3) @@ -633,7 +633,7 @@ class CollationStringExpressionsSuite case class LuhncheckTestCase(q: String, c: String, r: Boolean) val testCases = Seq( LuhncheckTestCase("123", "UTF8_BINARY", r = false), - LuhncheckTestCase("000", "UTF8_BINARY_LCASE", r = true), + LuhncheckTestCase("000", "UTF8_LCASE", r = true), LuhncheckTestCase("111", "UNICODE", r = false), LuhncheckTestCase("222", "UNICODE_CI", r = false) ) @@ -682,28 +682,28 @@ class CollationStringExpressionsSuite } val checks = Seq( - SubstringTestCase("substr", "example", "1", Some("100"), "utf8_binary_lcase", Row("example")), + SubstringTestCase("substr", "example", "1", Some("100"), "utf8_lcase", Row("example")), SubstringTestCase("substr", "example", "2", Some("2"), "utf8_binary", Row("xa")), - SubstringTestCase("right", "", "1", None, "utf8_binary_lcase", Row("")), + SubstringTestCase("right", "", "1", None, "utf8_lcase", Row("")), SubstringTestCase("substr", "example", "0", Some("0"), "unicode", Row("")), SubstringTestCase("substr", "example", "-3", Some("2"), "unicode_ci", Row("pl")), - SubstringTestCase("substr", " a世a ", "2", Some("3"), "utf8_binary_lcase", Row("a世a")), + SubstringTestCase("substr", " a世a ", "2", Some("3"), "utf8_lcase", Row("a世a")), SubstringTestCase("left", " a世a ", "3", None, "utf8_binary", Row(" a世")), SubstringTestCase("right", " a世a ", "3", None, "unicode", Row("世a ")), SubstringTestCase("left", "ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ", "3", None, "unicode_ci", Row("ÀÃÂ")), - SubstringTestCase("right", "ÀÃÂĀĂȦÄäâãȻȻȻȻȻǢǼÆ", "3", None, "utf8_binary_lcase", Row("ǢǼÆ")), - SubstringTestCase("substr", "", "1", Some("1"), "utf8_binary_lcase", Row("")), + SubstringTestCase("right", "ÀÃÂĀĂȦÄäâãȻȻȻȻȻǢǼÆ", "3", None, "utf8_lcase", Row("ǢǼÆ")), + SubstringTestCase("substr", "", "1", Some("1"), "utf8_lcase", Row("")), SubstringTestCase("substr", "", "1", Some("1"), "unicode", Row("")), SubstringTestCase("left", "", "1", None, "utf8_binary", Row("")), - SubstringTestCase("left", "null", "1", None, "utf8_binary_lcase", Row(null)), + SubstringTestCase("left", "null", "1", None, "utf8_lcase", Row(null)), SubstringTestCase("right", "null", "1", None, "unicode", Row(null)), SubstringTestCase("substr", "null", "1", None, "utf8_binary", Row(null)), SubstringTestCase("substr", "null", "1", Some("1"), "unicode_ci", Row(null)), - SubstringTestCase("left", "null", "null", None, "utf8_binary_lcase", Row(null)), + SubstringTestCase("left", "null", "null", None, "utf8_lcase", Row(null)), SubstringTestCase("right", "null", "null", None, "unicode", Row(null)), SubstringTestCase("substr", "null", "null", Some("null"), "utf8_binary", Row(null)), SubstringTestCase("substr", "null", "null", None, "unicode_ci", Row(null)), - SubstringTestCase("left", "ÀÃÂȦÄäåäáâãȻȻȻǢǼÆ", "null", None, "utf8_binary_lcase", Row(null)), + SubstringTestCase("left", "ÀÃÂȦÄäåäáâãȻȻȻǢǼÆ", "null", None, "utf8_lcase", Row(null)), SubstringTestCase("right", "ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ", "null", None, "unicode", Row(null)), SubstringTestCase("substr", "ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ", "null", None, "utf8_binary", Row(null)), SubstringTestCase("substr", "", "null", None, "unicode_ci", Row(null)) @@ -722,10 +722,10 @@ class CollationStringExpressionsSuite val testCases = Seq( StringRPadTestCase("", 5, " ", "UTF8_BINARY", " "), StringRPadTestCase("abc", 5, " ", "UNICODE", "abc "), - StringRPadTestCase("Hello", 7, "Wörld", "UTF8_BINARY_LCASE", "HelloWö"), + StringRPadTestCase("Hello", 7, "Wörld", "UTF8_LCASE", "HelloWö"), StringRPadTestCase("1234567890", 5, "aaaAAa", "UNICODE_CI", "12345"), StringRPadTestCase("aaAA", 2, " ", "UTF8_BINARY", "aa"), - StringRPadTestCase("ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ℀℃", 2, "1", "UTF8_BINARY_LCASE", "ÀÃ"), + StringRPadTestCase("ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ℀℃", 2, "1", "UTF8_LCASE", "ÀÃ"), StringRPadTestCase("ĂȦÄäåäá", 20, "ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ", "UNICODE", "ĂȦÄäåäáÀÃÂĀĂȦÄäåäáâã"), StringRPadTestCase("aȦÄä", 8, "a1", "UNICODE_CI", "aȦÄäa1a1") ) @@ -745,7 +745,7 @@ class CollationStringExpressionsSuite }) // Collation mismatch val collationMismatch = intercept[AnalysisException] { - sql("SELECT rpad(collate('abcde', 'UNICODE_CI'),1,collate('C', 'UTF8_BINARY_LCASE'))") + sql("SELECT rpad(collate('abcde', 'UNICODE_CI'),1,collate('C', 'UTF8_LCASE'))") } assert(collationMismatch.getErrorClass === "COLLATION_MISMATCH.EXPLICIT") } @@ -756,10 +756,10 @@ class CollationStringExpressionsSuite val testCases = Seq( StringLPadTestCase("", 5, " ", "UTF8_BINARY", " "), StringLPadTestCase("abc", 5, " ", "UNICODE", " abc"), - StringLPadTestCase("Hello", 7, "Wörld", "UTF8_BINARY_LCASE", "WöHello"), + StringLPadTestCase("Hello", 7, "Wörld", "UTF8_LCASE", "WöHello"), StringLPadTestCase("1234567890", 5, "aaaAAa", "UNICODE_CI", "12345"), StringLPadTestCase("aaAA", 2, " ", "UTF8_BINARY", "aa"), - StringLPadTestCase("ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ℀℃", 2, "1", "UTF8_BINARY_LCASE", "ÀÃ"), + StringLPadTestCase("ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ℀℃", 2, "1", "UTF8_LCASE", "ÀÃ"), StringLPadTestCase("ĂȦÄäåäá", 20, "ÀÃÂĀĂȦÄäåäáâãȻȻȻȻȻǢǼÆ", "UNICODE", "ÀÃÂĀĂȦÄäåäáâãĂȦÄäåäá"), StringLPadTestCase("aȦÄä", 8, "a1", "UNICODE_CI", "a1a1aȦÄä") ) @@ -779,7 +779,7 @@ class CollationStringExpressionsSuite }) // Collation mismatch val collationMismatch = intercept[AnalysisException] { - sql("SELECT lpad(collate('abcde', 'UNICODE_CI'),1,collate('C', 'UTF8_BINARY_LCASE'))") + sql("SELECT lpad(collate('abcde', 'UNICODE_CI'),1,collate('C', 'UTF8_LCASE'))") } assert(collationMismatch.getErrorClass === "COLLATION_MISMATCH.EXPLICIT") } @@ -796,9 +796,9 @@ class CollationStringExpressionsSuite val testCases = Seq( // scalastyle:off StringLocateTestCase("aa", "aaads", 0, "UTF8_BINARY", 0), - StringLocateTestCase("aa", "Aaads", 0, "UTF8_BINARY_LCASE", 0), - StringLocateTestCase("界x", "test大千世界X大千世界", 1, "UTF8_BINARY_LCASE", 8), - StringLocateTestCase("aBc", "abcabc", 4, "UTF8_BINARY_LCASE", 4), + StringLocateTestCase("aa", "Aaads", 0, "UTF8_LCASE", 0), + StringLocateTestCase("界x", "test大千世界X大千世界", 1, "UTF8_LCASE", 8), + StringLocateTestCase("aBc", "abcabc", 4, "UTF8_LCASE", 4), StringLocateTestCase("aa", "Aaads", 0, "UNICODE", 0), StringLocateTestCase("abC", "abCabC", 2, "UNICODE", 4), StringLocateTestCase("aa", "Aaads", 0, "UNICODE_CI", 0), @@ -819,7 +819,7 @@ class CollationStringExpressionsSuite }) // Collation mismatch val collationMismatch = intercept[AnalysisException] { - sql("SELECT locate(collate('aBc', 'UTF8_BINARY'),collate('abcabc', 'UTF8_BINARY_LCASE'),4)") + sql("SELECT locate(collate('aBc', 'UTF8_BINARY'),collate('abcabc', 'UTF8_LCASE'),4)") } assert(collationMismatch.getErrorClass === "COLLATION_MISMATCH.EXPLICIT") } @@ -829,7 +829,7 @@ class CollationStringExpressionsSuite checkEvaluation( StringTrim(Literal.create( " asd ", StringType("UTF8_BINARY"))), "asd") checkEvaluation( - StringTrimLeft(Literal.create(" asd ", StringType("UTF8_BINARY_LCASE"))), "asd ") + StringTrimLeft(Literal.create(" asd ", StringType("UTF8_LCASE"))), "asd ") checkEvaluation(StringTrimRight( Literal.create(" asd ", StringType("UTF8_BINARY"))), " asd") @@ -841,8 +841,8 @@ class CollationStringExpressionsSuite "asd") checkEvaluation( StringTrimLeft( - Literal.create(" asd ", StringType("UTF8_BINARY_LCASE")), - Literal.create(" ", StringType("UTF8_BINARY_LCASE"))), + Literal.create(" asd ", StringType("UTF8_LCASE")), + Literal.create(" ", StringType("UTF8_LCASE"))), "asd ") checkEvaluation( StringTrimRight( @@ -857,8 +857,8 @@ class CollationStringExpressionsSuite "asd") checkEvaluation( StringTrimLeft( - Literal.create("xxasdxx", StringType("UTF8_BINARY_LCASE")), - Literal.create("x", StringType("UTF8_BINARY_LCASE"))), + Literal.create("xxasdxx", StringType("UTF8_LCASE")), + Literal.create("x", StringType("UTF8_LCASE"))), "asdxx") checkEvaluation( StringTrimRight( @@ -882,10 +882,10 @@ class CollationStringExpressionsSuite StringTrimTestCase("UTF8_BINARY", "LTRIM", "xxasdxx", true, "x", "asdxx"), StringTrimTestCase("UTF8_BINARY", "RTRIM", "xxasdxx", true, "x", "xxasd"), - StringTrimTestCase("UTF8_BINARY_LCASE", "TRIM", " asd ", true, null, null), - StringTrimTestCase("UTF8_BINARY_LCASE", "BTRIM", "xxasdxx", true, "x", "asd"), - StringTrimTestCase("UTF8_BINARY_LCASE", "LTRIM", "xxasdxx", true, "x", "asdxx"), - StringTrimTestCase("UTF8_BINARY_LCASE", "RTRIM", " asd ", false, null, " asd"), + StringTrimTestCase("UTF8_LCASE", "TRIM", " asd ", true, null, null), + StringTrimTestCase("UTF8_LCASE", "BTRIM", "xxasdxx", true, "x", "asd"), + StringTrimTestCase("UTF8_LCASE", "LTRIM", "xxasdxx", true, "x", "asdxx"), + StringTrimTestCase("UTF8_LCASE", "RTRIM", " asd ", false, null, " asd"), StringTrimTestCase("UTF8_BINARY", "TRIM", "xxasdxx", true, "x", "asd"), StringTrimTestCase("UTF8_BINARY", "BTRIM", "xxasdxx", true, "x", "asd"), @@ -926,8 +926,8 @@ class CollationStringExpressionsSuite df = sql("SELECT TRIM(COLLATE('x', 'UTF8_BINARY'), COLLATE('xax', 'UTF8_BINARY'))"), expectedAnswer = Row("a")) checkAnswer( - df = sql("SELECT BTRIM(COLLATE('xax', 'UTF8_BINARY_LCASE'), " - + "COLLATE('x', 'UTF8_BINARY_LCASE'))"), + df = sql("SELECT BTRIM(COLLATE('xax', 'UTF8_LCASE'), " + + "COLLATE('x', 'UTF8_LCASE'))"), expectedAnswer = Row("a")) checkAnswer( df = sql("SELECT LTRIM(COLLATE('x', 'UTF8_BINARY'), COLLATE('xax', 'UTF8_BINARY'))"), @@ -937,7 +937,7 @@ class CollationStringExpressionsSuite df = sql("SELECT RTRIM('x', COLLATE('xax', 'UTF8_BINARY'))"), expectedAnswer = Row("xa")) checkAnswer( - df = sql("SELECT TRIM('x', COLLATE('xax', 'UTF8_BINARY_LCASE'))"), + df = sql("SELECT TRIM('x', COLLATE('xax', 'UTF8_LCASE'))"), expectedAnswer = Row("a")) checkAnswer( df = sql("SELECT BTRIM('xax', COLLATE('x', 'UTF8_BINARY'))"), @@ -947,7 +947,7 @@ class CollationStringExpressionsSuite df = sql("SELECT LTRIM(COLLATE('x', 'UTF8_BINARY'), 'xax')"), expectedAnswer = Row("ax")) checkAnswer( - df = sql("SELECT RTRIM(COLLATE('x', 'UTF8_BINARY_LCASE'), 'xax')"), + df = sql("SELECT RTRIM(COLLATE('x', 'UTF8_LCASE'), 'xax')"), expectedAnswer = Row("xa")) checkAnswer( df = sql("SELECT TRIM(COLLATE('x', 'UTF8_BINARY'), 'xax')"), @@ -957,14 +957,14 @@ class CollationStringExpressionsSuite test("StringTrim* functions - collation type mismatch") { List("TRIM", "LTRIM", "RTRIM").foreach(func => { val collationMismatch = intercept[AnalysisException] { - sql("SELECT " + func + "(COLLATE('x', 'UTF8_BINARY_LCASE'), " + sql("SELECT " + func + "(COLLATE('x', 'UTF8_LCASE'), " + "COLLATE('xxaaaxx', 'UTF8_BINARY'))") } assert(collationMismatch.getErrorClass === "COLLATION_MISMATCH.EXPLICIT") }) val collationMismatch = intercept[AnalysisException] { - sql("SELECT BTRIM(COLLATE('xxaaaxx', 'UTF8_BINARY'), COLLATE('x', 'UTF8_BINARY_LCASE'))") + sql("SELECT BTRIM(COLLATE('xxaaaxx', 'UTF8_BINARY'), COLLATE('x', 'UTF8_LCASE'))") } assert(collationMismatch.getErrorClass === "COLLATION_MISMATCH.EXPLICIT") } 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 bc97f0f023361..e2a11fc137c3d 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 @@ -43,7 +43,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { private val allFileBasedDataSources = collationPreservingSources ++ collationNonPreservingSources test("collate returns proper type") { - Seq("utf8_binary", "utf8_binary_lcase", "unicode", "unicode_ci").foreach { collationName => + Seq("utf8_binary", "utf8_lcase", "unicode", "unicode_ci").foreach { collationName => checkAnswer(sql(s"select 'aaa' collate $collationName"), Row("aaa")) val collationId = CollationFactory.collationNameToId(collationName) assert(sql(s"select 'aaa' collate $collationName").schema(0).dataType @@ -52,7 +52,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } test("collation name is case insensitive") { - Seq("uTf8_BiNaRy", "uTf8_BiNaRy_Lcase", "uNicOde", "UNICODE_ci").foreach { collationName => + Seq("uTf8_BiNaRy", "utf8_lcase", "uNicOde", "UNICODE_ci").foreach { collationName => checkAnswer(sql(s"select 'aaa' collate $collationName"), Row("aaa")) val collationId = CollationFactory.collationNameToId(collationName) assert(sql(s"select 'aaa' collate $collationName").schema(0).dataType @@ -61,7 +61,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } test("collation expression returns name of collation") { - Seq("utf8_binary", "utf8_binary_lcase", "unicode", "unicode_ci").foreach { collationName => + Seq("utf8_binary", "utf8_lcase", "unicode", "unicode_ci").foreach { collationName => checkAnswer( sql(s"select collation('aaa' collate $collationName)"), Row(collationName.toUpperCase())) } @@ -70,14 +70,14 @@ 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_lcase')").schema(0).dataType == - StringType("UTF8_BINARY_LCASE")) + assert(sql(s"select collate('aaa', 'utf8_lcase')").schema(0).dataType == + StringType("UTF8_LCASE")) } test("collate function syntax with default collation set") { - withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UTF8_BINARY_LCASE") { - assert(sql(s"select collate('aaa', 'utf8_binary_lcase')").schema(0).dataType == - StringType("UTF8_BINARY_LCASE")) + withSQLConf(SqlApiConf.DEFAULT_COLLATION -> "UTF8_LCASE") { + assert(sql(s"select collate('aaa', 'utf8_lcase')").schema(0).dataType == + StringType("UTF8_LCASE")) assert(sql(s"select collate('aaa', 'UNICODE')").schema(0).dataType == StringType("UNICODE")) } } @@ -189,9 +189,9 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { Seq( ("utf8_binary", "aaa", "AAA", false), ("utf8_binary", "aaa", "aaa", true), - ("utf8_binary_lcase", "aaa", "aaa", true), - ("utf8_binary_lcase", "aaa", "AAA", true), - ("utf8_binary_lcase", "aaa", "bbb", false), + ("utf8_lcase", "aaa", "aaa", true), + ("utf8_lcase", "aaa", "AAA", true), + ("utf8_lcase", "aaa", "bbb", false), ("unicode", "aaa", "aaa", true), ("unicode", "aaa", "AAA", false), ("unicode_CI", "aaa", "aaa", true), @@ -213,9 +213,9 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { ("utf8_binary", "AAA", "aaa", true), ("utf8_binary", "aaa", "aaa", false), ("utf8_binary", "aaa", "BBB", false), - ("utf8_binary_lcase", "aaa", "aaa", false), - ("utf8_binary_lcase", "AAA", "aaa", false), - ("utf8_binary_lcase", "aaa", "bbb", true), + ("utf8_lcase", "aaa", "aaa", false), + ("utf8_lcase", "AAA", "aaa", false), + ("utf8_lcase", "aaa", "bbb", true), ("unicode", "aaa", "aaa", false), ("unicode", "aaa", "AAA", true), ("unicode", "aaa", "BBB", true), @@ -287,9 +287,9 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { ("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_binary_lcase", Seq("aaa", "aaa"), Seq(Row(2, "aaa"))), - ("utf8_binary_lcase", Seq("AAA", "aaa"), Seq(Row(2, "AAA"))), - ("utf8_binary_lcase", 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"))), ("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"))), @@ -315,7 +315,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val tableNameBinary = "T_BINARY" withTable(tableNameNonBinary) { withTable(tableNameBinary) { - sql(s"CREATE TABLE $tableNameNonBinary (c STRING COLLATE UTF8_BINARY_LCASE) USING PARQUET") + sql(s"CREATE TABLE $tableNameNonBinary (c STRING COLLATE UTF8_LCASE) USING PARQUET") sql(s"INSERT INTO $tableNameNonBinary VALUES ('aaa')") sql(s"CREATE TABLE $tableNameBinary (c STRING COLLATE UTF8_BINARY) USING PARQUET") sql(s"INSERT INTO $tableNameBinary VALUES ('aaa')") @@ -345,7 +345,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("create table with collation") { val tableName = "dummy_tbl" - val collationName = "UTF8_BINARY_LCASE" + val collationName = "UTF8_LCASE" val collationId = CollationFactory.collationNameToId(collationName) allFileBasedDataSources.foreach { format => @@ -393,7 +393,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("add collated column with alter table") { val tableName = "alter_column_tbl" val defaultCollation = "UTF8_BINARY" - val collationName = "UTF8_BINARY_LCASE" + val collationName = "UTF8_LCASE" val collationId = CollationFactory.collationNameToId(collationName) withTable(tableName) { @@ -429,7 +429,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { withTable(tableName) { spark.sql( s""" - | CREATE TABLE $tableName(c1 STRING COLLATE UTF8_BINARY_LCASE, + | CREATE TABLE $tableName(c1 STRING COLLATE UTF8_LCASE, | c2 STRING COLLATE UNICODE, c3 STRING COLLATE UNICODE_CI, c4 STRING) | USING PARQUET |""".stripMargin) @@ -482,7 +482,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { // concat + in checkAnswer(sql(s"SELECT c1 FROM $tableName where c1 || 'a' " + - s"IN (COLLATE('aa', 'UTF8_BINARY_LCASE'))"), Seq(Row("a"), Row("A"))) + s"IN (COLLATE('aa', 'UTF8_LCASE'))"), Seq(Row("a"), Row("A"))) checkAnswer(sql(s"SELECT c1 FROM $tableName where (c1 || 'a') " + s"IN (COLLATE('aa', 'UTF8_BINARY'))"), Seq(Row("a"))) @@ -643,7 +643,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { spark.sql( s""" | CREATE TABLE $tableName(utf8_binary STRING COLLATE UTF8_BINARY, - | utf8_binary_lcase STRING COLLATE UTF8_BINARY_LCASE) + | utf8_lcase STRING COLLATE UTF8_LCASE) | USING PARQUET |""".stripMargin) sql(s"INSERT INTO $tableName VALUES ('aaa', 'aaa')") @@ -652,11 +652,11 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { sql(s"INSERT INTO $tableName VALUES ('BBB', 'BBB')") checkAnswer(sql(s"SELECT * FROM $tableName " + - s"WHERE utf8_binary_lcase IN " + - s"('aaa' COLLATE UTF8_BINARY_LCASE, 'bbb' COLLATE UTF8_BINARY_LCASE)"), + s"WHERE utf8_lcase IN " + + s"('aaa' COLLATE UTF8_LCASE, 'bbb' COLLATE UTF8_LCASE)"), Seq(Row("aaa", "aaa"), Row("AAA", "AAA"), Row("bbb", "bbb"), Row("BBB", "BBB"))) checkAnswer(sql(s"SELECT * FROM $tableName " + - s"WHERE utf8_binary_lcase IN ('aaa' COLLATE UTF8_BINARY_LCASE, 'bbb')"), + s"WHERE utf8_lcase IN ('aaa' COLLATE UTF8_LCASE, 'bbb')"), Seq(Row("aaa", "aaa"), Row("AAA", "AAA"), Row("bbb", "bbb"), Row("BBB", "BBB"))) } } @@ -669,7 +669,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { spark.sql( s""" | CREATE TABLE $tableName(c1 STRING COLLATE UNICODE, - | c2 STRING COLLATE UTF8_BINARY_LCASE) + | c2 STRING COLLATE UTF8_LCASE) | USING PARQUET |""".stripMargin) sql(s"INSERT INTO $tableName VALUES ('aaa', 'aaa')") @@ -691,7 +691,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("create v2 table with collation column") { val tableName = "testcat.table_name" - val collationName = "UTF8_BINARY_LCASE" + val collationName = "UTF8_LCASE" val collationId = CollationFactory.collationNameToId(collationName) withTable(tableName) { @@ -755,7 +755,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val schema = StructType(StructField( "col", - StringType(CollationFactory.collationNameToId("UTF8_BINARY_LCASE"))) :: Nil) + StringType(CollationFactory.collationNameToId("UTF8_LCASE"))) :: Nil) val df = spark.createDataFrame(sparkContext.parallelize(in), schema) df.repartition(10, df.col("col")).foreachPartition( @@ -908,13 +908,13 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val table = "table_agg" // array withTable(table) { - sql(s"create table $table (a array) using parquet") + sql(s"create table $table (a array) using parquet") sql(s"insert into $table values (array('aaa')), (array('AAA'))") checkAnswer(sql(s"select distinct a from $table"), Seq(Row(Seq("aaa")))) } // map doesn't support aggregation withTable(table) { - sql(s"create table $table (m map) using parquet") + sql(s"create table $table (m map) using parquet") val query = s"select distinct m from $table" checkError( exception = intercept[ExtendedAnalysisException](sql(query)), @@ -922,14 +922,14 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { parameters = Map( "colName" -> "`m`", "dataType" -> toSQLType(MapType( - StringType(CollationFactory.collationNameToId("UTF8_BINARY_LCASE")), + StringType(CollationFactory.collationNameToId("UTF8_LCASE")), StringType))), context = ExpectedContext(query, 0, query.length - 1) ) } // struct withTable(table) { - sql(s"create table $table (s struct) using parquet") + sql(s"create table $table (s struct) using parquet") sql(s"insert into $table values (named_struct('fld', 'aaa')), (named_struct('fld', 'AAA'))") checkAnswer(sql(s"select s.fld from $table group by s"), Seq(Row("aaa"))) } @@ -941,7 +941,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { // array withTable(tableLeft, tableRight) { Seq(tableLeft, tableRight).map(tab => - sql(s"create table $tab (a array) using parquet")) + sql(s"create table $tab (a array) using parquet")) Seq((tableLeft, "array('aaa')"), (tableRight, "array('AAA')")).map{ case (tab, data) => sql(s"insert into $tab values ($data)") } @@ -954,7 +954,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { // map doesn't support joins withTable(tableLeft, tableRight) { Seq(tableLeft, tableRight).map(tab => - sql(s"create table $tab (m map) using parquet")) + sql(s"create table $tab (m map) using parquet")) val query = s"select $tableLeft.m from $tableLeft join $tableRight on $tableLeft.m = $tableRight.m" val ctx = s"$tableLeft.m = $tableRight.m" @@ -964,7 +964,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { parameters = Map( "functionName" -> "`=`", "dataType" -> toSQLType(MapType( - StringType(CollationFactory.collationNameToId("UTF8_BINARY_LCASE")), + StringType(CollationFactory.collationNameToId("UTF8_LCASE")), StringType )), "sqlExpr" -> "\"(m = m)\""), @@ -973,7 +973,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { // struct withTable(tableLeft, tableRight) { Seq(tableLeft, tableRight).map(tab => - sql(s"create table $tab (s struct) using parquet")) + sql(s"create table $tab (s struct) using parquet")) Seq( (tableLeft, "named_struct('fld', 'aaa')"), (tableRight, "named_struct('fld', 'AAA')") @@ -989,37 +989,37 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { } test("Support operations on complex types containing collated strings") { - checkAnswer(sql("select reverse('abc' collate utf8_binary_lcase)"), Seq(Row("cba"))) + checkAnswer(sql("select reverse('abc' collate utf8_lcase)"), Seq(Row("cba"))) checkAnswer(sql( """ - |select reverse(array('a' collate utf8_binary_lcase, - |'b' collate utf8_binary_lcase)) + |select reverse(array('a' collate utf8_lcase, + |'b' collate utf8_lcase)) |""".stripMargin), Seq(Row(Seq("b", "a")))) checkAnswer(sql( """ - |select array_join(array('a' collate utf8_binary_lcase, - |'b' collate utf8_binary_lcase), ', ' collate utf8_binary_lcase) + |select array_join(array('a' collate utf8_lcase, + |'b' collate utf8_lcase), ', ' collate utf8_lcase) |""".stripMargin), Seq(Row("a, b"))) checkAnswer(sql( """ - |select array_join(array('a' collate utf8_binary_lcase, - |'b' collate utf8_binary_lcase, null), ', ' collate utf8_binary_lcase, - |'c' collate utf8_binary_lcase) + |select array_join(array('a' collate utf8_lcase, + |'b' collate utf8_lcase, null), ', ' collate utf8_lcase, + |'c' collate utf8_lcase) |""".stripMargin), Seq(Row("a, b, c"))) checkAnswer(sql( """ - |select concat('a' collate utf8_binary_lcase, 'b' collate utf8_binary_lcase) + |select concat('a' collate utf8_lcase, 'b' collate utf8_lcase) |""".stripMargin), Seq(Row("ab"))) checkAnswer(sql( """ - |select concat(array('a' collate utf8_binary_lcase, 'b' collate utf8_binary_lcase)) + |select concat(array('a' collate utf8_lcase, 'b' collate utf8_lcase)) |""".stripMargin), Seq(Row(Seq("a", "b")))) checkAnswer(sql( """ - |select map('a' collate utf8_binary_lcase, 1, 'b' collate utf8_binary_lcase, 2) - |['A' collate utf8_binary_lcase] + |select map('a' collate utf8_lcase, 1, 'b' collate utf8_lcase, 2) + |['A' collate utf8_lcase] |""".stripMargin), Seq(Row(1))) - val ctx = "map('aaa' collate utf8_binary_lcase, 1, 'AAA' collate utf8_binary_lcase, 2)['AaA']" + val ctx = "map('aaa' collate utf8_lcase, 1, 'AAA' collate utf8_lcase, 2)['AaA']" val query = s"select $ctx" checkError( exception = intercept[AnalysisException](sql(query)), @@ -1030,7 +1030,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { "inputSql" -> "\"AaA\"", "inputType" -> toSQLType(StringType), "requiredType" -> toSQLType(StringType( - CollationFactory.collationNameToId("UTF8_BINARY_LCASE"))) + CollationFactory.collationNameToId("UTF8_LCASE"))) ), context = ExpectedContext( fragment = ctx, @@ -1045,7 +1045,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val t2 = "T_BINARY" withTable(t1, t2) { - sql(s"CREATE TABLE $t1 (c STRING COLLATE UTF8_BINARY_LCASE, i int) USING PARQUET") + sql(s"CREATE TABLE $t1 (c STRING COLLATE UTF8_LCASE, i int) USING PARQUET") sql(s"INSERT INTO $t1 VALUES ('aA', 2), ('Aa', 1), ('ab', 3), ('aa', 1)") sql(s"CREATE TABLE $t2 (c STRING, i int) USING PARQUET") @@ -1067,7 +1067,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { case class HashJoinTestCase[R](collation: String, result: R) val testCases = Seq( HashJoinTestCase("UTF8_BINARY", Seq(Row("aa", 1, "aa", 2))), - HashJoinTestCase("UTF8_BINARY_LCASE", Seq(Row("aa", 1, "AA", 2), Row("aa", 1, "aa", 2))), + HashJoinTestCase("UTF8_LCASE", Seq(Row("aa", 1, "AA", 2), Row("aa", 1, "aa", 2))), HashJoinTestCase("UNICODE", Seq(Row("aa", 1, "aa", 2))), HashJoinTestCase("UNICODE_CI", Seq(Row("aa", 1, "AA", 2), Row("aa", 1, "aa", 2))) ) @@ -1119,7 +1119,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val testCases = Seq( HashJoinTestCase("UTF8_BINARY", Seq(Row(Seq("aa"), 1, Seq("aa"), 2))), - HashJoinTestCase("UTF8_BINARY_LCASE", + HashJoinTestCase("UTF8_LCASE", Seq(Row(Seq("aa"), 1, Seq("AA"), 2), Row(Seq("aa"), 1, Seq("aa"), 2))), HashJoinTestCase("UNICODE", Seq(Row(Seq("aa"), 1, Seq("aa"), 2))), @@ -1175,7 +1175,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val testCases = Seq( HashJoinTestCase("UTF8_BINARY", Seq(Row(Seq(Seq("aa")), 1, Seq(Seq("aa")), 2))), - HashJoinTestCase("UTF8_BINARY_LCASE", + HashJoinTestCase("UTF8_LCASE", Seq(Row(Seq(Seq("aa")), 1, Seq(Seq("AA")), 2), Row(Seq(Seq("aa")), 1, Seq(Seq("aa")), 2))), HashJoinTestCase("UNICODE", Seq(Row(Seq(Seq("aa")), 1, Seq(Seq("aa")), 2))), @@ -1234,7 +1234,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val testCases = Seq( HashJoinTestCase("UTF8_BINARY", Seq(Row(Row("aa"), 1, Row("aa"), 2))), - HashJoinTestCase("UTF8_BINARY_LCASE", + HashJoinTestCase("UTF8_LCASE", Seq(Row(Row("aa"), 1, Row("AA"), 2), Row(Row("aa"), 1, Row("aa"), 2))), HashJoinTestCase("UNICODE", Seq(Row(Row("aa"), 1, Row("aa"), 2))), @@ -1284,7 +1284,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { val testCases = Seq( HashJoinTestCase("UTF8_BINARY", Seq(Row(Row(Seq(Row("aa"))), 1, Row(Seq(Row("aa"))), 2))), - HashJoinTestCase("UTF8_BINARY_LCASE", + HashJoinTestCase("UTF8_LCASE", Seq(Row(Row(Seq(Row("aa"))), 1, Row(Seq(Row("AA"))), 2), Row(Row(Seq(Row("aa"))), 1, Row(Seq(Row("aa"))), 2))), HashJoinTestCase("UNICODE", @@ -1334,7 +1334,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { test("rewrite with collationkey should be an excludable rule") { val t1 = "T_1" val t2 = "T_2" - val collation = "UTF8_BINARY_LCASE" + val collation = "UTF8_LCASE" val collationRewriteJoinRule = "org.apache.spark.sql.catalyst.analysis.RewriteCollationJoin" withTable(t1, t2) { withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> collationRewriteJoinRule) { @@ -1380,9 +1380,9 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { "'a', 0, 1", "'a', 0, 1", Row("a", 0, 1, "a", 0, 1)), HashMultiJoinTestCase("STRING COLLATE UTF8_BINARY", "STRING COLLATE UTF8_BINARY", "'a', 'a', 1", "'a', 'a', 1", Row("a", "a", 1, "a", "a", 1)), - HashMultiJoinTestCase("STRING COLLATE UTF8_BINARY", "STRING COLLATE UTF8_BINARY_LCASE", + HashMultiJoinTestCase("STRING COLLATE UTF8_BINARY", "STRING COLLATE UTF8_LCASE", "'a', 'a', 1", "'a', 'A', 1", Row("a", "a", 1, "a", "A", 1)), - HashMultiJoinTestCase("STRING COLLATE UTF8_BINARY_LCASE", "STRING COLLATE UNICODE_CI", + HashMultiJoinTestCase("STRING COLLATE UTF8_LCASE", "STRING COLLATE UNICODE_CI", "'a', 'a', 1", "'A', 'A', 1", Row("a", "a", 1, "A", "A", 1)) ) @@ -1417,7 +1417,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { case class HllSketchAggTestCase[R](c: String, result: R) val testCases = Seq( HllSketchAggTestCase("UTF8_BINARY", 4), - HllSketchAggTestCase("UTF8_BINARY_LCASE", 3), + HllSketchAggTestCase("UTF8_LCASE", 3), HllSketchAggTestCase("UNICODE", 4), HllSketchAggTestCase("UNICODE_CI", 3) ) 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 49a33d1c92543..5fbe88a09e7cc 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 @@ -1248,7 +1248,7 @@ class FileBasedDataSourceSuite extends QueryTest Seq(format, "").foreach { conf => withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> conf) { withTempPath { path => - val collation = "'UTF8_BINARY_LCASE'" + val collation = "'UTF8_LCASE'" val df = sql( s"""SELECT | COLLATE(c, $collation) as c1, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSCollationQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSCollationQueryTestSuite.scala index a84dd9645bcc4..46a24acb475c4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSCollationQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSCollationQueryTestSuite.scala @@ -123,7 +123,7 @@ class TPCDSCollationQueryTestSuite extends QueryTest with TPCDSBase with SQLQuer val checks: Seq[Seq[CollationCheck]] = Seq( Seq( CaseSensitiveCollationCheck("tpcds_utf8", "UTF8_BINARY", "lower"), - CaseInsensitiveCollationCheck("tpcds_utf8_random", "UTF8_BINARY_LCASE", randomizeCase) + CaseInsensitiveCollationCheck("tpcds_utf8_random", "UTF8_LCASE", randomizeCase) ), Seq( CaseSensitiveCollationCheck("tpcds_unicode", "UNICODE", "lower"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CollationBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CollationBenchmark.scala index 03b638b786bfd..de3a6832652ae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CollationBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CollationBenchmark.scala @@ -24,7 +24,7 @@ import org.apache.spark.unsafe.types.UTF8String abstract class CollationBenchmarkBase extends BenchmarkBase { protected val collationTypes: Seq[String] = - Seq("UTF8_BINARY_LCASE", "UNICODE", "UTF8_BINARY", "UNICODE_CI") + Seq("UTF8_LCASE", "UNICODE", "UTF8_BINARY", "UNICODE_CI") def generateSeqInput(n: Long): Seq[UTF8String] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 216472ad23db6..6bc5917edd526 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -2444,8 +2444,8 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { sql("CREATE TABLE t1(col STRING) USING parquet") sql("INSERT INTO t1 VALUES ('a')") checkAnswer(sql("SELECT COLLATION(col) FROM t1"), Row("UTF8_BINARY")) - sql("ALTER TABLE t1 ALTER COLUMN col TYPE STRING COLLATE UTF8_BINARY_LCASE") - checkAnswer(sql("SELECT COLLATION(col) FROM t1"), Row("UTF8_BINARY_LCASE")) + sql("ALTER TABLE t1 ALTER COLUMN col TYPE STRING COLLATE UTF8_LCASE") + checkAnswer(sql("SELECT COLLATION(col) FROM t1"), Row("UTF8_LCASE")) // Invalid "ALTER COLUMN" to Integer. val alterInt = "ALTER TABLE t1 ALTER COLUMN col TYPE INTEGER" @@ -2455,7 +2455,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { }, errorClass = "NOT_SUPPORTED_CHANGE_COLUMN", parameters = Map( - "originType" -> "\"STRING COLLATE UTF8_BINARY_LCASE\"", + "originType" -> "\"STRING COLLATE UTF8_LCASE\"", "originName" -> "`col`", "table" -> "`spark_catalog`.`default`.`t1`", "newType" -> "\"INT\"", @@ -2468,8 +2468,8 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { sql("CREATE TABLE t2(col ARRAY) USING parquet") sql("INSERT INTO t2 VALUES (ARRAY('a'))") checkAnswer(sql("SELECT COLLATION(col[0]) FROM t2"), Row("UTF8_BINARY")) - sql("ALTER TABLE t2 ALTER COLUMN col TYPE ARRAY") - checkAnswer(sql("SELECT COLLATION(col[0]) FROM t2"), Row("UTF8_BINARY_LCASE")) + sql("ALTER TABLE t2 ALTER COLUMN col TYPE ARRAY") + checkAnswer(sql("SELECT COLLATION(col[0]) FROM t2"), Row("UTF8_LCASE")) // `MapType` with collation. sql("CREATE TABLE t3(col MAP) USING parquet") @@ -2478,23 +2478,23 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { sql( """ |ALTER TABLE t3 ALTER COLUMN col TYPE - |MAP""".stripMargin) - checkAnswer(sql("SELECT COLLATION(col['k']) FROM t3"), Row("UTF8_BINARY_LCASE")) + |MAP""".stripMargin) + checkAnswer(sql("SELECT COLLATION(col['k']) FROM t3"), Row("UTF8_LCASE")) // Invalid change of map key collation. val alterMap = "ALTER TABLE t3 ALTER COLUMN col TYPE " + - "MAP" + "MAP" checkError( exception = intercept[AnalysisException] { sql(alterMap) }, errorClass = "NOT_SUPPORTED_CHANGE_COLUMN", parameters = Map( - "originType" -> "\"MAP\"", + "originType" -> "\"MAP\"", "originName" -> "`col`", "table" -> "`spark_catalog`.`default`.`t3`", - "newType" -> "\"MAP\"", + "newType" -> "\"MAP\"", "newName" -> "`col`" ), context = ExpectedContext(fragment = alterMap, start = 0, stop = alterMap.length - 1) @@ -2504,8 +2504,8 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { sql("CREATE TABLE t4(col STRUCT) USING parquet") sql("INSERT INTO t4 VALUES (NAMED_STRUCT('a', 'value'))") checkAnswer(sql("SELECT COLLATION(col.a) FROM t4"), Row("UTF8_BINARY")) - sql("ALTER TABLE t4 ALTER COLUMN col TYPE STRUCT") - checkAnswer(sql("SELECT COLLATION(col.a) FROM t4"), Row("UTF8_BINARY_LCASE")) + sql("ALTER TABLE t4 ALTER COLUMN col TYPE STRUCT") + checkAnswer(sql("SELECT COLLATION(col.a) FROM t4"), Row("UTF8_LCASE")) } } @@ -2514,7 +2514,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { sql("CREATE TABLE t1(col STRING, i INTEGER) USING parquet PARTITIONED BY (col)") checkError( exception = intercept[AnalysisException] { - sql("ALTER TABLE t1 ALTER COLUMN col TYPE STRING COLLATE UTF8_BINARY_LCASE") + sql("ALTER TABLE t1 ALTER COLUMN col TYPE STRING COLLATE UTF8_LCASE") }, errorClass = "CANNOT_ALTER_PARTITION_COLUMN", sqlState = "428FR", @@ -2523,7 +2523,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { sql("CREATE TABLE t2(col STRING) USING parquet CLUSTERED BY (col) INTO 1 BUCKETS") checkError( exception = intercept[AnalysisException] { - sql("ALTER TABLE t2 ALTER COLUMN col TYPE STRING COLLATE UTF8_BINARY_LCASE") + sql("ALTER TABLE t2 ALTER COLUMN col TYPE STRING COLLATE UTF8_LCASE") }, errorClass = "CANNOT_ALTER_COLLATION_BUCKET_COLUMN", sqlState = "428FR", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala index 3c09dee990ebc..834225baf070e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala @@ -358,7 +358,7 @@ class DataSourceStrategySuite extends PlanTest with SharedSparkSession { testTranslateFilter(IsNotNull(colAttr), Some(sources.IsNotNull("col"))) } - for (collation <- Seq("UTF8_BINARY_LCASE", "UNICODE")) { + for (collation <- Seq("UTF8_LCASE", "UNICODE")) { test(s"SPARK-48431: Filter pushdown on columns with $collation collation") { val colAttr = $"col".string(collation) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala index 51c9b960a8eab..3762241719acd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala @@ -82,7 +82,7 @@ class SaveIntoDataSourceCommandSuite extends QueryTest with SharedSparkSession { val df = spark.range(1).selectExpr( "cast('a' as binary) a", "true b", "cast(1 as byte) c", "1.23 d", "'abc'", - "'abc' COLLATE UTF8_BINARY_LCASE") + "'abc' COLLATE UTF8_LCASE") dataSource.planForWriting(SaveMode.ErrorIfExists, df.logicalPlan) // Variant and Interval types are disallowed by default. 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 922894afa4e25..84348f7da67f1 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 @@ -66,12 +66,12 @@ class StateSchemaCompatibilityCheckerSuite extends SharedSparkSession { private val keySchemaWithCollation = new StructType() .add(StructField("key1", IntegerType, nullable = true)) - .add(StructField("key2", StringType("UTF8_BINARY_LCASE"), nullable = true)) + .add(StructField("key2", StringType("UTF8_LCASE"), nullable = true)) .add(StructField("key3", structSchema, nullable = true)) private val valueSchemaWithCollation = new StructType() .add(StructField("value1", IntegerType, nullable = true)) - .add(StructField("value2", StringType("UTF8_BINARY_LCASE"), nullable = true)) + .add(StructField("value2", StringType("UTF8_LCASE"), nullable = true)) .add(StructField("value3", structSchema, nullable = true)) // Checks on adding/removing (nested) field. 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 ea3c2d594cd5d..0639913c8f816 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 @@ -502,7 +502,7 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { } test("SPARK-47765: set collation") { - Seq("UNICODE", "UNICODE_CI", "utf8_binary_lcase", "utf8_binary").foreach { collation => + Seq("UNICODE", "UNICODE_CI", "utf8_lcase", "utf8_binary").foreach { collation => sql(s"set collation $collation") assert(spark.conf.get(SQLConf.DEFAULT_COLLATION) === collation.toUpperCase(Locale.ROOT)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala index 7110b70104f38..984279158944c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala @@ -514,7 +514,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { val inputData = MemoryStream[(String)] val result = inputData.toDF() .select(col("value") - .try_cast(StringType("UTF8_BINARY_LCASE")).as("str")) + .try_cast(StringType("UTF8_LCASE")).as("str")) .dropDuplicates("str") val ex = intercept[StreamingQueryException] { @@ -527,7 +527,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { ex.getCause.asInstanceOf[SparkUnsupportedOperationException], errorClass = "STATE_STORE_UNSUPPORTED_OPERATION_BINARY_INEQUALITY", parameters = Map( - "schema" -> ".+\"str\":\"spark.UTF8_BINARY_LCASE\".+" + "schema" -> ".+\"str\":\"spark.UTF8_LCASE\".+" ), matchPVals = true ) 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 fb118adbe2212..061b353879d14 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 @@ -1370,7 +1370,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi """ |CREATE TABLE parquet_streaming_tbl |( - | key STRING COLLATE UTF8_BINARY_LCASE, + | key STRING COLLATE UTF8_LCASE, | value_stream INTEGER |) USING parquet""".stripMargin) @@ -1396,7 +1396,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi test("SPARK-47776: streaming aggregation having binary inequality column in the grouping " + "key must be disallowed") { val tableName = "parquet_dummy_tbl" - val collationName = "UTF8_BINARY_LCASE" + val collationName = "UTF8_LCASE" withTable(tableName) { sql( @@ -1425,7 +1425,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi ex.getCause.asInstanceOf[SparkUnsupportedOperationException], errorClass = "STATE_STORE_UNSUPPORTED_OPERATION_BINARY_INEQUALITY", parameters = Map( - "schema" -> ".+\"c1\":\"spark.UTF8_BINARY_LCASE\".+" + "schema" -> ".+\"c1\":\"spark.UTF8_LCASE\".+" ), matchPVals = true ) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala index 2c42eaebd701e..8bb33e3383be1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala @@ -231,7 +231,7 @@ class HiveExternalCatalogSuite extends ExternalCatalogSuite { // perform alter table val newSchema = StructType(Seq( - StructField("col1", StringType("UTF8_BINARY_LCASE")) + StructField("col1", StringType("UTF8_LCASE")) )) catalog.alterTableDataSchema("db1", tableName, newSchema)