diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index d133b74da7cd6..8040169fcb2c8 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -295,8 +295,6 @@ jobs: INCLUDED_TAGS: ${{ matrix.included-tags }} HADOOP_PROFILE: ${{ matrix.hadoop }} HIVE_PROFILE: ${{ matrix.hive }} - # GitHub Actions' default miniconda to use in pip packaging test. - CONDA_PREFIX: /usr/share/miniconda GITHUB_PREV_SHA: ${{ github.event.before }} SPARK_LOCAL_IP: localhost NOLINT_ON_COMPILE: true @@ -539,8 +537,6 @@ jobs: MODULES_TO_TEST: ${{ matrix.modules }} HADOOP_PROFILE: ${{ inputs.hadoop }} HIVE_PROFILE: hive2.3 - # GitHub Actions' default miniconda to use in pip packaging test. - CONDA_PREFIX: /usr/share/miniconda GITHUB_PREV_SHA: ${{ github.event.before }} SPARK_LOCAL_IP: localhost SKIP_UNIDOC: true @@ -602,12 +598,18 @@ jobs: echo $py $py -m pip list done + - name: Install Conda for pip packaging test + if: contains(matrix.modules, 'pyspark-errors') + uses: conda-incubator/setup-miniconda@v3 + with: + miniforge-version: latest # Run the tests. - name: Run tests env: ${{ fromJSON(inputs.envs) }} shell: 'script -q -e -c "bash {0}"' run: | if [[ "$MODULES_TO_TEST" == *"pyspark-errors"* ]]; then + export PATH=$CONDA/bin:$PATH export SKIP_PACKAGING=false echo "Python Packaging Tests Enabled!" fi @@ -620,9 +622,10 @@ jobs: - name: Upload coverage to Codecov if: fromJSON(inputs.envs).PYSPARK_CODECOV == 'true' uses: codecov/codecov-action@v5 + env: + CODECOV_TOKEN: ${{ secrets.CODECOV_TOKEN }} with: files: ./python/coverage.xml - token: ${{ secrets.CODECOV_TOKEN }} flags: unittests name: PySpark verbose: true 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 ce416293131a1..81448dc95a374 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 @@ -467,9 +467,6 @@ private enum CaseSensitivity { */ private static final int CASE_SENSITIVITY_MASK = 0b1; - private static final String UTF8_BINARY_COLLATION_NAME = "UTF8_BINARY"; - private static final String UTF8_LCASE_COLLATION_NAME = "UTF8_LCASE"; - private static final int UTF8_BINARY_COLLATION_ID = new CollationSpecUTF8(CaseSensitivity.UNSPECIFIED, SpaceTrimming.NONE).collationId; private static final int UTF8_LCASE_COLLATION_ID = @@ -655,9 +652,9 @@ protected CollationMeta buildCollationMeta() { protected String normalizedCollationName() { StringBuilder builder = new StringBuilder(); if(caseSensitivity == CaseSensitivity.UNSPECIFIED){ - builder.append(UTF8_BINARY_COLLATION_NAME); + builder.append(CollationNames.UTF8_BINARY); } else{ - builder.append(UTF8_LCASE_COLLATION_NAME); + builder.append(CollationNames.UTF8_LCASE); } if (spaceTrimming != SpaceTrimming.NONE) { builder.append('_'); @@ -669,12 +666,12 @@ protected String normalizedCollationName() { static List listCollations() { CollationIdentifier UTF8_BINARY_COLLATION_IDENT = new CollationIdentifier( PROVIDER_SPARK, - UTF8_BINARY_COLLATION_NAME, + CollationNames.UTF8_BINARY, CollationSpecICU.ICU_VERSION ); CollationIdentifier UTF8_LCASE_COLLATION_IDENT = new CollationIdentifier( PROVIDER_SPARK, - UTF8_LCASE_COLLATION_NAME, + CollationNames.UTF8_LCASE, CollationSpecICU.ICU_VERSION ); return Arrays.asList(UTF8_BINARY_COLLATION_IDENT, UTF8_LCASE_COLLATION_IDENT); @@ -758,7 +755,7 @@ private enum AccentSensitivity { VersionInfo.ICU_VERSION.getMinor()); static { - ICULocaleMap.put("UNICODE", ULocale.ROOT); + ICULocaleMap.put(CollationNames.UNICODE, ULocale.ROOT); // ICU-implemented `ULocale`s which have corresponding `Collator` installed. ULocale[] locales = Collator.getAvailableULocales(); // Build locale names in format: language["_" optional script]["_" optional country code]. @@ -806,13 +803,13 @@ private enum AccentSensitivity { } private static final int UNICODE_COLLATION_ID = new CollationSpecICU( - "UNICODE", + CollationNames.UNICODE, CaseSensitivity.CS, AccentSensitivity.AS, SpaceTrimming.NONE).collationId; private static final int UNICODE_CI_COLLATION_ID = new CollationSpecICU( - "UNICODE", + CollationNames.UNICODE, CaseSensitivity.CI, AccentSensitivity.AS, SpaceTrimming.NONE).collationId; diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationNames.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationNames.java new file mode 100644 index 0000000000000..11e9e1a87e713 --- /dev/null +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationNames.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.util; + +public class CollationNames { + public static final String UTF8_BINARY = "UTF8_BINARY"; + public static final String UTF8_LCASE = "UTF8_LCASE"; + public static final String UNICODE = "UNICODE"; + public static final String UNICODE_CI = "UNICODE_CI"; +} 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 a696da8cf45b8..1db163c1c822d 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 @@ -26,6 +26,7 @@ import java.util.Map; import static org.junit.jupiter.api.Assertions.*; +import static org.apache.spark.sql.catalyst.util.CollationNames.*; // checkstyle.off: AvoidEscapedUnicodeCharacters public class CollationSupportSuite { @@ -37,7 +38,7 @@ public class CollationSupportSuite { * the specified collations (as often seen in some pass-through Spark expressions). */ private final String[] testSupportedCollations = - {"UTF8_BINARY", "UTF8_LCASE", "UNICODE", "UNICODE_CI"}; + {UTF8_BINARY, UTF8_LCASE, UNICODE, UNICODE_CI}; /** * Collation-aware UTF8String comparison and equality check. @@ -86,82 +87,82 @@ public void testCompare() throws SparkException { assertCompare("a", "ä", collationName, -1); } // Advanced tests. - assertCompare("äü", "bü", "UTF8_BINARY", 1); - assertCompare("bxx", "bü", "UTF8_BINARY", -1); - assertCompare("äü", "bü", "UTF8_LCASE", 1); - assertCompare("bxx", "bü", "UTF8_LCASE", -1); - assertCompare("äü", "bü", "UNICODE", -1); - assertCompare("bxx", "bü", "UNICODE", 1); - assertCompare("äü", "bü", "UNICODE_CI", -1); - assertCompare("bxx", "bü", "UNICODE_CI", 1); + assertCompare("äü", "bü", UTF8_BINARY, 1); + assertCompare("bxx", "bü", UTF8_BINARY, -1); + assertCompare("äü", "bü", UTF8_LCASE, 1); + assertCompare("bxx", "bü", UTF8_LCASE, -1); + assertCompare("äü", "bü", UNICODE, -1); + assertCompare("bxx", "bü", UNICODE, 1); + assertCompare("äü", "bü", UNICODE_CI, -1); + assertCompare("bxx", "bü", UNICODE_CI, 1); assertCompare("cČć", "ČćC", "SR_CI_AI", 0); // Case variation. - assertCompare("AbCd", "aBcD", "UTF8_BINARY", -1); - assertCompare("ABCD", "abcd", "UTF8_LCASE", 0); - assertCompare("AbcD", "aBCd", "UNICODE", 1); - assertCompare("abcd", "ABCD", "UNICODE_CI", 0); + assertCompare("AbCd", "aBcD", UTF8_BINARY, -1); + assertCompare("ABCD", "abcd", UTF8_LCASE, 0); + assertCompare("AbcD", "aBCd", UNICODE, 1); + assertCompare("abcd", "ABCD", UNICODE_CI, 0); // Accent variation. - assertCompare("aBćD", "ABĆD", "UTF8_BINARY", 1); - assertCompare("AbCδ", "ABCΔ", "UTF8_LCASE", 0); - assertCompare("äBCd", "ÄBCD", "UNICODE", -1); - assertCompare("Ab́cD", "AB́CD", "UNICODE_CI", 0); + assertCompare("aBćD", "ABĆD", UTF8_BINARY, 1); + assertCompare("AbCδ", "ABCΔ", UTF8_LCASE, 0); + assertCompare("äBCd", "ÄBCD", UNICODE, -1); + assertCompare("Ab́cD", "AB́CD", UNICODE_CI, 0); assertCompare("ÈÉÊË", "EeEe", "AF_CI_AI", 0); // One-to-many case mapping (e.g. Turkish dotted I). - assertCompare("i\u0307", "İ", "UTF8_BINARY", -1); - assertCompare("İ", "i\u0307", "UTF8_BINARY", 1); - assertCompare("i\u0307", "İ", "UTF8_LCASE", 0); - assertCompare("İ", "i\u0307", "UTF8_LCASE", 0); - assertCompare("i\u0307", "İ", "UNICODE", -1); - assertCompare("İ", "i\u0307", "UNICODE", 1); - assertCompare("i\u0307", "İ", "UNICODE_CI", 0); - assertCompare("İ", "i\u0307", "UNICODE_CI", 0); - assertCompare("i\u0307İ", "i\u0307İ", "UTF8_LCASE", 0); - assertCompare("i\u0307İ", "İi\u0307", "UTF8_LCASE", 0); - assertCompare("İi\u0307", "i\u0307İ", "UTF8_LCASE", 0); - assertCompare("İi\u0307", "İi\u0307", "UTF8_LCASE", 0); - assertCompare("i\u0307İ", "i\u0307İ", "UNICODE_CI", 0); - assertCompare("i\u0307İ", "İi\u0307", "UNICODE_CI", 0); - assertCompare("İi\u0307", "i\u0307İ", "UNICODE_CI", 0); - assertCompare("İi\u0307", "İi\u0307", "UNICODE_CI", 0); + assertCompare("i\u0307", "İ", UTF8_BINARY, -1); + assertCompare("İ", "i\u0307", UTF8_BINARY, 1); + assertCompare("i\u0307", "İ", UTF8_LCASE, 0); + assertCompare("İ", "i\u0307", UTF8_LCASE, 0); + assertCompare("i\u0307", "İ", UNICODE, -1); + assertCompare("İ", "i\u0307", UNICODE, 1); + assertCompare("i\u0307", "İ", UNICODE_CI, 0); + assertCompare("İ", "i\u0307", UNICODE_CI, 0); + assertCompare("i\u0307İ", "i\u0307İ", UTF8_LCASE, 0); + assertCompare("i\u0307İ", "İi\u0307", UTF8_LCASE, 0); + assertCompare("İi\u0307", "i\u0307İ", UTF8_LCASE, 0); + assertCompare("İi\u0307", "İi\u0307", UTF8_LCASE, 0); + assertCompare("i\u0307İ", "i\u0307İ", UNICODE_CI, 0); + assertCompare("i\u0307İ", "İi\u0307", UNICODE_CI, 0); + assertCompare("İi\u0307", "i\u0307İ", UNICODE_CI, 0); + assertCompare("İi\u0307", "İi\u0307", UNICODE_CI, 0); // Conditional case mapping (e.g. Greek sigmas). - assertCompare("ς", "σ", "UTF8_BINARY", -1); - assertCompare("ς", "Σ", "UTF8_BINARY", 1); - assertCompare("σ", "Σ", "UTF8_BINARY", 1); - assertCompare("ς", "σ", "UTF8_LCASE", 0); - assertCompare("ς", "Σ", "UTF8_LCASE", 0); - assertCompare("σ", "Σ", "UTF8_LCASE", 0); - assertCompare("ς", "σ", "UNICODE", 1); - assertCompare("ς", "Σ", "UNICODE", 1); - assertCompare("σ", "Σ", "UNICODE", -1); - assertCompare("ς", "σ", "UNICODE_CI", 0); - assertCompare("ς", "Σ", "UNICODE_CI", 0); - assertCompare("σ", "Σ", "UNICODE_CI", 0); + assertCompare("ς", "σ", UTF8_BINARY, -1); + assertCompare("ς", "Σ", UTF8_BINARY, 1); + assertCompare("σ", "Σ", UTF8_BINARY, 1); + assertCompare("ς", "σ", UTF8_LCASE, 0); + assertCompare("ς", "Σ", UTF8_LCASE, 0); + assertCompare("σ", "Σ", UTF8_LCASE, 0); + assertCompare("ς", "σ", UNICODE, 1); + assertCompare("ς", "Σ", UNICODE, 1); + assertCompare("σ", "Σ", UNICODE, -1); + assertCompare("ς", "σ", UNICODE_CI, 0); + assertCompare("ς", "Σ", UNICODE_CI, 0); + assertCompare("σ", "Σ", UNICODE_CI, 0); // Surrogate pairs. - assertCompare("a🙃b🙃c", "aaaaa", "UTF8_BINARY", 1); - assertCompare("a🙃b🙃c", "aaaaa", "UTF8_LCASE", 1); - assertCompare("a🙃b🙃c", "aaaaa", "UNICODE", -1); // != UTF8_BINARY - assertCompare("a🙃b🙃c", "aaaaa", "UNICODE_CI", -1); // != UTF8_LCASE - assertCompare("a🙃b🙃c", "a🙃b🙃c", "UTF8_BINARY", 0); - assertCompare("a🙃b🙃c", "a🙃b🙃c", "UTF8_LCASE", 0); - assertCompare("a🙃b🙃c", "a🙃b🙃c", "UNICODE", 0); - assertCompare("a🙃b🙃c", "a🙃b🙃c", "UNICODE_CI", 0); - assertCompare("a🙃b🙃c", "a🙃b🙃d", "UTF8_BINARY", -1); - assertCompare("a🙃b🙃c", "a🙃b🙃d", "UTF8_LCASE", -1); - assertCompare("a🙃b🙃c", "a🙃b🙃d", "UNICODE", -1); - assertCompare("a🙃b🙃c", "a🙃b🙃d", "UNICODE_CI", -1); + assertCompare("a🙃b🙃c", "aaaaa", UTF8_BINARY, 1); + assertCompare("a🙃b🙃c", "aaaaa", UTF8_LCASE, 1); + assertCompare("a🙃b🙃c", "aaaaa", UNICODE, -1); // != UTF8_BINARY + assertCompare("a🙃b🙃c", "aaaaa", UNICODE_CI, -1); // != UTF8_LCASE + assertCompare("a🙃b🙃c", "a🙃b🙃c", UTF8_BINARY, 0); + assertCompare("a🙃b🙃c", "a🙃b🙃c", UTF8_LCASE, 0); + assertCompare("a🙃b🙃c", "a🙃b🙃c", UNICODE, 0); + assertCompare("a🙃b🙃c", "a🙃b🙃c", UNICODE_CI, 0); + assertCompare("a🙃b🙃c", "a🙃b🙃d", UTF8_BINARY, -1); + assertCompare("a🙃b🙃c", "a🙃b🙃d", UTF8_LCASE, -1); + assertCompare("a🙃b🙃c", "a🙃b🙃d", UNICODE, -1); + assertCompare("a🙃b🙃c", "a🙃b🙃d", UNICODE_CI, -1); // Maximum code point. int maxCodePoint = Character.MAX_CODE_POINT; String maxCodePointStr = new String(Character.toChars(maxCodePoint)); for (int i = 0; i < maxCodePoint && Character.isValidCodePoint(i); ++i) { - assertCompare(new String(Character.toChars(i)), maxCodePointStr, "UTF8_BINARY", -1); - assertCompare(new String(Character.toChars(i)), maxCodePointStr, "UTF8_LCASE", -1); + assertCompare(new String(Character.toChars(i)), maxCodePointStr, UTF8_BINARY, -1); + assertCompare(new String(Character.toChars(i)), maxCodePointStr, UTF8_LCASE, -1); } // Minimum code point. int minCodePoint = Character.MIN_CODE_POINT; String minCodePointStr = new String(Character.toChars(minCodePoint)); for (int i = minCodePoint + 1; i <= maxCodePoint && Character.isValidCodePoint(i); ++i) { - assertCompare(new String(Character.toChars(i)), minCodePointStr, "UTF8_BINARY", 1); - assertCompare(new String(Character.toChars(i)), minCodePointStr, "UTF8_LCASE", 1); + assertCompare(new String(Character.toChars(i)), minCodePointStr, UTF8_BINARY, 1); + assertCompare(new String(Character.toChars(i)), minCodePointStr, UTF8_LCASE, 1); } } @@ -302,201 +303,201 @@ public void testContains() throws SparkException { assertContains("Здраво", "Здраво", collationName, true); } // Advanced tests. - assertContains("abcde", "bcd", "UTF8_BINARY", true); - assertContains("abcde", "bde", "UTF8_BINARY", false); - assertContains("abcde", "fgh", "UTF8_BINARY", false); - assertContains("abcde", "abcde", "UNICODE", true); - assertContains("abcde", "aBcDe", "UNICODE", false); - assertContains("abcde", "fghij", "UNICODE", 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); - assertContains("ab世De", "b世D", "UTF8_BINARY", true); - assertContains("ab世De", "B世d", "UTF8_BINARY", false); - assertContains("äbćδe", "bćδ", "UTF8_BINARY", true); - assertContains("äbćδe", "BcΔ", "UTF8_BINARY", false); - assertContains("ab世De", "ab世De", "UNICODE", true); - 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_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); - 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); + assertContains("abcde", "bcd", UTF8_BINARY, true); + assertContains("abcde", "bde", UTF8_BINARY, false); + assertContains("abcde", "fgh", UTF8_BINARY, false); + assertContains("abcde", "abcde", UNICODE, true); + assertContains("abcde", "aBcDe", UNICODE, false); + assertContains("abcde", "fghij", UNICODE, 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); + assertContains("ab世De", "b世D", UTF8_BINARY, true); + assertContains("ab世De", "B世d", UTF8_BINARY, false); + assertContains("äbćδe", "bćδ", UTF8_BINARY, true); + assertContains("äbćδe", "BcΔ", UTF8_BINARY, false); + assertContains("ab世De", "ab世De", UNICODE, true); + 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_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); + 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); assertContains("abčćd", "ABCCD", "SR_CI_AI", true); // Case variation. - assertContains("aBcDe", "bcd", "UTF8_BINARY", false); - assertContains("aBcDe", "BcD", "UTF8_BINARY", true); - assertContains("aBcDe", "abcde", "UNICODE", false); - assertContains("aBcDe", "aBcDe", "UNICODE", 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); + assertContains("aBcDe", "bcd", UTF8_BINARY, false); + assertContains("aBcDe", "BcD", UTF8_BINARY, true); + assertContains("aBcDe", "abcde", UNICODE, false); + assertContains("aBcDe", "aBcDe", UNICODE, 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. - assertContains("aBcDe", "bćd", "UTF8_BINARY", false); - assertContains("aBcDe", "BćD", "UTF8_BINARY", false); - assertContains("aBcDe", "abćde", "UNICODE", false); - assertContains("aBcDe", "aBćDe", "UNICODE", 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); + assertContains("aBcDe", "bćd", UTF8_BINARY, false); + assertContains("aBcDe", "BćD", UTF8_BINARY, false); + assertContains("aBcDe", "abćde", UNICODE, false); + assertContains("aBcDe", "aBćDe", UNICODE, 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); assertContains("abEEE", "Bèêë", "AF_CI_AI", true); // One-to-many case mapping (e.g. Turkish dotted I). - assertContains("i\u0307", "i", "UNICODE_CI", false); - assertContains("i\u0307", "\u0307", "UNICODE_CI", false); - assertContains("i\u0307", "İ", "UNICODE_CI", true); - assertContains("İ", "i", "UNICODE_CI", false); - assertContains("adi̇os", "io", "UNICODE_CI", false); - assertContains("adi̇os", "Io", "UNICODE_CI", false); - assertContains("adi̇os", "i\u0307o", "UNICODE_CI", true); - assertContains("adi̇os", "İo", "UNICODE_CI", true); - assertContains("adİos", "io", "UNICODE_CI", false); - assertContains("adİos", "Io", "UNICODE_CI", false); - assertContains("adİos", "i\u0307o", "UNICODE_CI", true); - assertContains("adİos", "İo", "UNICODE_CI", true); - assertContains("i\u0307", "i", "UTF8_LCASE", true); // != UNICODE_CI - assertContains("İ", "\u0307", "UTF8_LCASE", false); - assertContains("İ", "i", "UTF8_LCASE", false); - assertContains("i\u0307", "\u0307", "UTF8_LCASE", true); // != UNICODE_CI - assertContains("i\u0307", "İ", "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\u0307o", "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\u0307o", "UTF8_LCASE", true); - assertContains("adİos", "İo", "UTF8_LCASE", true); + assertContains("i\u0307", "i", UNICODE_CI, false); + assertContains("i\u0307", "\u0307", UNICODE_CI, false); + assertContains("i\u0307", "İ", UNICODE_CI, true); + assertContains("İ", "i", UNICODE_CI, false); + assertContains("adi̇os", "io", UNICODE_CI, false); + assertContains("adi̇os", "Io", UNICODE_CI, false); + assertContains("adi̇os", "i\u0307o", UNICODE_CI, true); + assertContains("adi̇os", "İo", UNICODE_CI, true); + assertContains("adİos", "io", UNICODE_CI, false); + assertContains("adİos", "Io", UNICODE_CI, false); + assertContains("adİos", "i\u0307o", UNICODE_CI, true); + assertContains("adİos", "İo", UNICODE_CI, true); + assertContains("i\u0307", "i", UTF8_LCASE, true); // != UNICODE_CI + assertContains("İ", "\u0307", UTF8_LCASE, false); + assertContains("İ", "i", UTF8_LCASE, false); + assertContains("i\u0307", "\u0307", UTF8_LCASE, true); // != UNICODE_CI + assertContains("i\u0307", "İ", 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\u0307o", 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\u0307o", UTF8_LCASE, true); + assertContains("adİos", "İo", UTF8_LCASE, true); // Conditional case mapping (e.g. Greek sigmas). - assertContains("σ", "σ", "UTF8_BINARY", true); - assertContains("σ", "ς", "UTF8_BINARY", false); - assertContains("σ", "Σ", "UTF8_BINARY", false); - assertContains("ς", "σ", "UTF8_BINARY", false); - assertContains("ς", "ς", "UTF8_BINARY", true); - assertContains("ς", "Σ", "UTF8_BINARY", false); - assertContains("Σ", "σ", "UTF8_BINARY", false); - assertContains("Σ", "ς", "UTF8_BINARY", false); - assertContains("Σ", "Σ", "UTF8_BINARY", true); - assertContains("σ", "σ", "UTF8_LCASE", true); - assertContains("σ", "ς", "UTF8_LCASE", true); - assertContains("σ", "Σ", "UTF8_LCASE", true); - assertContains("ς", "σ", "UTF8_LCASE", true); - assertContains("ς", "ς", "UTF8_LCASE", true); - assertContains("ς", "Σ", "UTF8_LCASE", true); - assertContains("Σ", "σ", "UTF8_LCASE", true); - assertContains("Σ", "ς", "UTF8_LCASE", true); - assertContains("Σ", "Σ", "UTF8_LCASE", true); - assertContains("σ", "σ", "UNICODE", true); - assertContains("σ", "ς", "UNICODE", false); - assertContains("σ", "Σ", "UNICODE", false); - assertContains("ς", "σ", "UNICODE", false); - assertContains("ς", "ς", "UNICODE", true); - assertContains("ς", "Σ", "UNICODE", false); - assertContains("Σ", "σ", "UNICODE", false); - assertContains("Σ", "ς", "UNICODE", false); - assertContains("Σ", "Σ", "UNICODE", true); - assertContains("σ", "σ", "UNICODE_CI", true); - assertContains("σ", "ς", "UNICODE_CI", true); - assertContains("σ", "Σ", "UNICODE_CI", true); - assertContains("ς", "σ", "UNICODE_CI", true); - assertContains("ς", "ς", "UNICODE_CI", true); - assertContains("ς", "Σ", "UNICODE_CI", true); - assertContains("Σ", "σ", "UNICODE_CI", true); - assertContains("Σ", "ς", "UNICODE_CI", true); - assertContains("Σ", "Σ", "UNICODE_CI", true); - assertContains("ΣΑΛΑΤΑ", "Σ", "UTF8_BINARY", true); - assertContains("ΣΑΛΑΤΑ", "σ", "UTF8_BINARY", false); - assertContains("ΣΑΛΑΤΑ", "ς", "UTF8_BINARY", false); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UTF8_BINARY", true); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UTF8_BINARY", false); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UTF8_BINARY", false); - assertContains("ΣΑΛΑΤΑ", "Σ", "UTF8_LCASE", true); - assertContains("ΣΑΛΑΤΑ", "σ", "UTF8_LCASE", true); - assertContains("ΣΑΛΑΤΑ", "ς", "UTF8_LCASE", true); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UTF8_LCASE", true); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UTF8_LCASE", true); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UTF8_LCASE", true); - assertContains("ΣΑΛΑΤΑ", "Σ", "UNICODE", true); - assertContains("ΣΑΛΑΤΑ", "σ", "UNICODE", false); - assertContains("ΣΑΛΑΤΑ", "ς", "UNICODE", false); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UNICODE", true); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UNICODE", false); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UNICODE", false); - assertContains("ΣΑΛΑΤΑ", "Σ", "UNICODE_CI", true); - assertContains("ΣΑΛΑΤΑ", "σ", "UNICODE_CI", true); - assertContains("ΣΑΛΑΤΑ", "ς", "UNICODE_CI", true); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UNICODE_CI", true); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UNICODE_CI", true); - assertContains("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UNICODE_CI", true); + assertContains("σ", "σ", UTF8_BINARY, true); + assertContains("σ", "ς", UTF8_BINARY, false); + assertContains("σ", "Σ", UTF8_BINARY, false); + assertContains("ς", "σ", UTF8_BINARY, false); + assertContains("ς", "ς", UTF8_BINARY, true); + assertContains("ς", "Σ", UTF8_BINARY, false); + assertContains("Σ", "σ", UTF8_BINARY, false); + assertContains("Σ", "ς", UTF8_BINARY, false); + assertContains("Σ", "Σ", UTF8_BINARY, true); + assertContains("σ", "σ", UTF8_LCASE, true); + assertContains("σ", "ς", UTF8_LCASE, true); + assertContains("σ", "Σ", UTF8_LCASE, true); + assertContains("ς", "σ", UTF8_LCASE, true); + assertContains("ς", "ς", UTF8_LCASE, true); + assertContains("ς", "Σ", UTF8_LCASE, true); + assertContains("Σ", "σ", UTF8_LCASE, true); + assertContains("Σ", "ς", UTF8_LCASE, true); + assertContains("Σ", "Σ", UTF8_LCASE, true); + assertContains("σ", "σ", UNICODE, true); + assertContains("σ", "ς", UNICODE, false); + assertContains("σ", "Σ", UNICODE, false); + assertContains("ς", "σ", UNICODE, false); + assertContains("ς", "ς", UNICODE, true); + assertContains("ς", "Σ", UNICODE, false); + assertContains("Σ", "σ", UNICODE, false); + assertContains("Σ", "ς", UNICODE, false); + assertContains("Σ", "Σ", UNICODE, true); + assertContains("σ", "σ", UNICODE_CI, true); + assertContains("σ", "ς", UNICODE_CI, true); + assertContains("σ", "Σ", UNICODE_CI, true); + assertContains("ς", "σ", UNICODE_CI, true); + assertContains("ς", "ς", UNICODE_CI, true); + assertContains("ς", "Σ", UNICODE_CI, true); + assertContains("Σ", "σ", UNICODE_CI, true); + assertContains("Σ", "ς", UNICODE_CI, true); + assertContains("Σ", "Σ", UNICODE_CI, true); + assertContains("ΣΑΛΑΤΑ", "Σ", UTF8_BINARY, true); + assertContains("ΣΑΛΑΤΑ", "σ", UTF8_BINARY, false); + assertContains("ΣΑΛΑΤΑ", "ς", UTF8_BINARY, false); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UTF8_BINARY, true); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "σ", UTF8_BINARY, false); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "ς", UTF8_BINARY, false); + assertContains("ΣΑΛΑΤΑ", "Σ", UTF8_LCASE, true); + assertContains("ΣΑΛΑΤΑ", "σ", UTF8_LCASE, true); + assertContains("ΣΑΛΑΤΑ", "ς", UTF8_LCASE, true); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UTF8_LCASE, true); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "σ", UTF8_LCASE, true); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "ς", UTF8_LCASE, true); + assertContains("ΣΑΛΑΤΑ", "Σ", UNICODE, true); + assertContains("ΣΑΛΑΤΑ", "σ", UNICODE, false); + assertContains("ΣΑΛΑΤΑ", "ς", UNICODE, false); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UNICODE, true); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "σ", UNICODE, false); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "ς", UNICODE, false); + assertContains("ΣΑΛΑΤΑ", "Σ", UNICODE_CI, true); + assertContains("ΣΑΛΑΤΑ", "σ", UNICODE_CI, true); + assertContains("ΣΑΛΑΤΑ", "ς", UNICODE_CI, true); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UNICODE_CI, true); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "σ", UNICODE_CI, true); + assertContains("ΘΑΛΑΣΣΙΝΟΣ", "ς", UNICODE_CI, true); // Surrogate pairs. - assertContains("a🙃b🙃c", "x", "UTF8_BINARY", false); - assertContains("a🙃b🙃c", "x", "UTF8_LCASE", false); - assertContains("a🙃b🙃c", "x", "UNICODE", false); - assertContains("a🙃b🙃c", "x", "UNICODE_CI", false); - assertContains("a🙃b🙃c", "b", "UTF8_BINARY", true); - assertContains("a🙃b🙃c", "b", "UTF8_LCASE", true); - assertContains("a🙃b🙃c", "b", "UNICODE", true); - assertContains("a🙃b🙃c", "b", "UNICODE_CI", true); - assertContains("a🙃b🙃c", "a🙃b", "UTF8_BINARY", true); - assertContains("a🙃b🙃c", "a🙃b", "UTF8_LCASE", true); - assertContains("a🙃b🙃c", "a🙃b", "UNICODE", true); - assertContains("a🙃b🙃c", "a🙃b", "UNICODE_CI", true); - assertContains("a🙃b🙃c", "b🙃c", "UTF8_BINARY", true); - assertContains("a🙃b🙃c", "b🙃c", "UTF8_LCASE", true); - assertContains("a🙃b🙃c", "b🙃c", "UNICODE", true); - assertContains("a🙃b🙃c", "b🙃c", "UNICODE_CI", true); - assertContains("a🙃b🙃c", "a🙃b🙃c", "UTF8_BINARY", true); - assertContains("a🙃b🙃c", "a🙃b🙃c", "UTF8_LCASE", true); - assertContains("a🙃b🙃c", "a🙃b🙃c", "UNICODE", true); - assertContains("a🙃b🙃c", "a🙃b🙃c", "UNICODE_CI", true); - assertContains("😀😆😃😄", "😄😆", "UTF8_BINARY", false); - assertContains("😀😆😃😄", "😄😆", "UTF8_LCASE", false); - assertContains("😀😆😃😄", "😄😆", "UNICODE", false); - assertContains("😀😆😃😄", "😄😆", "UNICODE_CI", false); - assertContains("😀😆😃😄", "😆😃", "UTF8_BINARY", true); - assertContains("😀😆😃😄", "😆😃", "UTF8_LCASE", true); - assertContains("😀😆😃😄", "😆😃", "UNICODE", true); - assertContains("😀😆😃😄", "😆😃", "UNICODE_CI", true); - assertContains("😀😆😃😄", "😀😆", "UTF8_BINARY", true); - assertContains("😀😆😃😄", "😀😆", "UTF8_LCASE", true); - assertContains("😀😆😃😄", "😀😆", "UNICODE", true); - assertContains("😀😆😃😄", "😀😆", "UNICODE_CI", true); - assertContains("😀😆😃😄", "😃😄", "UTF8_BINARY", true); - assertContains("😀😆😃😄", "😃😄", "UTF8_LCASE", true); - assertContains("😀😆😃😄", "😃😄", "UNICODE", true); - assertContains("😀😆😃😄", "😃😄", "UNICODE_CI", true); - assertContains("😀😆😃😄", "😀😆😃😄", "UTF8_BINARY", true); - assertContains("😀😆😃😄", "😀😆😃😄", "UTF8_LCASE", true); - assertContains("😀😆😃😄", "😀😆😃😄", "UNICODE", true); - assertContains("😀😆😃😄", "😀😆😃😄", "UNICODE_CI", true); - assertContains("𐐅", "𐐅", "UTF8_BINARY", true); - assertContains("𐐅", "𐐅", "UTF8_LCASE", true); - assertContains("𐐅", "𐐅", "UNICODE", true); - assertContains("𐐅", "𐐅", "UNICODE_CI", true); - assertContains("𐐅", "𐐭", "UTF8_BINARY", false); - assertContains("𐐅", "𐐭", "UTF8_LCASE", true); - assertContains("𐐅", "𐐭", "UNICODE", false); - assertContains("𐐅", "𐐭", "UNICODE_CI", true); - assertContains("𝔸", "𝔸", "UTF8_BINARY", true); - assertContains("𝔸", "𝔸", "UTF8_LCASE", true); - assertContains("𝔸", "𝔸", "UNICODE", true); - assertContains("𝔸", "𝔸", "UNICODE_CI", true); + assertContains("a🙃b🙃c", "x", UTF8_BINARY, false); + assertContains("a🙃b🙃c", "x", UTF8_LCASE, false); + assertContains("a🙃b🙃c", "x", UNICODE, false); + assertContains("a🙃b🙃c", "x", UNICODE_CI, false); + assertContains("a🙃b🙃c", "b", UTF8_BINARY, true); + assertContains("a🙃b🙃c", "b", UTF8_LCASE, true); + assertContains("a🙃b🙃c", "b", UNICODE, true); + assertContains("a🙃b🙃c", "b", UNICODE_CI, true); + assertContains("a🙃b🙃c", "a🙃b", UTF8_BINARY, true); + assertContains("a🙃b🙃c", "a🙃b", UTF8_LCASE, true); + assertContains("a🙃b🙃c", "a🙃b", UNICODE, true); + assertContains("a🙃b🙃c", "a🙃b", UNICODE_CI, true); + assertContains("a🙃b🙃c", "b🙃c", UTF8_BINARY, true); + assertContains("a🙃b🙃c", "b🙃c", UTF8_LCASE, true); + assertContains("a🙃b🙃c", "b🙃c", UNICODE, true); + assertContains("a🙃b🙃c", "b🙃c", UNICODE_CI, true); + assertContains("a🙃b🙃c", "a🙃b🙃c", UTF8_BINARY, true); + assertContains("a🙃b🙃c", "a🙃b🙃c", UTF8_LCASE, true); + assertContains("a🙃b🙃c", "a🙃b🙃c", UNICODE, true); + assertContains("a🙃b🙃c", "a🙃b🙃c", UNICODE_CI, true); + assertContains("😀😆😃😄", "😄😆", UTF8_BINARY, false); + assertContains("😀😆😃😄", "😄😆", UTF8_LCASE, false); + assertContains("😀😆😃😄", "😄😆", UNICODE, false); + assertContains("😀😆😃😄", "😄😆", UNICODE_CI, false); + assertContains("😀😆😃😄", "😆😃", UTF8_BINARY, true); + assertContains("😀😆😃😄", "😆😃", UTF8_LCASE, true); + assertContains("😀😆😃😄", "😆😃", UNICODE, true); + assertContains("😀😆😃😄", "😆😃", UNICODE_CI, true); + assertContains("😀😆😃😄", "😀😆", UTF8_BINARY, true); + assertContains("😀😆😃😄", "😀😆", UTF8_LCASE, true); + assertContains("😀😆😃😄", "😀😆", UNICODE, true); + assertContains("😀😆😃😄", "😀😆", UNICODE_CI, true); + assertContains("😀😆😃😄", "😃😄", UTF8_BINARY, true); + assertContains("😀😆😃😄", "😃😄", UTF8_LCASE, true); + assertContains("😀😆😃😄", "😃😄", UNICODE, true); + assertContains("😀😆😃😄", "😃😄", UNICODE_CI, true); + assertContains("😀😆😃😄", "😀😆😃😄", UTF8_BINARY, true); + assertContains("😀😆😃😄", "😀😆😃😄", UTF8_LCASE, true); + assertContains("😀😆😃😄", "😀😆😃😄", UNICODE, true); + assertContains("😀😆😃😄", "😀😆😃😄", UNICODE_CI, true); + assertContains("𐐅", "𐐅", UTF8_BINARY, true); + assertContains("𐐅", "𐐅", UTF8_LCASE, true); + assertContains("𐐅", "𐐅", UNICODE, true); + assertContains("𐐅", "𐐅", UNICODE_CI, true); + assertContains("𐐅", "𐐭", UTF8_BINARY, false); + assertContains("𐐅", "𐐭", UTF8_LCASE, true); + assertContains("𐐅", "𐐭", UNICODE, false); + assertContains("𐐅", "𐐭", UNICODE_CI, true); + assertContains("𝔸", "𝔸", UTF8_BINARY, true); + assertContains("𝔸", "𝔸", UTF8_LCASE, true); + assertContains("𝔸", "𝔸", UNICODE, true); + assertContains("𝔸", "𝔸", UNICODE_CI, true); } /** @@ -549,211 +550,211 @@ public void testStartsWith() throws SparkException { assertStartsWith("Здраво", "Здраво", collationName, true); } // Advanced tests. - assertStartsWith("abcde", "abc", "UTF8_BINARY", true); - assertStartsWith("abcde", "abd", "UTF8_BINARY", false); - assertStartsWith("abcde", "fgh", "UTF8_BINARY", false); - assertStartsWith("abcde", "abcde", "UNICODE", true); - assertStartsWith("abcde", "aBcDe", "UNICODE", false); - assertStartsWith("abcde", "fghij", "UNICODE", 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); - assertStartsWith("abcde", "123", "UNICODE_CI", false); - assertStartsWith("ab世De", "ab世", "UTF8_BINARY", true); - assertStartsWith("ab世De", "aB世", "UTF8_BINARY", false); - assertStartsWith("äbćδe", "äbć", "UTF8_BINARY", true); - assertStartsWith("äbćδe", "äBc", "UTF8_BINARY", false); - assertStartsWith("ab世De", "ab世De", "UNICODE", true); - 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_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); - 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); + assertStartsWith("abcde", "abc", UTF8_BINARY, true); + assertStartsWith("abcde", "abd", UTF8_BINARY, false); + assertStartsWith("abcde", "fgh", UTF8_BINARY, false); + assertStartsWith("abcde", "abcde", UNICODE, true); + assertStartsWith("abcde", "aBcDe", UNICODE, false); + assertStartsWith("abcde", "fghij", UNICODE, 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); + assertStartsWith("abcde", "123", UNICODE_CI, false); + assertStartsWith("ab世De", "ab世", UTF8_BINARY, true); + assertStartsWith("ab世De", "aB世", UTF8_BINARY, false); + assertStartsWith("äbćδe", "äbć", UTF8_BINARY, true); + assertStartsWith("äbćδe", "äBc", UTF8_BINARY, false); + assertStartsWith("ab世De", "ab世De", UNICODE, true); + 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_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); + 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); assertStartsWith("Ћао", "Ца", "sr_Cyrl_CI_AI", false); assertStartsWith("Ћао", "ћа", "sr_Cyrl_CI_AI", true); assertStartsWith("Ćao", "Ca", "SR_CI", false); assertStartsWith("Ćao", "Ca", "SR_CI_AI", true); assertStartsWith("Ćao", "Ća", "SR", true); // Case variation. - assertStartsWith("aBcDe", "abc", "UTF8_BINARY", false); - assertStartsWith("aBcDe", "aBc", "UTF8_BINARY", true); - assertStartsWith("aBcDe", "abcde", "UNICODE", false); - assertStartsWith("aBcDe", "aBcDe", "UNICODE", 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); + assertStartsWith("aBcDe", "abc", UTF8_BINARY, false); + assertStartsWith("aBcDe", "aBc", UTF8_BINARY, true); + assertStartsWith("aBcDe", "abcde", UNICODE, false); + assertStartsWith("aBcDe", "aBcDe", UNICODE, 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. - assertStartsWith("aBcDe", "abć", "UTF8_BINARY", false); - assertStartsWith("aBcDe", "aBć", "UTF8_BINARY", false); - assertStartsWith("aBcDe", "abćde", "UNICODE", false); - assertStartsWith("aBcDe", "aBćDe", "UNICODE", 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); + assertStartsWith("aBcDe", "abć", UTF8_BINARY, false); + assertStartsWith("aBcDe", "aBć", UTF8_BINARY, false); + assertStartsWith("aBcDe", "abćde", UNICODE, false); + assertStartsWith("aBcDe", "aBćDe", UNICODE, 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); // One-to-many case mapping (e.g. Turkish dotted I). - assertStartsWith("i\u0307", "i", "UNICODE_CI", false); - assertStartsWith("i\u0307", "İ", "UNICODE_CI", true); - assertStartsWith("İ", "i", "UNICODE_CI", false); - assertStartsWith("İİİ", "i̇i̇", "UNICODE_CI", true); - assertStartsWith("İİİ", "i̇i", "UNICODE_CI", false); - assertStartsWith("İi̇İ", "i̇İ", "UNICODE_CI", true); - assertStartsWith("i̇İi̇i̇", "İi̇İi", "UNICODE_CI", false); - assertStartsWith("i̇onic", "io", "UNICODE_CI", false); - assertStartsWith("i̇onic", "Io", "UNICODE_CI", false); - assertStartsWith("i̇onic", "i\u0307o", "UNICODE_CI", true); - assertStartsWith("i̇onic", "İo", "UNICODE_CI", true); - assertStartsWith("İonic", "io", "UNICODE_CI", false); - assertStartsWith("İonic", "Io", "UNICODE_CI", false); - assertStartsWith("İonic", "i\u0307o", "UNICODE_CI", true); - assertStartsWith("İonic", "İo", "UNICODE_CI", true); - assertStartsWith("i\u0307", "i", "UTF8_LCASE", true); // != UNICODE_CI - assertStartsWith("i\u0307", "İ", "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\u0307o", "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\u0307o", "UTF8_LCASE", true); - assertStartsWith("İonic", "İo", "UTF8_LCASE", true); - assertStartsWith("oİ", "oİ", "UTF8_LCASE", true); - assertStartsWith("oİ", "oi̇", "UTF8_LCASE", true); + assertStartsWith("i\u0307", "i", UNICODE_CI, false); + assertStartsWith("i\u0307", "İ", UNICODE_CI, true); + assertStartsWith("İ", "i", UNICODE_CI, false); + assertStartsWith("İİİ", "i̇i̇", UNICODE_CI, true); + assertStartsWith("İİİ", "i̇i", UNICODE_CI, false); + assertStartsWith("İi̇İ", "i̇İ", UNICODE_CI, true); + assertStartsWith("i̇İi̇i̇", "İi̇İi", UNICODE_CI, false); + assertStartsWith("i̇onic", "io", UNICODE_CI, false); + assertStartsWith("i̇onic", "Io", UNICODE_CI, false); + assertStartsWith("i̇onic", "i\u0307o", UNICODE_CI, true); + assertStartsWith("i̇onic", "İo", UNICODE_CI, true); + assertStartsWith("İonic", "io", UNICODE_CI, false); + assertStartsWith("İonic", "Io", UNICODE_CI, false); + assertStartsWith("İonic", "i\u0307o", UNICODE_CI, true); + assertStartsWith("İonic", "İo", UNICODE_CI, true); + assertStartsWith("i\u0307", "i", UTF8_LCASE, true); // != UNICODE_CI + assertStartsWith("i\u0307", "İ", 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\u0307o", 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\u0307o", UTF8_LCASE, true); + assertStartsWith("İonic", "İo", UTF8_LCASE, true); + assertStartsWith("oİ", "oİ", UTF8_LCASE, true); + assertStartsWith("oİ", "oi̇", UTF8_LCASE, true); // Conditional case mapping (e.g. Greek sigmas). - assertStartsWith("σ", "σ", "UTF8_BINARY", true); - assertStartsWith("σ", "ς", "UTF8_BINARY", false); - assertStartsWith("σ", "Σ", "UTF8_BINARY", false); - assertStartsWith("ς", "σ", "UTF8_BINARY", false); - assertStartsWith("ς", "ς", "UTF8_BINARY", true); - assertStartsWith("ς", "Σ", "UTF8_BINARY", false); - assertStartsWith("Σ", "σ", "UTF8_BINARY", false); - assertStartsWith("Σ", "ς", "UTF8_BINARY", false); - assertStartsWith("Σ", "Σ", "UTF8_BINARY", true); - assertStartsWith("σ", "σ", "UTF8_LCASE", true); - assertStartsWith("σ", "ς", "UTF8_LCASE", true); - assertStartsWith("σ", "Σ", "UTF8_LCASE", true); - assertStartsWith("ς", "σ", "UTF8_LCASE", true); - assertStartsWith("ς", "ς", "UTF8_LCASE", true); - assertStartsWith("ς", "Σ", "UTF8_LCASE", true); - assertStartsWith("Σ", "σ", "UTF8_LCASE", true); - assertStartsWith("Σ", "ς", "UTF8_LCASE", true); - assertStartsWith("Σ", "Σ", "UTF8_LCASE", true); - assertStartsWith("σ", "σ", "UNICODE", true); - assertStartsWith("σ", "ς", "UNICODE", false); - assertStartsWith("σ", "Σ", "UNICODE", false); - assertStartsWith("ς", "σ", "UNICODE", false); - assertStartsWith("ς", "ς", "UNICODE", true); - assertStartsWith("ς", "Σ", "UNICODE", false); - assertStartsWith("Σ", "σ", "UNICODE", false); - assertStartsWith("Σ", "ς", "UNICODE", false); - assertStartsWith("Σ", "Σ", "UNICODE", true); - assertStartsWith("σ", "σ", "UNICODE_CI", true); - assertStartsWith("σ", "ς", "UNICODE_CI", true); - assertStartsWith("σ", "Σ", "UNICODE_CI", true); - assertStartsWith("ς", "σ", "UNICODE_CI", true); - assertStartsWith("ς", "ς", "UNICODE_CI", true); - assertStartsWith("ς", "Σ", "UNICODE_CI", true); - assertStartsWith("Σ", "σ", "UNICODE_CI", true); - assertStartsWith("Σ", "ς", "UNICODE_CI", true); - assertStartsWith("Σ", "Σ", "UNICODE_CI", true); - assertStartsWith("ΣΑΛΑΤΑ", "Σ", "UTF8_BINARY", true); - assertStartsWith("ΣΑΛΑΤΑ", "σ", "UTF8_BINARY", false); - assertStartsWith("ΣΑΛΑΤΑ", "ς", "UTF8_BINARY", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UTF8_BINARY", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UTF8_BINARY", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UTF8_BINARY", false); - assertStartsWith("ΣΑΛΑΤΑ", "Σ", "UTF8_LCASE", true); - assertStartsWith("ΣΑΛΑΤΑ", "σ", "UTF8_LCASE", true); - assertStartsWith("ΣΑΛΑΤΑ", "ς", "UTF8_LCASE", true); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UTF8_LCASE", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UTF8_LCASE", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UTF8_LCASE", false); - assertStartsWith("ΣΑΛΑΤΑ", "Σ", "UNICODE", true); - assertStartsWith("ΣΑΛΑΤΑ", "σ", "UNICODE", false); - assertStartsWith("ΣΑΛΑΤΑ", "ς", "UNICODE", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UNICODE", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UNICODE", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UNICODE", false); - assertStartsWith("ΣΑΛΑΤΑ", "Σ", "UNICODE_CI", true); - assertStartsWith("ΣΑΛΑΤΑ", "σ", "UNICODE_CI", true); - assertStartsWith("ΣΑΛΑΤΑ", "ς", "UNICODE_CI", true); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UNICODE_CI", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UNICODE_CI", false); - assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UNICODE_CI", false); + assertStartsWith("σ", "σ", UTF8_BINARY, true); + assertStartsWith("σ", "ς", UTF8_BINARY, false); + assertStartsWith("σ", "Σ", UTF8_BINARY, false); + assertStartsWith("ς", "σ", UTF8_BINARY, false); + assertStartsWith("ς", "ς", UTF8_BINARY, true); + assertStartsWith("ς", "Σ", UTF8_BINARY, false); + assertStartsWith("Σ", "σ", UTF8_BINARY, false); + assertStartsWith("Σ", "ς", UTF8_BINARY, false); + assertStartsWith("Σ", "Σ", UTF8_BINARY, true); + assertStartsWith("σ", "σ", UTF8_LCASE, true); + assertStartsWith("σ", "ς", UTF8_LCASE, true); + assertStartsWith("σ", "Σ", UTF8_LCASE, true); + assertStartsWith("ς", "σ", UTF8_LCASE, true); + assertStartsWith("ς", "ς", UTF8_LCASE, true); + assertStartsWith("ς", "Σ", UTF8_LCASE, true); + assertStartsWith("Σ", "σ", UTF8_LCASE, true); + assertStartsWith("Σ", "ς", UTF8_LCASE, true); + assertStartsWith("Σ", "Σ", UTF8_LCASE, true); + assertStartsWith("σ", "σ", UNICODE, true); + assertStartsWith("σ", "ς", UNICODE, false); + assertStartsWith("σ", "Σ", UNICODE, false); + assertStartsWith("ς", "σ", UNICODE, false); + assertStartsWith("ς", "ς", UNICODE, true); + assertStartsWith("ς", "Σ", UNICODE, false); + assertStartsWith("Σ", "σ", UNICODE, false); + assertStartsWith("Σ", "ς", UNICODE, false); + assertStartsWith("Σ", "Σ", UNICODE, true); + assertStartsWith("σ", "σ", UNICODE_CI, true); + assertStartsWith("σ", "ς", UNICODE_CI, true); + assertStartsWith("σ", "Σ", UNICODE_CI, true); + assertStartsWith("ς", "σ", UNICODE_CI, true); + assertStartsWith("ς", "ς", UNICODE_CI, true); + assertStartsWith("ς", "Σ", UNICODE_CI, true); + assertStartsWith("Σ", "σ", UNICODE_CI, true); + assertStartsWith("Σ", "ς", UNICODE_CI, true); + assertStartsWith("Σ", "Σ", UNICODE_CI, true); + assertStartsWith("ΣΑΛΑΤΑ", "Σ", UTF8_BINARY, true); + assertStartsWith("ΣΑΛΑΤΑ", "σ", UTF8_BINARY, false); + assertStartsWith("ΣΑΛΑΤΑ", "ς", UTF8_BINARY, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UTF8_BINARY, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", UTF8_BINARY, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", UTF8_BINARY, false); + assertStartsWith("ΣΑΛΑΤΑ", "Σ", UTF8_LCASE, true); + assertStartsWith("ΣΑΛΑΤΑ", "σ", UTF8_LCASE, true); + assertStartsWith("ΣΑΛΑΤΑ", "ς", UTF8_LCASE, true); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UTF8_LCASE, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", UTF8_LCASE, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", UTF8_LCASE, false); + assertStartsWith("ΣΑΛΑΤΑ", "Σ", UNICODE, true); + assertStartsWith("ΣΑΛΑΤΑ", "σ", UNICODE, false); + assertStartsWith("ΣΑΛΑΤΑ", "ς", UNICODE, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UNICODE, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", UNICODE, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", UNICODE, false); + assertStartsWith("ΣΑΛΑΤΑ", "Σ", UNICODE_CI, true); + assertStartsWith("ΣΑΛΑΤΑ", "σ", UNICODE_CI, true); + assertStartsWith("ΣΑΛΑΤΑ", "ς", UNICODE_CI, true); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UNICODE_CI, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", UNICODE_CI, false); + assertStartsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", UNICODE_CI, false); // Surrogate pairs. - assertStartsWith("a🙃b🙃c", "x", "UTF8_BINARY", false); - assertStartsWith("a🙃b🙃c", "x", "UTF8_LCASE", false); - assertStartsWith("a🙃b🙃c", "x", "UNICODE", false); - assertStartsWith("a🙃b🙃c", "x", "UNICODE_CI", false); - assertStartsWith("a🙃b🙃c", "b", "UTF8_BINARY", false); - assertStartsWith("a🙃b🙃c", "b", "UTF8_LCASE", false); - assertStartsWith("a🙃b🙃c", "b", "UNICODE", false); - assertStartsWith("a🙃b🙃c", "b", "UNICODE_CI", false); - assertStartsWith("a🙃b🙃c", "a🙃b", "UTF8_BINARY", true); - assertStartsWith("a🙃b🙃c", "a🙃b", "UTF8_LCASE", true); - assertStartsWith("a🙃b🙃c", "a🙃b", "UNICODE", true); - assertStartsWith("a🙃b🙃c", "a🙃b", "UNICODE_CI", true); - assertStartsWith("a🙃b🙃c", "b🙃c", "UTF8_BINARY", false); - assertStartsWith("a🙃b🙃c", "b🙃c", "UTF8_LCASE", false); - assertStartsWith("a🙃b🙃c", "b🙃c", "UNICODE", false); - assertStartsWith("a🙃b🙃c", "b🙃c", "UNICODE_CI", false); - assertStartsWith("a🙃b🙃c", "a🙃b🙃c", "UTF8_BINARY", true); - assertStartsWith("a🙃b🙃c", "a🙃b🙃c", "UTF8_LCASE", true); - assertStartsWith("a🙃b🙃c", "a🙃b🙃c", "UNICODE", true); - assertStartsWith("a🙃b🙃c", "a🙃b🙃c", "UNICODE_CI", true); - assertStartsWith("😀😆😃😄", "😄😆", "UTF8_BINARY", false); - assertStartsWith("😀😆😃😄", "😄😆", "UTF8_LCASE", false); - assertStartsWith("😀😆😃😄", "😄😆", "UNICODE", false); - assertStartsWith("😀😆😃😄", "😄😆", "UNICODE_CI", false); - assertStartsWith("😀😆😃😄", "😆😃", "UTF8_BINARY", false); - assertStartsWith("😀😆😃😄", "😆😃", "UTF8_LCASE", false); - assertStartsWith("😀😆😃😄", "😆😃", "UNICODE", false); - assertStartsWith("😀😆😃😄", "😆😃", "UNICODE_CI", false); - assertStartsWith("😀😆😃😄", "😀😆", "UTF8_BINARY", true); - assertStartsWith("😀😆😃😄", "😀😆", "UTF8_LCASE", true); - assertStartsWith("😀😆😃😄", "😀😆", "UNICODE", true); - assertStartsWith("😀😆😃😄", "😀😆", "UNICODE_CI", true); - assertStartsWith("😀😆😃😄", "😃😄", "UTF8_BINARY", false); - assertStartsWith("😀😆😃😄", "😃😄", "UTF8_LCASE", false); - assertStartsWith("😀😆😃😄", "😃😄", "UNICODE", false); - assertStartsWith("😀😆😃😄", "😃😄", "UNICODE_CI", false); - assertStartsWith("😀😆😃😄", "😀😆😃😄", "UTF8_BINARY", true); - assertStartsWith("😀😆😃😄", "😀😆😃😄", "UTF8_LCASE", true); - assertStartsWith("😀😆😃😄", "😀😆😃😄", "UNICODE", true); - assertStartsWith("😀😆😃😄", "😀😆😃😄", "UNICODE_CI", true); - assertStartsWith("𐐅", "𐐅", "UTF8_BINARY", true); - assertStartsWith("𐐅", "𐐅", "UTF8_LCASE", true); - assertStartsWith("𐐅", "𐐅", "UNICODE", true); - assertStartsWith("𐐅", "𐐅", "UNICODE_CI", true); - assertStartsWith("𐐅", "𐐭", "UTF8_BINARY", false); - assertStartsWith("𐐅", "𐐭", "UTF8_LCASE", true); - assertStartsWith("𐐅", "𐐭", "UNICODE", false); - assertStartsWith("𐐅", "𐐭", "UNICODE_CI", true); - assertStartsWith("𝔸", "𝔸", "UTF8_BINARY", true); - assertStartsWith("𝔸", "𝔸", "UTF8_LCASE", true); - assertStartsWith("𝔸", "𝔸", "UNICODE", true); - assertStartsWith("𝔸", "𝔸", "UNICODE_CI", true); + assertStartsWith("a🙃b🙃c", "x", UTF8_BINARY, false); + assertStartsWith("a🙃b🙃c", "x", UTF8_LCASE, false); + assertStartsWith("a🙃b🙃c", "x", UNICODE, false); + assertStartsWith("a🙃b🙃c", "x", UNICODE_CI, false); + assertStartsWith("a🙃b🙃c", "b", UTF8_BINARY, false); + assertStartsWith("a🙃b🙃c", "b", UTF8_LCASE, false); + assertStartsWith("a🙃b🙃c", "b", UNICODE, false); + assertStartsWith("a🙃b🙃c", "b", UNICODE_CI, false); + assertStartsWith("a🙃b🙃c", "a🙃b", UTF8_BINARY, true); + assertStartsWith("a🙃b🙃c", "a🙃b", UTF8_LCASE, true); + assertStartsWith("a🙃b🙃c", "a🙃b", UNICODE, true); + assertStartsWith("a🙃b🙃c", "a🙃b", UNICODE_CI, true); + assertStartsWith("a🙃b🙃c", "b🙃c", UTF8_BINARY, false); + assertStartsWith("a🙃b🙃c", "b🙃c", UTF8_LCASE, false); + assertStartsWith("a🙃b🙃c", "b🙃c", UNICODE, false); + assertStartsWith("a🙃b🙃c", "b🙃c", UNICODE_CI, false); + assertStartsWith("a🙃b🙃c", "a🙃b🙃c", UTF8_BINARY, true); + assertStartsWith("a🙃b🙃c", "a🙃b🙃c", UTF8_LCASE, true); + assertStartsWith("a🙃b🙃c", "a🙃b🙃c", UNICODE, true); + assertStartsWith("a🙃b🙃c", "a🙃b🙃c", UNICODE_CI, true); + assertStartsWith("😀😆😃😄", "😄😆", UTF8_BINARY, false); + assertStartsWith("😀😆😃😄", "😄😆", UTF8_LCASE, false); + assertStartsWith("😀😆😃😄", "😄😆", UNICODE, false); + assertStartsWith("😀😆😃😄", "😄😆", UNICODE_CI, false); + assertStartsWith("😀😆😃😄", "😆😃", UTF8_BINARY, false); + assertStartsWith("😀😆😃😄", "😆😃", UTF8_LCASE, false); + assertStartsWith("😀😆😃😄", "😆😃", UNICODE, false); + assertStartsWith("😀😆😃😄", "😆😃", UNICODE_CI, false); + assertStartsWith("😀😆😃😄", "😀😆", UTF8_BINARY, true); + assertStartsWith("😀😆😃😄", "😀😆", UTF8_LCASE, true); + assertStartsWith("😀😆😃😄", "😀😆", UNICODE, true); + assertStartsWith("😀😆😃😄", "😀😆", UNICODE_CI, true); + assertStartsWith("😀😆😃😄", "😃😄", UTF8_BINARY, false); + assertStartsWith("😀😆😃😄", "😃😄", UTF8_LCASE, false); + assertStartsWith("😀😆😃😄", "😃😄", UNICODE, false); + assertStartsWith("😀😆😃😄", "😃😄", UNICODE_CI, false); + assertStartsWith("😀😆😃😄", "😀😆😃😄", UTF8_BINARY, true); + assertStartsWith("😀😆😃😄", "😀😆😃😄", UTF8_LCASE, true); + assertStartsWith("😀😆😃😄", "😀😆😃😄", UNICODE, true); + assertStartsWith("😀😆😃😄", "😀😆😃😄", UNICODE_CI, true); + assertStartsWith("𐐅", "𐐅", UTF8_BINARY, true); + assertStartsWith("𐐅", "𐐅", UTF8_LCASE, true); + assertStartsWith("𐐅", "𐐅", UNICODE, true); + assertStartsWith("𐐅", "𐐅", UNICODE_CI, true); + assertStartsWith("𐐅", "𐐭", UTF8_BINARY, false); + assertStartsWith("𐐅", "𐐭", UTF8_LCASE, true); + assertStartsWith("𐐅", "𐐭", UNICODE, false); + assertStartsWith("𐐅", "𐐭", UNICODE_CI, true); + assertStartsWith("𝔸", "𝔸", UTF8_BINARY, true); + assertStartsWith("𝔸", "𝔸", UTF8_LCASE, true); + assertStartsWith("𝔸", "𝔸", UNICODE, true); + assertStartsWith("𝔸", "𝔸", UNICODE_CI, true); } /** @@ -806,212 +807,212 @@ public void testEndsWith() throws SparkException { assertEndsWith("Здраво", "Здраво", collationName, true); } // Advanced tests. - assertEndsWith("abcde", "cde", "UTF8_BINARY", true); - assertEndsWith("abcde", "bde", "UTF8_BINARY", false); - assertEndsWith("abcde", "fgh", "UTF8_BINARY", false); - assertEndsWith("abcde", "abcde", "UNICODE", true); - assertEndsWith("abcde", "aBcDe", "UNICODE", false); - assertEndsWith("abcde", "fghij", "UNICODE", 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); - assertEndsWith("abcde", "123", "UNICODE_CI", false); - assertEndsWith("ab世De", "世De", "UTF8_BINARY", true); - assertEndsWith("ab世De", "世dE", "UTF8_BINARY", false); - assertEndsWith("äbćδe", "ćδe", "UTF8_BINARY", true); - assertEndsWith("äbćδe", "cΔé", "UTF8_BINARY", false); - assertEndsWith("ab世De", "ab世De", "UNICODE", true); - 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_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); - 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); + assertEndsWith("abcde", "cde", UTF8_BINARY, true); + assertEndsWith("abcde", "bde", UTF8_BINARY, false); + assertEndsWith("abcde", "fgh", UTF8_BINARY, false); + assertEndsWith("abcde", "abcde", UNICODE, true); + assertEndsWith("abcde", "aBcDe", UNICODE, false); + assertEndsWith("abcde", "fghij", UNICODE, 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); + assertEndsWith("abcde", "123", UNICODE_CI, false); + assertEndsWith("ab世De", "世De", UTF8_BINARY, true); + assertEndsWith("ab世De", "世dE", UTF8_BINARY, false); + assertEndsWith("äbćδe", "ćδe", UTF8_BINARY, true); + assertEndsWith("äbćδe", "cΔé", UTF8_BINARY, false); + assertEndsWith("ab世De", "ab世De", UNICODE, true); + 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_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); + 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); assertEndsWith("Ћевапчићи", "цици", "sr_Cyrl_CI_AI", false); assertEndsWith("Ћевапчићи", "чИЋи", "sr_Cyrl_CI_AI", true); assertEndsWith("Ćevapčići", "cici", "SR_CI", false); assertEndsWith("Ćevapčići", "cici", "SR_CI_AI", true); assertEndsWith("Ćevapčići", "čići", "SR", true); // Case variation. - assertEndsWith("aBcDe", "cde", "UTF8_BINARY", false); - assertEndsWith("aBcDe", "cDe", "UTF8_BINARY", true); - assertEndsWith("aBcDe", "abcde", "UNICODE", false); - assertEndsWith("aBcDe", "aBcDe", "UNICODE", 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); + assertEndsWith("aBcDe", "cde", UTF8_BINARY, false); + assertEndsWith("aBcDe", "cDe", UTF8_BINARY, true); + assertEndsWith("aBcDe", "abcde", UNICODE, false); + assertEndsWith("aBcDe", "aBcDe", UNICODE, 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. - assertEndsWith("aBcDe", "ćde", "UTF8_BINARY", false); - assertEndsWith("aBcDe", "ćDe", "UTF8_BINARY", false); - assertEndsWith("aBcDe", "abćde", "UNICODE", false); - assertEndsWith("aBcDe", "aBćDe", "UNICODE", 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); + assertEndsWith("aBcDe", "ćde", UTF8_BINARY, false); + assertEndsWith("aBcDe", "ćDe", UTF8_BINARY, false); + assertEndsWith("aBcDe", "abćde", UNICODE, false); + assertEndsWith("aBcDe", "aBćDe", UNICODE, 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); // One-to-many case mapping (e.g. Turkish dotted I). - assertEndsWith("i\u0307", "\u0307", "UNICODE_CI", false); - assertEndsWith("i\u0307", "İ", "UNICODE_CI", true); - assertEndsWith("İ", "i", "UNICODE_CI", false); - assertEndsWith("İİİ", "i̇i̇", "UNICODE_CI", true); - assertEndsWith("İİİ", "ii̇", "UNICODE_CI", false); - assertEndsWith("İi̇İ", "İi̇", "UNICODE_CI", true); - assertEndsWith("i̇İi̇i̇", "\u0307İi̇İ", "UNICODE_CI", false); - assertEndsWith("the i\u0307o", "io", "UNICODE_CI", false); - assertEndsWith("the i\u0307o", "Io", "UNICODE_CI", false); - assertEndsWith("the i\u0307o", "i\u0307o", "UNICODE_CI", true); - assertEndsWith("the i\u0307o", "İo", "UNICODE_CI", true); - assertEndsWith("the İo", "io", "UNICODE_CI", false); - assertEndsWith("the İo", "Io", "UNICODE_CI", false); - assertEndsWith("the İo", "i\u0307o", "UNICODE_CI", true); - assertEndsWith("the İo", "İo", "UNICODE_CI", true); - assertEndsWith("i\u0307", "\u0307", "UTF8_LCASE", true); // != UNICODE_CI - assertEndsWith("i\u0307", "İ", "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\u0307o", "io", "UTF8_LCASE", false); - assertEndsWith("the i\u0307o", "Io", "UTF8_LCASE", false); - assertEndsWith("the i\u0307o", "i\u0307o", "UTF8_LCASE", true); - assertEndsWith("the i\u0307o", "İo", "UTF8_LCASE", true); - assertEndsWith("the İo", "io", "UTF8_LCASE", false); - assertEndsWith("the İo", "Io", "UTF8_LCASE", false); - assertEndsWith("the İo", "i\u0307o", "UTF8_LCASE", true); - assertEndsWith("the İo", "İo", "UTF8_LCASE", true); - assertEndsWith("İo", "İo", "UTF8_LCASE", true); - assertEndsWith("İo", "i̇o", "UTF8_LCASE", true); + assertEndsWith("i\u0307", "\u0307", UNICODE_CI, false); + assertEndsWith("i\u0307", "İ", UNICODE_CI, true); + assertEndsWith("İ", "i", UNICODE_CI, false); + assertEndsWith("İİİ", "i̇i̇", UNICODE_CI, true); + assertEndsWith("İİİ", "ii̇", UNICODE_CI, false); + assertEndsWith("İi̇İ", "İi̇", UNICODE_CI, true); + assertEndsWith("i̇İi̇i̇", "\u0307İi̇İ", UNICODE_CI, false); + assertEndsWith("the i\u0307o", "io", UNICODE_CI, false); + assertEndsWith("the i\u0307o", "Io", UNICODE_CI, false); + assertEndsWith("the i\u0307o", "i\u0307o", UNICODE_CI, true); + assertEndsWith("the i\u0307o", "İo", UNICODE_CI, true); + assertEndsWith("the İo", "io", UNICODE_CI, false); + assertEndsWith("the İo", "Io", UNICODE_CI, false); + assertEndsWith("the İo", "i\u0307o", UNICODE_CI, true); + assertEndsWith("the İo", "İo", UNICODE_CI, true); + assertEndsWith("i\u0307", "\u0307", UTF8_LCASE, true); // != UNICODE_CI + assertEndsWith("i\u0307", "İ", 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\u0307o", "io", UTF8_LCASE, false); + assertEndsWith("the i\u0307o", "Io", UTF8_LCASE, false); + assertEndsWith("the i\u0307o", "i\u0307o", UTF8_LCASE, true); + assertEndsWith("the i\u0307o", "İo", UTF8_LCASE, true); + assertEndsWith("the İo", "io", UTF8_LCASE, false); + assertEndsWith("the İo", "Io", UTF8_LCASE, false); + assertEndsWith("the İo", "i\u0307o", UTF8_LCASE, true); + assertEndsWith("the İo", "İo", UTF8_LCASE, true); + assertEndsWith("İo", "İo", UTF8_LCASE, true); + assertEndsWith("İo", "i̇o", UTF8_LCASE, true); // Conditional case mapping (e.g. Greek sigmas). - assertEndsWith("σ", "σ", "UTF8_BINARY", true); - assertEndsWith("σ", "ς", "UTF8_BINARY", false); - assertEndsWith("σ", "Σ", "UTF8_BINARY", false); - assertEndsWith("ς", "σ", "UTF8_BINARY", false); - assertEndsWith("ς", "ς", "UTF8_BINARY", true); - assertEndsWith("ς", "Σ", "UTF8_BINARY", false); - assertEndsWith("Σ", "σ", "UTF8_BINARY", false); - assertEndsWith("Σ", "ς", "UTF8_BINARY", false); - assertEndsWith("Σ", "Σ", "UTF8_BINARY", true); - assertEndsWith("σ", "σ", "UTF8_LCASE", true); - assertEndsWith("σ", "ς", "UTF8_LCASE", true); - assertEndsWith("σ", "Σ", "UTF8_LCASE", true); - assertEndsWith("ς", "σ", "UTF8_LCASE", true); - assertEndsWith("ς", "ς", "UTF8_LCASE", true); - assertEndsWith("ς", "Σ", "UTF8_LCASE", true); - assertEndsWith("Σ", "σ", "UTF8_LCASE", true); - assertEndsWith("Σ", "ς", "UTF8_LCASE", true); - assertEndsWith("Σ", "Σ", "UTF8_LCASE", true); - assertEndsWith("σ", "σ", "UNICODE", true); - assertEndsWith("σ", "ς", "UNICODE", false); - assertEndsWith("σ", "Σ", "UNICODE", false); - assertEndsWith("ς", "σ", "UNICODE", false); - assertEndsWith("ς", "ς", "UNICODE", true); - assertEndsWith("ς", "Σ", "UNICODE", false); - assertEndsWith("Σ", "σ", "UNICODE", false); - assertEndsWith("Σ", "ς", "UNICODE", false); - assertEndsWith("Σ", "Σ", "UNICODE", true); - assertEndsWith("σ", "σ", "UNICODE_CI", true); - assertEndsWith("σ", "ς", "UNICODE_CI", true); - assertEndsWith("σ", "Σ", "UNICODE_CI", true); - assertEndsWith("ς", "σ", "UNICODE_CI", true); - assertEndsWith("ς", "ς", "UNICODE_CI", true); - assertEndsWith("ς", "Σ", "UNICODE_CI", true); - assertEndsWith("Σ", "σ", "UNICODE_CI", true); - assertEndsWith("Σ", "ς", "UNICODE_CI", true); - assertEndsWith("Σ", "Σ", "UNICODE_CI", true); - assertEndsWith("ΣΑΛΑΤΑ", "Σ", "UTF8_BINARY", false); - assertEndsWith("ΣΑΛΑΤΑ", "σ", "UTF8_BINARY", false); - assertEndsWith("ΣΑΛΑΤΑ", "ς", "UTF8_BINARY", false); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UTF8_BINARY", true); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UTF8_BINARY", false); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UTF8_BINARY", false); - assertEndsWith("ΣΑΛΑΤΑ", "Σ", "UTF8_LCASE", false); - assertEndsWith("ΣΑΛΑΤΑ", "σ", "UTF8_LCASE", false); - assertEndsWith("ΣΑΛΑΤΑ", "ς", "UTF8_LCASE", false); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UTF8_LCASE", true); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UTF8_LCASE", true); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UTF8_LCASE", true); - assertEndsWith("ΣΑΛΑΤΑ", "Σ", "UNICODE", false); - assertEndsWith("ΣΑΛΑΤΑ", "σ", "UNICODE", false); - assertEndsWith("ΣΑΛΑΤΑ", "ς", "UNICODE", false); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UNICODE", true); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UNICODE", false); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UNICODE", false); - assertEndsWith("ΣΑΛΑΤΑ", "Σ", "UNICODE_CI", false); - assertEndsWith("ΣΑΛΑΤΑ", "σ", "UNICODE_CI", false); - assertEndsWith("ΣΑΛΑΤΑ", "ς", "UNICODE_CI", false); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", "UNICODE_CI", true); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", "UNICODE_CI", true); - assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", "UNICODE_CI", true); + assertEndsWith("σ", "σ", UTF8_BINARY, true); + assertEndsWith("σ", "ς", UTF8_BINARY, false); + assertEndsWith("σ", "Σ", UTF8_BINARY, false); + assertEndsWith("ς", "σ", UTF8_BINARY, false); + assertEndsWith("ς", "ς", UTF8_BINARY, true); + assertEndsWith("ς", "Σ", UTF8_BINARY, false); + assertEndsWith("Σ", "σ", UTF8_BINARY, false); + assertEndsWith("Σ", "ς", UTF8_BINARY, false); + assertEndsWith("Σ", "Σ", UTF8_BINARY, true); + assertEndsWith("σ", "σ", UTF8_LCASE, true); + assertEndsWith("σ", "ς", UTF8_LCASE, true); + assertEndsWith("σ", "Σ", UTF8_LCASE, true); + assertEndsWith("ς", "σ", UTF8_LCASE, true); + assertEndsWith("ς", "ς", UTF8_LCASE, true); + assertEndsWith("ς", "Σ", UTF8_LCASE, true); + assertEndsWith("Σ", "σ", UTF8_LCASE, true); + assertEndsWith("Σ", "ς", UTF8_LCASE, true); + assertEndsWith("Σ", "Σ", UTF8_LCASE, true); + assertEndsWith("σ", "σ", UNICODE, true); + assertEndsWith("σ", "ς", UNICODE, false); + assertEndsWith("σ", "Σ", UNICODE, false); + assertEndsWith("ς", "σ", UNICODE, false); + assertEndsWith("ς", "ς", UNICODE, true); + assertEndsWith("ς", "Σ", UNICODE, false); + assertEndsWith("Σ", "σ", UNICODE, false); + assertEndsWith("Σ", "ς", UNICODE, false); + assertEndsWith("Σ", "Σ", UNICODE, true); + assertEndsWith("σ", "σ", UNICODE_CI, true); + assertEndsWith("σ", "ς", UNICODE_CI, true); + assertEndsWith("σ", "Σ", UNICODE_CI, true); + assertEndsWith("ς", "σ", UNICODE_CI, true); + assertEndsWith("ς", "ς", UNICODE_CI, true); + assertEndsWith("ς", "Σ", UNICODE_CI, true); + assertEndsWith("Σ", "σ", UNICODE_CI, true); + assertEndsWith("Σ", "ς", UNICODE_CI, true); + assertEndsWith("Σ", "Σ", UNICODE_CI, true); + assertEndsWith("ΣΑΛΑΤΑ", "Σ", UTF8_BINARY, false); + assertEndsWith("ΣΑΛΑΤΑ", "σ", UTF8_BINARY, false); + assertEndsWith("ΣΑΛΑΤΑ", "ς", UTF8_BINARY, false); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UTF8_BINARY, true); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", UTF8_BINARY, false); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", UTF8_BINARY, false); + assertEndsWith("ΣΑΛΑΤΑ", "Σ", UTF8_LCASE, false); + assertEndsWith("ΣΑΛΑΤΑ", "σ", UTF8_LCASE, false); + assertEndsWith("ΣΑΛΑΤΑ", "ς", UTF8_LCASE, false); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UTF8_LCASE, true); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", UTF8_LCASE, true); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", UTF8_LCASE, true); + assertEndsWith("ΣΑΛΑΤΑ", "Σ", UNICODE, false); + assertEndsWith("ΣΑΛΑΤΑ", "σ", UNICODE, false); + assertEndsWith("ΣΑΛΑΤΑ", "ς", UNICODE, false); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UNICODE, true); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", UNICODE, false); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", UNICODE, false); + assertEndsWith("ΣΑΛΑΤΑ", "Σ", UNICODE_CI, false); + assertEndsWith("ΣΑΛΑΤΑ", "σ", UNICODE_CI, false); + assertEndsWith("ΣΑΛΑΤΑ", "ς", UNICODE_CI, false); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "Σ", UNICODE_CI, true); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "σ", UNICODE_CI, true); + assertEndsWith("ΘΑΛΑΣΣΙΝΟΣ", "ς", UNICODE_CI, true); // Surrogate pairs. - assertEndsWith("a🙃b🙃c", "x", "UTF8_BINARY", false); - assertEndsWith("a🙃b🙃c", "x", "UTF8_LCASE", false); - assertEndsWith("a🙃b🙃c", "x", "UNICODE", false); - assertEndsWith("a🙃b🙃c", "x", "UNICODE_CI", false); - assertEndsWith("a🙃b🙃c", "b", "UTF8_BINARY", false); - assertEndsWith("a🙃b🙃c", "b", "UTF8_LCASE", false); - assertEndsWith("a🙃b🙃c", "b", "UNICODE", false); - assertEndsWith("a🙃b🙃c", "b", "UNICODE_CI", false); - assertEndsWith("a🙃b🙃c", "a🙃b", "UTF8_BINARY", false); - assertEndsWith("a🙃b🙃c", "a🙃b", "UTF8_LCASE", false); - assertEndsWith("a🙃b🙃c", "a🙃b", "UNICODE", false); - assertEndsWith("a🙃b🙃c", "a🙃b", "UNICODE_CI", false); - assertEndsWith("a🙃b🙃c", "b🙃c", "UTF8_BINARY", true); - assertEndsWith("a🙃b🙃c", "b🙃c", "UTF8_LCASE", true); - assertEndsWith("a🙃b🙃c", "b🙃c", "UNICODE", true); - assertEndsWith("a🙃b🙃c", "b🙃c", "UNICODE_CI", true); - assertEndsWith("a🙃b🙃c", "a🙃b🙃c", "UTF8_BINARY", true); - assertEndsWith("a🙃b🙃c", "a🙃b🙃c", "UTF8_LCASE", true); - assertEndsWith("a🙃b🙃c", "a🙃b🙃c", "UNICODE", true); - assertEndsWith("a🙃b🙃c", "a🙃b🙃c", "UNICODE_CI", true); - assertEndsWith("😀😆😃😄", "😄😆", "UTF8_BINARY", false); - assertEndsWith("😀😆😃😄", "😄😆", "UTF8_LCASE", false); - assertEndsWith("😀😆😃😄", "😄😆", "UNICODE", false); - assertEndsWith("😀😆😃😄", "😄😆", "UNICODE_CI", false); - assertEndsWith("😀😆😃😄", "😆😃", "UTF8_BINARY", false); - assertEndsWith("😀😆😃😄", "😆😃", "UTF8_LCASE", false); - assertEndsWith("😀😆😃😄", "😆😃", "UNICODE", false); - assertEndsWith("😀😆😃😄", "😆😃", "UNICODE_CI", false); - assertEndsWith("😀😆😃😄", "😀😆", "UTF8_BINARY", false); - assertEndsWith("😀😆😃😄", "😀😆", "UTF8_LCASE", false); - assertEndsWith("😀😆😃😄", "😀😆", "UNICODE", false); - assertEndsWith("😀😆😃😄", "😀😆", "UNICODE_CI", false); - assertEndsWith("😀😆😃😄", "😃😄", "UTF8_BINARY", true); - assertEndsWith("😀😆😃😄", "😃😄", "UTF8_LCASE", true); - assertEndsWith("😀😆😃😄", "😃😄", "UNICODE", true); - assertEndsWith("😀😆😃😄", "😃😄", "UNICODE_CI", true); - assertEndsWith("😀😆😃😄", "😀😆😃😄", "UTF8_BINARY", true); - assertEndsWith("😀😆😃😄", "😀😆😃😄", "UTF8_LCASE", true); - assertEndsWith("😀😆😃😄", "😀😆😃😄", "UNICODE", true); - assertEndsWith("😀😆😃😄", "😀😆😃😄", "UNICODE_CI", true); - assertEndsWith("𐐅", "𐐅", "UTF8_BINARY", true); - assertEndsWith("𐐅", "𐐅", "UTF8_LCASE", true); - assertEndsWith("𐐅", "𐐅", "UNICODE", true); - assertEndsWith("𐐅", "𐐅", "UNICODE_CI", true); - assertEndsWith("𐐅", "𐐭", "UTF8_BINARY", false); - assertEndsWith("𐐅", "𐐭", "UTF8_LCASE", true); - assertEndsWith("𐐅", "𐐭", "UNICODE", false); - assertEndsWith("𐐅", "𐐭", "UNICODE_CI", true); - assertEndsWith("𝔸", "𝔸", "UTF8_BINARY", true); - assertEndsWith("𝔸", "𝔸", "UTF8_LCASE", true); - assertEndsWith("𝔸", "𝔸", "UNICODE", true); - assertEndsWith("𝔸", "𝔸", "UNICODE_CI", true); + assertEndsWith("a🙃b🙃c", "x", UTF8_BINARY, false); + assertEndsWith("a🙃b🙃c", "x", UTF8_LCASE, false); + assertEndsWith("a🙃b🙃c", "x", UNICODE, false); + assertEndsWith("a🙃b🙃c", "x", UNICODE_CI, false); + assertEndsWith("a🙃b🙃c", "b", UTF8_BINARY, false); + assertEndsWith("a🙃b🙃c", "b", UTF8_LCASE, false); + assertEndsWith("a🙃b🙃c", "b", UNICODE, false); + assertEndsWith("a🙃b🙃c", "b", UNICODE_CI, false); + assertEndsWith("a🙃b🙃c", "a🙃b", UTF8_BINARY, false); + assertEndsWith("a🙃b🙃c", "a🙃b", UTF8_LCASE, false); + assertEndsWith("a🙃b🙃c", "a🙃b", UNICODE, false); + assertEndsWith("a🙃b🙃c", "a🙃b", UNICODE_CI, false); + assertEndsWith("a🙃b🙃c", "b🙃c", UTF8_BINARY, true); + assertEndsWith("a🙃b🙃c", "b🙃c", UTF8_LCASE, true); + assertEndsWith("a🙃b🙃c", "b🙃c", UNICODE, true); + assertEndsWith("a🙃b🙃c", "b🙃c", UNICODE_CI, true); + assertEndsWith("a🙃b🙃c", "a🙃b🙃c", UTF8_BINARY, true); + assertEndsWith("a🙃b🙃c", "a🙃b🙃c", UTF8_LCASE, true); + assertEndsWith("a🙃b🙃c", "a🙃b🙃c", UNICODE, true); + assertEndsWith("a🙃b🙃c", "a🙃b🙃c", UNICODE_CI, true); + assertEndsWith("😀😆😃😄", "😄😆", UTF8_BINARY, false); + assertEndsWith("😀😆😃😄", "😄😆", UTF8_LCASE, false); + assertEndsWith("😀😆😃😄", "😄😆", UNICODE, false); + assertEndsWith("😀😆😃😄", "😄😆", UNICODE_CI, false); + assertEndsWith("😀😆😃😄", "😆😃", UTF8_BINARY, false); + assertEndsWith("😀😆😃😄", "😆😃", UTF8_LCASE, false); + assertEndsWith("😀😆😃😄", "😆😃", UNICODE, false); + assertEndsWith("😀😆😃😄", "😆😃", UNICODE_CI, false); + assertEndsWith("😀😆😃😄", "😀😆", UTF8_BINARY, false); + assertEndsWith("😀😆😃😄", "😀😆", UTF8_LCASE, false); + assertEndsWith("😀😆😃😄", "😀😆", UNICODE, false); + assertEndsWith("😀😆😃😄", "😀😆", UNICODE_CI, false); + assertEndsWith("😀😆😃😄", "😃😄", UTF8_BINARY, true); + assertEndsWith("😀😆😃😄", "😃😄", UTF8_LCASE, true); + assertEndsWith("😀😆😃😄", "😃😄", UNICODE, true); + assertEndsWith("😀😆😃😄", "😃😄", UNICODE_CI, true); + assertEndsWith("😀😆😃😄", "😀😆😃😄", UTF8_BINARY, true); + assertEndsWith("😀😆😃😄", "😀😆😃😄", UTF8_LCASE, true); + assertEndsWith("😀😆😃😄", "😀😆😃😄", UNICODE, true); + assertEndsWith("😀😆😃😄", "😀😆😃😄", UNICODE_CI, true); + assertEndsWith("𐐅", "𐐅", UTF8_BINARY, true); + assertEndsWith("𐐅", "𐐅", UTF8_LCASE, true); + assertEndsWith("𐐅", "𐐅", UNICODE, true); + assertEndsWith("𐐅", "𐐅", UNICODE_CI, true); + assertEndsWith("𐐅", "𐐭", UTF8_BINARY, false); + assertEndsWith("𐐅", "𐐭", UTF8_LCASE, true); + assertEndsWith("𐐅", "𐐭", UNICODE, false); + assertEndsWith("𐐅", "𐐭", UNICODE_CI, true); + assertEndsWith("𝔸", "𝔸", UTF8_BINARY, true); + assertEndsWith("𝔸", "𝔸", UTF8_LCASE, true); + assertEndsWith("𝔸", "𝔸", UNICODE, true); + assertEndsWith("𝔸", "𝔸", UNICODE_CI, true); } /** @@ -1057,158 +1058,158 @@ public void testStringSplitSQL() throws SparkException { var array_AOB = new UTF8String[] { UTF8String.fromString("A𐐅B") }; var array_AoB = new UTF8String[] { UTF8String.fromString("A𐐭B") }; // Empty strings. - assertStringSplitSQL("", "", "UTF8_BINARY", empty_match); - assertStringSplitSQL("abc", "", "UTF8_BINARY", array_abc); - assertStringSplitSQL("", "abc", "UTF8_BINARY", empty_match); - assertStringSplitSQL("", "", "UNICODE", empty_match); - assertStringSplitSQL("abc", "", "UNICODE", array_abc); - assertStringSplitSQL("", "abc", "UNICODE", 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); + assertStringSplitSQL("", "", UTF8_BINARY, empty_match); + assertStringSplitSQL("abc", "", UTF8_BINARY, array_abc); + assertStringSplitSQL("", "abc", UTF8_BINARY, empty_match); + assertStringSplitSQL("", "", UNICODE, empty_match); + assertStringSplitSQL("abc", "", UNICODE, array_abc); + assertStringSplitSQL("", "abc", UNICODE, 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); // Basic tests. - assertStringSplitSQL("1a2", "a", "UTF8_BINARY", array_1_2); - assertStringSplitSQL("1a2", "A", "UTF8_BINARY", array_1a2); - assertStringSplitSQL("1a2", "b", "UTF8_BINARY", array_1a2); - assertStringSplitSQL("1a2", "1a2", "UNICODE", full_match); - assertStringSplitSQL("1a2", "1A2", "UNICODE", array_1a2); - assertStringSplitSQL("1a2", "3b4", "UNICODE", 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); - assertStringSplitSQL("1a2", "123", "UNICODE_CI", array_1a2); + assertStringSplitSQL("1a2", "a", UTF8_BINARY, array_1_2); + assertStringSplitSQL("1a2", "A", UTF8_BINARY, array_1a2); + assertStringSplitSQL("1a2", "b", UTF8_BINARY, array_1a2); + assertStringSplitSQL("1a2", "1a2", UNICODE, full_match); + assertStringSplitSQL("1a2", "1A2", UNICODE, array_1a2); + assertStringSplitSQL("1a2", "3b4", UNICODE, 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); + assertStringSplitSQL("1a2", "123", UNICODE_CI, array_1a2); // Advanced tests. - assertStringSplitSQL("äb世De", "b世D", "UTF8_BINARY", array_a_e); - assertStringSplitSQL("äb世De", "B世d", "UTF8_BINARY", array_special); - assertStringSplitSQL("äbćδe", "bćδ", "UTF8_BINARY", array_a_e); - assertStringSplitSQL("äbćδe", "BcΔ", "UTF8_BINARY", array_abcde); - assertStringSplitSQL("äb世De", "äb世De", "UNICODE", full_match); - 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_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); - assertStringSplitSQL("äbćδe", "ÄBcΔÉ", "UNICODE_CI", array_abcde); + assertStringSplitSQL("äb世De", "b世D", UTF8_BINARY, array_a_e); + assertStringSplitSQL("äb世De", "B世d", UTF8_BINARY, array_special); + assertStringSplitSQL("äbćδe", "bćδ", UTF8_BINARY, array_a_e); + assertStringSplitSQL("äbćδe", "BcΔ", UTF8_BINARY, array_abcde); + assertStringSplitSQL("äb世De", "äb世De", UNICODE, full_match); + 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_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); + assertStringSplitSQL("äbćδe", "ÄBcΔÉ", UNICODE_CI, array_abcde); // Case variation. - assertStringSplitSQL("AaXbB", "x", "UTF8_BINARY", array_AaXbB); - 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_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); + assertStringSplitSQL("AaXbB", "x", UTF8_BINARY, array_AaXbB); + 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_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. - assertStringSplitSQL("aBcDe", "bćd", "UTF8_BINARY", array_aBcDe); - 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_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); + assertStringSplitSQL("aBcDe", "bćd", UTF8_BINARY, array_aBcDe); + 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_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); // One-to-many case mapping (e.g. Turkish dotted I). - assertStringSplitSQL("İ", "i", "UTF8_BINARY", array_Turkish_uppercase_dotted_I); - assertStringSplitSQL("İ", "i", "UTF8_LCASE", array_Turkish_uppercase_dotted_I); - assertStringSplitSQL("İ", "i", "UNICODE", array_Turkish_uppercase_dotted_I); - assertStringSplitSQL("İ", "i", "UNICODE_CI", array_Turkish_uppercase_dotted_I); - assertStringSplitSQL("İ", "\u0307", "UTF8_BINARY", array_Turkish_uppercase_dotted_I); - assertStringSplitSQL("İ", "\u0307", "UTF8_LCASE", array_Turkish_uppercase_dotted_I); - assertStringSplitSQL("İ", "\u0307", "UNICODE", array_Turkish_uppercase_dotted_I); - assertStringSplitSQL("İ", "\u0307", "UNICODE_CI", array_Turkish_uppercase_dotted_I); - assertStringSplitSQL("i\u0307", "i", "UTF8_BINARY", array_dot); - assertStringSplitSQL("i\u0307", "i", "UTF8_LCASE", array_dot); - assertStringSplitSQL("i\u0307", "i", "UNICODE", array_Turkish_lowercase_dotted_i); - assertStringSplitSQL("i\u0307", "i", "UNICODE_CI", array_Turkish_lowercase_dotted_i); - assertStringSplitSQL("i\u0307", "\u0307", "UTF8_BINARY", array_i); - assertStringSplitSQL("i\u0307", "\u0307", "UTF8_LCASE", array_i); - assertStringSplitSQL("i\u0307", "\u0307", "UNICODE", array_Turkish_lowercase_dotted_i); - assertStringSplitSQL("i\u0307", "\u0307", "UNICODE_CI", array_Turkish_lowercase_dotted_i); - assertStringSplitSQL("AİB", "İ", "UTF8_BINARY", array_A_B); - assertStringSplitSQL("AİB", "İ", "UTF8_LCASE", array_A_B); - assertStringSplitSQL("AİB", "İ", "UNICODE", array_A_B); - assertStringSplitSQL("AİB", "İ", "UNICODE_CI", array_A_B); - assertStringSplitSQL("AİB", "i\u0307", "UTF8_BINARY", array_AIB); - assertStringSplitSQL("AİB", "i\u0307", "UTF8_LCASE", array_A_B); - assertStringSplitSQL("AİB", "i\u0307", "UNICODE", array_AIB); - assertStringSplitSQL("AİB", "i\u0307", "UNICODE_CI", array_A_B); - assertStringSplitSQL("Ai\u0307B", "İ", "UTF8_BINARY", array_AiB); - assertStringSplitSQL("Ai\u0307B", "İ", "UTF8_LCASE", array_A_B); - assertStringSplitSQL("Ai\u0307B", "İ", "UNICODE", array_AiB); - assertStringSplitSQL("Ai\u0307B", "İ", "UNICODE_CI", array_A_B); - assertStringSplitSQL("Ai\u0307B", "i\u0307", "UTF8_BINARY", array_A_B); - assertStringSplitSQL("Ai\u0307B", "i\u0307", "UTF8_LCASE", array_A_B); - assertStringSplitSQL("Ai\u0307B", "i\u0307", "UNICODE", array_A_B); - assertStringSplitSQL("Ai\u0307B", "i\u0307", "UNICODE_CI", array_A_B); + assertStringSplitSQL("İ", "i", UTF8_BINARY, array_Turkish_uppercase_dotted_I); + assertStringSplitSQL("İ", "i", UTF8_LCASE, array_Turkish_uppercase_dotted_I); + assertStringSplitSQL("İ", "i", UNICODE, array_Turkish_uppercase_dotted_I); + assertStringSplitSQL("İ", "i", UNICODE_CI, array_Turkish_uppercase_dotted_I); + assertStringSplitSQL("İ", "\u0307", UTF8_BINARY, array_Turkish_uppercase_dotted_I); + assertStringSplitSQL("İ", "\u0307", UTF8_LCASE, array_Turkish_uppercase_dotted_I); + assertStringSplitSQL("İ", "\u0307", UNICODE, array_Turkish_uppercase_dotted_I); + assertStringSplitSQL("İ", "\u0307", UNICODE_CI, array_Turkish_uppercase_dotted_I); + assertStringSplitSQL("i\u0307", "i", UTF8_BINARY, array_dot); + assertStringSplitSQL("i\u0307", "i", UTF8_LCASE, array_dot); + assertStringSplitSQL("i\u0307", "i", UNICODE, array_Turkish_lowercase_dotted_i); + assertStringSplitSQL("i\u0307", "i", UNICODE_CI, array_Turkish_lowercase_dotted_i); + assertStringSplitSQL("i\u0307", "\u0307", UTF8_BINARY, array_i); + assertStringSplitSQL("i\u0307", "\u0307", UTF8_LCASE, array_i); + assertStringSplitSQL("i\u0307", "\u0307", UNICODE, array_Turkish_lowercase_dotted_i); + assertStringSplitSQL("i\u0307", "\u0307", UNICODE_CI, array_Turkish_lowercase_dotted_i); + assertStringSplitSQL("AİB", "İ", UTF8_BINARY, array_A_B); + assertStringSplitSQL("AİB", "İ", UTF8_LCASE, array_A_B); + assertStringSplitSQL("AİB", "İ", UNICODE, array_A_B); + assertStringSplitSQL("AİB", "İ", UNICODE_CI, array_A_B); + assertStringSplitSQL("AİB", "i\u0307", UTF8_BINARY, array_AIB); + assertStringSplitSQL("AİB", "i\u0307", UTF8_LCASE, array_A_B); + assertStringSplitSQL("AİB", "i\u0307", UNICODE, array_AIB); + assertStringSplitSQL("AİB", "i\u0307", UNICODE_CI, array_A_B); + assertStringSplitSQL("Ai\u0307B", "İ", UTF8_BINARY, array_AiB); + assertStringSplitSQL("Ai\u0307B", "İ", UTF8_LCASE, array_A_B); + assertStringSplitSQL("Ai\u0307B", "İ", UNICODE, array_AiB); + assertStringSplitSQL("Ai\u0307B", "İ", UNICODE_CI, array_A_B); + assertStringSplitSQL("Ai\u0307B", "i\u0307", UTF8_BINARY, array_A_B); + assertStringSplitSQL("Ai\u0307B", "i\u0307", UTF8_LCASE, array_A_B); + assertStringSplitSQL("Ai\u0307B", "i\u0307", UNICODE, array_A_B); + assertStringSplitSQL("Ai\u0307B", "i\u0307", UNICODE_CI, array_A_B); // Conditional case mapping (e.g. Greek sigmas). - assertStringSplitSQL("σ", "σ", "UTF8_BINARY", full_match); - assertStringSplitSQL("σ", "σ", "UTF8_LCASE", full_match); - assertStringSplitSQL("σ", "σ", "UNICODE", full_match); - assertStringSplitSQL("σ", "σ", "UNICODE_CI", full_match); - assertStringSplitSQL("σ", "ς", "UTF8_BINARY", array_small_nonfinal_sigma); - assertStringSplitSQL("σ", "ς", "UTF8_LCASE", full_match); - assertStringSplitSQL("σ", "ς", "UNICODE", array_small_nonfinal_sigma); - assertStringSplitSQL("σ", "ς", "UNICODE_CI", full_match); - assertStringSplitSQL("σ", "Σ", "UTF8_BINARY", array_small_nonfinal_sigma); - assertStringSplitSQL("σ", "Σ", "UTF8_LCASE", full_match); - assertStringSplitSQL("σ", "Σ", "UNICODE", array_small_nonfinal_sigma); - assertStringSplitSQL("σ", "Σ", "UNICODE_CI", full_match); - assertStringSplitSQL("ς", "σ", "UTF8_BINARY", array_small_final_sigma); - assertStringSplitSQL("ς", "σ", "UTF8_LCASE", full_match); - assertStringSplitSQL("ς", "σ", "UNICODE", array_small_final_sigma); - assertStringSplitSQL("ς", "σ", "UNICODE_CI", full_match); - assertStringSplitSQL("ς", "ς", "UTF8_BINARY", full_match); - assertStringSplitSQL("ς", "ς", "UTF8_LCASE", full_match); - assertStringSplitSQL("ς", "ς", "UNICODE", full_match); - assertStringSplitSQL("ς", "ς", "UNICODE_CI", full_match); - assertStringSplitSQL("ς", "Σ", "UTF8_BINARY", array_small_final_sigma); - assertStringSplitSQL("ς", "Σ", "UTF8_LCASE", full_match); - assertStringSplitSQL("ς", "Σ", "UNICODE", array_small_final_sigma); - assertStringSplitSQL("ς", "Σ", "UNICODE_CI", full_match); - assertStringSplitSQL("Σ", "σ", "UTF8_BINARY", array_capital_sigma); - assertStringSplitSQL("Σ", "σ", "UTF8_LCASE", full_match); - assertStringSplitSQL("Σ", "σ", "UNICODE", array_capital_sigma); - assertStringSplitSQL("Σ", "σ", "UNICODE_CI", full_match); - assertStringSplitSQL("Σ", "ς", "UTF8_BINARY", array_capital_sigma); - assertStringSplitSQL("Σ", "ς", "UTF8_LCASE", full_match); - assertStringSplitSQL("Σ", "ς", "UNICODE", array_capital_sigma); - assertStringSplitSQL("Σ", "ς", "UNICODE_CI", full_match); - assertStringSplitSQL("Σ", "Σ", "UTF8_BINARY", full_match); - assertStringSplitSQL("Σ", "Σ", "UTF8_LCASE", full_match); - assertStringSplitSQL("Σ", "Σ", "UNICODE", full_match); - assertStringSplitSQL("Σ", "Σ", "UNICODE_CI", full_match); + assertStringSplitSQL("σ", "σ", UTF8_BINARY, full_match); + assertStringSplitSQL("σ", "σ", UTF8_LCASE, full_match); + assertStringSplitSQL("σ", "σ", UNICODE, full_match); + assertStringSplitSQL("σ", "σ", UNICODE_CI, full_match); + assertStringSplitSQL("σ", "ς", UTF8_BINARY, array_small_nonfinal_sigma); + assertStringSplitSQL("σ", "ς", UTF8_LCASE, full_match); + assertStringSplitSQL("σ", "ς", UNICODE, array_small_nonfinal_sigma); + assertStringSplitSQL("σ", "ς", UNICODE_CI, full_match); + assertStringSplitSQL("σ", "Σ", UTF8_BINARY, array_small_nonfinal_sigma); + assertStringSplitSQL("σ", "Σ", UTF8_LCASE, full_match); + assertStringSplitSQL("σ", "Σ", UNICODE, array_small_nonfinal_sigma); + assertStringSplitSQL("σ", "Σ", UNICODE_CI, full_match); + assertStringSplitSQL("ς", "σ", UTF8_BINARY, array_small_final_sigma); + assertStringSplitSQL("ς", "σ", UTF8_LCASE, full_match); + assertStringSplitSQL("ς", "σ", UNICODE, array_small_final_sigma); + assertStringSplitSQL("ς", "σ", UNICODE_CI, full_match); + assertStringSplitSQL("ς", "ς", UTF8_BINARY, full_match); + assertStringSplitSQL("ς", "ς", UTF8_LCASE, full_match); + assertStringSplitSQL("ς", "ς", UNICODE, full_match); + assertStringSplitSQL("ς", "ς", UNICODE_CI, full_match); + assertStringSplitSQL("ς", "Σ", UTF8_BINARY, array_small_final_sigma); + assertStringSplitSQL("ς", "Σ", UTF8_LCASE, full_match); + assertStringSplitSQL("ς", "Σ", UNICODE, array_small_final_sigma); + assertStringSplitSQL("ς", "Σ", UNICODE_CI, full_match); + assertStringSplitSQL("Σ", "σ", UTF8_BINARY, array_capital_sigma); + assertStringSplitSQL("Σ", "σ", UTF8_LCASE, full_match); + assertStringSplitSQL("Σ", "σ", UNICODE, array_capital_sigma); + assertStringSplitSQL("Σ", "σ", UNICODE_CI, full_match); + assertStringSplitSQL("Σ", "ς", UTF8_BINARY, array_capital_sigma); + assertStringSplitSQL("Σ", "ς", UTF8_LCASE, full_match); + assertStringSplitSQL("Σ", "ς", UNICODE, array_capital_sigma); + assertStringSplitSQL("Σ", "ς", UNICODE_CI, full_match); + assertStringSplitSQL("Σ", "Σ", UTF8_BINARY, full_match); + assertStringSplitSQL("Σ", "Σ", UTF8_LCASE, full_match); + assertStringSplitSQL("Σ", "Σ", UNICODE, full_match); + assertStringSplitSQL("Σ", "Σ", UNICODE_CI, full_match); // Surrogate pairs. - assertStringSplitSQL("a🙃b🙃c", "🙃", "UTF8_BINARY", array_a_b_c); - assertStringSplitSQL("a🙃b🙃c", "🙃", "UTF8_LCASE", array_a_b_c); - assertStringSplitSQL("a🙃b🙃c", "🙃", "UNICODE", array_a_b_c); - assertStringSplitSQL("a🙃b🙃c", "🙃", "UNICODE_CI", array_a_b_c); - assertStringSplitSQL("😀😆😃😄", "😆😃", "UTF8_BINARY", array_emojis); - assertStringSplitSQL("😀😆😃😄", "😆😃", "UTF8_LCASE", array_emojis); - assertStringSplitSQL("😀😆😃😄", "😆😃", "UNICODE", array_emojis); - assertStringSplitSQL("😀😆😃😄", "😆😃", "UNICODE_CI", array_emojis); - assertStringSplitSQL("A𐐅B", "𐐅", "UTF8_BINARY", array_A_B); - assertStringSplitSQL("A𐐅B", "𐐅", "UTF8_LCASE", array_A_B); - assertStringSplitSQL("A𐐅B", "𐐅", "UNICODE", array_A_B); - assertStringSplitSQL("A𐐅B", "𐐅", "UNICODE_CI", array_A_B); - assertStringSplitSQL("A𐐅B", "𐐭", "UTF8_BINARY", array_AOB); - assertStringSplitSQL("A𐐅B", "𐐭", "UTF8_LCASE", array_A_B); - assertStringSplitSQL("A𐐅B", "𐐭", "UNICODE", array_AOB); - assertStringSplitSQL("A𐐅B", "𐐭", "UNICODE_CI", array_A_B); - assertStringSplitSQL("A𐐭B", "𐐅", "UTF8_BINARY", array_AoB); - assertStringSplitSQL("A𐐭B", "𐐅", "UTF8_LCASE", array_A_B); - assertStringSplitSQL("A𐐭B", "𐐅", "UNICODE", array_AoB); - assertStringSplitSQL("A𐐭B", "𐐅", "UNICODE_CI", array_A_B); + assertStringSplitSQL("a🙃b🙃c", "🙃", UTF8_BINARY, array_a_b_c); + assertStringSplitSQL("a🙃b🙃c", "🙃", UTF8_LCASE, array_a_b_c); + assertStringSplitSQL("a🙃b🙃c", "🙃", UNICODE, array_a_b_c); + assertStringSplitSQL("a🙃b🙃c", "🙃", UNICODE_CI, array_a_b_c); + assertStringSplitSQL("😀😆😃😄", "😆😃", UTF8_BINARY, array_emojis); + assertStringSplitSQL("😀😆😃😄", "😆😃", UTF8_LCASE, array_emojis); + assertStringSplitSQL("😀😆😃😄", "😆😃", UNICODE, array_emojis); + assertStringSplitSQL("😀😆😃😄", "😆😃", UNICODE_CI, array_emojis); + assertStringSplitSQL("A𐐅B", "𐐅", UTF8_BINARY, array_A_B); + assertStringSplitSQL("A𐐅B", "𐐅", UTF8_LCASE, array_A_B); + assertStringSplitSQL("A𐐅B", "𐐅", UNICODE, array_A_B); + assertStringSplitSQL("A𐐅B", "𐐅", UNICODE_CI, array_A_B); + assertStringSplitSQL("A𐐅B", "𐐭", UTF8_BINARY, array_AOB); + assertStringSplitSQL("A𐐅B", "𐐭", UTF8_LCASE, array_A_B); + assertStringSplitSQL("A𐐅B", "𐐭", UNICODE, array_AOB); + assertStringSplitSQL("A𐐅B", "𐐭", UNICODE_CI, array_A_B); + assertStringSplitSQL("A𐐭B", "𐐅", UTF8_BINARY, array_AoB); + assertStringSplitSQL("A𐐭B", "𐐅", UTF8_LCASE, array_A_B); + assertStringSplitSQL("A𐐭B", "𐐅", UNICODE, array_AoB); + assertStringSplitSQL("A𐐭B", "𐐅", UNICODE_CI, array_A_B); } /** @@ -1391,156 +1392,156 @@ public void testInitCap() throws SparkException { assertInitCap("θαλασσινος", collationName, "Θαλασσινος"); } // Advanced tests. - assertInitCap("aBćDe", "UTF8_BINARY", "Abćde"); - assertInitCap("aBćDe", "UTF8_LCASE", "Abćde"); - assertInitCap("aBćDe", "UNICODE", "Abćde"); - assertInitCap("aBćDe", "UNICODE_CI", "Abćde"); - assertInitCap("ab世De", "UTF8_BINARY", "Ab世de"); - assertInitCap("ab世De", "UTF8_LCASE", "Ab世De"); - assertInitCap("ab世De", "UNICODE", "Ab世De"); - assertInitCap("ab世De", "UNICODE_CI", "Ab世De"); - assertInitCap("äbćδe", "UTF8_BINARY", "Äbćδe"); - assertInitCap("äbćδe", "UTF8_LCASE", "Äbćδe"); - assertInitCap("äbćδe", "UNICODE", "Äbćδe"); - assertInitCap("äbćδe", "UNICODE_CI", "Äbćδe"); - assertInitCap("ÄBĆΔE", "UTF8_BINARY", "Äbćδe"); - assertInitCap("ÄBĆΔE", "UTF8_LCASE", "Äbćδe"); - assertInitCap("ÄBĆΔE", "UNICODE", "Äbćδe"); - assertInitCap("ÄBĆΔE", "UNICODE_CI", "Äbćδe"); + assertInitCap("aBćDe", UTF8_BINARY, "Abćde"); + assertInitCap("aBćDe", UTF8_LCASE, "Abćde"); + assertInitCap("aBćDe", UNICODE, "Abćde"); + assertInitCap("aBćDe", UNICODE_CI, "Abćde"); + assertInitCap("ab世De", UTF8_BINARY, "Ab世de"); + assertInitCap("ab世De", UTF8_LCASE, "Ab世De"); + assertInitCap("ab世De", UNICODE, "Ab世De"); + assertInitCap("ab世De", UNICODE_CI, "Ab世De"); + assertInitCap("äbćδe", UTF8_BINARY, "Äbćδe"); + assertInitCap("äbćδe", UTF8_LCASE, "Äbćδe"); + assertInitCap("äbćδe", UNICODE, "Äbćδe"); + assertInitCap("äbćδe", UNICODE_CI, "Äbćδe"); + assertInitCap("ÄBĆΔE", UTF8_BINARY, "Äbćδe"); + assertInitCap("ÄBĆΔE", UTF8_LCASE, "Äbćδe"); + assertInitCap("ÄBĆΔE", UNICODE, "Äbćδe"); + assertInitCap("ÄBĆΔE", UNICODE_CI, "Äbćδe"); assertInitCap("êéfgh", "AF_CI_AI", "Êéfgh"); assertInitCap("öoAÄ", "DE_CI_AI", "Öoaä"); // Case-variable character length - assertInitCap("İo", "UTF8_BINARY", "İo", "I\u0307o"); - 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_LCASE", "I\u0307o"); - assertInitCap("i\u0307o", "UNICODE", "I\u0307o"); - assertInitCap("i\u0307o", "UNICODE_CI", "I\u0307o"); + assertInitCap("İo", UTF8_BINARY, "İo", "I\u0307o"); + 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_LCASE, "I\u0307o"); + assertInitCap("i\u0307o", UNICODE, "I\u0307o"); + assertInitCap("i\u0307o", UNICODE_CI, "I\u0307o"); // Different possible word boundaries - assertInitCap("aB 世 de", "UTF8_BINARY", "Ab 世 De"); - assertInitCap("aB 世 de", "UTF8_LCASE", "Ab 世 De"); - assertInitCap("aB 世 de", "UNICODE", "Ab 世 De"); - assertInitCap("aB 世 de", "UNICODE_CI", "Ab 世 De"); + assertInitCap("aB 世 de", UTF8_BINARY, "Ab 世 De"); + assertInitCap("aB 世 de", UTF8_LCASE, "Ab 世 De"); + assertInitCap("aB 世 de", UNICODE, "Ab 世 De"); + assertInitCap("aB 世 de", UNICODE_CI, "Ab 世 De"); // One-to-many case mapping (e.g. Turkish dotted I). - assertInitCap("İ", "UTF8_BINARY", "İ", "I\u0307"); - assertInitCap("İ", "UTF8_LCASE", "İ"); - assertInitCap("İ", "UNICODE", "İ"); - assertInitCap("İ", "UNICODE_CI", "İ"); - assertInitCap("I\u0307", "UTF8_BINARY","I\u0307"); - assertInitCap("I\u0307", "UTF8_LCASE","I\u0307"); - assertInitCap("I\u0307", "UNICODE","I\u0307"); - assertInitCap("I\u0307", "UNICODE_CI","I\u0307"); - assertInitCap("İonic", "UTF8_BINARY", "İonic", "I\u0307onic"); - assertInitCap("İonic", "UTF8_LCASE", "İonic"); - assertInitCap("İonic", "UNICODE", "İonic"); - assertInitCap("İonic", "UNICODE_CI", "İonic"); - assertInitCap("i\u0307onic", "UTF8_BINARY","I\u0307onic"); - assertInitCap("i\u0307onic", "UTF8_LCASE","I\u0307onic"); - assertInitCap("i\u0307onic", "UNICODE","I\u0307onic"); - assertInitCap("i\u0307onic", "UNICODE_CI","I\u0307onic"); - assertInitCap("FIDELİO", "UTF8_BINARY", "Fideli\u0307o"); - assertInitCap("FIDELİO", "UTF8_LCASE", "Fideli\u0307o"); - assertInitCap("FIDELİO", "UNICODE", "Fideli\u0307o"); - assertInitCap("FIDELİO", "UNICODE_CI", "Fideli\u0307o"); + assertInitCap("İ", UTF8_BINARY, "İ", "I\u0307"); + assertInitCap("İ", UTF8_LCASE, "İ"); + assertInitCap("İ", UNICODE, "İ"); + assertInitCap("İ", UNICODE_CI, "İ"); + assertInitCap("I\u0307", UTF8_BINARY,"I\u0307"); + assertInitCap("I\u0307", UTF8_LCASE,"I\u0307"); + assertInitCap("I\u0307", UNICODE,"I\u0307"); + assertInitCap("I\u0307", UNICODE_CI,"I\u0307"); + assertInitCap("İonic", UTF8_BINARY, "İonic", "I\u0307onic"); + assertInitCap("İonic", UTF8_LCASE, "İonic"); + assertInitCap("İonic", UNICODE, "İonic"); + assertInitCap("İonic", UNICODE_CI, "İonic"); + assertInitCap("i\u0307onic", UTF8_BINARY,"I\u0307onic"); + assertInitCap("i\u0307onic", UTF8_LCASE,"I\u0307onic"); + assertInitCap("i\u0307onic", UNICODE,"I\u0307onic"); + assertInitCap("i\u0307onic", UNICODE_CI,"I\u0307onic"); + assertInitCap("FIDELİO", UTF8_BINARY, "Fideli\u0307o"); + assertInitCap("FIDELİO", UTF8_LCASE, "Fideli\u0307o"); + assertInitCap("FIDELİO", UNICODE, "Fideli\u0307o"); + assertInitCap("FIDELİO", UNICODE_CI, "Fideli\u0307o"); // Surrogate pairs. - assertInitCap("a🙃B🙃c", "UTF8_BINARY", "A🙃b🙃c"); - assertInitCap("a🙃B🙃c", "UTF8_LCASE", "A🙃B🙃C"); - assertInitCap("a🙃B🙃c", "UNICODE", "A🙃B🙃C"); - assertInitCap("a🙃B🙃c", "UNICODE_CI", "A🙃B🙃C"); - assertInitCap("😄 😆", "UTF8_BINARY", "😄 😆"); - assertInitCap("😄 😆", "UTF8_LCASE", "😄 😆"); - assertInitCap("😄 😆", "UNICODE", "😄 😆"); - assertInitCap("😄 😆", "UNICODE_CI", "😄 😆"); - assertInitCap("😀😆😃😄", "UTF8_BINARY", "😀😆😃😄"); - assertInitCap("😀😆😃😄", "UTF8_LCASE", "😀😆😃😄"); - assertInitCap("😀😆😃😄", "UNICODE", "😀😆😃😄"); - assertInitCap("😀😆😃😄", "UNICODE_CI", "😀😆😃😄"); - assertInitCap("𝔸", "UTF8_BINARY", "𝔸"); - assertInitCap("𝔸", "UTF8_LCASE", "𝔸"); - assertInitCap("𝔸", "UNICODE", "𝔸"); - assertInitCap("𝔸", "UNICODE_CI", "𝔸"); - assertInitCap("𐐅", "UTF8_BINARY", "\uD801\uDC05", "𐐭"); - assertInitCap("𐐅", "UTF8_LCASE", "𐐅"); - assertInitCap("𐐅", "UNICODE", "𐐅"); - assertInitCap("𐐅", "UNICODE_CI", "𐐅"); - assertInitCap("𐐭", "UTF8_BINARY", "\uD801\uDC05", "𐐭"); - assertInitCap("𐐭", "UTF8_LCASE", "𐐅"); - assertInitCap("𐐭", "UNICODE", "𐐅"); - assertInitCap("𐐭", "UNICODE_CI", "𐐅"); - assertInitCap("𐐭𝔸", "UTF8_BINARY", "\uD801\uDC05\uD835\uDD38", "𐐭𝔸"); - assertInitCap("𐐭𝔸", "UTF8_LCASE", "𐐅𝔸"); - assertInitCap("𐐭𝔸", "UNICODE", "𐐅𝔸"); - assertInitCap("𐐭𝔸", "UNICODE_CI", "𐐅𝔸"); + assertInitCap("a🙃B🙃c", UTF8_BINARY, "A🙃b🙃c"); + assertInitCap("a🙃B🙃c", UTF8_LCASE, "A🙃B🙃C"); + assertInitCap("a🙃B🙃c", UNICODE, "A🙃B🙃C"); + assertInitCap("a🙃B🙃c", UNICODE_CI, "A🙃B🙃C"); + assertInitCap("😄 😆", UTF8_BINARY, "😄 😆"); + assertInitCap("😄 😆", UTF8_LCASE, "😄 😆"); + assertInitCap("😄 😆", UNICODE, "😄 😆"); + assertInitCap("😄 😆", UNICODE_CI, "😄 😆"); + assertInitCap("😀😆😃😄", UTF8_BINARY, "😀😆😃😄"); + assertInitCap("😀😆😃😄", UTF8_LCASE, "😀😆😃😄"); + assertInitCap("😀😆😃😄", UNICODE, "😀😆😃😄"); + assertInitCap("😀😆😃😄", UNICODE_CI, "😀😆😃😄"); + assertInitCap("𝔸", UTF8_BINARY, "𝔸"); + assertInitCap("𝔸", UTF8_LCASE, "𝔸"); + assertInitCap("𝔸", UNICODE, "𝔸"); + assertInitCap("𝔸", UNICODE_CI, "𝔸"); + assertInitCap("𐐅", UTF8_BINARY, "\uD801\uDC05", "𐐭"); + assertInitCap("𐐅", UTF8_LCASE, "𐐅"); + assertInitCap("𐐅", UNICODE, "𐐅"); + assertInitCap("𐐅", UNICODE_CI, "𐐅"); + assertInitCap("𐐭", UTF8_BINARY, "\uD801\uDC05", "𐐭"); + assertInitCap("𐐭", UTF8_LCASE, "𐐅"); + assertInitCap("𐐭", UNICODE, "𐐅"); + assertInitCap("𐐭", UNICODE_CI, "𐐅"); + assertInitCap("𐐭𝔸", UTF8_BINARY, "\uD801\uDC05\uD835\uDD38", "𐐭𝔸"); + assertInitCap("𐐭𝔸", UTF8_LCASE, "𐐅𝔸"); + assertInitCap("𐐭𝔸", UNICODE, "𐐅𝔸"); + assertInitCap("𐐭𝔸", UNICODE_CI, "𐐅𝔸"); // Ligatures. - assertInitCap("ß fi ffi ff st ῗ", "UTF8_BINARY", "Ss Fi Ffi Ff St Ϊ͂", "ß fi ffi ff st ῗ"); - assertInitCap("ß fi ffi ff st ῗ", "UTF8_LCASE", "Ss Fi Ffi Ff St \u0399\u0308\u0342"); - assertInitCap("ß fi ffi ff st ῗ", "UNICODE", "Ss Fi Ffi Ff St \u0399\u0308\u0342"); - assertInitCap("ß fi ffi ff st ῗ", "UNICODE", "Ss Fi Ffi Ff St \u0399\u0308\u0342"); - assertInitCap("œ ǽ", "UTF8_BINARY", "Œ Ǽ", "Œ Ǽ"); + assertInitCap("ß fi ffi ff st ῗ", UTF8_BINARY, "Ss Fi Ffi Ff St Ϊ͂", "ß fi ffi ff st ῗ"); + assertInitCap("ß fi ffi ff st ῗ", UTF8_LCASE, "Ss Fi Ffi Ff St \u0399\u0308\u0342"); + assertInitCap("ß fi ffi ff st ῗ", UNICODE, "Ss Fi Ffi Ff St \u0399\u0308\u0342"); + assertInitCap("ß fi ffi ff st ῗ", UNICODE, "Ss Fi Ffi Ff St \u0399\u0308\u0342"); + assertInitCap("œ ǽ", UTF8_BINARY, "Œ Ǽ", "Œ Ǽ"); // 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_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_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_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_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_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_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_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_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_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_LCASE", "Džaba Ljubav Njegova"); - assertInitCap("džaba Ljubav NJegova", "UNICODE_CI", "Džaba Ljubav Njegova"); - assertInitCap("ß fi ffi ff st ΣΗΜΕΡΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", "UTF8_BINARY", + assertInitCap("dzDZDz", UTF8_BINARY, "Dzdzdz"); + assertInitCap("dzDZDz", UNICODE, "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_LCASE, "Džaba Ljubav Njegova"); + assertInitCap("džaba Ljubav NJegova", UNICODE_CI, "Džaba Ljubav Njegova"); + assertInitCap("ß fi ffi ff st ΣΗΜΕΡΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", UTF8_BINARY, "Ss Fi Ffi Ff St Σημερινος Ασημενιος İota","ß fi ffi ff st Σημερινος Ασημενιος I\u0307ota"); - assertInitCap("ß fi ffi ff st ΣΗΜΕΡΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", "UTF8_LCASE", + assertInitCap("ß fi ffi ff st ΣΗΜΕΡΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", UTF8_LCASE, "Ss Fi Ffi Ff St Σημερινος Ασημενιος İota"); - assertInitCap("ß fi ffi ff st ΣΗΜΕΡΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", "UNICODE", + assertInitCap("ß fi ffi ff st ΣΗΜΕΡΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", UNICODE, "Ss Fi Ffi Ff St Σημερινος Ασημενιος İota"); - assertInitCap("ß fi ffi ff st ΣΗΜΕΡςΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", "UNICODE_CI", + assertInitCap("ß fi ffi ff st ΣΗΜΕΡςΙΝΟΣ ΑΣΗΜΕΝΙΟΣ İOTA", UNICODE_CI, "Ss Fi Ffi Ff St Σημερςινος Ασημενιος İota"); // Characters that map to multiple characters when titlecased and lowercased. - assertInitCap("ß fi ffi ff st İOTA", "UTF8_BINARY", "Ss Fi Ffi Ff St İota", "ß fi ffi ff st İota"); - assertInitCap("ß fi ffi ff st OİOTA", "UTF8_BINARY", + assertInitCap("ß fi ffi ff st İOTA", UTF8_BINARY, "Ss Fi Ffi Ff St İota", "ß fi ffi ff st İota"); + assertInitCap("ß fi ffi ff st OİOTA", UTF8_BINARY, "Ss Fi Ffi Ff St Oi\u0307ota", "ß fi ffi ff st Oi̇ota"); // Lowercasing Greek letter sigma ('Σ') when case-ignorable character present. - assertInitCap("`Σ", "UTF8_BINARY", "`σ", "`σ"); - assertInitCap("1`Σ`` AΣ", "UTF8_BINARY", "1`σ`` Aς", "1`σ`` Aς"); - assertInitCap("a1`Σ``", "UTF8_BINARY", "A1`σ``", "A1`σ``"); - assertInitCap("a`Σ``", "UTF8_BINARY", "A`ς``", "A`σ``"); - assertInitCap("a`Σ``1", "UTF8_BINARY", "A`ς``1", "A`σ``1"); - assertInitCap("a`Σ``A", "UTF8_BINARY", "A`σ``a", "A`σ``a"); - assertInitCap("ΘΑ�Σ�ΟΣ�", "UTF8_BINARY", "Θα�σ�ος�", "Θα�σ�ος�"); - assertInitCap("ΘΑᵩΣ�ΟᵩΣᵩ�", "UTF8_BINARY", "Θαᵩς�οᵩςᵩ�", "Θαᵩς�οᵩςᵩ�"); - assertInitCap("ΘΑ�ᵩΣ�ΟᵩΣᵩ�", "UTF8_BINARY", "Θα�ᵩσ�οᵩςᵩ�", "Θα�ᵩσ�οᵩςᵩ�"); - assertInitCap("ΘΑ�ᵩΣᵩ�ΟᵩΣᵩ�", "UTF8_BINARY", "Θα�ᵩσᵩ�οᵩςᵩ�", "Θα�ᵩσᵩ�οᵩςᵩ�"); - assertInitCap("ΘΑ�Σ�Ο�Σ�", "UTF8_BINARY", "Θα�σ�ο�σ�", "Θα�σ�ο�σ�"); + assertInitCap("`Σ", UTF8_BINARY, "`σ", "`σ"); + assertInitCap("1`Σ`` AΣ", UTF8_BINARY, "1`σ`` Aς", "1`σ`` Aς"); + assertInitCap("a1`Σ``", UTF8_BINARY, "A1`σ``", "A1`σ``"); + assertInitCap("a`Σ``", UTF8_BINARY, "A`ς``", "A`σ``"); + assertInitCap("a`Σ``1", UTF8_BINARY, "A`ς``1", "A`σ``1"); + assertInitCap("a`Σ``A", UTF8_BINARY, "A`σ``a", "A`σ``a"); + assertInitCap("ΘΑ�Σ�ΟΣ�", UTF8_BINARY, "Θα�σ�ος�", "Θα�σ�ος�"); + assertInitCap("ΘΑᵩΣ�ΟᵩΣᵩ�", UTF8_BINARY, "Θαᵩς�οᵩςᵩ�", "Θαᵩς�οᵩςᵩ�"); + assertInitCap("ΘΑ�ᵩΣ�ΟᵩΣᵩ�", UTF8_BINARY, "Θα�ᵩσ�οᵩςᵩ�", "Θα�ᵩσ�οᵩςᵩ�"); + assertInitCap("ΘΑ�ᵩΣᵩ�ΟᵩΣᵩ�", UTF8_BINARY, "Θα�ᵩσᵩ�οᵩςᵩ�", "Θα�ᵩσᵩ�οᵩςᵩ�"); + assertInitCap("ΘΑ�Σ�Ο�Σ�", UTF8_BINARY, "Θα�σ�ο�σ�", "Θα�σ�ο�σ�"); // Disallowed bytes and invalid sequences. assertInitCap(UTF8String.fromBytes(new byte[] { (byte)0xC0, (byte)0xC1, (byte)0xF5}).toString(), - "UTF8_BINARY", "���", "���"); + UTF8_BINARY, "���", "���"); assertInitCap(UTF8String.fromBytes( new byte[]{(byte)0xC0, (byte)0xC1, (byte)0xF5, 0x20, 0x61, 0x41, (byte)0xC0}).toString(), - "UTF8_BINARY", + UTF8_BINARY, "��� Aa�", "��� Aa�"); assertInitCap(UTF8String.fromBytes(new byte[]{(byte)0xC2,(byte)0xC2}).toString(), - "UTF8_BINARY", "��", "��"); + UTF8_BINARY, "��", "��"); assertInitCap(UTF8String.fromBytes( new byte[]{0x61, 0x41, (byte)0xC2, (byte)0xC2, 0x41}).toString(), - "UTF8_BINARY", + UTF8_BINARY, "Aa��a", "Aa��a"); } @@ -1559,147 +1560,147 @@ private void assertStringInstr(String string, String substring, @Test public void testStringInstr() throws SparkException { // Empty strings. - assertStringInstr("", "", "UTF8_BINARY", 1); - assertStringInstr("", "", "UTF8_LCASE", 1); - assertStringInstr("", "", "UNICODE_CI", 1); - assertStringInstr("", "", "UNICODE", 1); - assertStringInstr("a", "", "UTF8_BINARY", 1); - assertStringInstr("a", "", "UTF8_LCASE", 1); - assertStringInstr("a", "", "UNICODE", 1); - assertStringInstr("a", "", "UNICODE_CI", 1); - assertStringInstr("", "x", "UTF8_BINARY", 0); - assertStringInstr("", "x", "UTF8_LCASE", 0); - assertStringInstr("", "x", "UNICODE", 0); - assertStringInstr("", "x", "UNICODE_CI", 0); + assertStringInstr("", "", UTF8_BINARY, 1); + assertStringInstr("", "", UTF8_LCASE, 1); + assertStringInstr("", "", UNICODE_CI, 1); + assertStringInstr("", "", UNICODE, 1); + assertStringInstr("a", "", UTF8_BINARY, 1); + assertStringInstr("a", "", UTF8_LCASE, 1); + assertStringInstr("a", "", UNICODE, 1); + assertStringInstr("a", "", UNICODE_CI, 1); + assertStringInstr("", "x", UTF8_BINARY, 0); + assertStringInstr("", "x", UTF8_LCASE, 0); + assertStringInstr("", "x", UNICODE, 0); + assertStringInstr("", "x", UNICODE_CI, 0); // Basic tests. - assertStringInstr("aaads", "aa", "UTF8_BINARY", 1); - assertStringInstr("aaads", "aa", "UTF8_LCASE", 1); - assertStringInstr("aaads", "aa", "UNICODE", 1); - assertStringInstr("aaads", "aa", "UNICODE_CI", 1); - assertStringInstr("aaads", "ds", "UTF8_BINARY", 4); - assertStringInstr("aaads", "ds", "UTF8_LCASE", 4); - assertStringInstr("aaads", "ds", "UNICODE", 4); - assertStringInstr("aaads", "ds", "UNICODE_CI", 4); - assertStringInstr("aaads", "Aa", "UTF8_BINARY", 0); - assertStringInstr("aaads", "Aa", "UTF8_LCASE", 1); - assertStringInstr("aaads", "Aa", "UNICODE", 0); - assertStringInstr("aaads", "Aa", "UNICODE_CI", 1); - assertStringInstr("aaaDs", "de", "UTF8_BINARY", 0); - assertStringInstr("aaaDs", "de", "UTF8_LCASE", 0); - assertStringInstr("aaaDs", "de", "UNICODE", 0); - assertStringInstr("aaaDs", "de", "UNICODE_CI", 0); - assertStringInstr("aaaDs", "ds", "UTF8_BINARY", 0); - assertStringInstr("aaaDs", "ds", "UTF8_LCASE", 4); - assertStringInstr("aaaDs", "ds", "UNICODE", 0); - assertStringInstr("aaaDs", "ds", "UNICODE_CI", 4); - assertStringInstr("aaadS", "Ds", "UTF8_BINARY", 0); - assertStringInstr("aaadS", "Ds", "UTF8_LCASE", 4); - assertStringInstr("aaadS", "Ds", "UNICODE", 0); - assertStringInstr("aaadS", "Ds", "UNICODE_CI", 4); + assertStringInstr("aaads", "aa", UTF8_BINARY, 1); + assertStringInstr("aaads", "aa", UTF8_LCASE, 1); + assertStringInstr("aaads", "aa", UNICODE, 1); + assertStringInstr("aaads", "aa", UNICODE_CI, 1); + assertStringInstr("aaads", "ds", UTF8_BINARY, 4); + assertStringInstr("aaads", "ds", UTF8_LCASE, 4); + assertStringInstr("aaads", "ds", UNICODE, 4); + assertStringInstr("aaads", "ds", UNICODE_CI, 4); + assertStringInstr("aaads", "Aa", UTF8_BINARY, 0); + assertStringInstr("aaads", "Aa", UTF8_LCASE, 1); + assertStringInstr("aaads", "Aa", UNICODE, 0); + assertStringInstr("aaads", "Aa", UNICODE_CI, 1); + assertStringInstr("aaaDs", "de", UTF8_BINARY, 0); + assertStringInstr("aaaDs", "de", UTF8_LCASE, 0); + assertStringInstr("aaaDs", "de", UNICODE, 0); + assertStringInstr("aaaDs", "de", UNICODE_CI, 0); + assertStringInstr("aaaDs", "ds", UTF8_BINARY, 0); + assertStringInstr("aaaDs", "ds", UTF8_LCASE, 4); + assertStringInstr("aaaDs", "ds", UNICODE, 0); + assertStringInstr("aaaDs", "ds", UNICODE_CI, 4); + assertStringInstr("aaadS", "Ds", UTF8_BINARY, 0); + assertStringInstr("aaadS", "Ds", UTF8_LCASE, 4); + assertStringInstr("aaadS", "Ds", UNICODE, 0); + assertStringInstr("aaadS", "Ds", UNICODE_CI, 4); assertStringInstr("aaaČŠčšcs", "cs", "SR", 8); assertStringInstr("aaaČŠčšcs", "cs", "SR_CI_AI", 4); // Advanced tests. - assertStringInstr("test大千世界X大千世界", "大千", "UTF8_BINARY", 5); - assertStringInstr("test大千世界X大千世界", "大千", "UTF8_LCASE", 5); - assertStringInstr("test大千世界X大千世界", "大千", "UNICODE", 5); - assertStringInstr("test大千世界X大千世界", "大千", "UNICODE_CI", 5); - assertStringInstr("test大千世界X大千世界", "界X", "UTF8_BINARY", 8); - assertStringInstr("test大千世界X大千世界", "界X", "UTF8_LCASE", 8); - assertStringInstr("test大千世界X大千世界", "界X", "UNICODE", 8); - assertStringInstr("test大千世界X大千世界", "界X", "UNICODE_CI", 8); - assertStringInstr("test大千世界X大千世界", "界x", "UTF8_BINARY", 0); - assertStringInstr("test大千世界X大千世界", "界x", "UTF8_LCASE", 8); - assertStringInstr("test大千世界X大千世界", "界x", "UNICODE", 0); - assertStringInstr("test大千世界X大千世界", "界x", "UNICODE_CI", 8); - assertStringInstr("test大千世界X大千世界", "界y", "UTF8_BINARY", 0); - assertStringInstr("test大千世界X大千世界", "界y", "UTF8_LCASE", 0); - assertStringInstr("test大千世界X大千世界", "界y", "UNICODE", 0); - assertStringInstr("test大千世界X大千世界", "界y", "UNICODE_CI", 0); + assertStringInstr("test大千世界X大千世界", "大千", UTF8_BINARY, 5); + assertStringInstr("test大千世界X大千世界", "大千", UTF8_LCASE, 5); + assertStringInstr("test大千世界X大千世界", "大千", UNICODE, 5); + assertStringInstr("test大千世界X大千世界", "大千", UNICODE_CI, 5); + assertStringInstr("test大千世界X大千世界", "界X", UTF8_BINARY, 8); + assertStringInstr("test大千世界X大千世界", "界X", UTF8_LCASE, 8); + assertStringInstr("test大千世界X大千世界", "界X", UNICODE, 8); + assertStringInstr("test大千世界X大千世界", "界X", UNICODE_CI, 8); + assertStringInstr("test大千世界X大千世界", "界x", UTF8_BINARY, 0); + assertStringInstr("test大千世界X大千世界", "界x", UTF8_LCASE, 8); + assertStringInstr("test大千世界X大千世界", "界x", UNICODE, 0); + assertStringInstr("test大千世界X大千世界", "界x", UNICODE_CI, 8); + assertStringInstr("test大千世界X大千世界", "界y", UTF8_BINARY, 0); + assertStringInstr("test大千世界X大千世界", "界y", UTF8_LCASE, 0); + assertStringInstr("test大千世界X大千世界", "界y", UNICODE, 0); + assertStringInstr("test大千世界X大千世界", "界y", UNICODE_CI, 0); // One-to-many case mapping (e.g. Turkish dotted I). - assertStringInstr("i\u0307", "i", "UNICODE_CI", 0); - assertStringInstr("i\u0307", "\u0307", "UNICODE_CI", 0); - assertStringInstr("i\u0307", "İ", "UNICODE_CI", 1); - assertStringInstr("İ", "i", "UNICODE_CI", 0); - assertStringInstr("İoi̇o12", "i\u0307o", "UNICODE_CI", 1); - assertStringInstr("i̇oİo12", "İo", "UNICODE_CI", 1); - assertStringInstr("abİoi̇o", "i\u0307o", "UNICODE_CI", 3); - 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\u0307", "i", "UTF8_LCASE", 1); // != UNICODE_CI - assertStringInstr("i\u0307", "\u0307", "UTF8_LCASE", 2); // != UNICODE_CI - assertStringInstr("i\u0307", "İ", "UTF8_LCASE", 1); - assertStringInstr("İ", "i", "UTF8_LCASE", 0); - assertStringInstr("İoi̇o12", "i\u0307o", "UTF8_LCASE", 1); - assertStringInstr("i̇oİo12", "İo", "UTF8_LCASE", 1); - assertStringInstr("abİoi̇o", "i\u0307o", "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); + assertStringInstr("i\u0307", "i", UNICODE_CI, 0); + assertStringInstr("i\u0307", "\u0307", UNICODE_CI, 0); + assertStringInstr("i\u0307", "İ", UNICODE_CI, 1); + assertStringInstr("İ", "i", UNICODE_CI, 0); + assertStringInstr("İoi̇o12", "i\u0307o", UNICODE_CI, 1); + assertStringInstr("i̇oİo12", "İo", UNICODE_CI, 1); + assertStringInstr("abİoi̇o", "i\u0307o", UNICODE_CI, 3); + 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\u0307", "i", UTF8_LCASE, 1); // != UNICODE_CI + assertStringInstr("i\u0307", "\u0307", UTF8_LCASE, 2); // != UNICODE_CI + assertStringInstr("i\u0307", "İ", UTF8_LCASE, 1); + assertStringInstr("İ", "i", UTF8_LCASE, 0); + assertStringInstr("İoi̇o12", "i\u0307o", UTF8_LCASE, 1); + assertStringInstr("i̇oİo12", "İo", UTF8_LCASE, 1); + assertStringInstr("abİoi̇o", "i\u0307o", 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); // Conditional case mapping (e.g. Greek sigmas). - assertStringInstr("σ", "σ", "UTF8_BINARY", 1); - assertStringInstr("σ", "ς", "UTF8_BINARY", 0); - assertStringInstr("σ", "Σ", "UTF8_BINARY", 0); - assertStringInstr("ς", "σ", "UTF8_BINARY", 0); - assertStringInstr("ς", "ς", "UTF8_BINARY", 1); - assertStringInstr("ς", "Σ", "UTF8_BINARY", 0); - assertStringInstr("Σ", "σ", "UTF8_BINARY", 0); - assertStringInstr("Σ", "ς", "UTF8_BINARY", 0); - assertStringInstr("Σ", "Σ", "UTF8_BINARY", 1); - assertStringInstr("σ", "σ", "UTF8_LCASE", 1); - assertStringInstr("σ", "ς", "UTF8_LCASE", 1); - assertStringInstr("σ", "Σ", "UTF8_LCASE", 1); - assertStringInstr("ς", "σ", "UTF8_LCASE", 1); - assertStringInstr("ς", "ς", "UTF8_LCASE", 1); - assertStringInstr("ς", "Σ", "UTF8_LCASE", 1); - assertStringInstr("Σ", "σ", "UTF8_LCASE", 1); - assertStringInstr("Σ", "ς", "UTF8_LCASE", 1); - assertStringInstr("Σ", "Σ", "UTF8_LCASE", 1); - assertStringInstr("σ", "σ", "UNICODE", 1); - assertStringInstr("σ", "ς", "UNICODE", 0); - assertStringInstr("σ", "Σ", "UNICODE", 0); - assertStringInstr("ς", "σ", "UNICODE", 0); - assertStringInstr("ς", "ς", "UNICODE", 1); - assertStringInstr("ς", "Σ", "UNICODE", 0); - assertStringInstr("Σ", "σ", "UNICODE", 0); - assertStringInstr("Σ", "ς", "UNICODE", 0); - assertStringInstr("Σ", "Σ", "UNICODE", 1); - assertStringInstr("σ", "σ", "UNICODE_CI", 1); - assertStringInstr("σ", "ς", "UNICODE_CI", 1); - assertStringInstr("σ", "Σ", "UNICODE_CI", 1); - assertStringInstr("ς", "σ", "UNICODE_CI", 1); - assertStringInstr("ς", "ς", "UNICODE_CI", 1); - assertStringInstr("ς", "Σ", "UNICODE_CI", 1); - assertStringInstr("Σ", "σ", "UNICODE_CI", 1); - assertStringInstr("Σ", "ς", "UNICODE_CI", 1); - assertStringInstr("Σ", "Σ", "UNICODE_CI", 1); + assertStringInstr("σ", "σ", UTF8_BINARY, 1); + assertStringInstr("σ", "ς", UTF8_BINARY, 0); + assertStringInstr("σ", "Σ", UTF8_BINARY, 0); + assertStringInstr("ς", "σ", UTF8_BINARY, 0); + assertStringInstr("ς", "ς", UTF8_BINARY, 1); + assertStringInstr("ς", "Σ", UTF8_BINARY, 0); + assertStringInstr("Σ", "σ", UTF8_BINARY, 0); + assertStringInstr("Σ", "ς", UTF8_BINARY, 0); + assertStringInstr("Σ", "Σ", UTF8_BINARY, 1); + assertStringInstr("σ", "σ", UTF8_LCASE, 1); + assertStringInstr("σ", "ς", UTF8_LCASE, 1); + assertStringInstr("σ", "Σ", UTF8_LCASE, 1); + assertStringInstr("ς", "σ", UTF8_LCASE, 1); + assertStringInstr("ς", "ς", UTF8_LCASE, 1); + assertStringInstr("ς", "Σ", UTF8_LCASE, 1); + assertStringInstr("Σ", "σ", UTF8_LCASE, 1); + assertStringInstr("Σ", "ς", UTF8_LCASE, 1); + assertStringInstr("Σ", "Σ", UTF8_LCASE, 1); + assertStringInstr("σ", "σ", UNICODE, 1); + assertStringInstr("σ", "ς", UNICODE, 0); + assertStringInstr("σ", "Σ", UNICODE, 0); + assertStringInstr("ς", "σ", UNICODE, 0); + assertStringInstr("ς", "ς", UNICODE, 1); + assertStringInstr("ς", "Σ", UNICODE, 0); + assertStringInstr("Σ", "σ", UNICODE, 0); + assertStringInstr("Σ", "ς", UNICODE, 0); + assertStringInstr("Σ", "Σ", UNICODE, 1); + assertStringInstr("σ", "σ", UNICODE_CI, 1); + assertStringInstr("σ", "ς", UNICODE_CI, 1); + assertStringInstr("σ", "Σ", UNICODE_CI, 1); + assertStringInstr("ς", "σ", UNICODE_CI, 1); + assertStringInstr("ς", "ς", UNICODE_CI, 1); + assertStringInstr("ς", "Σ", UNICODE_CI, 1); + assertStringInstr("Σ", "σ", UNICODE_CI, 1); + assertStringInstr("Σ", "ς", UNICODE_CI, 1); + assertStringInstr("Σ", "Σ", UNICODE_CI, 1); // Surrogate pairs. - assertStringInstr("a🙃b", "a", "UTF8_BINARY", 1); - assertStringInstr("a🙃b", "a", "UTF8_LCASE", 1); - assertStringInstr("a🙃b", "a", "UNICODE", 1); - assertStringInstr("a🙃b", "a", "UNICODE_CI", 1); - assertStringInstr("a🙃b", "🙃", "UTF8_BINARY", 2); - assertStringInstr("a🙃b", "🙃", "UTF8_LCASE", 2); - assertStringInstr("a🙃b", "🙃", "UNICODE", 2); - assertStringInstr("a🙃b", "🙃", "UNICODE_CI", 2); - assertStringInstr("a🙃b", "b", "UTF8_BINARY", 3); - assertStringInstr("a🙃b", "b", "UTF8_LCASE", 3); - assertStringInstr("a🙃b", "b", "UNICODE", 3); - assertStringInstr("a🙃b", "b", "UNICODE_CI", 3); - assertStringInstr("a🙃🙃b", "🙃", "UTF8_BINARY", 2); - assertStringInstr("a🙃🙃b", "🙃", "UTF8_LCASE", 2); - assertStringInstr("a🙃🙃b", "🙃", "UNICODE", 2); - assertStringInstr("a🙃🙃b", "🙃", "UNICODE_CI", 2); - assertStringInstr("a🙃🙃b", "b", "UTF8_BINARY", 4); - assertStringInstr("a🙃🙃b", "b", "UTF8_LCASE", 4); - assertStringInstr("a🙃🙃b", "b", "UNICODE", 4); - assertStringInstr("a🙃🙃b", "b", "UNICODE_CI", 4); - assertStringInstr("a🙃x🙃b", "b", "UTF8_BINARY", 5); - assertStringInstr("a🙃x🙃b", "b", "UTF8_LCASE", 5); - assertStringInstr("a🙃x🙃b", "b", "UNICODE", 5); - assertStringInstr("a🙃x🙃b", "b", "UNICODE_CI", 5); + assertStringInstr("a🙃b", "a", UTF8_BINARY, 1); + assertStringInstr("a🙃b", "a", UTF8_LCASE, 1); + assertStringInstr("a🙃b", "a", UNICODE, 1); + assertStringInstr("a🙃b", "a", UNICODE_CI, 1); + assertStringInstr("a🙃b", "🙃", UTF8_BINARY, 2); + assertStringInstr("a🙃b", "🙃", UTF8_LCASE, 2); + assertStringInstr("a🙃b", "🙃", UNICODE, 2); + assertStringInstr("a🙃b", "🙃", UNICODE_CI, 2); + assertStringInstr("a🙃b", "b", UTF8_BINARY, 3); + assertStringInstr("a🙃b", "b", UTF8_LCASE, 3); + assertStringInstr("a🙃b", "b", UNICODE, 3); + assertStringInstr("a🙃b", "b", UNICODE_CI, 3); + assertStringInstr("a🙃🙃b", "🙃", UTF8_BINARY, 2); + assertStringInstr("a🙃🙃b", "🙃", UTF8_LCASE, 2); + assertStringInstr("a🙃🙃b", "🙃", UNICODE, 2); + assertStringInstr("a🙃🙃b", "🙃", UNICODE_CI, 2); + assertStringInstr("a🙃🙃b", "b", UTF8_BINARY, 4); + assertStringInstr("a🙃🙃b", "b", UTF8_LCASE, 4); + assertStringInstr("a🙃🙃b", "b", UNICODE, 4); + assertStringInstr("a🙃🙃b", "b", UNICODE_CI, 4); + assertStringInstr("a🙃x🙃b", "b", UTF8_BINARY, 5); + assertStringInstr("a🙃x🙃b", "b", UTF8_LCASE, 5); + assertStringInstr("a🙃x🙃b", "b", UNICODE, 5); + assertStringInstr("a🙃x🙃b", "b", UNICODE_CI, 5); } /** @@ -1717,256 +1718,256 @@ private void assertFindInSet(String word, UTF8String set, String collationName, @Test public void testFindInSet() throws SparkException { // Empty strings. - assertFindInSet("", UTF8String.fromString(""), "UTF8_BINARY", 1); - assertFindInSet("", UTF8String.fromString(""), "UTF8_LCASE", 1); - assertFindInSet("", UTF8String.fromString(""), "UNICODE", 1); - assertFindInSet("", UTF8String.fromString(""), "UNICODE_CI", 1); - assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_BINARY", 0); - assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_LCASE", 0); - assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE", 0); - assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE_CI", 0); - assertFindInSet("", UTF8String.fromString(",abc,b,ab,c,def"), "UTF8_BINARY", 1); - assertFindInSet("", UTF8String.fromString(",abc,b,ab,c,def"), "UTF8_LCASE", 1); - assertFindInSet("", UTF8String.fromString(",abc,b,ab,c,def"), "UNICODE", 1); - assertFindInSet("", UTF8String.fromString(",abc,b,ab,c,def"), "UNICODE_CI", 1); - assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def,"), "UTF8_BINARY", 6); - assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def,"), "UTF8_LCASE", 6); - assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def,"), "UNICODE", 6); - assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def,"), "UNICODE_CI", 6); - assertFindInSet("", UTF8String.fromString("abc"), "UTF8_BINARY", 0); - assertFindInSet("", UTF8String.fromString("abc"), "UTF8_LCASE", 0); - assertFindInSet("", UTF8String.fromString("abc"), "UNICODE", 0); - assertFindInSet("", UTF8String.fromString("abc"), "UNICODE_CI", 0); + assertFindInSet("", UTF8String.fromString(""), UTF8_BINARY, 1); + assertFindInSet("", UTF8String.fromString(""), UTF8_LCASE, 1); + assertFindInSet("", UTF8String.fromString(""), UNICODE, 1); + assertFindInSet("", UTF8String.fromString(""), UNICODE_CI, 1); + assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def"), UTF8_BINARY, 0); + assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def"), UTF8_LCASE, 0); + assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def"), UNICODE, 0); + assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def"), UNICODE_CI, 0); + assertFindInSet("", UTF8String.fromString(",abc,b,ab,c,def"), UTF8_BINARY, 1); + assertFindInSet("", UTF8String.fromString(",abc,b,ab,c,def"), UTF8_LCASE, 1); + assertFindInSet("", UTF8String.fromString(",abc,b,ab,c,def"), UNICODE, 1); + assertFindInSet("", UTF8String.fromString(",abc,b,ab,c,def"), UNICODE_CI, 1); + assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def,"), UTF8_BINARY, 6); + assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def,"), UTF8_LCASE, 6); + assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def,"), UNICODE, 6); + assertFindInSet("", UTF8String.fromString("abc,b,ab,c,def,"), UNICODE_CI, 6); + assertFindInSet("", UTF8String.fromString("abc"), UTF8_BINARY, 0); + assertFindInSet("", UTF8String.fromString("abc"), UTF8_LCASE, 0); + assertFindInSet("", UTF8String.fromString("abc"), UNICODE, 0); + assertFindInSet("", UTF8String.fromString("abc"), UNICODE_CI, 0); // Basic tests. - assertFindInSet("xx", UTF8String.fromString("xx"), "UTF8_BINARY", 1); - assertFindInSet("xx", UTF8String.fromString("xx"), "UTF8_LCASE", 1); - assertFindInSet("xx", UTF8String.fromString("xx"), "UNICODE", 1); - assertFindInSet("xx", UTF8String.fromString("xx"), "UNICODE_CI", 1); - assertFindInSet("a", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_BINARY", 0); - assertFindInSet("a", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_LCASE", 0); - assertFindInSet("a", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE", 0); - assertFindInSet("a", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE_CI", 0); - assertFindInSet("abc", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_BINARY", 1); - assertFindInSet("abc", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_LCASE", 1); - assertFindInSet("abc", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE", 1); - assertFindInSet("abc", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE_CI", 1); - assertFindInSet("abcd", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_BINARY", 0); - assertFindInSet("abcd", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_LCASE", 0); - assertFindInSet("abcd", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE", 0); - assertFindInSet("abcd", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE_CI", 0); - assertFindInSet("def", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_BINARY", 5); - assertFindInSet("def", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_LCASE", 5); - assertFindInSet("def", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE", 5); - assertFindInSet("def", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE_CI", 5); - assertFindInSet("xyz", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_BINARY", 0); - assertFindInSet("xyz", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_LCASE", 0); - assertFindInSet("xyz", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE", 0); - assertFindInSet("xyz", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE_CI", 0); - assertFindInSet("Ab", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_BINARY", 0); - assertFindInSet("Ab", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_LCASE", 3); - assertFindInSet("Ab", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE", 0); - assertFindInSet("Ab", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE_CI", 3); - assertFindInSet("d,ef", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_BINARY", 0); - assertFindInSet("d,ef", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_LCASE", 0); - assertFindInSet("d,ef", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE", 0); - assertFindInSet("d,ef", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE_CI", 0); - assertFindInSet("C", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_BINARY", 0); - assertFindInSet("C", UTF8String.fromString("abc,b,ab,c,def"), "UTF8_LCASE", 4); - assertFindInSet("C", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE", 0); - assertFindInSet("C", UTF8String.fromString("abc,b,ab,c,def"), "UNICODE_CI", 4); + assertFindInSet("xx", UTF8String.fromString("xx"), UTF8_BINARY, 1); + assertFindInSet("xx", UTF8String.fromString("xx"), UTF8_LCASE, 1); + assertFindInSet("xx", UTF8String.fromString("xx"), UNICODE, 1); + assertFindInSet("xx", UTF8String.fromString("xx"), UNICODE_CI, 1); + assertFindInSet("a", UTF8String.fromString("abc,b,ab,c,def"), UTF8_BINARY, 0); + assertFindInSet("a", UTF8String.fromString("abc,b,ab,c,def"), UTF8_LCASE, 0); + assertFindInSet("a", UTF8String.fromString("abc,b,ab,c,def"), UNICODE, 0); + assertFindInSet("a", UTF8String.fromString("abc,b,ab,c,def"), UNICODE_CI, 0); + assertFindInSet("abc", UTF8String.fromString("abc,b,ab,c,def"), UTF8_BINARY, 1); + assertFindInSet("abc", UTF8String.fromString("abc,b,ab,c,def"), UTF8_LCASE, 1); + assertFindInSet("abc", UTF8String.fromString("abc,b,ab,c,def"), UNICODE, 1); + assertFindInSet("abc", UTF8String.fromString("abc,b,ab,c,def"), UNICODE_CI, 1); + assertFindInSet("abcd", UTF8String.fromString("abc,b,ab,c,def"), UTF8_BINARY, 0); + assertFindInSet("abcd", UTF8String.fromString("abc,b,ab,c,def"), UTF8_LCASE, 0); + assertFindInSet("abcd", UTF8String.fromString("abc,b,ab,c,def"), UNICODE, 0); + assertFindInSet("abcd", UTF8String.fromString("abc,b,ab,c,def"), UNICODE_CI, 0); + assertFindInSet("def", UTF8String.fromString("abc,b,ab,c,def"), UTF8_BINARY, 5); + assertFindInSet("def", UTF8String.fromString("abc,b,ab,c,def"), UTF8_LCASE, 5); + assertFindInSet("def", UTF8String.fromString("abc,b,ab,c,def"), UNICODE, 5); + assertFindInSet("def", UTF8String.fromString("abc,b,ab,c,def"), UNICODE_CI, 5); + assertFindInSet("xyz", UTF8String.fromString("abc,b,ab,c,def"), UTF8_BINARY, 0); + assertFindInSet("xyz", UTF8String.fromString("abc,b,ab,c,def"), UTF8_LCASE, 0); + assertFindInSet("xyz", UTF8String.fromString("abc,b,ab,c,def"), UNICODE, 0); + assertFindInSet("xyz", UTF8String.fromString("abc,b,ab,c,def"), UNICODE_CI, 0); + assertFindInSet("Ab", UTF8String.fromString("abc,b,ab,c,def"), UTF8_BINARY, 0); + assertFindInSet("Ab", UTF8String.fromString("abc,b,ab,c,def"), UTF8_LCASE, 3); + assertFindInSet("Ab", UTF8String.fromString("abc,b,ab,c,def"), UNICODE, 0); + assertFindInSet("Ab", UTF8String.fromString("abc,b,ab,c,def"), UNICODE_CI, 3); + assertFindInSet("d,ef", UTF8String.fromString("abc,b,ab,c,def"), UTF8_BINARY, 0); + assertFindInSet("d,ef", UTF8String.fromString("abc,b,ab,c,def"), UTF8_LCASE, 0); + assertFindInSet("d,ef", UTF8String.fromString("abc,b,ab,c,def"), UNICODE, 0); + assertFindInSet("d,ef", UTF8String.fromString("abc,b,ab,c,def"), UNICODE_CI, 0); + assertFindInSet("C", UTF8String.fromString("abc,b,ab,c,def"), UTF8_BINARY, 0); + assertFindInSet("C", UTF8String.fromString("abc,b,ab,c,def"), UTF8_LCASE, 4); + assertFindInSet("C", UTF8String.fromString("abc,b,ab,c,def"), UNICODE, 0); + assertFindInSet("C", UTF8String.fromString("abc,b,ab,c,def"), UNICODE_CI, 4); // Advanced tests. - assertFindInSet("大", UTF8String.fromString("test,大千,世,界X,大,千,世界"), "UTF8_BINARY", 5); - assertFindInSet("大", UTF8String.fromString("test,大千,世,界X,大,千,世界"), "UTF8_LCASE", 5); - assertFindInSet("大", UTF8String.fromString("test,大千,世,界X,大,千,世界"), "UNICODE", 5); - assertFindInSet("大", UTF8String.fromString("test,大千,世,界X,大,千,世界"), "UNICODE_CI", 5); - assertFindInSet("界x", UTF8String.fromString("test,大千,世,界X,大,千,世界"), "UTF8_BINARY", 0); - assertFindInSet("界x", UTF8String.fromString("test,大千,世,界X,大,千,世界"), "UTF8_LCASE", 4); - assertFindInSet("界x", UTF8String.fromString("test,大千,世,界X,大,千,世界"), "UNICODE", 0); - assertFindInSet("界x", UTF8String.fromString("test,大千,世,界X,大,千,世界"), "UNICODE_CI", 4); - assertFindInSet("界x", UTF8String.fromString("test,大千,界Xx,世,界X,大,千,世界"), "UTF8_BINARY", 0); - assertFindInSet("界x", UTF8String.fromString("test,大千,界Xx,世,界X,大,千,世界"), "UTF8_LCASE", 5); - assertFindInSet("界x", UTF8String.fromString("test,大千,界Xx,世,界X,大,千,世界"), "UNICODE", 0); - assertFindInSet("界x", UTF8String.fromString("test,大千,界Xx,世,界X,大,千,世界"), "UNICODE_CI", 5); + assertFindInSet("大", UTF8String.fromString("test,大千,世,界X,大,千,世界"), UTF8_BINARY, 5); + assertFindInSet("大", UTF8String.fromString("test,大千,世,界X,大,千,世界"), UTF8_LCASE, 5); + assertFindInSet("大", UTF8String.fromString("test,大千,世,界X,大,千,世界"), UNICODE, 5); + assertFindInSet("大", UTF8String.fromString("test,大千,世,界X,大,千,世界"), UNICODE_CI, 5); + assertFindInSet("界x", UTF8String.fromString("test,大千,世,界X,大,千,世界"), UTF8_BINARY, 0); + assertFindInSet("界x", UTF8String.fromString("test,大千,世,界X,大,千,世界"), UTF8_LCASE, 4); + assertFindInSet("界x", UTF8String.fromString("test,大千,世,界X,大,千,世界"), UNICODE, 0); + assertFindInSet("界x", UTF8String.fromString("test,大千,世,界X,大,千,世界"), UNICODE_CI, 4); + assertFindInSet("界x", UTF8String.fromString("test,大千,界Xx,世,界X,大,千,世界"), UTF8_BINARY, 0); + assertFindInSet("界x", UTF8String.fromString("test,大千,界Xx,世,界X,大,千,世界"), UTF8_LCASE, 5); + assertFindInSet("界x", UTF8String.fromString("test,大千,界Xx,世,界X,大,千,世界"), UNICODE, 0); + assertFindInSet("界x", UTF8String.fromString("test,大千,界Xx,世,界X,大,千,世界"), UNICODE_CI, 5); // One-to-many case mapping (e.g. Turkish dotted I). - assertFindInSet("i\u0307", UTF8String.fromString("İ"), "UTF8_BINARY", 0); - assertFindInSet("i\u0307", UTF8String.fromString("İ"), "UTF8_LCASE", 1); - assertFindInSet("i\u0307", UTF8String.fromString("İ"), "UNICODE", 0); - assertFindInSet("i\u0307", UTF8String.fromString("İ"), "UNICODE_CI", 1); - assertFindInSet("i", UTF8String.fromString("İ"), "UTF8_BINARY", 0); - assertFindInSet("i", UTF8String.fromString("İ"), "UTF8_LCASE", 0); - assertFindInSet("i", UTF8String.fromString("İ"), "UNICODE", 0); - assertFindInSet("i", UTF8String.fromString("İ"), "UNICODE_CI", 0); - assertFindInSet("i\u0307", UTF8String.fromString("i\u0307"), "UTF8_BINARY", 1); - assertFindInSet("i\u0307", UTF8String.fromString("i\u0307"), "UTF8_LCASE", 1); - assertFindInSet("i\u0307", UTF8String.fromString("i\u0307"), "UNICODE", 1); - assertFindInSet("i\u0307", UTF8String.fromString("i\u0307"), "UNICODE_CI", 1); - assertFindInSet("i", UTF8String.fromString("i\u0307"), "UTF8_BINARY", 0); - assertFindInSet("i", UTF8String.fromString("i\u0307"), "UTF8_LCASE", 0); - assertFindInSet("i", UTF8String.fromString("i\u0307"), "UNICODE", 0); - assertFindInSet("i", UTF8String.fromString("i\u0307"), "UNICODE_CI", 0); - assertFindInSet("i\u0307", UTF8String.fromString("İ,"), "UTF8_BINARY", 0); - assertFindInSet("i\u0307", UTF8String.fromString("İ,"), "UTF8_LCASE", 1); - assertFindInSet("i\u0307", UTF8String.fromString("İ,"), "UNICODE", 0); - assertFindInSet("i\u0307", UTF8String.fromString("İ,"), "UNICODE_CI", 1); - assertFindInSet("i", UTF8String.fromString("İ,"), "UTF8_BINARY", 0); - assertFindInSet("i", UTF8String.fromString("İ,"), "UTF8_LCASE", 0); - assertFindInSet("i", UTF8String.fromString("İ,"), "UNICODE", 0); - assertFindInSet("i", UTF8String.fromString("İ,"), "UNICODE_CI", 0); - assertFindInSet("i\u0307", UTF8String.fromString("i\u0307,"), "UTF8_BINARY", 1); - assertFindInSet("i\u0307", UTF8String.fromString("i\u0307,"), "UTF8_LCASE", 1); - assertFindInSet("i\u0307", UTF8String.fromString("i\u0307,"), "UNICODE", 1); - assertFindInSet("i\u0307", UTF8String.fromString("i\u0307,"), "UNICODE_CI", 1); - assertFindInSet("i", UTF8String.fromString("i\u0307,"), "UTF8_BINARY", 0); - assertFindInSet("i", UTF8String.fromString("i\u0307,"), "UTF8_LCASE", 0); - assertFindInSet("i", UTF8String.fromString("i\u0307,"), "UNICODE", 0); - assertFindInSet("i", UTF8String.fromString("i\u0307,"), "UNICODE_CI", 0); - assertFindInSet("i\u0307", UTF8String.fromString("ab,İ"), "UTF8_BINARY", 0); - assertFindInSet("i\u0307", UTF8String.fromString("ab,İ"), "UTF8_LCASE", 2); - assertFindInSet("i\u0307", UTF8String.fromString("ab,İ"), "UNICODE", 0); - assertFindInSet("i\u0307", UTF8String.fromString("ab,İ"), "UNICODE_CI", 2); - assertFindInSet("i", UTF8String.fromString("ab,İ"), "UTF8_BINARY", 0); - assertFindInSet("i", UTF8String.fromString("ab,İ"), "UTF8_LCASE", 0); - assertFindInSet("i", UTF8String.fromString("ab,İ"), "UNICODE", 0); - assertFindInSet("i", UTF8String.fromString("ab,İ"), "UNICODE_CI", 0); - assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307"), "UTF8_BINARY", 2); - assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307"), "UTF8_LCASE", 2); - assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307"), "UNICODE", 2); - assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307"), "UNICODE_CI", 2); - assertFindInSet("i", UTF8String.fromString("ab,i\u0307"), "UTF8_BINARY", 0); - assertFindInSet("i", UTF8String.fromString("ab,i\u0307"), "UTF8_LCASE", 0); - assertFindInSet("i", UTF8String.fromString("ab,i\u0307"), "UNICODE", 0); - assertFindInSet("i", UTF8String.fromString("ab,i\u0307"), "UNICODE_CI", 0); - assertFindInSet("İ", UTF8String.fromString("ab,i\u0307"), "UTF8_BINARY", 0); - assertFindInSet("İ", UTF8String.fromString("ab,i\u0307"), "UTF8_LCASE", 2); - assertFindInSet("İ", UTF8String.fromString("ab,i\u0307"), "UNICODE", 0); - assertFindInSet("İ", UTF8String.fromString("ab,i\u0307"), "UNICODE_CI", 2); - assertFindInSet("i\u0307", UTF8String.fromString("ab,İ,12"), "UTF8_BINARY", 0); - assertFindInSet("i\u0307", UTF8String.fromString("ab,İ,12"), "UTF8_LCASE", 2); - assertFindInSet("i\u0307", UTF8String.fromString("ab,İ,12"), "UNICODE", 0); - assertFindInSet("i\u0307", UTF8String.fromString("ab,İ,12"), "UNICODE_CI", 2); - assertFindInSet("i", UTF8String.fromString("ab,İ,12"), "UTF8_BINARY", 0); - assertFindInSet("i", UTF8String.fromString("ab,İ,12"), "UTF8_LCASE", 0); - assertFindInSet("i", UTF8String.fromString("ab,İ,12"), "UNICODE", 0); - assertFindInSet("i", UTF8String.fromString("ab,İ,12"), "UNICODE_CI", 0); - assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307,12"), "UTF8_BINARY", 2); - assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307,12"), "UTF8_LCASE", 2); - assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307,12"), "UNICODE", 2); - assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307,12"), "UNICODE_CI", 2); - assertFindInSet("i", UTF8String.fromString("ab,i\u0307,12"), "UTF8_BINARY", 0); - assertFindInSet("i", UTF8String.fromString("ab,i\u0307,12"), "UTF8_LCASE", 0); - assertFindInSet("i", UTF8String.fromString("ab,i\u0307,12"), "UNICODE", 0); - assertFindInSet("i", UTF8String.fromString("ab,i\u0307,12"), "UNICODE_CI", 0); - assertFindInSet("i\u0307o", UTF8String.fromString("ab,İo,12"), "UTF8_BINARY", 0); - assertFindInSet("i\u0307o", UTF8String.fromString("ab,İo,12"), "UTF8_LCASE", 2); - assertFindInSet("i\u0307o", UTF8String.fromString("ab,İo,12"), "UNICODE", 0); - assertFindInSet("i\u0307o", UTF8String.fromString("ab,İo,12"), "UNICODE_CI", 2); - assertFindInSet("İo", UTF8String.fromString("ab,i\u0307o,12"), "UTF8_BINARY", 0); - assertFindInSet("İo", UTF8String.fromString("ab,i\u0307o,12"), "UTF8_LCASE", 2); - assertFindInSet("İo", UTF8String.fromString("ab,i\u0307o,12"), "UNICODE", 0); - assertFindInSet("İo", UTF8String.fromString("ab,i\u0307o,12"), "UNICODE_CI", 2); + assertFindInSet("i\u0307", UTF8String.fromString("İ"), UTF8_BINARY, 0); + assertFindInSet("i\u0307", UTF8String.fromString("İ"), UTF8_LCASE, 1); + assertFindInSet("i\u0307", UTF8String.fromString("İ"), UNICODE, 0); + assertFindInSet("i\u0307", UTF8String.fromString("İ"), UNICODE_CI, 1); + assertFindInSet("i", UTF8String.fromString("İ"), UTF8_BINARY, 0); + assertFindInSet("i", UTF8String.fromString("İ"), UTF8_LCASE, 0); + assertFindInSet("i", UTF8String.fromString("İ"), UNICODE, 0); + assertFindInSet("i", UTF8String.fromString("İ"), UNICODE_CI, 0); + assertFindInSet("i\u0307", UTF8String.fromString("i\u0307"), UTF8_BINARY, 1); + assertFindInSet("i\u0307", UTF8String.fromString("i\u0307"), UTF8_LCASE, 1); + assertFindInSet("i\u0307", UTF8String.fromString("i\u0307"), UNICODE, 1); + assertFindInSet("i\u0307", UTF8String.fromString("i\u0307"), UNICODE_CI, 1); + assertFindInSet("i", UTF8String.fromString("i\u0307"), UTF8_BINARY, 0); + assertFindInSet("i", UTF8String.fromString("i\u0307"), UTF8_LCASE, 0); + assertFindInSet("i", UTF8String.fromString("i\u0307"), UNICODE, 0); + assertFindInSet("i", UTF8String.fromString("i\u0307"), UNICODE_CI, 0); + assertFindInSet("i\u0307", UTF8String.fromString("İ,"), UTF8_BINARY, 0); + assertFindInSet("i\u0307", UTF8String.fromString("İ,"), UTF8_LCASE, 1); + assertFindInSet("i\u0307", UTF8String.fromString("İ,"), UNICODE, 0); + assertFindInSet("i\u0307", UTF8String.fromString("İ,"), UNICODE_CI, 1); + assertFindInSet("i", UTF8String.fromString("İ,"), UTF8_BINARY, 0); + assertFindInSet("i", UTF8String.fromString("İ,"), UTF8_LCASE, 0); + assertFindInSet("i", UTF8String.fromString("İ,"), UNICODE, 0); + assertFindInSet("i", UTF8String.fromString("İ,"), UNICODE_CI, 0); + assertFindInSet("i\u0307", UTF8String.fromString("i\u0307,"), UTF8_BINARY, 1); + assertFindInSet("i\u0307", UTF8String.fromString("i\u0307,"), UTF8_LCASE, 1); + assertFindInSet("i\u0307", UTF8String.fromString("i\u0307,"), UNICODE, 1); + assertFindInSet("i\u0307", UTF8String.fromString("i\u0307,"), UNICODE_CI, 1); + assertFindInSet("i", UTF8String.fromString("i\u0307,"), UTF8_BINARY, 0); + assertFindInSet("i", UTF8String.fromString("i\u0307,"), UTF8_LCASE, 0); + assertFindInSet("i", UTF8String.fromString("i\u0307,"), UNICODE, 0); + assertFindInSet("i", UTF8String.fromString("i\u0307,"), UNICODE_CI, 0); + assertFindInSet("i\u0307", UTF8String.fromString("ab,İ"), UTF8_BINARY, 0); + assertFindInSet("i\u0307", UTF8String.fromString("ab,İ"), UTF8_LCASE, 2); + assertFindInSet("i\u0307", UTF8String.fromString("ab,İ"), UNICODE, 0); + assertFindInSet("i\u0307", UTF8String.fromString("ab,İ"), UNICODE_CI, 2); + assertFindInSet("i", UTF8String.fromString("ab,İ"), UTF8_BINARY, 0); + assertFindInSet("i", UTF8String.fromString("ab,İ"), UTF8_LCASE, 0); + assertFindInSet("i", UTF8String.fromString("ab,İ"), UNICODE, 0); + assertFindInSet("i", UTF8String.fromString("ab,İ"), UNICODE_CI, 0); + assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307"), UTF8_BINARY, 2); + assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307"), UTF8_LCASE, 2); + assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307"), UNICODE, 2); + assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307"), UNICODE_CI, 2); + assertFindInSet("i", UTF8String.fromString("ab,i\u0307"), UTF8_BINARY, 0); + assertFindInSet("i", UTF8String.fromString("ab,i\u0307"), UTF8_LCASE, 0); + assertFindInSet("i", UTF8String.fromString("ab,i\u0307"), UNICODE, 0); + assertFindInSet("i", UTF8String.fromString("ab,i\u0307"), UNICODE_CI, 0); + assertFindInSet("İ", UTF8String.fromString("ab,i\u0307"), UTF8_BINARY, 0); + assertFindInSet("İ", UTF8String.fromString("ab,i\u0307"), UTF8_LCASE, 2); + assertFindInSet("İ", UTF8String.fromString("ab,i\u0307"), UNICODE, 0); + assertFindInSet("İ", UTF8String.fromString("ab,i\u0307"), UNICODE_CI, 2); + assertFindInSet("i\u0307", UTF8String.fromString("ab,İ,12"), UTF8_BINARY, 0); + assertFindInSet("i\u0307", UTF8String.fromString("ab,İ,12"), UTF8_LCASE, 2); + assertFindInSet("i\u0307", UTF8String.fromString("ab,İ,12"), UNICODE, 0); + assertFindInSet("i\u0307", UTF8String.fromString("ab,İ,12"), UNICODE_CI, 2); + assertFindInSet("i", UTF8String.fromString("ab,İ,12"), UTF8_BINARY, 0); + assertFindInSet("i", UTF8String.fromString("ab,İ,12"), UTF8_LCASE, 0); + assertFindInSet("i", UTF8String.fromString("ab,İ,12"), UNICODE, 0); + assertFindInSet("i", UTF8String.fromString("ab,İ,12"), UNICODE_CI, 0); + assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307,12"), UTF8_BINARY, 2); + assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307,12"), UTF8_LCASE, 2); + assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307,12"), UNICODE, 2); + assertFindInSet("i\u0307", UTF8String.fromString("ab,i\u0307,12"), UNICODE_CI, 2); + assertFindInSet("i", UTF8String.fromString("ab,i\u0307,12"), UTF8_BINARY, 0); + assertFindInSet("i", UTF8String.fromString("ab,i\u0307,12"), UTF8_LCASE, 0); + assertFindInSet("i", UTF8String.fromString("ab,i\u0307,12"), UNICODE, 0); + assertFindInSet("i", UTF8String.fromString("ab,i\u0307,12"), UNICODE_CI, 0); + assertFindInSet("i\u0307o", UTF8String.fromString("ab,İo,12"), UTF8_BINARY, 0); + assertFindInSet("i\u0307o", UTF8String.fromString("ab,İo,12"), UTF8_LCASE, 2); + assertFindInSet("i\u0307o", UTF8String.fromString("ab,İo,12"), UNICODE, 0); + assertFindInSet("i\u0307o", UTF8String.fromString("ab,İo,12"), UNICODE_CI, 2); + assertFindInSet("İo", UTF8String.fromString("ab,i\u0307o,12"), UTF8_BINARY, 0); + assertFindInSet("İo", UTF8String.fromString("ab,i\u0307o,12"), UTF8_LCASE, 2); + assertFindInSet("İo", UTF8String.fromString("ab,i\u0307o,12"), UNICODE, 0); + assertFindInSet("İo", UTF8String.fromString("ab,i\u0307o,12"), UNICODE_CI, 2); // Conditional case mapping (e.g. Greek sigmas). - assertFindInSet("σ", UTF8String.fromString("σ"), "UTF8_BINARY", 1); - assertFindInSet("σ", UTF8String.fromString("ς"), "UTF8_BINARY", 0); - assertFindInSet("σ", UTF8String.fromString("Σ"), "UTF8_BINARY", 0); - assertFindInSet("ς", UTF8String.fromString("σ"), "UTF8_BINARY", 0); - assertFindInSet("ς", UTF8String.fromString("ς"), "UTF8_BINARY", 1); - assertFindInSet("ς", UTF8String.fromString("Σ"), "UTF8_BINARY", 0); - assertFindInSet("Σ", UTF8String.fromString("σ"), "UTF8_BINARY", 0); - assertFindInSet("Σ", UTF8String.fromString("ς"), "UTF8_BINARY", 0); - assertFindInSet("Σ", UTF8String.fromString("Σ"), "UTF8_BINARY", 1); - assertFindInSet("σ", UTF8String.fromString("σ"), "UTF8_LCASE", 1); - assertFindInSet("σ", UTF8String.fromString("ς"), "UTF8_LCASE", 1); - assertFindInSet("σ", UTF8String.fromString("Σ"), "UTF8_LCASE", 1); - assertFindInSet("ς", UTF8String.fromString("σ"), "UTF8_LCASE", 1); - assertFindInSet("ς", UTF8String.fromString("ς"), "UTF8_LCASE", 1); - assertFindInSet("ς", UTF8String.fromString("Σ"), "UTF8_LCASE", 1); - assertFindInSet("Σ", UTF8String.fromString("σ"), "UTF8_LCASE", 1); - assertFindInSet("Σ", UTF8String.fromString("ς"), "UTF8_LCASE", 1); - assertFindInSet("Σ", UTF8String.fromString("Σ"), "UTF8_LCASE", 1); - assertFindInSet("σ", UTF8String.fromString("σ"), "UNICODE", 1); - assertFindInSet("σ", UTF8String.fromString("ς"), "UNICODE", 0); - assertFindInSet("σ", UTF8String.fromString("Σ"), "UNICODE", 0); - assertFindInSet("ς", UTF8String.fromString("σ"), "UNICODE", 0); - assertFindInSet("ς", UTF8String.fromString("ς"), "UNICODE", 1); - assertFindInSet("ς", UTF8String.fromString("Σ"), "UNICODE", 0); - assertFindInSet("Σ", UTF8String.fromString("σ"), "UNICODE", 0); - assertFindInSet("Σ", UTF8String.fromString("ς"), "UNICODE", 0); - assertFindInSet("Σ", UTF8String.fromString("Σ"), "UNICODE", 1); - assertFindInSet("σ", UTF8String.fromString("σ"), "UNICODE_CI", 1); - assertFindInSet("σ", UTF8String.fromString("ς"), "UNICODE_CI", 1); - assertFindInSet("σ", UTF8String.fromString("Σ"), "UNICODE_CI", 1); - assertFindInSet("ς", UTF8String.fromString("σ"), "UNICODE_CI", 1); - assertFindInSet("ς", UTF8String.fromString("ς"), "UNICODE_CI", 1); - assertFindInSet("ς", UTF8String.fromString("Σ"), "UNICODE_CI", 1); - assertFindInSet("Σ", UTF8String.fromString("σ"), "UNICODE_CI", 1); - assertFindInSet("Σ", UTF8String.fromString("ς"), "UNICODE_CI", 1); - assertFindInSet("Σ", UTF8String.fromString("Σ"), "UNICODE_CI", 1); + assertFindInSet("σ", UTF8String.fromString("σ"), UTF8_BINARY, 1); + assertFindInSet("σ", UTF8String.fromString("ς"), UTF8_BINARY, 0); + assertFindInSet("σ", UTF8String.fromString("Σ"), UTF8_BINARY, 0); + assertFindInSet("ς", UTF8String.fromString("σ"), UTF8_BINARY, 0); + assertFindInSet("ς", UTF8String.fromString("ς"), UTF8_BINARY, 1); + assertFindInSet("ς", UTF8String.fromString("Σ"), UTF8_BINARY, 0); + assertFindInSet("Σ", UTF8String.fromString("σ"), UTF8_BINARY, 0); + assertFindInSet("Σ", UTF8String.fromString("ς"), UTF8_BINARY, 0); + assertFindInSet("Σ", UTF8String.fromString("Σ"), UTF8_BINARY, 1); + assertFindInSet("σ", UTF8String.fromString("σ"), UTF8_LCASE, 1); + assertFindInSet("σ", UTF8String.fromString("ς"), UTF8_LCASE, 1); + assertFindInSet("σ", UTF8String.fromString("Σ"), UTF8_LCASE, 1); + assertFindInSet("ς", UTF8String.fromString("σ"), UTF8_LCASE, 1); + assertFindInSet("ς", UTF8String.fromString("ς"), UTF8_LCASE, 1); + assertFindInSet("ς", UTF8String.fromString("Σ"), UTF8_LCASE, 1); + assertFindInSet("Σ", UTF8String.fromString("σ"), UTF8_LCASE, 1); + assertFindInSet("Σ", UTF8String.fromString("ς"), UTF8_LCASE, 1); + assertFindInSet("Σ", UTF8String.fromString("Σ"), UTF8_LCASE, 1); + assertFindInSet("σ", UTF8String.fromString("σ"), UNICODE, 1); + assertFindInSet("σ", UTF8String.fromString("ς"), UNICODE, 0); + assertFindInSet("σ", UTF8String.fromString("Σ"), UNICODE, 0); + assertFindInSet("ς", UTF8String.fromString("σ"), UNICODE, 0); + assertFindInSet("ς", UTF8String.fromString("ς"), UNICODE, 1); + assertFindInSet("ς", UTF8String.fromString("Σ"), UNICODE, 0); + assertFindInSet("Σ", UTF8String.fromString("σ"), UNICODE, 0); + assertFindInSet("Σ", UTF8String.fromString("ς"), UNICODE, 0); + assertFindInSet("Σ", UTF8String.fromString("Σ"), UNICODE, 1); + assertFindInSet("σ", UTF8String.fromString("σ"), UNICODE_CI, 1); + assertFindInSet("σ", UTF8String.fromString("ς"), UNICODE_CI, 1); + assertFindInSet("σ", UTF8String.fromString("Σ"), UNICODE_CI, 1); + assertFindInSet("ς", UTF8String.fromString("σ"), UNICODE_CI, 1); + assertFindInSet("ς", UTF8String.fromString("ς"), UNICODE_CI, 1); + assertFindInSet("ς", UTF8String.fromString("Σ"), UNICODE_CI, 1); + assertFindInSet("Σ", UTF8String.fromString("σ"), UNICODE_CI, 1); + assertFindInSet("Σ", UTF8String.fromString("ς"), UNICODE_CI, 1); + assertFindInSet("Σ", UTF8String.fromString("Σ"), UNICODE_CI, 1); // Surrogate pairs. - assertFindInSet("a", UTF8String.fromString("a🙃,b,🙃c"), "UTF8_BINARY", 0); - assertFindInSet("a", UTF8String.fromString("a🙃,b,🙃c"), "UTF8_LCASE", 0); - assertFindInSet("a", UTF8String.fromString("a🙃,b,🙃c"), "UNICODE", 0); - assertFindInSet("a", UTF8String.fromString("a🙃,b,🙃c"), "UNICODE_CI", 0); - assertFindInSet("a🙃", UTF8String.fromString("a🙃,b,🙃c"), "UTF8_BINARY", 1); - assertFindInSet("a🙃", UTF8String.fromString("a🙃,b,🙃c"), "UTF8_LCASE", 1); - assertFindInSet("a🙃", UTF8String.fromString("a🙃,b,🙃c"), "UNICODE", 1); - assertFindInSet("a🙃", UTF8String.fromString("a🙃,b,🙃c"), "UNICODE_CI", 1); - assertFindInSet("b", UTF8String.fromString("a🙃,b,🙃c"), "UTF8_BINARY", 2); - assertFindInSet("b", UTF8String.fromString("a🙃,b,🙃c"), "UTF8_LCASE", 2); - assertFindInSet("b", UTF8String.fromString("a🙃,b,🙃c"), "UNICODE", 2); - assertFindInSet("b", UTF8String.fromString("a🙃,b,🙃c"), "UNICODE_CI", 2); - assertFindInSet("🙃c", UTF8String.fromString("a🙃,b,🙃c"), "UTF8_BINARY", 3); - assertFindInSet("🙃c", UTF8String.fromString("a🙃,b,🙃c"), "UTF8_LCASE", 3); - assertFindInSet("🙃c", UTF8String.fromString("a🙃,b,🙃c"), "UNICODE", 3); - assertFindInSet("🙃c", UTF8String.fromString("a🙃,b,🙃c"), "UNICODE_CI", 3); - assertFindInSet("😄😆", UTF8String.fromString("😀😆,😃😄"), "UTF8_BINARY", 0); - assertFindInSet("😄😆", UTF8String.fromString("😀😆,😃😄"), "UTF8_LCASE", 0); - assertFindInSet("😄😆", UTF8String.fromString("😀😆,😃😄"), "UNICODE", 0); - assertFindInSet("😄😆", UTF8String.fromString("😀😆,😃😄"), "UNICODE_CI", 0); - assertFindInSet("😀😆", UTF8String.fromString("😀😆,😃😄"), "UTF8_BINARY", 1); - assertFindInSet("😀😆", UTF8String.fromString("😀😆,😃😄"), "UTF8_LCASE", 1); - assertFindInSet("😀😆", UTF8String.fromString("😀😆,😃😄"), "UNICODE", 1); - assertFindInSet("😀😆", UTF8String.fromString("😀😆,😃😄"), "UNICODE_CI", 1); - assertFindInSet("😃😄", UTF8String.fromString("😀😆,😃😄"), "UTF8_BINARY", 2); - assertFindInSet("😃😄", UTF8String.fromString("😀😆,😃😄"), "UTF8_LCASE", 2); - assertFindInSet("😃😄", UTF8String.fromString("😀😆,😃😄"), "UNICODE", 2); - assertFindInSet("😃😄", UTF8String.fromString("😀😆,😃😄"), "UNICODE_CI", 2); - assertFindInSet("x", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_BINARY", 0); - assertFindInSet("x", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_LCASE", 0); - assertFindInSet("x", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE", 0); - assertFindInSet("x", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE_CI", 0); - assertFindInSet("a", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_BINARY", 1); - assertFindInSet("a", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_LCASE", 1); - assertFindInSet("a", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE", 1); - assertFindInSet("a", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE_CI", 1); - assertFindInSet("A", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_BINARY", 0); - assertFindInSet("A", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_LCASE", 1); - assertFindInSet("A", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE", 0); - assertFindInSet("A", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE_CI", 1); - assertFindInSet("𝔸", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_BINARY", 3); - assertFindInSet("𝔸", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_LCASE", 3); - assertFindInSet("𝔸", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE", 3); - assertFindInSet("𝔸", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE_CI", 1); - assertFindInSet("𐐅", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_BINARY", 2); - assertFindInSet("𐐅", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_LCASE", 2); - assertFindInSet("𐐅", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE", 2); - assertFindInSet("𐐅", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE_CI", 2); - assertFindInSet("𐐭", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_BINARY", 0); - assertFindInSet("𐐭", UTF8String.fromString("a,𐐅,𝔸"), "UTF8_LCASE", 2); - assertFindInSet("𐐭", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE", 0); - assertFindInSet("𐐭", UTF8String.fromString("a,𐐅,𝔸"), "UNICODE_CI", 2); + assertFindInSet("a", UTF8String.fromString("a🙃,b,🙃c"), UTF8_BINARY, 0); + assertFindInSet("a", UTF8String.fromString("a🙃,b,🙃c"), UTF8_LCASE, 0); + assertFindInSet("a", UTF8String.fromString("a🙃,b,🙃c"), UNICODE, 0); + assertFindInSet("a", UTF8String.fromString("a🙃,b,🙃c"), UNICODE_CI, 0); + assertFindInSet("a🙃", UTF8String.fromString("a🙃,b,🙃c"), UTF8_BINARY, 1); + assertFindInSet("a🙃", UTF8String.fromString("a🙃,b,🙃c"), UTF8_LCASE, 1); + assertFindInSet("a🙃", UTF8String.fromString("a🙃,b,🙃c"), UNICODE, 1); + assertFindInSet("a🙃", UTF8String.fromString("a🙃,b,🙃c"), UNICODE_CI, 1); + assertFindInSet("b", UTF8String.fromString("a🙃,b,🙃c"), UTF8_BINARY, 2); + assertFindInSet("b", UTF8String.fromString("a🙃,b,🙃c"), UTF8_LCASE, 2); + assertFindInSet("b", UTF8String.fromString("a🙃,b,🙃c"), UNICODE, 2); + assertFindInSet("b", UTF8String.fromString("a🙃,b,🙃c"), UNICODE_CI, 2); + assertFindInSet("🙃c", UTF8String.fromString("a🙃,b,🙃c"), UTF8_BINARY, 3); + assertFindInSet("🙃c", UTF8String.fromString("a🙃,b,🙃c"), UTF8_LCASE, 3); + assertFindInSet("🙃c", UTF8String.fromString("a🙃,b,🙃c"), UNICODE, 3); + assertFindInSet("🙃c", UTF8String.fromString("a🙃,b,🙃c"), UNICODE_CI, 3); + assertFindInSet("😄😆", UTF8String.fromString("😀😆,😃😄"), UTF8_BINARY, 0); + assertFindInSet("😄😆", UTF8String.fromString("😀😆,😃😄"), UTF8_LCASE, 0); + assertFindInSet("😄😆", UTF8String.fromString("😀😆,😃😄"), UNICODE, 0); + assertFindInSet("😄😆", UTF8String.fromString("😀😆,😃😄"), UNICODE_CI, 0); + assertFindInSet("😀😆", UTF8String.fromString("😀😆,😃😄"), UTF8_BINARY, 1); + assertFindInSet("😀😆", UTF8String.fromString("😀😆,😃😄"), UTF8_LCASE, 1); + assertFindInSet("😀😆", UTF8String.fromString("😀😆,😃😄"), UNICODE, 1); + assertFindInSet("😀😆", UTF8String.fromString("😀😆,😃😄"), UNICODE_CI, 1); + assertFindInSet("😃😄", UTF8String.fromString("😀😆,😃😄"), UTF8_BINARY, 2); + assertFindInSet("😃😄", UTF8String.fromString("😀😆,😃😄"), UTF8_LCASE, 2); + assertFindInSet("😃😄", UTF8String.fromString("😀😆,😃😄"), UNICODE, 2); + assertFindInSet("😃😄", UTF8String.fromString("😀😆,😃😄"), UNICODE_CI, 2); + assertFindInSet("x", UTF8String.fromString("a,𐐅,𝔸"), UTF8_BINARY, 0); + assertFindInSet("x", UTF8String.fromString("a,𐐅,𝔸"), UTF8_LCASE, 0); + assertFindInSet("x", UTF8String.fromString("a,𐐅,𝔸"), UNICODE, 0); + assertFindInSet("x", UTF8String.fromString("a,𐐅,𝔸"), UNICODE_CI, 0); + assertFindInSet("a", UTF8String.fromString("a,𐐅,𝔸"), UTF8_BINARY, 1); + assertFindInSet("a", UTF8String.fromString("a,𐐅,𝔸"), UTF8_LCASE, 1); + assertFindInSet("a", UTF8String.fromString("a,𐐅,𝔸"), UNICODE, 1); + assertFindInSet("a", UTF8String.fromString("a,𐐅,𝔸"), UNICODE_CI, 1); + assertFindInSet("A", UTF8String.fromString("a,𐐅,𝔸"), UTF8_BINARY, 0); + assertFindInSet("A", UTF8String.fromString("a,𐐅,𝔸"), UTF8_LCASE, 1); + assertFindInSet("A", UTF8String.fromString("a,𐐅,𝔸"), UNICODE, 0); + assertFindInSet("A", UTF8String.fromString("a,𐐅,𝔸"), UNICODE_CI, 1); + assertFindInSet("𝔸", UTF8String.fromString("a,𐐅,𝔸"), UTF8_BINARY, 3); + assertFindInSet("𝔸", UTF8String.fromString("a,𐐅,𝔸"), UTF8_LCASE, 3); + assertFindInSet("𝔸", UTF8String.fromString("a,𐐅,𝔸"), UNICODE, 3); + assertFindInSet("𝔸", UTF8String.fromString("a,𐐅,𝔸"), UNICODE_CI, 1); + assertFindInSet("𐐅", UTF8String.fromString("a,𐐅,𝔸"), UTF8_BINARY, 2); + assertFindInSet("𐐅", UTF8String.fromString("a,𐐅,𝔸"), UTF8_LCASE, 2); + assertFindInSet("𐐅", UTF8String.fromString("a,𐐅,𝔸"), UNICODE, 2); + assertFindInSet("𐐅", UTF8String.fromString("a,𐐅,𝔸"), UNICODE_CI, 2); + assertFindInSet("𐐭", UTF8String.fromString("a,𐐅,𝔸"), UTF8_BINARY, 0); + assertFindInSet("𐐭", UTF8String.fromString("a,𐐅,𝔸"), UTF8_LCASE, 2); + assertFindInSet("𐐭", UTF8String.fromString("a,𐐅,𝔸"), UNICODE, 0); + assertFindInSet("𐐭", UTF8String.fromString("a,𐐅,𝔸"), UNICODE_CI, 2); // Invalid UTF8 strings assertFindInSet("C", UTF8String.fromBytes( new byte[] { 0x41, (byte) 0xC2, 0x2C, 0x42, 0x2C, 0x43, 0x2C, 0x43, 0x2C, 0x56 }), - "UTF8_BINARY", 3); + UTF8_BINARY, 3); assertFindInSet("c", UTF8String.fromBytes( new byte[] { 0x41, (byte) 0xC2, 0x2C, 0x42, 0x2C, 0x43, 0x2C, 0x43, 0x2C, 0x56 }), - "UTF8_LCASE", 2); + UTF8_LCASE, 2); assertFindInSet("C", UTF8String.fromBytes( new byte[] { 0x41, (byte) 0xC2, 0x2C, 0x42, 0x2C, 0x43, 0x2C, 0x43, 0x2C, 0x56 }), - "UNICODE", 2); + UNICODE, 2); assertFindInSet("c", UTF8String.fromBytes( new byte[] { 0x41, (byte) 0xC2, 0x2C, 0x42, 0x2C, 0x43, 0x2C, 0x43, 0x2C, 0x56 }), - "UNICODE_CI", 2); + UNICODE_CI, 2); } /** @@ -1986,145 +1987,145 @@ private void assertStringReplace(String source, String search, String replace, @Test public void testStringReplace() throws SparkException { // Empty strings. - assertStringReplace("", "", "", "UTF8_BINARY", ""); - assertStringReplace("", "", "", "UTF8_LCASE", ""); - assertStringReplace("", "", "", "UNICODE", ""); - assertStringReplace("", "", "", "UNICODE_CI", ""); - assertStringReplace("abc", "", "", "UTF8_BINARY", "abc"); - assertStringReplace("abc", "", "", "UTF8_LCASE", "abc"); - assertStringReplace("abc", "", "", "UNICODE", "abc"); - assertStringReplace("abc", "", "", "UNICODE_CI", "abc"); - assertStringReplace("", "x", "", "UTF8_BINARY", ""); - assertStringReplace("", "x", "", "UTF8_LCASE", ""); - assertStringReplace("", "x", "", "UNICODE", ""); - assertStringReplace("", "x", "", "UNICODE_CI", ""); - assertStringReplace("", "", "x", "UTF8_BINARY", ""); - assertStringReplace("", "", "x", "UTF8_LCASE", ""); - assertStringReplace("", "", "x", "UNICODE", ""); - assertStringReplace("", "", "x", "UNICODE_CI", ""); - assertStringReplace("", "b", "x", "UTF8_BINARY", ""); - assertStringReplace("", "b", "x", "UTF8_LCASE", ""); - assertStringReplace("", "b", "x", "UNICODE", ""); - assertStringReplace("", "b", "x", "UNICODE_CI", ""); - assertStringReplace("abc", "b", "", "UTF8_BINARY", "ac"); - assertStringReplace("abc", "b", "", "UTF8_LCASE", "ac"); - assertStringReplace("abc", "b", "", "UNICODE", "ac"); - assertStringReplace("abc", "b", "", "UNICODE_CI", "ac"); - assertStringReplace("abc", "", "x", "UTF8_BINARY", "abc"); - assertStringReplace("abc", "", "x", "UTF8_LCASE", "abc"); - assertStringReplace("abc", "", "x", "UNICODE", "abc"); - assertStringReplace("abc", "", "x", "UNICODE_CI", "abc"); + assertStringReplace("", "", "", UTF8_BINARY, ""); + assertStringReplace("", "", "", UTF8_LCASE, ""); + assertStringReplace("", "", "", UNICODE, ""); + assertStringReplace("", "", "", UNICODE_CI, ""); + assertStringReplace("abc", "", "", UTF8_BINARY, "abc"); + assertStringReplace("abc", "", "", UTF8_LCASE, "abc"); + assertStringReplace("abc", "", "", UNICODE, "abc"); + assertStringReplace("abc", "", "", UNICODE_CI, "abc"); + assertStringReplace("", "x", "", UTF8_BINARY, ""); + assertStringReplace("", "x", "", UTF8_LCASE, ""); + assertStringReplace("", "x", "", UNICODE, ""); + assertStringReplace("", "x", "", UNICODE_CI, ""); + assertStringReplace("", "", "x", UTF8_BINARY, ""); + assertStringReplace("", "", "x", UTF8_LCASE, ""); + assertStringReplace("", "", "x", UNICODE, ""); + assertStringReplace("", "", "x", UNICODE_CI, ""); + assertStringReplace("", "b", "x", UTF8_BINARY, ""); + assertStringReplace("", "b", "x", UTF8_LCASE, ""); + assertStringReplace("", "b", "x", UNICODE, ""); + assertStringReplace("", "b", "x", UNICODE_CI, ""); + assertStringReplace("abc", "b", "", UTF8_BINARY, "ac"); + assertStringReplace("abc", "b", "", UTF8_LCASE, "ac"); + assertStringReplace("abc", "b", "", UNICODE, "ac"); + assertStringReplace("abc", "b", "", UNICODE_CI, "ac"); + assertStringReplace("abc", "", "x", UTF8_BINARY, "abc"); + assertStringReplace("abc", "", "x", UTF8_LCASE, "abc"); + assertStringReplace("abc", "", "x", UNICODE, "abc"); + assertStringReplace("abc", "", "x", UNICODE_CI, "abc"); // Basic tests. - assertStringReplace("replace", "pl", "", "UTF8_BINARY", "reace"); - assertStringReplace("replace", "pl", "", "UTF8_LCASE", "reace"); - assertStringReplace("replace", "pl", "", "UNICODE", "reace"); - assertStringReplace("replace", "pl", "", "UNICODE_CI", "reace"); - assertStringReplace("replace", "", "123", "UTF8_BINARY", "replace"); - assertStringReplace("replace", "", "123", "UTF8_LCASE", "replace"); - assertStringReplace("replace", "", "123", "UNICODE", "replace"); - assertStringReplace("replace", "", "123", "UNICODE_CI", "replace"); - assertStringReplace("abcabc", "b", "12", "UTF8_BINARY", "a12ca12c"); - assertStringReplace("abcabc", "b", "12", "UTF8_LCASE", "a12ca12c"); - assertStringReplace("abcabc", "b", "12", "UNICODE", "a12ca12c"); - assertStringReplace("abcabc", "b", "12", "UNICODE_CI", "a12ca12c"); - assertStringReplace("replace", "plx", "123", "UTF8_BINARY", "replace"); - assertStringReplace("replace", "plx", "123", "UTF8_LCASE", "replace"); - assertStringReplace("replace", "plx", "123", "UNICODE", "replace"); - assertStringReplace("replace", "plx", "123", "UNICODE_CI", "replace"); - assertStringReplace("Replace", "re", "", "UTF8_BINARY", "Replace"); - assertStringReplace("Replace", "re", "", "UTF8_LCASE", "place"); - assertStringReplace("Replace", "re", "", "UNICODE", "Replace"); - assertStringReplace("Replace", "re", "", "UNICODE_CI", "place"); - assertStringReplace("abcdabcd", "Bc", "", "UTF8_BINARY", "abcdabcd"); - assertStringReplace("abcdabcd", "Bc", "", "UTF8_LCASE", "adad"); - assertStringReplace("abcdabcd", "Bc", "", "UNICODE", "abcdabcd"); - assertStringReplace("abcdabcd", "Bc", "", "UNICODE_CI", "adad"); - assertStringReplace("AbcdabCd", "Bc", "", "UTF8_BINARY", "AbcdabCd"); - assertStringReplace("AbcdabCd", "Bc", "", "UTF8_LCASE", "Adad"); - assertStringReplace("AbcdabCd", "Bc", "", "UNICODE", "AbcdabCd"); - assertStringReplace("AbcdabCd", "Bc", "", "UNICODE_CI", "Adad"); + assertStringReplace("replace", "pl", "", UTF8_BINARY, "reace"); + assertStringReplace("replace", "pl", "", UTF8_LCASE, "reace"); + assertStringReplace("replace", "pl", "", UNICODE, "reace"); + assertStringReplace("replace", "pl", "", UNICODE_CI, "reace"); + assertStringReplace("replace", "", "123", UTF8_BINARY, "replace"); + assertStringReplace("replace", "", "123", UTF8_LCASE, "replace"); + assertStringReplace("replace", "", "123", UNICODE, "replace"); + assertStringReplace("replace", "", "123", UNICODE_CI, "replace"); + assertStringReplace("abcabc", "b", "12", UTF8_BINARY, "a12ca12c"); + assertStringReplace("abcabc", "b", "12", UTF8_LCASE, "a12ca12c"); + assertStringReplace("abcabc", "b", "12", UNICODE, "a12ca12c"); + assertStringReplace("abcabc", "b", "12", UNICODE_CI, "a12ca12c"); + assertStringReplace("replace", "plx", "123", UTF8_BINARY, "replace"); + assertStringReplace("replace", "plx", "123", UTF8_LCASE, "replace"); + assertStringReplace("replace", "plx", "123", UNICODE, "replace"); + assertStringReplace("replace", "plx", "123", UNICODE_CI, "replace"); + assertStringReplace("Replace", "re", "", UTF8_BINARY, "Replace"); + assertStringReplace("Replace", "re", "", UTF8_LCASE, "place"); + assertStringReplace("Replace", "re", "", UNICODE, "Replace"); + assertStringReplace("Replace", "re", "", UNICODE_CI, "place"); + assertStringReplace("abcdabcd", "Bc", "", UTF8_BINARY, "abcdabcd"); + assertStringReplace("abcdabcd", "Bc", "", UTF8_LCASE, "adad"); + assertStringReplace("abcdabcd", "Bc", "", UNICODE, "abcdabcd"); + assertStringReplace("abcdabcd", "Bc", "", UNICODE_CI, "adad"); + assertStringReplace("AbcdabCd", "Bc", "", UTF8_BINARY, "AbcdabCd"); + assertStringReplace("AbcdabCd", "Bc", "", UTF8_LCASE, "Adad"); + assertStringReplace("AbcdabCd", "Bc", "", UNICODE, "AbcdabCd"); + assertStringReplace("AbcdabCd", "Bc", "", UNICODE_CI, "Adad"); // Advanced tests. - assertStringReplace("abcdabcd", "bc", "", "UTF8_BINARY", "adad"); - assertStringReplace("r世eplace", "pl", "123", "UTF8_BINARY", "r世e123ace"); - assertStringReplace("世Replace", "re", "", "UTF8_BINARY", "世Replace"); - assertStringReplace("r世eplace", "pl", "xx", "UTF8_LCASE", "r世exxace"); - assertStringReplace("repl世ace", "PL", "AB", "UTF8_LCASE", "reAB世ace"); - assertStringReplace("re世place", "世", "x", "UTF8_LCASE", "rexplace"); - assertStringReplace("re世place", "plx", "123", "UNICODE", "re世place"); - assertStringReplace("replace世", "", "123", "UNICODE", "replace世"); - assertStringReplace("aBc世abc", "b", "12", "UNICODE", "aBc世a12c"); - assertStringReplace("aBc世abc", "b", "12", "UNICODE_CI", "a12c世a12c"); - assertStringReplace("a世Bcdabcd", "bC", "", "UNICODE_CI", "a世dad"); - assertStringReplace("repl世ace", "Pl", "", "UNICODE_CI", "re世ace"); + assertStringReplace("abcdabcd", "bc", "", UTF8_BINARY, "adad"); + assertStringReplace("r世eplace", "pl", "123", UTF8_BINARY, "r世e123ace"); + assertStringReplace("世Replace", "re", "", UTF8_BINARY, "世Replace"); + assertStringReplace("r世eplace", "pl", "xx", UTF8_LCASE, "r世exxace"); + assertStringReplace("repl世ace", "PL", "AB", UTF8_LCASE, "reAB世ace"); + assertStringReplace("re世place", "世", "x", UTF8_LCASE, "rexplace"); + assertStringReplace("re世place", "plx", "123", UNICODE, "re世place"); + assertStringReplace("replace世", "", "123", UNICODE, "replace世"); + assertStringReplace("aBc世abc", "b", "12", UNICODE, "aBc世a12c"); + assertStringReplace("aBc世abc", "b", "12", UNICODE_CI, "a12c世a12c"); + assertStringReplace("a世Bcdabcd", "bC", "", UNICODE_CI, "a世dad"); + assertStringReplace("repl世ace", "Pl", "", UNICODE_CI, "re世ace"); assertStringReplace("abcčšdabĆŠscd", "cs", "", "SR_CI_AI", "abcdabscd"); // One-to-many case mapping (e.g. Turkish dotted I). - assertStringReplace("abi̇12", "i", "X", "UNICODE_CI", "abi̇12"); - assertStringReplace("abi̇12", "\u0307", "X", "UNICODE_CI", "abi̇12"); - assertStringReplace("abi̇12", "İ", "X", "UNICODE_CI", "abX12"); - assertStringReplace("abİ12", "i", "X", "UNICODE_CI", "abİ12"); - assertStringReplace("İi̇İi̇İi̇", "i\u0307", "x", "UNICODE_CI", "xxxxxx"); - assertStringReplace("İi̇İi̇İi̇", "i", "x", "UNICODE_CI", "İi̇İi̇İi̇"); - assertStringReplace("abİo12i̇o", "i\u0307o", "xx", "UNICODE_CI", "abxx12xx"); - assertStringReplace("abi̇o12i̇o", "İo", "yy", "UNICODE_CI", "abyy12yy"); - assertStringReplace("abi̇12", "i", "X", "UTF8_LCASE", "abX\u030712"); // != UNICODE_CI - assertStringReplace("abi̇12", "\u0307", "X", "UTF8_LCASE", "abiX12"); // != UNICODE_CI - assertStringReplace("abi̇12", "İ", "X", "UTF8_LCASE", "abX12"); - assertStringReplace("abİ12", "i", "X", "UTF8_LCASE", "abİ12"); - assertStringReplace("İi̇İi̇İi̇", "i\u0307", "x", "UTF8_LCASE", "xxxxxx"); - assertStringReplace("İi̇İi̇İi̇", "i", "x", "UTF8_LCASE", + assertStringReplace("abi̇12", "i", "X", UNICODE_CI, "abi̇12"); + assertStringReplace("abi̇12", "\u0307", "X", UNICODE_CI, "abi̇12"); + assertStringReplace("abi̇12", "İ", "X", UNICODE_CI, "abX12"); + assertStringReplace("abİ12", "i", "X", UNICODE_CI, "abİ12"); + assertStringReplace("İi̇İi̇İi̇", "i\u0307", "x", UNICODE_CI, "xxxxxx"); + assertStringReplace("İi̇İi̇İi̇", "i", "x", UNICODE_CI, "İi̇İi̇İi̇"); + assertStringReplace("abİo12i̇o", "i\u0307o", "xx", UNICODE_CI, "abxx12xx"); + assertStringReplace("abi̇o12i̇o", "İo", "yy", UNICODE_CI, "abyy12yy"); + assertStringReplace("abi̇12", "i", "X", UTF8_LCASE, "abX\u030712"); // != UNICODE_CI + assertStringReplace("abi̇12", "\u0307", "X", UTF8_LCASE, "abiX12"); // != UNICODE_CI + assertStringReplace("abi̇12", "İ", "X", UTF8_LCASE, "abX12"); + assertStringReplace("abİ12", "i", "X", UTF8_LCASE, "abİ12"); + assertStringReplace("İi̇İi̇İi̇", "i\u0307", "x", UTF8_LCASE, "xxxxxx"); + assertStringReplace("İi̇İi̇İi̇", "i", "x", UTF8_LCASE, "İx\u0307İx\u0307İx\u0307"); // != UNICODE_CI - assertStringReplace("abİo12i̇o", "i\u0307o", "xx", "UTF8_LCASE", "abxx12xx"); - assertStringReplace("abi̇o12i̇o", "İo", "yy", "UTF8_LCASE", "abyy12yy"); + assertStringReplace("abİo12i̇o", "i\u0307o", "xx", UTF8_LCASE, "abxx12xx"); + assertStringReplace("abi̇o12i̇o", "İo", "yy", UTF8_LCASE, "abyy12yy"); // Conditional case mapping (e.g. Greek sigmas). - assertStringReplace("σ", "σ", "x", "UTF8_BINARY", "x"); - assertStringReplace("σ", "ς", "x", "UTF8_BINARY", "σ"); - assertStringReplace("σ", "Σ", "x", "UTF8_BINARY", "σ"); - assertStringReplace("ς", "σ", "x", "UTF8_BINARY", "ς"); - assertStringReplace("ς", "ς", "x", "UTF8_BINARY", "x"); - assertStringReplace("ς", "Σ", "x", "UTF8_BINARY", "ς"); - assertStringReplace("Σ", "σ", "x", "UTF8_BINARY", "Σ"); - assertStringReplace("Σ", "ς", "x", "UTF8_BINARY", "Σ"); - assertStringReplace("Σ", "Σ", "x", "UTF8_BINARY", "x"); - assertStringReplace("σ", "σ", "x", "UTF8_LCASE", "x"); - assertStringReplace("σ", "ς", "x", "UTF8_LCASE", "x"); - assertStringReplace("σ", "Σ", "x", "UTF8_LCASE", "x"); - assertStringReplace("ς", "σ", "x", "UTF8_LCASE", "x"); - assertStringReplace("ς", "ς", "x", "UTF8_LCASE", "x"); - assertStringReplace("ς", "Σ", "x", "UTF8_LCASE", "x"); - assertStringReplace("Σ", "σ", "x", "UTF8_LCASE", "x"); - assertStringReplace("Σ", "ς", "x", "UTF8_LCASE", "x"); - assertStringReplace("Σ", "Σ", "x", "UTF8_LCASE", "x"); - assertStringReplace("σ", "σ", "x", "UNICODE", "x"); - assertStringReplace("σ", "ς", "x", "UNICODE", "σ"); - assertStringReplace("σ", "Σ", "x", "UNICODE", "σ"); - assertStringReplace("ς", "σ", "x", "UNICODE", "ς"); - assertStringReplace("ς", "ς", "x", "UNICODE", "x"); - assertStringReplace("ς", "Σ", "x", "UNICODE", "ς"); - assertStringReplace("Σ", "σ", "x", "UNICODE", "Σ"); - assertStringReplace("Σ", "ς", "x", "UNICODE", "Σ"); - assertStringReplace("Σ", "Σ", "x", "UNICODE", "x"); - assertStringReplace("σ", "σ", "x", "UNICODE_CI", "x"); - assertStringReplace("σ", "ς", "x", "UNICODE_CI", "x"); - assertStringReplace("σ", "Σ", "x", "UNICODE_CI", "x"); - assertStringReplace("ς", "σ", "x", "UNICODE_CI", "x"); - assertStringReplace("ς", "ς", "x", "UNICODE_CI", "x"); - assertStringReplace("ς", "Σ", "x", "UNICODE_CI", "x"); - assertStringReplace("Σ", "σ", "x", "UNICODE_CI", "x"); - assertStringReplace("Σ", "ς", "x", "UNICODE_CI", "x"); - assertStringReplace("Σ", "Σ", "x", "UNICODE_CI", "x"); + assertStringReplace("σ", "σ", "x", UTF8_BINARY, "x"); + assertStringReplace("σ", "ς", "x", UTF8_BINARY, "σ"); + assertStringReplace("σ", "Σ", "x", UTF8_BINARY, "σ"); + assertStringReplace("ς", "σ", "x", UTF8_BINARY, "ς"); + assertStringReplace("ς", "ς", "x", UTF8_BINARY, "x"); + assertStringReplace("ς", "Σ", "x", UTF8_BINARY, "ς"); + assertStringReplace("Σ", "σ", "x", UTF8_BINARY, "Σ"); + assertStringReplace("Σ", "ς", "x", UTF8_BINARY, "Σ"); + assertStringReplace("Σ", "Σ", "x", UTF8_BINARY, "x"); + assertStringReplace("σ", "σ", "x", UTF8_LCASE, "x"); + assertStringReplace("σ", "ς", "x", UTF8_LCASE, "x"); + assertStringReplace("σ", "Σ", "x", UTF8_LCASE, "x"); + assertStringReplace("ς", "σ", "x", UTF8_LCASE, "x"); + assertStringReplace("ς", "ς", "x", UTF8_LCASE, "x"); + assertStringReplace("ς", "Σ", "x", UTF8_LCASE, "x"); + assertStringReplace("Σ", "σ", "x", UTF8_LCASE, "x"); + assertStringReplace("Σ", "ς", "x", UTF8_LCASE, "x"); + assertStringReplace("Σ", "Σ", "x", UTF8_LCASE, "x"); + assertStringReplace("σ", "σ", "x", UNICODE, "x"); + assertStringReplace("σ", "ς", "x", UNICODE, "σ"); + assertStringReplace("σ", "Σ", "x", UNICODE, "σ"); + assertStringReplace("ς", "σ", "x", UNICODE, "ς"); + assertStringReplace("ς", "ς", "x", UNICODE, "x"); + assertStringReplace("ς", "Σ", "x", UNICODE, "ς"); + assertStringReplace("Σ", "σ", "x", UNICODE, "Σ"); + assertStringReplace("Σ", "ς", "x", UNICODE, "Σ"); + assertStringReplace("Σ", "Σ", "x", UNICODE, "x"); + assertStringReplace("σ", "σ", "x", UNICODE_CI, "x"); + assertStringReplace("σ", "ς", "x", UNICODE_CI, "x"); + assertStringReplace("σ", "Σ", "x", UNICODE_CI, "x"); + assertStringReplace("ς", "σ", "x", UNICODE_CI, "x"); + assertStringReplace("ς", "ς", "x", UNICODE_CI, "x"); + assertStringReplace("ς", "Σ", "x", UNICODE_CI, "x"); + assertStringReplace("Σ", "σ", "x", UNICODE_CI, "x"); + assertStringReplace("Σ", "ς", "x", UNICODE_CI, "x"); + assertStringReplace("Σ", "Σ", "x", UNICODE_CI, "x"); // Surrogate pairs. - assertStringReplace("a🙃b", "a", "x", "UTF8_BINARY", "x🙃b"); - assertStringReplace("a🙃b", "b", "x", "UTF8_BINARY", "a🙃x"); - assertStringReplace("a🙃b", "🙃", "x", "UTF8_BINARY", "axb"); - assertStringReplace("a🙃b", "b", "c", "UTF8_LCASE", "a🙃c"); - assertStringReplace("a🙃b", "b", "x", "UTF8_LCASE", "a🙃x"); - assertStringReplace("a🙃b", "🙃", "x", "UTF8_LCASE", "axb"); - assertStringReplace("a🙃b", "b", "c", "UNICODE", "a🙃c"); - assertStringReplace("a🙃b", "b", "x", "UNICODE", "a🙃x"); - assertStringReplace("a🙃b", "🙃", "x", "UNICODE", "axb"); - assertStringReplace("a🙃b", "b", "c", "UNICODE_CI", "a🙃c"); - assertStringReplace("a🙃b", "b", "x", "UNICODE_CI", "a🙃x"); - assertStringReplace("a🙃b", "🙃", "x", "UNICODE_CI", "axb"); + assertStringReplace("a🙃b", "a", "x", UTF8_BINARY, "x🙃b"); + assertStringReplace("a🙃b", "b", "x", UTF8_BINARY, "a🙃x"); + assertStringReplace("a🙃b", "🙃", "x", UTF8_BINARY, "axb"); + assertStringReplace("a🙃b", "b", "c", UTF8_LCASE, "a🙃c"); + assertStringReplace("a🙃b", "b", "x", UTF8_LCASE, "a🙃x"); + assertStringReplace("a🙃b", "🙃", "x", UTF8_LCASE, "axb"); + assertStringReplace("a🙃b", "b", "c", UNICODE, "a🙃c"); + assertStringReplace("a🙃b", "b", "x", UNICODE, "a🙃x"); + assertStringReplace("a🙃b", "🙃", "x", UNICODE, "axb"); + assertStringReplace("a🙃b", "b", "c", UNICODE_CI, "a🙃c"); + assertStringReplace("a🙃b", "b", "x", UNICODE_CI, "a🙃x"); + assertStringReplace("a🙃b", "🙃", "x", UNICODE_CI, "axb"); } /** @@ -2145,293 +2146,293 @@ private void assertStringLocate(String substring, String string, int start, @Test public void testStringLocate() throws SparkException { // Empty strings. - assertStringLocate("", "", -1, "UTF8_BINARY", 1); - assertStringLocate("", "", -1, "UTF8_LCASE", 1); - assertStringLocate("", "", -1, "UNICODE", 1); - assertStringLocate("", "", -1, "UNICODE_CI", 1); - assertStringLocate("", "", 0, "UTF8_BINARY", 1); - assertStringLocate("", "", 0, "UTF8_LCASE", 1); - assertStringLocate("", "", 0, "UNICODE", 1); - assertStringLocate("", "", 0, "UNICODE_CI", 1); - assertStringLocate("", "", 1, "UTF8_BINARY", 1); - assertStringLocate("", "", 1, "UTF8_LCASE", 1); - assertStringLocate("", "", 1, "UNICODE", 1); - assertStringLocate("", "", 1, "UNICODE_CI", 1); - assertStringLocate("a", "", -1, "UTF8_BINARY", 0); - assertStringLocate("a", "", -1, "UTF8_LCASE", 0); - assertStringLocate("a", "", -1, "UNICODE", 0); - assertStringLocate("a", "", -1, "UNICODE_CI", 0); - assertStringLocate("a", "", 0, "UTF8_BINARY", 0); - assertStringLocate("a", "", 0, "UTF8_LCASE", 0); - assertStringLocate("a", "", 0, "UNICODE", 0); - assertStringLocate("a", "", 0, "UNICODE_CI", 0); - assertStringLocate("a", "", 1, "UTF8_BINARY", 0); - assertStringLocate("a", "", 1, "UTF8_LCASE", 0); - assertStringLocate("a", "", 1, "UNICODE", 0); - assertStringLocate("a", "", 1, "UNICODE_CI", 0); - assertStringLocate("", "x", -1, "UTF8_BINARY", 1); - assertStringLocate("", "x", -1, "UTF8_LCASE", 1); - assertStringLocate("", "x", -1, "UNICODE", 1); - assertStringLocate("", "x", -1, "UNICODE_CI", 1); - assertStringLocate("", "x", 0, "UTF8_BINARY", 1); - assertStringLocate("", "x", 0, "UTF8_LCASE", 1); - assertStringLocate("", "x", 0, "UNICODE", 1); - assertStringLocate("", "x", 0, "UNICODE_CI", 1); - assertStringLocate("", "x", 1, "UTF8_BINARY", 1); - assertStringLocate("", "x", 1, "UTF8_LCASE", 1); - assertStringLocate("", "x", 1, "UNICODE", 1); - assertStringLocate("", "x", 1, "UNICODE_CI", 1); + assertStringLocate("", "", -1, UTF8_BINARY, 1); + assertStringLocate("", "", -1, UTF8_LCASE, 1); + assertStringLocate("", "", -1, UNICODE, 1); + assertStringLocate("", "", -1, UNICODE_CI, 1); + assertStringLocate("", "", 0, UTF8_BINARY, 1); + assertStringLocate("", "", 0, UTF8_LCASE, 1); + assertStringLocate("", "", 0, UNICODE, 1); + assertStringLocate("", "", 0, UNICODE_CI, 1); + assertStringLocate("", "", 1, UTF8_BINARY, 1); + assertStringLocate("", "", 1, UTF8_LCASE, 1); + assertStringLocate("", "", 1, UNICODE, 1); + assertStringLocate("", "", 1, UNICODE_CI, 1); + assertStringLocate("a", "", -1, UTF8_BINARY, 0); + assertStringLocate("a", "", -1, UTF8_LCASE, 0); + assertStringLocate("a", "", -1, UNICODE, 0); + assertStringLocate("a", "", -1, UNICODE_CI, 0); + assertStringLocate("a", "", 0, UTF8_BINARY, 0); + assertStringLocate("a", "", 0, UTF8_LCASE, 0); + assertStringLocate("a", "", 0, UNICODE, 0); + assertStringLocate("a", "", 0, UNICODE_CI, 0); + assertStringLocate("a", "", 1, UTF8_BINARY, 0); + assertStringLocate("a", "", 1, UTF8_LCASE, 0); + assertStringLocate("a", "", 1, UNICODE, 0); + assertStringLocate("a", "", 1, UNICODE_CI, 0); + assertStringLocate("", "x", -1, UTF8_BINARY, 1); + assertStringLocate("", "x", -1, UTF8_LCASE, 1); + assertStringLocate("", "x", -1, UNICODE, 1); + assertStringLocate("", "x", -1, UNICODE_CI, 1); + assertStringLocate("", "x", 0, UTF8_BINARY, 1); + assertStringLocate("", "x", 0, UTF8_LCASE, 1); + assertStringLocate("", "x", 0, UNICODE, 1); + assertStringLocate("", "x", 0, UNICODE_CI, 1); + assertStringLocate("", "x", 1, UTF8_BINARY, 1); + assertStringLocate("", "x", 1, UTF8_LCASE, 1); + assertStringLocate("", "x", 1, UNICODE, 1); + assertStringLocate("", "x", 1, UNICODE_CI, 1); // Basic tests. - assertStringLocate("aa", "aaads", 1, "UTF8_BINARY", 1); - assertStringLocate("aa", "aaads", 1, "UTF8_LCASE", 1); - assertStringLocate("aa", "aaads", 1, "UNICODE", 1); - assertStringLocate("aa", "aaads", 1, "UNICODE_CI", 1); - assertStringLocate("aa", "aaads", 2, "UTF8_BINARY", 2); - assertStringLocate("aa", "aaads", 2, "UTF8_LCASE", 2); - assertStringLocate("aa", "aaads", 2, "UNICODE", 2); - assertStringLocate("aa", "aaads", 2, "UNICODE_CI", 2); - assertStringLocate("aa", "aaads", 3, "UTF8_BINARY", 0); - assertStringLocate("aa", "aaads", 3, "UTF8_LCASE", 0); - assertStringLocate("aa", "aaads", 3, "UNICODE", 0); - assertStringLocate("aa", "aaads", 3, "UNICODE_CI", 0); - assertStringLocate("Aa", "aaads", 1, "UTF8_BINARY", 0); - assertStringLocate("Aa", "aaads", 1, "UTF8_LCASE", 1); - assertStringLocate("Aa", "aaads", 1, "UNICODE", 0); - assertStringLocate("Aa", "aaads", 1, "UNICODE_CI", 1); - assertStringLocate("Aa", "aaads", 2, "UTF8_BINARY", 0); - assertStringLocate("Aa", "aaads", 2, "UTF8_LCASE", 2); - assertStringLocate("Aa", "aaads", 2, "UNICODE", 0); - assertStringLocate("Aa", "aaads", 2, "UNICODE_CI", 2); - assertStringLocate("Aa", "aaads", 3, "UTF8_BINARY", 0); - assertStringLocate("Aa", "aaads", 3, "UTF8_LCASE", 0); - assertStringLocate("Aa", "aaads", 3, "UNICODE", 0); - assertStringLocate("Aa", "aaads", 3, "UNICODE_CI", 0); - assertStringLocate("Aa", "aAads", 1, "UTF8_BINARY", 2); - assertStringLocate("Aa", "aAads", 1, "UTF8_LCASE", 1); - assertStringLocate("Aa", "aAads", 1, "UNICODE", 2); - assertStringLocate("Aa", "aAads", 1, "UNICODE_CI", 1); - assertStringLocate("AA", "aaads", 1, "UTF8_BINARY", 0); - assertStringLocate("AA", "aaads", 1, "UTF8_LCASE", 1); - assertStringLocate("AA", "aaads", 1, "UNICODE", 0); - assertStringLocate("AA", "aaads", 1, "UNICODE_CI", 1); - assertStringLocate("aa", "aAads", 2, "UTF8_BINARY", 0); - assertStringLocate("aa", "aAads", 2, "UTF8_LCASE", 2); - assertStringLocate("aa", "aAads", 2, "UNICODE", 0); - assertStringLocate("aa", "aAads", 2, "UNICODE_CI", 2); - assertStringLocate("aa", "aaAds", 3, "UTF8_BINARY", 0); - assertStringLocate("aa", "aaAds", 3, "UTF8_LCASE", 0); - assertStringLocate("aa", "aaAds", 3, "UNICODE", 0); - assertStringLocate("aa", "aaAds", 3, "UNICODE_CI", 0); - assertStringLocate("abC", "abcabc", 1, "UTF8_BINARY", 0); - assertStringLocate("abC", "abcabc", 1, "UTF8_LCASE", 1); - assertStringLocate("abC", "abcabc", 1, "UNICODE", 0); - assertStringLocate("abC", "abcabc", 1, "UNICODE_CI", 1); - assertStringLocate("abC", "abCabc", 2, "UTF8_BINARY", 0); - assertStringLocate("abC", "abCabc", 2, "UTF8_LCASE", 4); - assertStringLocate("abC", "abCabc", 2, "UNICODE", 0); - assertStringLocate("abC", "abCabc", 2, "UNICODE_CI", 4); - assertStringLocate("abc", "abcabc", 1, "UTF8_BINARY", 1); - assertStringLocate("abc", "abcabc", 1, "UTF8_LCASE", 1); - assertStringLocate("abc", "abcabc", 1, "UNICODE", 1); - assertStringLocate("abc", "abcabc", 1, "UNICODE_CI", 1); - assertStringLocate("abc", "abcabc", 2, "UTF8_BINARY", 4); - assertStringLocate("abc", "abcabc", 2, "UTF8_LCASE", 4); - assertStringLocate("abc", "abcabc", 2, "UNICODE", 4); - assertStringLocate("abc", "abcabc", 2, "UNICODE_CI", 4); - assertStringLocate("abc", "abcabc", 3, "UTF8_BINARY", 4); - assertStringLocate("abc", "abcabc", 3, "UTF8_LCASE", 4); - assertStringLocate("abc", "abcabc", 3, "UNICODE", 4); - assertStringLocate("abc", "abcabc", 3, "UNICODE_CI", 4); - assertStringLocate("abc", "abcabc", 4, "UTF8_BINARY", 4); - assertStringLocate("abc", "abcabc", 4, "UTF8_LCASE", 4); - assertStringLocate("abc", "abcabc", 4, "UNICODE", 4); - assertStringLocate("abc", "abcabc", 4, "UNICODE_CI", 4); - assertStringLocate("aa", "Aaads", 1, "UTF8_BINARY", 2); - assertStringLocate("aa", "Aaads", 1, "UTF8_LCASE", 1); - assertStringLocate("aa", "Aaads", 1, "UNICODE", 2); - assertStringLocate("aa", "Aaads", 1, "UNICODE_CI", 1); + assertStringLocate("aa", "aaads", 1, UTF8_BINARY, 1); + assertStringLocate("aa", "aaads", 1, UTF8_LCASE, 1); + assertStringLocate("aa", "aaads", 1, UNICODE, 1); + assertStringLocate("aa", "aaads", 1, UNICODE_CI, 1); + assertStringLocate("aa", "aaads", 2, UTF8_BINARY, 2); + assertStringLocate("aa", "aaads", 2, UTF8_LCASE, 2); + assertStringLocate("aa", "aaads", 2, UNICODE, 2); + assertStringLocate("aa", "aaads", 2, UNICODE_CI, 2); + assertStringLocate("aa", "aaads", 3, UTF8_BINARY, 0); + assertStringLocate("aa", "aaads", 3, UTF8_LCASE, 0); + assertStringLocate("aa", "aaads", 3, UNICODE, 0); + assertStringLocate("aa", "aaads", 3, UNICODE_CI, 0); + assertStringLocate("Aa", "aaads", 1, UTF8_BINARY, 0); + assertStringLocate("Aa", "aaads", 1, UTF8_LCASE, 1); + assertStringLocate("Aa", "aaads", 1, UNICODE, 0); + assertStringLocate("Aa", "aaads", 1, UNICODE_CI, 1); + assertStringLocate("Aa", "aaads", 2, UTF8_BINARY, 0); + assertStringLocate("Aa", "aaads", 2, UTF8_LCASE, 2); + assertStringLocate("Aa", "aaads", 2, UNICODE, 0); + assertStringLocate("Aa", "aaads", 2, UNICODE_CI, 2); + assertStringLocate("Aa", "aaads", 3, UTF8_BINARY, 0); + assertStringLocate("Aa", "aaads", 3, UTF8_LCASE, 0); + assertStringLocate("Aa", "aaads", 3, UNICODE, 0); + assertStringLocate("Aa", "aaads", 3, UNICODE_CI, 0); + assertStringLocate("Aa", "aAads", 1, UTF8_BINARY, 2); + assertStringLocate("Aa", "aAads", 1, UTF8_LCASE, 1); + assertStringLocate("Aa", "aAads", 1, UNICODE, 2); + assertStringLocate("Aa", "aAads", 1, UNICODE_CI, 1); + assertStringLocate("AA", "aaads", 1, UTF8_BINARY, 0); + assertStringLocate("AA", "aaads", 1, UTF8_LCASE, 1); + assertStringLocate("AA", "aaads", 1, UNICODE, 0); + assertStringLocate("AA", "aaads", 1, UNICODE_CI, 1); + assertStringLocate("aa", "aAads", 2, UTF8_BINARY, 0); + assertStringLocate("aa", "aAads", 2, UTF8_LCASE, 2); + assertStringLocate("aa", "aAads", 2, UNICODE, 0); + assertStringLocate("aa", "aAads", 2, UNICODE_CI, 2); + assertStringLocate("aa", "aaAds", 3, UTF8_BINARY, 0); + assertStringLocate("aa", "aaAds", 3, UTF8_LCASE, 0); + assertStringLocate("aa", "aaAds", 3, UNICODE, 0); + assertStringLocate("aa", "aaAds", 3, UNICODE_CI, 0); + assertStringLocate("abC", "abcabc", 1, UTF8_BINARY, 0); + assertStringLocate("abC", "abcabc", 1, UTF8_LCASE, 1); + assertStringLocate("abC", "abcabc", 1, UNICODE, 0); + assertStringLocate("abC", "abcabc", 1, UNICODE_CI, 1); + assertStringLocate("abC", "abCabc", 2, UTF8_BINARY, 0); + assertStringLocate("abC", "abCabc", 2, UTF8_LCASE, 4); + assertStringLocate("abC", "abCabc", 2, UNICODE, 0); + assertStringLocate("abC", "abCabc", 2, UNICODE_CI, 4); + assertStringLocate("abc", "abcabc", 1, UTF8_BINARY, 1); + assertStringLocate("abc", "abcabc", 1, UTF8_LCASE, 1); + assertStringLocate("abc", "abcabc", 1, UNICODE, 1); + assertStringLocate("abc", "abcabc", 1, UNICODE_CI, 1); + assertStringLocate("abc", "abcabc", 2, UTF8_BINARY, 4); + assertStringLocate("abc", "abcabc", 2, UTF8_LCASE, 4); + assertStringLocate("abc", "abcabc", 2, UNICODE, 4); + assertStringLocate("abc", "abcabc", 2, UNICODE_CI, 4); + assertStringLocate("abc", "abcabc", 3, UTF8_BINARY, 4); + assertStringLocate("abc", "abcabc", 3, UTF8_LCASE, 4); + assertStringLocate("abc", "abcabc", 3, UNICODE, 4); + assertStringLocate("abc", "abcabc", 3, UNICODE_CI, 4); + assertStringLocate("abc", "abcabc", 4, UTF8_BINARY, 4); + assertStringLocate("abc", "abcabc", 4, UTF8_LCASE, 4); + assertStringLocate("abc", "abcabc", 4, UNICODE, 4); + assertStringLocate("abc", "abcabc", 4, UNICODE_CI, 4); + assertStringLocate("aa", "Aaads", 1, UTF8_BINARY, 2); + assertStringLocate("aa", "Aaads", 1, UTF8_LCASE, 1); + assertStringLocate("aa", "Aaads", 1, UNICODE, 2); + assertStringLocate("aa", "Aaads", 1, UNICODE_CI, 1); assertStringLocate("ćČ", "CćČČćCČĆČcČcććČč", 3, "SR", 14); assertStringLocate("ćČ", "CćČČćCČĆČcČcććČč", 3, "SR_CI_AI", 3); // Advanced tests. - assertStringLocate("界x", "test大千世界X大千世界", 1, "UTF8_BINARY", 0); - assertStringLocate("界X", "test大千世界X大千世界", 1, "UTF8_BINARY", 8); - assertStringLocate("界", "test大千世界X大千世界", 13, "UTF8_BINARY", 13); - assertStringLocate("界x", "test大千世界X大千世界", 1, "UTF8_LCASE", 8); - assertStringLocate("界X", "test大千世界Xtest大千世界", 1, "UTF8_LCASE", 8); - assertStringLocate("界", "test大千世界X大千世界", 13, "UTF8_LCASE", 13); - assertStringLocate("大千", "test大千世界大千世界", 1, "UTF8_LCASE", 5); - assertStringLocate("大千", "test大千世界大千世界", 9, "UTF8_LCASE", 9); - assertStringLocate("大千", "大千世界大千世界", 1, "UTF8_LCASE", 1); - assertStringLocate("界x", "test大千世界X大千世界", 1, "UNICODE", 0); - assertStringLocate("界X", "test大千世界X大千世界", 1, "UNICODE", 8); - assertStringLocate("界", "test大千世界X大千世界", 13, "UNICODE", 13); - assertStringLocate("界x", "test大千世界X大千世界", 1, "UNICODE_CI", 8); - assertStringLocate("界", "test大千世界X大千世界", 13, "UNICODE_CI", 13); - assertStringLocate("大千", "test大千世界大千世界", 1, "UNICODE_CI", 5); - assertStringLocate("大千", "test大千世界大千世界", 9, "UNICODE_CI", 9); - assertStringLocate("大千", "大千世界大千世界", 1, "UNICODE_CI", 1); + assertStringLocate("界x", "test大千世界X大千世界", 1, UTF8_BINARY, 0); + assertStringLocate("界X", "test大千世界X大千世界", 1, UTF8_BINARY, 8); + assertStringLocate("界", "test大千世界X大千世界", 13, UTF8_BINARY, 13); + assertStringLocate("界x", "test大千世界X大千世界", 1, UTF8_LCASE, 8); + assertStringLocate("界X", "test大千世界Xtest大千世界", 1, UTF8_LCASE, 8); + assertStringLocate("界", "test大千世界X大千世界", 13, UTF8_LCASE, 13); + assertStringLocate("大千", "test大千世界大千世界", 1, UTF8_LCASE, 5); + assertStringLocate("大千", "test大千世界大千世界", 9, UTF8_LCASE, 9); + assertStringLocate("大千", "大千世界大千世界", 1, UTF8_LCASE, 1); + assertStringLocate("界x", "test大千世界X大千世界", 1, UNICODE, 0); + assertStringLocate("界X", "test大千世界X大千世界", 1, UNICODE, 8); + assertStringLocate("界", "test大千世界X大千世界", 13, UNICODE, 13); + assertStringLocate("界x", "test大千世界X大千世界", 1, UNICODE_CI, 8); + assertStringLocate("界", "test大千世界X大千世界", 13, UNICODE_CI, 13); + assertStringLocate("大千", "test大千世界大千世界", 1, UNICODE_CI, 5); + assertStringLocate("大千", "test大千世界大千世界", 9, UNICODE_CI, 9); + assertStringLocate("大千", "大千世界大千世界", 1, UNICODE_CI, 1); // One-to-many case mapping (e.g. Turkish dotted I). - assertStringLocate("\u0307", "i\u0307", 1, "UTF8_BINARY", 2); - assertStringLocate("\u0307", "İ", 1, "UTF8_LCASE", 0); // != UTF8_BINARY - assertStringLocate("i", "i\u0307", 1, "UNICODE_CI", 0); - assertStringLocate("\u0307", "i\u0307", 1, "UNICODE_CI", 0); - assertStringLocate("i\u0307", "i", 1, "UNICODE_CI", 0); - assertStringLocate("İ", "i\u0307", 1, "UNICODE_CI", 1); - assertStringLocate("İ", "i", 1, "UNICODE_CI", 0); - assertStringLocate("i", "i\u0307", 1, "UTF8_LCASE", 1); // != UNICODE_CI - assertStringLocate("\u0307", "i\u0307", 1, "UTF8_LCASE", 2); // != UNICODE_CI - assertStringLocate("i\u0307", "i", 1, "UTF8_LCASE", 0); - assertStringLocate("İ", "i\u0307", 1, "UTF8_LCASE", 1); - assertStringLocate("İ", "i", 1, "UTF8_LCASE", 0); - assertStringLocate("i\u0307o", "İo世界大千世界", 1, "UNICODE_CI", 1); - assertStringLocate("i\u0307o", "大千İo世界大千世界", 1, "UNICODE_CI", 3); - assertStringLocate("i\u0307o", "世界İo大千世界大千İo", 4, "UNICODE_CI", 11); - assertStringLocate("İo", "i̇o世界大千世界", 1, "UNICODE_CI", 1); - assertStringLocate("İo", "大千i̇o世界大千世界", 1, "UNICODE_CI", 3); - assertStringLocate("İo", "世界i̇o大千世界大千i̇o", 4, "UNICODE_CI", 12); + assertStringLocate("\u0307", "i\u0307", 1, UTF8_BINARY, 2); + assertStringLocate("\u0307", "İ", 1, UTF8_LCASE, 0); // != UTF8_BINARY + assertStringLocate("i", "i\u0307", 1, UNICODE_CI, 0); + assertStringLocate("\u0307", "i\u0307", 1, UNICODE_CI, 0); + assertStringLocate("i\u0307", "i", 1, UNICODE_CI, 0); + assertStringLocate("İ", "i\u0307", 1, UNICODE_CI, 1); + assertStringLocate("İ", "i", 1, UNICODE_CI, 0); + assertStringLocate("i", "i\u0307", 1, UTF8_LCASE, 1); // != UNICODE_CI + assertStringLocate("\u0307", "i\u0307", 1, UTF8_LCASE, 2); // != UNICODE_CI + assertStringLocate("i\u0307", "i", 1, UTF8_LCASE, 0); + assertStringLocate("İ", "i\u0307", 1, UTF8_LCASE, 1); + assertStringLocate("İ", "i", 1, UTF8_LCASE, 0); + assertStringLocate("i\u0307o", "İo世界大千世界", 1, UNICODE_CI, 1); + assertStringLocate("i\u0307o", "大千İo世界大千世界", 1, UNICODE_CI, 3); + assertStringLocate("i\u0307o", "世界İo大千世界大千İo", 4, UNICODE_CI, 11); + assertStringLocate("İo", "i̇o世界大千世界", 1, UNICODE_CI, 1); + assertStringLocate("İo", "大千i̇o世界大千世界", 1, UNICODE_CI, 3); + assertStringLocate("İo", "世界i̇o大千世界大千i̇o", 4, UNICODE_CI, 12); // Conditional case mapping (e.g. Greek sigmas). - assertStringLocate("σ", "σ", 1, "UTF8_BINARY", 1); - assertStringLocate("σ", "ς", 1, "UTF8_BINARY", 0); - assertStringLocate("σ", "Σ", 1, "UTF8_BINARY", 0); - assertStringLocate("ς", "σ", 1, "UTF8_BINARY", 0); - assertStringLocate("ς", "ς", 1, "UTF8_BINARY", 1); - assertStringLocate("ς", "Σ", 1, "UTF8_BINARY", 0); - assertStringLocate("Σ", "σ", 1, "UTF8_BINARY", 0); - assertStringLocate("Σ", "ς", 1, "UTF8_BINARY", 0); - assertStringLocate("Σ", "Σ", 1, "UTF8_BINARY", 1); - assertStringLocate("σ", "σ", 1, "UTF8_LCASE", 1); - assertStringLocate("σ", "ς", 1, "UTF8_LCASE", 1); - assertStringLocate("σ", "Σ", 1, "UTF8_LCASE", 1); - assertStringLocate("ς", "σ", 1, "UTF8_LCASE", 1); - assertStringLocate("ς", "ς", 1, "UTF8_LCASE", 1); - assertStringLocate("ς", "Σ", 1, "UTF8_LCASE", 1); - assertStringLocate("Σ", "σ", 1, "UTF8_LCASE", 1); - assertStringLocate("Σ", "ς", 1, "UTF8_LCASE", 1); - assertStringLocate("Σ", "Σ", 1, "UTF8_LCASE", 1); - assertStringLocate("σ", "σ", 1, "UNICODE", 1); - assertStringLocate("σ", "ς", 1, "UNICODE", 0); - assertStringLocate("σ", "Σ", 1, "UNICODE", 0); - assertStringLocate("ς", "σ", 1, "UNICODE", 0); - assertStringLocate("ς", "ς", 1, "UNICODE", 1); - assertStringLocate("ς", "Σ", 1, "UNICODE", 0); - assertStringLocate("Σ", "σ", 1, "UNICODE", 0); - assertStringLocate("Σ", "ς", 1, "UNICODE", 0); - assertStringLocate("Σ", "Σ", 1, "UNICODE", 1); - assertStringLocate("σ", "σ", 1, "UNICODE_CI", 1); - assertStringLocate("σ", "ς", 1, "UNICODE_CI", 1); - assertStringLocate("σ", "Σ", 1, "UNICODE_CI", 1); - assertStringLocate("ς", "σ", 1, "UNICODE_CI", 1); - assertStringLocate("ς", "ς", 1, "UNICODE_CI", 1); - assertStringLocate("ς", "Σ", 1, "UNICODE_CI", 1); - assertStringLocate("Σ", "σ", 1, "UNICODE_CI", 1); - assertStringLocate("Σ", "ς", 1, "UNICODE_CI", 1); - assertStringLocate("Σ", "Σ", 1, "UNICODE_CI", 1); + assertStringLocate("σ", "σ", 1, UTF8_BINARY, 1); + assertStringLocate("σ", "ς", 1, UTF8_BINARY, 0); + assertStringLocate("σ", "Σ", 1, UTF8_BINARY, 0); + assertStringLocate("ς", "σ", 1, UTF8_BINARY, 0); + assertStringLocate("ς", "ς", 1, UTF8_BINARY, 1); + assertStringLocate("ς", "Σ", 1, UTF8_BINARY, 0); + assertStringLocate("Σ", "σ", 1, UTF8_BINARY, 0); + assertStringLocate("Σ", "ς", 1, UTF8_BINARY, 0); + assertStringLocate("Σ", "Σ", 1, UTF8_BINARY, 1); + assertStringLocate("σ", "σ", 1, UTF8_LCASE, 1); + assertStringLocate("σ", "ς", 1, UTF8_LCASE, 1); + assertStringLocate("σ", "Σ", 1, UTF8_LCASE, 1); + assertStringLocate("ς", "σ", 1, UTF8_LCASE, 1); + assertStringLocate("ς", "ς", 1, UTF8_LCASE, 1); + assertStringLocate("ς", "Σ", 1, UTF8_LCASE, 1); + assertStringLocate("Σ", "σ", 1, UTF8_LCASE, 1); + assertStringLocate("Σ", "ς", 1, UTF8_LCASE, 1); + assertStringLocate("Σ", "Σ", 1, UTF8_LCASE, 1); + assertStringLocate("σ", "σ", 1, UNICODE, 1); + assertStringLocate("σ", "ς", 1, UNICODE, 0); + assertStringLocate("σ", "Σ", 1, UNICODE, 0); + assertStringLocate("ς", "σ", 1, UNICODE, 0); + assertStringLocate("ς", "ς", 1, UNICODE, 1); + assertStringLocate("ς", "Σ", 1, UNICODE, 0); + assertStringLocate("Σ", "σ", 1, UNICODE, 0); + assertStringLocate("Σ", "ς", 1, UNICODE, 0); + assertStringLocate("Σ", "Σ", 1, UNICODE, 1); + assertStringLocate("σ", "σ", 1, UNICODE_CI, 1); + assertStringLocate("σ", "ς", 1, UNICODE_CI, 1); + assertStringLocate("σ", "Σ", 1, UNICODE_CI, 1); + assertStringLocate("ς", "σ", 1, UNICODE_CI, 1); + assertStringLocate("ς", "ς", 1, UNICODE_CI, 1); + assertStringLocate("ς", "Σ", 1, UNICODE_CI, 1); + assertStringLocate("Σ", "σ", 1, UNICODE_CI, 1); + assertStringLocate("Σ", "ς", 1, UNICODE_CI, 1); + assertStringLocate("Σ", "Σ", 1, UNICODE_CI, 1); // Surrogate pairs. - assertStringLocate("a", "a🙃b", 1, "UTF8_BINARY", 1); - assertStringLocate("a", "a🙃b", 1, "UTF8_LCASE", 1); - assertStringLocate("a", "a🙃b", 1, "UNICODE", 1); - assertStringLocate("a", "a🙃b", 1, "UNICODE_CI", 1); - assertStringLocate("a", "a🙃b", 2, "UTF8_BINARY", 0); - assertStringLocate("a", "a🙃b", 2, "UTF8_LCASE", 0); - assertStringLocate("a", "a🙃b", 2, "UNICODE", 0); - assertStringLocate("a", "a🙃b", 2, "UNICODE_CI", 0); - assertStringLocate("a", "a🙃b", 3, "UTF8_BINARY", 0); - assertStringLocate("a", "a🙃b", 3, "UTF8_LCASE", 0); - assertStringLocate("a", "a🙃b", 3, "UNICODE", 0); - assertStringLocate("a", "a🙃b", 3, "UNICODE_CI", 0); - assertStringLocate("🙃", "a🙃b", 1, "UTF8_BINARY", 2); - assertStringLocate("🙃", "a🙃b", 1, "UTF8_LCASE", 2); - assertStringLocate("🙃", "a🙃b", 1, "UNICODE", 2); - assertStringLocate("🙃", "a🙃b", 1, "UNICODE_CI", 2); - assertStringLocate("🙃", "a🙃b", 2, "UTF8_BINARY", 2); - assertStringLocate("🙃", "a🙃b", 2, "UTF8_LCASE", 2); - assertStringLocate("🙃", "a🙃b", 2, "UNICODE", 2); - assertStringLocate("🙃", "a🙃b", 2, "UNICODE_CI", 2); - assertStringLocate("🙃", "a🙃b", 3, "UTF8_BINARY", 0); - assertStringLocate("🙃", "a🙃b", 3, "UTF8_LCASE", 0); - assertStringLocate("🙃", "a🙃b", 3, "UNICODE", 0); - assertStringLocate("🙃", "a🙃b", 3, "UNICODE_CI", 0); - assertStringLocate("b", "a🙃b", 1, "UTF8_BINARY", 3); - assertStringLocate("b", "a🙃b", 1, "UTF8_LCASE", 3); - assertStringLocate("b", "a🙃b", 1, "UNICODE", 3); - assertStringLocate("b", "a🙃b", 1, "UNICODE_CI", 3); - assertStringLocate("b", "a🙃b", 2, "UTF8_BINARY", 3); - assertStringLocate("b", "a🙃b", 2, "UTF8_LCASE", 3); - assertStringLocate("b", "a🙃b", 2, "UNICODE", 3); - assertStringLocate("b", "a🙃b", 2, "UNICODE_CI", 3); - assertStringLocate("b", "a🙃b", 3, "UTF8_BINARY", 3); - assertStringLocate("b", "a🙃b", 3, "UTF8_LCASE", 3); - assertStringLocate("b", "a🙃b", 3, "UNICODE", 3); - assertStringLocate("b", "a🙃b", 3, "UNICODE_CI", 3); - assertStringLocate("🙃", "a🙃🙃b", 1, "UTF8_BINARY", 2); - assertStringLocate("🙃", "a🙃🙃b", 1, "UTF8_LCASE", 2); - assertStringLocate("🙃", "a🙃🙃b", 1, "UNICODE", 2); - assertStringLocate("🙃", "a🙃🙃b", 1, "UNICODE_CI", 2); - assertStringLocate("🙃", "a🙃🙃b", 2, "UTF8_BINARY", 2); - assertStringLocate("🙃", "a🙃🙃b", 2, "UTF8_LCASE", 2); - assertStringLocate("🙃", "a🙃🙃b", 2, "UNICODE", 2); - assertStringLocate("🙃", "a🙃🙃b", 2, "UNICODE_CI", 2); - assertStringLocate("🙃", "a🙃🙃b", 3, "UTF8_BINARY", 3); - assertStringLocate("🙃", "a🙃🙃b", 3, "UTF8_LCASE", 3); - assertStringLocate("🙃", "a🙃🙃b", 3, "UNICODE", 3); - assertStringLocate("🙃", "a🙃🙃b", 3, "UNICODE_CI", 3); - assertStringLocate("🙃", "a🙃🙃b", 4, "UTF8_BINARY", 0); - assertStringLocate("🙃", "a🙃🙃b", 4, "UTF8_LCASE", 0); - assertStringLocate("🙃", "a🙃🙃b", 4, "UNICODE", 0); - assertStringLocate("🙃", "a🙃🙃b", 4, "UNICODE_CI", 0); - assertStringLocate("b", "a🙃🙃b", 1, "UTF8_BINARY", 4); - assertStringLocate("b", "a🙃🙃b", 1, "UTF8_LCASE", 4); - assertStringLocate("b", "a🙃🙃b", 1, "UNICODE", 4); - assertStringLocate("b", "a🙃🙃b", 1, "UNICODE_CI", 4); - assertStringLocate("b", "a🙃🙃b", 2, "UTF8_BINARY", 4); - assertStringLocate("b", "a🙃🙃b", 2, "UTF8_LCASE", 4); - assertStringLocate("b", "a🙃🙃b", 2, "UNICODE", 4); - assertStringLocate("b", "a🙃🙃b", 2, "UNICODE_CI", 4); - assertStringLocate("b", "a🙃🙃b", 3, "UTF8_BINARY", 4); - assertStringLocate("b", "a🙃🙃b", 3, "UTF8_LCASE", 4); - assertStringLocate("b", "a🙃🙃b", 3, "UNICODE", 4); - assertStringLocate("b", "a🙃🙃b", 3, "UNICODE_CI", 4); - assertStringLocate("b", "a🙃🙃b", 4, "UTF8_BINARY", 4); - assertStringLocate("b", "a🙃🙃b", 4, "UTF8_LCASE", 4); - assertStringLocate("b", "a🙃🙃b", 4, "UNICODE", 4); - assertStringLocate("b", "a🙃🙃b", 4, "UNICODE_CI", 4); - assertStringLocate("b", "a🙃x🙃b", 1, "UTF8_BINARY", 5); - assertStringLocate("b", "a🙃x🙃b", 1, "UTF8_LCASE", 5); - assertStringLocate("b", "a🙃x🙃b", 1, "UNICODE", 5); - assertStringLocate("b", "a🙃x🙃b", 1, "UNICODE_CI", 5); - assertStringLocate("b", "a🙃x🙃b", 2, "UTF8_BINARY", 5); - assertStringLocate("b", "a🙃x🙃b", 2, "UTF8_LCASE", 5); - assertStringLocate("b", "a🙃x🙃b", 2, "UNICODE", 5); - assertStringLocate("b", "a🙃x🙃b", 2, "UNICODE_CI", 5); - assertStringLocate("b", "a🙃x🙃b", 3, "UTF8_BINARY", 5); - assertStringLocate("b", "a🙃x🙃b", 3, "UTF8_LCASE", 5); - assertStringLocate("b", "a🙃x🙃b", 3, "UNICODE", 5); - assertStringLocate("b", "a🙃x🙃b", 3, "UNICODE_CI", 5); - assertStringLocate("b", "a🙃x🙃b", 4, "UTF8_BINARY", 5); - assertStringLocate("b", "a🙃x🙃b", 4, "UTF8_LCASE", 5); - assertStringLocate("b", "a🙃x🙃b", 4, "UNICODE", 5); - assertStringLocate("b", "a🙃x🙃b", 4, "UNICODE_CI", 5); + assertStringLocate("a", "a🙃b", 1, UTF8_BINARY, 1); + assertStringLocate("a", "a🙃b", 1, UTF8_LCASE, 1); + assertStringLocate("a", "a🙃b", 1, UNICODE, 1); + assertStringLocate("a", "a🙃b", 1, UNICODE_CI, 1); + assertStringLocate("a", "a🙃b", 2, UTF8_BINARY, 0); + assertStringLocate("a", "a🙃b", 2, UTF8_LCASE, 0); + assertStringLocate("a", "a🙃b", 2, UNICODE, 0); + assertStringLocate("a", "a🙃b", 2, UNICODE_CI, 0); + assertStringLocate("a", "a🙃b", 3, UTF8_BINARY, 0); + assertStringLocate("a", "a🙃b", 3, UTF8_LCASE, 0); + assertStringLocate("a", "a🙃b", 3, UNICODE, 0); + assertStringLocate("a", "a🙃b", 3, UNICODE_CI, 0); + assertStringLocate("🙃", "a🙃b", 1, UTF8_BINARY, 2); + assertStringLocate("🙃", "a🙃b", 1, UTF8_LCASE, 2); + assertStringLocate("🙃", "a🙃b", 1, UNICODE, 2); + assertStringLocate("🙃", "a🙃b", 1, UNICODE_CI, 2); + assertStringLocate("🙃", "a🙃b", 2, UTF8_BINARY, 2); + assertStringLocate("🙃", "a🙃b", 2, UTF8_LCASE, 2); + assertStringLocate("🙃", "a🙃b", 2, UNICODE, 2); + assertStringLocate("🙃", "a🙃b", 2, UNICODE_CI, 2); + assertStringLocate("🙃", "a🙃b", 3, UTF8_BINARY, 0); + assertStringLocate("🙃", "a🙃b", 3, UTF8_LCASE, 0); + assertStringLocate("🙃", "a🙃b", 3, UNICODE, 0); + assertStringLocate("🙃", "a🙃b", 3, UNICODE_CI, 0); + assertStringLocate("b", "a🙃b", 1, UTF8_BINARY, 3); + assertStringLocate("b", "a🙃b", 1, UTF8_LCASE, 3); + assertStringLocate("b", "a🙃b", 1, UNICODE, 3); + assertStringLocate("b", "a🙃b", 1, UNICODE_CI, 3); + assertStringLocate("b", "a🙃b", 2, UTF8_BINARY, 3); + assertStringLocate("b", "a🙃b", 2, UTF8_LCASE, 3); + assertStringLocate("b", "a🙃b", 2, UNICODE, 3); + assertStringLocate("b", "a🙃b", 2, UNICODE_CI, 3); + assertStringLocate("b", "a🙃b", 3, UTF8_BINARY, 3); + assertStringLocate("b", "a🙃b", 3, UTF8_LCASE, 3); + assertStringLocate("b", "a🙃b", 3, UNICODE, 3); + assertStringLocate("b", "a🙃b", 3, UNICODE_CI, 3); + assertStringLocate("🙃", "a🙃🙃b", 1, UTF8_BINARY, 2); + assertStringLocate("🙃", "a🙃🙃b", 1, UTF8_LCASE, 2); + assertStringLocate("🙃", "a🙃🙃b", 1, UNICODE, 2); + assertStringLocate("🙃", "a🙃🙃b", 1, UNICODE_CI, 2); + assertStringLocate("🙃", "a🙃🙃b", 2, UTF8_BINARY, 2); + assertStringLocate("🙃", "a🙃🙃b", 2, UTF8_LCASE, 2); + assertStringLocate("🙃", "a🙃🙃b", 2, UNICODE, 2); + assertStringLocate("🙃", "a🙃🙃b", 2, UNICODE_CI, 2); + assertStringLocate("🙃", "a🙃🙃b", 3, UTF8_BINARY, 3); + assertStringLocate("🙃", "a🙃🙃b", 3, UTF8_LCASE, 3); + assertStringLocate("🙃", "a🙃🙃b", 3, UNICODE, 3); + assertStringLocate("🙃", "a🙃🙃b", 3, UNICODE_CI, 3); + assertStringLocate("🙃", "a🙃🙃b", 4, UTF8_BINARY, 0); + assertStringLocate("🙃", "a🙃🙃b", 4, UTF8_LCASE, 0); + assertStringLocate("🙃", "a🙃🙃b", 4, UNICODE, 0); + assertStringLocate("🙃", "a🙃🙃b", 4, UNICODE_CI, 0); + assertStringLocate("b", "a🙃🙃b", 1, UTF8_BINARY, 4); + assertStringLocate("b", "a🙃🙃b", 1, UTF8_LCASE, 4); + assertStringLocate("b", "a🙃🙃b", 1, UNICODE, 4); + assertStringLocate("b", "a🙃🙃b", 1, UNICODE_CI, 4); + assertStringLocate("b", "a🙃🙃b", 2, UTF8_BINARY, 4); + assertStringLocate("b", "a🙃🙃b", 2, UTF8_LCASE, 4); + assertStringLocate("b", "a🙃🙃b", 2, UNICODE, 4); + assertStringLocate("b", "a🙃🙃b", 2, UNICODE_CI, 4); + assertStringLocate("b", "a🙃🙃b", 3, UTF8_BINARY, 4); + assertStringLocate("b", "a🙃🙃b", 3, UTF8_LCASE, 4); + assertStringLocate("b", "a🙃🙃b", 3, UNICODE, 4); + assertStringLocate("b", "a🙃🙃b", 3, UNICODE_CI, 4); + assertStringLocate("b", "a🙃🙃b", 4, UTF8_BINARY, 4); + assertStringLocate("b", "a🙃🙃b", 4, UTF8_LCASE, 4); + assertStringLocate("b", "a🙃🙃b", 4, UNICODE, 4); + assertStringLocate("b", "a🙃🙃b", 4, UNICODE_CI, 4); + assertStringLocate("b", "a🙃x🙃b", 1, UTF8_BINARY, 5); + assertStringLocate("b", "a🙃x🙃b", 1, UTF8_LCASE, 5); + assertStringLocate("b", "a🙃x🙃b", 1, UNICODE, 5); + assertStringLocate("b", "a🙃x🙃b", 1, UNICODE_CI, 5); + assertStringLocate("b", "a🙃x🙃b", 2, UTF8_BINARY, 5); + assertStringLocate("b", "a🙃x🙃b", 2, UTF8_LCASE, 5); + assertStringLocate("b", "a🙃x🙃b", 2, UNICODE, 5); + assertStringLocate("b", "a🙃x🙃b", 2, UNICODE_CI, 5); + assertStringLocate("b", "a🙃x🙃b", 3, UTF8_BINARY, 5); + assertStringLocate("b", "a🙃x🙃b", 3, UTF8_LCASE, 5); + assertStringLocate("b", "a🙃x🙃b", 3, UNICODE, 5); + assertStringLocate("b", "a🙃x🙃b", 3, UNICODE_CI, 5); + assertStringLocate("b", "a🙃x🙃b", 4, UTF8_BINARY, 5); + assertStringLocate("b", "a🙃x🙃b", 4, UTF8_LCASE, 5); + assertStringLocate("b", "a🙃x🙃b", 4, UNICODE, 5); + assertStringLocate("b", "a🙃x🙃b", 4, UNICODE_CI, 5); // Out of bounds test cases. - assertStringLocate("a", "asd", 4, "UTF8_BINARY", 0); - assertStringLocate("a", "asd", 4, "UTF8_LCASE", 0); - assertStringLocate("a", "asd", 4, "UNICODE", 0); - assertStringLocate("a", "asd", 4, "UNICODE_CI", 0); - assertStringLocate("a", "asd", 100, "UTF8_BINARY", 0); - assertStringLocate("a", "asd", 100, "UTF8_LCASE", 0); - assertStringLocate("a", "asd", 100, "UNICODE", 0); - assertStringLocate("a", "asd", 100, "UNICODE_CI", 0); - assertStringLocate("a", "🙃🙃", 4, "UTF8_BINARY", 0); - assertStringLocate("a", "🙃🙃", 4, "UTF8_LCASE", 0); - assertStringLocate("a", "🙃🙃", 4, "UNICODE", 0); - assertStringLocate("a", "🙃🙃", 4, "UNICODE_CI", 0); - assertStringLocate("", "asd", 100, "UTF8_BINARY", 1); - assertStringLocate("", "asd", 100, "UTF8_LCASE", 1); - assertStringLocate("", "asd", 100, "UNICODE", 1); - assertStringLocate("", "asd", 100, "UNICODE_CI", 1); - assertStringLocate("asd", "", 100, "UTF8_BINARY", 0); - assertStringLocate("asd", "", 100, "UTF8_LCASE", 0); - assertStringLocate("asd", "", 100, "UNICODE", 0); - assertStringLocate("asd", "", 100, "UNICODE_CI", 0); + assertStringLocate("a", "asd", 4, UTF8_BINARY, 0); + assertStringLocate("a", "asd", 4, UTF8_LCASE, 0); + assertStringLocate("a", "asd", 4, UNICODE, 0); + assertStringLocate("a", "asd", 4, UNICODE_CI, 0); + assertStringLocate("a", "asd", 100, UTF8_BINARY, 0); + assertStringLocate("a", "asd", 100, UTF8_LCASE, 0); + assertStringLocate("a", "asd", 100, UNICODE, 0); + assertStringLocate("a", "asd", 100, UNICODE_CI, 0); + assertStringLocate("a", "🙃🙃", 4, UTF8_BINARY, 0); + assertStringLocate("a", "🙃🙃", 4, UTF8_LCASE, 0); + assertStringLocate("a", "🙃🙃", 4, UNICODE, 0); + assertStringLocate("a", "🙃🙃", 4, UNICODE_CI, 0); + assertStringLocate("", "asd", 100, UTF8_BINARY, 1); + assertStringLocate("", "asd", 100, UTF8_LCASE, 1); + assertStringLocate("", "asd", 100, UNICODE, 1); + assertStringLocate("", "asd", 100, UNICODE_CI, 1); + assertStringLocate("asd", "", 100, UTF8_BINARY, 0); + assertStringLocate("asd", "", 100, UTF8_LCASE, 0); + assertStringLocate("asd", "", 100, UNICODE, 0); + assertStringLocate("asd", "", 100, UNICODE_CI, 0); } /** @@ -2450,292 +2451,292 @@ private void assertSubstringIndex(String string, String delimiter, int count, @Test public void testSubstringIndex() throws SparkException { // Empty strings. - assertSubstringIndex("", "", 0, "UTF8_BINARY", ""); - assertSubstringIndex("", "", 0, "UTF8_LCASE", ""); - assertSubstringIndex("", "", 0, "UNICODE", ""); - assertSubstringIndex("", "", 0, "UNICODE_CI", ""); - assertSubstringIndex("", "", 1, "UTF8_BINARY", ""); - assertSubstringIndex("", "", 1, "UTF8_LCASE", ""); - assertSubstringIndex("", "", 1, "UNICODE", ""); - assertSubstringIndex("", "", 1, "UNICODE_CI", ""); - assertSubstringIndex("", "", -1, "UTF8_BINARY", ""); - assertSubstringIndex("", "", -1, "UTF8_LCASE", ""); - assertSubstringIndex("", "", -1, "UNICODE", ""); - assertSubstringIndex("", "", -1, "UNICODE_CI", ""); - assertSubstringIndex("", "x", 0, "UTF8_BINARY", ""); - assertSubstringIndex("", "x", 0, "UTF8_LCASE", ""); - assertSubstringIndex("", "x", 0, "UNICODE", ""); - assertSubstringIndex("", "x", 0, "UNICODE_CI", ""); - assertSubstringIndex("", "x", 1, "UTF8_BINARY", ""); - assertSubstringIndex("", "x", 1, "UTF8_LCASE", ""); - assertSubstringIndex("", "x", 1, "UNICODE", ""); - assertSubstringIndex("", "x", 1, "UNICODE_CI", ""); - assertSubstringIndex("", "x", -1, "UTF8_BINARY", ""); - assertSubstringIndex("", "x", -1, "UTF8_LCASE", ""); - assertSubstringIndex("", "x", -1, "UNICODE", ""); - assertSubstringIndex("", "x", -1, "UNICODE_CI", ""); - assertSubstringIndex("abc", "", 0, "UTF8_BINARY", ""); - assertSubstringIndex("abc", "", 0, "UTF8_LCASE", ""); - assertSubstringIndex("abc", "", 0, "UNICODE", ""); - assertSubstringIndex("abc", "", 0, "UNICODE_CI", ""); - assertSubstringIndex("abc", "", 1, "UTF8_BINARY", ""); - assertSubstringIndex("abc", "", 1, "UTF8_LCASE", ""); - assertSubstringIndex("abc", "", 1, "UNICODE", ""); - assertSubstringIndex("abc", "", 1, "UNICODE_CI", ""); - assertSubstringIndex("abc", "", -1, "UTF8_BINARY", ""); - assertSubstringIndex("abc", "", -1, "UTF8_LCASE", ""); - assertSubstringIndex("abc", "", -1, "UNICODE", ""); - assertSubstringIndex("abc", "", -1, "UNICODE_CI", ""); + assertSubstringIndex("", "", 0, UTF8_BINARY, ""); + assertSubstringIndex("", "", 0, UTF8_LCASE, ""); + assertSubstringIndex("", "", 0, UNICODE, ""); + assertSubstringIndex("", "", 0, UNICODE_CI, ""); + assertSubstringIndex("", "", 1, UTF8_BINARY, ""); + assertSubstringIndex("", "", 1, UTF8_LCASE, ""); + assertSubstringIndex("", "", 1, UNICODE, ""); + assertSubstringIndex("", "", 1, UNICODE_CI, ""); + assertSubstringIndex("", "", -1, UTF8_BINARY, ""); + assertSubstringIndex("", "", -1, UTF8_LCASE, ""); + assertSubstringIndex("", "", -1, UNICODE, ""); + assertSubstringIndex("", "", -1, UNICODE_CI, ""); + assertSubstringIndex("", "x", 0, UTF8_BINARY, ""); + assertSubstringIndex("", "x", 0, UTF8_LCASE, ""); + assertSubstringIndex("", "x", 0, UNICODE, ""); + assertSubstringIndex("", "x", 0, UNICODE_CI, ""); + assertSubstringIndex("", "x", 1, UTF8_BINARY, ""); + assertSubstringIndex("", "x", 1, UTF8_LCASE, ""); + assertSubstringIndex("", "x", 1, UNICODE, ""); + assertSubstringIndex("", "x", 1, UNICODE_CI, ""); + assertSubstringIndex("", "x", -1, UTF8_BINARY, ""); + assertSubstringIndex("", "x", -1, UTF8_LCASE, ""); + assertSubstringIndex("", "x", -1, UNICODE, ""); + assertSubstringIndex("", "x", -1, UNICODE_CI, ""); + assertSubstringIndex("abc", "", 0, UTF8_BINARY, ""); + assertSubstringIndex("abc", "", 0, UTF8_LCASE, ""); + assertSubstringIndex("abc", "", 0, UNICODE, ""); + assertSubstringIndex("abc", "", 0, UNICODE_CI, ""); + assertSubstringIndex("abc", "", 1, UTF8_BINARY, ""); + assertSubstringIndex("abc", "", 1, UTF8_LCASE, ""); + assertSubstringIndex("abc", "", 1, UNICODE, ""); + assertSubstringIndex("abc", "", 1, UNICODE_CI, ""); + assertSubstringIndex("abc", "", -1, UTF8_BINARY, ""); + assertSubstringIndex("abc", "", -1, UTF8_LCASE, ""); + assertSubstringIndex("abc", "", -1, UNICODE, ""); + assertSubstringIndex("abc", "", -1, UNICODE_CI, ""); // Basic tests. - assertSubstringIndex("axbxc", "a", 1, "UTF8_BINARY", ""); - assertSubstringIndex("axbxc", "a", 1, "UTF8_LCASE", ""); - assertSubstringIndex("axbxc", "a", 1, "UNICODE", ""); - assertSubstringIndex("axbxc", "a", 1, "UNICODE_CI", ""); - assertSubstringIndex("axbxc", "x", 1, "UTF8_BINARY", "a"); - assertSubstringIndex("axbxc", "x", 1, "UTF8_LCASE", "a"); - assertSubstringIndex("axbxc", "x", 1, "UNICODE", "a"); - assertSubstringIndex("axbxc", "x", 1, "UNICODE_CI", "a"); - assertSubstringIndex("axbxc", "b", 1, "UTF8_BINARY", "ax"); - assertSubstringIndex("axbxc", "b", 1, "UTF8_LCASE", "ax"); - assertSubstringIndex("axbxc", "b", 1, "UNICODE", "ax"); - assertSubstringIndex("axbxc", "b", 1, "UNICODE_CI", "ax"); - assertSubstringIndex("axbxc", "x", 2, "UTF8_BINARY", "axb"); - assertSubstringIndex("axbxc", "x", 2, "UTF8_LCASE", "axb"); - assertSubstringIndex("axbxc", "x", 2, "UNICODE", "axb"); - assertSubstringIndex("axbxc", "x", 2, "UNICODE_CI", "axb"); - assertSubstringIndex("axbxc", "c", 1, "UTF8_BINARY", "axbx"); - assertSubstringIndex("axbxc", "c", 1, "UTF8_LCASE", "axbx"); - assertSubstringIndex("axbxc", "c", 1, "UNICODE", "axbx"); - assertSubstringIndex("axbxc", "c", 1, "UNICODE_CI", "axbx"); - assertSubstringIndex("axbxc", "x", 3, "UTF8_BINARY", "axbxc"); - assertSubstringIndex("axbxc", "x", 3, "UTF8_LCASE", "axbxc"); - assertSubstringIndex("axbxc", "x", 3, "UNICODE", "axbxc"); - assertSubstringIndex("axbxc", "x", 3, "UNICODE_CI", "axbxc"); - assertSubstringIndex("axbxc", "d", 1, "UTF8_BINARY", "axbxc"); - assertSubstringIndex("axbxc", "d", 1, "UTF8_LCASE", "axbxc"); - assertSubstringIndex("axbxc", "d", 1, "UNICODE", "axbxc"); - assertSubstringIndex("axbxc", "d", 1, "UNICODE_CI", "axbxc"); - assertSubstringIndex("axbxc", "c", -1, "UTF8_BINARY", ""); - assertSubstringIndex("axbxc", "c", -1, "UTF8_LCASE", ""); - assertSubstringIndex("axbxc", "c", -1, "UNICODE", ""); - assertSubstringIndex("axbxc", "c", -1, "UNICODE_CI", ""); - assertSubstringIndex("axbxc", "x", -1, "UTF8_BINARY", "c"); - assertSubstringIndex("axbxc", "x", -1, "UTF8_LCASE", "c"); - assertSubstringIndex("axbxc", "x", -1, "UNICODE", "c"); - assertSubstringIndex("axbxc", "x", -1, "UNICODE_CI", "c"); - assertSubstringIndex("axbxc", "b", -1, "UTF8_BINARY", "xc"); - assertSubstringIndex("axbxc", "b", -1, "UTF8_LCASE", "xc"); - assertSubstringIndex("axbxc", "b", -1, "UNICODE", "xc"); - assertSubstringIndex("axbxc", "b", -1, "UNICODE_CI", "xc"); - assertSubstringIndex("axbxc", "x", -2, "UTF8_BINARY", "bxc"); - assertSubstringIndex("axbxc", "x", -2, "UTF8_LCASE", "bxc"); - assertSubstringIndex("axbxc", "x", -2, "UNICODE", "bxc"); - assertSubstringIndex("axbxc", "x", -2, "UNICODE_CI", "bxc"); - assertSubstringIndex("axbxc", "a", -1, "UTF8_BINARY", "xbxc"); - assertSubstringIndex("axbxc", "a", -1, "UTF8_LCASE", "xbxc"); - assertSubstringIndex("axbxc", "a", -1, "UNICODE", "xbxc"); - assertSubstringIndex("axbxc", "a", -1, "UNICODE_CI", "xbxc"); - assertSubstringIndex("axbxc", "x", -3, "UTF8_BINARY", "axbxc"); - assertSubstringIndex("axbxc", "x", -3, "UTF8_LCASE", "axbxc"); - assertSubstringIndex("axbxc", "x", -3, "UNICODE", "axbxc"); - assertSubstringIndex("axbxc", "x", -3, "UNICODE_CI", "axbxc"); - assertSubstringIndex("axbxc", "d", -1, "UTF8_BINARY", "axbxc"); - assertSubstringIndex("axbxc", "d", -1, "UTF8_LCASE", "axbxc"); - assertSubstringIndex("axbxc", "d", -1, "UNICODE", "axbxc"); - assertSubstringIndex("axbxc", "d", -1, "UNICODE_CI", "axbxc"); + assertSubstringIndex("axbxc", "a", 1, UTF8_BINARY, ""); + assertSubstringIndex("axbxc", "a", 1, UTF8_LCASE, ""); + assertSubstringIndex("axbxc", "a", 1, UNICODE, ""); + assertSubstringIndex("axbxc", "a", 1, UNICODE_CI, ""); + assertSubstringIndex("axbxc", "x", 1, UTF8_BINARY, "a"); + assertSubstringIndex("axbxc", "x", 1, UTF8_LCASE, "a"); + assertSubstringIndex("axbxc", "x", 1, UNICODE, "a"); + assertSubstringIndex("axbxc", "x", 1, UNICODE_CI, "a"); + assertSubstringIndex("axbxc", "b", 1, UTF8_BINARY, "ax"); + assertSubstringIndex("axbxc", "b", 1, UTF8_LCASE, "ax"); + assertSubstringIndex("axbxc", "b", 1, UNICODE, "ax"); + assertSubstringIndex("axbxc", "b", 1, UNICODE_CI, "ax"); + assertSubstringIndex("axbxc", "x", 2, UTF8_BINARY, "axb"); + assertSubstringIndex("axbxc", "x", 2, UTF8_LCASE, "axb"); + assertSubstringIndex("axbxc", "x", 2, UNICODE, "axb"); + assertSubstringIndex("axbxc", "x", 2, UNICODE_CI, "axb"); + assertSubstringIndex("axbxc", "c", 1, UTF8_BINARY, "axbx"); + assertSubstringIndex("axbxc", "c", 1, UTF8_LCASE, "axbx"); + assertSubstringIndex("axbxc", "c", 1, UNICODE, "axbx"); + assertSubstringIndex("axbxc", "c", 1, UNICODE_CI, "axbx"); + assertSubstringIndex("axbxc", "x", 3, UTF8_BINARY, "axbxc"); + assertSubstringIndex("axbxc", "x", 3, UTF8_LCASE, "axbxc"); + assertSubstringIndex("axbxc", "x", 3, UNICODE, "axbxc"); + assertSubstringIndex("axbxc", "x", 3, UNICODE_CI, "axbxc"); + assertSubstringIndex("axbxc", "d", 1, UTF8_BINARY, "axbxc"); + assertSubstringIndex("axbxc", "d", 1, UTF8_LCASE, "axbxc"); + assertSubstringIndex("axbxc", "d", 1, UNICODE, "axbxc"); + assertSubstringIndex("axbxc", "d", 1, UNICODE_CI, "axbxc"); + assertSubstringIndex("axbxc", "c", -1, UTF8_BINARY, ""); + assertSubstringIndex("axbxc", "c", -1, UTF8_LCASE, ""); + assertSubstringIndex("axbxc", "c", -1, UNICODE, ""); + assertSubstringIndex("axbxc", "c", -1, UNICODE_CI, ""); + assertSubstringIndex("axbxc", "x", -1, UTF8_BINARY, "c"); + assertSubstringIndex("axbxc", "x", -1, UTF8_LCASE, "c"); + assertSubstringIndex("axbxc", "x", -1, UNICODE, "c"); + assertSubstringIndex("axbxc", "x", -1, UNICODE_CI, "c"); + assertSubstringIndex("axbxc", "b", -1, UTF8_BINARY, "xc"); + assertSubstringIndex("axbxc", "b", -1, UTF8_LCASE, "xc"); + assertSubstringIndex("axbxc", "b", -1, UNICODE, "xc"); + assertSubstringIndex("axbxc", "b", -1, UNICODE_CI, "xc"); + assertSubstringIndex("axbxc", "x", -2, UTF8_BINARY, "bxc"); + assertSubstringIndex("axbxc", "x", -2, UTF8_LCASE, "bxc"); + assertSubstringIndex("axbxc", "x", -2, UNICODE, "bxc"); + assertSubstringIndex("axbxc", "x", -2, UNICODE_CI, "bxc"); + assertSubstringIndex("axbxc", "a", -1, UTF8_BINARY, "xbxc"); + assertSubstringIndex("axbxc", "a", -1, UTF8_LCASE, "xbxc"); + assertSubstringIndex("axbxc", "a", -1, UNICODE, "xbxc"); + assertSubstringIndex("axbxc", "a", -1, UNICODE_CI, "xbxc"); + assertSubstringIndex("axbxc", "x", -3, UTF8_BINARY, "axbxc"); + assertSubstringIndex("axbxc", "x", -3, UTF8_LCASE, "axbxc"); + assertSubstringIndex("axbxc", "x", -3, UNICODE, "axbxc"); + assertSubstringIndex("axbxc", "x", -3, UNICODE_CI, "axbxc"); + assertSubstringIndex("axbxc", "d", -1, UTF8_BINARY, "axbxc"); + assertSubstringIndex("axbxc", "d", -1, UTF8_LCASE, "axbxc"); + assertSubstringIndex("axbxc", "d", -1, UNICODE, "axbxc"); + assertSubstringIndex("axbxc", "d", -1, UNICODE_CI, "axbxc"); // Advanced tests. - 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_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"); - assertSubstringIndex("wwwYapacheYorg", "Y", 1, "UNICODE", "www"); - assertSubstringIndex("wwwYapacheYorg", "y", 1, "UNICODE", "wwwYapacheYorg"); - assertSubstringIndex("wwwGapacheGorg", "g", 1, "UNICODE", "wwwGapacheGor"); - assertSubstringIndex("GwwwGapacheGorG", "G", 3, "UNICODE", "GwwwGapache"); - assertSubstringIndex("wwwGapacheGorG", "G", -3, "UNICODE", "apacheGorG"); - assertSubstringIndex("www.apache.org", ".", 0, "UNICODE", ""); - assertSubstringIndex("www.apache.org", ".", -3, "UNICODE", "www.apache.org"); - assertSubstringIndex("www.apache.org", ".", -2, "UNICODE", "apache.org"); - assertSubstringIndex("www.apache.org", ".", -1, "UNICODE", "org"); - assertSubstringIndex("", ".", -2, "UNICODE", ""); - assertSubstringIndex("test大千世界X大千世界", "X", -1, "UNICODE", "大千世界"); - assertSubstringIndex("test大千世界X大千世界", "X", 1, "UNICODE", "test大千世界"); - assertSubstringIndex("大x千世界大千世x界", "x", 1, "UNICODE", "大"); - assertSubstringIndex("大x千世界大千世x界", "x", -1, "UNICODE", "界"); - assertSubstringIndex("大x千世界大千世x界", "x", -2, "UNICODE", "千世界大千世x界"); - assertSubstringIndex("大千世界大千世界", "千", 2, "UNICODE", "大千世界大"); - assertSubstringIndex("www||apache||org", "||", 2, "UNICODE", "www||apache"); - assertSubstringIndex("AaAaAaAaAa", "aa", 2, "UNICODE_CI", "A"); - assertSubstringIndex("www.apache.org", ".", 3, "UNICODE_CI", "www.apache.org"); - assertSubstringIndex("wwwXapacheXorg", "x", 2, "UNICODE_CI", "wwwXapache"); - assertSubstringIndex("wwwxapacheXorg", "X", 1, "UNICODE_CI", "www"); - assertSubstringIndex("www.apache.org", ".", 0, "UNICODE_CI", ""); - assertSubstringIndex("wwwGapacheGorg", "G", 3, "UNICODE_CI", "wwwGapacheGor"); - assertSubstringIndex("gwwwGapacheGorg", "g", 3, "UNICODE_CI", "gwwwGapache"); - assertSubstringIndex("gwwwGapacheGorg", "g", -3, "UNICODE_CI", "apacheGorg"); - assertSubstringIndex("www.apache.ORG", ".", -3, "UNICODE_CI", "www.apache.ORG"); - assertSubstringIndex("wwwmapacheMorg", "M", -2, "UNICODE_CI", "apacheMorg"); - assertSubstringIndex("www.apache.org", ".", -1, "UNICODE_CI", "org"); - assertSubstringIndex("", ".", -2, "UNICODE_CI", ""); - assertSubstringIndex("test大千世界X大千世界", "X", -1, "UNICODE_CI", "大千世界"); - assertSubstringIndex("test大千世界X大千世界", "X", 1, "UNICODE_CI", "test大千世界"); - assertSubstringIndex("test大千世界大千世界", "千", 2, "UNICODE_CI", "test大千世界大"); - assertSubstringIndex("www||APACHE||org", "||", 2, "UNICODE_CI", "www||APACHE"); + 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_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"); + assertSubstringIndex("wwwYapacheYorg", "Y", 1, UNICODE, "www"); + assertSubstringIndex("wwwYapacheYorg", "y", 1, UNICODE, "wwwYapacheYorg"); + assertSubstringIndex("wwwGapacheGorg", "g", 1, UNICODE, "wwwGapacheGor"); + assertSubstringIndex("GwwwGapacheGorG", "G", 3, UNICODE, "GwwwGapache"); + assertSubstringIndex("wwwGapacheGorG", "G", -3, UNICODE, "apacheGorG"); + assertSubstringIndex("www.apache.org", ".", 0, UNICODE, ""); + assertSubstringIndex("www.apache.org", ".", -3, UNICODE, "www.apache.org"); + assertSubstringIndex("www.apache.org", ".", -2, UNICODE, "apache.org"); + assertSubstringIndex("www.apache.org", ".", -1, UNICODE, "org"); + assertSubstringIndex("", ".", -2, UNICODE, ""); + assertSubstringIndex("test大千世界X大千世界", "X", -1, UNICODE, "大千世界"); + assertSubstringIndex("test大千世界X大千世界", "X", 1, UNICODE, "test大千世界"); + assertSubstringIndex("大x千世界大千世x界", "x", 1, UNICODE, "大"); + assertSubstringIndex("大x千世界大千世x界", "x", -1, UNICODE, "界"); + assertSubstringIndex("大x千世界大千世x界", "x", -2, UNICODE, "千世界大千世x界"); + assertSubstringIndex("大千世界大千世界", "千", 2, UNICODE, "大千世界大"); + assertSubstringIndex("www||apache||org", "||", 2, UNICODE, "www||apache"); + assertSubstringIndex("AaAaAaAaAa", "aa", 2, UNICODE_CI, "A"); + assertSubstringIndex("www.apache.org", ".", 3, UNICODE_CI, "www.apache.org"); + assertSubstringIndex("wwwXapacheXorg", "x", 2, UNICODE_CI, "wwwXapache"); + assertSubstringIndex("wwwxapacheXorg", "X", 1, UNICODE_CI, "www"); + assertSubstringIndex("www.apache.org", ".", 0, UNICODE_CI, ""); + assertSubstringIndex("wwwGapacheGorg", "G", 3, UNICODE_CI, "wwwGapacheGor"); + assertSubstringIndex("gwwwGapacheGorg", "g", 3, UNICODE_CI, "gwwwGapache"); + assertSubstringIndex("gwwwGapacheGorg", "g", -3, UNICODE_CI, "apacheGorg"); + assertSubstringIndex("www.apache.ORG", ".", -3, UNICODE_CI, "www.apache.ORG"); + assertSubstringIndex("wwwmapacheMorg", "M", -2, UNICODE_CI, "apacheMorg"); + assertSubstringIndex("www.apache.org", ".", -1, UNICODE_CI, "org"); + assertSubstringIndex("", ".", -2, UNICODE_CI, ""); + assertSubstringIndex("test大千世界X大千世界", "X", -1, UNICODE_CI, "大千世界"); + assertSubstringIndex("test大千世界X大千世界", "X", 1, UNICODE_CI, "test大千世界"); + assertSubstringIndex("test大千世界大千世界", "千", 2, UNICODE_CI, "test大千世界大"); + assertSubstringIndex("www||APACHE||org", "||", 2, UNICODE_CI, "www||APACHE"); assertSubstringIndex("wwwèapacheËorg", "Ê", -3, "AF_CI_AI", "apacheËorg"); // One-to-many case mapping (e.g. Turkish dotted I). - assertSubstringIndex("abİo12", "i\u0307o", 1, "UNICODE_CI", "ab"); - assertSubstringIndex("abİo12", "i\u0307o", -1, "UNICODE_CI", "12"); - assertSubstringIndex("abi̇o12", "İo", 1, "UNICODE_CI", "ab"); - assertSubstringIndex("abi̇o12", "İo", -1, "UNICODE_CI", "12"); - assertSubstringIndex("ai̇bi̇o12", "İo", 1, "UNICODE_CI", "ai̇b"); - assertSubstringIndex("ai̇bi̇o12i̇o", "İo", 2, "UNICODE_CI", "ai̇bi̇o12"); - assertSubstringIndex("ai̇bi̇o12i̇o", "İo", -1, "UNICODE_CI", ""); - assertSubstringIndex("ai̇bi̇o12i̇o", "İo", -2, "UNICODE_CI", "12i̇o"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", -4, "UNICODE_CI", "İo12İoi̇o"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i\u0307o", -4, "UNICODE_CI", "İo12İoi̇o"); - assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", -4, "UNICODE_CI", "i̇o12i̇oİo"); - assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i\u0307o", -4, "UNICODE_CI", "i̇o12i̇oİo"); - assertSubstringIndex("abi̇12", "i", 1, "UNICODE_CI", "abi̇12"); - assertSubstringIndex("abi̇12", "\u0307", 1, "UNICODE_CI", "abi̇12"); - assertSubstringIndex("abi̇12", "İ", 1, "UNICODE_CI", "ab"); - assertSubstringIndex("abİ12", "i", 1, "UNICODE_CI", "abİ12"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", -4, "UNICODE_CI", "İo12İoi̇o"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i\u0307o", -4, "UNICODE_CI", "İo12İoi̇o"); - assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", -4, "UNICODE_CI", "i̇o12i̇oİo"); - assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i\u0307o", -4, "UNICODE_CI", "i̇o12i̇oİo"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", 3, "UNICODE_CI", "ai̇bi̇oİo12"); - assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i\u0307o", 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\u0307o", 3, "UNICODE_CI", "ai̇bİoi̇o12"); - 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\u0307o", -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\u0307o", -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\u0307o", 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\u0307o", 3, "UTF8_LCASE", "ai̇bİoi̇o12"); - assertSubstringIndex("bİoi̇o12i̇o", "\u0307oi", 1, "UTF8_LCASE", "bİoi̇o12i̇o"); + assertSubstringIndex("abİo12", "i\u0307o", 1, UNICODE_CI, "ab"); + assertSubstringIndex("abİo12", "i\u0307o", -1, UNICODE_CI, "12"); + assertSubstringIndex("abi̇o12", "İo", 1, UNICODE_CI, "ab"); + assertSubstringIndex("abi̇o12", "İo", -1, UNICODE_CI, "12"); + assertSubstringIndex("ai̇bi̇o12", "İo", 1, UNICODE_CI, "ai̇b"); + assertSubstringIndex("ai̇bi̇o12i̇o", "İo", 2, UNICODE_CI, "ai̇bi̇o12"); + assertSubstringIndex("ai̇bi̇o12i̇o", "İo", -1, UNICODE_CI, ""); + assertSubstringIndex("ai̇bi̇o12i̇o", "İo", -2, UNICODE_CI, "12i̇o"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", -4, UNICODE_CI, "İo12İoi̇o"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i\u0307o", -4, UNICODE_CI, "İo12İoi̇o"); + assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", -4, UNICODE_CI, "i̇o12i̇oİo"); + assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i\u0307o", -4, UNICODE_CI, "i̇o12i̇oİo"); + assertSubstringIndex("abi̇12", "i", 1, UNICODE_CI, "abi̇12"); + assertSubstringIndex("abi̇12", "\u0307", 1, UNICODE_CI, "abi̇12"); + assertSubstringIndex("abi̇12", "İ", 1, UNICODE_CI, "ab"); + assertSubstringIndex("abİ12", "i", 1, UNICODE_CI, "abİ12"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", -4, UNICODE_CI, "İo12İoi̇o"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i\u0307o", -4, UNICODE_CI, "İo12İoi̇o"); + assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "İo", -4, UNICODE_CI, "i̇o12i̇oİo"); + assertSubstringIndex("ai̇bİoi̇o12i̇oİo", "i\u0307o", -4, UNICODE_CI, "i̇o12i̇oİo"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "İo", 3, UNICODE_CI, "ai̇bi̇oİo12"); + assertSubstringIndex("ai̇bi̇oİo12İoi̇o", "i\u0307o", 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\u0307o", 3, UNICODE_CI, "ai̇bİoi̇o12"); + 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\u0307o", -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\u0307o", -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\u0307o", 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\u0307o", 3, UTF8_LCASE, "ai̇bİoi̇o12"); + assertSubstringIndex("bİoi̇o12i̇o", "\u0307oi", 1, UTF8_LCASE, "bİoi̇o12i̇o"); // Conditional case mapping (e.g. Greek sigmas). - assertSubstringIndex("σ", "σ", 1, "UTF8_BINARY", ""); - assertSubstringIndex("σ", "ς", 1, "UTF8_BINARY", "σ"); - assertSubstringIndex("σ", "Σ", 1, "UTF8_BINARY", "σ"); - assertSubstringIndex("ς", "σ", 1, "UTF8_BINARY", "ς"); - assertSubstringIndex("ς", "ς", 1, "UTF8_BINARY", ""); - assertSubstringIndex("ς", "Σ", 1, "UTF8_BINARY", "ς"); - assertSubstringIndex("Σ", "σ", 1, "UTF8_BINARY", "Σ"); - assertSubstringIndex("Σ", "ς", 1, "UTF8_BINARY", "Σ"); - assertSubstringIndex("Σ", "Σ", 1, "UTF8_BINARY", ""); - assertSubstringIndex("σ", "σ", 1, "UTF8_LCASE", ""); - assertSubstringIndex("σ", "ς", 1, "UTF8_LCASE", ""); - assertSubstringIndex("σ", "Σ", 1, "UTF8_LCASE", ""); - assertSubstringIndex("ς", "σ", 1, "UTF8_LCASE", ""); - assertSubstringIndex("ς", "ς", 1, "UTF8_LCASE", ""); - assertSubstringIndex("ς", "Σ", 1, "UTF8_LCASE", ""); - assertSubstringIndex("Σ", "σ", 1, "UTF8_LCASE", ""); - assertSubstringIndex("Σ", "ς", 1, "UTF8_LCASE", ""); - assertSubstringIndex("Σ", "Σ", 1, "UTF8_LCASE", ""); - assertSubstringIndex("σ", "σ", 1, "UNICODE", ""); - assertSubstringIndex("σ", "ς", 1, "UNICODE", "σ"); - assertSubstringIndex("σ", "Σ", 1, "UNICODE", "σ"); - assertSubstringIndex("ς", "σ", 1, "UNICODE", "ς"); - assertSubstringIndex("ς", "ς", 1, "UNICODE", ""); - assertSubstringIndex("ς", "Σ", 1, "UNICODE", "ς"); - assertSubstringIndex("Σ", "σ", 1, "UNICODE", "Σ"); - assertSubstringIndex("Σ", "ς", 1, "UNICODE", "Σ"); - assertSubstringIndex("Σ", "Σ", 1, "UNICODE", ""); - assertSubstringIndex("σ", "σ", 1, "UNICODE_CI", ""); - assertSubstringIndex("σ", "ς", 1, "UNICODE_CI", ""); - assertSubstringIndex("σ", "Σ", 1, "UNICODE_CI", ""); - assertSubstringIndex("ς", "σ", 1, "UNICODE_CI", ""); - assertSubstringIndex("ς", "ς", 1, "UNICODE_CI", ""); - assertSubstringIndex("ς", "Σ", 1, "UNICODE_CI", ""); - assertSubstringIndex("Σ", "σ", 1, "UNICODE_CI", ""); - assertSubstringIndex("Σ", "ς", 1, "UNICODE_CI", ""); - assertSubstringIndex("Σ", "Σ", 1, "UNICODE_CI", ""); + assertSubstringIndex("σ", "σ", 1, UTF8_BINARY, ""); + assertSubstringIndex("σ", "ς", 1, UTF8_BINARY, "σ"); + assertSubstringIndex("σ", "Σ", 1, UTF8_BINARY, "σ"); + assertSubstringIndex("ς", "σ", 1, UTF8_BINARY, "ς"); + assertSubstringIndex("ς", "ς", 1, UTF8_BINARY, ""); + assertSubstringIndex("ς", "Σ", 1, UTF8_BINARY, "ς"); + assertSubstringIndex("Σ", "σ", 1, UTF8_BINARY, "Σ"); + assertSubstringIndex("Σ", "ς", 1, UTF8_BINARY, "Σ"); + assertSubstringIndex("Σ", "Σ", 1, UTF8_BINARY, ""); + assertSubstringIndex("σ", "σ", 1, UTF8_LCASE, ""); + assertSubstringIndex("σ", "ς", 1, UTF8_LCASE, ""); + assertSubstringIndex("σ", "Σ", 1, UTF8_LCASE, ""); + assertSubstringIndex("ς", "σ", 1, UTF8_LCASE, ""); + assertSubstringIndex("ς", "ς", 1, UTF8_LCASE, ""); + assertSubstringIndex("ς", "Σ", 1, UTF8_LCASE, ""); + assertSubstringIndex("Σ", "σ", 1, UTF8_LCASE, ""); + assertSubstringIndex("Σ", "ς", 1, UTF8_LCASE, ""); + assertSubstringIndex("Σ", "Σ", 1, UTF8_LCASE, ""); + assertSubstringIndex("σ", "σ", 1, UNICODE, ""); + assertSubstringIndex("σ", "ς", 1, UNICODE, "σ"); + assertSubstringIndex("σ", "Σ", 1, UNICODE, "σ"); + assertSubstringIndex("ς", "σ", 1, UNICODE, "ς"); + assertSubstringIndex("ς", "ς", 1, UNICODE, ""); + assertSubstringIndex("ς", "Σ", 1, UNICODE, "ς"); + assertSubstringIndex("Σ", "σ", 1, UNICODE, "Σ"); + assertSubstringIndex("Σ", "ς", 1, UNICODE, "Σ"); + assertSubstringIndex("Σ", "Σ", 1, UNICODE, ""); + assertSubstringIndex("σ", "σ", 1, UNICODE_CI, ""); + assertSubstringIndex("σ", "ς", 1, UNICODE_CI, ""); + assertSubstringIndex("σ", "Σ", 1, UNICODE_CI, ""); + assertSubstringIndex("ς", "σ", 1, UNICODE_CI, ""); + assertSubstringIndex("ς", "ς", 1, UNICODE_CI, ""); + assertSubstringIndex("ς", "Σ", 1, UNICODE_CI, ""); + assertSubstringIndex("Σ", "σ", 1, UNICODE_CI, ""); + assertSubstringIndex("Σ", "ς", 1, UNICODE_CI, ""); + assertSubstringIndex("Σ", "Σ", 1, UNICODE_CI, ""); // Surrogate pairs. - assertSubstringIndex("a🙃b🙃c", "a", 1, "UTF8_BINARY", ""); - assertSubstringIndex("a🙃b🙃c", "a", 1, "UTF8_LCASE", ""); - assertSubstringIndex("a🙃b🙃c", "a", 1, "UNICODE", ""); - assertSubstringIndex("a🙃b🙃c", "a", 1, "UNICODE_CI", ""); - assertSubstringIndex("a🙃b🙃c", "🙃", 1, "UTF8_BINARY", "a"); - assertSubstringIndex("a🙃b🙃c", "🙃", 1, "UTF8_LCASE", "a"); - assertSubstringIndex("a🙃b🙃c", "🙃", 1, "UNICODE", "a"); - assertSubstringIndex("a🙃b🙃c", "🙃", 1, "UNICODE_CI", "a"); - assertSubstringIndex("a🙃b🙃c", "b", 1, "UTF8_BINARY", "a🙃"); - assertSubstringIndex("a🙃b🙃c", "b", 1, "UTF8_LCASE", "a🙃"); - assertSubstringIndex("a🙃b🙃c", "b", 1, "UNICODE", "a🙃"); - assertSubstringIndex("a🙃b🙃c", "b", 1, "UNICODE_CI", "a🙃"); - assertSubstringIndex("a🙃b🙃c", "🙃", 2, "UTF8_BINARY", "a🙃b"); - assertSubstringIndex("a🙃b🙃c", "🙃", 2, "UTF8_LCASE", "a🙃b"); - assertSubstringIndex("a🙃b🙃c", "🙃", 2, "UNICODE", "a🙃b"); - assertSubstringIndex("a🙃b🙃c", "🙃", 2, "UNICODE_CI", "a🙃b"); - assertSubstringIndex("a🙃b🙃c", "c", 1, "UTF8_BINARY", "a🙃b🙃"); - assertSubstringIndex("a🙃b🙃c", "c", 1, "UTF8_LCASE", "a🙃b🙃"); - assertSubstringIndex("a🙃b🙃c", "c", 1, "UNICODE", "a🙃b🙃"); - assertSubstringIndex("a🙃b🙃c", "c", 1, "UNICODE_CI", "a🙃b🙃"); - assertSubstringIndex("a🙃b🙃c", "🙃", 3, "UTF8_BINARY", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", 3, "UTF8_LCASE", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", 3, "UNICODE", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", 3, "UNICODE_CI", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "d", 1, "UTF8_BINARY", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "d", 1, "UTF8_LCASE", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "d", 1, "UNICODE", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "d", 1, "UNICODE_CI", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "c", -1, "UTF8_BINARY", ""); - assertSubstringIndex("a🙃b🙃c", "c", -1, "UTF8_LCASE", ""); - assertSubstringIndex("a🙃b🙃c", "c", -1, "UNICODE", ""); - assertSubstringIndex("a🙃b🙃c", "c", -1, "UNICODE_CI", ""); - assertSubstringIndex("a🙃b🙃c", "🙃", -1, "UTF8_BINARY", "c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -1, "UTF8_LCASE", "c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -1, "UNICODE", "c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -1, "UNICODE_CI", "c"); - assertSubstringIndex("a🙃b🙃c", "b", -1, "UTF8_BINARY", "🙃c"); - assertSubstringIndex("a🙃b🙃c", "b", -1, "UTF8_LCASE", "🙃c"); - assertSubstringIndex("a🙃b🙃c", "b", -1, "UNICODE", "🙃c"); - assertSubstringIndex("a🙃b🙃c", "b", -1, "UNICODE_CI", "🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -2, "UTF8_BINARY", "b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -2, "UTF8_LCASE", "b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -2, "UNICODE", "b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -2, "UNICODE_CI", "b🙃c"); - assertSubstringIndex("a🙃b🙃c", "a", -1, "UTF8_BINARY", "🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "a", -1, "UTF8_LCASE", "🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "a", -1, "UNICODE", "🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "a", -1, "UNICODE_CI", "🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -3, "UTF8_BINARY", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -3, "UTF8_LCASE", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -3, "UNICODE", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "🙃", -3, "UNICODE_CI", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "d", -1, "UTF8_BINARY", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "d", -1, "UTF8_LCASE", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "d", -1, "UNICODE", "a🙃b🙃c"); - assertSubstringIndex("a🙃b🙃c", "d", -1, "UNICODE_CI", "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "a", 1, UTF8_BINARY, ""); + assertSubstringIndex("a🙃b🙃c", "a", 1, UTF8_LCASE, ""); + assertSubstringIndex("a🙃b🙃c", "a", 1, UNICODE, ""); + assertSubstringIndex("a🙃b🙃c", "a", 1, UNICODE_CI, ""); + assertSubstringIndex("a🙃b🙃c", "🙃", 1, UTF8_BINARY, "a"); + assertSubstringIndex("a🙃b🙃c", "🙃", 1, UTF8_LCASE, "a"); + assertSubstringIndex("a🙃b🙃c", "🙃", 1, UNICODE, "a"); + assertSubstringIndex("a🙃b🙃c", "🙃", 1, UNICODE_CI, "a"); + assertSubstringIndex("a🙃b🙃c", "b", 1, UTF8_BINARY, "a🙃"); + assertSubstringIndex("a🙃b🙃c", "b", 1, UTF8_LCASE, "a🙃"); + assertSubstringIndex("a🙃b🙃c", "b", 1, UNICODE, "a🙃"); + assertSubstringIndex("a🙃b🙃c", "b", 1, UNICODE_CI, "a🙃"); + assertSubstringIndex("a🙃b🙃c", "🙃", 2, UTF8_BINARY, "a🙃b"); + assertSubstringIndex("a🙃b🙃c", "🙃", 2, UTF8_LCASE, "a🙃b"); + assertSubstringIndex("a🙃b🙃c", "🙃", 2, UNICODE, "a🙃b"); + assertSubstringIndex("a🙃b🙃c", "🙃", 2, UNICODE_CI, "a🙃b"); + assertSubstringIndex("a🙃b🙃c", "c", 1, UTF8_BINARY, "a🙃b🙃"); + assertSubstringIndex("a🙃b🙃c", "c", 1, UTF8_LCASE, "a🙃b🙃"); + assertSubstringIndex("a🙃b🙃c", "c", 1, UNICODE, "a🙃b🙃"); + assertSubstringIndex("a🙃b🙃c", "c", 1, UNICODE_CI, "a🙃b🙃"); + assertSubstringIndex("a🙃b🙃c", "🙃", 3, UTF8_BINARY, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", 3, UTF8_LCASE, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", 3, UNICODE, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", 3, UNICODE_CI, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "d", 1, UTF8_BINARY, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "d", 1, UTF8_LCASE, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "d", 1, UNICODE, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "d", 1, UNICODE_CI, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "c", -1, UTF8_BINARY, ""); + assertSubstringIndex("a🙃b🙃c", "c", -1, UTF8_LCASE, ""); + assertSubstringIndex("a🙃b🙃c", "c", -1, UNICODE, ""); + assertSubstringIndex("a🙃b🙃c", "c", -1, UNICODE_CI, ""); + assertSubstringIndex("a🙃b🙃c", "🙃", -1, UTF8_BINARY, "c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -1, UTF8_LCASE, "c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -1, UNICODE, "c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -1, UNICODE_CI, "c"); + assertSubstringIndex("a🙃b🙃c", "b", -1, UTF8_BINARY, "🙃c"); + assertSubstringIndex("a🙃b🙃c", "b", -1, UTF8_LCASE, "🙃c"); + assertSubstringIndex("a🙃b🙃c", "b", -1, UNICODE, "🙃c"); + assertSubstringIndex("a🙃b🙃c", "b", -1, UNICODE_CI, "🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -2, UTF8_BINARY, "b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -2, UTF8_LCASE, "b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -2, UNICODE, "b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -2, UNICODE_CI, "b🙃c"); + assertSubstringIndex("a🙃b🙃c", "a", -1, UTF8_BINARY, "🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "a", -1, UTF8_LCASE, "🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "a", -1, UNICODE, "🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "a", -1, UNICODE_CI, "🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -3, UTF8_BINARY, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -3, UTF8_LCASE, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -3, UNICODE, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "🙃", -3, UNICODE_CI, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "d", -1, UTF8_BINARY, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "d", -1, UTF8_LCASE, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "d", -1, UNICODE, "a🙃b🙃c"); + assertSubstringIndex("a🙃b🙃c", "d", -1, UNICODE_CI, "a🙃b🙃c"); } /** @@ -2776,279 +2777,279 @@ private void assertStringTrim(String collationName, String sourceString, String @Test public void testStringTrim() throws SparkException { // Basic tests. - assertStringTrim("UTF8_BINARY", "", "", ""); - assertStringTrim("UTF8_BINARY", "", "xyz", ""); - assertStringTrim("UTF8_BINARY", "asd", "", "asd"); - assertStringTrim("UTF8_BINARY", "asd", null, "asd"); - assertStringTrim("UTF8_BINARY", " asd ", null, "asd"); - assertStringTrim("UTF8_BINARY", " a世a ", null, "a世a"); - assertStringTrim("UTF8_BINARY", "asd", "x", "asd"); - assertStringTrim("UTF8_BINARY", "xxasdxx", "x", "asd"); - assertStringTrim("UTF8_BINARY", "xa世ax", "x", "a世a"); - assertStringTrim("UTF8_LCASE", "", "", ""); - assertStringTrim("UTF8_LCASE", "", "xyz", ""); - assertStringTrim("UTF8_LCASE", "asd", "", "asd"); - 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"); - assertStringTrim("UNICODE", "", "", ""); - assertStringTrim("UNICODE", "", "xyz", ""); - assertStringTrim("UNICODE", "asd", "", "asd"); - assertStringTrim("UNICODE", "asd", null, "asd"); - assertStringTrim("UNICODE", " asd ", null, "asd"); - assertStringTrim("UNICODE", " a世a ", null, "a世a"); - assertStringTrim("UNICODE", "asd", "x", "asd"); - assertStringTrim("UNICODE", "xxasdxx", "x", "asd"); - assertStringTrim("UNICODE", "xa世ax", "x", "a世a"); - assertStringTrim("UNICODE_CI", "", "", ""); - assertStringTrim("UNICODE_CI", "", "xyz", ""); - assertStringTrim("UNICODE_CI", "asd", "", "asd"); - assertStringTrim("UNICODE_CI", "asd", null, "asd"); - assertStringTrim("UNICODE_CI", " asd ", null, "asd"); - assertStringTrim("UNICODE_CI", " a世a ", null, "a世a"); - assertStringTrim("UNICODE_CI", "asd", "x", "asd"); - assertStringTrim("UNICODE_CI", "xxasdxx", "x", "asd"); - assertStringTrim("UNICODE_CI", "xa世ax", "x", "a世a"); + assertStringTrim(UTF8_BINARY, "", "", ""); + assertStringTrim(UTF8_BINARY, "", "xyz", ""); + assertStringTrim(UTF8_BINARY, "asd", "", "asd"); + assertStringTrim(UTF8_BINARY, "asd", null, "asd"); + assertStringTrim(UTF8_BINARY, " asd ", null, "asd"); + assertStringTrim(UTF8_BINARY, " a世a ", null, "a世a"); + assertStringTrim(UTF8_BINARY, "asd", "x", "asd"); + assertStringTrim(UTF8_BINARY, "xxasdxx", "x", "asd"); + assertStringTrim(UTF8_BINARY, "xa世ax", "x", "a世a"); + assertStringTrim(UTF8_LCASE, "", "", ""); + assertStringTrim(UTF8_LCASE, "", "xyz", ""); + assertStringTrim(UTF8_LCASE, "asd", "", "asd"); + 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"); + assertStringTrim(UNICODE, "", "", ""); + assertStringTrim(UNICODE, "", "xyz", ""); + assertStringTrim(UNICODE, "asd", "", "asd"); + assertStringTrim(UNICODE, "asd", null, "asd"); + assertStringTrim(UNICODE, " asd ", null, "asd"); + assertStringTrim(UNICODE, " a世a ", null, "a世a"); + assertStringTrim(UNICODE, "asd", "x", "asd"); + assertStringTrim(UNICODE, "xxasdxx", "x", "asd"); + assertStringTrim(UNICODE, "xa世ax", "x", "a世a"); + assertStringTrim(UNICODE_CI, "", "", ""); + assertStringTrim(UNICODE_CI, "", "xyz", ""); + assertStringTrim(UNICODE_CI, "asd", "", "asd"); + assertStringTrim(UNICODE_CI, "asd", null, "asd"); + assertStringTrim(UNICODE_CI, " asd ", null, "asd"); + assertStringTrim(UNICODE_CI, " a世a ", null, "a世a"); + assertStringTrim(UNICODE_CI, "asd", "x", "asd"); + assertStringTrim(UNICODE_CI, "xxasdxx", "x", "asd"); + assertStringTrim(UNICODE_CI, "xa世ax", "x", "a世a"); // Case variation. - assertStringTrim("UTF8_BINARY", "asd", "A", "asd"); - assertStringTrim("UTF8_BINARY", "ddsXXXaa", "asd", "XXX"); - assertStringTrim("UTF8_BINARY", "ASD", "a", "ASD"); - assertStringTrim("UTF8_LCASE", "asd", "A", "sd"); - assertStringTrim("UTF8_LCASE", "ASD", "a", "SD"); - assertStringTrim("UTF8_LCASE", "ddsXXXaa", "ASD", "XXX"); - assertStringTrim("UNICODE", "asd", "A", "asd"); - assertStringTrim("UNICODE", "ASD", "a", "ASD"); - assertStringTrim("UNICODE", "ddsXXXaa", "asd", "XXX"); - assertStringTrim("UNICODE_CI", "asd", "A", "sd"); - assertStringTrim("UNICODE_CI", "ASD", "a", "SD"); - assertStringTrim("UNICODE_CI", "ddsXXXaa", "ASD", "XXX"); + assertStringTrim(UTF8_BINARY, "asd", "A", "asd"); + assertStringTrim(UTF8_BINARY, "ddsXXXaa", "asd", "XXX"); + assertStringTrim(UTF8_BINARY, "ASD", "a", "ASD"); + assertStringTrim(UTF8_LCASE, "asd", "A", "sd"); + assertStringTrim(UTF8_LCASE, "ASD", "a", "SD"); + assertStringTrim(UTF8_LCASE, "ddsXXXaa", "ASD", "XXX"); + assertStringTrim(UNICODE, "asd", "A", "asd"); + assertStringTrim(UNICODE, "ASD", "a", "ASD"); + assertStringTrim(UNICODE, "ddsXXXaa", "asd", "XXX"); + assertStringTrim(UNICODE_CI, "asd", "A", "sd"); + assertStringTrim(UNICODE_CI, "ASD", "a", "SD"); + assertStringTrim(UNICODE_CI, "ddsXXXaa", "ASD", "XXX"); assertStringTrim("SR_CI_AI", "cSCšćČXXXsčšČŠsć", "čš", "XXX"); // One-to-many case mapping (e.g. Turkish dotted I).. - assertStringTrim("UTF8_BINARY", "ẞaaaẞ", "ß", "ẞaaaẞ"); - assertStringTrim("UTF8_BINARY", "ßaaaß", "ẞ", "ßaaaß"); - assertStringTrim("UTF8_BINARY", "Ëaaaẞ", "Ëẞ", "aaa"); - assertStringTrim("UTF8_LCASE", "ẞaaaẞ", "ß", "aaa"); - assertStringTrim("UTF8_LCASE", "ßaaaß", "ẞ", "aaa"); - assertStringTrim("UTF8_LCASE", "Ëaaaẞ", "Ëẞ", "aaa"); - assertStringTrim("UNICODE", "ẞaaaẞ", "ß", "ẞaaaẞ"); - assertStringTrim("UNICODE", "ßaaaß", "ẞ", "ßaaaß"); - assertStringTrim("UNICODE", "Ëaaaẞ", "Ëẞ", "aaa"); - assertStringTrim("UNICODE_CI", "ẞaaaẞ", "ß", "aaa"); - assertStringTrim("UNICODE_CI", "ßaaaß", "ẞ", "aaa"); - assertStringTrim("UNICODE_CI", "Ëaaaẞ", "Ëẞ", "aaa"); + assertStringTrim(UTF8_BINARY, "ẞaaaẞ", "ß", "ẞaaaẞ"); + assertStringTrim(UTF8_BINARY, "ßaaaß", "ẞ", "ßaaaß"); + assertStringTrim(UTF8_BINARY, "Ëaaaẞ", "Ëẞ", "aaa"); + assertStringTrim(UTF8_LCASE, "ẞaaaẞ", "ß", "aaa"); + assertStringTrim(UTF8_LCASE, "ßaaaß", "ẞ", "aaa"); + assertStringTrim(UTF8_LCASE, "Ëaaaẞ", "Ëẞ", "aaa"); + assertStringTrim(UNICODE, "ẞaaaẞ", "ß", "ẞaaaẞ"); + assertStringTrim(UNICODE, "ßaaaß", "ẞ", "ßaaaß"); + assertStringTrim(UNICODE, "Ëaaaẞ", "Ëẞ", "aaa"); + assertStringTrim(UNICODE_CI, "ẞaaaẞ", "ß", "aaa"); + assertStringTrim(UNICODE_CI, "ßaaaß", "ẞ", "aaa"); + assertStringTrim(UNICODE_CI, "Ëaaaẞ", "Ëẞ", "aaa"); // One-to-many case mapping (e.g. Turkish dotted I). - assertStringTrim("UTF8_BINARY", "i", "i", ""); - assertStringTrim("UTF8_BINARY", "iii", "I", "iii"); - assertStringTrim("UTF8_BINARY", "I", "iii", "I"); - assertStringTrim("UTF8_BINARY", "ixi", "i", "x"); - assertStringTrim("UTF8_BINARY", "i", "İ", "i"); - assertStringTrim("UTF8_BINARY", "i\u0307", "İ", "i\u0307"); - assertStringTrim("UTF8_BINARY", "ii\u0307", "İi", "\u0307"); - assertStringTrim("UTF8_BINARY", "iii\u0307", "İi", "\u0307"); - assertStringTrim("UTF8_BINARY", "iiii\u0307", "iİ", "\u0307"); - assertStringTrim("UTF8_BINARY", "ii\u0307ii\u0307", "iİ", "\u0307ii\u0307"); - assertStringTrim("UTF8_BINARY", "i\u0307", "i", "\u0307"); - assertStringTrim("UTF8_BINARY", "i\u0307", "\u0307", "i"); - assertStringTrim("UTF8_BINARY", "i\u0307", "i\u0307", ""); - assertStringTrim("UTF8_BINARY", "i\u0307i\u0307", "i\u0307", ""); - assertStringTrim("UTF8_BINARY", "i\u0307\u0307", "i\u0307", ""); - assertStringTrim("UTF8_BINARY", "i\u0307i", "i\u0307", ""); - assertStringTrim("UTF8_BINARY", "i\u0307i", "İ", "i\u0307i"); - assertStringTrim("UTF8_BINARY", "i\u0307İ", "i\u0307", "İ"); - assertStringTrim("UTF8_BINARY", "i\u0307İ", "İ", "i\u0307"); - assertStringTrim("UTF8_BINARY", "İ", "İ", ""); - assertStringTrim("UTF8_BINARY", "IXi", "İ", "IXi"); - assertStringTrim("UTF8_BINARY", "ix\u0307", "Ixİ", "ix\u0307"); - assertStringTrim("UTF8_BINARY", "i\u0307x", "IXİ", "i\u0307x"); - assertStringTrim("UTF8_BINARY", "i\u0307x", "ix\u0307İ", ""); - assertStringTrim("UTF8_BINARY", "İ", "i", "İ"); - assertStringTrim("UTF8_BINARY", "İ", "\u0307", "İ"); - assertStringTrim("UTF8_BINARY", "Ixİ", "i\u0307", "Ixİ"); - assertStringTrim("UTF8_BINARY", "IXİ", "ix\u0307", "IXİ"); - assertStringTrim("UTF8_BINARY", "xi\u0307", "\u0307IX", "xi"); - assertStringTrim("UTF8_LCASE", "i", "i", ""); - assertStringTrim("UTF8_LCASE", "iii", "I", ""); - assertStringTrim("UTF8_LCASE", "I", "iii", ""); - assertStringTrim("UTF8_LCASE", "ixi", "i", "x"); - assertStringTrim("UTF8_LCASE", "i", "İ", "i"); - assertStringTrim("UTF8_LCASE", "i\u0307", "İ", ""); - assertStringTrim("UTF8_LCASE", "ii\u0307", "İi", ""); - assertStringTrim("UTF8_LCASE", "iii\u0307", "İi", ""); - assertStringTrim("UTF8_LCASE", "iiii\u0307", "iİ", ""); - assertStringTrim("UTF8_LCASE", "ii\u0307ii\u0307", "iİ", ""); - assertStringTrim("UTF8_LCASE", "i\u0307", "i", "\u0307"); - assertStringTrim("UTF8_LCASE", "i\u0307", "\u0307", "i"); - assertStringTrim("UTF8_LCASE", "i\u0307", "i\u0307", ""); - assertStringTrim("UTF8_LCASE", "i\u0307i\u0307", "i\u0307", ""); - assertStringTrim("UTF8_LCASE", "i\u0307\u0307", "i\u0307", ""); - assertStringTrim("UTF8_LCASE", "i\u0307i", "i\u0307", ""); - assertStringTrim("UTF8_LCASE", "i\u0307i", "İ", "i"); - assertStringTrim("UTF8_LCASE", "i\u0307İ", "i\u0307", "İ"); - assertStringTrim("UTF8_LCASE", "i\u0307İ", "İ", ""); - assertStringTrim("UTF8_LCASE", "İ", "İ", ""); - assertStringTrim("UTF8_LCASE", "IXi", "İ", "IXi"); - assertStringTrim("UTF8_LCASE", "ix\u0307", "Ixİ", "\u0307"); - assertStringTrim("UTF8_LCASE", "i\u0307x", "IXİ", ""); - assertStringTrim("UTF8_LCASE", "i\u0307x", "I\u0307xİ", ""); - assertStringTrim("UTF8_LCASE", "İ", "i", "İ"); - assertStringTrim("UTF8_LCASE", "İ", "\u0307", "İ"); - assertStringTrim("UTF8_LCASE", "Ixİ", "i\u0307", "xİ"); - assertStringTrim("UTF8_LCASE", "IXİ", "ix\u0307", "İ"); - assertStringTrim("UTF8_LCASE", "xi\u0307", "\u0307IX", ""); - assertStringTrim("UNICODE", "i", "i", ""); - assertStringTrim("UNICODE", "iii", "I", "iii"); - assertStringTrim("UNICODE", "I", "iii", "I"); - assertStringTrim("UNICODE", "ixi", "i", "x"); - assertStringTrim("UNICODE", "i", "İ", "i"); - assertStringTrim("UNICODE", "i\u0307", "İ", "i\u0307"); - assertStringTrim("UNICODE", "ii\u0307", "İi", "i\u0307"); - assertStringTrim("UNICODE", "iii\u0307", "İi", "i\u0307"); - assertStringTrim("UNICODE", "iiii\u0307", "iİ", "i\u0307"); - assertStringTrim("UNICODE", "ii\u0307ii\u0307", "iİ", "i\u0307ii\u0307"); - assertStringTrim("UNICODE", "i\u0307", "i", "i\u0307"); - assertStringTrim("UNICODE", "i\u0307", "\u0307", "i\u0307"); - assertStringTrim("UNICODE", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrim("UNICODE", "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); - assertStringTrim("UNICODE", "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); - assertStringTrim("UNICODE", "i\u0307i", "i\u0307", "i\u0307"); - assertStringTrim("UNICODE", "i\u0307i", "İ", "i\u0307i"); - assertStringTrim("UNICODE", "i\u0307İ", "i\u0307", "i\u0307İ"); - assertStringTrim("UNICODE", "i\u0307İ", "İ", "i\u0307"); - assertStringTrim("UNICODE", "İ", "İ", ""); - assertStringTrim("UNICODE", "IXi", "İ", "IXi"); - assertStringTrim("UNICODE", "ix\u0307", "Ixİ", "ix\u0307"); - assertStringTrim("UNICODE", "i\u0307x", "IXİ", "i\u0307x"); - assertStringTrim("UNICODE", "i\u0307x", "ix\u0307İ", "i\u0307"); - assertStringTrim("UNICODE", "İ", "i", "İ"); - assertStringTrim("UNICODE", "İ", "\u0307", "İ"); - assertStringTrim("UNICODE", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrim("UNICODE", "Ixİ", "i\u0307", "Ixİ"); - assertStringTrim("UNICODE", "IXİ", "ix\u0307", "IXİ"); - assertStringTrim("UNICODE", "xi\u0307", "\u0307IX", "xi\u0307"); - assertStringTrim("UNICODE_CI", "i", "i", ""); - assertStringTrim("UNICODE_CI", "iii", "I", ""); - assertStringTrim("UNICODE_CI", "I", "iii", ""); - assertStringTrim("UNICODE_CI", "ixi", "i", "x"); - assertStringTrim("UNICODE_CI", "i", "İ", "i"); - assertStringTrim("UNICODE_CI", "i\u0307", "İ", ""); - assertStringTrim("UNICODE_CI", "ii\u0307", "İi", ""); - assertStringTrim("UNICODE_CI", "iii\u0307", "İi", ""); - assertStringTrim("UNICODE_CI", "iiii\u0307", "iİ", ""); - assertStringTrim("UNICODE_CI", "ii\u0307ii\u0307", "iİ", ""); - assertStringTrim("UNICODE_CI", "i\u0307", "i", "i\u0307"); - assertStringTrim("UNICODE_CI", "i\u0307", "\u0307", "i\u0307"); - assertStringTrim("UNICODE_CI", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrim("UNICODE_CI", "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); - assertStringTrim("UNICODE_CI", "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); - assertStringTrim("UNICODE_CI", "i\u0307i", "i\u0307", "i\u0307"); - assertStringTrim("UNICODE_CI", "i\u0307i", "İ", "i"); - assertStringTrim("UNICODE_CI", "i\u0307İ", "i\u0307", "i\u0307İ"); - assertStringTrim("UNICODE_CI", "i\u0307İ", "İ", ""); - assertStringTrim("UNICODE_CI", "İ", "İ", ""); - assertStringTrim("UNICODE_CI", "IXi", "İ", "IXi"); - assertStringTrim("UNICODE_CI", "ix\u0307", "Ixİ", "x\u0307"); - assertStringTrim("UNICODE_CI", "i\u0307x", "IXİ", ""); - assertStringTrim("UNICODE_CI", "i\u0307x", "I\u0307xİ", ""); - assertStringTrim("UNICODE_CI", "İ", "i", "İ"); - assertStringTrim("UNICODE_CI", "İ", "\u0307", "İ"); - assertStringTrim("UNICODE_CI", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrim("UNICODE_CI", "Ixİ", "i\u0307", "xİ"); - assertStringTrim("UNICODE_CI", "IXİ", "ix\u0307", "İ"); - assertStringTrim("UNICODE_CI", "xi\u0307", "\u0307IX", "i\u0307"); + assertStringTrim(UTF8_BINARY, "i", "i", ""); + assertStringTrim(UTF8_BINARY, "iii", "I", "iii"); + assertStringTrim(UTF8_BINARY, "I", "iii", "I"); + assertStringTrim(UTF8_BINARY, "ixi", "i", "x"); + assertStringTrim(UTF8_BINARY, "i", "İ", "i"); + assertStringTrim(UTF8_BINARY, "i\u0307", "İ", "i\u0307"); + assertStringTrim(UTF8_BINARY, "ii\u0307", "İi", "\u0307"); + assertStringTrim(UTF8_BINARY, "iii\u0307", "İi", "\u0307"); + assertStringTrim(UTF8_BINARY, "iiii\u0307", "iİ", "\u0307"); + assertStringTrim(UTF8_BINARY, "ii\u0307ii\u0307", "iİ", "\u0307ii\u0307"); + assertStringTrim(UTF8_BINARY, "i\u0307", "i", "\u0307"); + assertStringTrim(UTF8_BINARY, "i\u0307", "\u0307", "i"); + assertStringTrim(UTF8_BINARY, "i\u0307", "i\u0307", ""); + assertStringTrim(UTF8_BINARY, "i\u0307i\u0307", "i\u0307", ""); + assertStringTrim(UTF8_BINARY, "i\u0307\u0307", "i\u0307", ""); + assertStringTrim(UTF8_BINARY, "i\u0307i", "i\u0307", ""); + assertStringTrim(UTF8_BINARY, "i\u0307i", "İ", "i\u0307i"); + assertStringTrim(UTF8_BINARY, "i\u0307İ", "i\u0307", "İ"); + assertStringTrim(UTF8_BINARY, "i\u0307İ", "İ", "i\u0307"); + assertStringTrim(UTF8_BINARY, "İ", "İ", ""); + assertStringTrim(UTF8_BINARY, "IXi", "İ", "IXi"); + assertStringTrim(UTF8_BINARY, "ix\u0307", "Ixİ", "ix\u0307"); + assertStringTrim(UTF8_BINARY, "i\u0307x", "IXİ", "i\u0307x"); + assertStringTrim(UTF8_BINARY, "i\u0307x", "ix\u0307İ", ""); + assertStringTrim(UTF8_BINARY, "İ", "i", "İ"); + assertStringTrim(UTF8_BINARY, "İ", "\u0307", "İ"); + assertStringTrim(UTF8_BINARY, "Ixİ", "i\u0307", "Ixİ"); + assertStringTrim(UTF8_BINARY, "IXİ", "ix\u0307", "IXİ"); + assertStringTrim(UTF8_BINARY, "xi\u0307", "\u0307IX", "xi"); + assertStringTrim(UTF8_LCASE, "i", "i", ""); + assertStringTrim(UTF8_LCASE, "iii", "I", ""); + assertStringTrim(UTF8_LCASE, "I", "iii", ""); + assertStringTrim(UTF8_LCASE, "ixi", "i", "x"); + assertStringTrim(UTF8_LCASE, "i", "İ", "i"); + assertStringTrim(UTF8_LCASE, "i\u0307", "İ", ""); + assertStringTrim(UTF8_LCASE, "ii\u0307", "İi", ""); + assertStringTrim(UTF8_LCASE, "iii\u0307", "İi", ""); + assertStringTrim(UTF8_LCASE, "iiii\u0307", "iİ", ""); + assertStringTrim(UTF8_LCASE, "ii\u0307ii\u0307", "iİ", ""); + assertStringTrim(UTF8_LCASE, "i\u0307", "i", "\u0307"); + assertStringTrim(UTF8_LCASE, "i\u0307", "\u0307", "i"); + assertStringTrim(UTF8_LCASE, "i\u0307", "i\u0307", ""); + assertStringTrim(UTF8_LCASE, "i\u0307i\u0307", "i\u0307", ""); + assertStringTrim(UTF8_LCASE, "i\u0307\u0307", "i\u0307", ""); + assertStringTrim(UTF8_LCASE, "i\u0307i", "i\u0307", ""); + assertStringTrim(UTF8_LCASE, "i\u0307i", "İ", "i"); + assertStringTrim(UTF8_LCASE, "i\u0307İ", "i\u0307", "İ"); + assertStringTrim(UTF8_LCASE, "i\u0307İ", "İ", ""); + assertStringTrim(UTF8_LCASE, "İ", "İ", ""); + assertStringTrim(UTF8_LCASE, "IXi", "İ", "IXi"); + assertStringTrim(UTF8_LCASE, "ix\u0307", "Ixİ", "\u0307"); + assertStringTrim(UTF8_LCASE, "i\u0307x", "IXİ", ""); + assertStringTrim(UTF8_LCASE, "i\u0307x", "I\u0307xİ", ""); + assertStringTrim(UTF8_LCASE, "İ", "i", "İ"); + assertStringTrim(UTF8_LCASE, "İ", "\u0307", "İ"); + assertStringTrim(UTF8_LCASE, "Ixİ", "i\u0307", "xİ"); + assertStringTrim(UTF8_LCASE, "IXİ", "ix\u0307", "İ"); + assertStringTrim(UTF8_LCASE, "xi\u0307", "\u0307IX", ""); + assertStringTrim(UNICODE, "i", "i", ""); + assertStringTrim(UNICODE, "iii", "I", "iii"); + assertStringTrim(UNICODE, "I", "iii", "I"); + assertStringTrim(UNICODE, "ixi", "i", "x"); + assertStringTrim(UNICODE, "i", "İ", "i"); + assertStringTrim(UNICODE, "i\u0307", "İ", "i\u0307"); + assertStringTrim(UNICODE, "ii\u0307", "İi", "i\u0307"); + assertStringTrim(UNICODE, "iii\u0307", "İi", "i\u0307"); + assertStringTrim(UNICODE, "iiii\u0307", "iİ", "i\u0307"); + assertStringTrim(UNICODE, "ii\u0307ii\u0307", "iİ", "i\u0307ii\u0307"); + assertStringTrim(UNICODE, "i\u0307", "i", "i\u0307"); + assertStringTrim(UNICODE, "i\u0307", "\u0307", "i\u0307"); + assertStringTrim(UNICODE, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrim(UNICODE, "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); + assertStringTrim(UNICODE, "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); + assertStringTrim(UNICODE, "i\u0307i", "i\u0307", "i\u0307"); + assertStringTrim(UNICODE, "i\u0307i", "İ", "i\u0307i"); + assertStringTrim(UNICODE, "i\u0307İ", "i\u0307", "i\u0307İ"); + assertStringTrim(UNICODE, "i\u0307İ", "İ", "i\u0307"); + assertStringTrim(UNICODE, "İ", "İ", ""); + assertStringTrim(UNICODE, "IXi", "İ", "IXi"); + assertStringTrim(UNICODE, "ix\u0307", "Ixİ", "ix\u0307"); + assertStringTrim(UNICODE, "i\u0307x", "IXİ", "i\u0307x"); + assertStringTrim(UNICODE, "i\u0307x", "ix\u0307İ", "i\u0307"); + assertStringTrim(UNICODE, "İ", "i", "İ"); + assertStringTrim(UNICODE, "İ", "\u0307", "İ"); + assertStringTrim(UNICODE, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrim(UNICODE, "Ixİ", "i\u0307", "Ixİ"); + assertStringTrim(UNICODE, "IXİ", "ix\u0307", "IXİ"); + assertStringTrim(UNICODE, "xi\u0307", "\u0307IX", "xi\u0307"); + assertStringTrim(UNICODE_CI, "i", "i", ""); + assertStringTrim(UNICODE_CI, "iii", "I", ""); + assertStringTrim(UNICODE_CI, "I", "iii", ""); + assertStringTrim(UNICODE_CI, "ixi", "i", "x"); + assertStringTrim(UNICODE_CI, "i", "İ", "i"); + assertStringTrim(UNICODE_CI, "i\u0307", "İ", ""); + assertStringTrim(UNICODE_CI, "ii\u0307", "İi", ""); + assertStringTrim(UNICODE_CI, "iii\u0307", "İi", ""); + assertStringTrim(UNICODE_CI, "iiii\u0307", "iİ", ""); + assertStringTrim(UNICODE_CI, "ii\u0307ii\u0307", "iİ", ""); + assertStringTrim(UNICODE_CI, "i\u0307", "i", "i\u0307"); + assertStringTrim(UNICODE_CI, "i\u0307", "\u0307", "i\u0307"); + assertStringTrim(UNICODE_CI, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrim(UNICODE_CI, "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); + assertStringTrim(UNICODE_CI, "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); + assertStringTrim(UNICODE_CI, "i\u0307i", "i\u0307", "i\u0307"); + assertStringTrim(UNICODE_CI, "i\u0307i", "İ", "i"); + assertStringTrim(UNICODE_CI, "i\u0307İ", "i\u0307", "i\u0307İ"); + assertStringTrim(UNICODE_CI, "i\u0307İ", "İ", ""); + assertStringTrim(UNICODE_CI, "İ", "İ", ""); + assertStringTrim(UNICODE_CI, "IXi", "İ", "IXi"); + assertStringTrim(UNICODE_CI, "ix\u0307", "Ixİ", "x\u0307"); + assertStringTrim(UNICODE_CI, "i\u0307x", "IXİ", ""); + assertStringTrim(UNICODE_CI, "i\u0307x", "I\u0307xİ", ""); + assertStringTrim(UNICODE_CI, "İ", "i", "İ"); + assertStringTrim(UNICODE_CI, "İ", "\u0307", "İ"); + assertStringTrim(UNICODE_CI, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrim(UNICODE_CI, "Ixİ", "i\u0307", "xİ"); + assertStringTrim(UNICODE_CI, "IXİ", "ix\u0307", "İ"); + assertStringTrim(UNICODE_CI, "xi\u0307", "\u0307IX", "i\u0307"); // Conditional case mapping (e.g. Greek sigmas). - assertStringTrim("UTF8_BINARY", "ςxς", "σ", "ςxς"); - assertStringTrim("UTF8_BINARY", "ςxς", "ς", "x"); - assertStringTrim("UTF8_BINARY", "ςxς", "Σ", "ςxς"); - assertStringTrim("UTF8_BINARY", "σxσ", "σ", "x"); - assertStringTrim("UTF8_BINARY", "σxσ", "ς", "σxσ"); - assertStringTrim("UTF8_BINARY", "σxσ", "Σ", "σxσ"); - assertStringTrim("UTF8_BINARY", "ΣxΣ", "σ", "ΣxΣ"); - assertStringTrim("UTF8_BINARY", "ΣxΣ", "ς", "ΣxΣ"); - assertStringTrim("UTF8_BINARY", "ΣxΣ", "Σ", "x"); - assertStringTrim("UTF8_LCASE", "ςxς", "σ", "x"); - assertStringTrim("UTF8_LCASE", "ςxς", "ς", "x"); - assertStringTrim("UTF8_LCASE", "ςxς", "Σ", "x"); - assertStringTrim("UTF8_LCASE", "σxσ", "σ", "x"); - assertStringTrim("UTF8_LCASE", "σxσ", "ς", "x"); - assertStringTrim("UTF8_LCASE", "σxσ", "Σ", "x"); - assertStringTrim("UTF8_LCASE", "ΣxΣ", "σ", "x"); - assertStringTrim("UTF8_LCASE", "ΣxΣ", "ς", "x"); - assertStringTrim("UTF8_LCASE", "ΣxΣ", "Σ", "x"); - assertStringTrim("UNICODE", "ςxς", "σ", "ςxς"); - assertStringTrim("UNICODE", "ςxς", "ς", "x"); - assertStringTrim("UNICODE", "ςxς", "Σ", "ςxς"); - assertStringTrim("UNICODE", "σxσ", "σ", "x"); - assertStringTrim("UNICODE", "σxσ", "ς", "σxσ"); - assertStringTrim("UNICODE", "σxσ", "Σ", "σxσ"); - assertStringTrim("UNICODE", "ΣxΣ", "σ", "ΣxΣ"); - assertStringTrim("UNICODE", "ΣxΣ", "ς", "ΣxΣ"); - assertStringTrim("UNICODE", "ΣxΣ", "Σ", "x"); - assertStringTrim("UNICODE_CI", "ςxς", "σ", "x"); - assertStringTrim("UNICODE_CI", "ςxς", "ς", "x"); - assertStringTrim("UNICODE_CI", "ςxς", "Σ", "x"); - assertStringTrim("UNICODE_CI", "σxσ", "σ", "x"); - assertStringTrim("UNICODE_CI", "σxσ", "ς", "x"); - assertStringTrim("UNICODE_CI", "σxσ", "Σ", "x"); - assertStringTrim("UNICODE_CI", "ΣxΣ", "σ", "x"); - assertStringTrim("UNICODE_CI", "ΣxΣ", "ς", "x"); - assertStringTrim("UNICODE_CI", "ΣxΣ", "Σ", "x"); + assertStringTrim(UTF8_BINARY, "ςxς", "σ", "ςxς"); + assertStringTrim(UTF8_BINARY, "ςxς", "ς", "x"); + assertStringTrim(UTF8_BINARY, "ςxς", "Σ", "ςxς"); + assertStringTrim(UTF8_BINARY, "σxσ", "σ", "x"); + assertStringTrim(UTF8_BINARY, "σxσ", "ς", "σxσ"); + assertStringTrim(UTF8_BINARY, "σxσ", "Σ", "σxσ"); + assertStringTrim(UTF8_BINARY, "ΣxΣ", "σ", "ΣxΣ"); + assertStringTrim(UTF8_BINARY, "ΣxΣ", "ς", "ΣxΣ"); + assertStringTrim(UTF8_BINARY, "ΣxΣ", "Σ", "x"); + assertStringTrim(UTF8_LCASE, "ςxς", "σ", "x"); + assertStringTrim(UTF8_LCASE, "ςxς", "ς", "x"); + assertStringTrim(UTF8_LCASE, "ςxς", "Σ", "x"); + assertStringTrim(UTF8_LCASE, "σxσ", "σ", "x"); + assertStringTrim(UTF8_LCASE, "σxσ", "ς", "x"); + assertStringTrim(UTF8_LCASE, "σxσ", "Σ", "x"); + assertStringTrim(UTF8_LCASE, "ΣxΣ", "σ", "x"); + assertStringTrim(UTF8_LCASE, "ΣxΣ", "ς", "x"); + assertStringTrim(UTF8_LCASE, "ΣxΣ", "Σ", "x"); + assertStringTrim(UNICODE, "ςxς", "σ", "ςxς"); + assertStringTrim(UNICODE, "ςxς", "ς", "x"); + assertStringTrim(UNICODE, "ςxς", "Σ", "ςxς"); + assertStringTrim(UNICODE, "σxσ", "σ", "x"); + assertStringTrim(UNICODE, "σxσ", "ς", "σxσ"); + assertStringTrim(UNICODE, "σxσ", "Σ", "σxσ"); + assertStringTrim(UNICODE, "ΣxΣ", "σ", "ΣxΣ"); + assertStringTrim(UNICODE, "ΣxΣ", "ς", "ΣxΣ"); + assertStringTrim(UNICODE, "ΣxΣ", "Σ", "x"); + assertStringTrim(UNICODE_CI, "ςxς", "σ", "x"); + assertStringTrim(UNICODE_CI, "ςxς", "ς", "x"); + assertStringTrim(UNICODE_CI, "ςxς", "Σ", "x"); + assertStringTrim(UNICODE_CI, "σxσ", "σ", "x"); + assertStringTrim(UNICODE_CI, "σxσ", "ς", "x"); + assertStringTrim(UNICODE_CI, "σxσ", "Σ", "x"); + assertStringTrim(UNICODE_CI, "ΣxΣ", "σ", "x"); + assertStringTrim(UNICODE_CI, "ΣxΣ", "ς", "x"); + assertStringTrim(UNICODE_CI, "ΣxΣ", "Σ", "x"); // Unicode normalization. - assertStringTrim("UTF8_BINARY", "åβγδa\u030A", "å", "βγδa\u030A"); - assertStringTrim("UTF8_LCASE", "åβγδa\u030A", "Å", "βγδa\u030A"); - assertStringTrim("UNICODE", "åβγδa\u030A", "å", "βγδ"); - assertStringTrim("UNICODE_CI", "åβγδa\u030A", "Å", "βγδ"); + assertStringTrim(UTF8_BINARY, "åβγδa\u030A", "å", "βγδa\u030A"); + assertStringTrim(UTF8_LCASE, "åβγδa\u030A", "Å", "βγδa\u030A"); + assertStringTrim(UNICODE, "åβγδa\u030A", "å", "βγδ"); + assertStringTrim(UNICODE_CI, "åβγδa\u030A", "Å", "βγδ"); // Surrogate pairs. - assertStringTrim("UTF8_BINARY", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrim("UTF8_LCASE", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrim("UNICODE", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrim("UNICODE_CI", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrim("UTF8_BINARY", "a🙃b🙃c", "ac", "🙃b🙃"); - assertStringTrim("UTF8_LCASE", "a🙃b🙃c", "ac", "🙃b🙃"); - assertStringTrim("UNICODE", "a🙃b🙃c", "ac", "🙃b🙃"); - assertStringTrim("UNICODE_CI", "a🙃b🙃c", "ac", "🙃b🙃"); - assertStringTrim("UTF8_BINARY", "a🙃b🙃c", "a🙃c", "b"); - assertStringTrim("UTF8_LCASE", "a🙃b🙃c", "a🙃c", "b"); - assertStringTrim("UNICODE", "a🙃b🙃c", "a🙃c", "b"); - assertStringTrim("UNICODE_CI", "a🙃b🙃c", "a🙃c", "b"); - assertStringTrim("UTF8_BINARY", "a🙃b🙃c", "abc🙃", ""); - assertStringTrim("UTF8_LCASE", "a🙃b🙃c", "abc🙃", ""); - assertStringTrim("UNICODE", "a🙃b🙃c", "abc🙃", ""); - assertStringTrim("UNICODE_CI", "a🙃b🙃c", "abc🙃", ""); - assertStringTrim("UTF8_BINARY", "😀😆😃😄", "😀😄", "😆😃"); - assertStringTrim("UTF8_LCASE", "😀😆😃😄", "😀😄", "😆😃"); - assertStringTrim("UNICODE", "😀😆😃😄", "😀😄", "😆😃"); - assertStringTrim("UNICODE_CI", "😀😆😃😄", "😀😄", "😆😃"); - assertStringTrim("UTF8_BINARY", "😀😆😃😄", "😃😄", "😀😆"); - assertStringTrim("UTF8_LCASE", "😀😆😃😄", "😃😄", "😀😆"); - assertStringTrim("UNICODE", "😀😆😃😄", "😃😄", "😀😆"); - assertStringTrim("UNICODE_CI", "😀😆😃😄", "😃😄", "😀😆"); - assertStringTrim("UTF8_BINARY", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrim("UTF8_LCASE", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrim("UNICODE", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrim("UNICODE_CI", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrim("UTF8_BINARY", "𐐅", "𐐅", ""); - assertStringTrim("UTF8_LCASE", "𐐅", "𐐅", ""); - assertStringTrim("UNICODE", "𐐅", "𐐅", ""); - assertStringTrim("UNICODE_CI", "𐐅", "𐐅", ""); - assertStringTrim("UTF8_BINARY", "𐐅", "𐐭", "𐐅"); - assertStringTrim("UTF8_LCASE", "𐐅", "𐐭", ""); - assertStringTrim("UNICODE", "𐐅", "𐐭", "𐐅"); - assertStringTrim("UNICODE_CI", "𐐅", "𐐭", ""); - assertStringTrim("UTF8_BINARY", "𝔸", "𝔸", ""); - assertStringTrim("UTF8_LCASE", "𝔸", "𝔸", ""); - assertStringTrim("UNICODE", "𝔸", "𝔸", ""); - assertStringTrim("UNICODE_CI", "𝔸", "𝔸", ""); - assertStringTrim("UTF8_BINARY", "𝔸", "A", "𝔸"); - assertStringTrim("UTF8_LCASE", "𝔸", "A", "𝔸"); - assertStringTrim("UNICODE", "𝔸", "A", "𝔸"); - assertStringTrim("UNICODE_CI", "𝔸", "A", ""); - assertStringTrim("UTF8_BINARY", "𝔸", "a", "𝔸"); - assertStringTrim("UTF8_LCASE", "𝔸", "a", "𝔸"); - assertStringTrim("UNICODE", "𝔸", "a", "𝔸"); - assertStringTrim("UNICODE_CI", "𝔸", "a", ""); + assertStringTrim(UTF8_BINARY, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrim(UTF8_LCASE, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrim(UNICODE, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrim(UNICODE_CI, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrim(UTF8_BINARY, "a🙃b🙃c", "ac", "🙃b🙃"); + assertStringTrim(UTF8_LCASE, "a🙃b🙃c", "ac", "🙃b🙃"); + assertStringTrim(UNICODE, "a🙃b🙃c", "ac", "🙃b🙃"); + assertStringTrim(UNICODE_CI, "a🙃b🙃c", "ac", "🙃b🙃"); + assertStringTrim(UTF8_BINARY, "a🙃b🙃c", "a🙃c", "b"); + assertStringTrim(UTF8_LCASE, "a🙃b🙃c", "a🙃c", "b"); + assertStringTrim(UNICODE, "a🙃b🙃c", "a🙃c", "b"); + assertStringTrim(UNICODE_CI, "a🙃b🙃c", "a🙃c", "b"); + assertStringTrim(UTF8_BINARY, "a🙃b🙃c", "abc🙃", ""); + assertStringTrim(UTF8_LCASE, "a🙃b🙃c", "abc🙃", ""); + assertStringTrim(UNICODE, "a🙃b🙃c", "abc🙃", ""); + assertStringTrim(UNICODE_CI, "a🙃b🙃c", "abc🙃", ""); + assertStringTrim(UTF8_BINARY, "😀😆😃😄", "😀😄", "😆😃"); + assertStringTrim(UTF8_LCASE, "😀😆😃😄", "😀😄", "😆😃"); + assertStringTrim(UNICODE, "😀😆😃😄", "😀😄", "😆😃"); + assertStringTrim(UNICODE_CI, "😀😆😃😄", "😀😄", "😆😃"); + assertStringTrim(UTF8_BINARY, "😀😆😃😄", "😃😄", "😀😆"); + assertStringTrim(UTF8_LCASE, "😀😆😃😄", "😃😄", "😀😆"); + assertStringTrim(UNICODE, "😀😆😃😄", "😃😄", "😀😆"); + assertStringTrim(UNICODE_CI, "😀😆😃😄", "😃😄", "😀😆"); + assertStringTrim(UTF8_BINARY, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrim(UTF8_LCASE, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrim(UNICODE, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrim(UNICODE_CI, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrim(UTF8_BINARY, "𐐅", "𐐅", ""); + assertStringTrim(UTF8_LCASE, "𐐅", "𐐅", ""); + assertStringTrim(UNICODE, "𐐅", "𐐅", ""); + assertStringTrim(UNICODE_CI, "𐐅", "𐐅", ""); + assertStringTrim(UTF8_BINARY, "𐐅", "𐐭", "𐐅"); + assertStringTrim(UTF8_LCASE, "𐐅", "𐐭", ""); + assertStringTrim(UNICODE, "𐐅", "𐐭", "𐐅"); + assertStringTrim(UNICODE_CI, "𐐅", "𐐭", ""); + assertStringTrim(UTF8_BINARY, "𝔸", "𝔸", ""); + assertStringTrim(UTF8_LCASE, "𝔸", "𝔸", ""); + assertStringTrim(UNICODE, "𝔸", "𝔸", ""); + assertStringTrim(UNICODE_CI, "𝔸", "𝔸", ""); + assertStringTrim(UTF8_BINARY, "𝔸", "A", "𝔸"); + assertStringTrim(UTF8_LCASE, "𝔸", "A", "𝔸"); + assertStringTrim(UNICODE, "𝔸", "A", "𝔸"); + assertStringTrim(UNICODE_CI, "𝔸", "A", ""); + assertStringTrim(UTF8_BINARY, "𝔸", "a", "𝔸"); + assertStringTrim(UTF8_LCASE, "𝔸", "a", "𝔸"); + assertStringTrim(UNICODE, "𝔸", "a", "𝔸"); + assertStringTrim(UNICODE_CI, "𝔸", "a", ""); } /** @@ -3078,277 +3079,277 @@ private void assertStringTrimLeft(String collationName, String sourceString, Str @Test public void testStringTrimLeft() throws SparkException { // Basic tests - UTF8_BINARY. - assertStringTrimLeft("UTF8_BINARY", "", "", ""); - assertStringTrimLeft("UTF8_BINARY", "", "xyz", ""); - assertStringTrimLeft("UTF8_BINARY", "asd", "", "asd"); - assertStringTrimLeft("UTF8_BINARY", "asd", null, "asd"); - assertStringTrimLeft("UTF8_BINARY", " asd ", null, "asd "); - assertStringTrimLeft("UTF8_BINARY", " a世a ", null, "a世a "); - assertStringTrimLeft("UTF8_BINARY", "asd", "x", "asd"); - assertStringTrimLeft("UTF8_BINARY", "xxasdxx", "x", "asdxx"); - assertStringTrimLeft("UTF8_BINARY", "xa世ax", "x", "a世ax"); + assertStringTrimLeft(UTF8_BINARY, "", "", ""); + assertStringTrimLeft(UTF8_BINARY, "", "xyz", ""); + assertStringTrimLeft(UTF8_BINARY, "asd", "", "asd"); + assertStringTrimLeft(UTF8_BINARY, "asd", null, "asd"); + assertStringTrimLeft(UTF8_BINARY, " asd ", null, "asd "); + assertStringTrimLeft(UTF8_BINARY, " a世a ", null, "a世a "); + assertStringTrimLeft(UTF8_BINARY, "asd", "x", "asd"); + assertStringTrimLeft(UTF8_BINARY, "xxasdxx", "x", "asdxx"); + assertStringTrimLeft(UTF8_BINARY, "xa世ax", "x", "a世ax"); // Basic tests - UTF8_LCASE. - assertStringTrimLeft("UTF8_LCASE", "", "", ""); - assertStringTrimLeft("UTF8_LCASE", "", "xyz", ""); - assertStringTrimLeft("UTF8_LCASE", "asd", "", "asd"); - 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"); + assertStringTrimLeft(UTF8_LCASE, "", "", ""); + assertStringTrimLeft(UTF8_LCASE, "", "xyz", ""); + assertStringTrimLeft(UTF8_LCASE, "asd", "", "asd"); + 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"); // Basic tests - UNICODE. - assertStringTrimLeft("UNICODE", "", "", ""); - assertStringTrimLeft("UNICODE", "", "xyz", ""); - assertStringTrimLeft("UNICODE", "asd", "", "asd"); - assertStringTrimLeft("UNICODE", "asd", null, "asd"); - assertStringTrimLeft("UNICODE", " asd ", null, "asd "); - assertStringTrimLeft("UNICODE", " a世a ", null, "a世a "); - assertStringTrimLeft("UNICODE", "asd", "x", "asd"); - assertStringTrimLeft("UNICODE", "xxasdxx", "x", "asdxx"); - assertStringTrimLeft("UNICODE", "xa世ax", "x", "a世ax"); + assertStringTrimLeft(UNICODE, "", "", ""); + assertStringTrimLeft(UNICODE, "", "xyz", ""); + assertStringTrimLeft(UNICODE, "asd", "", "asd"); + assertStringTrimLeft(UNICODE, "asd", null, "asd"); + assertStringTrimLeft(UNICODE, " asd ", null, "asd "); + assertStringTrimLeft(UNICODE, " a世a ", null, "a世a "); + assertStringTrimLeft(UNICODE, "asd", "x", "asd"); + assertStringTrimLeft(UNICODE, "xxasdxx", "x", "asdxx"); + assertStringTrimLeft(UNICODE, "xa世ax", "x", "a世ax"); // Basic tests - UNICODE_CI. - assertStringTrimLeft("UNICODE_CI", "", "", ""); - assertStringTrimLeft("UNICODE_CI", "", "xyz", ""); - assertStringTrimLeft("UNICODE_CI", "asd", "", "asd"); - assertStringTrimLeft("UNICODE_CI", "asd", null, "asd"); - assertStringTrimLeft("UNICODE_CI", " asd ", null, "asd "); - assertStringTrimLeft("UNICODE_CI", " a世a ", null, "a世a "); - assertStringTrimLeft("UNICODE_CI", "asd", "x", "asd"); - assertStringTrimLeft("UNICODE_CI", "xxasdxx", "x", "asdxx"); - assertStringTrimLeft("UNICODE_CI", "xa世ax", "x", "a世ax"); + assertStringTrimLeft(UNICODE_CI, "", "", ""); + assertStringTrimLeft(UNICODE_CI, "", "xyz", ""); + assertStringTrimLeft(UNICODE_CI, "asd", "", "asd"); + assertStringTrimLeft(UNICODE_CI, "asd", null, "asd"); + assertStringTrimLeft(UNICODE_CI, " asd ", null, "asd "); + assertStringTrimLeft(UNICODE_CI, " a世a ", null, "a世a "); + assertStringTrimLeft(UNICODE_CI, "asd", "x", "asd"); + assertStringTrimLeft(UNICODE_CI, "xxasdxx", "x", "asdxx"); + assertStringTrimLeft(UNICODE_CI, "xa世ax", "x", "a世ax"); // Case variation. - assertStringTrimLeft("UTF8_BINARY", "ddsXXXaa", "asd", "XXXaa"); - assertStringTrimLeft("UTF8_LCASE", "ddsXXXaa", "aSd", "XXXaa"); - assertStringTrimLeft("UNICODE", "ddsXXXaa", "asd", "XXXaa"); - assertStringTrimLeft("UNICODE_CI", "ddsXXXaa", "aSd", "XXXaa"); + assertStringTrimLeft(UTF8_BINARY, "ddsXXXaa", "asd", "XXXaa"); + assertStringTrimLeft(UTF8_LCASE, "ddsXXXaa", "aSd", "XXXaa"); + assertStringTrimLeft(UNICODE, "ddsXXXaa", "asd", "XXXaa"); + assertStringTrimLeft(UNICODE_CI, "ddsXXXaa", "aSd", "XXXaa"); // One-to-many case mapping (e.g. Turkish dotted I).. - assertStringTrimLeft("UTF8_BINARY", "ẞaaaẞ", "ß", "ẞaaaẞ"); - assertStringTrimLeft("UTF8_BINARY", "ßaaaß", "ẞ", "ßaaaß"); - assertStringTrimLeft("UTF8_BINARY", "Ëaaaẞ", "Ëẞ", "aaaẞ"); - assertStringTrimLeft("UTF8_LCASE", "ẞaaaẞ", "ß", "aaaẞ"); - assertStringTrimLeft("UTF8_LCASE", "ßaaaß", "ẞ", "aaaß"); - assertStringTrimLeft("UTF8_LCASE", "Ëaaaẞ", "Ëẞ", "aaaẞ"); - assertStringTrimLeft("UNICODE", "ẞaaaẞ", "ß", "ẞaaaẞ"); - assertStringTrimLeft("UNICODE", "ßaaaß", "ẞ", "ßaaaß"); - assertStringTrimLeft("UNICODE", "Ëaaaẞ", "Ëẞ", "aaaẞ"); - assertStringTrimLeft("UNICODE_CI", "ẞaaaẞ", "ß", "aaaẞ"); - assertStringTrimLeft("UNICODE_CI", "ßaaaß", "ẞ", "aaaß"); - assertStringTrimLeft("UNICODE_CI", "Ëaaaẞ", "Ëẞ", "aaaẞ"); + assertStringTrimLeft(UTF8_BINARY, "ẞaaaẞ", "ß", "ẞaaaẞ"); + assertStringTrimLeft(UTF8_BINARY, "ßaaaß", "ẞ", "ßaaaß"); + assertStringTrimLeft(UTF8_BINARY, "Ëaaaẞ", "Ëẞ", "aaaẞ"); + assertStringTrimLeft(UTF8_LCASE, "ẞaaaẞ", "ß", "aaaẞ"); + assertStringTrimLeft(UTF8_LCASE, "ßaaaß", "ẞ", "aaaß"); + assertStringTrimLeft(UTF8_LCASE, "Ëaaaẞ", "Ëẞ", "aaaẞ"); + assertStringTrimLeft(UNICODE, "ẞaaaẞ", "ß", "ẞaaaẞ"); + assertStringTrimLeft(UNICODE, "ßaaaß", "ẞ", "ßaaaß"); + assertStringTrimLeft(UNICODE, "Ëaaaẞ", "Ëẞ", "aaaẞ"); + assertStringTrimLeft(UNICODE_CI, "ẞaaaẞ", "ß", "aaaẞ"); + assertStringTrimLeft(UNICODE_CI, "ßaaaß", "ẞ", "aaaß"); + assertStringTrimLeft(UNICODE_CI, "Ëaaaẞ", "Ëẞ", "aaaẞ"); // One-to-many case mapping (e.g. Turkish dotted I). - assertStringTrimLeft("UTF8_BINARY", "i", "i", ""); - assertStringTrimLeft("UTF8_BINARY", "iii", "I", "iii"); - assertStringTrimLeft("UTF8_BINARY", "I", "iii", "I"); - assertStringTrimLeft("UTF8_BINARY", "ixi", "i", "xi"); - assertStringTrimLeft("UTF8_BINARY", "i", "İ", "i"); - assertStringTrimLeft("UTF8_BINARY", "i\u0307", "İ", "i\u0307"); - assertStringTrimLeft("UTF8_BINARY", "ii\u0307", "İi", "\u0307"); - assertStringTrimLeft("UTF8_BINARY", "iii\u0307", "İi", "\u0307"); - assertStringTrimLeft("UTF8_BINARY", "iiii\u0307", "iİ", "\u0307"); - assertStringTrimLeft("UTF8_BINARY", "ii\u0307ii\u0307", "iİ", "\u0307ii\u0307"); - assertStringTrimLeft("UTF8_BINARY", "i\u0307", "i", "\u0307"); - assertStringTrimLeft("UTF8_BINARY", "i\u0307", "\u0307", "i\u0307"); - assertStringTrimLeft("UTF8_BINARY", "i\u0307", "i\u0307", ""); - assertStringTrimLeft("UTF8_BINARY", "i\u0307i\u0307", "i\u0307", ""); - assertStringTrimLeft("UTF8_BINARY", "i\u0307\u0307", "i\u0307", ""); - assertStringTrimLeft("UTF8_BINARY", "i\u0307i", "i\u0307", ""); - assertStringTrimLeft("UTF8_BINARY", "i\u0307i", "İ", "i\u0307i"); - assertStringTrimLeft("UTF8_BINARY", "i\u0307İ", "i\u0307", "İ"); - assertStringTrimLeft("UTF8_BINARY", "i\u0307İ", "İ", "i\u0307İ"); - assertStringTrimLeft("UTF8_BINARY", "İ", "İ", ""); - assertStringTrimLeft("UTF8_BINARY", "IXi", "İ", "IXi"); - assertStringTrimLeft("UTF8_BINARY", "ix\u0307", "Ixİ", "ix\u0307"); - assertStringTrimLeft("UTF8_BINARY", "i\u0307x", "IXİ", "i\u0307x"); - assertStringTrimLeft("UTF8_BINARY", "i\u0307x", "ix\u0307İ", ""); - assertStringTrimLeft("UTF8_BINARY", "İ", "i", "İ"); - assertStringTrimLeft("UTF8_BINARY", "İ", "\u0307", "İ"); - assertStringTrimLeft("UTF8_BINARY", "Ixİ", "i\u0307", "Ixİ"); - assertStringTrimLeft("UTF8_BINARY", "IXİ", "ix\u0307", "IXİ"); - assertStringTrimLeft("UTF8_BINARY", "xi\u0307", "\u0307IX", "xi\u0307"); - assertStringTrimLeft("UTF8_LCASE", "i", "i", ""); - assertStringTrimLeft("UTF8_LCASE", "iii", "I", ""); - assertStringTrimLeft("UTF8_LCASE", "I", "iii", ""); - assertStringTrimLeft("UTF8_LCASE", "ixi", "i", "xi"); - assertStringTrimLeft("UTF8_LCASE", "i", "İ", "i"); - assertStringTrimLeft("UTF8_LCASE", "i\u0307", "İ", ""); - assertStringTrimLeft("UTF8_LCASE", "ii\u0307", "İi", ""); - assertStringTrimLeft("UTF8_LCASE", "iii\u0307", "İi", ""); - assertStringTrimLeft("UTF8_LCASE", "iiii\u0307", "iİ", ""); - assertStringTrimLeft("UTF8_LCASE", "ii\u0307ii\u0307", "iİ", ""); - assertStringTrimLeft("UTF8_LCASE", "i\u0307", "i", "\u0307"); - assertStringTrimLeft("UTF8_LCASE", "i\u0307", "\u0307", "i\u0307"); - assertStringTrimLeft("UTF8_LCASE", "i\u0307", "i\u0307", ""); - assertStringTrimLeft("UTF8_LCASE", "i\u0307i\u0307", "i\u0307", ""); - assertStringTrimLeft("UTF8_LCASE", "i\u0307\u0307", "i\u0307", ""); - assertStringTrimLeft("UTF8_LCASE", "i\u0307i", "i\u0307", ""); - assertStringTrimLeft("UTF8_LCASE", "i\u0307i", "İ", "i"); - assertStringTrimLeft("UTF8_LCASE", "i\u0307İ", "i\u0307", "İ"); - assertStringTrimLeft("UTF8_LCASE", "i\u0307İ", "İ", ""); - assertStringTrimLeft("UTF8_LCASE", "İ", "İ", ""); - assertStringTrimLeft("UTF8_LCASE", "IXi", "İ", "IXi"); - assertStringTrimLeft("UTF8_LCASE", "ix\u0307", "Ixİ", "\u0307"); - assertStringTrimLeft("UTF8_LCASE", "i\u0307x", "IXİ", ""); - assertStringTrimLeft("UTF8_LCASE", "i\u0307x", "I\u0307xİ", ""); - assertStringTrimLeft("UTF8_LCASE", "İ", "i", "İ"); - assertStringTrimLeft("UTF8_LCASE", "İ", "\u0307", "İ"); - assertStringTrimLeft("UTF8_LCASE", "Ixİ", "i\u0307", "xİ"); - assertStringTrimLeft("UTF8_LCASE", "IXİ", "ix\u0307", "İ"); - assertStringTrimLeft("UTF8_LCASE", "xi\u0307", "\u0307IX", ""); - assertStringTrimLeft("UNICODE", "i", "i", ""); - assertStringTrimLeft("UNICODE", "iii", "I", "iii"); - assertStringTrimLeft("UNICODE", "I", "iii", "I"); - assertStringTrimLeft("UNICODE", "ixi", "i", "xi"); - assertStringTrimLeft("UNICODE", "i", "İ", "i"); - assertStringTrimLeft("UNICODE", "i\u0307", "İ", "i\u0307"); - assertStringTrimLeft("UNICODE", "ii\u0307", "İi", "i\u0307"); - assertStringTrimLeft("UNICODE", "iii\u0307", "İi", "i\u0307"); - assertStringTrimLeft("UNICODE", "iiii\u0307", "iİ", "i\u0307"); - assertStringTrimLeft("UNICODE", "ii\u0307ii\u0307", "iİ", "i\u0307ii\u0307"); - assertStringTrimLeft("UNICODE", "i\u0307", "i", "i\u0307"); - assertStringTrimLeft("UNICODE", "i\u0307", "\u0307", "i\u0307"); - assertStringTrimLeft("UNICODE", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrimLeft("UNICODE", "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); - assertStringTrimLeft("UNICODE", "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); - assertStringTrimLeft("UNICODE", "i\u0307i", "i\u0307", "i\u0307i"); - assertStringTrimLeft("UNICODE", "i\u0307i", "İ", "i\u0307i"); - assertStringTrimLeft("UNICODE", "i\u0307İ", "i\u0307", "i\u0307İ"); - assertStringTrimLeft("UNICODE", "i\u0307İ", "İ", "i\u0307İ"); - assertStringTrimLeft("UNICODE", "İ", "İ", ""); - assertStringTrimLeft("UNICODE", "IXi", "İ", "IXi"); - assertStringTrimLeft("UNICODE", "ix\u0307", "Ixİ", "ix\u0307"); - assertStringTrimLeft("UNICODE", "i\u0307x", "IXİ", "i\u0307x"); - assertStringTrimLeft("UNICODE", "i\u0307x", "ix\u0307İ", "i\u0307x"); - assertStringTrimLeft("UNICODE", "İ", "i", "İ"); - assertStringTrimLeft("UNICODE", "İ", "\u0307", "İ"); - assertStringTrimLeft("UNICODE", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrimLeft("UNICODE", "Ixİ", "i\u0307", "Ixİ"); - assertStringTrimLeft("UNICODE", "IXİ", "ix\u0307", "IXİ"); - assertStringTrimLeft("UNICODE", "xi\u0307", "\u0307IX", "xi\u0307"); - assertStringTrimLeft("UNICODE_CI", "i", "i", ""); - assertStringTrimLeft("UNICODE_CI", "iii", "I", ""); - assertStringTrimLeft("UNICODE_CI", "I", "iii", ""); - assertStringTrimLeft("UNICODE_CI", "ixi", "i", "xi"); - assertStringTrimLeft("UNICODE_CI", "i", "İ", "i"); - assertStringTrimLeft("UNICODE_CI", "i\u0307", "İ", ""); - assertStringTrimLeft("UNICODE_CI", "ii\u0307", "İi", ""); - assertStringTrimLeft("UNICODE_CI", "iii\u0307", "İi", ""); - assertStringTrimLeft("UNICODE_CI", "iiii\u0307", "iİ", ""); - assertStringTrimLeft("UNICODE_CI", "ii\u0307ii\u0307", "iİ", ""); - assertStringTrimLeft("UNICODE_CI", "i\u0307", "i", "i\u0307"); - assertStringTrimLeft("UNICODE_CI", "i\u0307", "\u0307", "i\u0307"); - assertStringTrimLeft("UNICODE_CI", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrimLeft("UNICODE_CI", "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); - assertStringTrimLeft("UNICODE_CI", "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); - assertStringTrimLeft("UNICODE_CI", "i\u0307i", "i\u0307", "i\u0307i"); - assertStringTrimLeft("UNICODE_CI", "i\u0307i", "İ", "i"); - assertStringTrimLeft("UNICODE_CI", "i\u0307İ", "i\u0307", "i\u0307İ"); - assertStringTrimLeft("UNICODE_CI", "i\u0307İ", "İ", ""); - assertStringTrimLeft("UNICODE_CI", "İ", "İ", ""); - assertStringTrimLeft("UNICODE_CI", "IXi", "İ", "IXi"); - assertStringTrimLeft("UNICODE_CI", "ix\u0307", "Ixİ", "x\u0307"); - assertStringTrimLeft("UNICODE_CI", "i\u0307x", "IXİ", ""); - assertStringTrimLeft("UNICODE_CI", "i\u0307x", "I\u0307xİ", ""); - assertStringTrimLeft("UNICODE_CI", "İ", "i", "İ"); - assertStringTrimLeft("UNICODE_CI", "İ", "\u0307", "İ"); - assertStringTrimLeft("UNICODE_CI", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrimLeft("UNICODE_CI", "Ixİ", "i\u0307", "xİ"); - assertStringTrimLeft("UNICODE_CI", "IXİ", "ix\u0307", "İ"); - assertStringTrimLeft("UNICODE_CI", "xi\u0307", "\u0307IX", "i\u0307"); + assertStringTrimLeft(UTF8_BINARY, "i", "i", ""); + assertStringTrimLeft(UTF8_BINARY, "iii", "I", "iii"); + assertStringTrimLeft(UTF8_BINARY, "I", "iii", "I"); + assertStringTrimLeft(UTF8_BINARY, "ixi", "i", "xi"); + assertStringTrimLeft(UTF8_BINARY, "i", "İ", "i"); + assertStringTrimLeft(UTF8_BINARY, "i\u0307", "İ", "i\u0307"); + assertStringTrimLeft(UTF8_BINARY, "ii\u0307", "İi", "\u0307"); + assertStringTrimLeft(UTF8_BINARY, "iii\u0307", "İi", "\u0307"); + assertStringTrimLeft(UTF8_BINARY, "iiii\u0307", "iİ", "\u0307"); + assertStringTrimLeft(UTF8_BINARY, "ii\u0307ii\u0307", "iİ", "\u0307ii\u0307"); + assertStringTrimLeft(UTF8_BINARY, "i\u0307", "i", "\u0307"); + assertStringTrimLeft(UTF8_BINARY, "i\u0307", "\u0307", "i\u0307"); + assertStringTrimLeft(UTF8_BINARY, "i\u0307", "i\u0307", ""); + assertStringTrimLeft(UTF8_BINARY, "i\u0307i\u0307", "i\u0307", ""); + assertStringTrimLeft(UTF8_BINARY, "i\u0307\u0307", "i\u0307", ""); + assertStringTrimLeft(UTF8_BINARY, "i\u0307i", "i\u0307", ""); + assertStringTrimLeft(UTF8_BINARY, "i\u0307i", "İ", "i\u0307i"); + assertStringTrimLeft(UTF8_BINARY, "i\u0307İ", "i\u0307", "İ"); + assertStringTrimLeft(UTF8_BINARY, "i\u0307İ", "İ", "i\u0307İ"); + assertStringTrimLeft(UTF8_BINARY, "İ", "İ", ""); + assertStringTrimLeft(UTF8_BINARY, "IXi", "İ", "IXi"); + assertStringTrimLeft(UTF8_BINARY, "ix\u0307", "Ixİ", "ix\u0307"); + assertStringTrimLeft(UTF8_BINARY, "i\u0307x", "IXİ", "i\u0307x"); + assertStringTrimLeft(UTF8_BINARY, "i\u0307x", "ix\u0307İ", ""); + assertStringTrimLeft(UTF8_BINARY, "İ", "i", "İ"); + assertStringTrimLeft(UTF8_BINARY, "İ", "\u0307", "İ"); + assertStringTrimLeft(UTF8_BINARY, "Ixİ", "i\u0307", "Ixİ"); + assertStringTrimLeft(UTF8_BINARY, "IXİ", "ix\u0307", "IXİ"); + assertStringTrimLeft(UTF8_BINARY, "xi\u0307", "\u0307IX", "xi\u0307"); + assertStringTrimLeft(UTF8_LCASE, "i", "i", ""); + assertStringTrimLeft(UTF8_LCASE, "iii", "I", ""); + assertStringTrimLeft(UTF8_LCASE, "I", "iii", ""); + assertStringTrimLeft(UTF8_LCASE, "ixi", "i", "xi"); + assertStringTrimLeft(UTF8_LCASE, "i", "İ", "i"); + assertStringTrimLeft(UTF8_LCASE, "i\u0307", "İ", ""); + assertStringTrimLeft(UTF8_LCASE, "ii\u0307", "İi", ""); + assertStringTrimLeft(UTF8_LCASE, "iii\u0307", "İi", ""); + assertStringTrimLeft(UTF8_LCASE, "iiii\u0307", "iİ", ""); + assertStringTrimLeft(UTF8_LCASE, "ii\u0307ii\u0307", "iİ", ""); + assertStringTrimLeft(UTF8_LCASE, "i\u0307", "i", "\u0307"); + assertStringTrimLeft(UTF8_LCASE, "i\u0307", "\u0307", "i\u0307"); + assertStringTrimLeft(UTF8_LCASE, "i\u0307", "i\u0307", ""); + assertStringTrimLeft(UTF8_LCASE, "i\u0307i\u0307", "i\u0307", ""); + assertStringTrimLeft(UTF8_LCASE, "i\u0307\u0307", "i\u0307", ""); + assertStringTrimLeft(UTF8_LCASE, "i\u0307i", "i\u0307", ""); + assertStringTrimLeft(UTF8_LCASE, "i\u0307i", "İ", "i"); + assertStringTrimLeft(UTF8_LCASE, "i\u0307İ", "i\u0307", "İ"); + assertStringTrimLeft(UTF8_LCASE, "i\u0307İ", "İ", ""); + assertStringTrimLeft(UTF8_LCASE, "İ", "İ", ""); + assertStringTrimLeft(UTF8_LCASE, "IXi", "İ", "IXi"); + assertStringTrimLeft(UTF8_LCASE, "ix\u0307", "Ixİ", "\u0307"); + assertStringTrimLeft(UTF8_LCASE, "i\u0307x", "IXİ", ""); + assertStringTrimLeft(UTF8_LCASE, "i\u0307x", "I\u0307xİ", ""); + assertStringTrimLeft(UTF8_LCASE, "İ", "i", "İ"); + assertStringTrimLeft(UTF8_LCASE, "İ", "\u0307", "İ"); + assertStringTrimLeft(UTF8_LCASE, "Ixİ", "i\u0307", "xİ"); + assertStringTrimLeft(UTF8_LCASE, "IXİ", "ix\u0307", "İ"); + assertStringTrimLeft(UTF8_LCASE, "xi\u0307", "\u0307IX", ""); + assertStringTrimLeft(UNICODE, "i", "i", ""); + assertStringTrimLeft(UNICODE, "iii", "I", "iii"); + assertStringTrimLeft(UNICODE, "I", "iii", "I"); + assertStringTrimLeft(UNICODE, "ixi", "i", "xi"); + assertStringTrimLeft(UNICODE, "i", "İ", "i"); + assertStringTrimLeft(UNICODE, "i\u0307", "İ", "i\u0307"); + assertStringTrimLeft(UNICODE, "ii\u0307", "İi", "i\u0307"); + assertStringTrimLeft(UNICODE, "iii\u0307", "İi", "i\u0307"); + assertStringTrimLeft(UNICODE, "iiii\u0307", "iİ", "i\u0307"); + assertStringTrimLeft(UNICODE, "ii\u0307ii\u0307", "iİ", "i\u0307ii\u0307"); + assertStringTrimLeft(UNICODE, "i\u0307", "i", "i\u0307"); + assertStringTrimLeft(UNICODE, "i\u0307", "\u0307", "i\u0307"); + assertStringTrimLeft(UNICODE, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrimLeft(UNICODE, "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); + assertStringTrimLeft(UNICODE, "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); + assertStringTrimLeft(UNICODE, "i\u0307i", "i\u0307", "i\u0307i"); + assertStringTrimLeft(UNICODE, "i\u0307i", "İ", "i\u0307i"); + assertStringTrimLeft(UNICODE, "i\u0307İ", "i\u0307", "i\u0307İ"); + assertStringTrimLeft(UNICODE, "i\u0307İ", "İ", "i\u0307İ"); + assertStringTrimLeft(UNICODE, "İ", "İ", ""); + assertStringTrimLeft(UNICODE, "IXi", "İ", "IXi"); + assertStringTrimLeft(UNICODE, "ix\u0307", "Ixİ", "ix\u0307"); + assertStringTrimLeft(UNICODE, "i\u0307x", "IXİ", "i\u0307x"); + assertStringTrimLeft(UNICODE, "i\u0307x", "ix\u0307İ", "i\u0307x"); + assertStringTrimLeft(UNICODE, "İ", "i", "İ"); + assertStringTrimLeft(UNICODE, "İ", "\u0307", "İ"); + assertStringTrimLeft(UNICODE, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrimLeft(UNICODE, "Ixİ", "i\u0307", "Ixİ"); + assertStringTrimLeft(UNICODE, "IXİ", "ix\u0307", "IXİ"); + assertStringTrimLeft(UNICODE, "xi\u0307", "\u0307IX", "xi\u0307"); + assertStringTrimLeft(UNICODE_CI, "i", "i", ""); + assertStringTrimLeft(UNICODE_CI, "iii", "I", ""); + assertStringTrimLeft(UNICODE_CI, "I", "iii", ""); + assertStringTrimLeft(UNICODE_CI, "ixi", "i", "xi"); + assertStringTrimLeft(UNICODE_CI, "i", "İ", "i"); + assertStringTrimLeft(UNICODE_CI, "i\u0307", "İ", ""); + assertStringTrimLeft(UNICODE_CI, "ii\u0307", "İi", ""); + assertStringTrimLeft(UNICODE_CI, "iii\u0307", "İi", ""); + assertStringTrimLeft(UNICODE_CI, "iiii\u0307", "iİ", ""); + assertStringTrimLeft(UNICODE_CI, "ii\u0307ii\u0307", "iİ", ""); + assertStringTrimLeft(UNICODE_CI, "i\u0307", "i", "i\u0307"); + assertStringTrimLeft(UNICODE_CI, "i\u0307", "\u0307", "i\u0307"); + assertStringTrimLeft(UNICODE_CI, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrimLeft(UNICODE_CI, "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); + assertStringTrimLeft(UNICODE_CI, "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); + assertStringTrimLeft(UNICODE_CI, "i\u0307i", "i\u0307", "i\u0307i"); + assertStringTrimLeft(UNICODE_CI, "i\u0307i", "İ", "i"); + assertStringTrimLeft(UNICODE_CI, "i\u0307İ", "i\u0307", "i\u0307İ"); + assertStringTrimLeft(UNICODE_CI, "i\u0307İ", "İ", ""); + assertStringTrimLeft(UNICODE_CI, "İ", "İ", ""); + assertStringTrimLeft(UNICODE_CI, "IXi", "İ", "IXi"); + assertStringTrimLeft(UNICODE_CI, "ix\u0307", "Ixİ", "x\u0307"); + assertStringTrimLeft(UNICODE_CI, "i\u0307x", "IXİ", ""); + assertStringTrimLeft(UNICODE_CI, "i\u0307x", "I\u0307xİ", ""); + assertStringTrimLeft(UNICODE_CI, "İ", "i", "İ"); + assertStringTrimLeft(UNICODE_CI, "İ", "\u0307", "İ"); + assertStringTrimLeft(UNICODE_CI, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrimLeft(UNICODE_CI, "Ixİ", "i\u0307", "xİ"); + assertStringTrimLeft(UNICODE_CI, "IXİ", "ix\u0307", "İ"); + assertStringTrimLeft(UNICODE_CI, "xi\u0307", "\u0307IX", "i\u0307"); // Conditional case mapping (e.g. Greek sigmas). - assertStringTrimLeft("UTF8_BINARY", "ςxς", "σ", "ςxς"); - assertStringTrimLeft("UTF8_BINARY", "ςxς", "ς", "xς"); - assertStringTrimLeft("UTF8_BINARY", "ςxς", "Σ", "ςxς"); - assertStringTrimLeft("UTF8_BINARY", "σxσ", "σ", "xσ"); - assertStringTrimLeft("UTF8_BINARY", "σxσ", "ς", "σxσ"); - assertStringTrimLeft("UTF8_BINARY", "σxσ", "Σ", "σxσ"); - assertStringTrimLeft("UTF8_BINARY", "ΣxΣ", "σ", "ΣxΣ"); - assertStringTrimLeft("UTF8_BINARY", "ΣxΣ", "ς", "ΣxΣ"); - assertStringTrimLeft("UTF8_BINARY", "ΣxΣ", "Σ", "xΣ"); - assertStringTrimLeft("UTF8_LCASE", "ςxς", "σ", "xς"); - assertStringTrimLeft("UTF8_LCASE", "ςxς", "ς", "xς"); - assertStringTrimLeft("UTF8_LCASE", "ςxς", "Σ", "xς"); - assertStringTrimLeft("UTF8_LCASE", "σxσ", "σ", "xσ"); - assertStringTrimLeft("UTF8_LCASE", "σxσ", "ς", "xσ"); - assertStringTrimLeft("UTF8_LCASE", "σxσ", "Σ", "xσ"); - assertStringTrimLeft("UTF8_LCASE", "ΣxΣ", "σ", "xΣ"); - assertStringTrimLeft("UTF8_LCASE", "ΣxΣ", "ς", "xΣ"); - assertStringTrimLeft("UTF8_LCASE", "ΣxΣ", "Σ", "xΣ"); - assertStringTrimLeft("UNICODE", "ςxς", "σ", "ςxς"); - assertStringTrimLeft("UNICODE", "ςxς", "ς", "xς"); - assertStringTrimLeft("UNICODE", "ςxς", "Σ", "ςxς"); - assertStringTrimLeft("UNICODE", "σxσ", "σ", "xσ"); - assertStringTrimLeft("UNICODE", "σxσ", "ς", "σxσ"); - assertStringTrimLeft("UNICODE", "σxσ", "Σ", "σxσ"); - assertStringTrimLeft("UNICODE", "ΣxΣ", "σ", "ΣxΣ"); - assertStringTrimLeft("UNICODE", "ΣxΣ", "ς", "ΣxΣ"); - assertStringTrimLeft("UNICODE", "ΣxΣ", "Σ", "xΣ"); - assertStringTrimLeft("UNICODE_CI", "ςxς", "σ", "xς"); - assertStringTrimLeft("UNICODE_CI", "ςxς", "ς", "xς"); - assertStringTrimLeft("UNICODE_CI", "ςxς", "Σ", "xς"); - assertStringTrimLeft("UNICODE_CI", "σxσ", "σ", "xσ"); - assertStringTrimLeft("UNICODE_CI", "σxσ", "ς", "xσ"); - assertStringTrimLeft("UNICODE_CI", "σxσ", "Σ", "xσ"); - assertStringTrimLeft("UNICODE_CI", "ΣxΣ", "σ", "xΣ"); - assertStringTrimLeft("UNICODE_CI", "ΣxΣ", "ς", "xΣ"); - assertStringTrimLeft("UNICODE_CI", "ΣxΣ", "Σ", "xΣ"); + assertStringTrimLeft(UTF8_BINARY, "ςxς", "σ", "ςxς"); + assertStringTrimLeft(UTF8_BINARY, "ςxς", "ς", "xς"); + assertStringTrimLeft(UTF8_BINARY, "ςxς", "Σ", "ςxς"); + assertStringTrimLeft(UTF8_BINARY, "σxσ", "σ", "xσ"); + assertStringTrimLeft(UTF8_BINARY, "σxσ", "ς", "σxσ"); + assertStringTrimLeft(UTF8_BINARY, "σxσ", "Σ", "σxσ"); + assertStringTrimLeft(UTF8_BINARY, "ΣxΣ", "σ", "ΣxΣ"); + assertStringTrimLeft(UTF8_BINARY, "ΣxΣ", "ς", "ΣxΣ"); + assertStringTrimLeft(UTF8_BINARY, "ΣxΣ", "Σ", "xΣ"); + assertStringTrimLeft(UTF8_LCASE, "ςxς", "σ", "xς"); + assertStringTrimLeft(UTF8_LCASE, "ςxς", "ς", "xς"); + assertStringTrimLeft(UTF8_LCASE, "ςxς", "Σ", "xς"); + assertStringTrimLeft(UTF8_LCASE, "σxσ", "σ", "xσ"); + assertStringTrimLeft(UTF8_LCASE, "σxσ", "ς", "xσ"); + assertStringTrimLeft(UTF8_LCASE, "σxσ", "Σ", "xσ"); + assertStringTrimLeft(UTF8_LCASE, "ΣxΣ", "σ", "xΣ"); + assertStringTrimLeft(UTF8_LCASE, "ΣxΣ", "ς", "xΣ"); + assertStringTrimLeft(UTF8_LCASE, "ΣxΣ", "Σ", "xΣ"); + assertStringTrimLeft(UNICODE, "ςxς", "σ", "ςxς"); + assertStringTrimLeft(UNICODE, "ςxς", "ς", "xς"); + assertStringTrimLeft(UNICODE, "ςxς", "Σ", "ςxς"); + assertStringTrimLeft(UNICODE, "σxσ", "σ", "xσ"); + assertStringTrimLeft(UNICODE, "σxσ", "ς", "σxσ"); + assertStringTrimLeft(UNICODE, "σxσ", "Σ", "σxσ"); + assertStringTrimLeft(UNICODE, "ΣxΣ", "σ", "ΣxΣ"); + assertStringTrimLeft(UNICODE, "ΣxΣ", "ς", "ΣxΣ"); + assertStringTrimLeft(UNICODE, "ΣxΣ", "Σ", "xΣ"); + assertStringTrimLeft(UNICODE_CI, "ςxς", "σ", "xς"); + assertStringTrimLeft(UNICODE_CI, "ςxς", "ς", "xς"); + assertStringTrimLeft(UNICODE_CI, "ςxς", "Σ", "xς"); + assertStringTrimLeft(UNICODE_CI, "σxσ", "σ", "xσ"); + assertStringTrimLeft(UNICODE_CI, "σxσ", "ς", "xσ"); + assertStringTrimLeft(UNICODE_CI, "σxσ", "Σ", "xσ"); + assertStringTrimLeft(UNICODE_CI, "ΣxΣ", "σ", "xΣ"); + assertStringTrimLeft(UNICODE_CI, "ΣxΣ", "ς", "xΣ"); + assertStringTrimLeft(UNICODE_CI, "ΣxΣ", "Σ", "xΣ"); // Unicode normalization. - assertStringTrimLeft("UTF8_BINARY", "åβγδa\u030A", "å", "βγδa\u030A"); - assertStringTrimLeft("UTF8_LCASE", "åβγδa\u030A", "Å", "βγδa\u030A"); - assertStringTrimLeft("UNICODE", "åβγδa\u030A", "å", "βγδa\u030A"); - assertStringTrimLeft("UNICODE_CI", "åβγδa\u030A", "Å", "βγδa\u030A"); + assertStringTrimLeft(UTF8_BINARY, "åβγδa\u030A", "å", "βγδa\u030A"); + assertStringTrimLeft(UTF8_LCASE, "åβγδa\u030A", "Å", "βγδa\u030A"); + assertStringTrimLeft(UNICODE, "åβγδa\u030A", "å", "βγδa\u030A"); + assertStringTrimLeft(UNICODE_CI, "åβγδa\u030A", "Å", "βγδa\u030A"); // Surrogate pairs. - assertStringTrimLeft("UTF8_BINARY", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrimLeft("UTF8_LCASE", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrimLeft("UNICODE", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrimLeft("UNICODE_CI", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrimLeft("UTF8_BINARY", "a🙃b🙃c", "a", "🙃b🙃c"); - assertStringTrimLeft("UTF8_LCASE", "a🙃b🙃c", "a", "🙃b🙃c"); - assertStringTrimLeft("UNICODE", "a🙃b🙃c", "a", "🙃b🙃c"); - assertStringTrimLeft("UNICODE_CI", "a🙃b🙃c", "a", "🙃b🙃c"); - assertStringTrimLeft("UTF8_BINARY", "a🙃b🙃c", "a🙃", "b🙃c"); - assertStringTrimLeft("UTF8_LCASE", "a🙃b🙃c", "a🙃", "b🙃c"); - assertStringTrimLeft("UNICODE", "a🙃b🙃c", "a🙃", "b🙃c"); - assertStringTrimLeft("UNICODE_CI", "a🙃b🙃c", "a🙃", "b🙃c"); - assertStringTrimLeft("UTF8_BINARY", "a🙃b🙃c", "a🙃b", "c"); - assertStringTrimLeft("UTF8_LCASE", "a🙃b🙃c", "a🙃b", "c"); - assertStringTrimLeft("UNICODE", "a🙃b🙃c", "a🙃b", "c"); - assertStringTrimLeft("UNICODE_CI", "a🙃b🙃c", "a🙃b", "c"); - assertStringTrimLeft("UTF8_BINARY", "a🙃b🙃c", "abc🙃", ""); - assertStringTrimLeft("UTF8_LCASE", "a🙃b🙃c", "abc🙃", ""); - assertStringTrimLeft("UNICODE", "a🙃b🙃c", "abc🙃", ""); - assertStringTrimLeft("UNICODE_CI", "a🙃b🙃c", "abc🙃", ""); - assertStringTrimLeft("UTF8_BINARY", "😀😆😃😄", "😆😃", "😀😆😃😄"); - assertStringTrimLeft("UTF8_LCASE", "😀😆😃😄", "😆😃", "😀😆😃😄"); - assertStringTrimLeft("UNICODE", "😀😆😃😄", "😆😃", "😀😆😃😄"); - assertStringTrimLeft("UNICODE_CI", "😀😆😃😄", "😆😃", "😀😆😃😄"); - assertStringTrimLeft("UTF8_BINARY", "😀😆😃😄", "😀😆", "😃😄"); - assertStringTrimLeft("UTF8_LCASE", "😀😆😃😄", "😀😆", "😃😄"); - assertStringTrimLeft("UNICODE", "😀😆😃😄", "😀😆", "😃😄"); - assertStringTrimLeft("UNICODE_CI", "😀😆😃😄", "😀😆", "😃😄"); - assertStringTrimLeft("UTF8_BINARY", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrimLeft("UTF8_LCASE", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrimLeft("UNICODE", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrimLeft("UNICODE_CI", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrimLeft("UTF8_BINARY", "𐐅", "𐐅", ""); - assertStringTrimLeft("UTF8_LCASE", "𐐅", "𐐅", ""); - assertStringTrimLeft("UNICODE", "𐐅", "𐐅", ""); - assertStringTrimLeft("UNICODE_CI", "𐐅", "𐐅", ""); - assertStringTrimLeft("UTF8_BINARY", "𐐅", "𐐭", "𐐅"); - assertStringTrimLeft("UTF8_LCASE", "𐐅", "𐐭", ""); - assertStringTrimLeft("UNICODE", "𐐅", "𐐭", "𐐅"); - assertStringTrimLeft("UNICODE_CI", "𐐅", "𐐭", ""); - assertStringTrimLeft("UTF8_BINARY", "𝔸", "𝔸", ""); - assertStringTrimLeft("UTF8_LCASE", "𝔸", "𝔸", ""); - assertStringTrimLeft("UNICODE", "𝔸", "𝔸", ""); - assertStringTrimLeft("UNICODE_CI", "𝔸", "𝔸", ""); - assertStringTrimLeft("UTF8_BINARY", "𝔸", "A", "𝔸"); - assertStringTrimLeft("UTF8_LCASE", "𝔸", "A", "𝔸"); - assertStringTrimLeft("UNICODE", "𝔸", "A", "𝔸"); - assertStringTrimLeft("UNICODE_CI", "𝔸", "A", ""); - assertStringTrimLeft("UTF8_BINARY", "𝔸", "a", "𝔸"); - assertStringTrimLeft("UTF8_LCASE", "𝔸", "a", "𝔸"); - assertStringTrimLeft("UNICODE", "𝔸", "a", "𝔸"); - assertStringTrimLeft("UNICODE_CI", "𝔸", "a", ""); + assertStringTrimLeft(UTF8_BINARY, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrimLeft(UTF8_LCASE, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrimLeft(UNICODE, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrimLeft(UNICODE_CI, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrimLeft(UTF8_BINARY, "a🙃b🙃c", "a", "🙃b🙃c"); + assertStringTrimLeft(UTF8_LCASE, "a🙃b🙃c", "a", "🙃b🙃c"); + assertStringTrimLeft(UNICODE, "a🙃b🙃c", "a", "🙃b🙃c"); + assertStringTrimLeft(UNICODE_CI, "a🙃b🙃c", "a", "🙃b🙃c"); + assertStringTrimLeft(UTF8_BINARY, "a🙃b🙃c", "a🙃", "b🙃c"); + assertStringTrimLeft(UTF8_LCASE, "a🙃b🙃c", "a🙃", "b🙃c"); + assertStringTrimLeft(UNICODE, "a🙃b🙃c", "a🙃", "b🙃c"); + assertStringTrimLeft(UNICODE_CI, "a🙃b🙃c", "a🙃", "b🙃c"); + assertStringTrimLeft(UTF8_BINARY, "a🙃b🙃c", "a🙃b", "c"); + assertStringTrimLeft(UTF8_LCASE, "a🙃b🙃c", "a🙃b", "c"); + assertStringTrimLeft(UNICODE, "a🙃b🙃c", "a🙃b", "c"); + assertStringTrimLeft(UNICODE_CI, "a🙃b🙃c", "a🙃b", "c"); + assertStringTrimLeft(UTF8_BINARY, "a🙃b🙃c", "abc🙃", ""); + assertStringTrimLeft(UTF8_LCASE, "a🙃b🙃c", "abc🙃", ""); + assertStringTrimLeft(UNICODE, "a🙃b🙃c", "abc🙃", ""); + assertStringTrimLeft(UNICODE_CI, "a🙃b🙃c", "abc🙃", ""); + assertStringTrimLeft(UTF8_BINARY, "😀😆😃😄", "😆😃", "😀😆😃😄"); + assertStringTrimLeft(UTF8_LCASE, "😀😆😃😄", "😆😃", "😀😆😃😄"); + assertStringTrimLeft(UNICODE, "😀😆😃😄", "😆😃", "😀😆😃😄"); + assertStringTrimLeft(UNICODE_CI, "😀😆😃😄", "😆😃", "😀😆😃😄"); + assertStringTrimLeft(UTF8_BINARY, "😀😆😃😄", "😀😆", "😃😄"); + assertStringTrimLeft(UTF8_LCASE, "😀😆😃😄", "😀😆", "😃😄"); + assertStringTrimLeft(UNICODE, "😀😆😃😄", "😀😆", "😃😄"); + assertStringTrimLeft(UNICODE_CI, "😀😆😃😄", "😀😆", "😃😄"); + assertStringTrimLeft(UTF8_BINARY, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrimLeft(UTF8_LCASE, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrimLeft(UNICODE, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrimLeft(UNICODE_CI, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrimLeft(UTF8_BINARY, "𐐅", "𐐅", ""); + assertStringTrimLeft(UTF8_LCASE, "𐐅", "𐐅", ""); + assertStringTrimLeft(UNICODE, "𐐅", "𐐅", ""); + assertStringTrimLeft(UNICODE_CI, "𐐅", "𐐅", ""); + assertStringTrimLeft(UTF8_BINARY, "𐐅", "𐐭", "𐐅"); + assertStringTrimLeft(UTF8_LCASE, "𐐅", "𐐭", ""); + assertStringTrimLeft(UNICODE, "𐐅", "𐐭", "𐐅"); + assertStringTrimLeft(UNICODE_CI, "𐐅", "𐐭", ""); + assertStringTrimLeft(UTF8_BINARY, "𝔸", "𝔸", ""); + assertStringTrimLeft(UTF8_LCASE, "𝔸", "𝔸", ""); + assertStringTrimLeft(UNICODE, "𝔸", "𝔸", ""); + assertStringTrimLeft(UNICODE_CI, "𝔸", "𝔸", ""); + assertStringTrimLeft(UTF8_BINARY, "𝔸", "A", "𝔸"); + assertStringTrimLeft(UTF8_LCASE, "𝔸", "A", "𝔸"); + assertStringTrimLeft(UNICODE, "𝔸", "A", "𝔸"); + assertStringTrimLeft(UNICODE_CI, "𝔸", "A", ""); + assertStringTrimLeft(UTF8_BINARY, "𝔸", "a", "𝔸"); + assertStringTrimLeft(UTF8_LCASE, "𝔸", "a", "𝔸"); + assertStringTrimLeft(UNICODE, "𝔸", "a", "𝔸"); + assertStringTrimLeft(UNICODE_CI, "𝔸", "a", ""); } /** @@ -3378,274 +3379,274 @@ private void assertStringTrimRight(String collationName, String sourceString, St @Test public void testStringTrimRight() throws SparkException { // Basic tests. - assertStringTrimRight("UTF8_BINARY", "", "", ""); - assertStringTrimRight("UTF8_BINARY", "", "xyz", ""); - assertStringTrimRight("UTF8_BINARY", "asd", "", "asd"); - assertStringTrimRight("UTF8_BINARY", "asd", null, "asd"); - assertStringTrimRight("UTF8_BINARY", " asd ", null, " asd"); - assertStringTrimRight("UTF8_BINARY", " a世a ", null, " a世a"); - assertStringTrimRight("UTF8_BINARY", "asd", "x", "asd"); - assertStringTrimRight("UTF8_BINARY", "xxasdxx", "x", "xxasd"); - assertStringTrimRight("UTF8_BINARY", "xa世ax", "x", "xa世a"); - assertStringTrimRight("UTF8_LCASE", "", "", ""); - assertStringTrimRight("UTF8_LCASE", "", "xyz", ""); - assertStringTrimRight("UTF8_LCASE", "asd", "", "asd"); - 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"); - assertStringTrimRight("UNICODE", "", "", ""); - assertStringTrimRight("UNICODE", "", "xyz", ""); - assertStringTrimRight("UNICODE", "asd", "", "asd"); - assertStringTrimRight("UNICODE", "asd", null, "asd"); - assertStringTrimRight("UNICODE", " asd ", null, " asd"); - assertStringTrimRight("UNICODE", " a世a ", null, " a世a"); - assertStringTrimRight("UNICODE", "asd", "x", "asd"); - assertStringTrimRight("UNICODE", "xxasdxx", "x", "xxasd"); - assertStringTrimRight("UNICODE", "xa世ax", "x", "xa世a"); - assertStringTrimRight("UNICODE_CI", "", "", ""); - assertStringTrimRight("UNICODE_CI", "", "xyz", ""); - assertStringTrimRight("UNICODE_CI", "asd", "", "asd"); - assertStringTrimRight("UNICODE_CI", "asd", null, "asd"); - assertStringTrimRight("UNICODE_CI", " asd ", null, " asd"); - assertStringTrimRight("UNICODE_CI", " a世a ", null, " a世a"); - assertStringTrimRight("UNICODE_CI", "asd", "x", "asd"); - assertStringTrimRight("UNICODE_CI", "xxasdxx", "x", "xxasd"); - assertStringTrimRight("UNICODE_CI", "xa世ax", "x", "xa世a"); + assertStringTrimRight(UTF8_BINARY, "", "", ""); + assertStringTrimRight(UTF8_BINARY, "", "xyz", ""); + assertStringTrimRight(UTF8_BINARY, "asd", "", "asd"); + assertStringTrimRight(UTF8_BINARY, "asd", null, "asd"); + assertStringTrimRight(UTF8_BINARY, " asd ", null, " asd"); + assertStringTrimRight(UTF8_BINARY, " a世a ", null, " a世a"); + assertStringTrimRight(UTF8_BINARY, "asd", "x", "asd"); + assertStringTrimRight(UTF8_BINARY, "xxasdxx", "x", "xxasd"); + assertStringTrimRight(UTF8_BINARY, "xa世ax", "x", "xa世a"); + assertStringTrimRight(UTF8_LCASE, "", "", ""); + assertStringTrimRight(UTF8_LCASE, "", "xyz", ""); + assertStringTrimRight(UTF8_LCASE, "asd", "", "asd"); + 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"); + assertStringTrimRight(UNICODE, "", "", ""); + assertStringTrimRight(UNICODE, "", "xyz", ""); + assertStringTrimRight(UNICODE, "asd", "", "asd"); + assertStringTrimRight(UNICODE, "asd", null, "asd"); + assertStringTrimRight(UNICODE, " asd ", null, " asd"); + assertStringTrimRight(UNICODE, " a世a ", null, " a世a"); + assertStringTrimRight(UNICODE, "asd", "x", "asd"); + assertStringTrimRight(UNICODE, "xxasdxx", "x", "xxasd"); + assertStringTrimRight(UNICODE, "xa世ax", "x", "xa世a"); + assertStringTrimRight(UNICODE_CI, "", "", ""); + assertStringTrimRight(UNICODE_CI, "", "xyz", ""); + assertStringTrimRight(UNICODE_CI, "asd", "", "asd"); + assertStringTrimRight(UNICODE_CI, "asd", null, "asd"); + assertStringTrimRight(UNICODE_CI, " asd ", null, " asd"); + assertStringTrimRight(UNICODE_CI, " a世a ", null, " a世a"); + assertStringTrimRight(UNICODE_CI, "asd", "x", "asd"); + assertStringTrimRight(UNICODE_CI, "xxasdxx", "x", "xxasd"); + assertStringTrimRight(UNICODE_CI, "xa世ax", "x", "xa世a"); // Case variation. - assertStringTrimRight("UTF8_BINARY", "ddsXXXaa", "asd", "ddsXXX"); - assertStringTrimRight("UTF8_LCASE", "ddsXXXaa", "AsD", "ddsXXX"); - assertStringTrimRight("UNICODE", "ddsXXXaa", "asd", "ddsXXX"); - assertStringTrimRight("UNICODE_CI", "ddsXXXaa", "AsD", "ddsXXX"); + assertStringTrimRight(UTF8_BINARY, "ddsXXXaa", "asd", "ddsXXX"); + assertStringTrimRight(UTF8_LCASE, "ddsXXXaa", "AsD", "ddsXXX"); + assertStringTrimRight(UNICODE, "ddsXXXaa", "asd", "ddsXXX"); + assertStringTrimRight(UNICODE_CI, "ddsXXXaa", "AsD", "ddsXXX"); // One-to-many case mapping (e.g. Turkish dotted I).. - assertStringTrimRight("UTF8_BINARY", "ẞaaaẞ", "ß", "ẞaaaẞ"); - assertStringTrimRight("UTF8_BINARY", "ßaaaß", "ẞ", "ßaaaß"); - assertStringTrimRight("UTF8_BINARY", "Ëaaaẞ", "Ëẞ", "Ëaaa"); - assertStringTrimRight("UTF8_LCASE", "ẞaaaẞ", "ß", "ẞaaa"); - assertStringTrimRight("UTF8_LCASE", "ßaaaß", "ẞ", "ßaaa"); - assertStringTrimRight("UTF8_LCASE", "Ëaaaẞ", "Ëẞ", "Ëaaa"); - assertStringTrimRight("UNICODE", "ẞaaaẞ", "ß", "ẞaaaẞ"); - assertStringTrimRight("UNICODE", "ßaaaß", "ẞ", "ßaaaß"); - assertStringTrimRight("UNICODE", "Ëaaaẞ", "Ëẞ", "Ëaaa"); - assertStringTrimRight("UNICODE_CI", "ẞaaaẞ", "ß", "ẞaaa"); - assertStringTrimRight("UNICODE_CI", "ßaaaß", "ẞ", "ßaaa"); - assertStringTrimRight("UNICODE_CI", "Ëaaaẞ", "Ëẞ", "Ëaaa"); + assertStringTrimRight(UTF8_BINARY, "ẞaaaẞ", "ß", "ẞaaaẞ"); + assertStringTrimRight(UTF8_BINARY, "ßaaaß", "ẞ", "ßaaaß"); + assertStringTrimRight(UTF8_BINARY, "Ëaaaẞ", "Ëẞ", "Ëaaa"); + assertStringTrimRight(UTF8_LCASE, "ẞaaaẞ", "ß", "ẞaaa"); + assertStringTrimRight(UTF8_LCASE, "ßaaaß", "ẞ", "ßaaa"); + assertStringTrimRight(UTF8_LCASE, "Ëaaaẞ", "Ëẞ", "Ëaaa"); + assertStringTrimRight(UNICODE, "ẞaaaẞ", "ß", "ẞaaaẞ"); + assertStringTrimRight(UNICODE, "ßaaaß", "ẞ", "ßaaaß"); + assertStringTrimRight(UNICODE, "Ëaaaẞ", "Ëẞ", "Ëaaa"); + assertStringTrimRight(UNICODE_CI, "ẞaaaẞ", "ß", "ẞaaa"); + assertStringTrimRight(UNICODE_CI, "ßaaaß", "ẞ", "ßaaa"); + assertStringTrimRight(UNICODE_CI, "Ëaaaẞ", "Ëẞ", "Ëaaa"); // One-to-many case mapping (e.g. Turkish dotted I). - assertStringTrimRight("UTF8_BINARY", "i", "i", ""); - assertStringTrimRight("UTF8_BINARY", "iii", "I", "iii"); - assertStringTrimRight("UTF8_BINARY", "I", "iii", "I"); - assertStringTrimRight("UTF8_BINARY", "ixi", "i", "ix"); - assertStringTrimRight("UTF8_BINARY", "i", "İ", "i"); - assertStringTrimRight("UTF8_BINARY", "i\u0307", "İ", "i\u0307"); - assertStringTrimRight("UTF8_BINARY", "ii\u0307", "İi", "ii\u0307"); - assertStringTrimRight("UTF8_BINARY", "iii\u0307", "İi", "iii\u0307"); - assertStringTrimRight("UTF8_BINARY", "iiii\u0307", "iİ", "iiii\u0307"); - assertStringTrimRight("UTF8_BINARY", "ii\u0307ii\u0307", "iİ", "ii\u0307ii\u0307"); - assertStringTrimRight("UTF8_BINARY", "i\u0307", "i", "i\u0307"); - assertStringTrimRight("UTF8_BINARY", "i\u0307", "\u0307", "i"); - assertStringTrimRight("UTF8_BINARY", "i\u0307", "i\u0307", ""); - assertStringTrimRight("UTF8_BINARY", "i\u0307i\u0307", "i\u0307", ""); - assertStringTrimRight("UTF8_BINARY", "i\u0307\u0307", "i\u0307", ""); - assertStringTrimRight("UTF8_BINARY", "i\u0307i", "i\u0307", ""); - assertStringTrimRight("UTF8_BINARY", "i\u0307i", "İ", "i\u0307i"); - assertStringTrimRight("UTF8_BINARY", "i\u0307İ", "i\u0307", "i\u0307İ"); - assertStringTrimRight("UTF8_BINARY", "i\u0307İ", "İ", "i\u0307"); - assertStringTrimRight("UTF8_BINARY", "İ", "İ", ""); - assertStringTrimRight("UTF8_BINARY", "IXi", "İ", "IXi"); - assertStringTrimRight("UTF8_BINARY", "ix\u0307", "Ixİ", "ix\u0307"); - assertStringTrimRight("UTF8_BINARY", "i\u0307x", "IXİ", "i\u0307x"); - assertStringTrimRight("UTF8_BINARY", "i\u0307x", "ix\u0307İ", ""); - assertStringTrimRight("UTF8_BINARY", "İ", "i", "İ"); - assertStringTrimRight("UTF8_BINARY", "İ", "\u0307", "İ"); - assertStringTrimRight("UTF8_BINARY", "Ixİ", "i\u0307", "Ixİ"); - assertStringTrimRight("UTF8_BINARY", "IXİ", "ix\u0307", "IXİ"); - assertStringTrimRight("UTF8_BINARY", "xi\u0307", "\u0307IX", "xi"); - assertStringTrimRight("UTF8_LCASE", "i", "i", ""); - assertStringTrimRight("UTF8_LCASE", "iii", "I", ""); - assertStringTrimRight("UTF8_LCASE", "I", "iii", ""); - assertStringTrimRight("UTF8_LCASE", "ixi", "i", "ix"); - assertStringTrimRight("UTF8_LCASE", "i", "İ", "i"); - assertStringTrimRight("UTF8_LCASE", "i\u0307", "İ", ""); - assertStringTrimRight("UTF8_LCASE", "ii\u0307", "İi", ""); - assertStringTrimRight("UTF8_LCASE", "iii\u0307", "İi", ""); - assertStringTrimRight("UTF8_LCASE", "iiii\u0307", "iİ", ""); - assertStringTrimRight("UTF8_LCASE", "ii\u0307ii\u0307", "iİ", ""); - assertStringTrimRight("UTF8_LCASE", "i\u0307", "i", "i\u0307"); - assertStringTrimRight("UTF8_LCASE", "i\u0307", "\u0307", "i"); - assertStringTrimRight("UTF8_LCASE", "i\u0307", "i\u0307", ""); - assertStringTrimRight("UTF8_LCASE", "i\u0307i\u0307", "i\u0307", ""); - assertStringTrimRight("UTF8_LCASE", "i\u0307\u0307", "i\u0307", ""); - assertStringTrimRight("UTF8_LCASE", "i\u0307i", "i\u0307", ""); - assertStringTrimRight("UTF8_LCASE", "i\u0307i", "İ", "i\u0307i"); - assertStringTrimRight("UTF8_LCASE", "i\u0307İ", "i\u0307", "i\u0307İ"); - assertStringTrimRight("UTF8_LCASE", "i\u0307İ", "İ", ""); - assertStringTrimRight("UTF8_LCASE", "İ", "İ", ""); - assertStringTrimRight("UTF8_LCASE", "IXi", "İ", "IXi"); - assertStringTrimRight("UTF8_LCASE", "ix\u0307", "Ixİ", "ix\u0307"); - assertStringTrimRight("UTF8_LCASE", "i\u0307x", "IXİ", ""); - assertStringTrimRight("UTF8_LCASE", "i\u0307x", "I\u0307xİ", ""); - assertStringTrimRight("UTF8_LCASE", "İ", "i", "İ"); - assertStringTrimRight("UTF8_LCASE", "İ", "\u0307", "İ"); - assertStringTrimRight("UTF8_LCASE", "Ixİ", "i\u0307", "Ixİ"); - assertStringTrimRight("UTF8_LCASE", "IXİ", "ix\u0307", "IXİ"); - assertStringTrimRight("UTF8_LCASE", "xi\u0307", "\u0307IX", ""); - assertStringTrimRight("UNICODE", "i", "i", ""); - assertStringTrimRight("UNICODE", "iii", "I", "iii"); - assertStringTrimRight("UNICODE", "I", "iii", "I"); - assertStringTrimRight("UNICODE", "ixi", "i", "ix"); - assertStringTrimRight("UNICODE", "i", "İ", "i"); - assertStringTrimRight("UNICODE", "i\u0307", "İ", "i\u0307"); - assertStringTrimRight("UTF8_BINARY", "ii\u0307", "İi", "ii\u0307"); - assertStringTrimRight("UTF8_BINARY", "iii\u0307", "İi", "iii\u0307"); - assertStringTrimRight("UTF8_BINARY", "iiii\u0307", "iİ", "iiii\u0307"); - assertStringTrimRight("UTF8_BINARY", "ii\u0307ii\u0307", "iİ", "ii\u0307ii\u0307"); - assertStringTrimRight("UNICODE", "i\u0307", "i", "i\u0307"); - assertStringTrimRight("UNICODE", "i\u0307", "\u0307", "i\u0307"); - assertStringTrimRight("UNICODE", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrimRight("UNICODE", "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); - assertStringTrimRight("UNICODE", "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); - assertStringTrimRight("UNICODE", "i\u0307i", "i\u0307", "i\u0307"); - assertStringTrimRight("UNICODE", "i\u0307i", "İ", "i\u0307i"); - assertStringTrimRight("UNICODE", "i\u0307İ", "i\u0307", "i\u0307İ"); - assertStringTrimRight("UNICODE", "i\u0307İ", "İ", "i\u0307"); - assertStringTrimRight("UNICODE", "İ", "İ", ""); - assertStringTrimRight("UNICODE", "IXi", "İ", "IXi"); - assertStringTrimRight("UNICODE", "ix\u0307", "Ixİ", "ix\u0307"); - assertStringTrimRight("UNICODE", "i\u0307x", "IXİ", "i\u0307x"); - assertStringTrimRight("UNICODE", "i\u0307x", "ix\u0307İ", "i\u0307"); - assertStringTrimRight("UNICODE", "İ", "i", "İ"); - assertStringTrimRight("UNICODE", "İ", "\u0307", "İ"); - assertStringTrimRight("UNICODE", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrimRight("UNICODE", "Ixİ", "i\u0307", "Ixİ"); - assertStringTrimRight("UNICODE", "IXİ", "ix\u0307", "IXİ"); - assertStringTrimRight("UNICODE", "xi\u0307", "\u0307IX", "xi\u0307"); - assertStringTrimRight("UNICODE_CI", "i", "i", ""); - assertStringTrimRight("UNICODE_CI", "iii", "I", ""); - assertStringTrimRight("UNICODE_CI", "I", "iii", ""); - assertStringTrimRight("UNICODE_CI", "ixi", "i", "ix"); - assertStringTrimRight("UNICODE_CI", "i", "İ", "i"); - assertStringTrimRight("UNICODE_CI", "i\u0307", "İ", ""); - assertStringTrimRight("UNICODE_CI", "ii\u0307", "İi", ""); - assertStringTrimRight("UNICODE_CI", "iii\u0307", "İi", ""); - assertStringTrimRight("UNICODE_CI", "iiii\u0307", "iİ", ""); - assertStringTrimRight("UNICODE_CI", "ii\u0307ii\u0307", "iİ", ""); - assertStringTrimRight("UNICODE_CI", "i\u0307", "i", "i\u0307"); - assertStringTrimRight("UNICODE_CI", "i\u0307", "\u0307", "i\u0307"); - assertStringTrimRight("UNICODE_CI", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrimRight("UNICODE_CI", "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); - assertStringTrimRight("UNICODE_CI", "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); - assertStringTrimRight("UNICODE_CI", "i\u0307i", "i\u0307", "i\u0307"); - assertStringTrimRight("UNICODE_CI", "i\u0307i", "İ", "i\u0307i"); - assertStringTrimRight("UNICODE_CI", "i\u0307İ", "i\u0307", "i\u0307İ"); - assertStringTrimRight("UNICODE_CI", "i\u0307İ", "İ", ""); - assertStringTrimRight("UNICODE_CI", "İ", "İ", ""); - assertStringTrimRight("UNICODE_CI", "IXi", "İ", "IXi"); - assertStringTrimRight("UNICODE_CI", "ix\u0307", "Ixİ", "ix\u0307"); - assertStringTrimRight("UNICODE_CI", "i\u0307x", "IXİ", ""); - assertStringTrimRight("UNICODE_CI", "i\u0307x", "I\u0307xİ", ""); - assertStringTrimRight("UNICODE_CI", "İ", "i", "İ"); - assertStringTrimRight("UNICODE_CI", "İ", "\u0307", "İ"); - assertStringTrimRight("UNICODE_CI", "i\u0307", "i\u0307", "i\u0307"); - assertStringTrimRight("UNICODE_CI", "Ixİ", "i\u0307", "Ixİ"); - assertStringTrimRight("UNICODE_CI", "IXİ", "ix\u0307", "IXİ"); - assertStringTrimRight("UNICODE_CI", "xi\u0307", "\u0307IX", "xi\u0307"); + assertStringTrimRight(UTF8_BINARY, "i", "i", ""); + assertStringTrimRight(UTF8_BINARY, "iii", "I", "iii"); + assertStringTrimRight(UTF8_BINARY, "I", "iii", "I"); + assertStringTrimRight(UTF8_BINARY, "ixi", "i", "ix"); + assertStringTrimRight(UTF8_BINARY, "i", "İ", "i"); + assertStringTrimRight(UTF8_BINARY, "i\u0307", "İ", "i\u0307"); + assertStringTrimRight(UTF8_BINARY, "ii\u0307", "İi", "ii\u0307"); + assertStringTrimRight(UTF8_BINARY, "iii\u0307", "İi", "iii\u0307"); + assertStringTrimRight(UTF8_BINARY, "iiii\u0307", "iİ", "iiii\u0307"); + assertStringTrimRight(UTF8_BINARY, "ii\u0307ii\u0307", "iİ", "ii\u0307ii\u0307"); + assertStringTrimRight(UTF8_BINARY, "i\u0307", "i", "i\u0307"); + assertStringTrimRight(UTF8_BINARY, "i\u0307", "\u0307", "i"); + assertStringTrimRight(UTF8_BINARY, "i\u0307", "i\u0307", ""); + assertStringTrimRight(UTF8_BINARY, "i\u0307i\u0307", "i\u0307", ""); + assertStringTrimRight(UTF8_BINARY, "i\u0307\u0307", "i\u0307", ""); + assertStringTrimRight(UTF8_BINARY, "i\u0307i", "i\u0307", ""); + assertStringTrimRight(UTF8_BINARY, "i\u0307i", "İ", "i\u0307i"); + assertStringTrimRight(UTF8_BINARY, "i\u0307İ", "i\u0307", "i\u0307İ"); + assertStringTrimRight(UTF8_BINARY, "i\u0307İ", "İ", "i\u0307"); + assertStringTrimRight(UTF8_BINARY, "İ", "İ", ""); + assertStringTrimRight(UTF8_BINARY, "IXi", "İ", "IXi"); + assertStringTrimRight(UTF8_BINARY, "ix\u0307", "Ixİ", "ix\u0307"); + assertStringTrimRight(UTF8_BINARY, "i\u0307x", "IXİ", "i\u0307x"); + assertStringTrimRight(UTF8_BINARY, "i\u0307x", "ix\u0307İ", ""); + assertStringTrimRight(UTF8_BINARY, "İ", "i", "İ"); + assertStringTrimRight(UTF8_BINARY, "İ", "\u0307", "İ"); + assertStringTrimRight(UTF8_BINARY, "Ixİ", "i\u0307", "Ixİ"); + assertStringTrimRight(UTF8_BINARY, "IXİ", "ix\u0307", "IXİ"); + assertStringTrimRight(UTF8_BINARY, "xi\u0307", "\u0307IX", "xi"); + assertStringTrimRight(UTF8_LCASE, "i", "i", ""); + assertStringTrimRight(UTF8_LCASE, "iii", "I", ""); + assertStringTrimRight(UTF8_LCASE, "I", "iii", ""); + assertStringTrimRight(UTF8_LCASE, "ixi", "i", "ix"); + assertStringTrimRight(UTF8_LCASE, "i", "İ", "i"); + assertStringTrimRight(UTF8_LCASE, "i\u0307", "İ", ""); + assertStringTrimRight(UTF8_LCASE, "ii\u0307", "İi", ""); + assertStringTrimRight(UTF8_LCASE, "iii\u0307", "İi", ""); + assertStringTrimRight(UTF8_LCASE, "iiii\u0307", "iİ", ""); + assertStringTrimRight(UTF8_LCASE, "ii\u0307ii\u0307", "iİ", ""); + assertStringTrimRight(UTF8_LCASE, "i\u0307", "i", "i\u0307"); + assertStringTrimRight(UTF8_LCASE, "i\u0307", "\u0307", "i"); + assertStringTrimRight(UTF8_LCASE, "i\u0307", "i\u0307", ""); + assertStringTrimRight(UTF8_LCASE, "i\u0307i\u0307", "i\u0307", ""); + assertStringTrimRight(UTF8_LCASE, "i\u0307\u0307", "i\u0307", ""); + assertStringTrimRight(UTF8_LCASE, "i\u0307i", "i\u0307", ""); + assertStringTrimRight(UTF8_LCASE, "i\u0307i", "İ", "i\u0307i"); + assertStringTrimRight(UTF8_LCASE, "i\u0307İ", "i\u0307", "i\u0307İ"); + assertStringTrimRight(UTF8_LCASE, "i\u0307İ", "İ", ""); + assertStringTrimRight(UTF8_LCASE, "İ", "İ", ""); + assertStringTrimRight(UTF8_LCASE, "IXi", "İ", "IXi"); + assertStringTrimRight(UTF8_LCASE, "ix\u0307", "Ixİ", "ix\u0307"); + assertStringTrimRight(UTF8_LCASE, "i\u0307x", "IXİ", ""); + assertStringTrimRight(UTF8_LCASE, "i\u0307x", "I\u0307xİ", ""); + assertStringTrimRight(UTF8_LCASE, "İ", "i", "İ"); + assertStringTrimRight(UTF8_LCASE, "İ", "\u0307", "İ"); + assertStringTrimRight(UTF8_LCASE, "Ixİ", "i\u0307", "Ixİ"); + assertStringTrimRight(UTF8_LCASE, "IXİ", "ix\u0307", "IXİ"); + assertStringTrimRight(UTF8_LCASE, "xi\u0307", "\u0307IX", ""); + assertStringTrimRight(UNICODE, "i", "i", ""); + assertStringTrimRight(UNICODE, "iii", "I", "iii"); + assertStringTrimRight(UNICODE, "I", "iii", "I"); + assertStringTrimRight(UNICODE, "ixi", "i", "ix"); + assertStringTrimRight(UNICODE, "i", "İ", "i"); + assertStringTrimRight(UNICODE, "i\u0307", "İ", "i\u0307"); + assertStringTrimRight(UTF8_BINARY, "ii\u0307", "İi", "ii\u0307"); + assertStringTrimRight(UTF8_BINARY, "iii\u0307", "İi", "iii\u0307"); + assertStringTrimRight(UTF8_BINARY, "iiii\u0307", "iİ", "iiii\u0307"); + assertStringTrimRight(UTF8_BINARY, "ii\u0307ii\u0307", "iİ", "ii\u0307ii\u0307"); + assertStringTrimRight(UNICODE, "i\u0307", "i", "i\u0307"); + assertStringTrimRight(UNICODE, "i\u0307", "\u0307", "i\u0307"); + assertStringTrimRight(UNICODE, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrimRight(UNICODE, "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); + assertStringTrimRight(UNICODE, "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); + assertStringTrimRight(UNICODE, "i\u0307i", "i\u0307", "i\u0307"); + assertStringTrimRight(UNICODE, "i\u0307i", "İ", "i\u0307i"); + assertStringTrimRight(UNICODE, "i\u0307İ", "i\u0307", "i\u0307İ"); + assertStringTrimRight(UNICODE, "i\u0307İ", "İ", "i\u0307"); + assertStringTrimRight(UNICODE, "İ", "İ", ""); + assertStringTrimRight(UNICODE, "IXi", "İ", "IXi"); + assertStringTrimRight(UNICODE, "ix\u0307", "Ixİ", "ix\u0307"); + assertStringTrimRight(UNICODE, "i\u0307x", "IXİ", "i\u0307x"); + assertStringTrimRight(UNICODE, "i\u0307x", "ix\u0307İ", "i\u0307"); + assertStringTrimRight(UNICODE, "İ", "i", "İ"); + assertStringTrimRight(UNICODE, "İ", "\u0307", "İ"); + assertStringTrimRight(UNICODE, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrimRight(UNICODE, "Ixİ", "i\u0307", "Ixİ"); + assertStringTrimRight(UNICODE, "IXİ", "ix\u0307", "IXİ"); + assertStringTrimRight(UNICODE, "xi\u0307", "\u0307IX", "xi\u0307"); + assertStringTrimRight(UNICODE_CI, "i", "i", ""); + assertStringTrimRight(UNICODE_CI, "iii", "I", ""); + assertStringTrimRight(UNICODE_CI, "I", "iii", ""); + assertStringTrimRight(UNICODE_CI, "ixi", "i", "ix"); + assertStringTrimRight(UNICODE_CI, "i", "İ", "i"); + assertStringTrimRight(UNICODE_CI, "i\u0307", "İ", ""); + assertStringTrimRight(UNICODE_CI, "ii\u0307", "İi", ""); + assertStringTrimRight(UNICODE_CI, "iii\u0307", "İi", ""); + assertStringTrimRight(UNICODE_CI, "iiii\u0307", "iİ", ""); + assertStringTrimRight(UNICODE_CI, "ii\u0307ii\u0307", "iİ", ""); + assertStringTrimRight(UNICODE_CI, "i\u0307", "i", "i\u0307"); + assertStringTrimRight(UNICODE_CI, "i\u0307", "\u0307", "i\u0307"); + assertStringTrimRight(UNICODE_CI, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrimRight(UNICODE_CI, "i\u0307i\u0307", "i\u0307", "i\u0307i\u0307"); + assertStringTrimRight(UNICODE_CI, "i\u0307\u0307", "i\u0307", "i\u0307\u0307"); + assertStringTrimRight(UNICODE_CI, "i\u0307i", "i\u0307", "i\u0307"); + assertStringTrimRight(UNICODE_CI, "i\u0307i", "İ", "i\u0307i"); + assertStringTrimRight(UNICODE_CI, "i\u0307İ", "i\u0307", "i\u0307İ"); + assertStringTrimRight(UNICODE_CI, "i\u0307İ", "İ", ""); + assertStringTrimRight(UNICODE_CI, "İ", "İ", ""); + assertStringTrimRight(UNICODE_CI, "IXi", "İ", "IXi"); + assertStringTrimRight(UNICODE_CI, "ix\u0307", "Ixİ", "ix\u0307"); + assertStringTrimRight(UNICODE_CI, "i\u0307x", "IXİ", ""); + assertStringTrimRight(UNICODE_CI, "i\u0307x", "I\u0307xİ", ""); + assertStringTrimRight(UNICODE_CI, "İ", "i", "İ"); + assertStringTrimRight(UNICODE_CI, "İ", "\u0307", "İ"); + assertStringTrimRight(UNICODE_CI, "i\u0307", "i\u0307", "i\u0307"); + assertStringTrimRight(UNICODE_CI, "Ixİ", "i\u0307", "Ixİ"); + assertStringTrimRight(UNICODE_CI, "IXİ", "ix\u0307", "IXİ"); + assertStringTrimRight(UNICODE_CI, "xi\u0307", "\u0307IX", "xi\u0307"); // Conditional case mapping (e.g. Greek sigmas). - assertStringTrimRight("UTF8_BINARY", "ςxς", "σ", "ςxς"); - assertStringTrimRight("UTF8_BINARY", "ςxς", "ς", "ςx"); - assertStringTrimRight("UTF8_BINARY", "ςxς", "Σ", "ςxς"); - assertStringTrimRight("UTF8_BINARY", "σxσ", "σ", "σx"); - assertStringTrimRight("UTF8_BINARY", "σxσ", "ς", "σxσ"); - assertStringTrimRight("UTF8_BINARY", "σxσ", "Σ", "σxσ"); - assertStringTrimRight("UTF8_BINARY", "ΣxΣ", "σ", "ΣxΣ"); - assertStringTrimRight("UTF8_BINARY", "ΣxΣ", "ς", "ΣxΣ"); - assertStringTrimRight("UTF8_BINARY", "ΣxΣ", "Σ", "Σx"); - assertStringTrimRight("UTF8_LCASE", "ςxς", "σ", "ςx"); - assertStringTrimRight("UTF8_LCASE", "ςxς", "ς", "ςx"); - assertStringTrimRight("UTF8_LCASE", "ςxς", "Σ", "ςx"); - assertStringTrimRight("UTF8_LCASE", "σxσ", "σ", "σx"); - assertStringTrimRight("UTF8_LCASE", "σxσ", "ς", "σx"); - assertStringTrimRight("UTF8_LCASE", "σxσ", "Σ", "σx"); - assertStringTrimRight("UTF8_LCASE", "ΣxΣ", "σ", "Σx"); - assertStringTrimRight("UTF8_LCASE", "ΣxΣ", "ς", "Σx"); - assertStringTrimRight("UTF8_LCASE", "ΣxΣ", "Σ", "Σx"); - assertStringTrimRight("UNICODE", "ςxς", "σ", "ςxς"); - assertStringTrimRight("UNICODE", "ςxς", "ς", "ςx"); - assertStringTrimRight("UNICODE", "ςxς", "Σ", "ςxς"); - assertStringTrimRight("UNICODE", "σxσ", "σ", "σx"); - assertStringTrimRight("UNICODE", "σxσ", "ς", "σxσ"); - assertStringTrimRight("UNICODE", "σxσ", "Σ", "σxσ"); - assertStringTrimRight("UNICODE", "ΣxΣ", "σ", "ΣxΣ"); - assertStringTrimRight("UNICODE", "ΣxΣ", "ς", "ΣxΣ"); - assertStringTrimRight("UNICODE", "ΣxΣ", "Σ", "Σx"); - assertStringTrimRight("UNICODE_CI", "ςxς", "σ", "ςx"); - assertStringTrimRight("UNICODE_CI", "ςxς", "ς", "ςx"); - assertStringTrimRight("UNICODE_CI", "ςxς", "Σ", "ςx"); - assertStringTrimRight("UNICODE_CI", "σxσ", "σ", "σx"); - assertStringTrimRight("UNICODE_CI", "σxσ", "ς", "σx"); - assertStringTrimRight("UNICODE_CI", "σxσ", "Σ", "σx"); - assertStringTrimRight("UNICODE_CI", "ΣxΣ", "σ", "Σx"); - assertStringTrimRight("UNICODE_CI", "ΣxΣ", "ς", "Σx"); - assertStringTrimRight("UNICODE_CI", "ΣxΣ", "Σ", "Σx"); + assertStringTrimRight(UTF8_BINARY, "ςxς", "σ", "ςxς"); + assertStringTrimRight(UTF8_BINARY, "ςxς", "ς", "ςx"); + assertStringTrimRight(UTF8_BINARY, "ςxς", "Σ", "ςxς"); + assertStringTrimRight(UTF8_BINARY, "σxσ", "σ", "σx"); + assertStringTrimRight(UTF8_BINARY, "σxσ", "ς", "σxσ"); + assertStringTrimRight(UTF8_BINARY, "σxσ", "Σ", "σxσ"); + assertStringTrimRight(UTF8_BINARY, "ΣxΣ", "σ", "ΣxΣ"); + assertStringTrimRight(UTF8_BINARY, "ΣxΣ", "ς", "ΣxΣ"); + assertStringTrimRight(UTF8_BINARY, "ΣxΣ", "Σ", "Σx"); + assertStringTrimRight(UTF8_LCASE, "ςxς", "σ", "ςx"); + assertStringTrimRight(UTF8_LCASE, "ςxς", "ς", "ςx"); + assertStringTrimRight(UTF8_LCASE, "ςxς", "Σ", "ςx"); + assertStringTrimRight(UTF8_LCASE, "σxσ", "σ", "σx"); + assertStringTrimRight(UTF8_LCASE, "σxσ", "ς", "σx"); + assertStringTrimRight(UTF8_LCASE, "σxσ", "Σ", "σx"); + assertStringTrimRight(UTF8_LCASE, "ΣxΣ", "σ", "Σx"); + assertStringTrimRight(UTF8_LCASE, "ΣxΣ", "ς", "Σx"); + assertStringTrimRight(UTF8_LCASE, "ΣxΣ", "Σ", "Σx"); + assertStringTrimRight(UNICODE, "ςxς", "σ", "ςxς"); + assertStringTrimRight(UNICODE, "ςxς", "ς", "ςx"); + assertStringTrimRight(UNICODE, "ςxς", "Σ", "ςxς"); + assertStringTrimRight(UNICODE, "σxσ", "σ", "σx"); + assertStringTrimRight(UNICODE, "σxσ", "ς", "σxσ"); + assertStringTrimRight(UNICODE, "σxσ", "Σ", "σxσ"); + assertStringTrimRight(UNICODE, "ΣxΣ", "σ", "ΣxΣ"); + assertStringTrimRight(UNICODE, "ΣxΣ", "ς", "ΣxΣ"); + assertStringTrimRight(UNICODE, "ΣxΣ", "Σ", "Σx"); + assertStringTrimRight(UNICODE_CI, "ςxς", "σ", "ςx"); + assertStringTrimRight(UNICODE_CI, "ςxς", "ς", "ςx"); + assertStringTrimRight(UNICODE_CI, "ςxς", "Σ", "ςx"); + assertStringTrimRight(UNICODE_CI, "σxσ", "σ", "σx"); + assertStringTrimRight(UNICODE_CI, "σxσ", "ς", "σx"); + assertStringTrimRight(UNICODE_CI, "σxσ", "Σ", "σx"); + assertStringTrimRight(UNICODE_CI, "ΣxΣ", "σ", "Σx"); + assertStringTrimRight(UNICODE_CI, "ΣxΣ", "ς", "Σx"); + assertStringTrimRight(UNICODE_CI, "ΣxΣ", "Σ", "Σx"); // Unicode normalization. - assertStringTrimRight("UTF8_BINARY", "åβγδa\u030A", "å", "åβγδa\u030A"); - assertStringTrimRight("UTF8_LCASE", "åβγδa\u030A", "Å", "åβγδa\u030A"); - assertStringTrimRight("UNICODE", "åβγδa\u030A", "å", "åβγδ"); - assertStringTrimRight("UNICODE_CI", "åβγδa\u030A", "Å", "åβγδ"); + assertStringTrimRight(UTF8_BINARY, "åβγδa\u030A", "å", "åβγδa\u030A"); + assertStringTrimRight(UTF8_LCASE, "åβγδa\u030A", "Å", "åβγδa\u030A"); + assertStringTrimRight(UNICODE, "åβγδa\u030A", "å", "åβγδ"); + assertStringTrimRight(UNICODE_CI, "åβγδa\u030A", "Å", "åβγδ"); // Surrogate pairs. - assertStringTrimRight("UTF8_BINARY", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrimRight("UTF8_LCASE", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrimRight("UNICODE", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrimRight("UNICODE_CI", "a🙃b🙃c", "🙃", "a🙃b🙃c"); - assertStringTrimRight("UTF8_BINARY", "a🙃b🙃c", "c", "a🙃b🙃"); - assertStringTrimRight("UTF8_LCASE", "a🙃b🙃c", "c", "a🙃b🙃"); - assertStringTrimRight("UNICODE", "a🙃b🙃c", "c", "a🙃b🙃"); - assertStringTrimRight("UNICODE_CI", "a🙃b🙃c", "c", "a🙃b🙃"); - assertStringTrimRight("UTF8_BINARY", "a🙃b🙃c", "c🙃", "a🙃b"); - assertStringTrimRight("UTF8_LCASE", "a🙃b🙃c", "c🙃", "a🙃b"); - assertStringTrimRight("UNICODE", "a🙃b🙃c", "c🙃", "a🙃b"); - assertStringTrimRight("UNICODE_CI", "a🙃b🙃c", "c🙃", "a🙃b"); - assertStringTrimRight("UTF8_BINARY", "a🙃b🙃c", "c🙃b", "a"); - assertStringTrimRight("UTF8_LCASE", "a🙃b🙃c", "c🙃b", "a"); - assertStringTrimRight("UNICODE", "a🙃b🙃c", "c🙃b", "a"); - assertStringTrimRight("UNICODE_CI", "a🙃b🙃c", "c🙃b", "a"); - assertStringTrimRight("UTF8_BINARY", "a🙃b🙃c", "abc🙃", ""); - assertStringTrimRight("UTF8_LCASE", "a🙃b🙃c", "abc🙃", ""); - assertStringTrimRight("UNICODE", "a🙃b🙃c", "abc🙃", ""); - assertStringTrimRight("UNICODE_CI", "a🙃b🙃c", "abc🙃", ""); - assertStringTrimRight("UTF8_BINARY", "😀😆😃😄", "😆😃", "😀😆😃😄"); - assertStringTrimRight("UTF8_LCASE", "😀😆😃😄", "😆😃", "😀😆😃😄"); - assertStringTrimRight("UNICODE", "😀😆😃😄", "😆😃", "😀😆😃😄"); - assertStringTrimRight("UNICODE_CI", "😀😆😃😄", "😆😃", "😀😆😃😄"); - assertStringTrimRight("UTF8_BINARY", "😀😆😃😄", "😃😄", "😀😆"); - assertStringTrimRight("UTF8_LCASE", "😀😆😃😄", "😃😄", "😀😆"); - assertStringTrimRight("UNICODE", "😀😆😃😄", "😃😄", "😀😆"); - assertStringTrimRight("UNICODE_CI", "😀😆😃😄", "😃😄", "😀😆"); - assertStringTrimRight("UTF8_BINARY", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrimRight("UTF8_LCASE", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrimRight("UNICODE", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrimRight("UNICODE_CI", "😀😆😃😄", "😀😆😃😄", ""); - assertStringTrimRight("UTF8_BINARY", "𐐅", "𐐅", ""); - assertStringTrimRight("UTF8_LCASE", "𐐅", "𐐅", ""); - assertStringTrimRight("UNICODE", "𐐅", "𐐅", ""); - assertStringTrimRight("UNICODE_CI", "𐐅", "𐐅", ""); - assertStringTrimRight("UTF8_BINARY", "𐐅", "𐐭", "𐐅"); - assertStringTrimRight("UTF8_LCASE", "𐐅", "𐐭", ""); - assertStringTrimRight("UNICODE", "𐐅", "𐐭", "𐐅"); - assertStringTrimRight("UNICODE_CI", "𐐅", "𐐭", ""); - assertStringTrimRight("UTF8_BINARY", "𝔸", "𝔸", ""); - assertStringTrimRight("UTF8_LCASE", "𝔸", "𝔸", ""); - assertStringTrimRight("UNICODE", "𝔸", "𝔸", ""); - assertStringTrimRight("UNICODE_CI", "𝔸", "𝔸", ""); - assertStringTrimRight("UTF8_BINARY", "𝔸", "A", "𝔸"); - assertStringTrimRight("UTF8_LCASE", "𝔸", "A", "𝔸"); - assertStringTrimRight("UNICODE", "𝔸", "A", "𝔸"); - assertStringTrimRight("UNICODE_CI", "𝔸", "A", ""); - assertStringTrimRight("UTF8_BINARY", "𝔸", "a", "𝔸"); - assertStringTrimRight("UTF8_LCASE", "𝔸", "a", "𝔸"); - assertStringTrimRight("UNICODE", "𝔸", "a", "𝔸"); - assertStringTrimRight("UNICODE_CI", "𝔸", "a", ""); + assertStringTrimRight(UTF8_BINARY, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrimRight(UTF8_LCASE, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrimRight(UNICODE, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrimRight(UNICODE_CI, "a🙃b🙃c", "🙃", "a🙃b🙃c"); + assertStringTrimRight(UTF8_BINARY, "a🙃b🙃c", "c", "a🙃b🙃"); + assertStringTrimRight(UTF8_LCASE, "a🙃b🙃c", "c", "a🙃b🙃"); + assertStringTrimRight(UNICODE, "a🙃b🙃c", "c", "a🙃b🙃"); + assertStringTrimRight(UNICODE_CI, "a🙃b🙃c", "c", "a🙃b🙃"); + assertStringTrimRight(UTF8_BINARY, "a🙃b🙃c", "c🙃", "a🙃b"); + assertStringTrimRight(UTF8_LCASE, "a🙃b🙃c", "c🙃", "a🙃b"); + assertStringTrimRight(UNICODE, "a🙃b🙃c", "c🙃", "a🙃b"); + assertStringTrimRight(UNICODE_CI, "a🙃b🙃c", "c🙃", "a🙃b"); + assertStringTrimRight(UTF8_BINARY, "a🙃b🙃c", "c🙃b", "a"); + assertStringTrimRight(UTF8_LCASE, "a🙃b🙃c", "c🙃b", "a"); + assertStringTrimRight(UNICODE, "a🙃b🙃c", "c🙃b", "a"); + assertStringTrimRight(UNICODE_CI, "a🙃b🙃c", "c🙃b", "a"); + assertStringTrimRight(UTF8_BINARY, "a🙃b🙃c", "abc🙃", ""); + assertStringTrimRight(UTF8_LCASE, "a🙃b🙃c", "abc🙃", ""); + assertStringTrimRight(UNICODE, "a🙃b🙃c", "abc🙃", ""); + assertStringTrimRight(UNICODE_CI, "a🙃b🙃c", "abc🙃", ""); + assertStringTrimRight(UTF8_BINARY, "😀😆😃😄", "😆😃", "😀😆😃😄"); + assertStringTrimRight(UTF8_LCASE, "😀😆😃😄", "😆😃", "😀😆😃😄"); + assertStringTrimRight(UNICODE, "😀😆😃😄", "😆😃", "😀😆😃😄"); + assertStringTrimRight(UNICODE_CI, "😀😆😃😄", "😆😃", "😀😆😃😄"); + assertStringTrimRight(UTF8_BINARY, "😀😆😃😄", "😃😄", "😀😆"); + assertStringTrimRight(UTF8_LCASE, "😀😆😃😄", "😃😄", "😀😆"); + assertStringTrimRight(UNICODE, "😀😆😃😄", "😃😄", "😀😆"); + assertStringTrimRight(UNICODE_CI, "😀😆😃😄", "😃😄", "😀😆"); + assertStringTrimRight(UTF8_BINARY, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrimRight(UTF8_LCASE, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrimRight(UNICODE, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrimRight(UNICODE_CI, "😀😆😃😄", "😀😆😃😄", ""); + assertStringTrimRight(UTF8_BINARY, "𐐅", "𐐅", ""); + assertStringTrimRight(UTF8_LCASE, "𐐅", "𐐅", ""); + assertStringTrimRight(UNICODE, "𐐅", "𐐅", ""); + assertStringTrimRight(UNICODE_CI, "𐐅", "𐐅", ""); + assertStringTrimRight(UTF8_BINARY, "𐐅", "𐐭", "𐐅"); + assertStringTrimRight(UTF8_LCASE, "𐐅", "𐐭", ""); + assertStringTrimRight(UNICODE, "𐐅", "𐐭", "𐐅"); + assertStringTrimRight(UNICODE_CI, "𐐅", "𐐭", ""); + assertStringTrimRight(UTF8_BINARY, "𝔸", "𝔸", ""); + assertStringTrimRight(UTF8_LCASE, "𝔸", "𝔸", ""); + assertStringTrimRight(UNICODE, "𝔸", "𝔸", ""); + assertStringTrimRight(UNICODE_CI, "𝔸", "𝔸", ""); + assertStringTrimRight(UTF8_BINARY, "𝔸", "A", "𝔸"); + assertStringTrimRight(UTF8_LCASE, "𝔸", "A", "𝔸"); + assertStringTrimRight(UNICODE, "𝔸", "A", "𝔸"); + assertStringTrimRight(UNICODE_CI, "𝔸", "A", ""); + assertStringTrimRight(UTF8_BINARY, "𝔸", "a", "𝔸"); + assertStringTrimRight(UTF8_LCASE, "𝔸", "a", "𝔸"); + assertStringTrimRight(UNICODE, "𝔸", "a", "𝔸"); + assertStringTrimRight(UNICODE_CI, "𝔸", "a", ""); } /** @@ -3664,211 +3665,211 @@ private void assertStringTranslate(String inputString, String matchingString, @Test public void testStringTranslate() throws SparkException { // Empty strings. - assertStringTranslate("", "", "", "UTF8_BINARY", ""); - assertStringTranslate("", "", "", "UTF8_LCASE", ""); - assertStringTranslate("", "", "", "UNICODE", ""); - assertStringTranslate("", "", "", "UNICODE_CI", ""); - assertStringTranslate("abc", "", "", "UTF8_BINARY", "abc"); - assertStringTranslate("abc", "", "", "UTF8_LCASE", "abc"); - assertStringTranslate("abc", "", "", "UNICODE", "abc"); - assertStringTranslate("abc", "", "", "UNICODE_CI", "abc"); - assertStringTranslate("", "b", "", "UTF8_BINARY", ""); - assertStringTranslate("", "b", "", "UTF8_LCASE", ""); - assertStringTranslate("", "b", "", "UNICODE", ""); - assertStringTranslate("", "b", "", "UNICODE_CI", ""); - assertStringTranslate("", "", "x", "UTF8_BINARY", ""); - assertStringTranslate("", "", "x", "UTF8_LCASE", ""); - assertStringTranslate("", "", "x", "UNICODE", ""); - assertStringTranslate("", "", "x", "UNICODE_CI", ""); - assertStringTranslate("abc", "b", "", "UTF8_BINARY", "ac"); - assertStringTranslate("abc", "b", "", "UTF8_LCASE", "ac"); - assertStringTranslate("abc", "b", "", "UNICODE", "ac"); - assertStringTranslate("abc", "b", "", "UNICODE_CI", "ac"); - assertStringTranslate("abc", "", "x", "UTF8_BINARY", "abc"); - assertStringTranslate("abc", "", "x", "UTF8_LCASE", "abc"); - assertStringTranslate("abc", "", "x", "UNICODE", "abc"); - assertStringTranslate("abc", "", "x", "UNICODE_CI", "abc"); - assertStringTranslate("", "b", "x", "UTF8_BINARY", ""); - assertStringTranslate("", "b", "x", "UTF8_LCASE", ""); - assertStringTranslate("", "b", "x", "UNICODE", ""); - assertStringTranslate("", "b", "x", "UNICODE_CI", ""); + assertStringTranslate("", "", "", UTF8_BINARY, ""); + assertStringTranslate("", "", "", UTF8_LCASE, ""); + assertStringTranslate("", "", "", UNICODE, ""); + assertStringTranslate("", "", "", UNICODE_CI, ""); + assertStringTranslate("abc", "", "", UTF8_BINARY, "abc"); + assertStringTranslate("abc", "", "", UTF8_LCASE, "abc"); + assertStringTranslate("abc", "", "", UNICODE, "abc"); + assertStringTranslate("abc", "", "", UNICODE_CI, "abc"); + assertStringTranslate("", "b", "", UTF8_BINARY, ""); + assertStringTranslate("", "b", "", UTF8_LCASE, ""); + assertStringTranslate("", "b", "", UNICODE, ""); + assertStringTranslate("", "b", "", UNICODE_CI, ""); + assertStringTranslate("", "", "x", UTF8_BINARY, ""); + assertStringTranslate("", "", "x", UTF8_LCASE, ""); + assertStringTranslate("", "", "x", UNICODE, ""); + assertStringTranslate("", "", "x", UNICODE_CI, ""); + assertStringTranslate("abc", "b", "", UTF8_BINARY, "ac"); + assertStringTranslate("abc", "b", "", UTF8_LCASE, "ac"); + assertStringTranslate("abc", "b", "", UNICODE, "ac"); + assertStringTranslate("abc", "b", "", UNICODE_CI, "ac"); + assertStringTranslate("abc", "", "x", UTF8_BINARY, "abc"); + assertStringTranslate("abc", "", "x", UTF8_LCASE, "abc"); + assertStringTranslate("abc", "", "x", UNICODE, "abc"); + assertStringTranslate("abc", "", "x", UNICODE_CI, "abc"); + assertStringTranslate("", "b", "x", UTF8_BINARY, ""); + assertStringTranslate("", "b", "x", UTF8_LCASE, ""); + assertStringTranslate("", "b", "x", UNICODE, ""); + assertStringTranslate("", "b", "x", UNICODE_CI, ""); // Basic tests. - assertStringTranslate("abc", "b", "x", "UTF8_BINARY", "axc"); - assertStringTranslate("abc", "b", "x", "UTF8_LCASE", "axc"); - assertStringTranslate("abc", "b", "x", "UNICODE", "axc"); - assertStringTranslate("abc", "b", "x", "UNICODE_CI", "axc"); - assertStringTranslate("Translate", "Rnlt", "12", "UTF8_BINARY", "Tra2sae"); - assertStringTranslate("Translate", "Rnlt", "12", "UTF8_LCASE", "1a2sae"); - assertStringTranslate("Translate", "Rnlt", "12", "UNICODE", "Tra2sae"); - assertStringTranslate("Translate", "Rnlt", "12", "UNICODE_CI", "1a2sae"); - assertStringTranslate("Translate", "Rn", "1234", "UTF8_BINARY", "Tra2slate"); - assertStringTranslate("Translate", "Rn", "1234", "UTF8_LCASE", "T1a2slate"); - assertStringTranslate("Translate", "Rn", "1234", "UNICODE", "Tra2slate"); - assertStringTranslate("Translate", "Rn", "1234", "UNICODE_CI", "T1a2slate"); - assertStringTranslate("Translate", "Rnlt", "1234", "UTF8_BINARY", "Tra2s3a4e"); - assertStringTranslate("Translate", "Rnlt", "1234", "UTF8_LCASE", "41a2s3a4e"); - assertStringTranslate("Translate", "Rnlt", "1234", "UNICODE", "Tra2s3a4e"); - assertStringTranslate("Translate", "Rnlt", "1234", "UNICODE_CI", "41a2s3a4e"); - assertStringTranslate("TRanslate", "rnlt", "XxXx", "UTF8_BINARY", "TRaxsXaxe"); - assertStringTranslate("TRanslate", "rnlt", "XxXx", "UTF8_LCASE", "xXaxsXaxe"); - assertStringTranslate("TRanslate", "rnlt", "XxXx", "UNICODE", "TRaxsXaxe"); - assertStringTranslate("TRanslate", "rnlt", "XxXx", "UNICODE_CI", "xXaxsXaxe"); - assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", "UTF8_BINARY", "TxaxsXaxeX"); - assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", "UTF8_LCASE", "xxaxsXaxex"); - assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", "UNICODE", "TxaxsXaxeX"); - assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", "UNICODE_CI", "xxaxsXaxex"); - assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", "UTF8_BINARY", "TXaxsXaxex"); - assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", "UTF8_LCASE", "xXaxsXaxeX"); - assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", "UNICODE", "TXaxsXaxex"); - assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", "UNICODE_CI", "xXaxsXaxeX"); - assertStringTranslate("test大千世界X大千世界", "界x", "AB", "UTF8_BINARY", "test大千世AX大千世A"); - assertStringTranslate("test大千世界X大千世界", "界x", "AB", "UTF8_LCASE", "test大千世AB大千世A"); - assertStringTranslate("test大千世界X大千世界", "界x", "AB", "UNICODE", "test大千世AX大千世A"); - assertStringTranslate("test大千世界X大千世界", "界x", "AB", "UNICODE_CI", "test大千世AB大千世A"); - assertStringTranslate("大千世界test大千世界", "TEST", "abcd", "UTF8_BINARY", "大千世界test大千世界"); - assertStringTranslate("大千世界test大千世界", "TEST", "abcd", "UTF8_LCASE", "大千世界abca大千世界"); - assertStringTranslate("大千世界test大千世界", "TEST", "abcd", "UNICODE", "大千世界test大千世界"); - assertStringTranslate("大千世界test大千世界", "TEST", "abcd", "UNICODE_CI", "大千世界abca大千世界"); - assertStringTranslate("Test大千世界大千世界", "tT", "oO", "UTF8_BINARY", "Oeso大千世界大千世界"); - assertStringTranslate("Test大千世界大千世界", "tT", "oO", "UTF8_LCASE", "oeso大千世界大千世界"); - assertStringTranslate("Test大千世界大千世界", "tT", "oO", "UNICODE", "Oeso大千世界大千世界"); - assertStringTranslate("Test大千世界大千世界", "tT", "oO", "UNICODE_CI", "oeso大千世界大千世界"); - assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", "UTF8_BINARY", "大千世界大千世界oesO"); - assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", "UTF8_LCASE", "大千世界大千世界OesO"); - assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", "UNICODE", "大千世界大千世界oesO"); - assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", "UNICODE_CI", "大千世界大千世界OesO"); - assertStringTranslate("大千世界大千世界tesT", "大千", "世世", "UTF8_BINARY", "世世世界世世世界tesT"); - assertStringTranslate("大千世界大千世界tesT", "大千", "世世", "UTF8_LCASE", "世世世界世世世界tesT"); - assertStringTranslate("大千世界大千世界tesT", "大千", "世世", "UNICODE", "世世世界世世世界tesT"); - assertStringTranslate("大千世界大千世界tesT", "大千", "世世", "UNICODE_CI", "世世世界世世世界tesT"); - assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", "UTF8_BINARY", "Tr4234e"); - assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", "UTF8_LCASE", "14234e"); - assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", "UNICODE", "Tr4234e"); - assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", "UNICODE_CI", "14234e"); - assertStringTranslate("Translate", "Rnlt", "123495834634", "UTF8_BINARY", "Tra2s3a4e"); - assertStringTranslate("Translate", "Rnlt", "123495834634", "UTF8_LCASE", "41a2s3a4e"); - assertStringTranslate("Translate", "Rnlt", "123495834634", "UNICODE", "Tra2s3a4e"); - assertStringTranslate("Translate", "Rnlt", "123495834634", "UNICODE_CI", "41a2s3a4e"); - assertStringTranslate("abcdef", "abcde", "123", "UTF8_BINARY", "123f"); - assertStringTranslate("abcdef", "abcde", "123", "UTF8_LCASE", "123f"); - assertStringTranslate("abcdef", "abcde", "123", "UNICODE", "123f"); - assertStringTranslate("abcdef", "abcde", "123", "UNICODE_CI", "123f"); + assertStringTranslate("abc", "b", "x", UTF8_BINARY, "axc"); + assertStringTranslate("abc", "b", "x", UTF8_LCASE, "axc"); + assertStringTranslate("abc", "b", "x", UNICODE, "axc"); + assertStringTranslate("abc", "b", "x", UNICODE_CI, "axc"); + assertStringTranslate("Translate", "Rnlt", "12", UTF8_BINARY, "Tra2sae"); + assertStringTranslate("Translate", "Rnlt", "12", UTF8_LCASE, "1a2sae"); + assertStringTranslate("Translate", "Rnlt", "12", UNICODE, "Tra2sae"); + assertStringTranslate("Translate", "Rnlt", "12", UNICODE_CI, "1a2sae"); + assertStringTranslate("Translate", "Rn", "1234", UTF8_BINARY, "Tra2slate"); + assertStringTranslate("Translate", "Rn", "1234", UTF8_LCASE, "T1a2slate"); + assertStringTranslate("Translate", "Rn", "1234", UNICODE, "Tra2slate"); + assertStringTranslate("Translate", "Rn", "1234", UNICODE_CI, "T1a2slate"); + assertStringTranslate("Translate", "Rnlt", "1234", UTF8_BINARY, "Tra2s3a4e"); + assertStringTranslate("Translate", "Rnlt", "1234", UTF8_LCASE, "41a2s3a4e"); + assertStringTranslate("Translate", "Rnlt", "1234", UNICODE, "Tra2s3a4e"); + assertStringTranslate("Translate", "Rnlt", "1234", UNICODE_CI, "41a2s3a4e"); + assertStringTranslate("TRanslate", "rnlt", "XxXx", UTF8_BINARY, "TRaxsXaxe"); + assertStringTranslate("TRanslate", "rnlt", "XxXx", UTF8_LCASE, "xXaxsXaxe"); + assertStringTranslate("TRanslate", "rnlt", "XxXx", UNICODE, "TRaxsXaxe"); + assertStringTranslate("TRanslate", "rnlt", "XxXx", UNICODE_CI, "xXaxsXaxe"); + assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", UTF8_BINARY, "TxaxsXaxeX"); + assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", UTF8_LCASE, "xxaxsXaxex"); + assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", UNICODE, "TxaxsXaxeX"); + assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", UNICODE_CI, "xxaxsXaxex"); + assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", UTF8_BINARY, "TXaxsXaxex"); + assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", UTF8_LCASE, "xXaxsXaxeX"); + assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", UNICODE, "TXaxsXaxex"); + assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", UNICODE_CI, "xXaxsXaxeX"); + assertStringTranslate("test大千世界X大千世界", "界x", "AB", UTF8_BINARY, "test大千世AX大千世A"); + assertStringTranslate("test大千世界X大千世界", "界x", "AB", UTF8_LCASE, "test大千世AB大千世A"); + assertStringTranslate("test大千世界X大千世界", "界x", "AB", UNICODE, "test大千世AX大千世A"); + assertStringTranslate("test大千世界X大千世界", "界x", "AB", UNICODE_CI, "test大千世AB大千世A"); + assertStringTranslate("大千世界test大千世界", "TEST", "abcd", UTF8_BINARY, "大千世界test大千世界"); + assertStringTranslate("大千世界test大千世界", "TEST", "abcd", UTF8_LCASE, "大千世界abca大千世界"); + assertStringTranslate("大千世界test大千世界", "TEST", "abcd", UNICODE, "大千世界test大千世界"); + assertStringTranslate("大千世界test大千世界", "TEST", "abcd", UNICODE_CI, "大千世界abca大千世界"); + assertStringTranslate("Test大千世界大千世界", "tT", "oO", UTF8_BINARY, "Oeso大千世界大千世界"); + assertStringTranslate("Test大千世界大千世界", "tT", "oO", UTF8_LCASE, "oeso大千世界大千世界"); + assertStringTranslate("Test大千世界大千世界", "tT", "oO", UNICODE, "Oeso大千世界大千世界"); + assertStringTranslate("Test大千世界大千世界", "tT", "oO", UNICODE_CI, "oeso大千世界大千世界"); + assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", UTF8_BINARY, "大千世界大千世界oesO"); + assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", UTF8_LCASE, "大千世界大千世界OesO"); + assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", UNICODE, "大千世界大千世界oesO"); + assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", UNICODE_CI, "大千世界大千世界OesO"); + assertStringTranslate("大千世界大千世界tesT", "大千", "世世", UTF8_BINARY, "世世世界世世世界tesT"); + assertStringTranslate("大千世界大千世界tesT", "大千", "世世", UTF8_LCASE, "世世世界世世世界tesT"); + assertStringTranslate("大千世界大千世界tesT", "大千", "世世", UNICODE, "世世世界世世世界tesT"); + assertStringTranslate("大千世界大千世界tesT", "大千", "世世", UNICODE_CI, "世世世界世世世界tesT"); + assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", UTF8_BINARY, "Tr4234e"); + assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", UTF8_LCASE, "14234e"); + assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", UNICODE, "Tr4234e"); + assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", UNICODE_CI, "14234e"); + assertStringTranslate("Translate", "Rnlt", "123495834634", UTF8_BINARY, "Tra2s3a4e"); + assertStringTranslate("Translate", "Rnlt", "123495834634", UTF8_LCASE, "41a2s3a4e"); + assertStringTranslate("Translate", "Rnlt", "123495834634", UNICODE, "Tra2s3a4e"); + assertStringTranslate("Translate", "Rnlt", "123495834634", UNICODE_CI, "41a2s3a4e"); + assertStringTranslate("abcdef", "abcde", "123", UTF8_BINARY, "123f"); + assertStringTranslate("abcdef", "abcde", "123", UTF8_LCASE, "123f"); + assertStringTranslate("abcdef", "abcde", "123", UNICODE, "123f"); + assertStringTranslate("abcdef", "abcde", "123", UNICODE_CI, "123f"); assertStringTranslate("abcdëÈêf", "ÊèË", "123", "AF_CI", "abcd321f"); // One-to-many case mapping (e.g. Turkish dotted I). - assertStringTranslate("İ", "i\u0307", "xy", "UTF8_BINARY", "İ"); - assertStringTranslate("İ", "i\u0307", "xy", "UTF8_LCASE", "İ"); - assertStringTranslate("İ", "i\u0307", "xy", "UNICODE", "İ"); - assertStringTranslate("İ", "i\u0307", "xy", "UNICODE_CI", "İ"); - assertStringTranslate("i\u0307", "İ", "xy", "UTF8_BINARY", "i\u0307"); - assertStringTranslate("i\u0307", "İ", "xy", "UTF8_LCASE", "x"); - assertStringTranslate("i\u0307", "İ", "xy", "UNICODE", "i\u0307"); - assertStringTranslate("i\u0307", "İ", "xy", "UNICODE_CI", "x"); - assertStringTranslate("i\u030A", "İ", "x", "UTF8_BINARY", "i\u030A"); - assertStringTranslate("i\u030A", "İ", "x", "UTF8_LCASE", "i\u030A"); - assertStringTranslate("i\u030A", "İ", "x", "UNICODE", "i\u030A"); - assertStringTranslate("i\u030A", "İ", "x", "UNICODE_CI", "i\u030A"); - assertStringTranslate("i\u030A", "İi", "xy", "UTF8_BINARY", "y\u030A"); - assertStringTranslate("i\u030A", "İi", "xy", "UTF8_LCASE", "y\u030A"); - assertStringTranslate("i\u030A", "İi", "xy", "UNICODE", "i\u030A"); - assertStringTranslate("i\u030A", "İi", "xy", "UNICODE_CI", "i\u030A"); - assertStringTranslate("İi\u0307", "İi\u0307", "123", "UTF8_BINARY", "123"); - assertStringTranslate("İi\u0307", "İi\u0307", "123", "UTF8_LCASE", "11"); - assertStringTranslate("İi\u0307", "İi\u0307", "123", "UNICODE", "1i\u0307"); - assertStringTranslate("İi\u0307", "İi\u0307", "123", "UNICODE_CI", "11"); - assertStringTranslate("İi\u0307", "İyz", "123", "UTF8_BINARY", "1i\u0307"); - assertStringTranslate("İi\u0307", "İyz", "123", "UTF8_LCASE", "11"); - assertStringTranslate("İi\u0307", "İyz", "123", "UNICODE", "1i\u0307"); - assertStringTranslate("İi\u0307", "İyz", "123", "UNICODE_CI", "11"); - assertStringTranslate("İi\u0307", "xi\u0307", "123", "UTF8_BINARY", "İ23"); - assertStringTranslate("İi\u0307", "xi\u0307", "123", "UTF8_LCASE", "İ23"); - assertStringTranslate("İi\u0307", "xi\u0307", "123", "UNICODE", "İi\u0307"); - assertStringTranslate("İi\u0307", "xi\u0307", "123", "UNICODE_CI", "İi\u0307"); - assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", "UTF8_BINARY", "12bc3"); - assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", "UTF8_LCASE", "12bc3"); - assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", "UNICODE", "3bc3"); - assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", "UNICODE_CI", "3bc3"); - assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", "UTF8_BINARY", "a2bcå"); - assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", "UTF8_LCASE", "12bc3"); - assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", "UNICODE", "a\u030Abcå"); - assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", "UNICODE_CI", "3bc3"); - assertStringTranslate("a\u030AβφδI\u0307", "Iİaå", "1234", "UTF8_BINARY", "3\u030Aβφδ1\u0307"); - assertStringTranslate("A\u030Aβφδi\u0307", "Iİaå", "1234", "UTF8_LCASE", "3\u030Aβφδ2"); - assertStringTranslate("a\u030AβφδI\u0307", "Iİaå", "1234", "UNICODE", "4βφδ2"); - assertStringTranslate("A\u030Aβφδi\u0307", "Iİaå", "1234", "UNICODE_CI", "4βφδ2"); + assertStringTranslate("İ", "i\u0307", "xy", UTF8_BINARY, "İ"); + assertStringTranslate("İ", "i\u0307", "xy", UTF8_LCASE, "İ"); + assertStringTranslate("İ", "i\u0307", "xy", UNICODE, "İ"); + assertStringTranslate("İ", "i\u0307", "xy", UNICODE_CI, "İ"); + assertStringTranslate("i\u0307", "İ", "xy", UTF8_BINARY, "i\u0307"); + assertStringTranslate("i\u0307", "İ", "xy", UTF8_LCASE, "x"); + assertStringTranslate("i\u0307", "İ", "xy", UNICODE, "i\u0307"); + assertStringTranslate("i\u0307", "İ", "xy", UNICODE_CI, "x"); + assertStringTranslate("i\u030A", "İ", "x", UTF8_BINARY, "i\u030A"); + assertStringTranslate("i\u030A", "İ", "x", UTF8_LCASE, "i\u030A"); + assertStringTranslate("i\u030A", "İ", "x", UNICODE, "i\u030A"); + assertStringTranslate("i\u030A", "İ", "x", UNICODE_CI, "i\u030A"); + assertStringTranslate("i\u030A", "İi", "xy", UTF8_BINARY, "y\u030A"); + assertStringTranslate("i\u030A", "İi", "xy", UTF8_LCASE, "y\u030A"); + assertStringTranslate("i\u030A", "İi", "xy", UNICODE, "i\u030A"); + assertStringTranslate("i\u030A", "İi", "xy", UNICODE_CI, "i\u030A"); + assertStringTranslate("İi\u0307", "İi\u0307", "123", UTF8_BINARY, "123"); + assertStringTranslate("İi\u0307", "İi\u0307", "123", UTF8_LCASE, "11"); + assertStringTranslate("İi\u0307", "İi\u0307", "123", UNICODE, "1i\u0307"); + assertStringTranslate("İi\u0307", "İi\u0307", "123", UNICODE_CI, "11"); + assertStringTranslate("İi\u0307", "İyz", "123", UTF8_BINARY, "1i\u0307"); + assertStringTranslate("İi\u0307", "İyz", "123", UTF8_LCASE, "11"); + assertStringTranslate("İi\u0307", "İyz", "123", UNICODE, "1i\u0307"); + assertStringTranslate("İi\u0307", "İyz", "123", UNICODE_CI, "11"); + assertStringTranslate("İi\u0307", "xi\u0307", "123", UTF8_BINARY, "İ23"); + assertStringTranslate("İi\u0307", "xi\u0307", "123", UTF8_LCASE, "İ23"); + assertStringTranslate("İi\u0307", "xi\u0307", "123", UNICODE, "İi\u0307"); + assertStringTranslate("İi\u0307", "xi\u0307", "123", UNICODE_CI, "İi\u0307"); + assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", UTF8_BINARY, "12bc3"); + assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", UTF8_LCASE, "12bc3"); + assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", UNICODE, "3bc3"); + assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", UNICODE_CI, "3bc3"); + assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", UTF8_BINARY, "a2bcå"); + assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", UTF8_LCASE, "12bc3"); + assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", UNICODE, "a\u030Abcå"); + assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", UNICODE_CI, "3bc3"); + assertStringTranslate("a\u030AβφδI\u0307", "Iİaå", "1234", UTF8_BINARY, "3\u030Aβφδ1\u0307"); + assertStringTranslate("A\u030Aβφδi\u0307", "Iİaå", "1234", UTF8_LCASE, "3\u030Aβφδ2"); + assertStringTranslate("a\u030AβφδI\u0307", "Iİaå", "1234", UNICODE, "4βφδ2"); + assertStringTranslate("A\u030Aβφδi\u0307", "Iİaå", "1234", UNICODE_CI, "4βφδ2"); // Conditional case mapping (e.g. Greek sigmas). - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", "UTF8_BINARY", "σΥσΤΗΜΑΤΙΚΟσ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", "UTF8_LCASE", "σισΤιΜΑΤΙΚΟσ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", "UNICODE", "σΥσΤΗΜΑΤΙΚΟσ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", "UNICODE_CI", "σισΤιΜΑΤΙΚΟσ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", "UTF8_BINARY", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", "UTF8_LCASE", "σισΤιΜΑΤΙΚΟσ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", "UNICODE", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", "UNICODE_CI", "σισΤιΜΑΤΙΚΟσ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", "UTF8_BINARY", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", "UTF8_LCASE", "σισΤιΜΑΤΙΚΟσ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", "UNICODE", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", "UNICODE_CI", "σισΤιΜΑΤΙΚΟσ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", "UTF8_BINARY", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", "UTF8_LCASE", "ςιςΤιΜΑΤΙΚΟς"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", "UNICODE", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", "UNICODE_CI", "ςιςΤιΜΑΤΙΚΟς"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", "UTF8_BINARY", "ςΥςΤΗΜΑΤΙΚΟς"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", "UTF8_LCASE", "ςιςΤιΜΑΤΙΚΟς"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", "UNICODE", "ςΥςΤΗΜΑΤΙΚΟς"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", "UNICODE_CI", "ςιςΤιΜΑΤΙΚΟς"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", "UTF8_BINARY", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", "UTF8_LCASE", "ςιςΤιΜΑΤΙΚΟς"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", "UNICODE", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); - assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", "UNICODE_CI", "ςιςΤιΜΑΤΙΚΟς"); - assertStringTranslate("συστηματικος", "Συη", "σιι", "UTF8_BINARY", "σιστιματικος"); - assertStringTranslate("συστηματικος", "Συη", "σιι", "UTF8_LCASE", "σιστιματικοσ"); - assertStringTranslate("συστηματικος", "Συη", "σιι", "UNICODE", "σιστιματικος"); - assertStringTranslate("συστηματικος", "Συη", "σιι", "UNICODE_CI", "σιστιματικοσ"); - assertStringTranslate("συστηματικος", "συη", "σιι", "UTF8_BINARY", "σιστιματικος"); - assertStringTranslate("συστηματικος", "συη", "σιι", "UTF8_LCASE", "σιστιματικοσ"); - assertStringTranslate("συστηματικος", "συη", "σιι", "UNICODE", "σιστιματικος"); - assertStringTranslate("συστηματικος", "συη", "σιι", "UNICODE_CI", "σιστιματικοσ"); - assertStringTranslate("συστηματικος", "ςυη", "σιι", "UTF8_BINARY", "σιστιματικοσ"); - assertStringTranslate("συστηματικος", "ςυη", "σιι", "UTF8_LCASE", "σιστιματικοσ"); - assertStringTranslate("συστηματικος", "ςυη", "σιι", "UNICODE", "σιστιματικοσ"); - assertStringTranslate("συστηματικος", "ςυη", "σιι", "UNICODE_CI", "σιστιματικοσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", UTF8_BINARY, "σΥσΤΗΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", UTF8_LCASE, "σισΤιΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", UNICODE, "σΥσΤΗΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", UNICODE_CI, "σισΤιΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", UTF8_BINARY, "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", UTF8_LCASE, "σισΤιΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", UNICODE, "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", UNICODE_CI, "σισΤιΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", UTF8_BINARY, "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", UTF8_LCASE, "σισΤιΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", UNICODE, "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", UNICODE_CI, "σισΤιΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", UTF8_BINARY, "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", UTF8_LCASE, "ςιςΤιΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", UNICODE, "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", UNICODE_CI, "ςιςΤιΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", UTF8_BINARY, "ςΥςΤΗΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", UTF8_LCASE, "ςιςΤιΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", UNICODE, "ςΥςΤΗΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", UNICODE_CI, "ςιςΤιΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", UTF8_BINARY, "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", UTF8_LCASE, "ςιςΤιΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", UNICODE, "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", UNICODE_CI, "ςιςΤιΜΑΤΙΚΟς"); + assertStringTranslate("συστηματικος", "Συη", "σιι", UTF8_BINARY, "σιστιματικος"); + assertStringTranslate("συστηματικος", "Συη", "σιι", UTF8_LCASE, "σιστιματικοσ"); + assertStringTranslate("συστηματικος", "Συη", "σιι", UNICODE, "σιστιματικος"); + assertStringTranslate("συστηματικος", "Συη", "σιι", UNICODE_CI, "σιστιματικοσ"); + assertStringTranslate("συστηματικος", "συη", "σιι", UTF8_BINARY, "σιστιματικος"); + assertStringTranslate("συστηματικος", "συη", "σιι", UTF8_LCASE, "σιστιματικοσ"); + assertStringTranslate("συστηματικος", "συη", "σιι", UNICODE, "σιστιματικος"); + assertStringTranslate("συστηματικος", "συη", "σιι", UNICODE_CI, "σιστιματικοσ"); + assertStringTranslate("συστηματικος", "ςυη", "σιι", UTF8_BINARY, "σιστιματικοσ"); + assertStringTranslate("συστηματικος", "ςυη", "σιι", UTF8_LCASE, "σιστιματικοσ"); + assertStringTranslate("συστηματικος", "ςυη", "σιι", UNICODE, "σιστιματικοσ"); + assertStringTranslate("συστηματικος", "ςυη", "σιι", UNICODE_CI, "σιστιματικοσ"); // Surrogate pairs. - assertStringTranslate("a🙃b🙃c", "a", "x", "UTF8_BINARY", "x🙃b🙃c"); - assertStringTranslate("a🙃b🙃c", "a🙃", "xy", "UTF8_BINARY", "xybyc"); - assertStringTranslate("a🙃b🙃c", "a🙃b", "xyz", "UTF8_BINARY", "xyzyc"); - assertStringTranslate("a🙃b🙃c", "a🙃bc", "xyzw", "UTF8_BINARY", "xyzyw"); - assertStringTranslate("😀😆😃😄", "😄😆", "😅😂", "UTF8_BINARY", "😀😂😃😅"); - assertStringTranslate("😀😆😃😄", "😄😆", "😅😂", "UTF8_LCASE", "😀😂😃😅"); - assertStringTranslate("😀😆😃😄", "😄😆", "😅😂", "UNICODE", "😀😂😃😅"); - assertStringTranslate("😀😆😃😄", "😄😆", "😅😂", "UNICODE_CI", "😀😂😃😅"); - assertStringTranslate("𐐅", "𐐅", "x", "UTF8_BINARY", "x"); - assertStringTranslate("𐐅", "𐐅", "x", "UTF8_LCASE", "x"); - assertStringTranslate("𐐅", "𐐅", "x", "UNICODE", "x"); - assertStringTranslate("𐐅", "𐐅", "x", "UNICODE_CI", "x"); - assertStringTranslate("𐐅", "𐐭", "x", "UTF8_BINARY", "𐐅"); - assertStringTranslate("𐐅", "𐐭", "x", "UTF8_LCASE", "x"); - assertStringTranslate("𐐅", "𐐭", "x", "UNICODE", "𐐅"); - assertStringTranslate("𐐅", "𐐭", "x", "UNICODE_CI", "x"); - assertStringTranslate("A", "A", "𐐅", "UTF8_BINARY", "𐐅"); - assertStringTranslate("A", "A", "𐐅", "UTF8_LCASE", "𐐅"); - assertStringTranslate("A", "A", "𐐅", "UNICODE", "𐐅"); - assertStringTranslate("A", "A", "𐐅", "UNICODE_CI", "𐐅"); - assertStringTranslate("A", "a", "𐐅", "UTF8_BINARY", "A"); - assertStringTranslate("A", "a", "𐐅", "UTF8_LCASE", "𐐅"); - assertStringTranslate("A", "a", "𐐅", "UNICODE", "A"); - assertStringTranslate("A", "a", "𐐅", "UNICODE_CI", "𐐅"); - assertStringTranslate("a", "A", "𐐅", "UTF8_BINARY", "a"); - assertStringTranslate("a", "A", "𐐅", "UTF8_LCASE", "𐐅"); - assertStringTranslate("a", "A", "𐐅", "UNICODE", "a"); - assertStringTranslate("a", "A", "𐐅", "UNICODE_CI", "𐐅"); - assertStringTranslate("𝔸", "𝔸", "x", "UTF8_BINARY", "x"); - assertStringTranslate("𝔸", "𝔸", "x", "UTF8_LCASE", "x"); - assertStringTranslate("𝔸", "𝔸", "x", "UNICODE", "x"); - assertStringTranslate("𝔸", "𝔸", "x", "UNICODE_CI", "x"); - assertStringTranslate("𝔸", "𝕒", "x", "UTF8_BINARY", "𝔸"); - assertStringTranslate("𝔸", "𝕒", "x", "UTF8_LCASE", "𝔸"); - assertStringTranslate("𝔸", "𝕒", "x", "UNICODE", "𝔸"); - assertStringTranslate("𝔸", "𝕒", "x", "UNICODE_CI", "x"); + assertStringTranslate("a🙃b🙃c", "a", "x", UTF8_BINARY, "x🙃b🙃c"); + assertStringTranslate("a🙃b🙃c", "a🙃", "xy", UTF8_BINARY, "xybyc"); + assertStringTranslate("a🙃b🙃c", "a🙃b", "xyz", UTF8_BINARY, "xyzyc"); + assertStringTranslate("a🙃b🙃c", "a🙃bc", "xyzw", UTF8_BINARY, "xyzyw"); + assertStringTranslate("😀😆😃😄", "😄😆", "😅😂", UTF8_BINARY, "😀😂😃😅"); + assertStringTranslate("😀😆😃😄", "😄😆", "😅😂", UTF8_LCASE, "😀😂😃😅"); + assertStringTranslate("😀😆😃😄", "😄😆", "😅😂", UNICODE, "😀😂😃😅"); + assertStringTranslate("😀😆😃😄", "😄😆", "😅😂", UNICODE_CI, "😀😂😃😅"); + assertStringTranslate("𐐅", "𐐅", "x", UTF8_BINARY, "x"); + assertStringTranslate("𐐅", "𐐅", "x", UTF8_LCASE, "x"); + assertStringTranslate("𐐅", "𐐅", "x", UNICODE, "x"); + assertStringTranslate("𐐅", "𐐅", "x", UNICODE_CI, "x"); + assertStringTranslate("𐐅", "𐐭", "x", UTF8_BINARY, "𐐅"); + assertStringTranslate("𐐅", "𐐭", "x", UTF8_LCASE, "x"); + assertStringTranslate("𐐅", "𐐭", "x", UNICODE, "𐐅"); + assertStringTranslate("𐐅", "𐐭", "x", UNICODE_CI, "x"); + assertStringTranslate("A", "A", "𐐅", UTF8_BINARY, "𐐅"); + assertStringTranslate("A", "A", "𐐅", UTF8_LCASE, "𐐅"); + assertStringTranslate("A", "A", "𐐅", UNICODE, "𐐅"); + assertStringTranslate("A", "A", "𐐅", UNICODE_CI, "𐐅"); + assertStringTranslate("A", "a", "𐐅", UTF8_BINARY, "A"); + assertStringTranslate("A", "a", "𐐅", UTF8_LCASE, "𐐅"); + assertStringTranslate("A", "a", "𐐅", UNICODE, "A"); + assertStringTranslate("A", "a", "𐐅", UNICODE_CI, "𐐅"); + assertStringTranslate("a", "A", "𐐅", UTF8_BINARY, "a"); + assertStringTranslate("a", "A", "𐐅", UTF8_LCASE, "𐐅"); + assertStringTranslate("a", "A", "𐐅", UNICODE, "a"); + assertStringTranslate("a", "A", "𐐅", UNICODE_CI, "𐐅"); + assertStringTranslate("𝔸", "𝔸", "x", UTF8_BINARY, "x"); + assertStringTranslate("𝔸", "𝔸", "x", UTF8_LCASE, "x"); + assertStringTranslate("𝔸", "𝔸", "x", UNICODE, "x"); + assertStringTranslate("𝔸", "𝔸", "x", UNICODE_CI, "x"); + assertStringTranslate("𝔸", "𝕒", "x", UTF8_BINARY, "𝔸"); + assertStringTranslate("𝔸", "𝕒", "x", UTF8_LCASE, "𝔸"); + assertStringTranslate("𝔸", "𝕒", "x", UNICODE, "𝔸"); + assertStringTranslate("𝔸", "𝕒", "x", UNICODE_CI, "x"); } private Map buildDict(String matching, String replace) { diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 8b266e9d6ac11..3c11f5b360bb2 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -752,6 +752,24 @@ }, "sqlState" : "56K00" }, + "CONNECT_ML" : { + "message" : [ + "Generic Spark Connect ML error." + ], + "subClass" : { + "ATTRIBUTE_NOT_ALLOWED" : { + "message" : [ + " is not allowed to be accessed." + ] + }, + "UNSUPPORTED_EXCEPTION" : { + "message" : [ + "" + ] + } + }, + "sqlState" : "XX000" + }, "CONVERSION_INVALID_INPUT" : { "message" : [ "The value () cannot be converted to because it is malformed. Correct the value as per the syntax, or change its format. Use to tolerate malformed input and return NULL instead." @@ -3126,6 +3144,13 @@ ], "sqlState" : "42K08" }, + "INVALID_SQL_FUNCTION_PLAN_STRUCTURE" : { + "message" : [ + "Invalid SQL function plan structure", + "" + ], + "sqlState" : "XXKD0" + }, "INVALID_SQL_SYNTAX" : { "message" : [ "Invalid SQL syntax:" @@ -5757,6 +5782,12 @@ ], "sqlState" : "0A000" }, + "UNSUPPORTED_SQL_UDF_USAGE" : { + "message" : [ + "Using SQL function in is not supported." + ], + "sqlState" : "0A000" + }, "UNSUPPORTED_STREAMING_OPERATOR_WITHOUT_WATERMARK" : { "message" : [ " output mode not supported for on streaming DataFrames/DataSets without watermark." diff --git a/common/utils/src/main/resources/org/apache/spark/log4j2-defaults.properties b/common/utils/src/main/resources/org/apache/spark/log4j2-defaults.properties index 9be86b650d091..777c5f2b25915 100644 --- a/common/utils/src/main/resources/org/apache/spark/log4j2-defaults.properties +++ b/common/utils/src/main/resources/org/apache/spark/log4j2-defaults.properties @@ -22,8 +22,8 @@ rootLogger.appenderRef.stdout.ref = console appender.console.type = Console appender.console.name = console appender.console.target = SYSTEM_ERR -appender.console.layout.type = JsonTemplateLayout -appender.console.layout.eventTemplateUri = classpath:org/apache/spark/SparkLayout.json +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n%ex # Settings to quiet third party logs that are too verbose logger.jetty.name = org.sparkproject.jetty diff --git a/common/utils/src/main/resources/org/apache/spark/log4j2-pattern-layout-defaults.properties b/common/utils/src/main/resources/org/apache/spark/log4j2-json-layout.properties similarity index 94% rename from common/utils/src/main/resources/org/apache/spark/log4j2-pattern-layout-defaults.properties rename to common/utils/src/main/resources/org/apache/spark/log4j2-json-layout.properties index 777c5f2b25915..9be86b650d091 100644 --- a/common/utils/src/main/resources/org/apache/spark/log4j2-pattern-layout-defaults.properties +++ b/common/utils/src/main/resources/org/apache/spark/log4j2-json-layout.properties @@ -22,8 +22,8 @@ rootLogger.appenderRef.stdout.ref = console appender.console.type = Console appender.console.name = console appender.console.target = SYSTEM_ERR -appender.console.layout.type = PatternLayout -appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n%ex +appender.console.layout.type = JsonTemplateLayout +appender.console.layout.eventTemplateUri = classpath:org/apache/spark/SparkLayout.json # Settings to quiet third party logs that are too verbose logger.jetty.name = org.sparkproject.jetty diff --git a/common/utils/src/main/scala/org/apache/spark/internal/Logging.scala b/common/utils/src/main/scala/org/apache/spark/internal/Logging.scala index 7471b764bd2b3..4b60cb20f0732 100644 --- a/common/utils/src/main/scala/org/apache/spark/internal/Logging.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/Logging.scala @@ -337,9 +337,9 @@ trait Logging { if (Logging.defaultSparkLog4jConfig || Logging.islog4j2DefaultConfigured()) { Logging.defaultSparkLog4jConfig = true val defaultLogProps = if (Logging.isStructuredLoggingEnabled) { - "org/apache/spark/log4j2-defaults.properties" + "org/apache/spark/log4j2-json-layout.properties" } else { - "org/apache/spark/log4j2-pattern-layout-defaults.properties" + "org/apache/spark/log4j2-defaults.properties" } Option(SparkClassUtils.getSparkClassLoader.getResource(defaultLogProps)) match { case Some(url) => @@ -398,7 +398,7 @@ private[spark] object Logging { @volatile private var initialized = false @volatile private var defaultRootLevel: Level = null @volatile private var defaultSparkLog4jConfig = false - @volatile private var structuredLoggingEnabled = true + @volatile private var structuredLoggingEnabled = false @volatile private[spark] var sparkShellThresholdLevel: Level = null @volatile private[spark] var setLogLevelPrinted: Boolean = false diff --git a/common/utils/src/test/java/org/apache/spark/util/StructuredSparkLoggerSuite.java b/common/utils/src/test/java/org/apache/spark/util/StructuredSparkLoggerSuite.java index 6959fe11820ff..1fab167adfeb0 100644 --- a/common/utils/src/test/java/org/apache/spark/util/StructuredSparkLoggerSuite.java +++ b/common/utils/src/test/java/org/apache/spark/util/StructuredSparkLoggerSuite.java @@ -21,11 +21,27 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.logging.log4j.Level; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; + +import org.apache.spark.internal.Logging$; import org.apache.spark.internal.SparkLogger; import org.apache.spark.internal.SparkLoggerFactory; public class StructuredSparkLoggerSuite extends SparkLoggerSuiteBase { + // Enable Structured Logging before running the tests + @BeforeAll + public static void setup() { + Logging$.MODULE$.enableStructuredLogging(); + } + + // Disable Structured Logging after running the tests + @AfterAll + public static void teardown() { + Logging$.MODULE$.disableStructuredLogging(); + } + private static final SparkLogger LOGGER = SparkLoggerFactory.getLogger(StructuredSparkLoggerSuite.class); diff --git a/common/utils/src/test/scala/org/apache/spark/util/MDCSuite.scala b/common/utils/src/test/scala/org/apache/spark/util/MDCSuite.scala index 7631c25662219..9615eb2263636 100644 --- a/common/utils/src/test/scala/org/apache/spark/util/MDCSuite.scala +++ b/common/utils/src/test/scala/org/apache/spark/util/MDCSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.util import scala.jdk.CollectionConverters._ +import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite import org.apache.spark.internal.{Logging, MDC} @@ -26,7 +27,16 @@ import org.apache.spark.internal.LogKeys.{EXIT_CODE, OFFSET, RANGE} class MDCSuite extends AnyFunSuite // scalastyle:ignore funsuite - with Logging { + with Logging + with BeforeAndAfterAll { + + override def beforeAll(): Unit = { + Logging.enableStructuredLogging() + } + + override def afterAll(): Unit = { + Logging.disableStructuredLogging() + } test("check MDC message") { val log = log"This is a log, exitcode ${MDC(EXIT_CODE, 10086)}" diff --git a/common/utils/src/test/scala/org/apache/spark/util/PatternLoggingSuite.scala b/common/utils/src/test/scala/org/apache/spark/util/PatternLoggingSuite.scala index 2ba2b15c49f33..248136798b362 100644 --- a/common/utils/src/test/scala/org/apache/spark/util/PatternLoggingSuite.scala +++ b/common/utils/src/test/scala/org/apache/spark/util/PatternLoggingSuite.scala @@ -17,19 +17,16 @@ package org.apache.spark.util import org.apache.logging.log4j.Level -import org.scalatest.BeforeAndAfterAll import org.apache.spark.internal.Logging -class PatternLoggingSuite extends LoggingSuiteBase with BeforeAndAfterAll { +class PatternLoggingSuite extends LoggingSuiteBase { override def className: String = classOf[PatternLoggingSuite].getSimpleName override def logFilePath: String = "target/pattern.log" override def beforeAll(): Unit = Logging.disableStructuredLogging() - override def afterAll(): Unit = Logging.enableStructuredLogging() - override def expectedPatternForBasicMsg(level: Level): String = { s""".*$level $className: This is a log message\n""" } diff --git a/common/utils/src/test/scala/org/apache/spark/util/StructuredLoggingSuite.scala b/common/utils/src/test/scala/org/apache/spark/util/StructuredLoggingSuite.scala index 48951c2084f17..0026b696f0695 100644 --- a/common/utils/src/test/scala/org/apache/spark/util/StructuredLoggingSuite.scala +++ b/common/utils/src/test/scala/org/apache/spark/util/StructuredLoggingSuite.scala @@ -23,14 +23,21 @@ import java.nio.file.Files import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import org.apache.logging.log4j.Level +import org.scalatest.BeforeAndAfterAll import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite import org.apache.spark.internal.{LogEntry, Logging, LogKey, LogKeys, MDC, MessageWithContext} trait LoggingSuiteBase extends AnyFunSuite // scalastyle:ignore funsuite + with BeforeAndAfterAll with Logging { + override def afterAll(): Unit = { + super.afterAll() + Logging.disableStructuredLogging() + } + def className: String def logFilePath: String @@ -202,7 +209,7 @@ trait LoggingSuiteBase } } - private val customLog = log"${MDC(CustomLogKeys.CUSTOM_LOG_KEY, "Custom log message.")}" + private lazy val customLog = log"${MDC(CustomLogKeys.CUSTOM_LOG_KEY, "Custom log message.")}" test("Logging with custom LogKey") { Seq( (Level.ERROR, () => logError(customLog)), @@ -265,6 +272,13 @@ class StructuredLoggingSuite extends LoggingSuiteBase { override def className: String = classOf[StructuredLoggingSuite].getSimpleName override def logFilePath: String = "target/structured.log" + override def beforeAll(): Unit = { + super.beforeAll() + Logging.enableStructuredLogging() + } + + override def afterAll(): Unit = super.afterAll() + private val jsonMapper = new ObjectMapper().registerModule(DefaultScalaModule) private def compactAndToRegexPattern(json: String): String = { jsonMapper.readTree(json).toString. diff --git a/conf/log4j2.properties.pattern-layout-template b/conf/log4j2-json-layout.properties.template similarity index 80% rename from conf/log4j2.properties.pattern-layout-template rename to conf/log4j2-json-layout.properties.template index 011fca58c9b2a..76499bb6691e7 100644 --- a/conf/log4j2.properties.pattern-layout-template +++ b/conf/log4j2-json-layout.properties.template @@ -19,17 +19,11 @@ rootLogger.level = info rootLogger.appenderRef.stdout.ref = console -# In the pattern layout configuration below, we specify an explicit `%ex` conversion -# pattern for logging Throwables. If this was omitted, then (by default) Log4J would -# implicitly add an `%xEx` conversion pattern which logs stacktraces with additional -# class packaging information. That extra information can sometimes add a substantial -# performance overhead, so we disable it in our default logging config. -# For more information, see SPARK-39361. appender.console.type = Console appender.console.name = console appender.console.target = SYSTEM_ERR -appender.console.layout.type = PatternLayout -appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n%ex +appender.console.layout.type = JsonTemplateLayout +appender.console.layout.eventTemplateUri = classpath:org/apache/spark/SparkLayout.json # Set the default spark-shell/spark-sql log level to WARN. When running the # spark-shell/spark-sql, the log level for these classes is used to overwrite diff --git a/conf/log4j2.properties.template b/conf/log4j2.properties.template index 76499bb6691e7..011fca58c9b2a 100644 --- a/conf/log4j2.properties.template +++ b/conf/log4j2.properties.template @@ -19,11 +19,17 @@ rootLogger.level = info rootLogger.appenderRef.stdout.ref = console +# In the pattern layout configuration below, we specify an explicit `%ex` conversion +# pattern for logging Throwables. If this was omitted, then (by default) Log4J would +# implicitly add an `%xEx` conversion pattern which logs stacktraces with additional +# class packaging information. That extra information can sometimes add a substantial +# performance overhead, so we disable it in our default logging config. +# For more information, see SPARK-39361. appender.console.type = Console appender.console.name = console appender.console.target = SYSTEM_ERR -appender.console.layout.type = JsonTemplateLayout -appender.console.layout.eventTemplateUri = classpath:org/apache/spark/SparkLayout.json +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n%ex # Set the default spark-shell/spark-sql log level to WARN. When running the # spark-shell/spark-sql, the log level for these classes is used to overwrite diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala index 4b36d36983a5d..1d2165b668f61 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.{SparkException, SparkRuntimeException} +import org.apache.spark.SparkRuntimeException import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.{QueryTest, RemoteSparkSession} @@ -665,15 +665,52 @@ class DataFrameSubquerySuite extends QueryTest with RemoteSparkSession { withView("t1") { val t1 = table1() - // TODO(SPARK-50601): Fix the SparkConnectPlanner to support this case - checkError( - intercept[SparkException] { - t1.withColumn("scalar", spark.range(1).select($"c1".outer() + $"c2".outer()).scalar()) - .collect() - }, - "INTERNAL_ERROR", - parameters = Map("message" -> "Found the unresolved operator: .*"), - matchPVals = true) + checkAnswer( + t1.withColumn( + "scalar", + spark + .range(1) + .select($"c1".outer() + $"c2".outer()) + .scalar()), + t1.select($"*", ($"c1" + $"c2").as("scalar"))) + + checkAnswer( + t1.withColumn( + "scalar", + spark + .range(1) + .withColumn("c1", $"c1".outer()) + .select($"c1" + $"c2".outer()) + .scalar()), + t1.select($"*", ($"c1" + $"c2").as("scalar"))) + + checkAnswer( + t1.withColumn( + "scalar", + spark + .range(1) + .select($"c1".outer().as("c1")) + .withColumn("c2", $"c2".outer()) + .select($"c1" + $"c2") + .scalar()), + t1.select($"*", ($"c1" + $"c2").as("scalar"))) + } + } + + test("subquery in withColumnsRenamed") { + withView("t1") { + val t1 = table1() + + checkAnswer( + t1.withColumn( + "scalar", + spark + .range(1) + .select($"c1".outer().as("c1"), $"c2".outer().as("c2")) + .withColumnsRenamed(Map("c1" -> "x", "c2" -> "y")) + .select($"x" + $"y") + .scalar()), + t1.select($"*", ($"c1".as("x") + $"c2".as("y")).as("scalar"))) } } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala index 6fd664d905408..021b4fea26e2a 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala @@ -460,6 +460,14 @@ class KeyValueGroupedDatasetE2ETestSuite extends QueryTest with RemoteSparkSessi (5, "hello")) } + test("SPARK-50789: reduceGroups on unresolved plan") { + val ds = Seq("abc", "xyz", "hello").toDS().select("*").as[String] + checkDatasetUnorderly( + ds.groupByKey(_.length).reduceGroups(_ + _), + (3, "abcxyz"), + (5, "hello")) + } + test("groupby") { val ds = Seq(("a", 1, 10), ("a", 2, 20), ("b", 2, 1), ("b", 1, 2), ("c", 1, 1)) .toDF("key", "seq", "value") diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala index 8415444c10aac..19275326d6421 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala @@ -401,6 +401,13 @@ class UserDefinedFunctionE2ETestSuite extends QueryTest with RemoteSparkSession assert(ds.select(aggCol).head() == 135) // 45 + 90 } + test("SPARK-50789: UDAF custom Aggregator - toColumn on unresolved plan") { + val encoder = Encoders.product[UdafTestInput] + val aggCol = new CompleteUdafTestInputAggregator().toColumn + val ds = spark.range(10).withColumn("extra", col("id") * 2).select("*").as(encoder) + assert(ds.select(aggCol).head() == 135) // 45 + 90 + } + test("UDAF custom Aggregator - multiple extends - toColumn") { val encoder = Encoders.product[UdafTestInput] val aggCol = new CompleteGrandChildUdafTestInputAggregator().toColumn @@ -408,11 +415,24 @@ class UserDefinedFunctionE2ETestSuite extends QueryTest with RemoteSparkSession assert(ds.select(aggCol).head() == 540) // (45 + 90) * 4 } - test("UDAF custom aggregator - with rows - toColumn") { + test("SPARK-50789: UDAF custom Aggregator - multiple extends - toColumn on unresolved plan") { + val encoder = Encoders.product[UdafTestInput] + val aggCol = new CompleteGrandChildUdafTestInputAggregator().toColumn + val ds = spark.range(10).withColumn("extra", col("id") * 2).select("*").as(encoder) + assert(ds.select(aggCol).head() == 540) // (45 + 90) * 4 + } + + test("UDAF custom Aggregator - with rows - toColumn") { val ds = spark.range(10).withColumn("extra", col("id") * 2) assert(ds.select(RowAggregator.toColumn).head() == 405) assert(ds.agg(RowAggregator.toColumn).head().getLong(0) == 405) } + + test("SPARK-50789: UDAF custom Aggregator - with rows - toColumn on unresolved plan") { + val ds = spark.range(10).withColumn("extra", col("id") * 2).select("*") + assert(ds.select(RowAggregator.toColumn).head() == 405) + assert(ds.agg(RowAggregator.toColumn).head().getLong(0) == 405) + } } case class UdafTestInput(id: Long, extra: Long) diff --git a/connector/profiler/README.md b/connector/profiler/README.md index 1326fd55df097..4d97b15eb96ab 100644 --- a/connector/profiler/README.md +++ b/connector/profiler/README.md @@ -16,7 +16,7 @@ The profiler writes the jfr files to the executor's working directory in the exe Code profiling is currently only supported for * Linux (x64) -* Linux (arm 64) +* Linux (arm64) * Linux (musl, x64) * MacOS @@ -54,7 +54,7 @@ Then enable the profiling in the configuration. spark.executor.profiling.dfsDir (none) - An HDFS compatible path to which the profiler's output files are copied. The output files will be written as dfsDir/application_id/profile-appname-exec-executor_id.jfr
+ An HDFS compatible path to which the profiler's output files are copied. The output files will be written as dfsDir/{{APP_ID}}/profile-exec-{{EXECUTOR_ID}}.jfr
If no dfsDir is specified then the files are not copied over. Users should ensure there is sufficient disk space available otherwise it may lead to corrupt jfr files. 4.0.0 @@ -72,7 +72,7 @@ Then enable the profiling in the configuration. event=wall,interval=10ms,alloc=2m,lock=10ms,chunktime=300s Options to pass to the profiler. Detailed options are documented in the comments here: - Profiler arguments. + Profiler arguments. Note that the options to start, stop, specify output format, and output file do not have to be specified. 4.0.0 diff --git a/connector/profiler/src/main/scala/org/apache/spark/executor/profiler/ExecutorJVMProfiler.scala b/connector/profiler/src/main/scala/org/apache/spark/executor/profiler/ExecutorJVMProfiler.scala index 20b6db5221fa9..94e5b46c65881 100644 --- a/connector/profiler/src/main/scala/org/apache/spark/executor/profiler/ExecutorJVMProfiler.scala +++ b/connector/profiler/src/main/scala/org/apache/spark/executor/profiler/ExecutorJVMProfiler.scala @@ -17,17 +17,17 @@ package org.apache.spark.executor.profiler import java.io.{BufferedInputStream, FileInputStream, InputStream, IOException} -import java.net.URI import java.util.concurrent.{ScheduledExecutorService, TimeUnit} import one.profiler.{AsyncProfiler, AsyncProfilerLoader} import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} +import org.apache.hadoop.fs.permission.FsPermission import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys.PATH -import org.apache.spark.util.ThreadUtils +import org.apache.spark.util.{ThreadUtils, Utils} /** @@ -38,15 +38,26 @@ private[spark] class ExecutorJVMProfiler(conf: SparkConf, executorId: String) ex private var running = false private val enableProfiler = conf.get(EXECUTOR_PROFILING_ENABLED) private val profilerOptions = conf.get(EXECUTOR_PROFILING_OPTIONS) - private val profilerDfsDir = conf.get(EXECUTOR_PROFILING_DFS_DIR) + private val profilerDfsDirOpt = conf.get(EXECUTOR_PROFILING_DFS_DIR) private val profilerLocalDir = conf.get(EXECUTOR_PROFILING_LOCAL_DIR) private val writeInterval = conf.get(EXECUTOR_PROFILING_WRITE_INTERVAL) - private val startcmd = s"start,$profilerOptions,file=$profilerLocalDir/profile.jfr" - private val stopcmd = s"stop,$profilerOptions,file=$profilerLocalDir/profile.jfr" - private val dumpcmd = s"dump,$profilerOptions,file=$profilerLocalDir/profile.jfr" - private val resumecmd = s"resume,$profilerOptions,file=$profilerLocalDir/profile.jfr" + private val appId = try { + conf.getAppId + } catch { + case _: NoSuchElementException => "local-" + System.currentTimeMillis + } + private val appAttemptId = conf.getOption("spark.app.attempt.id") + private val baseName = Utils.nameForAppAndAttempt(appId, appAttemptId) + private val profileFile = s"profile-exec-$executorId.jfr" + + private val startcmd = s"start,$profilerOptions,file=$profilerLocalDir/$profileFile" + private val stopcmd = s"stop,$profilerOptions,file=$profilerLocalDir/$profileFile" + private val dumpcmd = s"dump,$profilerOptions,file=$profilerLocalDir/$profileFile" + private val resumecmd = s"resume,$profilerOptions,file=$profilerLocalDir/$profileFile" + private val PROFILER_FOLDER_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) + private val PROFILER_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("660", 8).toShort) private val UPLOAD_SIZE = 8 * 1024 * 1024 // 8 MB private var outputStream: FSDataOutputStream = _ private var inputStream: InputStream = _ @@ -89,28 +100,34 @@ private[spark] class ExecutorJVMProfiler(conf: SparkConf, executorId: String) ex } } + private def requireProfilerBaseDirAsDirectory(fs: FileSystem, profilerDfsDir: String): Unit = { + if (!fs.getFileStatus(new Path(profilerDfsDir)).isDirectory) { + throw new IllegalArgumentException( + s"Profiler DFS base directory $profilerDfsDir is not a directory.") + } + } + private def startWriting(): Unit = { - if (profilerDfsDir.isDefined) { - val applicationId = try { - conf.getAppId - } catch { - case _: NoSuchElementException => "local-" + System.currentTimeMillis + profilerDfsDirOpt.foreach { profilerDfsDir => + val profilerDirForApp = s"$profilerDfsDir/$baseName" + val profileOutputFile = s"$profilerDirForApp/$profileFile" + + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + val fs = Utils.getHadoopFileSystem(profilerDfsDir, hadoopConf) + + requireProfilerBaseDirAsDirectory(fs, profilerDfsDir) + + val profilerDirForAppPath = new Path(profilerDirForApp) + if (!fs.exists(profilerDirForAppPath)) { + // SPARK-30860: use the class method to avoid the umask causing permission issues + FileSystem.mkdirs(fs, profilerDirForAppPath, PROFILER_FOLDER_PERMISSIONS) } - val config = SparkHadoopUtil.get.newConfiguration(conf) - val appName = conf.get("spark.app.name").replace(" ", "-") - val profilerOutputDirname = profilerDfsDir.get - - val profileOutputFile = - s"$profilerOutputDirname/$applicationId/profile-$appName-exec-$executorId.jfr" - val fs = FileSystem.get(new URI(profileOutputFile), config); - val filenamePath = new Path(profileOutputFile) - outputStream = fs.create(filenamePath) + + outputStream = FileSystem.create(fs, new Path(profileOutputFile), PROFILER_FILE_PERMISSIONS) try { - if (fs.exists(filenamePath)) { - fs.delete(filenamePath, true) - } logInfo(log"Copying executor profiling file to ${MDC(PATH, profileOutputFile)}") - inputStream = new BufferedInputStream(new FileInputStream(s"$profilerLocalDir/profile.jfr")) + inputStream = new BufferedInputStream( + new FileInputStream(s"$profilerLocalDir/$profileFile")) threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("profilerOutputThread") threadpool.scheduleWithFixedDelay( new Runnable() { @@ -158,14 +175,14 @@ private[spark] class ExecutorJVMProfiler(conf: SparkConf, executorId: String) ex } catch { case e: IOException => logError("Exception occurred while writing some profiler output: ", e) case e @ (_: IllegalArgumentException | _: IllegalStateException) => - logError("Some profiler output not written." + - " Exception occurred in profiler native code: ", e) + logError("Some profiler output not written. " + + "Exception occurred in profiler native code: ", e) case e: Exception => logError("Some profiler output not written. Unexpected exception: ", e) } } private def finishWriting(): Unit = { - if (profilerDfsDir.isDefined && writing) { + if (profilerDfsDirOpt.isDefined && writing) { try { // shutdown background writer threadpool.shutdown() @@ -177,8 +194,8 @@ private[spark] class ExecutorJVMProfiler(conf: SparkConf, executorId: String) ex } catch { case _: InterruptedException => Thread.currentThread().interrupt() case e: IOException => - logWarning("Some profiling output not written." + - "Exception occurred while completing profiler output", e) + logWarning("Some profiling output not written. " + + "Exception occurred while completing profiler output: ", e) } writing = false } diff --git a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufSerializer.scala b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufSerializer.scala index 1c64e70755d5c..65e8cce0d056e 100644 --- a/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufSerializer.scala +++ b/connector/protobuf/src/main/scala/org/apache/spark/sql/protobuf/ProtobufSerializer.scala @@ -47,6 +47,9 @@ private[sql] class ProtobufSerializer( } private val converter: Any => Any = { + assert( + rootCatalystType.isInstanceOf[StructType], + "ProtobufSerializer's root catalyst type must be a struct type") val baseConverter = try { rootCatalystType match { diff --git a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala index 44a8339ac1f02..c1f5186b8333f 100644 --- a/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala +++ b/connector/protobuf/src/test/scala/org/apache/spark/sql/protobuf/ProtobufFunctionsSuite.scala @@ -1721,6 +1721,33 @@ class ProtobufFunctionsSuite extends QueryTest with SharedSparkSession with Prot } } + test("non-struct SQL type") { + val dfWithInt = spark + .range(1) + .select( + lit(9999).as("int_col") + ) + + val parseError = intercept[AnalysisException] { + dfWithInt.select( + to_protobuf_wrapper($"int_col", "SimpleMessageEnum", Some(testFileDesc))).collect() + } + val descMsg = testFileDesc.map("%02X".format(_)).mkString("") + checkError( + exception = parseError, + condition = "DATATYPE_MISMATCH.TYPE_CHECK_FAILURE_WITH_HINT", + parameters = Map( + "sqlExpr" -> + s"""\"to_protobuf(int_col, SimpleMessageEnum, X'$descMsg', NULL)\"""", + "msg" -> ("The first argument of the TO_PROTOBUF SQL function must be a struct type"), + "hint" -> "" + ), + queryContext = Array(ExpectedContext( + fragment = "fn", + callSitePattern = ".*")) + ) + } + test("test unsigned integer types") { // Test that we correctly handle unsigned integer parsing. // We're using Integer/Long's `MIN_VALUE` as it has a 1 in the sign bit. diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala index f3bb6d5af3358..990ab680f3aaf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala @@ -187,12 +187,7 @@ object EventLogFileWriter { } def nameForAppAndAttempt(appId: String, appAttemptId: Option[String]): String = { - val base = Utils.sanitizeDirName(appId) - if (appAttemptId.isDefined) { - base + "_" + Utils.sanitizeDirName(appAttemptId.get) - } else { - base - } + Utils.nameForAppAndAttempt(appId, appAttemptId) } def codecName(log: Path): Option[String] = { diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 6d51424f0baff..5dda7afc3ebcb 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -162,7 +162,7 @@ package object config { "PySpark shell.") .version("4.0.0") .booleanConf - .createWithDefault(true) + .createWithDefault(false) private[spark] val LEGACY_TASK_NAME_MDC_ENABLED = ConfigBuilder("spark.log.legacyTaskNameMdc.enabled") diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index e7b65bf1a4eff..1efe181a8c38a 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2689,7 +2689,7 @@ private[spark] object Utils * loading SparkConf. */ def resetStructuredLogging(sparkConf: SparkConf): Unit = { - if (sparkConf.getBoolean(STRUCTURED_LOGGING_ENABLED.key, defaultValue = true)) { + if (sparkConf.get(STRUCTURED_LOGGING_ENABLED)) { Logging.enableStructuredLogging() } else { Logging.disableStructuredLogging() @@ -2954,6 +2954,15 @@ private[spark] object Utils str.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase(Locale.ROOT) } + def nameForAppAndAttempt(appId: String, appAttemptId: Option[String]): String = { + val base = sanitizeDirName(appId) + if (appAttemptId.isDefined) { + base + "_" + sanitizeDirName(appAttemptId.get) + } else { + base + } + } + def isClientMode(conf: SparkConf): Boolean = { "client".equals(conf.get(SparkLauncher.DEPLOY_MODE, "client")) } diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 875499320b481..ec3ef63dbe874 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -103,7 +103,7 @@ httpcore/4.4.16//httpcore-4.4.16.jar icu4j/76.1//icu4j-76.1.jar ini4j/0.5.4//ini4j-0.5.4.jar istack-commons-runtime/3.0.8//istack-commons-runtime-3.0.8.jar -ivy/2.5.2//ivy-2.5.2.jar +ivy/2.5.3//ivy-2.5.3.jar j2objc-annotations/3.0.0//j2objc-annotations-3.0.0.jar jackson-annotations/2.18.2//jackson-annotations-2.18.2.jar jackson-core-asl/1.9.13//jackson-core-asl-1.9.13.jar diff --git a/dev/lint-scala b/dev/lint-scala index 23df146a8d1b4..30642a550401e 100755 --- a/dev/lint-scala +++ b/dev/lint-scala @@ -20,8 +20,10 @@ SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)" +set -e "$SCRIPT_DIR/scalastyle" "$1" +set +e # For Spark Connect, we actively enforce scalafmt and check that the produced diff is empty. ERRORS=$(./build/mvn \ -Pscala-2.13 \ diff --git a/dev/run-pip-tests b/dev/run-pip-tests index 91399ff1e25ea..f8a547b0c917c 100755 --- a/dev/run-pip-tests +++ b/dev/run-pip-tests @@ -87,10 +87,6 @@ for python in "${PYTHON_EXECS[@]}"; do VIRTUALENV_PATH="$VIRTUALENV_BASE"/$python rm -rf "$VIRTUALENV_PATH" if [ -n "$USE_CONDA" ]; then - if [ -f "$CONDA_PREFIX/etc/profile.d/conda.sh" ]; then - # See also https://github.com/conda/conda/issues/7980 - source "$CONDA_PREFIX/etc/profile.d/conda.sh" - fi conda create -y -p "$VIRTUALENV_PATH" python=$python numpy pandas pip setuptools source activate "$VIRTUALENV_PATH" || conda activate "$VIRTUALENV_PATH" else diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 71817a64b3581..d43feceb76648 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -686,6 +686,7 @@ def __hash__(self): "pyspark.ml.tests.connect.test_legacy_mode_classification", "pyspark.ml.tests.connect.test_legacy_mode_pipeline", "pyspark.ml.tests.connect.test_legacy_mode_tuning", + "pyspark.ml.tests.test_classification", ], excluded_python_implementations=[ "PyPy" # Skip these tests under PyPy since they require numpy and it isn't available there @@ -1106,6 +1107,7 @@ def __hash__(self): "pyspark.ml.tests.connect.test_connect_classification", "pyspark.ml.tests.connect.test_connect_pipeline", "pyspark.ml.tests.connect.test_connect_tuning", + "pyspark.ml.tests.connect.test_connect_spark_ml_classification", ], excluded_python_implementations=[ "PyPy" # Skip these tests under PyPy since they require numpy, pandas, and pyarrow and diff --git a/docs/configuration.md b/docs/configuration.md index 4a85c4f256a95..162165ffe68dd 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -2850,7 +2850,7 @@ Apart from these, the following properties are also available, and may be useful If set to "true", prevent Spark from scheduling tasks on executors that have been excluded due to too many task failures. The algorithm used to exclude executors and nodes can be further controlled by the other "spark.excludeOnFailure" configuration options. - This config will be overriden by "spark.excludeOnFailure.application.enabled" and + This config will be overridden by "spark.excludeOnFailure.application.enabled" and "spark.excludeOnFailure.taskAndStage.enabled" to specify exclusion enablement on individual levels. @@ -3751,15 +3751,20 @@ Note: When running Spark on YARN in `cluster` mode, environment variables need t # Configuring Logging -Spark uses [log4j](http://logging.apache.org/log4j/) for logging. You can configure it by adding a -`log4j2.properties` file in the `conf` directory. One way to start is to copy the existing templates `log4j2.properties.template` or `log4j2.properties.pattern-layout-template` located there. +Spark uses [log4j](http://logging.apache.org/log4j/) for logging. You can configure it by adding a `log4j2.properties` file in the `conf` directory. To get started, copy one of the provided templates: `log4j2.properties.template` (for plain text logging) or `log4j2-json-layout.properties.template` (for structured logging). + +## Plain Text Logging +The default logging format is plain text, using Log4j's [Pattern Layout](https://logging.apache.org/log4j/2.x/manual/pattern-layout.html). + +MDC (Mapped Diagnostic Context) information is not included by default in plain text logs. To include it, update the `PatternLayout` configuration in the `log4j2.properties` file. For example, add `%X{task_name}` to include the task name in logs. Additionally, use `spark.sparkContext.setLocalProperty("key", "value")` to add custom data to the MDC. ## Structured Logging -Starting from version 4.0.0, `spark-submit` has adopted the [JSON Template Layout](https://logging.apache.org/log4j/2.x/manual/json-template-layout.html) for logging, which outputs logs in JSON format. This format facilitates querying logs using Spark SQL with the JSON data source. Additionally, the logs include all Mapped Diagnostic Context (MDC) information for search and debugging purposes. +Starting with version 4.0.0, `spark-submit` supports optional structured logging using the [JSON Template Layout](https://logging.apache.org/log4j/2.x/manual/json-template-layout.html). This format enables efficient querying of logs with Spark SQL using the JSON data source and includes all MDC information for improved searchability and debugging. -To configure the layout of structured logging, start with the `log4j2.properties.template` file. +To enable structured logging and include MDC information, set the configuration `spark.log.structuredLogging.enabled` to `true` (default is `false`). For additional customization, copy `log4j2-json-layout.properties.template` to `conf/log4j2.properties` and adjust as needed. -To query Spark logs using Spark SQL, you can use the following code snippets: +### Querying Structured Logs with Spark SQL +To query structured logs in JSON format, use the following code snippet: **Python:** ```python @@ -3775,14 +3780,6 @@ import org.apache.spark.util.LogUtils.SPARK_LOG_SCHEMA val logDf = spark.read.schema(SPARK_LOG_SCHEMA).json("path/to/logs") ``` **Note**: If you're using the interactive shell (pyspark shell or spark-shell), you can omit the import statement in the code because SPARK_LOG_SCHEMA is already available in the shell's context. -## Plain Text Logging -If you prefer plain text logging, you have two options: -- Disable structured JSON logging by setting the Spark configuration `spark.log.structuredLogging.enabled` to `false`. -- Use a custom log4j configuration file. Rename `conf/log4j2.properties.pattern-layout-template` to `conf/log4j2.properties`. This reverts to the default configuration prior to Spark 4.0, which utilizes [PatternLayout](https://logging.apache.org/log4j/2.x/manual/layouts.html#PatternLayout) for logging all messages in plain text. - -MDC information is not included by default when with plain text logging. In order to print it in the logs, you can update the patternLayout in the file. For example, you can add `%X{task_name}` to print the task name in the logs. -Moreover, you can use `spark.sparkContext.setLocalProperty(s"mdc.$name", "value")` to add user specific data into MDC. -The key in MDC will be the string of `mdc.$name`. # Overriding configuration directory diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md index 49737392312a7..9dcf4ad8a2984 100644 --- a/docs/core-migration-guide.md +++ b/docs/core-migration-guide.md @@ -44,10 +44,6 @@ license: | - Since Spark 4.0, Spark uses the external shuffle service for deleting shuffle blocks for deallocated executors when the shuffle is no longer needed. To restore the legacy behavior, you can set `spark.shuffle.service.removeShuffle` to `false`. -- Starting with Spark 4.0, the default logging format for `spark-submit` has changed from plain text to JSON lines to improve log analysis. If you prefer plain text logs, you have two options: - - Set the Spark configuration `spark.log.structuredLogging.enabled` to `false`. For example, you can use `JDK_JAVA_OPTIONS=-Dspark.log.structuredLogging.enabled=false`. - - Use a custom log4j configuration file, such as renaming the template file `conf/log4j2.properties.pattern-layout-template` to `conf/log4j2.properties`. - - Since Spark 4.0, the MDC (Mapped Diagnostic Context) key for Spark task names in Spark logs has been changed from `mdc.taskName` to `task_name`. To use the key `mdc.taskName`, you can set `spark.log.legacyTaskNameMdc.enabled` to `true`. - Since Spark 4.0, Spark performs speculative executions less aggressively with `spark.speculation.multiplier=3` and `spark.speculation.quantile=0.9`. To restore the legacy behavior, you can set `spark.speculation.multiplier=1.5` and `spark.speculation.quantile=0.75`. diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index a0c73813612d0..c7f5d67a6cd85 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -394,7 +394,7 @@ spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.mount. spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.mount.readOnly=false ``` -To enable shuffle data recovery feature via the built-in `KubernetesLocalDiskShuffleDataIO` plugin, we need to have the followings. You may want to enable `spark.kubernetes.driver.waitToReusePersistentVolumeClaim` additionally. +To enable shuffle data recovery feature via the built-in `KubernetesLocalDiskShuffleDataIO` plugin, we need to have the following. You may want to enable `spark.kubernetes.driver.waitToReusePersistentVolumeClaim` additionally. ``` spark.kubernetes.executor.volumes.persistentVolumeClaim.spark-local-dir-1.mount.path=/data/spark-x/executor-x diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md index 7ad56bf0657b2..9a50db992f9c8 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt-add-jar.md @@ -54,9 +54,9 @@ ADD JAR '/some/other.jar'; ADD JAR "/path with space/abc.jar"; ADD JARS "/path with space/def.jar" '/path with space/ghi.jar'; ADD JAR "ivy://group:module:version"; -ADD JAR "ivy://group:module:version?transitive=false" -ADD JAR "ivy://group:module:version?transitive=true" -ADD JAR "ivy://group:module:version?exclude=group:module&transitive=true" +ADD JAR "ivy://group:module:version?transitive=false"; +ADD JAR "ivy://group:module:version?transitive=true"; +ADD JAR "ivy://group:module:version?exclude=group:module&transitive=true"; ``` ### Related Statements diff --git a/docs/sql-ref-syntax-ddl-alter-table.md b/docs/sql-ref-syntax-ddl-alter-table.md index adcfa8db06f12..28ecc44a5bf7e 100644 --- a/docs/sql-ref-syntax-ddl-alter-table.md +++ b/docs/sql-ref-syntax-ddl-alter-table.md @@ -673,12 +673,12 @@ ALTER TABLE loc_orc SET fileformat orc; ALTER TABLE p1 partition (month=2, day=2) SET fileformat parquet; -- Change the file Location -ALTER TABLE dbx.tab1 PARTITION (a='1', b='2') SET LOCATION '/path/to/part/ways' +ALTER TABLE dbx.tab1 PARTITION (a='1', b='2') SET LOCATION '/path/to/part/ways'; -- SET SERDE/ SERDE Properties ALTER TABLE test_tab SET SERDE 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'; -ALTER TABLE dbx.tab1 SET SERDE 'org.apache.hadoop' WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee') +ALTER TABLE dbx.tab1 SET SERDE 'org.apache.hadoop' WITH SERDEPROPERTIES ('k' = 'v', 'kay' = 'vee'); -- SET TABLE PROPERTIES ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('winner' = 'loser'); diff --git a/docs/sql-ref-syntax-qry-star.md b/docs/sql-ref-syntax-qry-star.md index 3a997dad644b9..c575727e820e8 100644 --- a/docs/sql-ref-syntax-qry-star.md +++ b/docs/sql-ref-syntax-qry-star.md @@ -21,7 +21,7 @@ license: | ### Description -A shorthand to name all the referencable columns in the FROM clause or a specific table reference's columns or fields in the FROM clause. +A shorthand to name all the referenceable columns in the FROM clause or a specific table reference's columns or fields in the FROM clause. The star clause is most frequently used in the SELECT list. Spark also supports its use in function invocation and certain n-ary operations within the SELECT list and WHERE clause. @@ -38,11 +38,11 @@ except_clause * **name** - If present limits the columns or fields to be named to those in the specified referencable field, column, or table. + If present limits the columns or fields to be named to those in the specified referenceable field, column, or table. * **except_clause** - Optionally prunes columns or fields from the referencable set of columns identified in the select_star clause. + Optionally prunes columns or fields from the referenceable set of columns identified in the select_star clause. * **column_name** diff --git a/mllib/src/main/resources/META-INF/services/org.apache.spark.ml.Estimator b/mllib/src/main/resources/META-INF/services/org.apache.spark.ml.Estimator new file mode 100644 index 0000000000000..e6902f62c4d60 --- /dev/null +++ b/mllib/src/main/resources/META-INF/services/org.apache.spark.ml.Estimator @@ -0,0 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Spark Connect ML uses ServiceLoader to find out the supported Spark Ml estimators. +# So register the supported estimator here if you're trying to add a new one. +org.apache.spark.ml.classification.LogisticRegression diff --git a/mllib/src/main/resources/META-INF/services/org.apache.spark.ml.Transformer b/mllib/src/main/resources/META-INF/services/org.apache.spark.ml.Transformer new file mode 100644 index 0000000000000..004ec8aeff8cf --- /dev/null +++ b/mllib/src/main/resources/META-INF/services/org.apache.spark.ml.Transformer @@ -0,0 +1,20 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Spark Connect ML uses ServiceLoader to find out the supported Spark Ml non-model transformer. +# So register the supported transformer here if you're trying to add a new one. +org.apache.spark.ml.feature.VectorAssembler diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ClassificationSummary.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ClassificationSummary.scala index 9f3428db484c2..88cfb703fca41 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ClassificationSummary.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ClassificationSummary.scala @@ -19,6 +19,7 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.Since import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.util.Summary import org.apache.spark.mllib.evaluation.{BinaryClassificationMetrics, MulticlassMetrics} import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.functions.{col, lit} @@ -28,7 +29,7 @@ import org.apache.spark.sql.types.DoubleType /** * Abstraction for multiclass classification results for a given model. */ -private[classification] trait ClassificationSummary extends Serializable { +private[classification] trait ClassificationSummary extends Summary with Serializable { /** * Dataframe output by the model's `transform` method. diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 055c1c4d4228e..43016a32e570b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -19,11 +19,11 @@ package org.apache.spark.ml.param import java.lang.reflect.Modifier import java.util.{List => JList} -import java.util.NoSuchElementException import scala.annotation.varargs import scala.collection.mutable import scala.jdk.CollectionConverters._ +import scala.reflect.ClassTag import org.json4s._ import org.json4s.jackson.JsonMethods._ @@ -45,9 +45,14 @@ import org.apache.spark.util.ArrayImplicits._ * See [[ParamValidators]] for factory methods for common validation functions. * @tparam T param value type */ -class Param[T](val parent: String, val name: String, val doc: String, val isValid: T => Boolean) +class Param[T: ClassTag]( + val parent: String, val name: String, val doc: String, val isValid: T => Boolean) extends Serializable { + // Spark Connect ML needs T type information which has been erased when compiling, + // Use classTag to preserve the T type. + val paramValueClassTag = implicitly[ClassTag[T]] + def this(parent: Identifiable, name: String, doc: String, isValid: T => Boolean) = this(parent.uid, name, doc, isValid) diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/Summary.scala b/mllib/src/main/scala/org/apache/spark/ml/util/Summary.scala new file mode 100644 index 0000000000000..6205fea92ef83 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/util/Summary.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.util + +import org.apache.spark.annotation.Since + +/** + * Trait for the Summary + * All the summaries should extend from this Summary in order to + * support connect. + */ +@Since("4.0.0") +private[spark] trait Summary diff --git a/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java b/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java index 1ad5f7a442daa..b3993c453e91f 100644 --- a/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java +++ b/mllib/src/test/java/org/apache/spark/ml/param/JavaTestParams.java @@ -21,6 +21,7 @@ import java.util.List; import org.apache.spark.ml.util.Identifiable$; +import scala.reflect.ClassTag; /** * A subclass of Params for testing. @@ -110,7 +111,7 @@ private void init() { ParamValidators.inRange(0.0, 1.0)); List validStrings = Arrays.asList("a", "b"); myStringParam_ = new Param<>(this, "myStringParam", "this is a string param", - ParamValidators.inArray(validStrings)); + ParamValidators.inArray(validStrings), ClassTag.apply(String.class)); myDoubleArrayParam_ = new DoubleArrayParam(this, "myDoubleArrayParam", "this is a double param"); diff --git a/pom.xml b/pom.xml index c64e01d4dcb7a..41a5ce0c5592c 100644 --- a/pom.xml +++ b/pom.xml @@ -125,7 +125,7 @@ 3.4.1 - 4.29.1 + 4.29.3 3.11.4 3.9.3 5.7.1 @@ -145,7 +145,7 @@ 4.0.1 0.10.0 - 2.5.2 + 2.5.3 2.0.8 - - - - diff --git a/python/docs/source/conf.py b/python/docs/source/conf.py index 5640ba151176d..20c13cd768deb 100644 --- a/python/docs/source/conf.py +++ b/python/docs/source/conf.py @@ -188,19 +188,19 @@ # a list of builtin themes. html_theme = 'pydata_sphinx_theme' -html_context = { - # When releasing a new Spark version, please update the file - # "site/static/versions.json" under the code repository "spark-website" - # (item should be added in order), and also set the local environment - # variable "RELEASE_VERSION". - "switcher_json_url": "https://spark.apache.org/static/versions.json", - "switcher_template_url": "https://spark.apache.org/docs/{version}/api/python/index.html", -} - # Theme options are theme-specific and customize the look and feel of a theme # further. For a list of options available for each theme, see the # documentation. html_theme_options = { + "check_switcher": False, + "switcher": { + # When releasing a new Spark version, please update the file + # "site/static/versions.json" under the code repository "spark-website" + # (item should be added in order), and also set the local environment + # variable "RELEASE_VERSION". + "json_url": "https://spark.apache.org/static/versions.json", + "version_match": release, + }, "header_links_before_dropdown": 6, "navbar_end": ["version-switcher", "theme-switcher", "navbar-icon-links"], "footer_start": ["spark_footer", "sphinx-version"], diff --git a/python/docs/source/reference/pyspark.sql/variant_val.rst b/python/docs/source/reference/pyspark.sql/variant_val.rst index 8630ae8aace14..883b4c8fdc3d5 100644 --- a/python/docs/source/reference/pyspark.sql/variant_val.rst +++ b/python/docs/source/reference/pyspark.sql/variant_val.rst @@ -26,3 +26,4 @@ VariantVal VariantVal.toPython VariantVal.toJson + VariantVal.parseJson diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index b7ae794d3d613..e003ba43ec7c8 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -62,6 +62,7 @@ HasSolver, HasParallelism, ) +from pyspark.ml.remote.util import try_remote_attribute_relation from pyspark.ml.tree import ( _DecisionTreeModel, _DecisionTreeParams, @@ -336,6 +337,7 @@ class _ClassificationSummary(JavaWrapper): @property @since("3.1.0") + @try_remote_attribute_relation def predictions(self) -> DataFrame: """ Dataframe outputted by the model's `transform` method. @@ -521,6 +523,7 @@ def scoreCol(self) -> str: return self._call_java("scoreCol") @property + @try_remote_attribute_relation def roc(self) -> DataFrame: """ Returns the receiver operating characteristic (ROC) curve, @@ -546,6 +549,7 @@ def areaUnderROC(self) -> float: @property @since("3.1.0") + @try_remote_attribute_relation def pr(self) -> DataFrame: """ Returns the precision-recall curve, which is a Dataframe @@ -556,6 +560,7 @@ def pr(self) -> DataFrame: @property @since("3.1.0") + @try_remote_attribute_relation def fMeasureByThreshold(self) -> DataFrame: """ Returns a dataframe with two fields (threshold, F-Measure) curve @@ -565,6 +570,7 @@ def fMeasureByThreshold(self) -> DataFrame: @property @since("3.1.0") + @try_remote_attribute_relation def precisionByThreshold(self) -> DataFrame: """ Returns a dataframe with two fields (threshold, precision) curve. @@ -575,6 +581,7 @@ def precisionByThreshold(self) -> DataFrame: @property @since("3.1.0") + @try_remote_attribute_relation def recallByThreshold(self) -> DataFrame: """ Returns a dataframe with two fields (threshold, recall) curve. diff --git a/python/pyspark/ml/remote/__init__.py b/python/pyspark/ml/remote/__init__.py new file mode 100644 index 0000000000000..cce3acad34a49 --- /dev/null +++ b/python/pyspark/ml/remote/__init__.py @@ -0,0 +1,16 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# diff --git a/python/pyspark/ml/remote/proto.py b/python/pyspark/ml/remote/proto.py new file mode 100644 index 0000000000000..3a81e74b6aec3 --- /dev/null +++ b/python/pyspark/ml/remote/proto.py @@ -0,0 +1,76 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +from typing import Optional, TYPE_CHECKING, List + +import pyspark.sql.connect.proto as pb2 +from pyspark.sql.connect.plan import LogicalPlan + +if TYPE_CHECKING: + from pyspark.sql.connect.client import SparkConnectClient + + +class TransformerRelation(LogicalPlan): + """A logical plan for transforming of a transformer which could be a cached model + or a non-model transformer like VectorAssembler.""" + + def __init__( + self, + child: Optional["LogicalPlan"], + name: str, + ml_params: pb2.MlParams, + uid: str = "", + is_model: bool = True, + ) -> None: + super().__init__(child) + self._name = name + self._ml_params = ml_params + self._uid = uid + self._is_model = is_model + + def plan(self, session: "SparkConnectClient") -> pb2.Relation: + assert self._child is not None + plan = self._create_proto_relation() + plan.ml_relation.transform.input.CopyFrom(self._child.plan(session)) + + if self._is_model: + plan.ml_relation.transform.obj_ref.CopyFrom(pb2.ObjectRef(id=self._name)) + else: + plan.ml_relation.transform.transformer.CopyFrom( + pb2.MlOperator(name=self._name, uid=self._uid, type=pb2.MlOperator.TRANSFORMER) + ) + + if self._ml_params is not None: + plan.ml_relation.transform.params.CopyFrom(self._ml_params) + + return plan + + +class AttributeRelation(LogicalPlan): + """A logical plan used in ML to represent an attribute of an instance, which + could be a model or a summary. This attribute returns a DataFrame. + """ + + def __init__(self, ref_id: str, methods: List[pb2.Fetch.Method]) -> None: + super().__init__(None) + self._ref_id = ref_id + self._methods = methods + + def plan(self, session: "SparkConnectClient") -> pb2.Relation: + plan = self._create_proto_relation() + plan.ml_relation.fetch.obj_ref.CopyFrom(pb2.ObjectRef(id=self._ref_id)) + plan.ml_relation.fetch.methods.extend(self._methods) + return plan diff --git a/python/pyspark/ml/remote/readwrite.py b/python/pyspark/ml/remote/readwrite.py new file mode 100644 index 0000000000000..9149ab3bfd454 --- /dev/null +++ b/python/pyspark/ml/remote/readwrite.py @@ -0,0 +1,134 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from typing import cast, Type, TYPE_CHECKING + +import pyspark.sql.connect.proto as pb2 +from pyspark.ml.remote.serialize import serialize_ml_params, deserialize, deserialize_param +from pyspark.ml.util import MLWriter, MLReader, RL +from pyspark.ml.wrapper import JavaWrapper + +if TYPE_CHECKING: + from pyspark.ml.util import JavaMLReadable, JavaMLWritable + from pyspark.core.context import SparkContext + + +class RemoteMLWriter(MLWriter): + def __init__(self, instance: "JavaMLWritable") -> None: + super().__init__() + self._instance = instance + + @property + def sc(self) -> "SparkContext": + raise RuntimeError("Accessing SparkContext is not supported on Connect") + + def save(self, path: str) -> None: + from pyspark.ml.wrapper import JavaModel, JavaEstimator + from pyspark.sql.connect.session import SparkSession + + session = SparkSession.getActiveSession() + assert session is not None + + # Spark Connect ML is built on scala Spark.ML, that means we're only + # supporting JavaModel or JavaEstimator or JavaEvaluator + if isinstance(self._instance, JavaModel): + model = cast("JavaModel", self._instance) + params = serialize_ml_params(model, session.client) + assert isinstance(model._java_obj, str) + writer = pb2.MlCommand.Write( + obj_ref=pb2.ObjectRef(id=model._java_obj), + params=params, + path=path, + should_overwrite=self.shouldOverwrite, + options=self.optionMap, + ) + elif isinstance(self._instance, JavaEstimator): + estimator = cast("JavaEstimator", self._instance) + params = serialize_ml_params(estimator, session.client) + assert isinstance(estimator._java_obj, str) + writer = pb2.MlCommand.Write( + operator=pb2.MlOperator( + name=estimator._java_obj, uid=estimator.uid, type=pb2.MlOperator.ESTIMATOR + ), + params=params, + path=path, + should_overwrite=self.shouldOverwrite, + options=self.optionMap, + ) + else: + raise NotImplementedError(f"Unsupported writing for {self._instance}") + + command = pb2.Command() + command.ml_command.write.CopyFrom(writer) + session.client.execute_command(command) + + +class RemoteMLReader(MLReader[RL]): + def __init__(self, clazz: Type["JavaMLReadable[RL]"]) -> None: + super().__init__() + self._clazz = clazz + + def load(self, path: str) -> RL: + from pyspark.sql.connect.session import SparkSession + from pyspark.ml.wrapper import JavaModel, JavaEstimator + + session = SparkSession.getActiveSession() + assert session is not None + # to get the java corresponding qualified class name + java_qualified_class_name = ( + self._clazz.__module__.replace("pyspark", "org.apache.spark") + + "." + + self._clazz.__name__ + ) + + if issubclass(self._clazz, JavaModel): + ml_type = pb2.MlOperator.MODEL + elif issubclass(self._clazz, JavaEstimator): + ml_type = pb2.MlOperator.ESTIMATOR + else: + raise ValueError(f"Unsupported reading for {java_qualified_class_name}") + + command = pb2.Command() + command.ml_command.read.CopyFrom( + pb2.MlCommand.Read( + operator=pb2.MlOperator(name=java_qualified_class_name, type=ml_type), path=path + ) + ) + (_, properties, _) = session.client.execute_command(command) + result = deserialize(properties) + + # Get the python type + def _get_class() -> Type[RL]: + parts = (self._clazz.__module__ + "." + self._clazz.__name__).split(".") + module = ".".join(parts[:-1]) + m = __import__(module, fromlist=[parts[-1]]) + return getattr(m, parts[-1]) + + py_type = _get_class() + # It must be JavaWrapper, since we're passing the string to the _java_obj + if issubclass(py_type, JavaWrapper): + if ml_type == pb2.MlOperator.MODEL: + session.client.add_ml_cache(result.obj_ref.id) + instance = py_type(result.obj_ref.id) + else: + instance = py_type() + instance._resetUid(result.uid) + params = {k: deserialize_param(v) for k, v in result.params.params.items()} + instance._set(**params) + return instance + else: + raise RuntimeError(f"Unsupported class {self._clazz}") diff --git a/python/pyspark/ml/remote/serialize.py b/python/pyspark/ml/remote/serialize.py new file mode 100644 index 0000000000000..69e3af1f4c787 --- /dev/null +++ b/python/pyspark/ml/remote/serialize.py @@ -0,0 +1,132 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +from typing import Any, List, TYPE_CHECKING, Mapping, Dict + +import pyspark.sql.connect.proto as pb2 +from pyspark.ml.linalg import ( + Vectors, + Matrices, + DenseVector, + SparseVector, + DenseMatrix, + SparseMatrix, +) +from pyspark.sql.connect.expressions import LiteralExpression + +if TYPE_CHECKING: + from pyspark.sql.connect.client import SparkConnectClient + from pyspark.ml.param import Params + + +def serialize_param(value: Any, client: "SparkConnectClient") -> pb2.Param: + if isinstance(value, DenseVector): + return pb2.Param(vector=pb2.Vector(dense=pb2.Vector.Dense(value=value.values.tolist()))) + elif isinstance(value, SparseVector): + return pb2.Param( + vector=pb2.Vector( + sparse=pb2.Vector.Sparse( + size=value.size, index=value.indices.tolist(), value=value.values.tolist() + ) + ) + ) + elif isinstance(value, DenseMatrix): + return pb2.Param( + matrix=pb2.Matrix( + dense=pb2.Matrix.Dense( + num_rows=value.numRows, num_cols=value.numCols, value=value.values.tolist() + ) + ) + ) + elif isinstance(value, SparseMatrix): + return pb2.Param( + matrix=pb2.Matrix( + sparse=pb2.Matrix.Sparse( + num_rows=value.numRows, + num_cols=value.numCols, + colptr=value.colPtrs.tolist(), + row_index=value.rowIndices.tolist(), + value=value.values.tolist(), + ) + ) + ) + else: + literal = LiteralExpression._from_value(value).to_plan(client).literal + return pb2.Param(literal=literal) + + +def serialize(client: "SparkConnectClient", *args: Any) -> List[Any]: + from pyspark.sql.connect.dataframe import DataFrame as ConnectDataFrame + + result = [] + for arg in args: + if isinstance(arg, ConnectDataFrame): + result.append(pb2.Fetch.Method.Args(input=arg._plan.plan(client))) + else: + result.append(pb2.Fetch.Method.Args(param=serialize_param(arg, client))) + return result + + +def deserialize_param(param: pb2.Param) -> Any: + if param.HasField("literal"): + return LiteralExpression._to_value(param.literal) + if param.HasField("vector"): + vector = param.vector + if vector.HasField("dense"): + return Vectors.dense(vector.dense.value) + elif vector.HasField("sparse"): + return Vectors.sparse(vector.sparse.size, vector.sparse.index, vector.sparse.value) + else: + raise ValueError("Unsupported vector type") + if param.HasField("matrix"): + matrix = param.matrix + if matrix.HasField("dense"): + return DenseMatrix( + matrix.dense.num_rows, + matrix.dense.num_cols, + matrix.dense.value, + matrix.dense.is_transposed, + ) + elif matrix.HasField("sparse"): + return Matrices.sparse( + matrix.sparse.num_rows, + matrix.sparse.num_cols, + matrix.sparse.colptr, + matrix.sparse.row_index, + matrix.sparse.value, + ) + else: + raise ValueError("Unsupported matrix type") + + raise ValueError("Unsupported param type") + + +def deserialize(ml_command_result_properties: Dict[str, Any]) -> Any: + ml_command_result = ml_command_result_properties["ml_command_result"] + if ml_command_result.HasField("operator_info"): + return ml_command_result.operator_info + + if ml_command_result.HasField("param"): + return deserialize_param(ml_command_result.param) + + raise ValueError("Unsupported result type") + + +def serialize_ml_params(instance: "Params", client: "SparkConnectClient") -> pb2.MlParams: + params: Mapping[str, pb2.Param] = { + k.name: serialize_param(v, client) for k, v in instance._paramMap.items() + } + return pb2.MlParams(params=params) diff --git a/python/pyspark/ml/remote/util.py b/python/pyspark/ml/remote/util.py new file mode 100644 index 0000000000000..cb34dae165863 --- /dev/null +++ b/python/pyspark/ml/remote/util.py @@ -0,0 +1,293 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +import functools +import os +from typing import Any, cast, TypeVar, Callable, TYPE_CHECKING, Type, List, Tuple + +import pyspark.sql.connect.proto as pb2 +from pyspark.ml.remote.serialize import serialize_ml_params, serialize, deserialize +from pyspark.sql import is_remote + +if TYPE_CHECKING: + from pyspark.sql.connect.dataframe import DataFrame as ConnectDataFrame + from pyspark.ml.wrapper import JavaWrapper, JavaEstimator + from pyspark.ml.util import JavaMLReadable, JavaMLWritable + +FuncT = TypeVar("FuncT", bound=Callable[..., Any]) + + +def _extract_id_methods(obj_identifier: str) -> Tuple[List[pb2.Fetch.Method], str]: + """Extract the obj reference id and the methods. Eg, model.summary""" + method_chain = obj_identifier.split(".") + obj_ref = method_chain[0] + methods: List[pb2.Fetch.Method] = [] + if len(method_chain) > 1: + methods = [pb2.Fetch.Method(method=m) for m in method_chain[1:]] + return methods, obj_ref + + +def try_remote_intermediate_result(f: FuncT) -> FuncT: + """Mark the function/property that returns the intermediate result of the remote call. + Eg, model.summary""" + + @functools.wraps(f) + def wrapped(self: "JavaWrapper") -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + return f"{self._java_obj}.{f.__name__}" + else: + return f(self) + + return cast(FuncT, wrapped) + + +def try_remote_attribute_relation(f: FuncT) -> FuncT: + """Mark the function/property that returns a Relation. + Eg, model.summary.roc""" + + @functools.wraps(f) + def wrapped(self: "JavaWrapper", *args: Any, **kwargs: Any) -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + # The attribute returns a dataframe, we need to wrap it + # in the AttributeRelation + from pyspark.ml.remote.proto import AttributeRelation + from pyspark.sql.connect.session import SparkSession + from pyspark.sql.connect.dataframe import DataFrame as ConnectDataFrame + + session = SparkSession.getActiveSession() + assert session is not None + + assert isinstance(self._java_obj, str) + + methods, obj_ref = _extract_id_methods(self._java_obj) + methods.append( + pb2.Fetch.Method(method=f.__name__, args=serialize(session.client, *args)) + ) + plan = AttributeRelation(obj_ref, methods) + return ConnectDataFrame(plan, session) + else: + return f(self, *args, **kwargs) + + return cast(FuncT, wrapped) + + +def try_remote_fit(f: FuncT) -> FuncT: + """Mark the function that fits a model.""" + + @functools.wraps(f) + def wrapped(self: "JavaEstimator", dataset: "ConnectDataFrame") -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + client = dataset.sparkSession.client + input = dataset._plan.plan(client) + assert isinstance(self._java_obj, str) + estimator = pb2.MlOperator( + name=self._java_obj, uid=self.uid, type=pb2.MlOperator.ESTIMATOR + ) + command = pb2.Command() + command.ml_command.fit.CopyFrom( + pb2.MlCommand.Fit( + estimator=estimator, + params=serialize_ml_params(self, client), + dataset=input, + ) + ) + (_, properties, _) = client.execute_command(command) + model_info = deserialize(properties) + client.add_ml_cache(model_info.obj_ref.id) + return model_info.obj_ref.id + else: + return f(self, dataset) + + return cast(FuncT, wrapped) + + +def try_remote_transform_relation(f: FuncT) -> FuncT: + """Mark the function/property that returns a relation for model transform.""" + + @functools.wraps(f) + def wrapped(self: "JavaWrapper", dataset: "ConnectDataFrame") -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + from pyspark.ml import Model, Transformer + from pyspark.sql.connect.session import SparkSession + from pyspark.sql.connect.dataframe import DataFrame as ConnectDataFrame + + session = SparkSession.getActiveSession() + assert session is not None + # Model is also a Transformer, so we much match Model first + if isinstance(self, Model): + params = serialize_ml_params(self, session.client) + from pyspark.ml.remote.proto import TransformerRelation + + assert isinstance(self._java_obj, str) + return ConnectDataFrame( + TransformerRelation( + child=dataset._plan, name=self._java_obj, ml_params=params, is_model=True + ), + session, + ) + elif isinstance(self, Transformer): + params = serialize_ml_params(self, session.client) + from pyspark.ml.remote.proto import TransformerRelation + + assert isinstance(self._java_obj, str) + return ConnectDataFrame( + TransformerRelation( + child=dataset._plan, + name=self._java_obj, + ml_params=params, + uid=self.uid, + is_model=False, + ), + session, + ) + else: + raise RuntimeError(f"Unsupported {self}") + else: + return f(self, dataset) + + return cast(FuncT, wrapped) + + +def try_remote_call(f: FuncT) -> FuncT: + """Mark the function/property for the remote call. + Eg, model.coefficients""" + + @functools.wraps(f) + def wrapped(self: "JavaWrapper", name: str, *args: Any) -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + """Launch a remote call if possible""" + from pyspark.sql.connect.session import SparkSession + + session = SparkSession.getActiveSession() + assert session is not None + assert isinstance(self._java_obj, str) + methods, obj_ref = _extract_id_methods(self._java_obj) + methods.append(pb2.Fetch.Method(method=name, args=serialize(session.client, *args))) + command = pb2.Command() + command.ml_command.fetch.CopyFrom( + pb2.Fetch(obj_ref=pb2.ObjectRef(id=obj_ref), methods=methods) + ) + (_, properties, _) = session.client.execute_command(command) + ml_command_result = properties["ml_command_result"] + if ml_command_result.HasField("summary"): + summary = ml_command_result.summary + session.client.add_ml_cache(summary) + return summary + else: + return deserialize(properties) + else: + return f(self, name, *args) + + return cast(FuncT, wrapped) + + +def try_remote_del(f: FuncT) -> FuncT: + """Mark the function/property to delete a model on the server side.""" + + @functools.wraps(f) + def wrapped(self: "JavaWrapper") -> Any: + try: + in_remote = is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ + except Exception: + return + + if in_remote: + # Delete the model if possible + model_id = self._java_obj + if model_id is not None and "." not in model_id: + try: + from pyspark.sql.connect.session import SparkSession + + session = SparkSession.getActiveSession() + if session is not None: + session.client.remove_ml_cache(model_id) + return + except Exception: + # SparkSession's down. + return + else: + return f(self) + + return cast(FuncT, wrapped) + + +def try_remote_return_java_class(f: FuncT) -> FuncT: + """Mark the function/property that returns none.""" + + @functools.wraps(f) + def wrapped(java_class: str, *args: Any) -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + return java_class + else: + return f(java_class, *args) + + return cast(FuncT, wrapped) + + +def try_remote_write(f: FuncT) -> FuncT: + """Mark the function that write an estimator/model or evaluator""" + + @functools.wraps(f) + def wrapped(self: "JavaMLWritable") -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + from pyspark.ml.remote.readwrite import RemoteMLWriter + + return RemoteMLWriter(self) + else: + return f(self) + + return cast(FuncT, wrapped) + + +def try_remote_read(f: FuncT) -> FuncT: + """Mark the function to read an estimator/model or evaluator""" + + @functools.wraps(f) + def wrapped(cls: Type["JavaMLReadable"]) -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + from pyspark.ml.remote.readwrite import RemoteMLReader + + return RemoteMLReader(cls) + else: + return f(cls) + + return cast(FuncT, wrapped) + + +def try_remote_intercept(f: FuncT) -> FuncT: + """Mark the function/property that returns none.""" + + @functools.wraps(f) + def wrapped(java_class: str, *args: Any) -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + return None + else: + return f(java_class, *args) + + return cast(FuncT, wrapped) + + +def try_remote_not_supporting(f: FuncT) -> FuncT: + """Mark the function/property that has not been supported yet""" + + @functools.wraps(f) + def wrapped(*args: Any) -> Any: + if is_remote() and "PYSPARK_NO_NAMESPACE_SHARE" not in os.environ: + raise NotImplementedError("") + else: + return f(*args) + + return cast(FuncT, wrapped) diff --git a/python/pyspark/ml/tests/connect/test_connect_spark_ml_classification.py b/python/pyspark/ml/tests/connect/test_connect_spark_ml_classification.py new file mode 100644 index 0000000000000..2000a38d9e616 --- /dev/null +++ b/python/pyspark/ml/tests/connect/test_connect_spark_ml_classification.py @@ -0,0 +1,49 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import os +import unittest + +from pyspark.ml.tests.test_classification import ClassificationTestsMixin +from pyspark.sql import SparkSession + + +class ClassificationTestsOnConnect(ClassificationTestsMixin, unittest.TestCase): + def setUp(self) -> None: + self.spark = SparkSession.builder.remote( + os.environ.get("SPARK_CONNECT_TESTING_REMOTE", "local[2]") + ).getOrCreate() + + def test_assert_remote_mode(self): + from pyspark.sql import is_remote + + self.assertTrue(is_remote()) + + def tearDown(self) -> None: + self.spark.stop() + + +if __name__ == "__main__": + from pyspark.ml.tests.connect.test_connect_spark_ml_classification import * # noqa: F401 + + try: + import xmlrunner # type: ignore[import] + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/ml/tests/test_algorithms.py b/python/pyspark/ml/tests/test_algorithms.py index eeb342c4238dd..d0e2600a9a8b3 100644 --- a/python/pyspark/ml/tests/test_algorithms.py +++ b/python/pyspark/ml/tests/test_algorithms.py @@ -29,93 +29,13 @@ ) from pyspark.ml.clustering import DistributedLDAModel, KMeans, LocalLDAModel, LDA, LDAModel from pyspark.ml.fpm import FPGrowth -from pyspark.ml.linalg import Matrices, Vectors, DenseVector +from pyspark.ml.linalg import Vectors, DenseVector from pyspark.ml.recommendation import ALS from pyspark.ml.regression import GeneralizedLinearRegression, LinearRegression from pyspark.sql import Row from pyspark.testing.mlutils import SparkSessionTestCase -class LogisticRegressionTest(SparkSessionTestCase): - def test_binomial_logistic_regression_with_bound(self): - df = self.spark.createDataFrame( - [ - (1.0, 1.0, Vectors.dense(0.0, 5.0)), - (0.0, 2.0, Vectors.dense(1.0, 2.0)), - (1.0, 3.0, Vectors.dense(2.0, 1.0)), - (0.0, 4.0, Vectors.dense(3.0, 3.0)), - ], - ["label", "weight", "features"], - ) - - lor = LogisticRegression( - regParam=0.01, - weightCol="weight", - lowerBoundsOnCoefficients=Matrices.dense(1, 2, [-1.0, -1.0]), - upperBoundsOnIntercepts=Vectors.dense(0.0), - ) - model = lor.fit(df) - self.assertTrue(np.allclose(model.coefficients.toArray(), [-0.2944, -0.0484], atol=1e-4)) - self.assertTrue(np.isclose(model.intercept, 0.0, atol=1e-4)) - - def test_multinomial_logistic_regression_with_bound(self): - data_path = "data/mllib/sample_multiclass_classification_data.txt" - df = self.spark.read.format("libsvm").load(data_path) - - lor = LogisticRegression( - regParam=0.01, - lowerBoundsOnCoefficients=Matrices.dense(3, 4, range(12)), - upperBoundsOnIntercepts=Vectors.dense(0.0, 0.0, 0.0), - ) - model = lor.fit(df) - expected = [ - [4.593, 4.5516, 9.0099, 12.2904], - [1.0, 8.1093, 7.0, 10.0], - [3.041, 5.0, 8.0, 11.0], - ] - for i in range(0, len(expected)): - self.assertTrue( - np.allclose(model.coefficientMatrix.toArray()[i], expected[i], atol=1e-4) - ) - self.assertTrue( - np.allclose(model.interceptVector.toArray(), [-0.9057, -1.1392, -0.0033], atol=1e-4) - ) - - def test_logistic_regression_with_threshold(self): - df = self.spark.createDataFrame( - [ - (1.0, 1.0, Vectors.dense(0.0, 5.0)), - (0.0, 2.0, Vectors.dense(1.0, 2.0)), - (1.0, 3.0, Vectors.dense(2.0, 1.0)), - (0.0, 4.0, Vectors.dense(3.0, 3.0)), - ], - ["label", "weight", "features"], - ) - - lor = LogisticRegression(weightCol="weight") - model = lor.fit(df) - - # status changes 1 - for t in [0.0, 0.1, 0.2, 0.5, 1.0]: - model.setThreshold(t).transform(df) - - # status changes 2 - [model.setThreshold(t).predict(Vectors.dense(0.0, 5.0)) for t in [0.0, 0.1, 0.2, 0.5, 1.0]] - - self.assertEqual( - [row.prediction for row in model.setThreshold(0.0).transform(df).collect()], - [1.0, 1.0, 1.0, 1.0], - ) - self.assertEqual( - [row.prediction for row in model.setThreshold(0.5).transform(df).collect()], - [0.0, 1.0, 1.0, 0.0], - ) - self.assertEqual( - [row.prediction for row in model.setThreshold(1.0).transform(df).collect()], - [0.0, 0.0, 0.0, 0.0], - ) - - class MultilayerPerceptronClassifierTest(SparkSessionTestCase): def test_raw_and_probability_prediction(self): data_path = "data/mllib/sample_multiclass_classification_data.txt" diff --git a/python/pyspark/ml/tests/test_classification.py b/python/pyspark/ml/tests/test_classification.py new file mode 100644 index 0000000000000..ee72e0394e3a0 --- /dev/null +++ b/python/pyspark/ml/tests/test_classification.py @@ -0,0 +1,304 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import os +import tempfile +import unittest +from shutil import rmtree + +import numpy as np + +from pyspark.ml.linalg import Vectors, Matrices +from pyspark.sql import SparkSession, DataFrame +from pyspark.ml.classification import ( + LogisticRegression, + LogisticRegressionModel, + LogisticRegressionSummary, + BinaryLogisticRegressionSummary, +) + + +class ClassificationTestsMixin: + def test_binomial_logistic_regression_with_bound(self): + df = self.spark.createDataFrame( + [ + (1.0, 1.0, Vectors.dense(0.0, 5.0)), + (0.0, 2.0, Vectors.dense(1.0, 2.0)), + (1.0, 3.0, Vectors.dense(2.0, 1.0)), + (0.0, 4.0, Vectors.dense(3.0, 3.0)), + ], + ["label", "weight", "features"], + ) + + lor = LogisticRegression( + regParam=0.01, + weightCol="weight", + lowerBoundsOnCoefficients=Matrices.dense(1, 2, [-1.0, -1.0]), + upperBoundsOnIntercepts=Vectors.dense(0.0), + ) + lor_model = lor.fit(df) + + def check_result(model: LogisticRegressionModel) -> None: + self.assertTrue( + np.allclose(model.coefficients.toArray(), [-0.2944, -0.0484], atol=1e-4) + ) + self.assertTrue(np.isclose(model.intercept, 0.0, atol=1e-4)) + + check_result(lor_model) + + # Model save + with tempfile.TemporaryDirectory(prefix="model_save") as tmp_dir: + local_path = os.path.join(tmp_dir, "model") + lor_model.write().save(local_path) + loaded_model = LogisticRegressionModel.load(local_path) + check_result(loaded_model) + + def test_multinomial_logistic_regression_with_bound(self): + data_path = "data/mllib/sample_multiclass_classification_data.txt" + df = self.spark.read.format("libsvm").load(data_path) + + lor = LogisticRegression( + regParam=0.01, + lowerBoundsOnCoefficients=Matrices.dense(3, 4, range(12)), + upperBoundsOnIntercepts=Vectors.dense(0.0, 0.0, 0.0), + ) + lor_model = lor.fit(df) + + def check_result(model: LogisticRegressionModel) -> None: + expected = [ + [4.593, 4.5516, 9.0099, 12.2904], + [1.0, 8.1093, 7.0, 10.0], + [3.041, 5.0, 8.0, 11.0], + ] + for i in range(0, len(expected)): + self.assertTrue( + np.allclose(model.coefficientMatrix.toArray()[i], expected[i], atol=1e-4) + ) + self.assertTrue( + np.allclose(model.interceptVector.toArray(), [-0.9057, -1.1392, -0.0033], atol=1e-4) + ) + + check_result(lor_model) + + # Model save + with tempfile.TemporaryDirectory(prefix="model_save") as tmp_dir: + local_path = os.path.join(tmp_dir, "model") + lor_model.write().save(local_path) + loaded_model = LogisticRegressionModel.load(local_path) + check_result(loaded_model) + + def test_logistic_regression_with_threshold(self): + df = self.spark.createDataFrame( + [ + (1.0, 1.0, Vectors.dense(0.0, 5.0)), + (0.0, 2.0, Vectors.dense(1.0, 2.0)), + (1.0, 3.0, Vectors.dense(2.0, 1.0)), + (0.0, 4.0, Vectors.dense(3.0, 3.0)), + ], + ["label", "weight", "features"], + ) + + lor = LogisticRegression(weightCol="weight") + model = lor.fit(df) + + # status changes 1 + for t in [0.0, 0.1, 0.2, 0.5, 1.0]: + model.setThreshold(t).transform(df) + + # status changes 2 + [model.setThreshold(t).predict(Vectors.dense(0.0, 5.0)) for t in [0.0, 0.1, 0.2, 0.5, 1.0]] + + self.assertEqual( + [row.prediction for row in model.setThreshold(0.0).transform(df).collect()], + [1.0, 1.0, 1.0, 1.0], + ) + self.assertEqual( + [row.prediction for row in model.setThreshold(0.5).transform(df).collect()], + [0.0, 1.0, 1.0, 0.0], + ) + self.assertEqual( + [row.prediction for row in model.setThreshold(1.0).transform(df).collect()], + [0.0, 0.0, 0.0, 0.0], + ) + + def test_binary_logistic_regression_summary(self): + df = self.spark.createDataFrame( + [(1.0, 2.0, Vectors.dense(1.0)), (0.0, 2.0, Vectors.sparse(1, [], []))], + ["label", "weight", "features"], + ) + lr = LogisticRegression(maxIter=5, regParam=0.01, weightCol="weight", fitIntercept=False) + model = lr.fit(df) + self.assertTrue(model.hasSummary) + s = model.summary + # test that api is callable and returns expected types + self.assertTrue(isinstance(s.predictions, DataFrame)) + self.assertEqual(s.probabilityCol, "probability") + self.assertEqual(s.labelCol, "label") + self.assertEqual(s.featuresCol, "features") + self.assertEqual(s.predictionCol, "prediction") + objHist = s.objectiveHistory + self.assertTrue(isinstance(objHist, list) and isinstance(objHist[0], float)) + self.assertGreater(s.totalIterations, 0) + self.assertTrue(isinstance(s.labels, list)) + self.assertTrue(isinstance(s.truePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.falsePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.precisionByLabel, list)) + self.assertTrue(isinstance(s.recallByLabel, list)) + self.assertTrue(isinstance(s.fMeasureByLabel(), list)) + self.assertTrue(isinstance(s.fMeasureByLabel(1.0), list)) + self.assertTrue(isinstance(s.roc, DataFrame)) + self.assertAlmostEqual(s.areaUnderROC, 1.0, 2) + self.assertTrue(isinstance(s.pr, DataFrame)) + self.assertTrue(isinstance(s.fMeasureByThreshold, DataFrame)) + self.assertTrue(isinstance(s.precisionByThreshold, DataFrame)) + self.assertTrue(isinstance(s.recallByThreshold, DataFrame)) + self.assertAlmostEqual(s.accuracy, 1.0, 2) + self.assertAlmostEqual(s.weightedTruePositiveRate, 1.0, 2) + self.assertAlmostEqual(s.weightedFalsePositiveRate, 0.0, 2) + self.assertAlmostEqual(s.weightedRecall, 1.0, 2) + self.assertAlmostEqual(s.weightedPrecision, 1.0, 2) + self.assertAlmostEqual(s.weightedFMeasure(), 1.0, 2) + self.assertAlmostEqual(s.weightedFMeasure(1.0), 1.0, 2) + + # test evaluation (with training dataset) produces a summary with same values + # one check is enough to verify a summary is returned, Scala version runs full test + sameSummary = model.evaluate(df) + self.assertTrue(isinstance(sameSummary, BinaryLogisticRegressionSummary)) + self.assertAlmostEqual(sameSummary.areaUnderROC, s.areaUnderROC) + self.assertEqual(sorted(sameSummary.predictions.collect()), sorted(s.predictions.collect())) + + def test_multiclass_logistic_regression_summary(self): + df = self.spark.createDataFrame( + [ + (1.0, 2.0, Vectors.dense(1.0)), + (0.0, 2.0, Vectors.sparse(1, [], [])), + (2.0, 2.0, Vectors.dense(2.0)), + (2.0, 2.0, Vectors.dense(1.9)), + ], + ["label", "weight", "features"], + ) + lr = LogisticRegression(maxIter=5, regParam=0.01, weightCol="weight", fitIntercept=False) + model = lr.fit(df) + self.assertTrue(model.hasSummary) + s = model.summary + # test that api is callable and returns expected types + self.assertTrue(isinstance(s.predictions, DataFrame)) + self.assertEqual(s.probabilityCol, "probability") + self.assertEqual(s.labelCol, "label") + self.assertEqual(s.featuresCol, "features") + self.assertEqual(s.predictionCol, "prediction") + objHist = s.objectiveHistory + self.assertTrue(isinstance(objHist, list) and isinstance(objHist[0], float)) + self.assertGreater(s.totalIterations, 0) + self.assertTrue(isinstance(s.labels, list)) + self.assertTrue(isinstance(s.truePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.falsePositiveRateByLabel, list)) + self.assertTrue(isinstance(s.precisionByLabel, list)) + self.assertTrue(isinstance(s.recallByLabel, list)) + self.assertTrue(isinstance(s.fMeasureByLabel(), list)) + self.assertTrue(isinstance(s.fMeasureByLabel(1.0), list)) + self.assertAlmostEqual(s.accuracy, 0.75, 2) + self.assertAlmostEqual(s.weightedTruePositiveRate, 0.75, 2) + self.assertAlmostEqual(s.weightedFalsePositiveRate, 0.25, 2) + self.assertAlmostEqual(s.weightedRecall, 0.75, 2) + self.assertAlmostEqual(s.weightedPrecision, 0.583, 2) + self.assertAlmostEqual(s.weightedFMeasure(), 0.65, 2) + self.assertAlmostEqual(s.weightedFMeasure(1.0), 0.65, 2) + + # test evaluation (with training dataset) produces a summary with same values + # one check is enough to verify a summary is returned, Scala version runs full test + sameSummary = model.evaluate(df) + self.assertTrue(isinstance(sameSummary, LogisticRegressionSummary)) + self.assertFalse(isinstance(sameSummary, BinaryLogisticRegressionSummary)) + self.assertAlmostEqual(sameSummary.accuracy, s.accuracy) + + # We can't use sorted(s.predictions.collect()), since the DenseVector doesn't support "<" + self.assertEqual( + sameSummary.predictions.coalesce(1).sort("label", "weight", "prediction").collect(), + s.predictions.coalesce(1).sort("label", "weight", "prediction").collect(), + ) + + def test_logistic_regression(self): + # test sparse/dense vector and matrix + lower_intercepts = Vectors.dense([1, 2, 3, 4]) + upper_intercepts = Vectors.sparse(4, [(1, 1.0), (3, 5.5)]) + lower_coefficients = Matrices.dense(3, 2, [0, 1, 4, 5, 9, 10]) + upper_coefficients = Matrices.sparse(1, 1, [0, 1], [0], [2.0]) + + lr = LogisticRegression( + maxIter=1, + lowerBoundsOnIntercepts=lower_intercepts, + upperBoundsOnIntercepts=upper_intercepts, + lowerBoundsOnCoefficients=lower_coefficients, + upperBoundsOnCoefficients=upper_coefficients, + ) + path = tempfile.mkdtemp() + lr_path = path + "/logreg" + lr.save(lr_path) + lr2 = LogisticRegression.load(lr_path) + self.assertEqual( + lr2.uid, + lr2.maxIter.parent, + "Loaded LogisticRegression instance uid (%s) " + "did not match Param's uid (%s)" % (lr2.uid, lr2.maxIter.parent), + ) + self.assertEqual( + lr._defaultParamMap[lr.maxIter], + lr2._defaultParamMap[lr2.maxIter], + "Loaded LogisticRegression instance default params did not match " + + "original defaults", + ) + self.assertEqual( + lr.getLowerBoundsOnIntercepts(), + lr2.getLowerBoundsOnIntercepts(), + ) + self.assertEqual( + lr.getUpperBoundsOnIntercepts(), + lr2.getUpperBoundsOnIntercepts(), + ) + self.assertEqual( + lr.getLowerBoundsOnCoefficients(), + lr2.getLowerBoundsOnCoefficients(), + ) + self.assertEqual( + lr.getUpperBoundsOnCoefficients(), + lr2.getUpperBoundsOnCoefficients(), + ) + try: + rmtree(path) + except OSError: + pass + + +class ClassificationTests(ClassificationTestsMixin, unittest.TestCase): + def setUp(self) -> None: + self.spark = SparkSession.builder.master("local[4]").getOrCreate() + + def tearDown(self) -> None: + self.spark.stop() + + +if __name__ == "__main__": + from pyspark.ml.tests.test_classification import * # noqa: F401,F403 + + try: + import xmlrunner # type: ignore[import] + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/ml/tests/test_persistence.py b/python/pyspark/ml/tests/test_persistence.py index 406180d9a6391..481c2f236d46f 100644 --- a/python/pyspark/ml/tests/test_persistence.py +++ b/python/pyspark/ml/tests/test_persistence.py @@ -153,29 +153,6 @@ def test_linear_regression_pmml_basic(self): self.assertIn("Apache Spark", pmml_text) self.assertIn("PMML", pmml_text) - def test_logistic_regression(self): - lr = LogisticRegression(maxIter=1) - path = tempfile.mkdtemp() - lr_path = path + "/logreg" - lr.save(lr_path) - lr2 = LogisticRegression.load(lr_path) - self.assertEqual( - lr2.uid, - lr2.maxIter.parent, - "Loaded LogisticRegression instance uid (%s) " - "did not match Param's uid (%s)" % (lr2.uid, lr2.maxIter.parent), - ) - self.assertEqual( - lr._defaultParamMap[lr.maxIter], - lr2._defaultParamMap[lr2.maxIter], - "Loaded LogisticRegression instance default params did not match " - + "original defaults", - ) - try: - rmtree(path) - except OSError: - pass - def test_kmeans(self): kmeans = KMeans(k=2, seed=1) path = tempfile.mkdtemp() diff --git a/python/pyspark/ml/tests/test_training_summary.py b/python/pyspark/ml/tests/test_training_summary.py index 5704d7186734f..e1c8f4197e3c7 100644 --- a/python/pyspark/ml/tests/test_training_summary.py +++ b/python/pyspark/ml/tests/test_training_summary.py @@ -18,14 +18,11 @@ import unittest from pyspark.ml.classification import ( - BinaryLogisticRegressionSummary, BinaryRandomForestClassificationSummary, FMClassifier, FMClassificationSummary, LinearSVC, LinearSVCSummary, - LogisticRegression, - LogisticRegressionSummary, MultilayerPerceptronClassifier, MultilayerPerceptronClassificationSummary, RandomForestClassificationSummary, @@ -122,94 +119,6 @@ def test_glr_summary(self): sameSummary = model.evaluate(df) self.assertAlmostEqual(sameSummary.deviance, s.deviance) - def test_binary_logistic_regression_summary(self): - df = self.spark.createDataFrame( - [(1.0, 2.0, Vectors.dense(1.0)), (0.0, 2.0, Vectors.sparse(1, [], []))], - ["label", "weight", "features"], - ) - lr = LogisticRegression(maxIter=5, regParam=0.01, weightCol="weight", fitIntercept=False) - model = lr.fit(df) - self.assertTrue(model.hasSummary) - s = model.summary - # test that api is callable and returns expected types - self.assertTrue(isinstance(s.predictions, DataFrame)) - self.assertEqual(s.probabilityCol, "probability") - self.assertEqual(s.labelCol, "label") - self.assertEqual(s.featuresCol, "features") - self.assertEqual(s.predictionCol, "prediction") - objHist = s.objectiveHistory - self.assertTrue(isinstance(objHist, list) and isinstance(objHist[0], float)) - self.assertGreater(s.totalIterations, 0) - self.assertTrue(isinstance(s.labels, list)) - self.assertTrue(isinstance(s.truePositiveRateByLabel, list)) - self.assertTrue(isinstance(s.falsePositiveRateByLabel, list)) - self.assertTrue(isinstance(s.precisionByLabel, list)) - self.assertTrue(isinstance(s.recallByLabel, list)) - self.assertTrue(isinstance(s.fMeasureByLabel(), list)) - self.assertTrue(isinstance(s.fMeasureByLabel(1.0), list)) - self.assertTrue(isinstance(s.roc, DataFrame)) - self.assertAlmostEqual(s.areaUnderROC, 1.0, 2) - self.assertTrue(isinstance(s.pr, DataFrame)) - self.assertTrue(isinstance(s.fMeasureByThreshold, DataFrame)) - self.assertTrue(isinstance(s.precisionByThreshold, DataFrame)) - self.assertTrue(isinstance(s.recallByThreshold, DataFrame)) - self.assertAlmostEqual(s.accuracy, 1.0, 2) - self.assertAlmostEqual(s.weightedTruePositiveRate, 1.0, 2) - self.assertAlmostEqual(s.weightedFalsePositiveRate, 0.0, 2) - self.assertAlmostEqual(s.weightedRecall, 1.0, 2) - self.assertAlmostEqual(s.weightedPrecision, 1.0, 2) - self.assertAlmostEqual(s.weightedFMeasure(), 1.0, 2) - self.assertAlmostEqual(s.weightedFMeasure(1.0), 1.0, 2) - # test evaluation (with training dataset) produces a summary with same values - # one check is enough to verify a summary is returned, Scala version runs full test - sameSummary = model.evaluate(df) - self.assertTrue(isinstance(sameSummary, BinaryLogisticRegressionSummary)) - self.assertAlmostEqual(sameSummary.areaUnderROC, s.areaUnderROC) - - def test_multiclass_logistic_regression_summary(self): - df = self.spark.createDataFrame( - [ - (1.0, 2.0, Vectors.dense(1.0)), - (0.0, 2.0, Vectors.sparse(1, [], [])), - (2.0, 2.0, Vectors.dense(2.0)), - (2.0, 2.0, Vectors.dense(1.9)), - ], - ["label", "weight", "features"], - ) - lr = LogisticRegression(maxIter=5, regParam=0.01, weightCol="weight", fitIntercept=False) - model = lr.fit(df) - self.assertTrue(model.hasSummary) - s = model.summary - # test that api is callable and returns expected types - self.assertTrue(isinstance(s.predictions, DataFrame)) - self.assertEqual(s.probabilityCol, "probability") - self.assertEqual(s.labelCol, "label") - self.assertEqual(s.featuresCol, "features") - self.assertEqual(s.predictionCol, "prediction") - objHist = s.objectiveHistory - self.assertTrue(isinstance(objHist, list) and isinstance(objHist[0], float)) - self.assertGreater(s.totalIterations, 0) - self.assertTrue(isinstance(s.labels, list)) - self.assertTrue(isinstance(s.truePositiveRateByLabel, list)) - self.assertTrue(isinstance(s.falsePositiveRateByLabel, list)) - self.assertTrue(isinstance(s.precisionByLabel, list)) - self.assertTrue(isinstance(s.recallByLabel, list)) - self.assertTrue(isinstance(s.fMeasureByLabel(), list)) - self.assertTrue(isinstance(s.fMeasureByLabel(1.0), list)) - self.assertAlmostEqual(s.accuracy, 0.75, 2) - self.assertAlmostEqual(s.weightedTruePositiveRate, 0.75, 2) - self.assertAlmostEqual(s.weightedFalsePositiveRate, 0.25, 2) - self.assertAlmostEqual(s.weightedRecall, 0.75, 2) - self.assertAlmostEqual(s.weightedPrecision, 0.583, 2) - self.assertAlmostEqual(s.weightedFMeasure(), 0.65, 2) - self.assertAlmostEqual(s.weightedFMeasure(1.0), 0.65, 2) - # test evaluation (with training dataset) produces a summary with same values - # one check is enough to verify a summary is returned, Scala version runs full test - sameSummary = model.evaluate(df) - self.assertTrue(isinstance(sameSummary, LogisticRegressionSummary)) - self.assertFalse(isinstance(sameSummary, BinaryLogisticRegressionSummary)) - self.assertAlmostEqual(sameSummary.accuracy, s.accuracy) - def test_linear_svc_summary(self): df = self.spark.createDataFrame( [(1.0, 2.0, Vectors.dense(1.0, 1.0, 1.0)), (0.0, 2.0, Vectors.dense(1.0, 2.0, 3.0))], diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index 9bbd64d2aef5a..3fe97f44619c0 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -37,6 +37,7 @@ from pyspark import since from pyspark.ml.common import inherit_doc +from pyspark.ml.remote.util import try_remote_intermediate_result, try_remote_write, try_remote_read from pyspark.sql import SparkSession from pyspark.sql.utils import is_remote from pyspark.util import VersionUtils @@ -270,6 +271,7 @@ class JavaMLWritable(MLWritable): (Private) Mixin for ML instances that provide :py:class:`JavaMLWriter`. """ + @try_remote_write def write(self) -> JavaMLWriter: """Returns an MLWriter instance for this ML instance.""" return JavaMLWriter(self) @@ -378,6 +380,7 @@ class JavaMLReadable(MLReadable[RL]): """ @classmethod + @try_remote_read def read(cls) -> JavaMLReader[RL]: """Returns an MLReader instance for this class.""" return JavaMLReader(cls) @@ -680,6 +683,7 @@ def hasSummary(self) -> bool: @property @since("2.1.0") + @try_remote_intermediate_result def summary(self) -> T: """ Gets summary of the model trained on the training set. An exception is thrown if diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py index eed7781dc71e3..e2bf25386c77a 100644 --- a/python/pyspark/ml/wrapper.py +++ b/python/pyspark/ml/wrapper.py @@ -19,7 +19,15 @@ from typing import Any, Generic, Optional, List, Type, TypeVar, TYPE_CHECKING from pyspark import since -from pyspark.sql import DataFrame +from pyspark.ml.remote.util import ( + try_remote_transform_relation, + try_remote_call, + try_remote_fit, + try_remote_del, + try_remote_return_java_class, + try_remote_intercept, +) +from pyspark.sql import DataFrame, is_remote from pyspark.ml import Estimator, Predictor, PredictionModel, Transformer, Model from pyspark.ml.base import _PredictorParams from pyspark.ml.param import Param, Params @@ -47,6 +55,7 @@ def __init__(self, java_obj: Optional["JavaObject"] = None): super(JavaWrapper, self).__init__() self._java_obj = java_obj + @try_remote_del def __del__(self) -> None: from pyspark.core.context import SparkContext @@ -63,6 +72,7 @@ def _create_from_java_class(cls: Type[JW], java_class: str, *args: Any) -> JW: java_obj = JavaWrapper._new_java_obj(java_class, *args) return cls(java_obj) + @try_remote_call def _call_java(self, name: str, *args: Any) -> Any: from pyspark.core.context import SparkContext @@ -74,6 +84,7 @@ def _call_java(self, name: str, *args: Any) -> Any: return _java2py(sc, m(*java_args)) @staticmethod + @try_remote_return_java_class def _new_java_obj(java_class: str, *args: Any) -> "JavaObject": """ Returns a new Java object. @@ -347,6 +358,7 @@ def copy(self: "JP", extra: Optional["ParamMap"] = None) -> "JP": that._transfer_params_to_java() return that + @try_remote_intercept def clear(self, param: Param) -> None: """ Clears a param from the param map if it has been explicitly set. @@ -372,6 +384,7 @@ def _create_model(self, java_model: "JavaObject") -> JM: """ raise NotImplementedError() + @try_remote_fit def _fit_java(self, dataset: DataFrame) -> "JavaObject": """ Fits a Java model to the input dataset. @@ -405,6 +418,7 @@ class JavaTransformer(JavaParams, Transformer, metaclass=ABCMeta): available as _java_obj. """ + @try_remote_transform_relation def _transform(self, dataset: DataFrame) -> DataFrame: assert self._java_obj is not None @@ -435,7 +449,7 @@ def __init__(self, java_model: Optional["JavaObject"] = None): other ML classes). """ super(JavaModel, self).__init__(java_model) - if java_model is not None: + if java_model is not None and not is_remote(): # SPARK-10931: This is a temporary fix to allow models to own params # from estimators. Eventually, these params should be in models through # using common base classes between estimators and models. diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index 162d309a6f40a..4a85ca26b64de 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -20,6 +20,8 @@ "SparkConnectClient", ] +import atexit + from pyspark.sql.connect.utils import check_dependencies check_dependencies(__name__) @@ -675,6 +677,9 @@ def __init__( self._progress_handlers: List[ProgressHandler] = [] + # cleanup ml cache if possible + atexit.register(self._cleanup_ml) + def register_progress_handler(self, handler: ProgressHandler) -> None: """ Register a progress handler to be called when a progress message is received. @@ -1479,6 +1484,8 @@ def handle_response( b.checkpoint_command_result.relation ) } + if b.HasField("ml_command_result"): + yield {"ml_command_result": b.ml_command_result} try: if self._use_reattachable_execute: @@ -1931,3 +1938,33 @@ def _create_profile(self, profile: pb2.ResourceProfile) -> int: (_, properties, _) = self.execute_command(cmd) profile_id = properties["create_resource_profile_command_result"] return profile_id + + def add_ml_cache(self, cache_id: str) -> None: + if not hasattr(self.thread_local, "ml_caches"): + self.thread_local.ml_caches = set() + self.thread_local.ml_caches.add(cache_id) + + def remove_ml_cache(self, cache_id: str) -> None: + if not hasattr(self.thread_local, "ml_caches"): + self.thread_local.ml_caches = set() + + if cache_id in self.thread_local.ml_caches: + self._delete_ml_cache(cache_id) + + def _delete_ml_cache(self, cache_id: str) -> None: + # try best to delete the cache + try: + command = pb2.Command() + command.ml_command.delete.obj_ref.CopyFrom(pb2.ObjectRef(id=cache_id)) + self.execute_command(command) + except Exception: + pass + + def _cleanup_ml(self) -> None: + if not hasattr(self.thread_local, "ml_caches"): + self.thread_local.ml_caches = set() + + self.disable_reattachable_execute() + # Todo add a pattern to delete all model in one command + for model_id in self.thread_local.ml_caches: + self._delete_ml_cache(model_id) diff --git a/python/pyspark/sql/connect/proto/__init__.py b/python/pyspark/sql/connect/proto/__init__.py index 3e8d074d963dc..0877696c2680e 100644 --- a/python/pyspark/sql/connect/proto/__init__.py +++ b/python/pyspark/sql/connect/proto/__init__.py @@ -23,3 +23,5 @@ from pyspark.sql.connect.proto.relations_pb2 import * from pyspark.sql.connect.proto.catalog_pb2 import * from pyspark.sql.connect.proto.common_pb2 import * +from pyspark.sql.connect.proto.ml_pb2 import * +from pyspark.sql.connect.proto.ml_common_pb2 import * diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 64c549ffe9565..2fbc4287db786 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -40,10 +40,11 @@ from pyspark.sql.connect.proto import expressions_pb2 as spark_dot_connect_dot_expressions__pb2 from pyspark.sql.connect.proto import relations_pb2 as spark_dot_connect_dot_relations__pb2 from pyspark.sql.connect.proto import types_pb2 as spark_dot_connect_dot_types__pb2 +from pyspark.sql.connect.proto import ml_pb2 as spark_dot_connect_dot_ml__pb2 DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\xf5\x14\n\x12\x41nalyzePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x11 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12\x42\n\x06schema\x18\x04 \x01(\x0b\x32(.spark.connect.AnalyzePlanRequest.SchemaH\x00R\x06schema\x12\x45\n\x07\x65xplain\x18\x05 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.ExplainH\x00R\x07\x65xplain\x12O\n\x0btree_string\x18\x06 \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.TreeStringH\x00R\ntreeString\x12\x46\n\x08is_local\x18\x07 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.IsLocalH\x00R\x07isLocal\x12R\n\x0cis_streaming\x18\x08 \x01(\x0b\x32-.spark.connect.AnalyzePlanRequest.IsStreamingH\x00R\x0bisStreaming\x12O\n\x0binput_files\x18\t \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.InputFilesH\x00R\ninputFiles\x12U\n\rspark_version\x18\n \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SparkVersionH\x00R\x0csparkVersion\x12I\n\tddl_parse\x18\x0b \x01(\x0b\x32*.spark.connect.AnalyzePlanRequest.DDLParseH\x00R\x08\x64\x64lParse\x12X\n\x0esame_semantics\x18\x0c \x01(\x0b\x32/.spark.connect.AnalyzePlanRequest.SameSemanticsH\x00R\rsameSemantics\x12U\n\rsemantic_hash\x18\r \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SemanticHashH\x00R\x0csemanticHash\x12\x45\n\x07persist\x18\x0e \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.PersistH\x00R\x07persist\x12K\n\tunpersist\x18\x0f \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.UnpersistH\x00R\tunpersist\x12_\n\x11get_storage_level\x18\x10 \x01(\x0b\x32\x31.spark.connect.AnalyzePlanRequest.GetStorageLevelH\x00R\x0fgetStorageLevel\x12M\n\x0bjson_to_ddl\x18\x12 \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.JsonToDDLH\x00R\tjsonToDdl\x1a\x31\n\x06Schema\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\xbb\x02\n\x07\x45xplain\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12X\n\x0c\x65xplain_mode\x18\x02 \x01(\x0e\x32\x35.spark.connect.AnalyzePlanRequest.Explain.ExplainModeR\x0b\x65xplainMode"\xac\x01\n\x0b\x45xplainMode\x12\x1c\n\x18\x45XPLAIN_MODE_UNSPECIFIED\x10\x00\x12\x17\n\x13\x45XPLAIN_MODE_SIMPLE\x10\x01\x12\x19\n\x15\x45XPLAIN_MODE_EXTENDED\x10\x02\x12\x18\n\x14\x45XPLAIN_MODE_CODEGEN\x10\x03\x12\x15\n\x11\x45XPLAIN_MODE_COST\x10\x04\x12\x1a\n\x16\x45XPLAIN_MODE_FORMATTED\x10\x05\x1aZ\n\nTreeString\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12\x19\n\x05level\x18\x02 \x01(\x05H\x00R\x05level\x88\x01\x01\x42\x08\n\x06_level\x1a\x32\n\x07IsLocal\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x36\n\x0bIsStreaming\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x35\n\nInputFiles\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x0e\n\x0cSparkVersion\x1a)\n\x08\x44\x44LParse\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlString\x1ay\n\rSameSemantics\x12\x34\n\x0btarget_plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\ntargetPlan\x12\x32\n\nother_plan\x18\x02 \x01(\x0b\x32\x13.spark.connect.PlanR\totherPlan\x1a\x37\n\x0cSemanticHash\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x97\x01\n\x07Persist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x45\n\rstorage_level\x18\x02 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level\x1an\n\tUnpersist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x1f\n\x08\x62locking\x18\x02 \x01(\x08H\x00R\x08\x62locking\x88\x01\x01\x42\x0b\n\t_blocking\x1a\x46\n\x0fGetStorageLevel\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1a,\n\tJsonToDDL\x12\x1f\n\x0bjson_string\x18\x01 \x01(\tR\njsonStringB\t\n\x07\x61nalyzeB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xca\x0e\n\x13\x41nalyzePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12\x43\n\x06schema\x18\x02 \x01(\x0b\x32).spark.connect.AnalyzePlanResponse.SchemaH\x00R\x06schema\x12\x46\n\x07\x65xplain\x18\x03 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.ExplainH\x00R\x07\x65xplain\x12P\n\x0btree_string\x18\x04 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.TreeStringH\x00R\ntreeString\x12G\n\x08is_local\x18\x05 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.IsLocalH\x00R\x07isLocal\x12S\n\x0cis_streaming\x18\x06 \x01(\x0b\x32..spark.connect.AnalyzePlanResponse.IsStreamingH\x00R\x0bisStreaming\x12P\n\x0binput_files\x18\x07 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.InputFilesH\x00R\ninputFiles\x12V\n\rspark_version\x18\x08 \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SparkVersionH\x00R\x0csparkVersion\x12J\n\tddl_parse\x18\t \x01(\x0b\x32+.spark.connect.AnalyzePlanResponse.DDLParseH\x00R\x08\x64\x64lParse\x12Y\n\x0esame_semantics\x18\n \x01(\x0b\x32\x30.spark.connect.AnalyzePlanResponse.SameSemanticsH\x00R\rsameSemantics\x12V\n\rsemantic_hash\x18\x0b \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SemanticHashH\x00R\x0csemanticHash\x12\x46\n\x07persist\x18\x0c \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.PersistH\x00R\x07persist\x12L\n\tunpersist\x18\r \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.UnpersistH\x00R\tunpersist\x12`\n\x11get_storage_level\x18\x0e \x01(\x0b\x32\x32.spark.connect.AnalyzePlanResponse.GetStorageLevelH\x00R\x0fgetStorageLevel\x12N\n\x0bjson_to_ddl\x18\x10 \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.JsonToDDLH\x00R\tjsonToDdl\x1a\x39\n\x06Schema\x12/\n\x06schema\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1a\x30\n\x07\x45xplain\x12%\n\x0e\x65xplain_string\x18\x01 \x01(\tR\rexplainString\x1a-\n\nTreeString\x12\x1f\n\x0btree_string\x18\x01 \x01(\tR\ntreeString\x1a$\n\x07IsLocal\x12\x19\n\x08is_local\x18\x01 \x01(\x08R\x07isLocal\x1a\x30\n\x0bIsStreaming\x12!\n\x0cis_streaming\x18\x01 \x01(\x08R\x0bisStreaming\x1a"\n\nInputFiles\x12\x14\n\x05\x66iles\x18\x01 \x03(\tR\x05\x66iles\x1a(\n\x0cSparkVersion\x12\x18\n\x07version\x18\x01 \x01(\tR\x07version\x1a;\n\x08\x44\x44LParse\x12/\n\x06parsed\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06parsed\x1a\'\n\rSameSemantics\x12\x16\n\x06result\x18\x01 \x01(\x08R\x06result\x1a&\n\x0cSemanticHash\x12\x16\n\x06result\x18\x01 \x01(\x05R\x06result\x1a\t\n\x07Persist\x1a\x0b\n\tUnpersist\x1aS\n\x0fGetStorageLevel\x12@\n\rstorage_level\x18\x01 \x01(\x0b\x32\x1b.spark.connect.StorageLevelR\x0cstorageLevel\x1a*\n\tJsonToDDL\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlStringB\x08\n\x06result"\xa3\x05\n\x12\x45xecutePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12&\n\x0coperation_id\x18\x06 \x01(\tH\x01R\x0boperationId\x88\x01\x01\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12X\n\x0frequest_options\x18\x05 \x03(\x0b\x32/.spark.connect.ExecutePlanRequest.RequestOptionR\x0erequestOptions\x12\x12\n\x04tags\x18\x07 \x03(\tR\x04tags\x1a\xa5\x01\n\rRequestOption\x12K\n\x10reattach_options\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ReattachOptionsH\x00R\x0freattachOptions\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x10\n\x0erequest_optionB)\n\'_client_observed_server_side_session_idB\x0f\n\r_operation_idB\x0e\n\x0c_client_type"\xe6\x16\n\x13\x45xecutePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12!\n\x0coperation_id\x18\x0c \x01(\tR\x0boperationId\x12\x1f\n\x0bresponse_id\x18\r \x01(\tR\nresponseId\x12P\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchH\x00R\narrowBatch\x12\x63\n\x12sql_command_result\x18\x05 \x01(\x0b\x32\x33.spark.connect.ExecutePlanResponse.SqlCommandResultH\x00R\x10sqlCommandResult\x12~\n#write_stream_operation_start_result\x18\x08 \x01(\x0b\x32..spark.connect.WriteStreamOperationStartResultH\x00R\x1fwriteStreamOperationStartResult\x12q\n\x1estreaming_query_command_result\x18\t \x01(\x0b\x32*.spark.connect.StreamingQueryCommandResultH\x00R\x1bstreamingQueryCommandResult\x12k\n\x1cget_resources_command_result\x18\n \x01(\x0b\x32(.spark.connect.GetResourcesCommandResultH\x00R\x19getResourcesCommandResult\x12\x87\x01\n&streaming_query_manager_command_result\x18\x0b \x01(\x0b\x32\x31.spark.connect.StreamingQueryManagerCommandResultH\x00R"streamingQueryManagerCommandResult\x12\x87\x01\n&streaming_query_listener_events_result\x18\x10 \x01(\x0b\x32\x31.spark.connect.StreamingQueryListenerEventsResultH\x00R"streamingQueryListenerEventsResult\x12\\\n\x0fresult_complete\x18\x0e \x01(\x0b\x32\x31.spark.connect.ExecutePlanResponse.ResultCompleteH\x00R\x0eresultComplete\x12\x87\x01\n&create_resource_profile_command_result\x18\x11 \x01(\x0b\x32\x31.spark.connect.CreateResourceProfileCommandResultH\x00R"createResourceProfileCommandResult\x12\x65\n\x12\x65xecution_progress\x18\x12 \x01(\x0b\x32\x34.spark.connect.ExecutePlanResponse.ExecutionProgressH\x00R\x11\x65xecutionProgress\x12\x64\n\x19\x63heckpoint_command_result\x18\x13 \x01(\x0b\x32&.spark.connect.CheckpointCommandResultH\x00R\x17\x63heckpointCommandResult\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x12]\n\x10observed_metrics\x18\x06 \x03(\x0b\x32\x32.spark.connect.ExecutePlanResponse.ObservedMetricsR\x0fobservedMetrics\x12/\n\x06schema\x18\x07 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1aG\n\x10SqlCommandResult\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1av\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x12&\n\x0cstart_offset\x18\x03 \x01(\x03H\x00R\x0bstartOffset\x88\x01\x01\x42\x0f\n\r_start_offset\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType\x1a\x8d\x01\n\x0fObservedMetrics\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x12\x12\n\x04keys\x18\x03 \x03(\tR\x04keys\x12\x17\n\x07plan_id\x18\x04 \x01(\x03R\x06planId\x1a\x10\n\x0eResultComplete\x1a\xcd\x02\n\x11\x45xecutionProgress\x12V\n\x06stages\x18\x01 \x03(\x0b\x32>.spark.connect.ExecutePlanResponse.ExecutionProgress.StageInfoR\x06stages\x12,\n\x12num_inflight_tasks\x18\x02 \x01(\x03R\x10numInflightTasks\x1a\xb1\x01\n\tStageInfo\x12\x19\n\x08stage_id\x18\x01 \x01(\x03R\x07stageId\x12\x1b\n\tnum_tasks\x18\x02 \x01(\x03R\x08numTasks\x12.\n\x13num_completed_tasks\x18\x03 \x01(\x03R\x11numCompletedTasks\x12(\n\x10input_bytes_read\x18\x04 \x01(\x03R\x0einputBytesRead\x12\x12\n\x04\x64one\x18\x05 \x01(\x08R\x04\x64oneB\x0f\n\rresponse_type"A\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x19\n\x05value\x18\x02 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"\xaf\t\n\rConfigRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\\\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1b\n\x06silent\x18\x02 \x01(\x08H\x00R\x06silent\x88\x01\x01\x42\t\n\x07_silent\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a?\n\x0eGetWithDefault\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xaf\x01\n\x0e\x43onfigResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x04 \x01(\tR\x13serverSideSessionId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings"\xea\x07\n\x13\x41\x64\x64\x41rtifactsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12$\n\x0b\x63lient_type\x18\x06 \x01(\tH\x02R\nclientType\x88\x01\x01\x12@\n\x05\x62\x61tch\x18\x03 \x01(\x0b\x32(.spark.connect.AddArtifactsRequest.BatchH\x00R\x05\x62\x61tch\x12Z\n\x0b\x62\x65gin_chunk\x18\x04 \x01(\x0b\x32\x37.spark.connect.AddArtifactsRequest.BeginChunkedArtifactH\x00R\nbeginChunk\x12H\n\x05\x63hunk\x18\x05 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkH\x00R\x05\x63hunk\x1a\x35\n\rArtifactChunk\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12\x10\n\x03\x63rc\x18\x02 \x01(\x03R\x03\x63rc\x1ao\n\x13SingleChunkArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x44\n\x04\x64\x61ta\x18\x02 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x04\x64\x61ta\x1a]\n\x05\x42\x61tch\x12T\n\tartifacts\x18\x01 \x03(\x0b\x32\x36.spark.connect.AddArtifactsRequest.SingleChunkArtifactR\tartifacts\x1a\xc1\x01\n\x14\x42\x65ginChunkedArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n\x0btotal_bytes\x18\x02 \x01(\x03R\ntotalBytes\x12\x1d\n\nnum_chunks\x18\x03 \x01(\x03R\tnumChunks\x12U\n\rinitial_chunk\x18\x04 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x0cinitialChunkB\t\n\x07payloadB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x02\n\x14\x41\x64\x64\x41rtifactsResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\tartifacts\x18\x01 \x03(\x0b\x32\x33.spark.connect.AddArtifactsResponse.ArtifactSummaryR\tartifacts\x1aQ\n\x0f\x41rtifactSummary\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12*\n\x11is_crc_successful\x18\x02 \x01(\x08R\x0fisCrcSuccessful"\xc6\x02\n\x17\x41rtifactStatusesRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12\x14\n\x05names\x18\x04 \x03(\tR\x05namesB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xe0\x02\n\x18\x41rtifactStatusesResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\x08statuses\x18\x01 \x03(\x0b\x32\x35.spark.connect.ArtifactStatusesResponse.StatusesEntryR\x08statuses\x1as\n\rStatusesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ArtifactStatusesResponse.ArtifactStatusR\x05value:\x02\x38\x01\x1a(\n\x0e\x41rtifactStatus\x12\x16\n\x06\x65xists\x18\x01 \x01(\x08R\x06\x65xists"\xdb\x04\n\x10InterruptRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12T\n\x0einterrupt_type\x18\x04 \x01(\x0e\x32-.spark.connect.InterruptRequest.InterruptTypeR\rinterruptType\x12%\n\roperation_tag\x18\x05 \x01(\tH\x00R\x0coperationTag\x12#\n\x0coperation_id\x18\x06 \x01(\tH\x00R\x0boperationId"\x80\x01\n\rInterruptType\x12\x1e\n\x1aINTERRUPT_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12INTERRUPT_TYPE_ALL\x10\x01\x12\x16\n\x12INTERRUPT_TYPE_TAG\x10\x02\x12\x1f\n\x1bINTERRUPT_TYPE_OPERATION_ID\x10\x03\x42\x0b\n\tinterruptB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x01\n\x11InterruptResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\'\n\x0finterrupted_ids\x18\x02 \x03(\tR\x0einterruptedIds"5\n\x0fReattachOptions\x12"\n\x0creattachable\x18\x01 \x01(\x08R\x0creattachable"\x96\x03\n\x16ReattachExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x06 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x12-\n\x10last_response_id\x18\x05 \x01(\tH\x02R\x0elastResponseId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x13\n\x11_last_response_id"\xc9\x04\n\x15ReleaseExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12R\n\x0brelease_all\x18\x05 \x01(\x0b\x32/.spark.connect.ReleaseExecuteRequest.ReleaseAllH\x00R\nreleaseAll\x12X\n\rrelease_until\x18\x06 \x01(\x0b\x32\x31.spark.connect.ReleaseExecuteRequest.ReleaseUntilH\x00R\x0creleaseUntil\x1a\x0c\n\nReleaseAll\x1a/\n\x0cReleaseUntil\x12\x1f\n\x0bresponse_id\x18\x01 \x01(\tR\nresponseIdB\t\n\x07releaseB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xa5\x01\n\x16ReleaseExecuteResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12&\n\x0coperation_id\x18\x02 \x01(\tH\x00R\x0boperationId\x88\x01\x01\x42\x0f\n\r_operation_id"\xd4\x01\n\x15ReleaseSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\'\n\x0f\x61llow_reconnect\x18\x04 \x01(\x08R\x0e\x61llowReconnectB\x0e\n\x0c_client_type"l\n\x16ReleaseSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId"\xcc\x02\n\x18\x46\x65tchErrorDetailsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x19\n\x08\x65rror_id\x18\x03 \x01(\tR\x07\x65rrorId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x93\x0c\n\x19\x46\x65tchErrorDetailsResponse\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\x1d\n\nsession_id\x18\x04 \x01(\tR\tsessionId\x12)\n\x0eroot_error_idx\x18\x01 \x01(\x05H\x00R\x0crootErrorIdx\x88\x01\x01\x12\x46\n\x06\x65rrors\x18\x02 \x03(\x0b\x32..spark.connect.FetchErrorDetailsResponse.ErrorR\x06\x65rrors\x1a\xae\x01\n\x11StackTraceElement\x12\'\n\x0f\x64\x65\x63laring_class\x18\x01 \x01(\tR\x0e\x64\x65\x63laringClass\x12\x1f\n\x0bmethod_name\x18\x02 \x01(\tR\nmethodName\x12 \n\tfile_name\x18\x03 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12\x1f\n\x0bline_number\x18\x04 \x01(\x05R\nlineNumberB\x0c\n\n_file_name\x1a\xf0\x02\n\x0cQueryContext\x12\x64\n\x0c\x63ontext_type\x18\n \x01(\x0e\x32\x41.spark.connect.FetchErrorDetailsResponse.QueryContext.ContextTypeR\x0b\x63ontextType\x12\x1f\n\x0bobject_type\x18\x01 \x01(\tR\nobjectType\x12\x1f\n\x0bobject_name\x18\x02 \x01(\tR\nobjectName\x12\x1f\n\x0bstart_index\x18\x03 \x01(\x05R\nstartIndex\x12\x1d\n\nstop_index\x18\x04 \x01(\x05R\tstopIndex\x12\x1a\n\x08\x66ragment\x18\x05 \x01(\tR\x08\x66ragment\x12\x1b\n\tcall_site\x18\x06 \x01(\tR\x08\x63\x61llSite\x12\x18\n\x07summary\x18\x07 \x01(\tR\x07summary"%\n\x0b\x43ontextType\x12\x07\n\x03SQL\x10\x00\x12\r\n\tDATAFRAME\x10\x01\x1a\x99\x03\n\x0eSparkThrowable\x12$\n\x0b\x65rror_class\x18\x01 \x01(\tH\x00R\nerrorClass\x88\x01\x01\x12}\n\x12message_parameters\x18\x02 \x03(\x0b\x32N.spark.connect.FetchErrorDetailsResponse.SparkThrowable.MessageParametersEntryR\x11messageParameters\x12\\\n\x0equery_contexts\x18\x03 \x03(\x0b\x32\x35.spark.connect.FetchErrorDetailsResponse.QueryContextR\rqueryContexts\x12 \n\tsql_state\x18\x04 \x01(\tH\x01R\x08sqlState\x88\x01\x01\x1a\x44\n\x16MessageParametersEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x0e\n\x0c_error_classB\x0c\n\n_sql_state\x1a\xdb\x02\n\x05\x45rror\x12\x30\n\x14\x65rror_type_hierarchy\x18\x01 \x03(\tR\x12\x65rrorTypeHierarchy\x12\x18\n\x07message\x18\x02 \x01(\tR\x07message\x12[\n\x0bstack_trace\x18\x03 \x03(\x0b\x32:.spark.connect.FetchErrorDetailsResponse.StackTraceElementR\nstackTrace\x12 \n\tcause_idx\x18\x04 \x01(\x05H\x00R\x08\x63\x61useIdx\x88\x01\x01\x12\x65\n\x0fspark_throwable\x18\x05 \x01(\x0b\x32\x37.spark.connect.FetchErrorDetailsResponse.SparkThrowableH\x01R\x0esparkThrowable\x88\x01\x01\x42\x0c\n\n_cause_idxB\x12\n\x10_spark_throwableB\x11\n\x0f_root_error_idx"Z\n\x17\x43heckpointCommandResult\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation2\xb2\x07\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x12[\n\x0c\x41\x64\x64\x41rtifacts\x12".spark.connect.AddArtifactsRequest\x1a#.spark.connect.AddArtifactsResponse"\x00(\x01\x12\x63\n\x0e\x41rtifactStatus\x12&.spark.connect.ArtifactStatusesRequest\x1a\'.spark.connect.ArtifactStatusesResponse"\x00\x12P\n\tInterrupt\x12\x1f.spark.connect.InterruptRequest\x1a .spark.connect.InterruptResponse"\x00\x12`\n\x0fReattachExecute\x12%.spark.connect.ReattachExecuteRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12_\n\x0eReleaseExecute\x12$.spark.connect.ReleaseExecuteRequest\x1a%.spark.connect.ReleaseExecuteResponse"\x00\x12_\n\x0eReleaseSession\x12$.spark.connect.ReleaseSessionRequest\x1a%.spark.connect.ReleaseSessionResponse"\x00\x12h\n\x11\x46\x65tchErrorDetails\x12\'.spark.connect.FetchErrorDetailsRequest\x1a(.spark.connect.FetchErrorDetailsResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto\x1a\x16spark/connect/ml.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\xf5\x14\n\x12\x41nalyzePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x11 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12\x42\n\x06schema\x18\x04 \x01(\x0b\x32(.spark.connect.AnalyzePlanRequest.SchemaH\x00R\x06schema\x12\x45\n\x07\x65xplain\x18\x05 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.ExplainH\x00R\x07\x65xplain\x12O\n\x0btree_string\x18\x06 \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.TreeStringH\x00R\ntreeString\x12\x46\n\x08is_local\x18\x07 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.IsLocalH\x00R\x07isLocal\x12R\n\x0cis_streaming\x18\x08 \x01(\x0b\x32-.spark.connect.AnalyzePlanRequest.IsStreamingH\x00R\x0bisStreaming\x12O\n\x0binput_files\x18\t \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.InputFilesH\x00R\ninputFiles\x12U\n\rspark_version\x18\n \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SparkVersionH\x00R\x0csparkVersion\x12I\n\tddl_parse\x18\x0b \x01(\x0b\x32*.spark.connect.AnalyzePlanRequest.DDLParseH\x00R\x08\x64\x64lParse\x12X\n\x0esame_semantics\x18\x0c \x01(\x0b\x32/.spark.connect.AnalyzePlanRequest.SameSemanticsH\x00R\rsameSemantics\x12U\n\rsemantic_hash\x18\r \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SemanticHashH\x00R\x0csemanticHash\x12\x45\n\x07persist\x18\x0e \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.PersistH\x00R\x07persist\x12K\n\tunpersist\x18\x0f \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.UnpersistH\x00R\tunpersist\x12_\n\x11get_storage_level\x18\x10 \x01(\x0b\x32\x31.spark.connect.AnalyzePlanRequest.GetStorageLevelH\x00R\x0fgetStorageLevel\x12M\n\x0bjson_to_ddl\x18\x12 \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.JsonToDDLH\x00R\tjsonToDdl\x1a\x31\n\x06Schema\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\xbb\x02\n\x07\x45xplain\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12X\n\x0c\x65xplain_mode\x18\x02 \x01(\x0e\x32\x35.spark.connect.AnalyzePlanRequest.Explain.ExplainModeR\x0b\x65xplainMode"\xac\x01\n\x0b\x45xplainMode\x12\x1c\n\x18\x45XPLAIN_MODE_UNSPECIFIED\x10\x00\x12\x17\n\x13\x45XPLAIN_MODE_SIMPLE\x10\x01\x12\x19\n\x15\x45XPLAIN_MODE_EXTENDED\x10\x02\x12\x18\n\x14\x45XPLAIN_MODE_CODEGEN\x10\x03\x12\x15\n\x11\x45XPLAIN_MODE_COST\x10\x04\x12\x1a\n\x16\x45XPLAIN_MODE_FORMATTED\x10\x05\x1aZ\n\nTreeString\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12\x19\n\x05level\x18\x02 \x01(\x05H\x00R\x05level\x88\x01\x01\x42\x08\n\x06_level\x1a\x32\n\x07IsLocal\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x36\n\x0bIsStreaming\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x35\n\nInputFiles\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x0e\n\x0cSparkVersion\x1a)\n\x08\x44\x44LParse\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlString\x1ay\n\rSameSemantics\x12\x34\n\x0btarget_plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\ntargetPlan\x12\x32\n\nother_plan\x18\x02 \x01(\x0b\x32\x13.spark.connect.PlanR\totherPlan\x1a\x37\n\x0cSemanticHash\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x97\x01\n\x07Persist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x45\n\rstorage_level\x18\x02 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level\x1an\n\tUnpersist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x1f\n\x08\x62locking\x18\x02 \x01(\x08H\x00R\x08\x62locking\x88\x01\x01\x42\x0b\n\t_blocking\x1a\x46\n\x0fGetStorageLevel\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1a,\n\tJsonToDDL\x12\x1f\n\x0bjson_string\x18\x01 \x01(\tR\njsonStringB\t\n\x07\x61nalyzeB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xca\x0e\n\x13\x41nalyzePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12\x43\n\x06schema\x18\x02 \x01(\x0b\x32).spark.connect.AnalyzePlanResponse.SchemaH\x00R\x06schema\x12\x46\n\x07\x65xplain\x18\x03 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.ExplainH\x00R\x07\x65xplain\x12P\n\x0btree_string\x18\x04 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.TreeStringH\x00R\ntreeString\x12G\n\x08is_local\x18\x05 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.IsLocalH\x00R\x07isLocal\x12S\n\x0cis_streaming\x18\x06 \x01(\x0b\x32..spark.connect.AnalyzePlanResponse.IsStreamingH\x00R\x0bisStreaming\x12P\n\x0binput_files\x18\x07 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.InputFilesH\x00R\ninputFiles\x12V\n\rspark_version\x18\x08 \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SparkVersionH\x00R\x0csparkVersion\x12J\n\tddl_parse\x18\t \x01(\x0b\x32+.spark.connect.AnalyzePlanResponse.DDLParseH\x00R\x08\x64\x64lParse\x12Y\n\x0esame_semantics\x18\n \x01(\x0b\x32\x30.spark.connect.AnalyzePlanResponse.SameSemanticsH\x00R\rsameSemantics\x12V\n\rsemantic_hash\x18\x0b \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SemanticHashH\x00R\x0csemanticHash\x12\x46\n\x07persist\x18\x0c \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.PersistH\x00R\x07persist\x12L\n\tunpersist\x18\r \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.UnpersistH\x00R\tunpersist\x12`\n\x11get_storage_level\x18\x0e \x01(\x0b\x32\x32.spark.connect.AnalyzePlanResponse.GetStorageLevelH\x00R\x0fgetStorageLevel\x12N\n\x0bjson_to_ddl\x18\x10 \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.JsonToDDLH\x00R\tjsonToDdl\x1a\x39\n\x06Schema\x12/\n\x06schema\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1a\x30\n\x07\x45xplain\x12%\n\x0e\x65xplain_string\x18\x01 \x01(\tR\rexplainString\x1a-\n\nTreeString\x12\x1f\n\x0btree_string\x18\x01 \x01(\tR\ntreeString\x1a$\n\x07IsLocal\x12\x19\n\x08is_local\x18\x01 \x01(\x08R\x07isLocal\x1a\x30\n\x0bIsStreaming\x12!\n\x0cis_streaming\x18\x01 \x01(\x08R\x0bisStreaming\x1a"\n\nInputFiles\x12\x14\n\x05\x66iles\x18\x01 \x03(\tR\x05\x66iles\x1a(\n\x0cSparkVersion\x12\x18\n\x07version\x18\x01 \x01(\tR\x07version\x1a;\n\x08\x44\x44LParse\x12/\n\x06parsed\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06parsed\x1a\'\n\rSameSemantics\x12\x16\n\x06result\x18\x01 \x01(\x08R\x06result\x1a&\n\x0cSemanticHash\x12\x16\n\x06result\x18\x01 \x01(\x05R\x06result\x1a\t\n\x07Persist\x1a\x0b\n\tUnpersist\x1aS\n\x0fGetStorageLevel\x12@\n\rstorage_level\x18\x01 \x01(\x0b\x32\x1b.spark.connect.StorageLevelR\x0cstorageLevel\x1a*\n\tJsonToDDL\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlStringB\x08\n\x06result"\xa3\x05\n\x12\x45xecutePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12&\n\x0coperation_id\x18\x06 \x01(\tH\x01R\x0boperationId\x88\x01\x01\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12X\n\x0frequest_options\x18\x05 \x03(\x0b\x32/.spark.connect.ExecutePlanRequest.RequestOptionR\x0erequestOptions\x12\x12\n\x04tags\x18\x07 \x03(\tR\x04tags\x1a\xa5\x01\n\rRequestOption\x12K\n\x10reattach_options\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ReattachOptionsH\x00R\x0freattachOptions\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x10\n\x0erequest_optionB)\n\'_client_observed_server_side_session_idB\x0f\n\r_operation_idB\x0e\n\x0c_client_type"\xb4\x17\n\x13\x45xecutePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12!\n\x0coperation_id\x18\x0c \x01(\tR\x0boperationId\x12\x1f\n\x0bresponse_id\x18\r \x01(\tR\nresponseId\x12P\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchH\x00R\narrowBatch\x12\x63\n\x12sql_command_result\x18\x05 \x01(\x0b\x32\x33.spark.connect.ExecutePlanResponse.SqlCommandResultH\x00R\x10sqlCommandResult\x12~\n#write_stream_operation_start_result\x18\x08 \x01(\x0b\x32..spark.connect.WriteStreamOperationStartResultH\x00R\x1fwriteStreamOperationStartResult\x12q\n\x1estreaming_query_command_result\x18\t \x01(\x0b\x32*.spark.connect.StreamingQueryCommandResultH\x00R\x1bstreamingQueryCommandResult\x12k\n\x1cget_resources_command_result\x18\n \x01(\x0b\x32(.spark.connect.GetResourcesCommandResultH\x00R\x19getResourcesCommandResult\x12\x87\x01\n&streaming_query_manager_command_result\x18\x0b \x01(\x0b\x32\x31.spark.connect.StreamingQueryManagerCommandResultH\x00R"streamingQueryManagerCommandResult\x12\x87\x01\n&streaming_query_listener_events_result\x18\x10 \x01(\x0b\x32\x31.spark.connect.StreamingQueryListenerEventsResultH\x00R"streamingQueryListenerEventsResult\x12\\\n\x0fresult_complete\x18\x0e \x01(\x0b\x32\x31.spark.connect.ExecutePlanResponse.ResultCompleteH\x00R\x0eresultComplete\x12\x87\x01\n&create_resource_profile_command_result\x18\x11 \x01(\x0b\x32\x31.spark.connect.CreateResourceProfileCommandResultH\x00R"createResourceProfileCommandResult\x12\x65\n\x12\x65xecution_progress\x18\x12 \x01(\x0b\x32\x34.spark.connect.ExecutePlanResponse.ExecutionProgressH\x00R\x11\x65xecutionProgress\x12\x64\n\x19\x63heckpoint_command_result\x18\x13 \x01(\x0b\x32&.spark.connect.CheckpointCommandResultH\x00R\x17\x63heckpointCommandResult\x12L\n\x11ml_command_result\x18\x14 \x01(\x0b\x32\x1e.spark.connect.MlCommandResultH\x00R\x0fmlCommandResult\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x12]\n\x10observed_metrics\x18\x06 \x03(\x0b\x32\x32.spark.connect.ExecutePlanResponse.ObservedMetricsR\x0fobservedMetrics\x12/\n\x06schema\x18\x07 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1aG\n\x10SqlCommandResult\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1av\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x12&\n\x0cstart_offset\x18\x03 \x01(\x03H\x00R\x0bstartOffset\x88\x01\x01\x42\x0f\n\r_start_offset\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType\x1a\x8d\x01\n\x0fObservedMetrics\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x12\x12\n\x04keys\x18\x03 \x03(\tR\x04keys\x12\x17\n\x07plan_id\x18\x04 \x01(\x03R\x06planId\x1a\x10\n\x0eResultComplete\x1a\xcd\x02\n\x11\x45xecutionProgress\x12V\n\x06stages\x18\x01 \x03(\x0b\x32>.spark.connect.ExecutePlanResponse.ExecutionProgress.StageInfoR\x06stages\x12,\n\x12num_inflight_tasks\x18\x02 \x01(\x03R\x10numInflightTasks\x1a\xb1\x01\n\tStageInfo\x12\x19\n\x08stage_id\x18\x01 \x01(\x03R\x07stageId\x12\x1b\n\tnum_tasks\x18\x02 \x01(\x03R\x08numTasks\x12.\n\x13num_completed_tasks\x18\x03 \x01(\x03R\x11numCompletedTasks\x12(\n\x10input_bytes_read\x18\x04 \x01(\x03R\x0einputBytesRead\x12\x12\n\x04\x64one\x18\x05 \x01(\x08R\x04\x64oneB\x0f\n\rresponse_type"A\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x19\n\x05value\x18\x02 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"\xaf\t\n\rConfigRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\\\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1b\n\x06silent\x18\x02 \x01(\x08H\x00R\x06silent\x88\x01\x01\x42\t\n\x07_silent\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a?\n\x0eGetWithDefault\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xaf\x01\n\x0e\x43onfigResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x04 \x01(\tR\x13serverSideSessionId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings"\xea\x07\n\x13\x41\x64\x64\x41rtifactsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12$\n\x0b\x63lient_type\x18\x06 \x01(\tH\x02R\nclientType\x88\x01\x01\x12@\n\x05\x62\x61tch\x18\x03 \x01(\x0b\x32(.spark.connect.AddArtifactsRequest.BatchH\x00R\x05\x62\x61tch\x12Z\n\x0b\x62\x65gin_chunk\x18\x04 \x01(\x0b\x32\x37.spark.connect.AddArtifactsRequest.BeginChunkedArtifactH\x00R\nbeginChunk\x12H\n\x05\x63hunk\x18\x05 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkH\x00R\x05\x63hunk\x1a\x35\n\rArtifactChunk\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12\x10\n\x03\x63rc\x18\x02 \x01(\x03R\x03\x63rc\x1ao\n\x13SingleChunkArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x44\n\x04\x64\x61ta\x18\x02 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x04\x64\x61ta\x1a]\n\x05\x42\x61tch\x12T\n\tartifacts\x18\x01 \x03(\x0b\x32\x36.spark.connect.AddArtifactsRequest.SingleChunkArtifactR\tartifacts\x1a\xc1\x01\n\x14\x42\x65ginChunkedArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n\x0btotal_bytes\x18\x02 \x01(\x03R\ntotalBytes\x12\x1d\n\nnum_chunks\x18\x03 \x01(\x03R\tnumChunks\x12U\n\rinitial_chunk\x18\x04 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x0cinitialChunkB\t\n\x07payloadB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x02\n\x14\x41\x64\x64\x41rtifactsResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\tartifacts\x18\x01 \x03(\x0b\x32\x33.spark.connect.AddArtifactsResponse.ArtifactSummaryR\tartifacts\x1aQ\n\x0f\x41rtifactSummary\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12*\n\x11is_crc_successful\x18\x02 \x01(\x08R\x0fisCrcSuccessful"\xc6\x02\n\x17\x41rtifactStatusesRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12\x14\n\x05names\x18\x04 \x03(\tR\x05namesB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xe0\x02\n\x18\x41rtifactStatusesResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\x08statuses\x18\x01 \x03(\x0b\x32\x35.spark.connect.ArtifactStatusesResponse.StatusesEntryR\x08statuses\x1as\n\rStatusesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ArtifactStatusesResponse.ArtifactStatusR\x05value:\x02\x38\x01\x1a(\n\x0e\x41rtifactStatus\x12\x16\n\x06\x65xists\x18\x01 \x01(\x08R\x06\x65xists"\xdb\x04\n\x10InterruptRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12T\n\x0einterrupt_type\x18\x04 \x01(\x0e\x32-.spark.connect.InterruptRequest.InterruptTypeR\rinterruptType\x12%\n\roperation_tag\x18\x05 \x01(\tH\x00R\x0coperationTag\x12#\n\x0coperation_id\x18\x06 \x01(\tH\x00R\x0boperationId"\x80\x01\n\rInterruptType\x12\x1e\n\x1aINTERRUPT_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12INTERRUPT_TYPE_ALL\x10\x01\x12\x16\n\x12INTERRUPT_TYPE_TAG\x10\x02\x12\x1f\n\x1bINTERRUPT_TYPE_OPERATION_ID\x10\x03\x42\x0b\n\tinterruptB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x01\n\x11InterruptResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\'\n\x0finterrupted_ids\x18\x02 \x03(\tR\x0einterruptedIds"5\n\x0fReattachOptions\x12"\n\x0creattachable\x18\x01 \x01(\x08R\x0creattachable"\x96\x03\n\x16ReattachExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x06 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x12-\n\x10last_response_id\x18\x05 \x01(\tH\x02R\x0elastResponseId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x13\n\x11_last_response_id"\xc9\x04\n\x15ReleaseExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12R\n\x0brelease_all\x18\x05 \x01(\x0b\x32/.spark.connect.ReleaseExecuteRequest.ReleaseAllH\x00R\nreleaseAll\x12X\n\rrelease_until\x18\x06 \x01(\x0b\x32\x31.spark.connect.ReleaseExecuteRequest.ReleaseUntilH\x00R\x0creleaseUntil\x1a\x0c\n\nReleaseAll\x1a/\n\x0cReleaseUntil\x12\x1f\n\x0bresponse_id\x18\x01 \x01(\tR\nresponseIdB\t\n\x07releaseB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xa5\x01\n\x16ReleaseExecuteResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12&\n\x0coperation_id\x18\x02 \x01(\tH\x00R\x0boperationId\x88\x01\x01\x42\x0f\n\r_operation_id"\xd4\x01\n\x15ReleaseSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\'\n\x0f\x61llow_reconnect\x18\x04 \x01(\x08R\x0e\x61llowReconnectB\x0e\n\x0c_client_type"l\n\x16ReleaseSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId"\xcc\x02\n\x18\x46\x65tchErrorDetailsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x19\n\x08\x65rror_id\x18\x03 \x01(\tR\x07\x65rrorId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x93\x0c\n\x19\x46\x65tchErrorDetailsResponse\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\x1d\n\nsession_id\x18\x04 \x01(\tR\tsessionId\x12)\n\x0eroot_error_idx\x18\x01 \x01(\x05H\x00R\x0crootErrorIdx\x88\x01\x01\x12\x46\n\x06\x65rrors\x18\x02 \x03(\x0b\x32..spark.connect.FetchErrorDetailsResponse.ErrorR\x06\x65rrors\x1a\xae\x01\n\x11StackTraceElement\x12\'\n\x0f\x64\x65\x63laring_class\x18\x01 \x01(\tR\x0e\x64\x65\x63laringClass\x12\x1f\n\x0bmethod_name\x18\x02 \x01(\tR\nmethodName\x12 \n\tfile_name\x18\x03 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12\x1f\n\x0bline_number\x18\x04 \x01(\x05R\nlineNumberB\x0c\n\n_file_name\x1a\xf0\x02\n\x0cQueryContext\x12\x64\n\x0c\x63ontext_type\x18\n \x01(\x0e\x32\x41.spark.connect.FetchErrorDetailsResponse.QueryContext.ContextTypeR\x0b\x63ontextType\x12\x1f\n\x0bobject_type\x18\x01 \x01(\tR\nobjectType\x12\x1f\n\x0bobject_name\x18\x02 \x01(\tR\nobjectName\x12\x1f\n\x0bstart_index\x18\x03 \x01(\x05R\nstartIndex\x12\x1d\n\nstop_index\x18\x04 \x01(\x05R\tstopIndex\x12\x1a\n\x08\x66ragment\x18\x05 \x01(\tR\x08\x66ragment\x12\x1b\n\tcall_site\x18\x06 \x01(\tR\x08\x63\x61llSite\x12\x18\n\x07summary\x18\x07 \x01(\tR\x07summary"%\n\x0b\x43ontextType\x12\x07\n\x03SQL\x10\x00\x12\r\n\tDATAFRAME\x10\x01\x1a\x99\x03\n\x0eSparkThrowable\x12$\n\x0b\x65rror_class\x18\x01 \x01(\tH\x00R\nerrorClass\x88\x01\x01\x12}\n\x12message_parameters\x18\x02 \x03(\x0b\x32N.spark.connect.FetchErrorDetailsResponse.SparkThrowable.MessageParametersEntryR\x11messageParameters\x12\\\n\x0equery_contexts\x18\x03 \x03(\x0b\x32\x35.spark.connect.FetchErrorDetailsResponse.QueryContextR\rqueryContexts\x12 \n\tsql_state\x18\x04 \x01(\tH\x01R\x08sqlState\x88\x01\x01\x1a\x44\n\x16MessageParametersEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x0e\n\x0c_error_classB\x0c\n\n_sql_state\x1a\xdb\x02\n\x05\x45rror\x12\x30\n\x14\x65rror_type_hierarchy\x18\x01 \x03(\tR\x12\x65rrorTypeHierarchy\x12\x18\n\x07message\x18\x02 \x01(\tR\x07message\x12[\n\x0bstack_trace\x18\x03 \x03(\x0b\x32:.spark.connect.FetchErrorDetailsResponse.StackTraceElementR\nstackTrace\x12 \n\tcause_idx\x18\x04 \x01(\x05H\x00R\x08\x63\x61useIdx\x88\x01\x01\x12\x65\n\x0fspark_throwable\x18\x05 \x01(\x0b\x32\x37.spark.connect.FetchErrorDetailsResponse.SparkThrowableH\x01R\x0esparkThrowable\x88\x01\x01\x42\x0c\n\n_cause_idxB\x12\n\x10_spark_throwableB\x11\n\x0f_root_error_idx"Z\n\x17\x43heckpointCommandResult\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation2\xb2\x07\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x12[\n\x0c\x41\x64\x64\x41rtifacts\x12".spark.connect.AddArtifactsRequest\x1a#.spark.connect.AddArtifactsResponse"\x00(\x01\x12\x63\n\x0e\x41rtifactStatus\x12&.spark.connect.ArtifactStatusesRequest\x1a\'.spark.connect.ArtifactStatusesResponse"\x00\x12P\n\tInterrupt\x12\x1f.spark.connect.InterruptRequest\x1a .spark.connect.InterruptResponse"\x00\x12`\n\x0fReattachExecute\x12%.spark.connect.ReattachExecuteRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12_\n\x0eReleaseExecute\x12$.spark.connect.ReleaseExecuteRequest\x1a%.spark.connect.ReleaseExecuteResponse"\x00\x12_\n\x0eReleaseSession\x12$.spark.connect.ReleaseSessionRequest\x1a%.spark.connect.ReleaseSessionResponse"\x00\x12h\n\x11\x46\x65tchErrorDetails\x12\'.spark.connect.FetchErrorDetailsRequest\x1a(.spark.connect.FetchErrorDetailsResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -68,190 +69,190 @@ _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" ]._serialized_options = b"8\001" - _globals["_PLAN"]._serialized_start = 219 - _globals["_PLAN"]._serialized_end = 335 - _globals["_USERCONTEXT"]._serialized_start = 337 - _globals["_USERCONTEXT"]._serialized_end = 459 - _globals["_ANALYZEPLANREQUEST"]._serialized_start = 462 - _globals["_ANALYZEPLANREQUEST"]._serialized_end = 3139 - _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_start = 1824 - _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_end = 1873 - _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_start = 1876 - _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_end = 2191 - _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_start = 2019 - _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_end = 2191 - _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_start = 2193 - _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_end = 2283 - _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_start = 2285 - _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_end = 2335 - _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_start = 2337 - _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_end = 2391 - _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_start = 2393 - _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_end = 2446 - _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_start = 2448 - _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_end = 2462 - _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_start = 2464 - _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_end = 2505 - _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_start = 2507 - _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_end = 2628 - _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_start = 2630 - _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_end = 2685 - _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_start = 2688 - _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_end = 2839 - _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_start = 2841 - _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_end = 2951 - _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_start = 2953 - _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_end = 3023 - _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_start = 3025 - _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_end = 3069 - _globals["_ANALYZEPLANRESPONSE"]._serialized_start = 3142 - _globals["_ANALYZEPLANRESPONSE"]._serialized_end = 5008 - _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_start = 4383 - _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_end = 4440 - _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_start = 4442 - _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_end = 4490 - _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_start = 4492 - _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_end = 4537 - _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_start = 4539 - _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_end = 4575 - _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_start = 4577 - _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_end = 4625 - _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_start = 4627 - _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_end = 4661 - _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_start = 4663 - _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_end = 4703 - _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_start = 4705 - _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_end = 4764 - _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_start = 4766 - _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_end = 4805 - _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_start = 4807 - _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_end = 4845 - _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_start = 2688 - _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_end = 2697 - _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_start = 2841 - _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_end = 2852 - _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_start = 4871 - _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_end = 4954 - _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_start = 4956 - _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_end = 4998 - _globals["_EXECUTEPLANREQUEST"]._serialized_start = 5011 - _globals["_EXECUTEPLANREQUEST"]._serialized_end = 5686 - _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_start = 5445 - _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_end = 5610 - _globals["_EXECUTEPLANRESPONSE"]._serialized_start = 5689 - _globals["_EXECUTEPLANRESPONSE"]._serialized_end = 8607 - _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_start = 7381 - _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_end = 7452 - _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_start = 7454 - _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_end = 7572 - _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_start = 7575 - _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_end = 8092 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_start = 7670 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_end = 8002 + _globals["_PLAN"]._serialized_start = 243 + _globals["_PLAN"]._serialized_end = 359 + _globals["_USERCONTEXT"]._serialized_start = 361 + _globals["_USERCONTEXT"]._serialized_end = 483 + _globals["_ANALYZEPLANREQUEST"]._serialized_start = 486 + _globals["_ANALYZEPLANREQUEST"]._serialized_end = 3163 + _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_start = 1848 + _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_end = 1897 + _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_start = 1900 + _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_end = 2215 + _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_start = 2043 + _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_end = 2215 + _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_start = 2217 + _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_end = 2307 + _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_start = 2309 + _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_end = 2359 + _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_start = 2361 + _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_end = 2415 + _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_start = 2417 + _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_end = 2470 + _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_start = 2472 + _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_end = 2486 + _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_start = 2488 + _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_end = 2529 + _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_start = 2531 + _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_end = 2652 + _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_start = 2654 + _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_end = 2709 + _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_start = 2712 + _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_end = 2863 + _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_start = 2865 + _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_end = 2975 + _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_start = 2977 + _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_end = 3047 + _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_start = 3049 + _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_end = 3093 + _globals["_ANALYZEPLANRESPONSE"]._serialized_start = 3166 + _globals["_ANALYZEPLANRESPONSE"]._serialized_end = 5032 + _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_start = 4407 + _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_end = 4464 + _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_start = 4466 + _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_end = 4514 + _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_start = 4516 + _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_end = 4561 + _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_start = 4563 + _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_end = 4599 + _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_start = 4601 + _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_end = 4649 + _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_start = 4651 + _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_end = 4685 + _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_start = 4687 + _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_end = 4727 + _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_start = 4729 + _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_end = 4788 + _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_start = 4790 + _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_end = 4829 + _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_start = 4831 + _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_end = 4869 + _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_start = 2712 + _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_end = 2721 + _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_start = 2865 + _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_end = 2876 + _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_start = 4895 + _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_end = 4978 + _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_start = 4980 + _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_end = 5022 + _globals["_EXECUTEPLANREQUEST"]._serialized_start = 5035 + _globals["_EXECUTEPLANREQUEST"]._serialized_end = 5710 + _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_start = 5469 + _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_end = 5634 + _globals["_EXECUTEPLANRESPONSE"]._serialized_start = 5713 + _globals["_EXECUTEPLANRESPONSE"]._serialized_end = 8709 + _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_start = 7483 + _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_end = 7554 + _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_start = 7556 + _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_end = 7674 + _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_start = 7677 + _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_end = 8194 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_start = 7772 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_end = 8104 _globals[ "_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY" - ]._serialized_start = 7879 + ]._serialized_start = 7981 _globals[ "_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY" - ]._serialized_end = 8002 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_start = 8004 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_end = 8092 - _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_start = 8095 - _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_end = 8236 - _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_start = 8238 - _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_end = 8254 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_start = 8257 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_end = 8590 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_start = 8413 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_end = 8590 - _globals["_KEYVALUE"]._serialized_start = 8609 - _globals["_KEYVALUE"]._serialized_end = 8674 - _globals["_CONFIGREQUEST"]._serialized_start = 8677 - _globals["_CONFIGREQUEST"]._serialized_end = 9876 - _globals["_CONFIGREQUEST_OPERATION"]._serialized_start = 8985 - _globals["_CONFIGREQUEST_OPERATION"]._serialized_end = 9483 - _globals["_CONFIGREQUEST_SET"]._serialized_start = 9485 - _globals["_CONFIGREQUEST_SET"]._serialized_end = 9577 - _globals["_CONFIGREQUEST_GET"]._serialized_start = 9579 - _globals["_CONFIGREQUEST_GET"]._serialized_end = 9604 - _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_start = 9606 - _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_end = 9669 - _globals["_CONFIGREQUEST_GETOPTION"]._serialized_start = 9671 - _globals["_CONFIGREQUEST_GETOPTION"]._serialized_end = 9702 - _globals["_CONFIGREQUEST_GETALL"]._serialized_start = 9704 - _globals["_CONFIGREQUEST_GETALL"]._serialized_end = 9752 - _globals["_CONFIGREQUEST_UNSET"]._serialized_start = 9754 - _globals["_CONFIGREQUEST_UNSET"]._serialized_end = 9781 - _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_start = 9783 - _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_end = 9817 - _globals["_CONFIGRESPONSE"]._serialized_start = 9879 - _globals["_CONFIGRESPONSE"]._serialized_end = 10054 - _globals["_ADDARTIFACTSREQUEST"]._serialized_start = 10057 - _globals["_ADDARTIFACTSREQUEST"]._serialized_end = 11059 - _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_start = 10532 - _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_end = 10585 - _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_start = 10587 - _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_end = 10698 - _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_start = 10700 - _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_end = 10793 - _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_start = 10796 - _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_end = 10989 - _globals["_ADDARTIFACTSRESPONSE"]._serialized_start = 11062 - _globals["_ADDARTIFACTSRESPONSE"]._serialized_end = 11334 - _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_start = 11253 - _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_end = 11334 - _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_start = 11337 - _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_end = 11663 - _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_start = 11666 - _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_end = 12018 - _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_start = 11861 - _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_end = 11976 - _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_start = 11978 - _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_end = 12018 - _globals["_INTERRUPTREQUEST"]._serialized_start = 12021 - _globals["_INTERRUPTREQUEST"]._serialized_end = 12624 - _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_start = 12424 - _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_end = 12552 - _globals["_INTERRUPTRESPONSE"]._serialized_start = 12627 - _globals["_INTERRUPTRESPONSE"]._serialized_end = 12771 - _globals["_REATTACHOPTIONS"]._serialized_start = 12773 - _globals["_REATTACHOPTIONS"]._serialized_end = 12826 - _globals["_REATTACHEXECUTEREQUEST"]._serialized_start = 12829 - _globals["_REATTACHEXECUTEREQUEST"]._serialized_end = 13235 - _globals["_RELEASEEXECUTEREQUEST"]._serialized_start = 13238 - _globals["_RELEASEEXECUTEREQUEST"]._serialized_end = 13823 - _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_start = 13692 - _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_end = 13704 - _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_start = 13706 - _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_end = 13753 - _globals["_RELEASEEXECUTERESPONSE"]._serialized_start = 13826 - _globals["_RELEASEEXECUTERESPONSE"]._serialized_end = 13991 - _globals["_RELEASESESSIONREQUEST"]._serialized_start = 13994 - _globals["_RELEASESESSIONREQUEST"]._serialized_end = 14206 - _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 14208 - _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 14316 - _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 14319 - _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 14651 - _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 14654 - _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 16209 - _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 14883 - _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 15057 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 15060 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 15428 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 15391 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 15428 - _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 15431 - _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 15840 + ]._serialized_end = 8104 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_start = 8106 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_end = 8194 + _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_start = 8197 + _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_end = 8338 + _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_start = 8340 + _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_end = 8356 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_start = 8359 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_end = 8692 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_start = 8515 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_end = 8692 + _globals["_KEYVALUE"]._serialized_start = 8711 + _globals["_KEYVALUE"]._serialized_end = 8776 + _globals["_CONFIGREQUEST"]._serialized_start = 8779 + _globals["_CONFIGREQUEST"]._serialized_end = 9978 + _globals["_CONFIGREQUEST_OPERATION"]._serialized_start = 9087 + _globals["_CONFIGREQUEST_OPERATION"]._serialized_end = 9585 + _globals["_CONFIGREQUEST_SET"]._serialized_start = 9587 + _globals["_CONFIGREQUEST_SET"]._serialized_end = 9679 + _globals["_CONFIGREQUEST_GET"]._serialized_start = 9681 + _globals["_CONFIGREQUEST_GET"]._serialized_end = 9706 + _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_start = 9708 + _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_end = 9771 + _globals["_CONFIGREQUEST_GETOPTION"]._serialized_start = 9773 + _globals["_CONFIGREQUEST_GETOPTION"]._serialized_end = 9804 + _globals["_CONFIGREQUEST_GETALL"]._serialized_start = 9806 + _globals["_CONFIGREQUEST_GETALL"]._serialized_end = 9854 + _globals["_CONFIGREQUEST_UNSET"]._serialized_start = 9856 + _globals["_CONFIGREQUEST_UNSET"]._serialized_end = 9883 + _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_start = 9885 + _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_end = 9919 + _globals["_CONFIGRESPONSE"]._serialized_start = 9981 + _globals["_CONFIGRESPONSE"]._serialized_end = 10156 + _globals["_ADDARTIFACTSREQUEST"]._serialized_start = 10159 + _globals["_ADDARTIFACTSREQUEST"]._serialized_end = 11161 + _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_start = 10634 + _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_end = 10687 + _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_start = 10689 + _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_end = 10800 + _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_start = 10802 + _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_end = 10895 + _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_start = 10898 + _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_end = 11091 + _globals["_ADDARTIFACTSRESPONSE"]._serialized_start = 11164 + _globals["_ADDARTIFACTSRESPONSE"]._serialized_end = 11436 + _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_start = 11355 + _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_end = 11436 + _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_start = 11439 + _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_end = 11765 + _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_start = 11768 + _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_end = 12120 + _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_start = 11963 + _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_end = 12078 + _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_start = 12080 + _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_end = 12120 + _globals["_INTERRUPTREQUEST"]._serialized_start = 12123 + _globals["_INTERRUPTREQUEST"]._serialized_end = 12726 + _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_start = 12526 + _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_end = 12654 + _globals["_INTERRUPTRESPONSE"]._serialized_start = 12729 + _globals["_INTERRUPTRESPONSE"]._serialized_end = 12873 + _globals["_REATTACHOPTIONS"]._serialized_start = 12875 + _globals["_REATTACHOPTIONS"]._serialized_end = 12928 + _globals["_REATTACHEXECUTEREQUEST"]._serialized_start = 12931 + _globals["_REATTACHEXECUTEREQUEST"]._serialized_end = 13337 + _globals["_RELEASEEXECUTEREQUEST"]._serialized_start = 13340 + _globals["_RELEASEEXECUTEREQUEST"]._serialized_end = 13925 + _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_start = 13794 + _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_end = 13806 + _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_start = 13808 + _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_end = 13855 + _globals["_RELEASEEXECUTERESPONSE"]._serialized_start = 13928 + _globals["_RELEASEEXECUTERESPONSE"]._serialized_end = 14093 + _globals["_RELEASESESSIONREQUEST"]._serialized_start = 14096 + _globals["_RELEASESESSIONREQUEST"]._serialized_end = 14308 + _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 14310 + _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 14418 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 14421 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 14753 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 14756 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 16311 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 14985 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 15159 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 15162 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 15530 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 15493 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 15530 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 15533 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 15942 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_start = 15742 + ]._serialized_start = 15844 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_end = 15810 - _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 15843 - _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 16190 - _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 16211 - _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 16301 - _globals["_SPARKCONNECTSERVICE"]._serialized_start = 16304 - _globals["_SPARKCONNECTSERVICE"]._serialized_end = 17250 + ]._serialized_end = 15912 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 15945 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 16292 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 16313 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 16403 + _globals["_SPARKCONNECTSERVICE"]._serialized_start = 16406 + _globals["_SPARKCONNECTSERVICE"]._serialized_end = 17352 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index adea62ed6b805..738339fa968ec 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -43,6 +43,7 @@ import google.protobuf.message import pyspark.sql.connect.proto.commands_pb2 import pyspark.sql.connect.proto.common_pb2 import pyspark.sql.connect.proto.expressions_pb2 +import pyspark.sql.connect.proto.ml_pb2 import pyspark.sql.connect.proto.relations_pb2 import pyspark.sql.connect.proto.types_pb2 import sys @@ -1581,6 +1582,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): CREATE_RESOURCE_PROFILE_COMMAND_RESULT_FIELD_NUMBER: builtins.int EXECUTION_PROGRESS_FIELD_NUMBER: builtins.int CHECKPOINT_COMMAND_RESULT_FIELD_NUMBER: builtins.int + ML_COMMAND_RESULT_FIELD_NUMBER: builtins.int EXTENSION_FIELD_NUMBER: builtins.int METRICS_FIELD_NUMBER: builtins.int OBSERVED_METRICS_FIELD_NUMBER: builtins.int @@ -1645,6 +1647,9 @@ class ExecutePlanResponse(google.protobuf.message.Message): def checkpoint_command_result(self) -> global___CheckpointCommandResult: """Response for command that checkpoints a DataFrame.""" @property + def ml_command_result(self) -> pyspark.sql.connect.proto.ml_pb2.MlCommandResult: + """ML command response""" + @property def extension(self) -> google.protobuf.any_pb2.Any: """Support arbitrary result objects.""" @property @@ -1686,6 +1691,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): | None = ..., execution_progress: global___ExecutePlanResponse.ExecutionProgress | None = ..., checkpoint_command_result: global___CheckpointCommandResult | None = ..., + ml_command_result: pyspark.sql.connect.proto.ml_pb2.MlCommandResult | None = ..., extension: google.protobuf.any_pb2.Any | None = ..., metrics: global___ExecutePlanResponse.Metrics | None = ..., observed_metrics: collections.abc.Iterable[global___ExecutePlanResponse.ObservedMetrics] @@ -1709,6 +1715,8 @@ class ExecutePlanResponse(google.protobuf.message.Message): b"get_resources_command_result", "metrics", b"metrics", + "ml_command_result", + b"ml_command_result", "response_type", b"response_type", "result_complete", @@ -1744,6 +1752,8 @@ class ExecutePlanResponse(google.protobuf.message.Message): b"get_resources_command_result", "metrics", b"metrics", + "ml_command_result", + b"ml_command_result", "observed_metrics", b"observed_metrics", "operation_id", @@ -1787,6 +1797,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): "create_resource_profile_command_result", "execution_progress", "checkpoint_command_result", + "ml_command_result", "extension", ] | None diff --git a/python/pyspark/sql/connect/proto/commands_pb2.py b/python/pyspark/sql/connect/proto/commands_pb2.py index a7fcc1d7e0908..57a770f0226d9 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.py +++ b/python/pyspark/sql/connect/proto/commands_pb2.py @@ -38,10 +38,11 @@ from pyspark.sql.connect.proto import common_pb2 as spark_dot_connect_dot_common__pb2 from pyspark.sql.connect.proto import expressions_pb2 as spark_dot_connect_dot_expressions__pb2 from pyspark.sql.connect.proto import relations_pb2 as spark_dot_connect_dot_relations__pb2 +from pyspark.sql.connect.proto import ml_pb2 as spark_dot_connect_dot_ml__pb2 DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1cspark/connect/commands.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto"\x90\r\n\x07\x43ommand\x12]\n\x11register_function\x18\x01 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x10registerFunction\x12H\n\x0fwrite_operation\x18\x02 \x01(\x0b\x32\x1d.spark.connect.WriteOperationH\x00R\x0ewriteOperation\x12_\n\x15\x63reate_dataframe_view\x18\x03 \x01(\x0b\x32).spark.connect.CreateDataFrameViewCommandH\x00R\x13\x63reateDataframeView\x12O\n\x12write_operation_v2\x18\x04 \x01(\x0b\x32\x1f.spark.connect.WriteOperationV2H\x00R\x10writeOperationV2\x12<\n\x0bsql_command\x18\x05 \x01(\x0b\x32\x19.spark.connect.SqlCommandH\x00R\nsqlCommand\x12k\n\x1cwrite_stream_operation_start\x18\x06 \x01(\x0b\x32(.spark.connect.WriteStreamOperationStartH\x00R\x19writeStreamOperationStart\x12^\n\x17streaming_query_command\x18\x07 \x01(\x0b\x32$.spark.connect.StreamingQueryCommandH\x00R\x15streamingQueryCommand\x12X\n\x15get_resources_command\x18\x08 \x01(\x0b\x32".spark.connect.GetResourcesCommandH\x00R\x13getResourcesCommand\x12t\n\x1fstreaming_query_manager_command\x18\t \x01(\x0b\x32+.spark.connect.StreamingQueryManagerCommandH\x00R\x1cstreamingQueryManagerCommand\x12m\n\x17register_table_function\x18\n \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R\x15registerTableFunction\x12\x81\x01\n$streaming_query_listener_bus_command\x18\x0b \x01(\x0b\x32/.spark.connect.StreamingQueryListenerBusCommandH\x00R streamingQueryListenerBusCommand\x12\x64\n\x14register_data_source\x18\x0c \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R\x12registerDataSource\x12t\n\x1f\x63reate_resource_profile_command\x18\r \x01(\x0b\x32+.spark.connect.CreateResourceProfileCommandH\x00R\x1c\x63reateResourceProfileCommand\x12Q\n\x12\x63heckpoint_command\x18\x0e \x01(\x0b\x32 .spark.connect.CheckpointCommandH\x00R\x11\x63heckpointCommand\x12\x84\x01\n%remove_cached_remote_relation_command\x18\x0f \x01(\x0b\x32\x30.spark.connect.RemoveCachedRemoteRelationCommandH\x00R!removeCachedRemoteRelationCommand\x12_\n\x18merge_into_table_command\x18\x10 \x01(\x0b\x32$.spark.connect.MergeIntoTableCommandH\x00R\x15mergeIntoTableCommand\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x0e\n\x0c\x63ommand_type"\xaa\x04\n\nSqlCommand\x12\x14\n\x03sql\x18\x01 \x01(\tB\x02\x18\x01R\x03sql\x12;\n\x04\x61rgs\x18\x02 \x03(\x0b\x32#.spark.connect.SqlCommand.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12Z\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32-.spark.connect.SqlCommand.NamedArgumentsEntryB\x02\x18\x01R\x0enamedArguments\x12\x42\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionB\x02\x18\x01R\x0cposArguments\x12-\n\x05input\x18\x06 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"\x96\x01\n\x1a\x43reateDataFrameViewCommand\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x1b\n\tis_global\x18\x03 \x01(\x08R\x08isGlobal\x12\x18\n\x07replace\x18\x04 \x01(\x08R\x07replace"\xca\x08\n\x0eWriteOperation\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1b\n\x06source\x18\x02 \x01(\tH\x01R\x06source\x88\x01\x01\x12\x14\n\x04path\x18\x03 \x01(\tH\x00R\x04path\x12?\n\x05table\x18\x04 \x01(\x0b\x32\'.spark.connect.WriteOperation.SaveTableH\x00R\x05table\x12:\n\x04mode\x18\x05 \x01(\x0e\x32&.spark.connect.WriteOperation.SaveModeR\x04mode\x12*\n\x11sort_column_names\x18\x06 \x03(\tR\x0fsortColumnNames\x12\x31\n\x14partitioning_columns\x18\x07 \x03(\tR\x13partitioningColumns\x12\x43\n\tbucket_by\x18\x08 \x01(\x0b\x32&.spark.connect.WriteOperation.BucketByR\x08\x62ucketBy\x12\x44\n\x07options\x18\t \x03(\x0b\x32*.spark.connect.WriteOperation.OptionsEntryR\x07options\x12-\n\x12\x63lustering_columns\x18\n \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x82\x02\n\tSaveTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12X\n\x0bsave_method\x18\x02 \x01(\x0e\x32\x37.spark.connect.WriteOperation.SaveTable.TableSaveMethodR\nsaveMethod"|\n\x0fTableSaveMethod\x12!\n\x1dTABLE_SAVE_METHOD_UNSPECIFIED\x10\x00\x12#\n\x1fTABLE_SAVE_METHOD_SAVE_AS_TABLE\x10\x01\x12!\n\x1dTABLE_SAVE_METHOD_INSERT_INTO\x10\x02\x1a[\n\x08\x42ucketBy\x12.\n\x13\x62ucket_column_names\x18\x01 \x03(\tR\x11\x62ucketColumnNames\x12\x1f\n\x0bnum_buckets\x18\x02 \x01(\x05R\nnumBuckets"\x89\x01\n\x08SaveMode\x12\x19\n\x15SAVE_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10SAVE_MODE_APPEND\x10\x01\x12\x17\n\x13SAVE_MODE_OVERWRITE\x10\x02\x12\x1d\n\x19SAVE_MODE_ERROR_IF_EXISTS\x10\x03\x12\x14\n\x10SAVE_MODE_IGNORE\x10\x04\x42\x0b\n\tsave_typeB\t\n\x07_source"\xdc\x06\n\x10WriteOperationV2\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\ntable_name\x18\x02 \x01(\tR\ttableName\x12\x1f\n\x08provider\x18\x03 \x01(\tH\x00R\x08provider\x88\x01\x01\x12L\n\x14partitioning_columns\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13partitioningColumns\x12\x46\n\x07options\x18\x05 \x03(\x0b\x32,.spark.connect.WriteOperationV2.OptionsEntryR\x07options\x12_\n\x10table_properties\x18\x06 \x03(\x0b\x32\x34.spark.connect.WriteOperationV2.TablePropertiesEntryR\x0ftableProperties\x12\x38\n\x04mode\x18\x07 \x01(\x0e\x32$.spark.connect.WriteOperationV2.ModeR\x04mode\x12J\n\x13overwrite_condition\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x12overwriteCondition\x12-\n\x12\x63lustering_columns\x18\t \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"\x9f\x01\n\x04Mode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\x0f\n\x0bMODE_CREATE\x10\x01\x12\x12\n\x0eMODE_OVERWRITE\x10\x02\x12\x1d\n\x19MODE_OVERWRITE_PARTITIONS\x10\x03\x12\x0f\n\x0bMODE_APPEND\x10\x04\x12\x10\n\x0cMODE_REPLACE\x10\x05\x12\x1a\n\x16MODE_CREATE_OR_REPLACE\x10\x06\x42\x0b\n\t_provider"\xd8\x06\n\x19WriteStreamOperationStart\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06\x66ormat\x18\x02 \x01(\tR\x06\x66ormat\x12O\n\x07options\x18\x03 \x03(\x0b\x32\x35.spark.connect.WriteStreamOperationStart.OptionsEntryR\x07options\x12:\n\x19partitioning_column_names\x18\x04 \x03(\tR\x17partitioningColumnNames\x12:\n\x18processing_time_interval\x18\x05 \x01(\tH\x00R\x16processingTimeInterval\x12%\n\ravailable_now\x18\x06 \x01(\x08H\x00R\x0c\x61vailableNow\x12\x14\n\x04once\x18\x07 \x01(\x08H\x00R\x04once\x12\x46\n\x1e\x63ontinuous_checkpoint_interval\x18\x08 \x01(\tH\x00R\x1c\x63ontinuousCheckpointInterval\x12\x1f\n\x0boutput_mode\x18\t \x01(\tR\noutputMode\x12\x1d\n\nquery_name\x18\n \x01(\tR\tqueryName\x12\x14\n\x04path\x18\x0b \x01(\tH\x01R\x04path\x12\x1f\n\ntable_name\x18\x0c \x01(\tH\x01R\ttableName\x12N\n\x0e\x66oreach_writer\x18\r \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\rforeachWriter\x12L\n\rforeach_batch\x18\x0e \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\x0c\x66oreachBatch\x12\x36\n\x17\x63lustering_column_names\x18\x0f \x03(\tR\x15\x63lusteringColumnNames\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07triggerB\x12\n\x10sink_destination"\xb3\x01\n\x18StreamingForeachFunction\x12\x43\n\x0fpython_function\x18\x01 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x0epythonFunction\x12\x46\n\x0escala_function\x18\x02 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\rscalaFunctionB\n\n\x08\x66unction"\xd4\x01\n\x1fWriteStreamOperationStartResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12<\n\x18query_started_event_json\x18\x03 \x01(\tH\x00R\x15queryStartedEventJson\x88\x01\x01\x42\x1b\n\x19_query_started_event_json"A\n\x18StreamingQueryInstanceId\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id\x12\x15\n\x06run_id\x18\x02 \x01(\tR\x05runId"\xf8\x04\n\x15StreamingQueryCommand\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x18\n\x06status\x18\x02 \x01(\x08H\x00R\x06status\x12%\n\rlast_progress\x18\x03 \x01(\x08H\x00R\x0clastProgress\x12)\n\x0frecent_progress\x18\x04 \x01(\x08H\x00R\x0erecentProgress\x12\x14\n\x04stop\x18\x05 \x01(\x08H\x00R\x04stop\x12\x34\n\x15process_all_available\x18\x06 \x01(\x08H\x00R\x13processAllAvailable\x12O\n\x07\x65xplain\x18\x07 \x01(\x0b\x32\x33.spark.connect.StreamingQueryCommand.ExplainCommandH\x00R\x07\x65xplain\x12\x1e\n\texception\x18\x08 \x01(\x08H\x00R\texception\x12k\n\x11\x61wait_termination\x18\t \x01(\x0b\x32<.spark.connect.StreamingQueryCommand.AwaitTerminationCommandH\x00R\x10\x61waitTermination\x1a,\n\x0e\x45xplainCommand\x12\x1a\n\x08\x65xtended\x18\x01 \x01(\x08R\x08\x65xtended\x1aL\n\x17\x41waitTerminationCommand\x12"\n\ntimeout_ms\x18\x02 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_msB\t\n\x07\x63ommand"\xf5\x08\n\x1bStreamingQueryCommandResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12Q\n\x06status\x18\x02 \x01(\x0b\x32\x37.spark.connect.StreamingQueryCommandResult.StatusResultH\x00R\x06status\x12j\n\x0frecent_progress\x18\x03 \x01(\x0b\x32?.spark.connect.StreamingQueryCommandResult.RecentProgressResultH\x00R\x0erecentProgress\x12T\n\x07\x65xplain\x18\x04 \x01(\x0b\x32\x38.spark.connect.StreamingQueryCommandResult.ExplainResultH\x00R\x07\x65xplain\x12Z\n\texception\x18\x05 \x01(\x0b\x32:.spark.connect.StreamingQueryCommandResult.ExceptionResultH\x00R\texception\x12p\n\x11\x61wait_termination\x18\x06 \x01(\x0b\x32\x41.spark.connect.StreamingQueryCommandResult.AwaitTerminationResultH\x00R\x10\x61waitTermination\x1a\xaa\x01\n\x0cStatusResult\x12%\n\x0estatus_message\x18\x01 \x01(\tR\rstatusMessage\x12*\n\x11is_data_available\x18\x02 \x01(\x08R\x0fisDataAvailable\x12*\n\x11is_trigger_active\x18\x03 \x01(\x08R\x0fisTriggerActive\x12\x1b\n\tis_active\x18\x04 \x01(\x08R\x08isActive\x1aH\n\x14RecentProgressResult\x12\x30\n\x14recent_progress_json\x18\x05 \x03(\tR\x12recentProgressJson\x1a\'\n\rExplainResult\x12\x16\n\x06result\x18\x01 \x01(\tR\x06result\x1a\xc5\x01\n\x0f\x45xceptionResult\x12\x30\n\x11\x65xception_message\x18\x01 \x01(\tH\x00R\x10\x65xceptionMessage\x88\x01\x01\x12$\n\x0b\x65rror_class\x18\x02 \x01(\tH\x01R\nerrorClass\x88\x01\x01\x12$\n\x0bstack_trace\x18\x03 \x01(\tH\x02R\nstackTrace\x88\x01\x01\x42\x14\n\x12_exception_messageB\x0e\n\x0c_error_classB\x0e\n\x0c_stack_trace\x1a\x38\n\x16\x41waitTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminatedB\r\n\x0bresult_type"\xbd\x06\n\x1cStreamingQueryManagerCommand\x12\x18\n\x06\x61\x63tive\x18\x01 \x01(\x08H\x00R\x06\x61\x63tive\x12\x1d\n\tget_query\x18\x02 \x01(\tH\x00R\x08getQuery\x12|\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32\x46.spark.connect.StreamingQueryManagerCommand.AwaitAnyTerminationCommandH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12n\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0b\x61\x64\x64Listener\x12t\n\x0fremove_listener\x18\x06 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0eremoveListener\x12\'\n\x0elist_listeners\x18\x07 \x01(\x08H\x00R\rlistListeners\x1aO\n\x1a\x41waitAnyTerminationCommand\x12"\n\ntimeout_ms\x18\x01 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_ms\x1a\xcd\x01\n\x1dStreamingQueryListenerCommand\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x12U\n\x17python_listener_payload\x18\x02 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x15pythonListenerPayload\x88\x01\x01\x12\x0e\n\x02id\x18\x03 \x01(\tR\x02idB\x1a\n\x18_python_listener_payloadB\t\n\x07\x63ommand"\xb4\x08\n"StreamingQueryManagerCommandResult\x12X\n\x06\x61\x63tive\x18\x01 \x01(\x0b\x32>.spark.connect.StreamingQueryManagerCommandResult.ActiveResultH\x00R\x06\x61\x63tive\x12`\n\x05query\x18\x02 \x01(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceH\x00R\x05query\x12\x81\x01\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32K.spark.connect.StreamingQueryManagerCommandResult.AwaitAnyTerminationResultH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12#\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x08H\x00R\x0b\x61\x64\x64Listener\x12)\n\x0fremove_listener\x18\x06 \x01(\x08H\x00R\x0eremoveListener\x12{\n\x0elist_listeners\x18\x07 \x01(\x0b\x32R.spark.connect.StreamingQueryManagerCommandResult.ListStreamingQueryListenerResultH\x00R\rlistListeners\x1a\x7f\n\x0c\x41\x63tiveResult\x12o\n\x0e\x61\x63tive_queries\x18\x01 \x03(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceR\ractiveQueries\x1as\n\x16StreamingQueryInstance\x12\x37\n\x02id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x02id\x12\x17\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x88\x01\x01\x42\x07\n\x05_name\x1a;\n\x19\x41waitAnyTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminated\x1aK\n\x1eStreamingQueryListenerInstance\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x1a\x45\n ListStreamingQueryListenerResult\x12!\n\x0clistener_ids\x18\x01 \x03(\tR\x0blistenerIdsB\r\n\x0bresult_type"\xad\x01\n StreamingQueryListenerBusCommand\x12;\n\x19\x61\x64\x64_listener_bus_listener\x18\x01 \x01(\x08H\x00R\x16\x61\x64\x64ListenerBusListener\x12\x41\n\x1cremove_listener_bus_listener\x18\x02 \x01(\x08H\x00R\x19removeListenerBusListenerB\t\n\x07\x63ommand"\x83\x01\n\x1bStreamingQueryListenerEvent\x12\x1d\n\nevent_json\x18\x01 \x01(\tR\teventJson\x12\x45\n\nevent_type\x18\x02 \x01(\x0e\x32&.spark.connect.StreamingQueryEventTypeR\teventType"\xcc\x01\n"StreamingQueryListenerEventsResult\x12\x42\n\x06\x65vents\x18\x01 \x03(\x0b\x32*.spark.connect.StreamingQueryListenerEventR\x06\x65vents\x12\x42\n\x1blistener_bus_listener_added\x18\x02 \x01(\x08H\x00R\x18listenerBusListenerAdded\x88\x01\x01\x42\x1e\n\x1c_listener_bus_listener_added"\x15\n\x13GetResourcesCommand"\xd4\x01\n\x19GetResourcesCommandResult\x12U\n\tresources\x18\x01 \x03(\x0b\x32\x37.spark.connect.GetResourcesCommandResult.ResourcesEntryR\tresources\x1a`\n\x0eResourcesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32".spark.connect.ResourceInformationR\x05value:\x02\x38\x01"X\n\x1c\x43reateResourceProfileCommand\x12\x38\n\x07profile\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ResourceProfileR\x07profile"C\n"CreateResourceProfileCommandResult\x12\x1d\n\nprofile_id\x18\x01 \x01(\x05R\tprofileId"d\n!RemoveCachedRemoteRelationCommand\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation"\xcd\x01\n\x11\x43heckpointCommand\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x14\n\x05local\x18\x02 \x01(\x08R\x05local\x12\x14\n\x05\x65\x61ger\x18\x03 \x01(\x08R\x05\x65\x61ger\x12\x45\n\rstorage_level\x18\x04 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level"\xe8\x03\n\x15MergeIntoTableCommand\x12*\n\x11target_table_name\x18\x01 \x01(\tR\x0ftargetTableName\x12\x43\n\x11source_table_plan\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x0fsourceTablePlan\x12\x42\n\x0fmerge_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0emergeCondition\x12>\n\rmatch_actions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cmatchActions\x12I\n\x13not_matched_actions\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x11notMatchedActions\x12[\n\x1dnot_matched_by_source_actions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x19notMatchedBySourceActions\x12\x32\n\x15with_schema_evolution\x18\x07 \x01(\x08R\x13withSchemaEvolution*\x85\x01\n\x17StreamingQueryEventType\x12\x1e\n\x1aQUERY_PROGRESS_UNSPECIFIED\x10\x00\x12\x18\n\x14QUERY_PROGRESS_EVENT\x10\x01\x12\x1a\n\x16QUERY_TERMINATED_EVENT\x10\x02\x12\x14\n\x10QUERY_IDLE_EVENT\x10\x03\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1cspark/connect/commands.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x16spark/connect/ml.proto"\xcb\r\n\x07\x43ommand\x12]\n\x11register_function\x18\x01 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x10registerFunction\x12H\n\x0fwrite_operation\x18\x02 \x01(\x0b\x32\x1d.spark.connect.WriteOperationH\x00R\x0ewriteOperation\x12_\n\x15\x63reate_dataframe_view\x18\x03 \x01(\x0b\x32).spark.connect.CreateDataFrameViewCommandH\x00R\x13\x63reateDataframeView\x12O\n\x12write_operation_v2\x18\x04 \x01(\x0b\x32\x1f.spark.connect.WriteOperationV2H\x00R\x10writeOperationV2\x12<\n\x0bsql_command\x18\x05 \x01(\x0b\x32\x19.spark.connect.SqlCommandH\x00R\nsqlCommand\x12k\n\x1cwrite_stream_operation_start\x18\x06 \x01(\x0b\x32(.spark.connect.WriteStreamOperationStartH\x00R\x19writeStreamOperationStart\x12^\n\x17streaming_query_command\x18\x07 \x01(\x0b\x32$.spark.connect.StreamingQueryCommandH\x00R\x15streamingQueryCommand\x12X\n\x15get_resources_command\x18\x08 \x01(\x0b\x32".spark.connect.GetResourcesCommandH\x00R\x13getResourcesCommand\x12t\n\x1fstreaming_query_manager_command\x18\t \x01(\x0b\x32+.spark.connect.StreamingQueryManagerCommandH\x00R\x1cstreamingQueryManagerCommand\x12m\n\x17register_table_function\x18\n \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R\x15registerTableFunction\x12\x81\x01\n$streaming_query_listener_bus_command\x18\x0b \x01(\x0b\x32/.spark.connect.StreamingQueryListenerBusCommandH\x00R streamingQueryListenerBusCommand\x12\x64\n\x14register_data_source\x18\x0c \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R\x12registerDataSource\x12t\n\x1f\x63reate_resource_profile_command\x18\r \x01(\x0b\x32+.spark.connect.CreateResourceProfileCommandH\x00R\x1c\x63reateResourceProfileCommand\x12Q\n\x12\x63heckpoint_command\x18\x0e \x01(\x0b\x32 .spark.connect.CheckpointCommandH\x00R\x11\x63heckpointCommand\x12\x84\x01\n%remove_cached_remote_relation_command\x18\x0f \x01(\x0b\x32\x30.spark.connect.RemoveCachedRemoteRelationCommandH\x00R!removeCachedRemoteRelationCommand\x12_\n\x18merge_into_table_command\x18\x10 \x01(\x0b\x32$.spark.connect.MergeIntoTableCommandH\x00R\x15mergeIntoTableCommand\x12\x39\n\nml_command\x18\x11 \x01(\x0b\x32\x18.spark.connect.MlCommandH\x00R\tmlCommand\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x0e\n\x0c\x63ommand_type"\xaa\x04\n\nSqlCommand\x12\x14\n\x03sql\x18\x01 \x01(\tB\x02\x18\x01R\x03sql\x12;\n\x04\x61rgs\x18\x02 \x03(\x0b\x32#.spark.connect.SqlCommand.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12Z\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32-.spark.connect.SqlCommand.NamedArgumentsEntryB\x02\x18\x01R\x0enamedArguments\x12\x42\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionB\x02\x18\x01R\x0cposArguments\x12-\n\x05input\x18\x06 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"\x96\x01\n\x1a\x43reateDataFrameViewCommand\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x1b\n\tis_global\x18\x03 \x01(\x08R\x08isGlobal\x12\x18\n\x07replace\x18\x04 \x01(\x08R\x07replace"\xca\x08\n\x0eWriteOperation\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1b\n\x06source\x18\x02 \x01(\tH\x01R\x06source\x88\x01\x01\x12\x14\n\x04path\x18\x03 \x01(\tH\x00R\x04path\x12?\n\x05table\x18\x04 \x01(\x0b\x32\'.spark.connect.WriteOperation.SaveTableH\x00R\x05table\x12:\n\x04mode\x18\x05 \x01(\x0e\x32&.spark.connect.WriteOperation.SaveModeR\x04mode\x12*\n\x11sort_column_names\x18\x06 \x03(\tR\x0fsortColumnNames\x12\x31\n\x14partitioning_columns\x18\x07 \x03(\tR\x13partitioningColumns\x12\x43\n\tbucket_by\x18\x08 \x01(\x0b\x32&.spark.connect.WriteOperation.BucketByR\x08\x62ucketBy\x12\x44\n\x07options\x18\t \x03(\x0b\x32*.spark.connect.WriteOperation.OptionsEntryR\x07options\x12-\n\x12\x63lustering_columns\x18\n \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x82\x02\n\tSaveTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12X\n\x0bsave_method\x18\x02 \x01(\x0e\x32\x37.spark.connect.WriteOperation.SaveTable.TableSaveMethodR\nsaveMethod"|\n\x0fTableSaveMethod\x12!\n\x1dTABLE_SAVE_METHOD_UNSPECIFIED\x10\x00\x12#\n\x1fTABLE_SAVE_METHOD_SAVE_AS_TABLE\x10\x01\x12!\n\x1dTABLE_SAVE_METHOD_INSERT_INTO\x10\x02\x1a[\n\x08\x42ucketBy\x12.\n\x13\x62ucket_column_names\x18\x01 \x03(\tR\x11\x62ucketColumnNames\x12\x1f\n\x0bnum_buckets\x18\x02 \x01(\x05R\nnumBuckets"\x89\x01\n\x08SaveMode\x12\x19\n\x15SAVE_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10SAVE_MODE_APPEND\x10\x01\x12\x17\n\x13SAVE_MODE_OVERWRITE\x10\x02\x12\x1d\n\x19SAVE_MODE_ERROR_IF_EXISTS\x10\x03\x12\x14\n\x10SAVE_MODE_IGNORE\x10\x04\x42\x0b\n\tsave_typeB\t\n\x07_source"\xdc\x06\n\x10WriteOperationV2\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\ntable_name\x18\x02 \x01(\tR\ttableName\x12\x1f\n\x08provider\x18\x03 \x01(\tH\x00R\x08provider\x88\x01\x01\x12L\n\x14partitioning_columns\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13partitioningColumns\x12\x46\n\x07options\x18\x05 \x03(\x0b\x32,.spark.connect.WriteOperationV2.OptionsEntryR\x07options\x12_\n\x10table_properties\x18\x06 \x03(\x0b\x32\x34.spark.connect.WriteOperationV2.TablePropertiesEntryR\x0ftableProperties\x12\x38\n\x04mode\x18\x07 \x01(\x0e\x32$.spark.connect.WriteOperationV2.ModeR\x04mode\x12J\n\x13overwrite_condition\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x12overwriteCondition\x12-\n\x12\x63lustering_columns\x18\t \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"\x9f\x01\n\x04Mode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\x0f\n\x0bMODE_CREATE\x10\x01\x12\x12\n\x0eMODE_OVERWRITE\x10\x02\x12\x1d\n\x19MODE_OVERWRITE_PARTITIONS\x10\x03\x12\x0f\n\x0bMODE_APPEND\x10\x04\x12\x10\n\x0cMODE_REPLACE\x10\x05\x12\x1a\n\x16MODE_CREATE_OR_REPLACE\x10\x06\x42\x0b\n\t_provider"\xd8\x06\n\x19WriteStreamOperationStart\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06\x66ormat\x18\x02 \x01(\tR\x06\x66ormat\x12O\n\x07options\x18\x03 \x03(\x0b\x32\x35.spark.connect.WriteStreamOperationStart.OptionsEntryR\x07options\x12:\n\x19partitioning_column_names\x18\x04 \x03(\tR\x17partitioningColumnNames\x12:\n\x18processing_time_interval\x18\x05 \x01(\tH\x00R\x16processingTimeInterval\x12%\n\ravailable_now\x18\x06 \x01(\x08H\x00R\x0c\x61vailableNow\x12\x14\n\x04once\x18\x07 \x01(\x08H\x00R\x04once\x12\x46\n\x1e\x63ontinuous_checkpoint_interval\x18\x08 \x01(\tH\x00R\x1c\x63ontinuousCheckpointInterval\x12\x1f\n\x0boutput_mode\x18\t \x01(\tR\noutputMode\x12\x1d\n\nquery_name\x18\n \x01(\tR\tqueryName\x12\x14\n\x04path\x18\x0b \x01(\tH\x01R\x04path\x12\x1f\n\ntable_name\x18\x0c \x01(\tH\x01R\ttableName\x12N\n\x0e\x66oreach_writer\x18\r \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\rforeachWriter\x12L\n\rforeach_batch\x18\x0e \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\x0c\x66oreachBatch\x12\x36\n\x17\x63lustering_column_names\x18\x0f \x03(\tR\x15\x63lusteringColumnNames\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07triggerB\x12\n\x10sink_destination"\xb3\x01\n\x18StreamingForeachFunction\x12\x43\n\x0fpython_function\x18\x01 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x0epythonFunction\x12\x46\n\x0escala_function\x18\x02 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\rscalaFunctionB\n\n\x08\x66unction"\xd4\x01\n\x1fWriteStreamOperationStartResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12<\n\x18query_started_event_json\x18\x03 \x01(\tH\x00R\x15queryStartedEventJson\x88\x01\x01\x42\x1b\n\x19_query_started_event_json"A\n\x18StreamingQueryInstanceId\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id\x12\x15\n\x06run_id\x18\x02 \x01(\tR\x05runId"\xf8\x04\n\x15StreamingQueryCommand\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x18\n\x06status\x18\x02 \x01(\x08H\x00R\x06status\x12%\n\rlast_progress\x18\x03 \x01(\x08H\x00R\x0clastProgress\x12)\n\x0frecent_progress\x18\x04 \x01(\x08H\x00R\x0erecentProgress\x12\x14\n\x04stop\x18\x05 \x01(\x08H\x00R\x04stop\x12\x34\n\x15process_all_available\x18\x06 \x01(\x08H\x00R\x13processAllAvailable\x12O\n\x07\x65xplain\x18\x07 \x01(\x0b\x32\x33.spark.connect.StreamingQueryCommand.ExplainCommandH\x00R\x07\x65xplain\x12\x1e\n\texception\x18\x08 \x01(\x08H\x00R\texception\x12k\n\x11\x61wait_termination\x18\t \x01(\x0b\x32<.spark.connect.StreamingQueryCommand.AwaitTerminationCommandH\x00R\x10\x61waitTermination\x1a,\n\x0e\x45xplainCommand\x12\x1a\n\x08\x65xtended\x18\x01 \x01(\x08R\x08\x65xtended\x1aL\n\x17\x41waitTerminationCommand\x12"\n\ntimeout_ms\x18\x02 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_msB\t\n\x07\x63ommand"\xf5\x08\n\x1bStreamingQueryCommandResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12Q\n\x06status\x18\x02 \x01(\x0b\x32\x37.spark.connect.StreamingQueryCommandResult.StatusResultH\x00R\x06status\x12j\n\x0frecent_progress\x18\x03 \x01(\x0b\x32?.spark.connect.StreamingQueryCommandResult.RecentProgressResultH\x00R\x0erecentProgress\x12T\n\x07\x65xplain\x18\x04 \x01(\x0b\x32\x38.spark.connect.StreamingQueryCommandResult.ExplainResultH\x00R\x07\x65xplain\x12Z\n\texception\x18\x05 \x01(\x0b\x32:.spark.connect.StreamingQueryCommandResult.ExceptionResultH\x00R\texception\x12p\n\x11\x61wait_termination\x18\x06 \x01(\x0b\x32\x41.spark.connect.StreamingQueryCommandResult.AwaitTerminationResultH\x00R\x10\x61waitTermination\x1a\xaa\x01\n\x0cStatusResult\x12%\n\x0estatus_message\x18\x01 \x01(\tR\rstatusMessage\x12*\n\x11is_data_available\x18\x02 \x01(\x08R\x0fisDataAvailable\x12*\n\x11is_trigger_active\x18\x03 \x01(\x08R\x0fisTriggerActive\x12\x1b\n\tis_active\x18\x04 \x01(\x08R\x08isActive\x1aH\n\x14RecentProgressResult\x12\x30\n\x14recent_progress_json\x18\x05 \x03(\tR\x12recentProgressJson\x1a\'\n\rExplainResult\x12\x16\n\x06result\x18\x01 \x01(\tR\x06result\x1a\xc5\x01\n\x0f\x45xceptionResult\x12\x30\n\x11\x65xception_message\x18\x01 \x01(\tH\x00R\x10\x65xceptionMessage\x88\x01\x01\x12$\n\x0b\x65rror_class\x18\x02 \x01(\tH\x01R\nerrorClass\x88\x01\x01\x12$\n\x0bstack_trace\x18\x03 \x01(\tH\x02R\nstackTrace\x88\x01\x01\x42\x14\n\x12_exception_messageB\x0e\n\x0c_error_classB\x0e\n\x0c_stack_trace\x1a\x38\n\x16\x41waitTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminatedB\r\n\x0bresult_type"\xbd\x06\n\x1cStreamingQueryManagerCommand\x12\x18\n\x06\x61\x63tive\x18\x01 \x01(\x08H\x00R\x06\x61\x63tive\x12\x1d\n\tget_query\x18\x02 \x01(\tH\x00R\x08getQuery\x12|\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32\x46.spark.connect.StreamingQueryManagerCommand.AwaitAnyTerminationCommandH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12n\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0b\x61\x64\x64Listener\x12t\n\x0fremove_listener\x18\x06 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0eremoveListener\x12\'\n\x0elist_listeners\x18\x07 \x01(\x08H\x00R\rlistListeners\x1aO\n\x1a\x41waitAnyTerminationCommand\x12"\n\ntimeout_ms\x18\x01 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_ms\x1a\xcd\x01\n\x1dStreamingQueryListenerCommand\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x12U\n\x17python_listener_payload\x18\x02 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x15pythonListenerPayload\x88\x01\x01\x12\x0e\n\x02id\x18\x03 \x01(\tR\x02idB\x1a\n\x18_python_listener_payloadB\t\n\x07\x63ommand"\xb4\x08\n"StreamingQueryManagerCommandResult\x12X\n\x06\x61\x63tive\x18\x01 \x01(\x0b\x32>.spark.connect.StreamingQueryManagerCommandResult.ActiveResultH\x00R\x06\x61\x63tive\x12`\n\x05query\x18\x02 \x01(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceH\x00R\x05query\x12\x81\x01\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32K.spark.connect.StreamingQueryManagerCommandResult.AwaitAnyTerminationResultH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12#\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x08H\x00R\x0b\x61\x64\x64Listener\x12)\n\x0fremove_listener\x18\x06 \x01(\x08H\x00R\x0eremoveListener\x12{\n\x0elist_listeners\x18\x07 \x01(\x0b\x32R.spark.connect.StreamingQueryManagerCommandResult.ListStreamingQueryListenerResultH\x00R\rlistListeners\x1a\x7f\n\x0c\x41\x63tiveResult\x12o\n\x0e\x61\x63tive_queries\x18\x01 \x03(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceR\ractiveQueries\x1as\n\x16StreamingQueryInstance\x12\x37\n\x02id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x02id\x12\x17\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x88\x01\x01\x42\x07\n\x05_name\x1a;\n\x19\x41waitAnyTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminated\x1aK\n\x1eStreamingQueryListenerInstance\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x1a\x45\n ListStreamingQueryListenerResult\x12!\n\x0clistener_ids\x18\x01 \x03(\tR\x0blistenerIdsB\r\n\x0bresult_type"\xad\x01\n StreamingQueryListenerBusCommand\x12;\n\x19\x61\x64\x64_listener_bus_listener\x18\x01 \x01(\x08H\x00R\x16\x61\x64\x64ListenerBusListener\x12\x41\n\x1cremove_listener_bus_listener\x18\x02 \x01(\x08H\x00R\x19removeListenerBusListenerB\t\n\x07\x63ommand"\x83\x01\n\x1bStreamingQueryListenerEvent\x12\x1d\n\nevent_json\x18\x01 \x01(\tR\teventJson\x12\x45\n\nevent_type\x18\x02 \x01(\x0e\x32&.spark.connect.StreamingQueryEventTypeR\teventType"\xcc\x01\n"StreamingQueryListenerEventsResult\x12\x42\n\x06\x65vents\x18\x01 \x03(\x0b\x32*.spark.connect.StreamingQueryListenerEventR\x06\x65vents\x12\x42\n\x1blistener_bus_listener_added\x18\x02 \x01(\x08H\x00R\x18listenerBusListenerAdded\x88\x01\x01\x42\x1e\n\x1c_listener_bus_listener_added"\x15\n\x13GetResourcesCommand"\xd4\x01\n\x19GetResourcesCommandResult\x12U\n\tresources\x18\x01 \x03(\x0b\x32\x37.spark.connect.GetResourcesCommandResult.ResourcesEntryR\tresources\x1a`\n\x0eResourcesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32".spark.connect.ResourceInformationR\x05value:\x02\x38\x01"X\n\x1c\x43reateResourceProfileCommand\x12\x38\n\x07profile\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ResourceProfileR\x07profile"C\n"CreateResourceProfileCommandResult\x12\x1d\n\nprofile_id\x18\x01 \x01(\x05R\tprofileId"d\n!RemoveCachedRemoteRelationCommand\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation"\xcd\x01\n\x11\x43heckpointCommand\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x14\n\x05local\x18\x02 \x01(\x08R\x05local\x12\x14\n\x05\x65\x61ger\x18\x03 \x01(\x08R\x05\x65\x61ger\x12\x45\n\rstorage_level\x18\x04 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level"\xe8\x03\n\x15MergeIntoTableCommand\x12*\n\x11target_table_name\x18\x01 \x01(\tR\x0ftargetTableName\x12\x43\n\x11source_table_plan\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x0fsourceTablePlan\x12\x42\n\x0fmerge_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0emergeCondition\x12>\n\rmatch_actions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cmatchActions\x12I\n\x13not_matched_actions\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x11notMatchedActions\x12[\n\x1dnot_matched_by_source_actions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x19notMatchedBySourceActions\x12\x32\n\x15with_schema_evolution\x18\x07 \x01(\x08R\x13withSchemaEvolution*\x85\x01\n\x17StreamingQueryEventType\x12\x1e\n\x1aQUERY_PROGRESS_UNSPECIFIED\x10\x00\x12\x18\n\x14QUERY_PROGRESS_EVENT\x10\x01\x12\x1a\n\x16QUERY_TERMINATED_EVENT\x10\x02\x12\x14\n\x10QUERY_IDLE_EVENT\x10\x03\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -78,114 +79,114 @@ _globals["_WRITESTREAMOPERATIONSTART_OPTIONSENTRY"]._serialized_options = b"8\001" _globals["_GETRESOURCESCOMMANDRESULT_RESOURCESENTRY"]._loaded_options = None _globals["_GETRESOURCESCOMMANDRESULT_RESOURCESENTRY"]._serialized_options = b"8\001" - _globals["_STREAMINGQUERYEVENTTYPE"]._serialized_start = 11252 - _globals["_STREAMINGQUERYEVENTTYPE"]._serialized_end = 11385 - _globals["_COMMAND"]._serialized_start = 167 - _globals["_COMMAND"]._serialized_end = 1847 - _globals["_SQLCOMMAND"]._serialized_start = 1850 - _globals["_SQLCOMMAND"]._serialized_end = 2404 - _globals["_SQLCOMMAND_ARGSENTRY"]._serialized_start = 2220 - _globals["_SQLCOMMAND_ARGSENTRY"]._serialized_end = 2310 - _globals["_SQLCOMMAND_NAMEDARGUMENTSENTRY"]._serialized_start = 2312 - _globals["_SQLCOMMAND_NAMEDARGUMENTSENTRY"]._serialized_end = 2404 - _globals["_CREATEDATAFRAMEVIEWCOMMAND"]._serialized_start = 2407 - _globals["_CREATEDATAFRAMEVIEWCOMMAND"]._serialized_end = 2557 - _globals["_WRITEOPERATION"]._serialized_start = 2560 - _globals["_WRITEOPERATION"]._serialized_end = 3658 - _globals["_WRITEOPERATION_OPTIONSENTRY"]._serialized_start = 3082 - _globals["_WRITEOPERATION_OPTIONSENTRY"]._serialized_end = 3140 - _globals["_WRITEOPERATION_SAVETABLE"]._serialized_start = 3143 - _globals["_WRITEOPERATION_SAVETABLE"]._serialized_end = 3401 - _globals["_WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD"]._serialized_start = 3277 - _globals["_WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD"]._serialized_end = 3401 - _globals["_WRITEOPERATION_BUCKETBY"]._serialized_start = 3403 - _globals["_WRITEOPERATION_BUCKETBY"]._serialized_end = 3494 - _globals["_WRITEOPERATION_SAVEMODE"]._serialized_start = 3497 - _globals["_WRITEOPERATION_SAVEMODE"]._serialized_end = 3634 - _globals["_WRITEOPERATIONV2"]._serialized_start = 3661 - _globals["_WRITEOPERATIONV2"]._serialized_end = 4521 - _globals["_WRITEOPERATIONV2_OPTIONSENTRY"]._serialized_start = 3082 - _globals["_WRITEOPERATIONV2_OPTIONSENTRY"]._serialized_end = 3140 - _globals["_WRITEOPERATIONV2_TABLEPROPERTIESENTRY"]._serialized_start = 4280 - _globals["_WRITEOPERATIONV2_TABLEPROPERTIESENTRY"]._serialized_end = 4346 - _globals["_WRITEOPERATIONV2_MODE"]._serialized_start = 4349 - _globals["_WRITEOPERATIONV2_MODE"]._serialized_end = 4508 - _globals["_WRITESTREAMOPERATIONSTART"]._serialized_start = 4524 - _globals["_WRITESTREAMOPERATIONSTART"]._serialized_end = 5380 - _globals["_WRITESTREAMOPERATIONSTART_OPTIONSENTRY"]._serialized_start = 3082 - _globals["_WRITESTREAMOPERATIONSTART_OPTIONSENTRY"]._serialized_end = 3140 - _globals["_STREAMINGFOREACHFUNCTION"]._serialized_start = 5383 - _globals["_STREAMINGFOREACHFUNCTION"]._serialized_end = 5562 - _globals["_WRITESTREAMOPERATIONSTARTRESULT"]._serialized_start = 5565 - _globals["_WRITESTREAMOPERATIONSTARTRESULT"]._serialized_end = 5777 - _globals["_STREAMINGQUERYINSTANCEID"]._serialized_start = 5779 - _globals["_STREAMINGQUERYINSTANCEID"]._serialized_end = 5844 - _globals["_STREAMINGQUERYCOMMAND"]._serialized_start = 5847 - _globals["_STREAMINGQUERYCOMMAND"]._serialized_end = 6479 - _globals["_STREAMINGQUERYCOMMAND_EXPLAINCOMMAND"]._serialized_start = 6346 - _globals["_STREAMINGQUERYCOMMAND_EXPLAINCOMMAND"]._serialized_end = 6390 - _globals["_STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND"]._serialized_start = 6392 - _globals["_STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND"]._serialized_end = 6468 - _globals["_STREAMINGQUERYCOMMANDRESULT"]._serialized_start = 6482 - _globals["_STREAMINGQUERYCOMMANDRESULT"]._serialized_end = 7623 - _globals["_STREAMINGQUERYCOMMANDRESULT_STATUSRESULT"]._serialized_start = 7065 - _globals["_STREAMINGQUERYCOMMANDRESULT_STATUSRESULT"]._serialized_end = 7235 - _globals["_STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT"]._serialized_start = 7237 - _globals["_STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT"]._serialized_end = 7309 - _globals["_STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT"]._serialized_start = 7311 - _globals["_STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT"]._serialized_end = 7350 - _globals["_STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT"]._serialized_start = 7353 - _globals["_STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT"]._serialized_end = 7550 - _globals["_STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT"]._serialized_start = 7552 - _globals["_STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT"]._serialized_end = 7608 - _globals["_STREAMINGQUERYMANAGERCOMMAND"]._serialized_start = 7626 - _globals["_STREAMINGQUERYMANAGERCOMMAND"]._serialized_end = 8455 - _globals["_STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND"]._serialized_start = 8157 - _globals["_STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND"]._serialized_end = 8236 - _globals["_STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND"]._serialized_start = 8239 - _globals["_STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND"]._serialized_end = 8444 - _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT"]._serialized_start = 8458 - _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT"]._serialized_end = 9534 - _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT"]._serialized_start = 9066 - _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT"]._serialized_end = 9193 - _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE"]._serialized_start = 9195 - _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE"]._serialized_end = 9310 + _globals["_STREAMINGQUERYEVENTTYPE"]._serialized_start = 11335 + _globals["_STREAMINGQUERYEVENTTYPE"]._serialized_end = 11468 + _globals["_COMMAND"]._serialized_start = 191 + _globals["_COMMAND"]._serialized_end = 1930 + _globals["_SQLCOMMAND"]._serialized_start = 1933 + _globals["_SQLCOMMAND"]._serialized_end = 2487 + _globals["_SQLCOMMAND_ARGSENTRY"]._serialized_start = 2303 + _globals["_SQLCOMMAND_ARGSENTRY"]._serialized_end = 2393 + _globals["_SQLCOMMAND_NAMEDARGUMENTSENTRY"]._serialized_start = 2395 + _globals["_SQLCOMMAND_NAMEDARGUMENTSENTRY"]._serialized_end = 2487 + _globals["_CREATEDATAFRAMEVIEWCOMMAND"]._serialized_start = 2490 + _globals["_CREATEDATAFRAMEVIEWCOMMAND"]._serialized_end = 2640 + _globals["_WRITEOPERATION"]._serialized_start = 2643 + _globals["_WRITEOPERATION"]._serialized_end = 3741 + _globals["_WRITEOPERATION_OPTIONSENTRY"]._serialized_start = 3165 + _globals["_WRITEOPERATION_OPTIONSENTRY"]._serialized_end = 3223 + _globals["_WRITEOPERATION_SAVETABLE"]._serialized_start = 3226 + _globals["_WRITEOPERATION_SAVETABLE"]._serialized_end = 3484 + _globals["_WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD"]._serialized_start = 3360 + _globals["_WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD"]._serialized_end = 3484 + _globals["_WRITEOPERATION_BUCKETBY"]._serialized_start = 3486 + _globals["_WRITEOPERATION_BUCKETBY"]._serialized_end = 3577 + _globals["_WRITEOPERATION_SAVEMODE"]._serialized_start = 3580 + _globals["_WRITEOPERATION_SAVEMODE"]._serialized_end = 3717 + _globals["_WRITEOPERATIONV2"]._serialized_start = 3744 + _globals["_WRITEOPERATIONV2"]._serialized_end = 4604 + _globals["_WRITEOPERATIONV2_OPTIONSENTRY"]._serialized_start = 3165 + _globals["_WRITEOPERATIONV2_OPTIONSENTRY"]._serialized_end = 3223 + _globals["_WRITEOPERATIONV2_TABLEPROPERTIESENTRY"]._serialized_start = 4363 + _globals["_WRITEOPERATIONV2_TABLEPROPERTIESENTRY"]._serialized_end = 4429 + _globals["_WRITEOPERATIONV2_MODE"]._serialized_start = 4432 + _globals["_WRITEOPERATIONV2_MODE"]._serialized_end = 4591 + _globals["_WRITESTREAMOPERATIONSTART"]._serialized_start = 4607 + _globals["_WRITESTREAMOPERATIONSTART"]._serialized_end = 5463 + _globals["_WRITESTREAMOPERATIONSTART_OPTIONSENTRY"]._serialized_start = 3165 + _globals["_WRITESTREAMOPERATIONSTART_OPTIONSENTRY"]._serialized_end = 3223 + _globals["_STREAMINGFOREACHFUNCTION"]._serialized_start = 5466 + _globals["_STREAMINGFOREACHFUNCTION"]._serialized_end = 5645 + _globals["_WRITESTREAMOPERATIONSTARTRESULT"]._serialized_start = 5648 + _globals["_WRITESTREAMOPERATIONSTARTRESULT"]._serialized_end = 5860 + _globals["_STREAMINGQUERYINSTANCEID"]._serialized_start = 5862 + _globals["_STREAMINGQUERYINSTANCEID"]._serialized_end = 5927 + _globals["_STREAMINGQUERYCOMMAND"]._serialized_start = 5930 + _globals["_STREAMINGQUERYCOMMAND"]._serialized_end = 6562 + _globals["_STREAMINGQUERYCOMMAND_EXPLAINCOMMAND"]._serialized_start = 6429 + _globals["_STREAMINGQUERYCOMMAND_EXPLAINCOMMAND"]._serialized_end = 6473 + _globals["_STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND"]._serialized_start = 6475 + _globals["_STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND"]._serialized_end = 6551 + _globals["_STREAMINGQUERYCOMMANDRESULT"]._serialized_start = 6565 + _globals["_STREAMINGQUERYCOMMANDRESULT"]._serialized_end = 7706 + _globals["_STREAMINGQUERYCOMMANDRESULT_STATUSRESULT"]._serialized_start = 7148 + _globals["_STREAMINGQUERYCOMMANDRESULT_STATUSRESULT"]._serialized_end = 7318 + _globals["_STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT"]._serialized_start = 7320 + _globals["_STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT"]._serialized_end = 7392 + _globals["_STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT"]._serialized_start = 7394 + _globals["_STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT"]._serialized_end = 7433 + _globals["_STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT"]._serialized_start = 7436 + _globals["_STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT"]._serialized_end = 7633 + _globals["_STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT"]._serialized_start = 7635 + _globals["_STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT"]._serialized_end = 7691 + _globals["_STREAMINGQUERYMANAGERCOMMAND"]._serialized_start = 7709 + _globals["_STREAMINGQUERYMANAGERCOMMAND"]._serialized_end = 8538 + _globals["_STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND"]._serialized_start = 8240 + _globals["_STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND"]._serialized_end = 8319 + _globals["_STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND"]._serialized_start = 8322 + _globals["_STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND"]._serialized_end = 8527 + _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT"]._serialized_start = 8541 + _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT"]._serialized_end = 9617 + _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT"]._serialized_start = 9149 + _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT"]._serialized_end = 9276 + _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE"]._serialized_start = 9278 + _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE"]._serialized_end = 9393 _globals[ "_STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT" - ]._serialized_start = 9312 - _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT"]._serialized_end = 9371 + ]._serialized_start = 9395 + _globals["_STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT"]._serialized_end = 9454 _globals[ "_STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE" - ]._serialized_start = 9373 + ]._serialized_start = 9456 _globals[ "_STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE" - ]._serialized_end = 9448 + ]._serialized_end = 9531 _globals[ "_STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT" - ]._serialized_start = 9450 + ]._serialized_start = 9533 _globals[ "_STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT" - ]._serialized_end = 9519 - _globals["_STREAMINGQUERYLISTENERBUSCOMMAND"]._serialized_start = 9537 - _globals["_STREAMINGQUERYLISTENERBUSCOMMAND"]._serialized_end = 9710 - _globals["_STREAMINGQUERYLISTENEREVENT"]._serialized_start = 9713 - _globals["_STREAMINGQUERYLISTENEREVENT"]._serialized_end = 9844 - _globals["_STREAMINGQUERYLISTENEREVENTSRESULT"]._serialized_start = 9847 - _globals["_STREAMINGQUERYLISTENEREVENTSRESULT"]._serialized_end = 10051 - _globals["_GETRESOURCESCOMMAND"]._serialized_start = 10053 - _globals["_GETRESOURCESCOMMAND"]._serialized_end = 10074 - _globals["_GETRESOURCESCOMMANDRESULT"]._serialized_start = 10077 - _globals["_GETRESOURCESCOMMANDRESULT"]._serialized_end = 10289 - _globals["_GETRESOURCESCOMMANDRESULT_RESOURCESENTRY"]._serialized_start = 10193 - _globals["_GETRESOURCESCOMMANDRESULT_RESOURCESENTRY"]._serialized_end = 10289 - _globals["_CREATERESOURCEPROFILECOMMAND"]._serialized_start = 10291 - _globals["_CREATERESOURCEPROFILECOMMAND"]._serialized_end = 10379 - _globals["_CREATERESOURCEPROFILECOMMANDRESULT"]._serialized_start = 10381 - _globals["_CREATERESOURCEPROFILECOMMANDRESULT"]._serialized_end = 10448 - _globals["_REMOVECACHEDREMOTERELATIONCOMMAND"]._serialized_start = 10450 - _globals["_REMOVECACHEDREMOTERELATIONCOMMAND"]._serialized_end = 10550 - _globals["_CHECKPOINTCOMMAND"]._serialized_start = 10553 - _globals["_CHECKPOINTCOMMAND"]._serialized_end = 10758 - _globals["_MERGEINTOTABLECOMMAND"]._serialized_start = 10761 - _globals["_MERGEINTOTABLECOMMAND"]._serialized_end = 11249 + ]._serialized_end = 9602 + _globals["_STREAMINGQUERYLISTENERBUSCOMMAND"]._serialized_start = 9620 + _globals["_STREAMINGQUERYLISTENERBUSCOMMAND"]._serialized_end = 9793 + _globals["_STREAMINGQUERYLISTENEREVENT"]._serialized_start = 9796 + _globals["_STREAMINGQUERYLISTENEREVENT"]._serialized_end = 9927 + _globals["_STREAMINGQUERYLISTENEREVENTSRESULT"]._serialized_start = 9930 + _globals["_STREAMINGQUERYLISTENEREVENTSRESULT"]._serialized_end = 10134 + _globals["_GETRESOURCESCOMMAND"]._serialized_start = 10136 + _globals["_GETRESOURCESCOMMAND"]._serialized_end = 10157 + _globals["_GETRESOURCESCOMMANDRESULT"]._serialized_start = 10160 + _globals["_GETRESOURCESCOMMANDRESULT"]._serialized_end = 10372 + _globals["_GETRESOURCESCOMMANDRESULT_RESOURCESENTRY"]._serialized_start = 10276 + _globals["_GETRESOURCESCOMMANDRESULT_RESOURCESENTRY"]._serialized_end = 10372 + _globals["_CREATERESOURCEPROFILECOMMAND"]._serialized_start = 10374 + _globals["_CREATERESOURCEPROFILECOMMAND"]._serialized_end = 10462 + _globals["_CREATERESOURCEPROFILECOMMANDRESULT"]._serialized_start = 10464 + _globals["_CREATERESOURCEPROFILECOMMANDRESULT"]._serialized_end = 10531 + _globals["_REMOVECACHEDREMOTERELATIONCOMMAND"]._serialized_start = 10533 + _globals["_REMOVECACHEDREMOTERELATIONCOMMAND"]._serialized_end = 10633 + _globals["_CHECKPOINTCOMMAND"]._serialized_start = 10636 + _globals["_CHECKPOINTCOMMAND"]._serialized_end = 10841 + _globals["_MERGEINTOTABLECOMMAND"]._serialized_start = 10844 + _globals["_MERGEINTOTABLECOMMAND"]._serialized_end = 11332 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/commands_pb2.pyi b/python/pyspark/sql/connect/proto/commands_pb2.pyi index 6192a29607cbf..906f1aad10574 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.pyi +++ b/python/pyspark/sql/connect/proto/commands_pb2.pyi @@ -42,6 +42,7 @@ import google.protobuf.internal.enum_type_wrapper import google.protobuf.message import pyspark.sql.connect.proto.common_pb2 import pyspark.sql.connect.proto.expressions_pb2 +import pyspark.sql.connect.proto.ml_pb2 import pyspark.sql.connect.proto.relations_pb2 import sys import typing @@ -104,6 +105,7 @@ class Command(google.protobuf.message.Message): CHECKPOINT_COMMAND_FIELD_NUMBER: builtins.int REMOVE_CACHED_REMOTE_RELATION_COMMAND_FIELD_NUMBER: builtins.int MERGE_INTO_TABLE_COMMAND_FIELD_NUMBER: builtins.int + ML_COMMAND_FIELD_NUMBER: builtins.int EXTENSION_FIELD_NUMBER: builtins.int @property def register_function( @@ -146,6 +148,8 @@ class Command(google.protobuf.message.Message): @property def merge_into_table_command(self) -> global___MergeIntoTableCommand: ... @property + def ml_command(self) -> pyspark.sql.connect.proto.ml_pb2.MlCommand: ... + @property def extension(self) -> google.protobuf.any_pb2.Any: """This field is used to mark extensions to the protocol. When plugins generate arbitrary Commands they can add them here. During the planning the correct resolution is done. @@ -174,6 +178,7 @@ class Command(google.protobuf.message.Message): remove_cached_remote_relation_command: global___RemoveCachedRemoteRelationCommand | None = ..., merge_into_table_command: global___MergeIntoTableCommand | None = ..., + ml_command: pyspark.sql.connect.proto.ml_pb2.MlCommand | None = ..., extension: google.protobuf.any_pb2.Any | None = ..., ) -> None: ... def HasField( @@ -193,6 +198,8 @@ class Command(google.protobuf.message.Message): b"get_resources_command", "merge_into_table_command", b"merge_into_table_command", + "ml_command", + b"ml_command", "register_data_source", b"register_data_source", "register_function", @@ -234,6 +241,8 @@ class Command(google.protobuf.message.Message): b"get_resources_command", "merge_into_table_command", b"merge_into_table_command", + "ml_command", + b"ml_command", "register_data_source", b"register_data_source", "register_function", @@ -278,6 +287,7 @@ class Command(google.protobuf.message.Message): "checkpoint_command", "remove_cached_remote_relation_command", "merge_into_table_command", + "ml_command", "extension", ] | None diff --git a/python/pyspark/sql/connect/proto/ml_common_pb2.py b/python/pyspark/sql/connect/proto/ml_common_pb2.py new file mode 100644 index 0000000000000..70e0e91652892 --- /dev/null +++ b/python/pyspark/sql/connect/proto/ml_common_pb2.py @@ -0,0 +1,80 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# -*- coding: utf-8 -*- +# Generated by the protocol buffer compiler. DO NOT EDIT! +# NO CHECKED-IN PROTOBUF GENCODE +# source: spark/connect/ml_common.proto +# Protobuf Python Version: 5.28.3 +"""Generated protocol buffer code.""" +from google.protobuf import descriptor as _descriptor +from google.protobuf import descriptor_pool as _descriptor_pool +from google.protobuf import runtime_version as _runtime_version +from google.protobuf import symbol_database as _symbol_database +from google.protobuf.internal import builder as _builder + +_runtime_version.ValidateProtobufRuntimeVersion( + _runtime_version.Domain.PUBLIC, 5, 28, 3, "", "spark/connect/ml_common.proto" +) +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + +from pyspark.sql.connect.proto import expressions_pb2 as spark_dot_connect_dot_expressions__pb2 + + +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\x1dspark/connect/ml_common.proto\x12\rspark.connect\x1a\x1fspark/connect/expressions.proto"\x98\x01\n\x08MlParams\x12;\n\x06params\x18\x01 \x03(\x0b\x32#.spark.connect.MlParams.ParamsEntryR\x06params\x1aO\n\x0bParamsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12*\n\x05value\x18\x02 \x01(\x0b\x32\x14.spark.connect.ParamR\x05value:\x02\x38\x01"\xb6\x01\n\x05Param\x12=\n\x07literal\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12/\n\x06vector\x18\x02 \x01(\x0b\x32\x15.spark.connect.VectorH\x00R\x06vector\x12/\n\x06matrix\x18\x03 \x01(\x0b\x32\x15.spark.connect.MatrixH\x00R\x06matrixB\x0c\n\nparam_type"\xc9\x01\n\nMlOperator\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x10\n\x03uid\x18\x02 \x01(\tR\x03uid\x12:\n\x04type\x18\x03 \x01(\x0e\x32&.spark.connect.MlOperator.OperatorTypeR\x04type"Y\n\x0cOperatorType\x12\x0f\n\x0bUNSPECIFIED\x10\x00\x12\r\n\tESTIMATOR\x10\x01\x12\x0f\n\x0bTRANSFORMER\x10\x02\x12\r\n\tEVALUATOR\x10\x03\x12\t\n\x05MODEL\x10\x04"\x1b\n\tObjectRef\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id"\xed\x01\n\x06Vector\x12\x33\n\x05\x64\x65nse\x18\x01 \x01(\x0b\x32\x1b.spark.connect.Vector.DenseH\x00R\x05\x64\x65nse\x12\x36\n\x06sparse\x18\x02 \x01(\x0b\x32\x1c.spark.connect.Vector.SparseH\x00R\x06sparse\x1a\x1d\n\x05\x44\x65nse\x12\x14\n\x05value\x18\x01 \x03(\x01R\x05value\x1aH\n\x06Sparse\x12\x12\n\x04size\x18\x01 \x01(\x05R\x04size\x12\x14\n\x05index\x18\x02 \x03(\x05R\x05index\x12\x14\n\x05value\x18\x03 \x03(\x01R\x05valueB\r\n\x0bvector_type"\xaf\x03\n\x06Matrix\x12\x33\n\x05\x64\x65nse\x18\x01 \x01(\x0b\x32\x1b.spark.connect.Matrix.DenseH\x00R\x05\x64\x65nse\x12\x36\n\x06sparse\x18\x02 \x01(\x0b\x32\x1c.spark.connect.Matrix.SparseH\x00R\x06sparse\x1ax\n\x05\x44\x65nse\x12\x19\n\x08num_rows\x18\x01 \x01(\x05R\x07numRows\x12\x19\n\x08num_cols\x18\x02 \x01(\x05R\x07numCols\x12\x14\n\x05value\x18\x03 \x03(\x01R\x05value\x12#\n\ris_transposed\x18\x04 \x01(\x08R\x0cisTransposed\x1a\xae\x01\n\x06Sparse\x12\x19\n\x08num_rows\x18\x01 \x01(\x05R\x07numRows\x12\x19\n\x08num_cols\x18\x02 \x01(\x05R\x07numCols\x12\x16\n\x06\x63olptr\x18\x03 \x03(\x05R\x06\x63olptr\x12\x1b\n\trow_index\x18\x04 \x03(\x05R\x08rowIndex\x12\x14\n\x05value\x18\x05 \x03(\x01R\x05value\x12#\n\ris_transposed\x18\x06 \x01(\x08R\x0cisTransposedB\r\n\x0bmatrix_typeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' +) + +_globals = globals() +_builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) +_builder.BuildTopDescriptorsAndMessages( + DESCRIPTOR, "pyspark.sql.connect.proto.ml_common_pb2", _globals +) +if not _descriptor._USE_C_DESCRIPTORS: + _globals["DESCRIPTOR"]._loaded_options = None + _globals[ + "DESCRIPTOR" + ]._serialized_options = b"\n\036org.apache.spark.connect.protoP\001Z\022internal/generated" + _globals["_MLPARAMS_PARAMSENTRY"]._loaded_options = None + _globals["_MLPARAMS_PARAMSENTRY"]._serialized_options = b"8\001" + _globals["_MLPARAMS"]._serialized_start = 82 + _globals["_MLPARAMS"]._serialized_end = 234 + _globals["_MLPARAMS_PARAMSENTRY"]._serialized_start = 155 + _globals["_MLPARAMS_PARAMSENTRY"]._serialized_end = 234 + _globals["_PARAM"]._serialized_start = 237 + _globals["_PARAM"]._serialized_end = 419 + _globals["_MLOPERATOR"]._serialized_start = 422 + _globals["_MLOPERATOR"]._serialized_end = 623 + _globals["_MLOPERATOR_OPERATORTYPE"]._serialized_start = 534 + _globals["_MLOPERATOR_OPERATORTYPE"]._serialized_end = 623 + _globals["_OBJECTREF"]._serialized_start = 625 + _globals["_OBJECTREF"]._serialized_end = 652 + _globals["_VECTOR"]._serialized_start = 655 + _globals["_VECTOR"]._serialized_end = 892 + _globals["_VECTOR_DENSE"]._serialized_start = 774 + _globals["_VECTOR_DENSE"]._serialized_end = 803 + _globals["_VECTOR_SPARSE"]._serialized_start = 805 + _globals["_VECTOR_SPARSE"]._serialized_end = 877 + _globals["_MATRIX"]._serialized_start = 895 + _globals["_MATRIX"]._serialized_end = 1326 + _globals["_MATRIX_DENSE"]._serialized_start = 1014 + _globals["_MATRIX_DENSE"]._serialized_end = 1134 + _globals["_MATRIX_SPARSE"]._serialized_start = 1137 + _globals["_MATRIX_SPARSE"]._serialized_end = 1311 +# @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/ml_common_pb2.pyi b/python/pyspark/sql/connect/proto/ml_common_pb2.pyi new file mode 100644 index 0000000000000..64029b6679f19 --- /dev/null +++ b/python/pyspark/sql/connect/proto/ml_common_pb2.pyi @@ -0,0 +1,427 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +""" +@generated by mypy-protobuf. Do not edit manually! +isort:skip_file + +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +""" +import builtins +import collections.abc +import google.protobuf.descriptor +import google.protobuf.internal.containers +import google.protobuf.internal.enum_type_wrapper +import google.protobuf.message +import pyspark.sql.connect.proto.expressions_pb2 +import sys +import typing + +if sys.version_info >= (3, 10): + import typing as typing_extensions +else: + import typing_extensions + +DESCRIPTOR: google.protobuf.descriptor.FileDescriptor + +class MlParams(google.protobuf.message.Message): + """MlParams stores param settings for ML Estimator / Transformer / Evaluator""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class ParamsEntry(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + KEY_FIELD_NUMBER: builtins.int + VALUE_FIELD_NUMBER: builtins.int + key: builtins.str + @property + def value(self) -> global___Param: ... + def __init__( + self, + *, + key: builtins.str = ..., + value: global___Param | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["value", b"value"] + ) -> builtins.bool: ... + def ClearField( + self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] + ) -> None: ... + + PARAMS_FIELD_NUMBER: builtins.int + @property + def params( + self, + ) -> google.protobuf.internal.containers.MessageMap[builtins.str, global___Param]: + """User-supplied params""" + def __init__( + self, + *, + params: collections.abc.Mapping[builtins.str, global___Param] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["params", b"params"]) -> None: ... + +global___MlParams = MlParams + +class Param(google.protobuf.message.Message): + """Represents the parameter type of the ML instance, or the returned value + of the attribute + """ + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + LITERAL_FIELD_NUMBER: builtins.int + VECTOR_FIELD_NUMBER: builtins.int + MATRIX_FIELD_NUMBER: builtins.int + @property + def literal(self) -> pyspark.sql.connect.proto.expressions_pb2.Expression.Literal: ... + @property + def vector(self) -> global___Vector: ... + @property + def matrix(self) -> global___Matrix: ... + def __init__( + self, + *, + literal: pyspark.sql.connect.proto.expressions_pb2.Expression.Literal | None = ..., + vector: global___Vector | None = ..., + matrix: global___Matrix | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "literal", + b"literal", + "matrix", + b"matrix", + "param_type", + b"param_type", + "vector", + b"vector", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "literal", + b"literal", + "matrix", + b"matrix", + "param_type", + b"param_type", + "vector", + b"vector", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["param_type", b"param_type"] + ) -> typing_extensions.Literal["literal", "vector", "matrix"] | None: ... + +global___Param = Param + +class MlOperator(google.protobuf.message.Message): + """MLOperator represents the ML operators like (Estimator, Transformer or Evaluator)""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class _OperatorType: + ValueType = typing.NewType("ValueType", builtins.int) + V: typing_extensions.TypeAlias = ValueType + + class _OperatorTypeEnumTypeWrapper( + google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[ + MlOperator._OperatorType.ValueType + ], + builtins.type, + ): # noqa: F821 + DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor + UNSPECIFIED: MlOperator._OperatorType.ValueType # 0 + ESTIMATOR: MlOperator._OperatorType.ValueType # 1 + TRANSFORMER: MlOperator._OperatorType.ValueType # 2 + EVALUATOR: MlOperator._OperatorType.ValueType # 3 + MODEL: MlOperator._OperatorType.ValueType # 4 + + class OperatorType(_OperatorType, metaclass=_OperatorTypeEnumTypeWrapper): ... + UNSPECIFIED: MlOperator.OperatorType.ValueType # 0 + ESTIMATOR: MlOperator.OperatorType.ValueType # 1 + TRANSFORMER: MlOperator.OperatorType.ValueType # 2 + EVALUATOR: MlOperator.OperatorType.ValueType # 3 + MODEL: MlOperator.OperatorType.ValueType # 4 + + NAME_FIELD_NUMBER: builtins.int + UID_FIELD_NUMBER: builtins.int + TYPE_FIELD_NUMBER: builtins.int + name: builtins.str + """The qualified name of the ML operator.""" + uid: builtins.str + """Unique id of the ML operator""" + type: global___MlOperator.OperatorType.ValueType + """Represents what the ML operator is""" + def __init__( + self, + *, + name: builtins.str = ..., + uid: builtins.str = ..., + type: global___MlOperator.OperatorType.ValueType = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["name", b"name", "type", b"type", "uid", b"uid"] + ) -> None: ... + +global___MlOperator = MlOperator + +class ObjectRef(google.protobuf.message.Message): + """Represents a reference to the cached object which could be a model + or summary evaluated by a model + """ + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + ID_FIELD_NUMBER: builtins.int + id: builtins.str + """The ID is used to lookup the object on the server side.""" + def __init__( + self, + *, + id: builtins.str = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["id", b"id"]) -> None: ... + +global___ObjectRef = ObjectRef + +class Vector(google.protobuf.message.Message): + """See pyspark.ml.linalg.Vector""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class Dense(google.protobuf.message.Message): + """See pyspark.ml.linalg.DenseVector""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + VALUE_FIELD_NUMBER: builtins.int + @property + def value( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.float]: ... + def __init__( + self, + *, + value: collections.abc.Iterable[builtins.float] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["value", b"value"]) -> None: ... + + class Sparse(google.protobuf.message.Message): + """See pyspark.ml.linalg.SparseVector""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + SIZE_FIELD_NUMBER: builtins.int + INDEX_FIELD_NUMBER: builtins.int + VALUE_FIELD_NUMBER: builtins.int + size: builtins.int + @property + def index( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.int]: ... + @property + def value( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.float]: ... + def __init__( + self, + *, + size: builtins.int = ..., + index: collections.abc.Iterable[builtins.int] | None = ..., + value: collections.abc.Iterable[builtins.float] | None = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "index", b"index", "size", b"size", "value", b"value" + ], + ) -> None: ... + + DENSE_FIELD_NUMBER: builtins.int + SPARSE_FIELD_NUMBER: builtins.int + @property + def dense(self) -> global___Vector.Dense: ... + @property + def sparse(self) -> global___Vector.Sparse: ... + def __init__( + self, + *, + dense: global___Vector.Dense | None = ..., + sparse: global___Vector.Sparse | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "dense", b"dense", "sparse", b"sparse", "vector_type", b"vector_type" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "dense", b"dense", "sparse", b"sparse", "vector_type", b"vector_type" + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["vector_type", b"vector_type"] + ) -> typing_extensions.Literal["dense", "sparse"] | None: ... + +global___Vector = Vector + +class Matrix(google.protobuf.message.Message): + """See pyspark.ml.linalg.Matrix""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class Dense(google.protobuf.message.Message): + """See pyspark.ml.linalg.DenseMatrix""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NUM_ROWS_FIELD_NUMBER: builtins.int + NUM_COLS_FIELD_NUMBER: builtins.int + VALUE_FIELD_NUMBER: builtins.int + IS_TRANSPOSED_FIELD_NUMBER: builtins.int + num_rows: builtins.int + num_cols: builtins.int + @property + def value( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.float]: ... + is_transposed: builtins.bool + def __init__( + self, + *, + num_rows: builtins.int = ..., + num_cols: builtins.int = ..., + value: collections.abc.Iterable[builtins.float] | None = ..., + is_transposed: builtins.bool = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "is_transposed", + b"is_transposed", + "num_cols", + b"num_cols", + "num_rows", + b"num_rows", + "value", + b"value", + ], + ) -> None: ... + + class Sparse(google.protobuf.message.Message): + """See pyspark.ml.linalg.SparseMatrix""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NUM_ROWS_FIELD_NUMBER: builtins.int + NUM_COLS_FIELD_NUMBER: builtins.int + COLPTR_FIELD_NUMBER: builtins.int + ROW_INDEX_FIELD_NUMBER: builtins.int + VALUE_FIELD_NUMBER: builtins.int + IS_TRANSPOSED_FIELD_NUMBER: builtins.int + num_rows: builtins.int + num_cols: builtins.int + @property + def colptr( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.int]: ... + @property + def row_index( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.int]: ... + @property + def value( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.float]: ... + is_transposed: builtins.bool + def __init__( + self, + *, + num_rows: builtins.int = ..., + num_cols: builtins.int = ..., + colptr: collections.abc.Iterable[builtins.int] | None = ..., + row_index: collections.abc.Iterable[builtins.int] | None = ..., + value: collections.abc.Iterable[builtins.float] | None = ..., + is_transposed: builtins.bool = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "colptr", + b"colptr", + "is_transposed", + b"is_transposed", + "num_cols", + b"num_cols", + "num_rows", + b"num_rows", + "row_index", + b"row_index", + "value", + b"value", + ], + ) -> None: ... + + DENSE_FIELD_NUMBER: builtins.int + SPARSE_FIELD_NUMBER: builtins.int + @property + def dense(self) -> global___Matrix.Dense: ... + @property + def sparse(self) -> global___Matrix.Sparse: ... + def __init__( + self, + *, + dense: global___Matrix.Dense | None = ..., + sparse: global___Matrix.Sparse | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "dense", b"dense", "matrix_type", b"matrix_type", "sparse", b"sparse" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "dense", b"dense", "matrix_type", b"matrix_type", "sparse", b"sparse" + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["matrix_type", b"matrix_type"] + ) -> typing_extensions.Literal["dense", "sparse"] | None: ... + +global___Matrix = Matrix diff --git a/python/pyspark/sql/connect/proto/ml_pb2.py b/python/pyspark/sql/connect/proto/ml_pb2.py new file mode 100644 index 0000000000000..46692010e3ffc --- /dev/null +++ b/python/pyspark/sql/connect/proto/ml_pb2.py @@ -0,0 +1,72 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# -*- coding: utf-8 -*- +# Generated by the protocol buffer compiler. DO NOT EDIT! +# NO CHECKED-IN PROTOBUF GENCODE +# source: spark/connect/ml.proto +# Protobuf Python Version: 5.28.3 +"""Generated protocol buffer code.""" +from google.protobuf import descriptor as _descriptor +from google.protobuf import descriptor_pool as _descriptor_pool +from google.protobuf import runtime_version as _runtime_version +from google.protobuf import symbol_database as _symbol_database +from google.protobuf.internal import builder as _builder + +_runtime_version.ValidateProtobufRuntimeVersion( + _runtime_version.Domain.PUBLIC, 5, 28, 3, "", "spark/connect/ml.proto" +) +# @@protoc_insertion_point(imports) + +_sym_db = _symbol_database.Default() + + +from pyspark.sql.connect.proto import expressions_pb2 as spark_dot_connect_dot_expressions__pb2 +from pyspark.sql.connect.proto import relations_pb2 as spark_dot_connect_dot_relations__pb2 +from pyspark.sql.connect.proto import ml_common_pb2 as spark_dot_connect_dot_ml__common__pb2 + + +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( + b'\n\x16spark/connect/ml.proto\x12\rspark.connect\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x1dspark/connect/ml_common.proto"\xc6\x07\n\tMlCommand\x12\x30\n\x03\x66it\x18\x01 \x01(\x0b\x32\x1c.spark.connect.MlCommand.FitH\x00R\x03\x66it\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x12\x39\n\x06\x64\x65lete\x18\x03 \x01(\x0b\x32\x1f.spark.connect.MlCommand.DeleteH\x00R\x06\x64\x65lete\x12\x36\n\x05write\x18\x04 \x01(\x0b\x32\x1e.spark.connect.MlCommand.WriteH\x00R\x05write\x12\x33\n\x04read\x18\x05 \x01(\x0b\x32\x1d.spark.connect.MlCommand.ReadH\x00R\x04read\x1a\xa2\x01\n\x03\x46it\x12\x37\n\testimator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorR\testimator\x12/\n\x06params\x18\x02 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06params\x12\x31\n\x07\x64\x61taset\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x07\x64\x61taset\x1a;\n\x06\x44\x65lete\x12\x31\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x06objRef\x1a\xf0\x02\n\x05Write\x12\x37\n\x08operator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x08operator\x12\x33\n\x07obj_ref\x18\x02 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12/\n\x06params\x18\x03 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06params\x12\x12\n\x04path\x18\x04 \x01(\tR\x04path\x12)\n\x10should_overwrite\x18\x05 \x01(\x08R\x0fshouldOverwrite\x12\x45\n\x07options\x18\x06 \x03(\x0b\x32+.spark.connect.MlCommand.Write.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x06\n\x04type\x1aQ\n\x04Read\x12\x35\n\x08operator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorR\x08operator\x12\x12\n\x04path\x18\x02 \x01(\tR\x04pathB\t\n\x07\x63ommand"\xe9\x02\n\x0fMlCommandResult\x12,\n\x05param\x18\x01 \x01(\x0b\x32\x14.spark.connect.ParamH\x00R\x05param\x12\x1a\n\x07summary\x18\x02 \x01(\tH\x00R\x07summary\x12T\n\roperator_info\x18\x03 \x01(\x0b\x32-.spark.connect.MlCommandResult.MlOperatorInfoH\x00R\x0coperatorInfo\x1a\xa6\x01\n\x0eMlOperatorInfo\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12\x14\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x12\x10\n\x03uid\x18\x03 \x01(\tR\x03uid\x12/\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06paramsB\x06\n\x04typeB\r\n\x0bresult_typeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' +) + +_globals = globals() +_builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) +_builder.BuildTopDescriptorsAndMessages(DESCRIPTOR, "pyspark.sql.connect.proto.ml_pb2", _globals) +if not _descriptor._USE_C_DESCRIPTORS: + _globals["DESCRIPTOR"]._loaded_options = None + _globals[ + "DESCRIPTOR" + ]._serialized_options = b"\n\036org.apache.spark.connect.protoP\001Z\022internal/generated" + _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._loaded_options = None + _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_options = b"8\001" + _globals["_MLCOMMAND"]._serialized_start = 137 + _globals["_MLCOMMAND"]._serialized_end = 1103 + _globals["_MLCOMMAND_FIT"]._serialized_start = 415 + _globals["_MLCOMMAND_FIT"]._serialized_end = 577 + _globals["_MLCOMMAND_DELETE"]._serialized_start = 579 + _globals["_MLCOMMAND_DELETE"]._serialized_end = 638 + _globals["_MLCOMMAND_WRITE"]._serialized_start = 641 + _globals["_MLCOMMAND_WRITE"]._serialized_end = 1009 + _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_start = 943 + _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_end = 1001 + _globals["_MLCOMMAND_READ"]._serialized_start = 1011 + _globals["_MLCOMMAND_READ"]._serialized_end = 1092 + _globals["_MLCOMMANDRESULT"]._serialized_start = 1106 + _globals["_MLCOMMANDRESULT"]._serialized_end = 1467 + _globals["_MLCOMMANDRESULT_MLOPERATORINFO"]._serialized_start = 1286 + _globals["_MLCOMMANDRESULT_MLOPERATORINFO"]._serialized_end = 1452 +# @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/ml_pb2.pyi b/python/pyspark/sql/connect/proto/ml_pb2.pyi new file mode 100644 index 0000000000000..95bfefb524e2a --- /dev/null +++ b/python/pyspark/sql/connect/proto/ml_pb2.pyi @@ -0,0 +1,393 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +""" +@generated by mypy-protobuf. Do not edit manually! +isort:skip_file + +Licensed to the Apache Software Foundation (ASF) under one or more +contributor license agreements. See the NOTICE file distributed with +this work for additional information regarding copyright ownership. +The ASF licenses this file to You under the Apache License, Version 2.0 +(the "License"); you may not use this file except in compliance with +the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +""" +import builtins +import collections.abc +import google.protobuf.descriptor +import google.protobuf.internal.containers +import google.protobuf.message +import pyspark.sql.connect.proto.ml_common_pb2 +import pyspark.sql.connect.proto.relations_pb2 +import sys + +if sys.version_info >= (3, 8): + import typing as typing_extensions +else: + import typing_extensions + +DESCRIPTOR: google.protobuf.descriptor.FileDescriptor + +class MlCommand(google.protobuf.message.Message): + """Command for ML""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class Fit(google.protobuf.message.Message): + """Command for estimator.fit(dataset)""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + ESTIMATOR_FIELD_NUMBER: builtins.int + PARAMS_FIELD_NUMBER: builtins.int + DATASET_FIELD_NUMBER: builtins.int + @property + def estimator(self) -> pyspark.sql.connect.proto.ml_common_pb2.MlOperator: + """Estimator information""" + @property + def params(self) -> pyspark.sql.connect.proto.ml_common_pb2.MlParams: + """parameters of the Estimator""" + @property + def dataset(self) -> pyspark.sql.connect.proto.relations_pb2.Relation: + """the training dataset""" + def __init__( + self, + *, + estimator: pyspark.sql.connect.proto.ml_common_pb2.MlOperator | None = ..., + params: pyspark.sql.connect.proto.ml_common_pb2.MlParams | None = ..., + dataset: pyspark.sql.connect.proto.relations_pb2.Relation | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "dataset", b"dataset", "estimator", b"estimator", "params", b"params" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "dataset", b"dataset", "estimator", b"estimator", "params", b"params" + ], + ) -> None: ... + + class Delete(google.protobuf.message.Message): + """Command to delete the cached object which could be a model + or summary evaluated by a model + """ + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + OBJ_REF_FIELD_NUMBER: builtins.int + @property + def obj_ref(self) -> pyspark.sql.connect.proto.ml_common_pb2.ObjectRef: ... + def __init__( + self, + *, + obj_ref: pyspark.sql.connect.proto.ml_common_pb2.ObjectRef | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["obj_ref", b"obj_ref"] + ) -> builtins.bool: ... + def ClearField( + self, field_name: typing_extensions.Literal["obj_ref", b"obj_ref"] + ) -> None: ... + + class Write(google.protobuf.message.Message): + """Command to write ML operator""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class OptionsEntry(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + KEY_FIELD_NUMBER: builtins.int + VALUE_FIELD_NUMBER: builtins.int + key: builtins.str + value: builtins.str + def __init__( + self, + *, + key: builtins.str = ..., + value: builtins.str = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] + ) -> None: ... + + OPERATOR_FIELD_NUMBER: builtins.int + OBJ_REF_FIELD_NUMBER: builtins.int + PARAMS_FIELD_NUMBER: builtins.int + PATH_FIELD_NUMBER: builtins.int + SHOULD_OVERWRITE_FIELD_NUMBER: builtins.int + OPTIONS_FIELD_NUMBER: builtins.int + @property + def operator(self) -> pyspark.sql.connect.proto.ml_common_pb2.MlOperator: + """Estimator or evaluator""" + @property + def obj_ref(self) -> pyspark.sql.connect.proto.ml_common_pb2.ObjectRef: + """The cached model""" + @property + def params(self) -> pyspark.sql.connect.proto.ml_common_pb2.MlParams: + """The parameters of operator which could be estimator/evaluator or a cached model""" + path: builtins.str + """Save the ML instance to the path""" + should_overwrite: builtins.bool + """Overwrites if the output path already exists.""" + @property + def options( + self, + ) -> google.protobuf.internal.containers.ScalarMap[builtins.str, builtins.str]: + """The options of the writer""" + def __init__( + self, + *, + operator: pyspark.sql.connect.proto.ml_common_pb2.MlOperator | None = ..., + obj_ref: pyspark.sql.connect.proto.ml_common_pb2.ObjectRef | None = ..., + params: pyspark.sql.connect.proto.ml_common_pb2.MlParams | None = ..., + path: builtins.str = ..., + should_overwrite: builtins.bool = ..., + options: collections.abc.Mapping[builtins.str, builtins.str] | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "obj_ref", b"obj_ref", "operator", b"operator", "params", b"params", "type", b"type" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "obj_ref", + b"obj_ref", + "operator", + b"operator", + "options", + b"options", + "params", + b"params", + "path", + b"path", + "should_overwrite", + b"should_overwrite", + "type", + b"type", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["type", b"type"] + ) -> typing_extensions.Literal["operator", "obj_ref"] | None: ... + + class Read(google.protobuf.message.Message): + """Command to load ML operator.""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + OPERATOR_FIELD_NUMBER: builtins.int + PATH_FIELD_NUMBER: builtins.int + @property + def operator(self) -> pyspark.sql.connect.proto.ml_common_pb2.MlOperator: + """ML operator information""" + path: builtins.str + """Load the ML instance from the input path""" + def __init__( + self, + *, + operator: pyspark.sql.connect.proto.ml_common_pb2.MlOperator | None = ..., + path: builtins.str = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["operator", b"operator"] + ) -> builtins.bool: ... + def ClearField( + self, field_name: typing_extensions.Literal["operator", b"operator", "path", b"path"] + ) -> None: ... + + FIT_FIELD_NUMBER: builtins.int + FETCH_FIELD_NUMBER: builtins.int + DELETE_FIELD_NUMBER: builtins.int + WRITE_FIELD_NUMBER: builtins.int + READ_FIELD_NUMBER: builtins.int + @property + def fit(self) -> global___MlCommand.Fit: ... + @property + def fetch(self) -> pyspark.sql.connect.proto.relations_pb2.Fetch: ... + @property + def delete(self) -> global___MlCommand.Delete: ... + @property + def write(self) -> global___MlCommand.Write: ... + @property + def read(self) -> global___MlCommand.Read: ... + def __init__( + self, + *, + fit: global___MlCommand.Fit | None = ..., + fetch: pyspark.sql.connect.proto.relations_pb2.Fetch | None = ..., + delete: global___MlCommand.Delete | None = ..., + write: global___MlCommand.Write | None = ..., + read: global___MlCommand.Read | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "command", + b"command", + "delete", + b"delete", + "fetch", + b"fetch", + "fit", + b"fit", + "read", + b"read", + "write", + b"write", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "command", + b"command", + "delete", + b"delete", + "fetch", + b"fetch", + "fit", + b"fit", + "read", + b"read", + "write", + b"write", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["command", b"command"] + ) -> typing_extensions.Literal["fit", "fetch", "delete", "write", "read"] | None: ... + +global___MlCommand = MlCommand + +class MlCommandResult(google.protobuf.message.Message): + """The result of MlCommand""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class MlOperatorInfo(google.protobuf.message.Message): + """Represents an operator info""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + OBJ_REF_FIELD_NUMBER: builtins.int + NAME_FIELD_NUMBER: builtins.int + UID_FIELD_NUMBER: builtins.int + PARAMS_FIELD_NUMBER: builtins.int + @property + def obj_ref(self) -> pyspark.sql.connect.proto.ml_common_pb2.ObjectRef: + """The cached object which could be a model or summary evaluated by a model""" + name: builtins.str + """Operator name""" + uid: builtins.str + @property + def params(self) -> pyspark.sql.connect.proto.ml_common_pb2.MlParams: ... + def __init__( + self, + *, + obj_ref: pyspark.sql.connect.proto.ml_common_pb2.ObjectRef | None = ..., + name: builtins.str = ..., + uid: builtins.str = ..., + params: pyspark.sql.connect.proto.ml_common_pb2.MlParams | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "name", b"name", "obj_ref", b"obj_ref", "params", b"params", "type", b"type" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "name", + b"name", + "obj_ref", + b"obj_ref", + "params", + b"params", + "type", + b"type", + "uid", + b"uid", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["type", b"type"] + ) -> typing_extensions.Literal["obj_ref", "name"] | None: ... + + PARAM_FIELD_NUMBER: builtins.int + SUMMARY_FIELD_NUMBER: builtins.int + OPERATOR_INFO_FIELD_NUMBER: builtins.int + @property + def param(self) -> pyspark.sql.connect.proto.ml_common_pb2.Param: + """The result of the attribute""" + summary: builtins.str + """Evaluate a Dataset in a model and return the cached ID of summary""" + @property + def operator_info(self) -> global___MlCommandResult.MlOperatorInfo: + """Operator information""" + def __init__( + self, + *, + param: pyspark.sql.connect.proto.ml_common_pb2.Param | None = ..., + summary: builtins.str = ..., + operator_info: global___MlCommandResult.MlOperatorInfo | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "operator_info", + b"operator_info", + "param", + b"param", + "result_type", + b"result_type", + "summary", + b"summary", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "operator_info", + b"operator_info", + "param", + b"param", + "result_type", + b"result_type", + "summary", + b"summary", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["result_type", b"result_type"] + ) -> typing_extensions.Literal["param", "summary", "operator_info"] | None: ... + +global___MlCommandResult = MlCommandResult diff --git a/python/pyspark/sql/connect/proto/relations_pb2.py b/python/pyspark/sql/connect/proto/relations_pb2.py index b7248d4b17080..4327d0240b355 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.py +++ b/python/pyspark/sql/connect/proto/relations_pb2.py @@ -39,10 +39,11 @@ from pyspark.sql.connect.proto import types_pb2 as spark_dot_connect_dot_types__pb2 from pyspark.sql.connect.proto import catalog_pb2 as spark_dot_connect_dot_catalog__pb2 from pyspark.sql.connect.proto import common_pb2 as spark_dot_connect_dot_common__pb2 +from pyspark.sql.connect.proto import ml_common_pb2 as spark_dot_connect_dot_ml__common__pb2 DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto\x1a\x1aspark/connect/common.proto"\xdd\x1c\n\x08Relation\x12\x35\n\x06\x63ommon\x18\x01 \x01(\x0b\x32\x1d.spark.connect.RelationCommonR\x06\x63ommon\x12)\n\x04read\x18\x02 \x01(\x0b\x32\x13.spark.connect.ReadH\x00R\x04read\x12\x32\n\x07project\x18\x03 \x01(\x0b\x32\x16.spark.connect.ProjectH\x00R\x07project\x12/\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x15.spark.connect.FilterH\x00R\x06\x66ilter\x12)\n\x04join\x18\x05 \x01(\x0b\x32\x13.spark.connect.JoinH\x00R\x04join\x12\x34\n\x06set_op\x18\x06 \x01(\x0b\x32\x1b.spark.connect.SetOperationH\x00R\x05setOp\x12)\n\x04sort\x18\x07 \x01(\x0b\x32\x13.spark.connect.SortH\x00R\x04sort\x12,\n\x05limit\x18\x08 \x01(\x0b\x32\x14.spark.connect.LimitH\x00R\x05limit\x12\x38\n\taggregate\x18\t \x01(\x0b\x32\x18.spark.connect.AggregateH\x00R\taggregate\x12&\n\x03sql\x18\n \x01(\x0b\x32\x12.spark.connect.SQLH\x00R\x03sql\x12\x45\n\x0elocal_relation\x18\x0b \x01(\x0b\x32\x1c.spark.connect.LocalRelationH\x00R\rlocalRelation\x12/\n\x06sample\x18\x0c \x01(\x0b\x32\x15.spark.connect.SampleH\x00R\x06sample\x12/\n\x06offset\x18\r \x01(\x0b\x32\x15.spark.connect.OffsetH\x00R\x06offset\x12>\n\x0b\x64\x65\x64uplicate\x18\x0e \x01(\x0b\x32\x1a.spark.connect.DeduplicateH\x00R\x0b\x64\x65\x64uplicate\x12,\n\x05range\x18\x0f \x01(\x0b\x32\x14.spark.connect.RangeH\x00R\x05range\x12\x45\n\x0esubquery_alias\x18\x10 \x01(\x0b\x32\x1c.spark.connect.SubqueryAliasH\x00R\rsubqueryAlias\x12>\n\x0brepartition\x18\x11 \x01(\x0b\x32\x1a.spark.connect.RepartitionH\x00R\x0brepartition\x12*\n\x05to_df\x18\x12 \x01(\x0b\x32\x13.spark.connect.ToDFH\x00R\x04toDf\x12U\n\x14with_columns_renamed\x18\x13 \x01(\x0b\x32!.spark.connect.WithColumnsRenamedH\x00R\x12withColumnsRenamed\x12<\n\x0bshow_string\x18\x14 \x01(\x0b\x32\x19.spark.connect.ShowStringH\x00R\nshowString\x12)\n\x04\x64rop\x18\x15 \x01(\x0b\x32\x13.spark.connect.DropH\x00R\x04\x64rop\x12)\n\x04tail\x18\x16 \x01(\x0b\x32\x13.spark.connect.TailH\x00R\x04tail\x12?\n\x0cwith_columns\x18\x17 \x01(\x0b\x32\x1a.spark.connect.WithColumnsH\x00R\x0bwithColumns\x12)\n\x04hint\x18\x18 \x01(\x0b\x32\x13.spark.connect.HintH\x00R\x04hint\x12\x32\n\x07unpivot\x18\x19 \x01(\x0b\x32\x16.spark.connect.UnpivotH\x00R\x07unpivot\x12\x36\n\tto_schema\x18\x1a \x01(\x0b\x32\x17.spark.connect.ToSchemaH\x00R\x08toSchema\x12\x64\n\x19repartition_by_expression\x18\x1b \x01(\x0b\x32&.spark.connect.RepartitionByExpressionH\x00R\x17repartitionByExpression\x12\x45\n\x0emap_partitions\x18\x1c \x01(\x0b\x32\x1c.spark.connect.MapPartitionsH\x00R\rmapPartitions\x12H\n\x0f\x63ollect_metrics\x18\x1d \x01(\x0b\x32\x1d.spark.connect.CollectMetricsH\x00R\x0e\x63ollectMetrics\x12,\n\x05parse\x18\x1e \x01(\x0b\x32\x14.spark.connect.ParseH\x00R\x05parse\x12\x36\n\tgroup_map\x18\x1f \x01(\x0b\x32\x17.spark.connect.GroupMapH\x00R\x08groupMap\x12=\n\x0c\x63o_group_map\x18 \x01(\x0b\x32\x19.spark.connect.CoGroupMapH\x00R\ncoGroupMap\x12\x45\n\x0ewith_watermark\x18! \x01(\x0b\x32\x1c.spark.connect.WithWatermarkH\x00R\rwithWatermark\x12\x63\n\x1a\x61pply_in_pandas_with_state\x18" \x01(\x0b\x32%.spark.connect.ApplyInPandasWithStateH\x00R\x16\x61pplyInPandasWithState\x12<\n\x0bhtml_string\x18# \x01(\x0b\x32\x19.spark.connect.HtmlStringH\x00R\nhtmlString\x12X\n\x15\x63\x61\x63hed_local_relation\x18$ \x01(\x0b\x32".spark.connect.CachedLocalRelationH\x00R\x13\x63\x61\x63hedLocalRelation\x12[\n\x16\x63\x61\x63hed_remote_relation\x18% \x01(\x0b\x32#.spark.connect.CachedRemoteRelationH\x00R\x14\x63\x61\x63hedRemoteRelation\x12\x8e\x01\n)common_inline_user_defined_table_function\x18& \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R$commonInlineUserDefinedTableFunction\x12\x37\n\nas_of_join\x18\' \x01(\x0b\x32\x17.spark.connect.AsOfJoinH\x00R\x08\x61sOfJoin\x12\x85\x01\n&common_inline_user_defined_data_source\x18( \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R!commonInlineUserDefinedDataSource\x12\x45\n\x0ewith_relations\x18) \x01(\x0b\x32\x1c.spark.connect.WithRelationsH\x00R\rwithRelations\x12\x38\n\ttranspose\x18* \x01(\x0b\x32\x18.spark.connect.TransposeH\x00R\ttranspose\x12w\n unresolved_table_valued_function\x18+ \x01(\x0b\x32,.spark.connect.UnresolvedTableValuedFunctionH\x00R\x1dunresolvedTableValuedFunction\x12?\n\x0clateral_join\x18, \x01(\x0b\x32\x1a.spark.connect.LateralJoinH\x00R\x0blateralJoin\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\t\n\x07Unknown"\x8e\x01\n\x0eRelationCommon\x12#\n\x0bsource_info\x18\x01 \x01(\tB\x02\x18\x01R\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12-\n\x06origin\x18\x03 \x01(\x0b\x32\x15.spark.connect.OriginR\x06originB\n\n\x08_plan_id"\xde\x03\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12O\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32&.spark.connect.SQL.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"u\n\rWithRelations\x12+\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04root\x12\x37\n\nreferences\x18\x02 \x03(\x0b\x32\x17.spark.connect.RelationR\nreferences"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xfe\x05\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x12J\n\rgrouping_sets\x18\x06 \x03(\x0b\x32%.spark.connect.Aggregate.GroupingSetsR\x0cgroupingSets\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1aL\n\x0cGroupingSets\x12<\n\x0cgrouping_set\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0bgroupingSet"\x9f\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04\x12\x1c\n\x18GROUP_TYPE_GROUPING_SETS\x10\x05"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xfe\x02\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12i\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryB\x02\x18\x01R\x10renameColumnsMap\x12\x42\n\x07renames\x18\x03 \x03(\x0b\x32(.spark.connect.WithColumnsRenamed.RenameR\x07renames\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x45\n\x06Rename\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12 \n\x0cnew_col_name\x18\x02 \x01(\tR\nnewColName"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"z\n\tTranspose\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12>\n\rindex_columns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cindexColumns"}\n\x1dUnresolvedTableValuedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xe8\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x12"\n\nprofile_id\x18\x04 \x01(\x05H\x01R\tprofileId\x88\x01\x01\x42\r\n\x0b_is_barrierB\r\n\x0b_profile_id"\xcd\x05\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x12?\n\x0cstate_schema\x18\n \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x0bstateSchema\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_confB\x0f\n\r_state_schema"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirection"\xe6\x01\n\x0bLateralJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinTypeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto\x1a\x1aspark/connect/common.proto\x1a\x1dspark/connect/ml_common.proto"\x9c\x1d\n\x08Relation\x12\x35\n\x06\x63ommon\x18\x01 \x01(\x0b\x32\x1d.spark.connect.RelationCommonR\x06\x63ommon\x12)\n\x04read\x18\x02 \x01(\x0b\x32\x13.spark.connect.ReadH\x00R\x04read\x12\x32\n\x07project\x18\x03 \x01(\x0b\x32\x16.spark.connect.ProjectH\x00R\x07project\x12/\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x15.spark.connect.FilterH\x00R\x06\x66ilter\x12)\n\x04join\x18\x05 \x01(\x0b\x32\x13.spark.connect.JoinH\x00R\x04join\x12\x34\n\x06set_op\x18\x06 \x01(\x0b\x32\x1b.spark.connect.SetOperationH\x00R\x05setOp\x12)\n\x04sort\x18\x07 \x01(\x0b\x32\x13.spark.connect.SortH\x00R\x04sort\x12,\n\x05limit\x18\x08 \x01(\x0b\x32\x14.spark.connect.LimitH\x00R\x05limit\x12\x38\n\taggregate\x18\t \x01(\x0b\x32\x18.spark.connect.AggregateH\x00R\taggregate\x12&\n\x03sql\x18\n \x01(\x0b\x32\x12.spark.connect.SQLH\x00R\x03sql\x12\x45\n\x0elocal_relation\x18\x0b \x01(\x0b\x32\x1c.spark.connect.LocalRelationH\x00R\rlocalRelation\x12/\n\x06sample\x18\x0c \x01(\x0b\x32\x15.spark.connect.SampleH\x00R\x06sample\x12/\n\x06offset\x18\r \x01(\x0b\x32\x15.spark.connect.OffsetH\x00R\x06offset\x12>\n\x0b\x64\x65\x64uplicate\x18\x0e \x01(\x0b\x32\x1a.spark.connect.DeduplicateH\x00R\x0b\x64\x65\x64uplicate\x12,\n\x05range\x18\x0f \x01(\x0b\x32\x14.spark.connect.RangeH\x00R\x05range\x12\x45\n\x0esubquery_alias\x18\x10 \x01(\x0b\x32\x1c.spark.connect.SubqueryAliasH\x00R\rsubqueryAlias\x12>\n\x0brepartition\x18\x11 \x01(\x0b\x32\x1a.spark.connect.RepartitionH\x00R\x0brepartition\x12*\n\x05to_df\x18\x12 \x01(\x0b\x32\x13.spark.connect.ToDFH\x00R\x04toDf\x12U\n\x14with_columns_renamed\x18\x13 \x01(\x0b\x32!.spark.connect.WithColumnsRenamedH\x00R\x12withColumnsRenamed\x12<\n\x0bshow_string\x18\x14 \x01(\x0b\x32\x19.spark.connect.ShowStringH\x00R\nshowString\x12)\n\x04\x64rop\x18\x15 \x01(\x0b\x32\x13.spark.connect.DropH\x00R\x04\x64rop\x12)\n\x04tail\x18\x16 \x01(\x0b\x32\x13.spark.connect.TailH\x00R\x04tail\x12?\n\x0cwith_columns\x18\x17 \x01(\x0b\x32\x1a.spark.connect.WithColumnsH\x00R\x0bwithColumns\x12)\n\x04hint\x18\x18 \x01(\x0b\x32\x13.spark.connect.HintH\x00R\x04hint\x12\x32\n\x07unpivot\x18\x19 \x01(\x0b\x32\x16.spark.connect.UnpivotH\x00R\x07unpivot\x12\x36\n\tto_schema\x18\x1a \x01(\x0b\x32\x17.spark.connect.ToSchemaH\x00R\x08toSchema\x12\x64\n\x19repartition_by_expression\x18\x1b \x01(\x0b\x32&.spark.connect.RepartitionByExpressionH\x00R\x17repartitionByExpression\x12\x45\n\x0emap_partitions\x18\x1c \x01(\x0b\x32\x1c.spark.connect.MapPartitionsH\x00R\rmapPartitions\x12H\n\x0f\x63ollect_metrics\x18\x1d \x01(\x0b\x32\x1d.spark.connect.CollectMetricsH\x00R\x0e\x63ollectMetrics\x12,\n\x05parse\x18\x1e \x01(\x0b\x32\x14.spark.connect.ParseH\x00R\x05parse\x12\x36\n\tgroup_map\x18\x1f \x01(\x0b\x32\x17.spark.connect.GroupMapH\x00R\x08groupMap\x12=\n\x0c\x63o_group_map\x18 \x01(\x0b\x32\x19.spark.connect.CoGroupMapH\x00R\ncoGroupMap\x12\x45\n\x0ewith_watermark\x18! \x01(\x0b\x32\x1c.spark.connect.WithWatermarkH\x00R\rwithWatermark\x12\x63\n\x1a\x61pply_in_pandas_with_state\x18" \x01(\x0b\x32%.spark.connect.ApplyInPandasWithStateH\x00R\x16\x61pplyInPandasWithState\x12<\n\x0bhtml_string\x18# \x01(\x0b\x32\x19.spark.connect.HtmlStringH\x00R\nhtmlString\x12X\n\x15\x63\x61\x63hed_local_relation\x18$ \x01(\x0b\x32".spark.connect.CachedLocalRelationH\x00R\x13\x63\x61\x63hedLocalRelation\x12[\n\x16\x63\x61\x63hed_remote_relation\x18% \x01(\x0b\x32#.spark.connect.CachedRemoteRelationH\x00R\x14\x63\x61\x63hedRemoteRelation\x12\x8e\x01\n)common_inline_user_defined_table_function\x18& \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R$commonInlineUserDefinedTableFunction\x12\x37\n\nas_of_join\x18\' \x01(\x0b\x32\x17.spark.connect.AsOfJoinH\x00R\x08\x61sOfJoin\x12\x85\x01\n&common_inline_user_defined_data_source\x18( \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R!commonInlineUserDefinedDataSource\x12\x45\n\x0ewith_relations\x18) \x01(\x0b\x32\x1c.spark.connect.WithRelationsH\x00R\rwithRelations\x12\x38\n\ttranspose\x18* \x01(\x0b\x32\x18.spark.connect.TransposeH\x00R\ttranspose\x12w\n unresolved_table_valued_function\x18+ \x01(\x0b\x32,.spark.connect.UnresolvedTableValuedFunctionH\x00R\x1dunresolvedTableValuedFunction\x12?\n\x0clateral_join\x18, \x01(\x0b\x32\x1a.spark.connect.LateralJoinH\x00R\x0blateralJoin\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12=\n\x0bml_relation\x18\xac\x02 \x01(\x0b\x32\x19.spark.connect.MlRelationH\x00R\nmlRelation\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\xf8\x02\n\nMlRelation\x12\x43\n\ttransform\x18\x01 \x01(\x0b\x32#.spark.connect.MlRelation.TransformH\x00R\ttransform\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x1a\xeb\x01\n\tTransform\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12=\n\x0btransformer\x18\x02 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x0btransformer\x12-\n\x05input\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06paramsB\n\n\x08operatorB\t\n\x07ml_type"\xbe\x02\n\x05\x46\x65tch\x12\x31\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x06objRef\x12\x35\n\x07methods\x18\x02 \x03(\x0b\x32\x1b.spark.connect.Fetch.MethodR\x07methods\x1a\xca\x01\n\x06Method\x12\x16\n\x06method\x18\x01 \x01(\tR\x06method\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32 .spark.connect.Fetch.Method.ArgsR\x04\x61rgs\x1ar\n\x04\x41rgs\x12,\n\x05param\x18\x01 \x01(\x0b\x32\x14.spark.connect.ParamH\x00R\x05param\x12/\n\x05input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x05inputB\x0b\n\targs_type"\t\n\x07Unknown"\x8e\x01\n\x0eRelationCommon\x12#\n\x0bsource_info\x18\x01 \x01(\tB\x02\x18\x01R\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12-\n\x06origin\x18\x03 \x01(\x0b\x32\x15.spark.connect.OriginR\x06originB\n\n\x08_plan_id"\xde\x03\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12O\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32&.spark.connect.SQL.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"u\n\rWithRelations\x12+\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04root\x12\x37\n\nreferences\x18\x02 \x03(\x0b\x32\x17.spark.connect.RelationR\nreferences"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xfe\x05\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x12J\n\rgrouping_sets\x18\x06 \x03(\x0b\x32%.spark.connect.Aggregate.GroupingSetsR\x0cgroupingSets\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1aL\n\x0cGroupingSets\x12<\n\x0cgrouping_set\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0bgroupingSet"\x9f\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04\x12\x1c\n\x18GROUP_TYPE_GROUPING_SETS\x10\x05"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xfe\x02\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12i\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryB\x02\x18\x01R\x10renameColumnsMap\x12\x42\n\x07renames\x18\x03 \x03(\x0b\x32(.spark.connect.WithColumnsRenamed.RenameR\x07renames\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x45\n\x06Rename\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12 \n\x0cnew_col_name\x18\x02 \x01(\tR\nnewColName"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"z\n\tTranspose\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12>\n\rindex_columns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cindexColumns"}\n\x1dUnresolvedTableValuedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xe8\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x12"\n\nprofile_id\x18\x04 \x01(\x05H\x01R\tprofileId\x88\x01\x01\x42\r\n\x0b_is_barrierB\r\n\x0b_profile_id"\xcd\x05\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x12?\n\x0cstate_schema\x18\n \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x0bstateSchema\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_confB\x0f\n\r_state_schema"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirection"\xe6\x01\n\x0bLateralJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinTypeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -77,160 +78,170 @@ ]._serialized_options = b"\030\001" _globals["_PARSE_OPTIONSENTRY"]._loaded_options = None _globals["_PARSE_OPTIONSENTRY"]._serialized_options = b"8\001" - _globals["_RELATION"]._serialized_start = 193 - _globals["_RELATION"]._serialized_end = 3870 - _globals["_UNKNOWN"]._serialized_start = 3872 - _globals["_UNKNOWN"]._serialized_end = 3881 - _globals["_RELATIONCOMMON"]._serialized_start = 3884 - _globals["_RELATIONCOMMON"]._serialized_end = 4026 - _globals["_SQL"]._serialized_start = 4029 - _globals["_SQL"]._serialized_end = 4507 - _globals["_SQL_ARGSENTRY"]._serialized_start = 4323 - _globals["_SQL_ARGSENTRY"]._serialized_end = 4413 - _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_start = 4415 - _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_end = 4507 - _globals["_WITHRELATIONS"]._serialized_start = 4509 - _globals["_WITHRELATIONS"]._serialized_end = 4626 - _globals["_READ"]._serialized_start = 4629 - _globals["_READ"]._serialized_end = 5292 - _globals["_READ_NAMEDTABLE"]._serialized_start = 4807 - _globals["_READ_NAMEDTABLE"]._serialized_end = 4999 - _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_start = 4941 - _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_end = 4999 - _globals["_READ_DATASOURCE"]._serialized_start = 5002 - _globals["_READ_DATASOURCE"]._serialized_end = 5279 - _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_start = 4941 - _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_end = 4999 - _globals["_PROJECT"]._serialized_start = 5294 - _globals["_PROJECT"]._serialized_end = 5411 - _globals["_FILTER"]._serialized_start = 5413 - _globals["_FILTER"]._serialized_end = 5525 - _globals["_JOIN"]._serialized_start = 5528 - _globals["_JOIN"]._serialized_end = 6189 - _globals["_JOIN_JOINDATATYPE"]._serialized_start = 5867 - _globals["_JOIN_JOINDATATYPE"]._serialized_end = 5959 - _globals["_JOIN_JOINTYPE"]._serialized_start = 5962 - _globals["_JOIN_JOINTYPE"]._serialized_end = 6170 - _globals["_SETOPERATION"]._serialized_start = 6192 - _globals["_SETOPERATION"]._serialized_end = 6671 - _globals["_SETOPERATION_SETOPTYPE"]._serialized_start = 6508 - _globals["_SETOPERATION_SETOPTYPE"]._serialized_end = 6622 - _globals["_LIMIT"]._serialized_start = 6673 - _globals["_LIMIT"]._serialized_end = 6749 - _globals["_OFFSET"]._serialized_start = 6751 - _globals["_OFFSET"]._serialized_end = 6830 - _globals["_TAIL"]._serialized_start = 6832 - _globals["_TAIL"]._serialized_end = 6907 - _globals["_AGGREGATE"]._serialized_start = 6910 - _globals["_AGGREGATE"]._serialized_end = 7676 - _globals["_AGGREGATE_PIVOT"]._serialized_start = 7325 - _globals["_AGGREGATE_PIVOT"]._serialized_end = 7436 - _globals["_AGGREGATE_GROUPINGSETS"]._serialized_start = 7438 - _globals["_AGGREGATE_GROUPINGSETS"]._serialized_end = 7514 - _globals["_AGGREGATE_GROUPTYPE"]._serialized_start = 7517 - _globals["_AGGREGATE_GROUPTYPE"]._serialized_end = 7676 - _globals["_SORT"]._serialized_start = 7679 - _globals["_SORT"]._serialized_end = 7839 - _globals["_DROP"]._serialized_start = 7842 - _globals["_DROP"]._serialized_end = 7983 - _globals["_DEDUPLICATE"]._serialized_start = 7986 - _globals["_DEDUPLICATE"]._serialized_end = 8226 - _globals["_LOCALRELATION"]._serialized_start = 8228 - _globals["_LOCALRELATION"]._serialized_end = 8317 - _globals["_CACHEDLOCALRELATION"]._serialized_start = 8319 - _globals["_CACHEDLOCALRELATION"]._serialized_end = 8391 - _globals["_CACHEDREMOTERELATION"]._serialized_start = 8393 - _globals["_CACHEDREMOTERELATION"]._serialized_end = 8448 - _globals["_SAMPLE"]._serialized_start = 8451 - _globals["_SAMPLE"]._serialized_end = 8724 - _globals["_RANGE"]._serialized_start = 8727 - _globals["_RANGE"]._serialized_end = 8872 - _globals["_SUBQUERYALIAS"]._serialized_start = 8874 - _globals["_SUBQUERYALIAS"]._serialized_end = 8988 - _globals["_REPARTITION"]._serialized_start = 8991 - _globals["_REPARTITION"]._serialized_end = 9133 - _globals["_SHOWSTRING"]._serialized_start = 9136 - _globals["_SHOWSTRING"]._serialized_end = 9278 - _globals["_HTMLSTRING"]._serialized_start = 9280 - _globals["_HTMLSTRING"]._serialized_end = 9394 - _globals["_STATSUMMARY"]._serialized_start = 9396 - _globals["_STATSUMMARY"]._serialized_end = 9488 - _globals["_STATDESCRIBE"]._serialized_start = 9490 - _globals["_STATDESCRIBE"]._serialized_end = 9571 - _globals["_STATCROSSTAB"]._serialized_start = 9573 - _globals["_STATCROSSTAB"]._serialized_end = 9674 - _globals["_STATCOV"]._serialized_start = 9676 - _globals["_STATCOV"]._serialized_end = 9772 - _globals["_STATCORR"]._serialized_start = 9775 - _globals["_STATCORR"]._serialized_end = 9912 - _globals["_STATAPPROXQUANTILE"]._serialized_start = 9915 - _globals["_STATAPPROXQUANTILE"]._serialized_end = 10079 - _globals["_STATFREQITEMS"]._serialized_start = 10081 - _globals["_STATFREQITEMS"]._serialized_end = 10206 - _globals["_STATSAMPLEBY"]._serialized_start = 10209 - _globals["_STATSAMPLEBY"]._serialized_end = 10518 - _globals["_STATSAMPLEBY_FRACTION"]._serialized_start = 10410 - _globals["_STATSAMPLEBY_FRACTION"]._serialized_end = 10509 - _globals["_NAFILL"]._serialized_start = 10521 - _globals["_NAFILL"]._serialized_end = 10655 - _globals["_NADROP"]._serialized_start = 10658 - _globals["_NADROP"]._serialized_end = 10792 - _globals["_NAREPLACE"]._serialized_start = 10795 - _globals["_NAREPLACE"]._serialized_end = 11091 - _globals["_NAREPLACE_REPLACEMENT"]._serialized_start = 10950 - _globals["_NAREPLACE_REPLACEMENT"]._serialized_end = 11091 - _globals["_TODF"]._serialized_start = 11093 - _globals["_TODF"]._serialized_end = 11181 - _globals["_WITHCOLUMNSRENAMED"]._serialized_start = 11184 - _globals["_WITHCOLUMNSRENAMED"]._serialized_end = 11566 - _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_start = 11428 - _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_end = 11495 - _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_start = 11497 - _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_end = 11566 - _globals["_WITHCOLUMNS"]._serialized_start = 11568 - _globals["_WITHCOLUMNS"]._serialized_end = 11687 - _globals["_WITHWATERMARK"]._serialized_start = 11690 - _globals["_WITHWATERMARK"]._serialized_end = 11824 - _globals["_HINT"]._serialized_start = 11827 - _globals["_HINT"]._serialized_end = 11959 - _globals["_UNPIVOT"]._serialized_start = 11962 - _globals["_UNPIVOT"]._serialized_end = 12289 - _globals["_UNPIVOT_VALUES"]._serialized_start = 12219 - _globals["_UNPIVOT_VALUES"]._serialized_end = 12278 - _globals["_TRANSPOSE"]._serialized_start = 12291 - _globals["_TRANSPOSE"]._serialized_end = 12413 - _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_start = 12415 - _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_end = 12540 - _globals["_TOSCHEMA"]._serialized_start = 12542 - _globals["_TOSCHEMA"]._serialized_end = 12648 - _globals["_REPARTITIONBYEXPRESSION"]._serialized_start = 12651 - _globals["_REPARTITIONBYEXPRESSION"]._serialized_end = 12854 - _globals["_MAPPARTITIONS"]._serialized_start = 12857 - _globals["_MAPPARTITIONS"]._serialized_end = 13089 - _globals["_GROUPMAP"]._serialized_start = 13092 - _globals["_GROUPMAP"]._serialized_end = 13809 - _globals["_COGROUPMAP"]._serialized_start = 13812 - _globals["_COGROUPMAP"]._serialized_end = 14338 - _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 14341 - _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 14698 - _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 14701 - _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 14945 - _globals["_PYTHONUDTF"]._serialized_start = 14948 - _globals["_PYTHONUDTF"]._serialized_end = 15125 - _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 15128 - _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 15279 - _globals["_PYTHONDATASOURCE"]._serialized_start = 15281 - _globals["_PYTHONDATASOURCE"]._serialized_end = 15356 - _globals["_COLLECTMETRICS"]._serialized_start = 15359 - _globals["_COLLECTMETRICS"]._serialized_end = 15495 - _globals["_PARSE"]._serialized_start = 15498 - _globals["_PARSE"]._serialized_end = 15886 - _globals["_PARSE_OPTIONSENTRY"]._serialized_start = 4941 - _globals["_PARSE_OPTIONSENTRY"]._serialized_end = 4999 - _globals["_PARSE_PARSEFORMAT"]._serialized_start = 15787 - _globals["_PARSE_PARSEFORMAT"]._serialized_end = 15875 - _globals["_ASOFJOIN"]._serialized_start = 15889 - _globals["_ASOFJOIN"]._serialized_end = 16364 - _globals["_LATERALJOIN"]._serialized_start = 16367 - _globals["_LATERALJOIN"]._serialized_end = 16597 + _globals["_RELATION"]._serialized_start = 224 + _globals["_RELATION"]._serialized_end = 3964 + _globals["_MLRELATION"]._serialized_start = 3967 + _globals["_MLRELATION"]._serialized_end = 4343 + _globals["_MLRELATION_TRANSFORM"]._serialized_start = 4097 + _globals["_MLRELATION_TRANSFORM"]._serialized_end = 4332 + _globals["_FETCH"]._serialized_start = 4346 + _globals["_FETCH"]._serialized_end = 4664 + _globals["_FETCH_METHOD"]._serialized_start = 4462 + _globals["_FETCH_METHOD"]._serialized_end = 4664 + _globals["_FETCH_METHOD_ARGS"]._serialized_start = 4550 + _globals["_FETCH_METHOD_ARGS"]._serialized_end = 4664 + _globals["_UNKNOWN"]._serialized_start = 4666 + _globals["_UNKNOWN"]._serialized_end = 4675 + _globals["_RELATIONCOMMON"]._serialized_start = 4678 + _globals["_RELATIONCOMMON"]._serialized_end = 4820 + _globals["_SQL"]._serialized_start = 4823 + _globals["_SQL"]._serialized_end = 5301 + _globals["_SQL_ARGSENTRY"]._serialized_start = 5117 + _globals["_SQL_ARGSENTRY"]._serialized_end = 5207 + _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_start = 5209 + _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_end = 5301 + _globals["_WITHRELATIONS"]._serialized_start = 5303 + _globals["_WITHRELATIONS"]._serialized_end = 5420 + _globals["_READ"]._serialized_start = 5423 + _globals["_READ"]._serialized_end = 6086 + _globals["_READ_NAMEDTABLE"]._serialized_start = 5601 + _globals["_READ_NAMEDTABLE"]._serialized_end = 5793 + _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_start = 5735 + _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_end = 5793 + _globals["_READ_DATASOURCE"]._serialized_start = 5796 + _globals["_READ_DATASOURCE"]._serialized_end = 6073 + _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_start = 5735 + _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_end = 5793 + _globals["_PROJECT"]._serialized_start = 6088 + _globals["_PROJECT"]._serialized_end = 6205 + _globals["_FILTER"]._serialized_start = 6207 + _globals["_FILTER"]._serialized_end = 6319 + _globals["_JOIN"]._serialized_start = 6322 + _globals["_JOIN"]._serialized_end = 6983 + _globals["_JOIN_JOINDATATYPE"]._serialized_start = 6661 + _globals["_JOIN_JOINDATATYPE"]._serialized_end = 6753 + _globals["_JOIN_JOINTYPE"]._serialized_start = 6756 + _globals["_JOIN_JOINTYPE"]._serialized_end = 6964 + _globals["_SETOPERATION"]._serialized_start = 6986 + _globals["_SETOPERATION"]._serialized_end = 7465 + _globals["_SETOPERATION_SETOPTYPE"]._serialized_start = 7302 + _globals["_SETOPERATION_SETOPTYPE"]._serialized_end = 7416 + _globals["_LIMIT"]._serialized_start = 7467 + _globals["_LIMIT"]._serialized_end = 7543 + _globals["_OFFSET"]._serialized_start = 7545 + _globals["_OFFSET"]._serialized_end = 7624 + _globals["_TAIL"]._serialized_start = 7626 + _globals["_TAIL"]._serialized_end = 7701 + _globals["_AGGREGATE"]._serialized_start = 7704 + _globals["_AGGREGATE"]._serialized_end = 8470 + _globals["_AGGREGATE_PIVOT"]._serialized_start = 8119 + _globals["_AGGREGATE_PIVOT"]._serialized_end = 8230 + _globals["_AGGREGATE_GROUPINGSETS"]._serialized_start = 8232 + _globals["_AGGREGATE_GROUPINGSETS"]._serialized_end = 8308 + _globals["_AGGREGATE_GROUPTYPE"]._serialized_start = 8311 + _globals["_AGGREGATE_GROUPTYPE"]._serialized_end = 8470 + _globals["_SORT"]._serialized_start = 8473 + _globals["_SORT"]._serialized_end = 8633 + _globals["_DROP"]._serialized_start = 8636 + _globals["_DROP"]._serialized_end = 8777 + _globals["_DEDUPLICATE"]._serialized_start = 8780 + _globals["_DEDUPLICATE"]._serialized_end = 9020 + _globals["_LOCALRELATION"]._serialized_start = 9022 + _globals["_LOCALRELATION"]._serialized_end = 9111 + _globals["_CACHEDLOCALRELATION"]._serialized_start = 9113 + _globals["_CACHEDLOCALRELATION"]._serialized_end = 9185 + _globals["_CACHEDREMOTERELATION"]._serialized_start = 9187 + _globals["_CACHEDREMOTERELATION"]._serialized_end = 9242 + _globals["_SAMPLE"]._serialized_start = 9245 + _globals["_SAMPLE"]._serialized_end = 9518 + _globals["_RANGE"]._serialized_start = 9521 + _globals["_RANGE"]._serialized_end = 9666 + _globals["_SUBQUERYALIAS"]._serialized_start = 9668 + _globals["_SUBQUERYALIAS"]._serialized_end = 9782 + _globals["_REPARTITION"]._serialized_start = 9785 + _globals["_REPARTITION"]._serialized_end = 9927 + _globals["_SHOWSTRING"]._serialized_start = 9930 + _globals["_SHOWSTRING"]._serialized_end = 10072 + _globals["_HTMLSTRING"]._serialized_start = 10074 + _globals["_HTMLSTRING"]._serialized_end = 10188 + _globals["_STATSUMMARY"]._serialized_start = 10190 + _globals["_STATSUMMARY"]._serialized_end = 10282 + _globals["_STATDESCRIBE"]._serialized_start = 10284 + _globals["_STATDESCRIBE"]._serialized_end = 10365 + _globals["_STATCROSSTAB"]._serialized_start = 10367 + _globals["_STATCROSSTAB"]._serialized_end = 10468 + _globals["_STATCOV"]._serialized_start = 10470 + _globals["_STATCOV"]._serialized_end = 10566 + _globals["_STATCORR"]._serialized_start = 10569 + _globals["_STATCORR"]._serialized_end = 10706 + _globals["_STATAPPROXQUANTILE"]._serialized_start = 10709 + _globals["_STATAPPROXQUANTILE"]._serialized_end = 10873 + _globals["_STATFREQITEMS"]._serialized_start = 10875 + _globals["_STATFREQITEMS"]._serialized_end = 11000 + _globals["_STATSAMPLEBY"]._serialized_start = 11003 + _globals["_STATSAMPLEBY"]._serialized_end = 11312 + _globals["_STATSAMPLEBY_FRACTION"]._serialized_start = 11204 + _globals["_STATSAMPLEBY_FRACTION"]._serialized_end = 11303 + _globals["_NAFILL"]._serialized_start = 11315 + _globals["_NAFILL"]._serialized_end = 11449 + _globals["_NADROP"]._serialized_start = 11452 + _globals["_NADROP"]._serialized_end = 11586 + _globals["_NAREPLACE"]._serialized_start = 11589 + _globals["_NAREPLACE"]._serialized_end = 11885 + _globals["_NAREPLACE_REPLACEMENT"]._serialized_start = 11744 + _globals["_NAREPLACE_REPLACEMENT"]._serialized_end = 11885 + _globals["_TODF"]._serialized_start = 11887 + _globals["_TODF"]._serialized_end = 11975 + _globals["_WITHCOLUMNSRENAMED"]._serialized_start = 11978 + _globals["_WITHCOLUMNSRENAMED"]._serialized_end = 12360 + _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_start = 12222 + _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_end = 12289 + _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_start = 12291 + _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_end = 12360 + _globals["_WITHCOLUMNS"]._serialized_start = 12362 + _globals["_WITHCOLUMNS"]._serialized_end = 12481 + _globals["_WITHWATERMARK"]._serialized_start = 12484 + _globals["_WITHWATERMARK"]._serialized_end = 12618 + _globals["_HINT"]._serialized_start = 12621 + _globals["_HINT"]._serialized_end = 12753 + _globals["_UNPIVOT"]._serialized_start = 12756 + _globals["_UNPIVOT"]._serialized_end = 13083 + _globals["_UNPIVOT_VALUES"]._serialized_start = 13013 + _globals["_UNPIVOT_VALUES"]._serialized_end = 13072 + _globals["_TRANSPOSE"]._serialized_start = 13085 + _globals["_TRANSPOSE"]._serialized_end = 13207 + _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_start = 13209 + _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_end = 13334 + _globals["_TOSCHEMA"]._serialized_start = 13336 + _globals["_TOSCHEMA"]._serialized_end = 13442 + _globals["_REPARTITIONBYEXPRESSION"]._serialized_start = 13445 + _globals["_REPARTITIONBYEXPRESSION"]._serialized_end = 13648 + _globals["_MAPPARTITIONS"]._serialized_start = 13651 + _globals["_MAPPARTITIONS"]._serialized_end = 13883 + _globals["_GROUPMAP"]._serialized_start = 13886 + _globals["_GROUPMAP"]._serialized_end = 14603 + _globals["_COGROUPMAP"]._serialized_start = 14606 + _globals["_COGROUPMAP"]._serialized_end = 15132 + _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 15135 + _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 15492 + _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 15495 + _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 15739 + _globals["_PYTHONUDTF"]._serialized_start = 15742 + _globals["_PYTHONUDTF"]._serialized_end = 15919 + _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 15922 + _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 16073 + _globals["_PYTHONDATASOURCE"]._serialized_start = 16075 + _globals["_PYTHONDATASOURCE"]._serialized_end = 16150 + _globals["_COLLECTMETRICS"]._serialized_start = 16153 + _globals["_COLLECTMETRICS"]._serialized_end = 16289 + _globals["_PARSE"]._serialized_start = 16292 + _globals["_PARSE"]._serialized_end = 16680 + _globals["_PARSE_OPTIONSENTRY"]._serialized_start = 5735 + _globals["_PARSE_OPTIONSENTRY"]._serialized_end = 5793 + _globals["_PARSE_PARSEFORMAT"]._serialized_start = 16581 + _globals["_PARSE_PARSEFORMAT"]._serialized_end = 16669 + _globals["_ASOFJOIN"]._serialized_start = 16683 + _globals["_ASOFJOIN"]._serialized_end = 17158 + _globals["_LATERALJOIN"]._serialized_start = 17161 + _globals["_LATERALJOIN"]._serialized_end = 17391 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/relations_pb2.pyi b/python/pyspark/sql/connect/proto/relations_pb2.pyi index 371d735b9e871..0c8cf8dd3eda8 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.pyi +++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi @@ -43,6 +43,7 @@ import google.protobuf.message import pyspark.sql.connect.proto.catalog_pb2 import pyspark.sql.connect.proto.common_pb2 import pyspark.sql.connect.proto.expressions_pb2 +import pyspark.sql.connect.proto.ml_common_pb2 import pyspark.sql.connect.proto.types_pb2 import sys import typing @@ -119,6 +120,7 @@ class Relation(google.protobuf.message.Message): FREQ_ITEMS_FIELD_NUMBER: builtins.int SAMPLE_BY_FIELD_NUMBER: builtins.int CATALOG_FIELD_NUMBER: builtins.int + ML_RELATION_FIELD_NUMBER: builtins.int EXTENSION_FIELD_NUMBER: builtins.int UNKNOWN_FIELD_NUMBER: builtins.int @property @@ -241,6 +243,9 @@ class Relation(google.protobuf.message.Message): def catalog(self) -> pyspark.sql.connect.proto.catalog_pb2.Catalog: """Catalog API (experimental / unstable)""" @property + def ml_relation(self) -> global___MlRelation: + """ML relation""" + @property def extension(self) -> google.protobuf.any_pb2.Any: """This field is used to mark extensions to the protocol. When plugins generate arbitrary relations they can add them here. During the planning the correct resolution is done. @@ -308,6 +313,7 @@ class Relation(google.protobuf.message.Message): freq_items: global___StatFreqItems | None = ..., sample_by: global___StatSampleBy | None = ..., catalog: pyspark.sql.connect.proto.catalog_pb2.Catalog | None = ..., + ml_relation: global___MlRelation | None = ..., extension: google.protobuf.any_pb2.Any | None = ..., unknown: global___Unknown | None = ..., ) -> None: ... @@ -376,6 +382,8 @@ class Relation(google.protobuf.message.Message): b"local_relation", "map_partitions", b"map_partitions", + "ml_relation", + b"ml_relation", "offset", b"offset", "parse", @@ -499,6 +507,8 @@ class Relation(google.protobuf.message.Message): b"local_relation", "map_partitions", b"map_partitions", + "ml_relation", + b"ml_relation", "offset", b"offset", "parse", @@ -616,6 +626,7 @@ class Relation(google.protobuf.message.Message): "freq_items", "sample_by", "catalog", + "ml_relation", "extension", "unknown", ] @@ -624,6 +635,198 @@ class Relation(google.protobuf.message.Message): global___Relation = Relation +class MlRelation(google.protobuf.message.Message): + """Relation to represent ML world""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class Transform(google.protobuf.message.Message): + """Relation to represent transform(input) of the operator + which could be a cached model or a new transformer + """ + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + OBJ_REF_FIELD_NUMBER: builtins.int + TRANSFORMER_FIELD_NUMBER: builtins.int + INPUT_FIELD_NUMBER: builtins.int + PARAMS_FIELD_NUMBER: builtins.int + @property + def obj_ref(self) -> pyspark.sql.connect.proto.ml_common_pb2.ObjectRef: + """Object reference""" + @property + def transformer(self) -> pyspark.sql.connect.proto.ml_common_pb2.MlOperator: + """Could be an ML transformer like VectorAssembler""" + @property + def input(self) -> global___Relation: + """the input dataframe""" + @property + def params(self) -> pyspark.sql.connect.proto.ml_common_pb2.MlParams: + """the operator specific parameters""" + def __init__( + self, + *, + obj_ref: pyspark.sql.connect.proto.ml_common_pb2.ObjectRef | None = ..., + transformer: pyspark.sql.connect.proto.ml_common_pb2.MlOperator | None = ..., + input: global___Relation | None = ..., + params: pyspark.sql.connect.proto.ml_common_pb2.MlParams | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "input", + b"input", + "obj_ref", + b"obj_ref", + "operator", + b"operator", + "params", + b"params", + "transformer", + b"transformer", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "input", + b"input", + "obj_ref", + b"obj_ref", + "operator", + b"operator", + "params", + b"params", + "transformer", + b"transformer", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["operator", b"operator"] + ) -> typing_extensions.Literal["obj_ref", "transformer"] | None: ... + + TRANSFORM_FIELD_NUMBER: builtins.int + FETCH_FIELD_NUMBER: builtins.int + @property + def transform(self) -> global___MlRelation.Transform: ... + @property + def fetch(self) -> global___Fetch: ... + def __init__( + self, + *, + transform: global___MlRelation.Transform | None = ..., + fetch: global___Fetch | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "fetch", b"fetch", "ml_type", b"ml_type", "transform", b"transform" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "fetch", b"fetch", "ml_type", b"ml_type", "transform", b"transform" + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["ml_type", b"ml_type"] + ) -> typing_extensions.Literal["transform", "fetch"] | None: ... + +global___MlRelation = MlRelation + +class Fetch(google.protobuf.message.Message): + """Message for fetching attribute from object on the server side. + Fetch can be represented as a Relation or a ML command + Command: model.coefficients, model.summary.weightedPrecision which + returns the final literal result + Relation: model.summary.roc which returns a DataFrame (Relation) + """ + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class Method(google.protobuf.message.Message): + """Represents a method with inclusion of method name and its arguments""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class Args(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + PARAM_FIELD_NUMBER: builtins.int + INPUT_FIELD_NUMBER: builtins.int + @property + def param(self) -> pyspark.sql.connect.proto.ml_common_pb2.Param: ... + @property + def input(self) -> global___Relation: ... + def __init__( + self, + *, + param: pyspark.sql.connect.proto.ml_common_pb2.Param | None = ..., + input: global___Relation | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "args_type", b"args_type", "input", b"input", "param", b"param" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "args_type", b"args_type", "input", b"input", "param", b"param" + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["args_type", b"args_type"] + ) -> typing_extensions.Literal["param", "input"] | None: ... + + METHOD_FIELD_NUMBER: builtins.int + ARGS_FIELD_NUMBER: builtins.int + method: builtins.str + """(Required) the method name""" + @property + def args( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + global___Fetch.Method.Args + ]: + """(Optional) the arguments of the method""" + def __init__( + self, + *, + method: builtins.str = ..., + args: collections.abc.Iterable[global___Fetch.Method.Args] | None = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["args", b"args", "method", b"method"] + ) -> None: ... + + OBJ_REF_FIELD_NUMBER: builtins.int + METHODS_FIELD_NUMBER: builtins.int + @property + def obj_ref(self) -> pyspark.sql.connect.proto.ml_common_pb2.ObjectRef: + """(Required) reference to the object on the server side""" + @property + def methods( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[global___Fetch.Method]: + """(Required) the calling method chains""" + def __init__( + self, + *, + obj_ref: pyspark.sql.connect.proto.ml_common_pb2.ObjectRef | None = ..., + methods: collections.abc.Iterable[global___Fetch.Method] | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["obj_ref", b"obj_ref"] + ) -> builtins.bool: ... + def ClearField( + self, field_name: typing_extensions.Literal["methods", b"methods", "obj_ref", b"obj_ref"] + ) -> None: ... + +global___Fetch = Fetch + class Unknown(google.protobuf.message.Message): """Used for testing purposes only.""" diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 5ab186b2957e7..f5bb269c23d6e 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -1467,7 +1467,9 @@ def createDataFrame( # type: ignore[misc] +-----+---+ |Alice| 1| +-----+---+ - >>> spark.createDataFrame(pandas.DataFrame([[1, 2]])).collect() # doctest: +SKIP + + >>> pdf = pandas.DataFrame([[1, 2]]) # doctest: +SKIP + >>> spark.createDataFrame(pdf).show() # doctest: +SKIP +---+---+ | 0| 1| +---+---+ @@ -1482,8 +1484,9 @@ def createDataFrame( # type: ignore[misc] +-----+---+ |Alice| 1| +-----+---+ + >>> table = pyarrow.table({'0': [1], '1': [2]}) # doctest: +SKIP - >>> spark.createDataFrame(table).collect() # doctest: +SKIP + >>> spark.createDataFrame(table).show() # doctest: +SKIP +---+---+ | 0| 1| +---+---+ diff --git a/python/pyspark/sql/table_arg.py b/python/pyspark/sql/table_arg.py index d4b5e1653c7a1..cacfd24b2f1ba 100644 --- a/python/pyspark/sql/table_arg.py +++ b/python/pyspark/sql/table_arg.py @@ -17,7 +17,6 @@ from typing import TYPE_CHECKING -from pyspark.sql.classic.column import _to_java_column, _to_seq from pyspark.sql.tvf_argument import TableValuedFunctionArgument from pyspark.sql.utils import get_active_spark_context @@ -32,6 +31,8 @@ def __init__(self, j_table_arg: "JavaObject"): self._j_table_arg = j_table_arg def partitionBy(self, *cols: "ColumnOrName") -> "TableArg": + from pyspark.sql.classic.column import _to_java_column, _to_seq + sc = get_active_spark_context() if len(cols) == 1 and isinstance(cols[0], list): cols = cols[0] @@ -40,6 +41,8 @@ def partitionBy(self, *cols: "ColumnOrName") -> "TableArg": return TableArg(new_j_table_arg) def orderBy(self, *cols: "ColumnOrName") -> "TableArg": + from pyspark.sql.classic.column import _to_java_column, _to_seq + sc = get_active_spark_context() if len(cols) == 1 and isinstance(cols[0], list): cols = cols[0] diff --git a/python/pyspark/sql/tests/connect/test_parity_subquery.py b/python/pyspark/sql/tests/connect/test_parity_subquery.py index dae60a354d20a..f3225fcb7f2dd 100644 --- a/python/pyspark/sql/tests/connect/test_parity_subquery.py +++ b/python/pyspark/sql/tests/connect/test_parity_subquery.py @@ -45,10 +45,6 @@ def test_scalar_subquery_with_missing_outer_reference(self): def test_subquery_in_unpivot(self): self.check_subquery_in_unpivot(None, None) - @unittest.skip("SPARK-50601: Fix the SparkConnectPlanner to support this case") - def test_subquery_in_with_columns(self): - super().test_subquery_in_with_columns() - if __name__ == "__main__": from pyspark.sql.tests.connect.test_parity_subquery import * # noqa: F401 diff --git a/python/pyspark/sql/tests/test_subquery.py b/python/pyspark/sql/tests/test_subquery.py index 99a22d7c29664..7c63ddb69458e 100644 --- a/python/pyspark/sql/tests/test_subquery.py +++ b/python/pyspark/sql/tests/test_subquery.py @@ -939,7 +939,44 @@ def test_subquery_in_with_columns(self): .select(sf.col("c1").outer() + sf.col("c2").outer()) .scalar(), ), - t1.withColumn("scalar", sf.col("c1") + sf.col("c2")), + t1.select("*", (sf.col("c1") + sf.col("c2")).alias("scalar")), + ) + assertDataFrameEqual( + t1.withColumn( + "scalar", + self.spark.range(1) + .withColumn("c1", sf.col("c1").outer()) + .select(sf.col("c1") + sf.col("c2").outer()) + .scalar(), + ), + t1.select("*", (sf.col("c1") + sf.col("c2")).alias("scalar")), + ) + assertDataFrameEqual( + t1.withColumn( + "scalar", + self.spark.range(1) + .select(sf.col("c1").outer().alias("c1")) + .withColumn("c2", sf.col("c2").outer()) + .select(sf.col("c1") + sf.col("c2")) + .scalar(), + ), + t1.select("*", (sf.col("c1") + sf.col("c2")).alias("scalar")), + ) + + def test_subquery_in_with_columns_renamed(self): + with self.tempView("t1"): + t1 = self.table1() + + assertDataFrameEqual( + t1.withColumn( + "scalar", + self.spark.range(1) + .select(sf.col("c1").outer().alias("c1"), sf.col("c2").outer().alias("c2")) + .withColumnsRenamed({"c1": "x", "c2": "y"}) + .select(sf.col("x") + sf.col("y")) + .scalar(), + ), + t1.select("*", (sf.col("c1").alias("x") + sf.col("c2").alias("y")).alias("scalar")), ) def test_subquery_in_drop(self): diff --git a/python/pyspark/sql/tests/test_types.py b/python/pyspark/sql/tests/test_types.py index 432ddd083c802..75c28ac0dec1d 100644 --- a/python/pyspark/sql/tests/test_types.py +++ b/python/pyspark/sql/tests/test_types.py @@ -2240,6 +2240,17 @@ def test_variant_type(self): PySparkValueError, lambda: str(VariantVal(bytes([32, 10, 1, 0, 0, 0]), metadata)) ) + # check parse_json + for key, json, obj in expected_values: + self.assertEqual(VariantVal.parseJson(json).toJson(), json) + self.assertEqual(VariantVal.parseJson(json).toPython(), obj) + + # compare the parse_json in Spark vs python. `json_str` contains all of `expected_values`. + parse_json_spark_output = variants[0] + parse_json_python_output = VariantVal.parseJson(json_str) + self.assertEqual(parse_json_spark_output.value, parse_json_python_output.value) + self.assertEqual(parse_json_spark_output.metadata, parse_json_python_output.metadata) + def test_to_ddl(self): schema = StructType().add("a", NullType()).add("b", BooleanType()).add("c", BinaryType()) self.assertEqual(schema.toDDL(), "a VOID,b BOOLEAN,c BINARY") diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index f40a8bf62b290..b913e05e16d2a 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1770,6 +1770,15 @@ def toJson(self, zone_id: str = "UTC") -> str: """ return VariantUtils.to_json(self.value, self.metadata, zone_id) + @classmethod + def parseJson(cls, json_str: str) -> "VariantVal": + """ + Convert the VariantVal to a nested Python object of Python data types. + :return: Python representation of the Variant nested structure + """ + (value, metadata) = VariantUtils.parse_json(json_str) + return VariantVal(value, metadata) + _atomic_types: List[Type[DataType]] = [ StringType, diff --git a/python/pyspark/sql/variant_utils.py b/python/pyspark/sql/variant_utils.py index 40cc69c1f0961..3025523064e1d 100644 --- a/python/pyspark/sql/variant_utils.py +++ b/python/pyspark/sql/variant_utils.py @@ -21,7 +21,7 @@ import json import struct from array import array -from typing import Any, Callable, Dict, List, Tuple +from typing import Any, Callable, Dict, List, NamedTuple, Tuple from pyspark.errors import PySparkValueError from zoneinfo import ZoneInfo @@ -108,8 +108,25 @@ class VariantUtils: # string size) + (size bytes of string content). LONG_STR = 16 + VERSION = 1 + # The lower 4 bits of the first metadata byte contain the version. + VERSION_MASK = 0x0F + + U8_MAX = 0xFF + U16_MAX = 0xFFFF + U24_MAX = 0xFFFFFF + U24_SIZE = 3 U32_SIZE = 4 + I8_MAX = 0x7F + I8_MIN = -0x80 + I16_MAX = 0x7FFF + I16_MIN = -0x8000 + I32_MAX = 0x7FFFFFFF + I32_MIN = -0x80000000 + I64_MAX = 0x7FFFFFFFFFFFFFFF + I64_MIN = -0x8000000000000000 + EPOCH = datetime.datetime( year=1970, month=1, day=1, hour=0, minute=0, second=0, tzinfo=datetime.timezone.utc ) @@ -140,6 +157,15 @@ def to_python(cls, value: bytes, metadata: bytes) -> str: """ return cls._to_python(value, metadata, 0) + @classmethod + def parse_json(cls, json_str: str) -> Tuple[bytes, bytes]: + """ + Parses the JSON string and creates the Variant binary (value, metadata) + :return: tuple of 2 binary values (value, metadata) + """ + builder = VariantBuilder() + return builder.build(json_str) + @classmethod def _read_long(cls, data: bytes, pos: int, num_bytes: int, signed: bool) -> int: cls._check_index(pos, len(data)) @@ -468,7 +494,10 @@ def _handle_object( value, offset_start + offset_size * i, offset_size, signed=False ) value_pos = data_start + offset - key_value_pos_list.append((cls._get_metadata_key(metadata, id), value_pos)) + if metadata is not None: + key_value_pos_list.append((cls._get_metadata_key(metadata, id), value_pos)) + else: + key_value_pos_list.append(("", value_pos)) return func(key_value_pos_list) @classmethod @@ -496,3 +525,297 @@ def _handle_array(cls, value: bytes, pos: int, func: Callable[[List[int]], Any]) element_pos = data_start + offset value_pos_list.append(element_pos) return func(value_pos_list) + + +class FieldEntry(NamedTuple): + """ + Info about an object field + """ + + key: str + id: int + offset: int + + +class VariantBuilder: + """ + A utility class for building VariantVal. + """ + + DEFAULT_SIZE_LIMIT = 16 * 1024 * 1024 + + def __init__(self, size_limit: int = DEFAULT_SIZE_LIMIT): + self.value = bytearray() + self.dictionary = dict[str, int]() + self.dictionary_keys = list[bytes]() + self.size_limit = size_limit + + def build(self, json_str: str) -> Tuple[bytes, bytes]: + parsed = json.loads(json_str, parse_float=self._handle_float) + self._process_parsed_json(parsed) + + num_keys = len(self.dictionary_keys) + dictionary_string_size = sum(len(key) for key in self.dictionary_keys) + + # Determine the number of bytes required per offset entry. + # The largest offset is the one-past-the-end value, which is total string size. It's very + # unlikely that the number of keys could be larger, but incorporate that into the + # calculation in case of pathological data. + max_size = max(dictionary_string_size, num_keys) + if max_size > self.size_limit: + raise PySparkValueError(errorClass="VARIANT_SIZE_LIMIT_EXCEEDED", messageParameters={}) + offset_size = self._get_integer_size(max_size) + + offset_start = 1 + offset_size + string_start = offset_start + (num_keys + 1) * offset_size + metadata_size = string_start + dictionary_string_size + if metadata_size > self.size_limit: + raise PySparkValueError(errorClass="VARIANT_SIZE_LIMIT_EXCEEDED", messageParameters={}) + + metadata = bytearray() + header_byte = VariantUtils.VERSION | ((offset_size - 1) << 6) + metadata.extend(header_byte.to_bytes(1, byteorder="little")) + metadata.extend(num_keys.to_bytes(offset_size, byteorder="little")) + # write offsets + current_offset = 0 + for key in self.dictionary_keys: + metadata.extend(current_offset.to_bytes(offset_size, byteorder="little")) + current_offset += len(key) + metadata.extend(current_offset.to_bytes(offset_size, byteorder="little")) + # write key data + for key in self.dictionary_keys: + metadata.extend(key) + return (bytes(self.value), bytes(metadata)) + + def _process_parsed_json(self, parsed: Any) -> None: + if type(parsed) is dict: + fields = list[FieldEntry]() + start = len(self.value) + for key, value in parsed.items(): + id = self._add_key(key) + fields.append(FieldEntry(key, id, len(self.value) - start)) + self._process_parsed_json(value) + self._finish_writing_object(start, fields) + elif type(parsed) is list: + offsets = [] + start = len(self.value) + for elem in parsed: + offsets.append(len(self.value) - start) + self._process_parsed_json(elem) + self._finish_writing_array(start, offsets) + elif type(parsed) is str: + self._append_string(parsed) + elif type(parsed) is int: + if not self._append_int(parsed): + self._process_parsed_json(self._handle_float(str(parsed))) + elif type(parsed) is float: + self._append_float(parsed) + elif type(parsed) is decimal.Decimal: + self._append_decimal(parsed) + elif type(parsed) is bool: + self._append_boolean(parsed) + elif parsed is None: + self._append_null() + else: + raise PySparkValueError(errorClass="MALFORMED_VARIANT", messageParameters={}) + + # Choose the smallest unsigned integer type that can store `value`. It must be within + # [0, U24_MAX]. + def _get_integer_size(self, value: int) -> int: + if value <= VariantUtils.U8_MAX: + return 1 + if value <= VariantUtils.U16_MAX: + return 2 + return VariantUtils.U24_SIZE + + def _check_capacity(self, additional: int) -> None: + required = len(self.value) + additional + if required > self.size_limit: + raise PySparkValueError(errorClass="VARIANT_SIZE_LIMIT_EXCEEDED", messageParameters={}) + + def _primitive_header(self, type: int) -> bytes: + return bytes([(type << 2) | VariantUtils.PRIMITIVE]) + + def _short_string_header(self, size: int) -> bytes: + return bytes([size << 2 | VariantUtils.SHORT_STR]) + + def _array_header(self, large_size: bool, offset_size: int) -> bytes: + return bytes( + [ + ( + (large_size << (VariantUtils.BASIC_TYPE_BITS + 2)) + | ((offset_size - 1) << VariantUtils.BASIC_TYPE_BITS) + | VariantUtils.ARRAY + ) + ] + ) + + def _object_header(self, large_size: bool, id_size: int, offset_size: int) -> bytes: + return bytes( + [ + ( + (large_size << (VariantUtils.BASIC_TYPE_BITS + 4)) + | ((id_size - 1) << (VariantUtils.BASIC_TYPE_BITS + 2)) + | ((offset_size - 1) << VariantUtils.BASIC_TYPE_BITS) + | VariantUtils.OBJECT + ) + ] + ) + + # Add a key to the variant dictionary. If the key already exists, the dictionary is + # not modified. In either case, return the id of the key. + def _add_key(self, key: str) -> int: + if key in self.dictionary: + return self.dictionary[key] + id = len(self.dictionary_keys) + self.dictionary[key] = id + self.dictionary_keys.append(key.encode("utf-8")) + return id + + def _handle_float(self, num_str: str) -> Any: + # a float can be a decimal if it only contains digits, '-', or '-'. + if all([ch.isdecimal() or ch == "-" or ch == "." for ch in num_str]): + dec = decimal.Decimal(num_str) + precision = len(dec.as_tuple().digits) + scale = -int(dec.as_tuple().exponent) + + if ( + scale <= VariantUtils.MAX_DECIMAL16_PRECISION + and precision <= VariantUtils.MAX_DECIMAL16_PRECISION + ): + return dec + return float(num_str) + + def _append_boolean(self, b: bool) -> None: + self._check_capacity(1) + self.value.extend(self._primitive_header(VariantUtils.TRUE if b else VariantUtils.FALSE)) + + def _append_null(self) -> None: + self._check_capacity(1) + self.value.extend(self._primitive_header(VariantUtils.NULL)) + + def _append_string(self, s: str) -> None: + text = s.encode("utf-8") + long_str = len(text) > VariantUtils.MAX_SHORT_STR_SIZE + additional = (1 + VariantUtils.U32_SIZE) if long_str else 1 + self._check_capacity(additional + len(text)) + if long_str: + self.value.extend(self._primitive_header(VariantUtils.LONG_STR)) + self.value.extend(len(text).to_bytes(VariantUtils.U32_SIZE, byteorder="little")) + else: + self.value.extend(self._short_string_header(len(text))) + self.value.extend(text) + + def _append_int(self, i: int) -> bool: + self._check_capacity(1 + 8) + if i >= VariantUtils.I8_MIN and i <= VariantUtils.I8_MAX: + self.value.extend(self._primitive_header(VariantUtils.INT1)) + self.value.extend(i.to_bytes(1, byteorder="little", signed=True)) + elif i >= VariantUtils.I16_MIN and i <= VariantUtils.I16_MAX: + self.value.extend(self._primitive_header(VariantUtils.INT2)) + self.value.extend(i.to_bytes(2, byteorder="little", signed=True)) + elif i >= VariantUtils.I32_MIN and i <= VariantUtils.I32_MAX: + self.value.extend(self._primitive_header(VariantUtils.INT4)) + self.value.extend(i.to_bytes(4, byteorder="little", signed=True)) + elif i >= VariantUtils.I64_MIN and i <= VariantUtils.I64_MAX: + self.value.extend(self._primitive_header(VariantUtils.INT8)) + self.value.extend(i.to_bytes(8, byteorder="little", signed=True)) + else: + return False + return True + + # Append a decimal value to the variant builder. The caller should guarantee that its precision + # and scale fit into `MAX_DECIMAL16_PRECISION`. + def _append_decimal(self, d: decimal.Decimal) -> None: + self._check_capacity(2 + 16) + precision = len(d.as_tuple().digits) + scale = -int(d.as_tuple().exponent) + unscaled = int("".join(map(str, d.as_tuple().digits))) + unscaled = -unscaled if d < 0 else unscaled + if ( + scale <= VariantUtils.MAX_DECIMAL4_PRECISION + and precision <= VariantUtils.MAX_DECIMAL4_PRECISION + ): + self.value.extend(self._primitive_header(VariantUtils.DECIMAL4)) + self.value.extend(scale.to_bytes(1, byteorder="little")) + self.value.extend(unscaled.to_bytes(4, byteorder="little", signed=True)) + elif ( + scale <= VariantUtils.MAX_DECIMAL8_PRECISION + and precision <= VariantUtils.MAX_DECIMAL8_PRECISION + ): + self.value.extend(self._primitive_header(VariantUtils.DECIMAL8)) + self.value.extend(scale.to_bytes(1, byteorder="little")) + self.value.extend(unscaled.to_bytes(8, byteorder="little", signed=True)) + else: + assert ( + scale <= VariantUtils.MAX_DECIMAL16_PRECISION + and precision <= VariantUtils.MAX_DECIMAL16_PRECISION + ) + self.value.extend(self._primitive_header(VariantUtils.DECIMAL16)) + self.value.extend(scale.to_bytes(1, byteorder="little")) + self.value.extend(unscaled.to_bytes(16, byteorder="little", signed=True)) + + def _append_float(self, f: float) -> None: + self._check_capacity(1 + 8) + self.value.extend(self._primitive_header(VariantUtils.DOUBLE)) + self.value.extend(struct.pack(" None: + data_size = len(self.value) - start + num_offsets = len(offsets) + large_size = num_offsets > VariantUtils.U8_MAX + size_bytes = VariantUtils.U32_SIZE if large_size else 1 + offset_size = self._get_integer_size(data_size) + # The space for header byte, object size, and offset list. + header_size = 1 + size_bytes + (num_offsets + 1) * offset_size + self._check_capacity(header_size) + self.value.extend(bytearray(header_size)) + # Shift the just-written element data to make room for the header section. + self.value[start + header_size :] = bytes(self.value[start : start + data_size]) + # Write the header byte, num offsets + offset_start = start + 1 + size_bytes + self.value[start : start + 1] = self._array_header(large_size, offset_size) + self.value[start + 1 : offset_start] = num_offsets.to_bytes(size_bytes, byteorder="little") + # write offset list + offset_list = bytearray() + for offset in offsets: + offset_list.extend(offset.to_bytes(offset_size, byteorder="little")) + offset_list.extend(data_size.to_bytes(offset_size, byteorder="little")) + self.value[offset_start : offset_start + len(offset_list)] = offset_list + + # Finish writing a variant object after all of its fields have already been written. + def _finish_writing_object(self, start: int, fields: List[FieldEntry]) -> None: + num_fields = len(fields) + # object fields are from a python dictionary, so keys are already distinct + fields.sort(key=lambda f: f.key) + max_id = 0 + for field in fields: + max_id = max(max_id, field.id) + + data_size = len(self.value) - start + large_size = num_fields > VariantUtils.U8_MAX + size_bytes = VariantUtils.U32_SIZE if large_size else 1 + id_size = self._get_integer_size(max_id) + offset_size = self._get_integer_size(data_size) + # The space for header byte, object size, id list, and offset list. + header_size = 1 + size_bytes + num_fields * id_size + (num_fields + 1) * offset_size + self._check_capacity(header_size) + self.value.extend(bytearray(header_size)) + # Shift the just-written field data to make room for the object header section. + self.value[start + header_size :] = self.value[start : start + data_size] + # Write the header byte, num fields, id list, offset list + self.value[start : start + 1] = self._object_header(large_size, id_size, offset_size) + self.value[start + 1 : start + 1 + size_bytes] = num_fields.to_bytes( + size_bytes, byteorder="little" + ) + id_start = start + 1 + size_bytes + offset_start = id_start + num_fields * id_size + id_list = bytearray() + offset_list = bytearray() + for field in fields: + id_list.extend(field.id.to_bytes(id_size, byteorder="little")) + offset_list.extend(field.offset.to_bytes(offset_size, byteorder="little")) + offset_list.extend(data_size.to_bytes(offset_size, byteorder="little")) + self.value[id_start : id_start + len(id_list)] = id_list + self.value[offset_start : offset_start + len(offset_list)] = offset_list diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java index 4200619d3c5f9..310d18ddb3486 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java @@ -51,7 +51,7 @@ public class ExpressionInfo { "window_funcs", "xml_funcs", "table_funcs", "url_funcs", "variant_funcs")); private static final Set validSources = - new HashSet<>(Arrays.asList("built-in", "hive", "python_udf", "scala_udf", + new HashSet<>(Arrays.asList("built-in", "hive", "python_udf", "scala_udf", "sql_udf", "java_udf", "python_udtf", "internal")); public String getClassName() { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java index aaac980bb332a..ac05981da5a24 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java @@ -188,6 +188,8 @@ public Object get(int ordinal, DataType dataType) { return getInt(ordinal); } else if (dataType instanceof TimestampType) { return getLong(ordinal); + } else if (dataType instanceof TimestampNTZType) { + return getLong(ordinal); } else if (dataType instanceof ArrayType) { return getArray(ordinal); } else if (dataType instanceof StructType) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 9282e0554a2d4..92cfc4119dd0c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -374,6 +374,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor BindProcedures :: ResolveTableSpec :: ValidateAndStripPipeExpressions :: + ResolveSQLFunctions :: ResolveAliases :: ResolveSubquery :: ResolveSubqueryColumnAliases :: @@ -2364,6 +2365,277 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor } } + /** + * This rule resolves SQL function expressions. It pulls out function inputs and place them + * in a separate [[Project]] node below the operator and replace the SQL function with its + * actual function body. SQL function expressions in [[Aggregate]] are handled in a special + * way. Non-aggregated SQL functions in the aggregate expressions of an Aggregate need to be + * pulled out into a Project above the Aggregate before replacing the SQL function expressions + * with actual function bodies. For example: + * + * Before: + * Aggregate [c1] [foo(c1), foo(max(c2)), sum(foo(c2)) AS sum] + * +- Relation [c1, c2] + * + * After: + * Project [foo(c1), foo(max_c2), sum] + * +- Aggregate [c1] [c1, max(c2) AS max_c2, sum(foo(c2)) AS sum] + * +- Relation [c1, c2] + */ + object ResolveSQLFunctions extends Rule[LogicalPlan] { + + private def hasSQLFunctionExpression(exprs: Seq[Expression]): Boolean = { + exprs.exists(_.find(_.isInstanceOf[SQLFunctionExpression]).nonEmpty) + } + + /** + * Check if the function input contains aggregate expressions. + */ + private def checkFunctionInput(f: SQLFunctionExpression): Unit = { + if (f.inputs.exists(AggregateExpression.containsAggregate)) { + // The input of a SQL function should not contain aggregate functions after + // `extractAndRewrite`. If there are aggregate functions, it means they are + // nested in another aggregate function, which is not allowed. + // For example: SELECT sum(foo(sum(c1))) FROM t + // We have to throw the error here because otherwise the query plan after + // resolving the SQL function will not be valid. + throw new AnalysisException( + errorClass = "NESTED_AGGREGATE_FUNCTION", + messageParameters = Map.empty) + } + } + + /** + * Resolve a SQL function expression as a logical plan check if it can be analyzed. + */ + private def resolve(f: SQLFunctionExpression): LogicalPlan = { + // Validate the SQL function input. + checkFunctionInput(f) + val plan = v1SessionCatalog.makeSQLFunctionPlan(f.name, f.function, f.inputs) + val resolved = SQLFunctionContext.withSQLFunction { + // Resolve the SQL function plan using its context. + val conf = new SQLConf() + f.function.getSQLConfigs.foreach { case (k, v) => conf.settings.put(k, v) } + SQLConf.withExistingConf(conf) { + executeSameContext(plan) + } + } + // Fail the analysis eagerly if a SQL function cannot be resolved using its input. + SimpleAnalyzer.checkAnalysis(resolved) + resolved + } + + /** + * Rewrite SQL function expressions into actual resolved function bodies and extract + * function inputs into the given project list. + */ + private def rewriteSQLFunctions[E <: Expression]( + expression: E, + projectList: ArrayBuffer[NamedExpression]): E = { + val newExpr = expression match { + case f: SQLFunctionExpression if !hasSQLFunctionExpression(f.inputs) && + // Make sure LateralColumnAliasReference in parameters is resolved and eliminated first. + // Otherwise, the projectList can contain the LateralColumnAliasReference, which will be + // pushed down to a Project without the 'referenced' alias by LCA present, leaving it + // unresolved. + !f.inputs.exists(_.containsPattern(LATERAL_COLUMN_ALIAS_REFERENCE)) => + withPosition(f) { + val plan = resolve(f) + // Extract the function input project list from the SQL function plan and + // inline the SQL function expression. + plan match { + case Project(body :: Nil, Project(aliases, _: LocalRelation)) => + projectList ++= aliases + SQLScalarFunction(f.function, aliases.map(_.toAttribute), body) + case o => + throw new AnalysisException( + errorClass = "INVALID_SQL_FUNCTION_PLAN_STRUCTURE", + messageParameters = Map("plan" -> o.toString)) + } + } + case o => o.mapChildren(rewriteSQLFunctions(_, projectList)) + } + newExpr.asInstanceOf[E] + } + + /** + * Check if the given expression contains expressions that should be extracted, + * i.e. non-aggregated SQL functions with non-foldable inputs. + */ + private def shouldExtract(e: Expression): Boolean = e match { + // Return false if the expression is already an aggregate expression. + case _: AggregateExpression => false + case _: SQLFunctionExpression => true + case _: LeafExpression => false + case o => o.children.exists(shouldExtract) + } + + /** + * Extract aggregate expressions from the given expression and replace + * them with attribute references. + * Example: + * Before: foo(c1) + foo(max(c2)) + max(foo(c2)) + * After: foo(c1) + foo(max_c2) + max_foo_c2 + * Extracted expressions: [c1, max(c2) AS max_c2, max(foo(c2)) AS max_foo_c2] + */ + private def extractAndRewrite[T <: Expression]( + expression: T, + extractedExprs: ArrayBuffer[NamedExpression]): T = { + val newExpr = expression match { + case e if !shouldExtract(e) => + val exprToAdd: NamedExpression = e match { + case o: OuterReference => Alias(o, toPrettySQL(o.e))() + case ne: NamedExpression => ne + case o => Alias(o, toPrettySQL(o))() + } + extractedExprs += exprToAdd + exprToAdd.toAttribute + case f: SQLFunctionExpression => + val newInputs = f.inputs.map(extractAndRewrite(_, extractedExprs)) + f.copy(inputs = newInputs) + case o => o.mapChildren(extractAndRewrite(_, extractedExprs)) + } + newExpr.asInstanceOf[T] + } + + /** + * Replace all [[SQLFunctionExpression]]s in an expression with attribute references + * from the aliasMap. + */ + private def replaceSQLFunctionWithAttr[T <: Expression]( + expr: T, + aliasMap: mutable.HashMap[Expression, Alias]): T = { + expr.transform { + case f: SQLFunctionExpression if aliasMap.contains(f.canonicalized) => + aliasMap(f.canonicalized).toAttribute + }.asInstanceOf[T] + } + + private def rewrite(plan: LogicalPlan): LogicalPlan = plan match { + // Return if a sub-tree does not contain SQLFunctionExpression. + case p: LogicalPlan if !p.containsPattern(SQL_FUNCTION_EXPRESSION) => p + + case f @ Filter(cond, a: Aggregate) + if !f.resolved || AggregateExpression.containsAggregate(cond) || + ResolveGroupingAnalytics.hasGroupingFunction(cond) || + cond.containsPattern(TEMP_RESOLVED_COLUMN) => + // If the filter's condition contains aggregate expressions or grouping expressions or temp + // resolved column, we cannot rewrite both the filter and the aggregate until they are + // resolved by ResolveAggregateFunctions or ResolveGroupingAnalytics, because rewriting SQL + // functions in aggregate can add an additional project on top of the aggregate + // which breaks the pattern matching in those rules. + f.copy(child = a.copy(child = rewrite(a.child))) + + case h @ UnresolvedHaving(_, a: Aggregate) => + // Similarly UnresolvedHaving should be resolved by ResolveAggregateFunctions first + // before rewriting aggregate. + h.copy(child = a.copy(child = rewrite(a.child))) + + case a: Aggregate if a.resolved && hasSQLFunctionExpression(a.expressions) => + val child = rewrite(a.child) + // Extract SQL functions in the grouping expressions and place them in a project list + // below the current aggregate. Also update their appearances in the aggregate expressions. + val bottomProjectList = ArrayBuffer.empty[NamedExpression] + val aliasMap = mutable.HashMap.empty[Expression, Alias] + val newGrouping = a.groupingExpressions.map { expr => + expr.transformDown { + case f: SQLFunctionExpression => + val alias = aliasMap.getOrElseUpdate(f.canonicalized, Alias(f, f.name)()) + bottomProjectList += alias + alias.toAttribute + } + } + val aggregateExpressions = a.aggregateExpressions.map( + replaceSQLFunctionWithAttr(_, aliasMap)) + + // Rewrite SQL functions in the aggregate expressions that are not wrapped in + // aggregate functions. They need to be extracted into a project list above the + // current aggregate. + val aggExprs = ArrayBuffer.empty[NamedExpression] + val topProjectList = aggregateExpressions.map(extractAndRewrite(_, aggExprs)) + + // Rewrite SQL functions in the new aggregate expressions that are wrapped inside + // aggregate functions. + val newAggExprs = aggExprs.map(rewriteSQLFunctions(_, bottomProjectList)) + + val bottomProject = if (bottomProjectList.nonEmpty) { + Project(child.output ++ bottomProjectList, child) + } else { + child + } + val newAgg = if (newGrouping.nonEmpty || newAggExprs.nonEmpty) { + a.copy( + groupingExpressions = newGrouping, + aggregateExpressions = newAggExprs.toSeq, + child = bottomProject) + } else { + bottomProject + } + if (topProjectList.nonEmpty) Project(topProjectList, newAgg) else newAgg + + case p: Project if p.resolved && hasSQLFunctionExpression(p.expressions) => + val newChild = rewrite(p.child) + val projectList = ArrayBuffer.empty[NamedExpression] + val newPList = p.projectList.map(rewriteSQLFunctions(_, projectList)) + if (newPList != newChild.output) { + p.copy(newPList, Project(newChild.output ++ projectList, newChild)) + } else { + assert(projectList.isEmpty) + p.copy(child = newChild) + } + + case f: Filter if f.resolved && hasSQLFunctionExpression(f.expressions) => + val newChild = rewrite(f.child) + val projectList = ArrayBuffer.empty[NamedExpression] + val newCond = rewriteSQLFunctions(f.condition, projectList) + if (newCond != f.condition) { + Project(f.output, Filter(newCond, Project(newChild.output ++ projectList, newChild))) + } else { + assert(projectList.isEmpty) + f.copy(child = newChild) + } + + case j: Join if j.resolved && hasSQLFunctionExpression(j.expressions) => + val newLeft = rewrite(j.left) + val newRight = rewrite(j.right) + val projectList = ArrayBuffer.empty[NamedExpression] + val joinCond = j.condition.map(rewriteSQLFunctions(_, projectList)) + if (joinCond != j.condition) { + // Join condition cannot have non-deterministic expressions. We can safely + // replace the aliases with the original SQL function input expressions. + val aliasMap = projectList.collect { case a: Alias => a.toAttribute -> a.child }.toMap + val newJoinCond = joinCond.map(_.transform { + case a: Attribute => aliasMap.getOrElse(a, a) + }) + j.copy(left = newLeft, right = newRight, condition = newJoinCond) + } else { + assert(projectList.isEmpty) + j.copy(left = newLeft, right = newRight) + } + + case o: LogicalPlan if o.resolved && hasSQLFunctionExpression(o.expressions) => + o.transformExpressionsWithPruning(_.containsPattern(SQL_FUNCTION_EXPRESSION)) { + case f: SQLFunctionExpression => + f.failAnalysis( + errorClass = "UNSUPPORTED_SQL_UDF_USAGE", + messageParameters = Map( + "functionName" -> toSQLId(f.function.name.nameParts), + "nodeName" -> o.nodeName.toString)) + } + + case p: LogicalPlan => p.mapChildren(rewrite) + } + + def apply(plan: LogicalPlan): LogicalPlan = { + // Only rewrite SQL functions when they are not in nested function calls. + if (SQLFunctionContext.get.nestedSQLFunctionDepth > 0) { + plan + } else { + rewrite(plan) + } + } + } + /** * Turns projections that contain aggregate expressions into aggregations. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 46ca8e793218b..0a68524c31241 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -1106,6 +1106,8 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB @scala.annotation.tailrec def cleanQueryInScalarSubquery(p: LogicalPlan): LogicalPlan = p match { case s: SubqueryAlias => cleanQueryInScalarSubquery(s.child) + // Skip SQL function node added by the Analyzer + case s: SQLFunctionNode => cleanQueryInScalarSubquery(s.child) case p: Project => cleanQueryInScalarSubquery(p.child) case h: ResolvedHint => cleanQueryInScalarSubquery(h.child) case child => child diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SQLFunctionExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SQLFunctionExpression.scala index fb6935d64d4c4..37981f47287da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SQLFunctionExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SQLFunctionExpression.scala @@ -18,8 +18,8 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.catalog.SQLFunction -import org.apache.spark.sql.catalyst.expressions.{Expression, Unevaluable} -import org.apache.spark.sql.catalyst.trees.TreePattern.{SQL_FUNCTION_EXPRESSION, TreePattern} +import org.apache.spark.sql.catalyst.expressions.{Expression, UnaryExpression, Unevaluable} +import org.apache.spark.sql.catalyst.trees.TreePattern.{SQL_FUNCTION_EXPRESSION, SQL_SCALAR_FUNCTION, TreePattern} import org.apache.spark.sql.types.DataType /** @@ -39,3 +39,52 @@ case class SQLFunctionExpression( newChildren: IndexedSeq[Expression]): SQLFunctionExpression = copy(inputs = newChildren) final override val nodePatterns: Seq[TreePattern] = Seq(SQL_FUNCTION_EXPRESSION) } + +/** + * A wrapper node for a SQL scalar function expression. + */ +case class SQLScalarFunction(function: SQLFunction, inputs: Seq[Expression], child: Expression) + extends UnaryExpression with Unevaluable { + override def dataType: DataType = child.dataType + override def toString: String = s"${function.name}(${inputs.mkString(", ")})" + override def sql: String = s"${function.name}(${inputs.map(_.sql).mkString(", ")})" + override protected def withNewChildInternal(newChild: Expression): SQLScalarFunction = { + copy(child = newChild) + } + final override val nodePatterns: Seq[TreePattern] = Seq(SQL_SCALAR_FUNCTION) + // The `inputs` is for display only and does not matter in execution. + override lazy val canonicalized: Expression = copy(inputs = Nil, child = child.canonicalized) + override lazy val deterministic: Boolean = { + function.deterministic.getOrElse(true) && children.forall(_.deterministic) + } +} + +/** + * Provide a way to keep state during analysis for resolving nested SQL functions. + * + * @param nestedSQLFunctionDepth The nested depth in the SQL function resolution. A SQL function + * expression should only be expanded as a [[SQLScalarFunction]] if + * the nested depth is 0. + */ +case class SQLFunctionContext(nestedSQLFunctionDepth: Int = 0) + +object SQLFunctionContext { + + private val value = new ThreadLocal[SQLFunctionContext]() { + override def initialValue: SQLFunctionContext = SQLFunctionContext() + } + + def get: SQLFunctionContext = value.get() + + def reset(): Unit = value.remove() + + private def set(context: SQLFunctionContext): Unit = value.set(context) + + def withSQLFunction[A](f: => A): A = { + val originContext = value.get() + val context = originContext.copy( + nestedSQLFunctionDepth = originContext.nestedSQLFunctionDepth + 1) + set(context) + try f finally { set(originContext) } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala index 8870befede4f4..f778915008dbb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/resolver/PlanLogger.scala @@ -46,7 +46,7 @@ class PlanLogger extends Logging { |${MDC( QUERY_PLAN, sideBySide( - unresolvedPlan.withNewChildren(resolvedPlan.children).treeString, + unresolvedPlan.treeString, resolvedPlan.treeString ).mkString("\n") )} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index b47af90c651a6..fabe551d054ca 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLId import org.apache.spark.sql.connector.catalog.TableWritePrivilege import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.types.{DataType, Metadata, StructType} -import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.sql.util.{CaseInsensitiveStringMap, SchemaUtils} import org.apache.spark.util.ArrayImplicits._ /** @@ -429,7 +429,7 @@ object UnresolvedFunction { * Represents all of the input attributes to a given relational operator, for example in * "SELECT * FROM ...". A [[Star]] gets automatically expanded during analysis. */ -abstract class Star extends LeafExpression with NamedExpression { +trait Star extends NamedExpression { override def name: String = throw new UnresolvedException("name") override def exprId: ExprId = throw new UnresolvedException("exprId") @@ -451,15 +451,20 @@ abstract class Star extends LeafExpression with NamedExpression { * This is also used to expand structs. For example: * "SELECT record.* from (SELECT struct(a,b,c) as record ...) * - * @param target an optional name that should be the target of the expansion. If omitted all - * targets' columns are produced. This can either be a table name or struct name. This - * is a list of identifiers that is the path of the expansion. - * - * This class provides the shared behavior between the classes for SELECT * ([[UnresolvedStar]]) - * and SELECT * EXCEPT ([[UnresolvedStarExceptOrReplace]]). [[UnresolvedStar]] is just a case class - * of this, while [[UnresolvedStarExceptOrReplace]] adds some additional logic to the expand method. + * This trait provides the shared behavior among the classes for SELECT * ([[UnresolvedStar]]) + * and SELECT * EXCEPT ([[UnresolvedStarExceptOrReplace]]), etc. [[UnresolvedStar]] is just a case + * class of this, while [[UnresolvedStarExceptOrReplace]] or other classes add some additional logic + * to the expand method. */ -abstract class UnresolvedStarBase(target: Option[Seq[String]]) extends Star with Unevaluable { +trait UnresolvedStarBase extends Star with Unevaluable { + + /** + * An optional name that should be the target of the expansion. If omitted all + * targets' columns are produced. This can either be a table name or struct name. This + * is a list of identifiers that is the path of the expansion. + */ + def target: Option[Seq[String]] + /** * Returns true if the nameParts is a subset of the last elements of qualifier of the attribute. * @@ -583,7 +588,7 @@ case class UnresolvedStarExceptOrReplace( target: Option[Seq[String]], excepts: Seq[Seq[String]], replacements: Option[Seq[NamedExpression]]) - extends UnresolvedStarBase(target) { + extends LeafExpression with UnresolvedStarBase { /** * We expand the * EXCEPT by the following three steps: @@ -712,6 +717,103 @@ case class UnresolvedStarExceptOrReplace( } } +/** + * Represents some of the input attributes to a given relational operator, for example in + * `df.withColumn`. + * + * @param colNames a list of column names that should be replaced or produced. + * + * @param exprs the corresponding expressions for `colNames`. + * + * @param explicitMetadata an optional list of explicit metadata to associate with the columns. + */ +case class UnresolvedStarWithColumns( + colNames: Seq[String], + exprs: Seq[Expression], + explicitMetadata: Option[Seq[Metadata]] = None) + extends UnresolvedStarBase { + + override def target: Option[Seq[String]] = None + override def children: Seq[Expression] = exprs + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression]): UnresolvedStarWithColumns = + copy(exprs = newChildren) + + override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = { + assert(colNames.size == exprs.size, + s"The size of column names: ${colNames.size} isn't equal to " + + s"the size of expressions: ${exprs.size}") + explicitMetadata.foreach { m => + assert(colNames.size == m.size, + s"The size of column names: ${colNames.size} isn't equal to " + + s"the size of metadata elements: ${m.size}") + } + + SchemaUtils.checkColumnNameDuplication(colNames, resolver) + + val expandedCols = super.expand(input, resolver) + + val columnSeq = explicitMetadata match { + case Some(ms) => colNames.zip(exprs).zip(ms.map(Some(_))) + case _ => colNames.zip(exprs).map((_, None)) + } + + val replacedAndExistingColumns = expandedCols.map { field => + columnSeq.find { case ((colName, _), _) => + resolver(field.name, colName) + } match { + case Some(((colName, expr), m)) => Alias(expr, colName)(explicitMetadata = m) + case _ => field + } + } + + val newColumns = columnSeq.filter { case ((colName, _), _) => + !expandedCols.exists(f => resolver(f.name, colName)) + }.map { + case ((colName, expr), m) => Alias(expr, colName)(explicitMetadata = m) + } + + replacedAndExistingColumns ++ newColumns + } +} + +/** + * Represents some of the input attributes to a given relational operator, for example in + * `df.withColumnRenamed`. + * + * @param existingNames a list of column names that should be replaced. + * If the column does not exist, it is ignored. + * + * @param newNames a list of new column names that should be used to replace the existing columns. + */ +case class UnresolvedStarWithColumnsRenames( + existingNames: Seq[String], + newNames: Seq[String]) + extends LeafExpression with UnresolvedStarBase { + + override def target: Option[Seq[String]] = None + + override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = { + assert(existingNames.size == newNames.size, + s"The size of existing column names: ${existingNames.size} isn't equal to " + + s"the size of new column names: ${newNames.size}") + + val expandedCols = super.expand(input, resolver) + + existingNames.zip(newNames).foldLeft(expandedCols) { + case (attrs, (existingName, newName)) => + attrs.map(attr => + if (resolver(attr.name, existingName)) { + Alias(attr, newName)() + } else { + attr + } + ) + } + } +} + /** * Represents all of the input attributes to a given relational operator, for example in * "SELECT * FROM ...". @@ -723,7 +825,8 @@ case class UnresolvedStarExceptOrReplace( * targets' columns are produced. This can either be a table name or struct name. This * is a list of identifiers that is the path of the expansion. */ -case class UnresolvedStar(target: Option[Seq[String]]) extends UnresolvedStarBase(target) +case class UnresolvedStar(target: Option[Seq[String]]) + extends LeafExpression with UnresolvedStarBase /** * Represents all of the input attributes to a given relational operator, for example in @@ -733,7 +836,7 @@ case class UnresolvedStar(target: Option[Seq[String]]) extends UnresolvedStarBas * tables' columns are produced. */ case class UnresolvedRegex(regexPattern: String, table: Option[String], caseSensitive: Boolean) - extends Star with Unevaluable { + extends LeafExpression with Star with Unevaluable { override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = { val pattern = if (caseSensitive) regexPattern else s"(?i)$regexPattern" table match { @@ -791,7 +894,8 @@ case class MultiAlias(child: Expression, names: Seq[String]) * * @param expressions Expressions to expand. */ -case class ResolvedStar(expressions: Seq[NamedExpression]) extends Star with Unevaluable { +case class ResolvedStar(expressions: Seq[NamedExpression]) + extends LeafExpression with Star with Unevaluable { override def newInstance(): NamedExpression = throw new UnresolvedException("newInstance") override def expand(input: LogicalPlan, resolver: Resolver): Seq[NamedExpression] = expressions override def toString: String = expressions.mkString("ResolvedStar(", ", ", ")") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 3c6dfe5ac8445..b123952c5f086 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -38,9 +38,9 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder import org.apache.spark.sql.catalyst.analysis.TableFunctionRegistry.TableFunctionBuilder import org.apache.spark.sql.catalyst.catalog.SQLFunction.parseDefault -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Cast, Expression, ExpressionInfo, NamedArgumentExpression, NamedExpression, UpCast} +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Cast, Expression, ExpressionInfo, NamedArgumentExpression, NamedExpression, ScalarSubquery, UpCast} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} -import org.apache.spark.sql.catalyst.plans.logical.{FunctionSignature, InputParameter, LogicalPlan, NamedParametersSupport, Project, SubqueryAlias, View} +import org.apache.spark.sql.catalyst.plans.logical.{FunctionSignature, InputParameter, LocalRelation, LogicalPlan, NamedParametersSupport, Project, SubqueryAlias, View} import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, StringUtils} import org.apache.spark.sql.connector.catalog.CatalogManager @@ -48,7 +48,7 @@ import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAM import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.GLOBAL_TEMP_DATABASE -import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.types.{MetadataBuilder, StructField, StructType} import org.apache.spark.sql.util.{CaseInsensitiveStringMap, PartitioningUtils} import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils @@ -1561,6 +1561,103 @@ class SessionCatalog( } } + /** + * Constructs a scalar SQL function logical plan. The logical plan will be used to + * construct actual expression from the function inputs and body. + * + * The body of a scalar SQL function can either be an expression or a query returns + * one single column. + * + * Example scalar SQL function with an expression: + * + * CREATE FUNCTION area(width DOUBLE, height DOUBLE) RETURNS DOUBLE + * RETURN width * height; + * + * Query: + * + * SELECT area(a, b) FROM t; + * + * SQL function plan: + * + * Project [CAST(width * height AS DOUBLE) AS area] + * +- Project [CAST(a AS DOUBLE) AS width, CAST(b AS DOUBLE) AS height] + * +- LocalRelation [a, b] + * + * Example scalar SQL function with a subquery: + * + * CREATE FUNCTION foo(x INT) RETURNS INT + * RETURN SELECT SUM(b) FROM t WHERE x = a; + * + * SELECT foo(a) FROM t; + * + * SQL function plan: + * + * Project [scalar-subquery AS foo] + * : +- Aggregate [] [sum(b)] + * : +- Filter [outer(x) = a] + * : +- Relation [a, b] + * +- Project [CAST(a AS INT) AS x] + * +- LocalRelation [a, b] + */ + def makeSQLFunctionPlan( + name: String, + function: SQLFunction, + input: Seq[Expression]): LogicalPlan = { + def metaForFuncInputAlias = { + new MetadataBuilder() + .putString("__funcInputAlias", "true") + .build() + } + assert(!function.isTableFunc) + val funcName = function.name.funcName + + // Use captured SQL configs when parsing a SQL function. + val conf = new SQLConf() + function.getSQLConfigs.foreach { case (k, v) => conf.settings.put(k, v) } + SQLConf.withExistingConf(conf) { + val inputParam = function.inputParam + val returnType = function.getScalarFuncReturnType + val (expression, query) = function.getExpressionAndQuery(parser, isTableFunc = false) + assert(expression.isDefined || query.isDefined) + + // Check function arguments + val paramSize = inputParam.map(_.size).getOrElse(0) + if (input.size > paramSize) { + throw QueryCompilationErrors.wrongNumArgsError( + name, paramSize.toString, input.size) + } + + val inputs = inputParam.map { param => + // Attributes referencing the input parameters inside the function can use the + // function name as a qualifier. E.G.: + // `create function foo(a int) returns int return foo.a` + val qualifier = Seq(funcName) + val paddedInput = input ++ + param.takeRight(paramSize - input.size).map { p => + val defaultExpr = p.getDefault() + if (defaultExpr.isDefined) { + Cast(parseDefault(defaultExpr.get, parser), p.dataType) + } else { + throw QueryCompilationErrors.wrongNumArgsError( + name, paramSize.toString, input.size) + } + } + + paddedInput.zip(param.fields).map { + case (expr, param) => + Alias(Cast(expr, param.dataType), param.name)( + qualifier = qualifier, + // mark the alias as function input + explicitMetadata = Some(metaForFuncInputAlias)) + } + }.getOrElse(Nil) + + val body = if (query.isDefined) ScalarSubquery(query.get) else expression.get + Project(Alias(Cast(body, returnType), funcName)() :: Nil, + Project(inputs, LocalRelation(inputs.flatMap(_.references)))) + } + } + /** * Constructs a [[TableFunctionBuilder]] based on the provided class that represents a function. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala index fe00184e843a3..a76ca7b15c278 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.catalog import com.fasterxml.jackson.annotation.JsonInclude.Include import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} -import com.fasterxml.jackson.module.scala.{DefaultScalaModule, ScalaObjectMapper} +import com.fasterxml.jackson.module.scala.{ClassTagExtensions, DefaultScalaModule} import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.FunctionIdentifier @@ -45,6 +45,14 @@ trait UserDefinedFunction { */ def properties: Map[String, String] + /** + * Get SQL configs from the function properties. + * Use this to restore the SQL configs that should be used for this function. + */ + def getSQLConfigs: Map[String, String] = { + UserDefinedFunction.propertiesToSQLConfigs(properties) + } + /** * Owner of the function */ @@ -118,7 +126,7 @@ object UserDefinedFunction { * Get a object mapper to serialize and deserialize function properties. */ private def getObjectMapper: ObjectMapper = { - val mapper = new ObjectMapper with ScalaObjectMapper + val mapper = new ObjectMapper with ClassTagExtensions mapper.setSerializationInclusion(Include.NON_ABSENT) mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) mapper.registerModule(DefaultScalaModule) @@ -142,4 +150,17 @@ object UserDefinedFunction { * Verify if the function is a [[UserDefinedFunction]]. */ def isUserDefinedFunction(className: String): Boolean = SQLFunction.isSQLFunction(className) + + /** + * Covert properties to SQL configs. + */ + def propertiesToSQLConfigs(properties: Map[String, String]): Map[String, String] = { + try { + for ((key, value) <- properties if key.startsWith(SQL_CONFIG_PREFIX)) + yield (key.substring(SQL_CONFIG_PREFIX.length), value) + } catch { + case e: Exception => throw SparkException.internalError( + "Corrupted user defined function SQL configs in catalog", cause = e) + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 2ebfcf781b97b..de828dfd3e280 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -63,29 +63,25 @@ trait MetadataMapSupport { protected def jsonToString( jsonMap: mutable.LinkedHashMap[String, JValue]): mutable.LinkedHashMap[String, String] = { - def removeWhitespace(str: String): String = { - str.replaceAll("\\s+$", "") - } - val map = new mutable.LinkedHashMap[String, String]() jsonMap.foreach { case (key, jValue) => val stringValue = jValue match { - case JString(value) => removeWhitespace(value) + case JString(value) => value case JArray(values) => values.map(_.values) .map { - case str: String => quoteIdentifier(removeWhitespace(str)) - case other => removeWhitespace(other.toString) + case str: String => quoteIdentifier(str) + case other => other.toString } .mkString("[", ", ", "]") case JObject(fields) => fields.map { case (k, v) => - s"$k=${removeWhitespace(v.values.toString)}" + s"$k=${v.values.toString}" } .mkString("[", ", ", "]") case JInt(value) => value.toString case JDouble(value) => value.toString - case _ => removeWhitespace(jValue.values.toString) + case _ => jValue.values.toString } map.put(key, stringValue) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 30c00f5bf96b8..de74bb2f8cd21 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -29,6 +29,7 @@ import com.google.common.util.concurrent.{ExecutionError, UncheckedExecutionExce import org.codehaus.commons.compiler.{CompileException, InternalCompilerException} import org.codehaus.janino.ClassBodyEvaluator import org.codehaus.janino.util.ClassFile +import org.codehaus.janino.util.ClassFile.CodeAttribute import org.apache.spark.{SparkException, SparkIllegalArgumentException, TaskContext, TaskKilledException} import org.apache.spark.executor.InputMetrics @@ -1578,9 +1579,6 @@ object CodeGenerator extends Logging { val classes = evaluator.getBytecodes.asScala // Then walk the classes to get at the method bytecode. - val codeAttr = Utils.classForName("org.codehaus.janino.util.ClassFile$CodeAttribute") - val codeAttrField = codeAttr.getDeclaredField("code") - codeAttrField.setAccessible(true) val codeStats = classes.map { case (_, classBytes) => val classCodeSize = classBytes.length CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.update(classCodeSize) @@ -1588,8 +1586,8 @@ object CodeGenerator extends Logging { val cf = new ClassFile(new ByteArrayInputStream(classBytes)) val constPoolSize = cf.getConstantPoolSize val methodCodeSizes = cf.methodInfos.asScala.flatMap { method => - method.getAttributes().filter(_.getClass eq codeAttr).map { a => - val byteCodeSize = codeAttrField.get(a).asInstanceOf[Array[Byte]].length + method.getAttributes.collect { case attr: CodeAttribute => + val byteCodeSize = attr.code.length CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.update(byteCodeSize) if (byteCodeSize > DEFAULT_JVM_HUGE_METHOD_LIMIT) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromProtobufSqlFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromProtobufSqlFunctions.scala index 96bcf49dbd097..380df38ab049b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromProtobufSqlFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/toFromProtobufSqlFunctions.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.util.ArrayBasedMapData import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.types.{BinaryType, MapType, NullType, StringType} +import org.apache.spark.sql.types.{BinaryType, MapType, NullType, StringType, StructType} import org.apache.spark.sql.util.ProtobufUtils import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils @@ -238,6 +238,15 @@ case class ToProtobuf( } override def checkInputDataTypes(): TypeCheckResult = { + val colTypeCheck = first.dataType match { + case _: StructType => None + case _ => + Some( + TypeCheckResult.TypeCheckFailure( + "The first argument of the TO_PROTOBUF SQL function must be a struct type") + ) + } + val messageNameCheck = messageName.dataType match { case _: StringType if messageName.foldable => None case _ => @@ -262,10 +271,11 @@ case class ToProtobuf( "strings to strings containing the options to use for converting the value to " + "Protobuf format")) } - - messageNameCheck.getOrElse( - descFilePathCheck.getOrElse( - optionsCheck.getOrElse(TypeCheckResult.TypeCheckSuccess) + colTypeCheck.getOrElse( + messageNameCheck.getOrElse( + descFilePathCheck.getOrElse( + optionsCheck.getOrElse(TypeCheckResult.TypeCheckSuccess) + ) ) ) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSQLFunctionNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSQLFunctionNode.scala new file mode 100644 index 0000000000000..d9da38b4c2af4 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSQLFunctionNode.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.SparkException +import org.apache.spark.sql.catalyst.analysis.{SQLFunctionExpression, SQLFunctionNode, SQLScalarFunction, SQLTableFunction} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule + +/** + * This rule removes [[SQLScalarFunction]] and [[SQLFunctionNode]] wrapper. They are respected + * till the end of analysis stage because we want to see which part of an analyzed logical + * plan is generated from a SQL function and also perform ACL checks. + */ +object EliminateSQLFunctionNode extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = { + // Include subqueries when eliminating SQL function expressions otherwise we might miss + // expressions in subqueries which can be inlined by the rule `OptimizeOneRowRelationSubquery`. + plan.transformWithSubqueries { + case SQLFunctionNode(_, child) => child + case f: SQLTableFunction => + throw SparkException.internalError( + s"SQL table function plan should be rewritten during analysis: $f") + case p: LogicalPlan => p.transformExpressions { + case f: SQLScalarFunction => f.child + case f: SQLFunctionExpression => + throw SparkException.internalError( + s"SQL function expression should be rewritten during analysis: $f") + } + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 8ee2226947ec9..9d269f37e58b9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -315,6 +315,7 @@ abstract class Optimizer(catalogManager: CatalogManager) EliminateSubqueryAliases, EliminatePipeOperators, EliminateView, + EliminateSQLFunctionNode, ReplaceExpressions, RewriteNonCorrelatedExists, PullOutGroupingExpressions, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala index d7ba596cf3995..13df749c6d584 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala @@ -25,8 +25,34 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans.logical._ object NormalizePlan extends PredicateHelper { - def apply(plan: LogicalPlan): LogicalPlan = - normalizePlan(normalizeExprIds(plan)) + def apply(plan: LogicalPlan): LogicalPlan = { + val withNormalizedInheritAnalysis = normalizeInheritAnalysisRules(plan) + val withNormalizedExprIds = normalizeExprIds(withNormalizedInheritAnalysis) + normalizePlan(withNormalizedExprIds) + } + + /** + * Normalize [[InheritAnalysisRules]] nodes by replacing them with their replacement expressions. + * This is necessary because fixed-point analyzer may produce non-deterministic results when + * resolving original expressions. For example, in a query like: + * + * {{{ SELECT assert_true(1) }}} + * + * Before resolution, we have [[UnresolvedFunction]] whose child is Literal(1). This child will + * first be converted to Cast(Literal(1), BooleanType) by type coercion. Because in this case + * [[Cast]] doesn't require timezone, the expression will be implicitly resolved. Because the + * child of initially unresolved function is resolved, the function can be converted to + * [[AssertTrue]], which is of type [[InheritAnalysisRules]]. However, because the only child of + * [[InheritAnalysisRules]] is the replacement expression, the original expression will be lost + * timezone will never be applied. This causes inconsistencies, because fixed-point semantic is + * to ALWAYS apply timezone, regardless of whether or not the Cast actually needs it. + */ + def normalizeInheritAnalysisRules(plan: LogicalPlan): LogicalPlan = { + plan transformAllExpressions { + case inheritAnalysisRules: InheritAnalysisRules => + inheritAnalysisRules.child + } + } /** * Since attribute references are given globally unique ids during analysis, @@ -102,14 +128,15 @@ object NormalizePlan extends PredicateHelper { .sortBy(_.hashCode()) .reduce(And) Join(left, right, newJoinType, Some(newCondition), hint) - case Project(outerProjectList, innerProject: Project) => - val normalizedInnerProjectList = normalizeProjectList(innerProject.projectList) - val orderedInnerProjectList = normalizedInnerProjectList.sortBy(_.name) - val newInnerProject = - Project(orderedInnerProjectList, innerProject.child) - Project(normalizeProjectList(outerProjectList), newInnerProject) case Project(projectList, child) => - Project(normalizeProjectList(projectList), child) + val projList = projectList + .map { e => + e.transformUp { + case g: GetViewColumnByNameAndOrdinal => g.copy(viewDDL = None) + } + } + .asInstanceOf[Seq[NamedExpression]] + Project(projList, child) case c: KeepAnalyzedQuery => c.storeAnalyzedQuery() case localRelation: LocalRelation if !localRelation.data.isEmpty => /** @@ -127,16 +154,6 @@ object NormalizePlan extends PredicateHelper { } } - private def normalizeProjectList(projectList: Seq[NamedExpression]): Seq[NamedExpression] = { - projectList - .map { e => - e.transformUp { - case g: GetViewColumnByNameAndOrdinal => g.copy(viewDDL = None) - } - } - .asInstanceOf[Seq[NamedExpression]] - } - /** * Rewrite [[BinaryComparison]] operator to keep order. The following cases will be * equivalent: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala index b56085ecae8d6..9856a26346f6a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala @@ -93,6 +93,7 @@ object TreePattern extends Enumeration { val SESSION_WINDOW: Value = Value val SORT: Value = Value val SQL_FUNCTION_EXPRESSION: Value = Value + val SQL_SCALAR_FUNCTION: Value = Value val SQL_TABLE_FUNCTION: Value = Value val SUBQUERY_ALIAS: Value = Value val SUM: Value = Value diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 6077e55561e62..727d54b6bbd2e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -43,7 +43,7 @@ import org.apache.spark.sql.catalyst.analysis.{HintErrorLogger, Resolver} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator import org.apache.spark.sql.catalyst.plans.logical.HintErrorHandler -import org.apache.spark.sql.catalyst.util.{CollationFactory, DateTimeUtils} +import org.apache.spark.sql.catalyst.util.{CollationFactory, CollationNames, DateTimeUtils} import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.types.{AtomicType, StringType, TimestampNTZType, TimestampType} @@ -5787,7 +5787,7 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def trimCollationEnabled: Boolean = getConf(TRIM_COLLATION_ENABLED) override def defaultStringType: StringType = { - if (getConf(DEFAULT_COLLATION).toUpperCase(Locale.ROOT) == "UTF8_BINARY") { + if (getConf(DEFAULT_COLLATION).toUpperCase(Locale.ROOT) == CollationNames.UTF8_BINARY) { StringType } else { StringType(getConf(DEFAULT_COLLATION)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/NormalizePlanSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/NormalizePlanSuite.scala index 80cdf19e4b73b..5ff66098107c2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/NormalizePlanSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/NormalizePlanSuite.scala @@ -18,29 +18,62 @@ package org.apache.spark.sql.catalyst.plans import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.SQLConfHelper import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.catalyst.expressions.{AssertTrue, Cast, If, Literal, TimeZoneAwareExpression} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.types.BooleanType -class NormalizePlanSuite extends SparkFunSuite{ +class NormalizePlanSuite extends SparkFunSuite with SQLConfHelper { - test("Normalize Project") { - val baselineCol1 = $"col1".int - val testCol1 = baselineCol1.newInstance() - val baselinePlan = LocalRelation(baselineCol1).select(baselineCol1) - val testPlan = LocalRelation(testCol1).select(testCol1) + test("Normalize InheritAnalysisRules expressions") { + val castWithoutTimezone = + Cast(child = Literal(1), dataType = BooleanType, ansiEnabled = conf.ansiEnabled) + val castWithTimezone = castWithoutTimezone.withTimeZone(conf.sessionLocalTimeZone) - assert(baselinePlan != testPlan) - assert(NormalizePlan(baselinePlan) == NormalizePlan(testPlan)) - } + val baselineExpression = AssertTrue(castWithTimezone) + val baselinePlan = LocalRelation().select(baselineExpression) + + val testExpression = AssertTrue(castWithoutTimezone) + val testPlan = LocalRelation().select(testExpression) + + // Before calling [[setTimezoneForAllExpression]], [[AssertTrue]] node will look like: + // + // AssertTrue(Cast(Literal(1)), message, If(Cast(Literal(1)), Literal(null), error)) + // + // Calling [[setTimezoneForAllExpression]] will only apply timezone to the second Cast node + // because [[InheritAnalysisRules]] only sees replacement expression as its child. This will + // cause the difference when comparing [[resolvedBaselinePlan]] and [[resolvedTestPlan]], + // therefore we need normalization. + + // Before applying timezone, no timezone is set. + testPlan.expressions.foreach { + case _ @ AssertTrue(firstCast: Cast, _, _ @ If(secondCast: Cast, _, _)) => + assert(firstCast.timeZoneId.isEmpty) + assert(secondCast.timeZoneId.isEmpty) + case _ => + } - test("Normalize ordering in a project list of an inner Project") { - val baselinePlan = - LocalRelation($"col1".int, $"col2".string).select($"col1", $"col2").select($"col1") - val testPlan = - LocalRelation($"col1".int, $"col2".string).select($"col2", $"col1").select($"col1") + val resolvedBaselinePlan = setTimezoneForAllExpression(baselinePlan) + val resolvedTestPlan = setTimezoneForAllExpression(testPlan) + + // After applying timezone, only the second cast gets timezone. + resolvedTestPlan.expressions.foreach { + case _ @ AssertTrue(firstCast: Cast, _, _ @ If(secondCast: Cast, _, _)) => + assert(firstCast.timeZoneId.isEmpty) + assert(secondCast.timeZoneId.isDefined) + case _ => + } + + // However, plans are still different. + assert(resolvedBaselinePlan != resolvedTestPlan) + assert(NormalizePlan(resolvedBaselinePlan) == NormalizePlan(resolvedTestPlan)) + } - assert(baselinePlan != testPlan) - assert(NormalizePlan(baselinePlan) == NormalizePlan(testPlan)) + private def setTimezoneForAllExpression(plan: LogicalPlan): LogicalPlan = { + plan.transformAllExpressions { + case e: TimeZoneAwareExpression if e.timeZoneId.isEmpty => + e.withTimeZone(conf.sessionLocalTimeZone) + } } } diff --git a/sql/connect/common/src/main/protobuf/spark/connect/base.proto b/sql/connect/common/src/main/protobuf/spark/connect/base.proto index 74413509ebc14..1d34011b20e78 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/base.proto @@ -25,6 +25,7 @@ import "spark/connect/common.proto"; import "spark/connect/expressions.proto"; import "spark/connect/relations.proto"; import "spark/connect/types.proto"; +import "spark/connect/ml.proto"; option java_multiple_files = true; option java_package = "org.apache.spark.connect.proto"; @@ -395,6 +396,9 @@ message ExecutePlanResponse { // Response for command that checkpoints a DataFrame. CheckpointCommandResult checkpoint_command_result = 19; + // ML command response + MlCommandResult ml_command_result = 20; + // Support arbitrary result objects. google.protobuf.Any extension = 999; } diff --git a/sql/connect/common/src/main/protobuf/spark/connect/commands.proto b/sql/connect/common/src/main/protobuf/spark/connect/commands.proto index a01d4369a7aed..10033b6400b53 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/commands.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/commands.proto @@ -21,6 +21,7 @@ import "google/protobuf/any.proto"; import "spark/connect/common.proto"; import "spark/connect/expressions.proto"; import "spark/connect/relations.proto"; +import "spark/connect/ml.proto"; package spark.connect; @@ -48,7 +49,7 @@ message Command { CheckpointCommand checkpoint_command = 14; RemoveCachedRemoteRelationCommand remove_cached_remote_relation_command = 15; MergeIntoTableCommand merge_into_table_command = 16; - + MlCommand ml_command = 17; // This field is used to mark extensions to the protocol. When plugins generate arbitrary // Commands they can add them here. During the planning the correct resolution is done. google.protobuf.Any extension = 999; diff --git a/sql/connect/common/src/main/protobuf/spark/connect/ml.proto b/sql/connect/common/src/main/protobuf/spark/connect/ml.proto new file mode 100644 index 0000000000000..591004b000daa --- /dev/null +++ b/sql/connect/common/src/main/protobuf/spark/connect/ml.proto @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +syntax = 'proto3'; + +package spark.connect; + +import "spark/connect/expressions.proto"; +import "spark/connect/relations.proto"; +import "spark/connect/ml_common.proto"; + +option java_multiple_files = true; +option java_package = "org.apache.spark.connect.proto"; +option go_package = "internal/generated"; + +// Command for ML +message MlCommand { + oneof command { + Fit fit = 1; + Fetch fetch = 2; + Delete delete = 3; + Write write = 4; + Read read = 5; + } + + // Command for estimator.fit(dataset) + message Fit { + // Estimator information + MlOperator estimator = 1; + // parameters of the Estimator + MlParams params = 2; + // the training dataset + Relation dataset = 3; + } + + // Command to delete the cached object which could be a model + // or summary evaluated by a model + message Delete { + ObjectRef obj_ref = 1; + } + + // Command to write ML operator + message Write { + // It could be an estimator/evaluator or the cached model + oneof type { + // Estimator or evaluator + MlOperator operator = 1; + // The cached model + ObjectRef obj_ref = 2; + } + // The parameters of operator which could be estimator/evaluator or a cached model + MlParams params = 3; + // Save the ML instance to the path + string path = 4; + // Overwrites if the output path already exists. + bool should_overwrite = 5; + // The options of the writer + map options = 6; + } + + // Command to load ML operator. + message Read { + // ML operator information + MlOperator operator = 1; + // Load the ML instance from the input path + string path = 2; + } +} + +// The result of MlCommand +message MlCommandResult { + oneof result_type { + // The result of the attribute + Param param = 1; + // Evaluate a Dataset in a model and return the cached ID of summary + string summary = 2; + // Operator information + MlOperatorInfo operator_info = 3; + } + + // Represents an operator info + message MlOperatorInfo { + oneof type { + // The cached object which could be a model or summary evaluated by a model + ObjectRef obj_ref = 1; + // Operator name + string name = 2; + } + string uid = 3; + MlParams params = 4; + } + +} diff --git a/sql/connect/common/src/main/protobuf/spark/connect/ml_common.proto b/sql/connect/common/src/main/protobuf/spark/connect/ml_common.proto new file mode 100644 index 0000000000000..f91c2489ed947 --- /dev/null +++ b/sql/connect/common/src/main/protobuf/spark/connect/ml_common.proto @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +syntax = 'proto3'; + +package spark.connect; + +import "spark/connect/expressions.proto"; + +option java_multiple_files = true; +option java_package = "org.apache.spark.connect.proto"; +option go_package = "internal/generated"; + +// MlParams stores param settings for ML Estimator / Transformer / Evaluator +message MlParams { + // User-supplied params + map params = 1; +} + +// Represents the parameter type of the ML instance, or the returned value +// of the attribute +message Param { + oneof param_type { + Expression.Literal literal = 1; + Vector vector = 2; + Matrix matrix = 3; + } +} + +// MLOperator represents the ML operators like (Estimator, Transformer or Evaluator) +message MlOperator { + // The qualified name of the ML operator. + string name = 1; + // Unique id of the ML operator + string uid = 2; + // Represents what the ML operator is + OperatorType type = 3; + enum OperatorType { + UNSPECIFIED = 0; + ESTIMATOR = 1; + TRANSFORMER = 2; + EVALUATOR = 3; + MODEL = 4; + } +} + +// Represents a reference to the cached object which could be a model +// or summary evaluated by a model +message ObjectRef { + // The ID is used to lookup the object on the server side. + string id = 1; +} + +// See pyspark.ml.linalg.Vector +message Vector { + oneof vector_type { + Dense dense = 1; + Sparse sparse = 2; + } + // See pyspark.ml.linalg.DenseVector + message Dense { + repeated double value = 1; + } + // See pyspark.ml.linalg.SparseVector + message Sparse { + int32 size = 1; + repeated int32 index = 2; + repeated double value = 3; + } +} + +// See pyspark.ml.linalg.Matrix +message Matrix { + oneof matrix_type { + Dense dense = 1; + Sparse sparse = 2; + } + // See pyspark.ml.linalg.DenseMatrix + message Dense { + int32 num_rows = 1; + int32 num_cols = 2; + repeated double value = 3; + bool is_transposed = 4; + } + // See pyspark.ml.linalg.SparseMatrix + message Sparse { + int32 num_rows = 1; + int32 num_cols = 2; + repeated int32 colptr = 3; + repeated int32 row_index = 4; + repeated double value = 5; + bool is_transposed = 6; + } +} diff --git a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto index 5ab9f64149f51..c2cbed0dd22ba 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -24,6 +24,7 @@ import "spark/connect/expressions.proto"; import "spark/connect/types.proto"; import "spark/connect/catalog.proto"; import "spark/connect/common.proto"; +import "spark/connect/ml_common.proto"; option java_multiple_files = true; option java_package = "org.apache.spark.connect.proto"; @@ -98,6 +99,9 @@ message Relation { // Catalog API (experimental / unstable) Catalog catalog = 200; + // ML relation + MlRelation ml_relation = 300; + // This field is used to mark extensions to the protocol. When plugins generate arbitrary // relations they can add them here. During the planning the correct resolution is done. google.protobuf.Any extension = 998; @@ -105,6 +109,55 @@ message Relation { } } +// Relation to represent ML world +message MlRelation { + oneof ml_type { + Transform transform = 1; + Fetch fetch = 2; + } + // Relation to represent transform(input) of the operator + // which could be a cached model or a new transformer + message Transform { + oneof operator { + // Object reference + ObjectRef obj_ref = 1; + // Could be an ML transformer like VectorAssembler + MlOperator transformer = 2; + } + // the input dataframe + Relation input = 3; + // the operator specific parameters + MlParams params = 4; + } +} + +// Message for fetching attribute from object on the server side. +// Fetch can be represented as a Relation or a ML command +// Command: model.coefficients, model.summary.weightedPrecision which +// returns the final literal result +// Relation: model.summary.roc which returns a DataFrame (Relation) +message Fetch { + // (Required) reference to the object on the server side + ObjectRef obj_ref = 1; + // (Required) the calling method chains + repeated Method methods = 2; + + // Represents a method with inclusion of method name and its arguments + message Method { + // (Required) the method name + string method = 1; + // (Optional) the arguments of the method + repeated Args args = 2; + + message Args { + oneof args_type { + Param param = 1; + Relation input = 2; + } + } + } +} + // Used for testing purposes only. message Unknown {} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala index 44b634af95ca9..21b5e057fb77e 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala @@ -241,14 +241,13 @@ private[connect] class ExecuteGrpcResponseSender[T <: Message]( // The state of interrupted, response and lastIndex are changed under executionObserver // monitor, and will notify upon state change. if (response.isEmpty) { + var timeout = Math.max(1, deadlineTimeMillis - System.currentTimeMillis()) // Wake up more frequently to send the progress updates. val progressTimeout = executeHolder.sessionHolder.session.sessionState.conf .getConf(CONNECT_PROGRESS_REPORT_INTERVAL) // If the progress feature is disabled, wait for the deadline. - val timeout = if (progressTimeout > 0) { - progressTimeout - } else { - Math.max(1, deadlineTimeMillis - System.currentTimeMillis()) + if (progressTimeout > 0L) { + timeout = Math.min(progressTimeout, timeout) } logTrace(s"Wait for response to become available with timeout=$timeout ms.") executionObserver.responseLock.wait(timeout) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLCache.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLCache.scala new file mode 100644 index 0000000000000..a036f8b67350d --- /dev/null +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLCache.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connect.ml + +import java.util.UUID +import java.util.concurrent.ConcurrentHashMap + +import org.apache.spark.internal.Logging + +/** + * MLCache is for caching ML objects, typically for models and summaries evaluated by a model. + */ +private[connect] class MLCache extends Logging { + private val cachedModel: ConcurrentHashMap[String, Object] = + new ConcurrentHashMap[String, Object]() + + /** + * Cache an object into a map of MLCache, and return its key + * @param obj + * the object to be cached + * @return + * the key + */ + def register(obj: Object): String = { + val objectId = UUID.randomUUID().toString + cachedModel.put(objectId, obj) + objectId + } + + /** + * Get the object by the key + * @param refId + * the key used to look up the corresponding object + * @return + * the cached object + */ + def get(refId: String): Object = { + cachedModel.get(refId) + } + + /** + * Remove the object from MLCache + * @param refId + * the key used to look up the corresponding object + */ + def remove(refId: String): Unit = { + cachedModel.remove(refId) + } + + /** + * Clear all the caches + */ + def clear(): Unit = { + cachedModel.clear() + } +} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLException.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLException.scala new file mode 100644 index 0000000000000..eb88bf9169d3d --- /dev/null +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLException.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connect.ml + +import org.apache.spark.SparkException + +private[spark] case class MlUnsupportedException(message: String) + extends SparkException( + errorClass = "CONNECT_ML.UNSUPPORTED_EXCEPTION", + messageParameters = Map("message" -> message), + cause = null) + +private[spark] case class MLAttributeNotAllowedException(attribute: String) + extends SparkException( + errorClass = "CONNECT_ML.ATTRIBUTE_NOT_ALLOWED", + messageParameters = Map("attribute" -> attribute), + cause = null) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLHandler.scala new file mode 100644 index 0000000000000..b4bc6bfdc66b4 --- /dev/null +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLHandler.scala @@ -0,0 +1,268 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connect.ml + +import scala.jdk.CollectionConverters.CollectionHasAsScala + +import org.apache.spark.connect.proto +import org.apache.spark.internal.Logging +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.util.{MLWritable, Summary} +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.connect.common.LiteralValueProtoConverter +import org.apache.spark.sql.connect.ml.Serializer.deserializeMethodArguments +import org.apache.spark.sql.connect.service.SessionHolder + +private case class Method( + name: String, + argValues: Array[Object] = Array.empty, + argClasses: Array[Class[_]] = Array.empty) + +/** + * Helper function to get the attribute from an object by reflection + */ +private class AttributeHelper( + val sessionHolder: SessionHolder, + val objRef: String, + val methods: Array[Method]) { + protected lazy val instance = sessionHolder.mlCache.get(objRef) + // Get the attribute by reflection + def getAttribute: Any = { + assert(methods.length >= 1) + methods.foldLeft(instance) { (obj, m) => + if (m.argValues.isEmpty) { + MLUtils.invokeMethodAllowed(obj, m.name) + } else { + MLUtils.invokeMethodAllowed(obj, m.name, m.argValues, m.argClasses) + } + } + } +} + +// Model specific attribute helper with transform supported +private class ModelAttributeHelper( + sessionHolder: SessionHolder, + objRef: String, + methods: Array[Method]) + extends AttributeHelper(sessionHolder, objRef, methods) { + + def transform(relation: proto.MlRelation.Transform): DataFrame = { + // Create a copied model to avoid concurrently modify model params. + val model = instance.asInstanceOf[Model[_]] + val copiedModel = model.copy(ParamMap.empty).asInstanceOf[Model[_]] + MLUtils.setInstanceParams(copiedModel, relation.getParams) + val inputDF = MLUtils.parseRelationProto(relation.getInput, sessionHolder) + copiedModel.transform(inputDF) + } +} + +private object AttributeHelper { + def parseMethods( + sessionHolder: SessionHolder, + methodsProto: Array[proto.Fetch.Method] = Array.empty): Array[Method] = { + methodsProto.map { m => + val (argValues, argClasses) = + deserializeMethodArguments(m.getArgsList.asScala.toArray, sessionHolder).unzip + Method(m.getMethod, argValues, argClasses) + } + } + def apply( + sessionHolder: SessionHolder, + objId: String, + methodsProto: Array[proto.Fetch.Method] = Array.empty): AttributeHelper = { + new AttributeHelper(sessionHolder, objId, parseMethods(sessionHolder, methodsProto)) + } +} + +private object ModelAttributeHelper { + def apply( + sessionHolder: SessionHolder, + objId: String, + methodsProto: Array[proto.Fetch.Method] = Array.empty): ModelAttributeHelper = { + new ModelAttributeHelper( + sessionHolder, + objId, + AttributeHelper.parseMethods(sessionHolder, methodsProto)) + } +} + +// MLHandler is a utility to group all ML operations +private[connect] object MLHandler extends Logging { + def handleMlCommand( + sessionHolder: SessionHolder, + mlCommand: proto.MlCommand): proto.MlCommandResult = { + + val mlCache = sessionHolder.mlCache + + mlCommand.getCommandCase match { + case proto.MlCommand.CommandCase.FIT => + val fitCmd = mlCommand.getFit + val estimatorProto = fitCmd.getEstimator + assert(estimatorProto.getType == proto.MlOperator.OperatorType.ESTIMATOR) + + val dataset = MLUtils.parseRelationProto(fitCmd.getDataset, sessionHolder) + val estimator = MLUtils.getEstimator(estimatorProto, Some(fitCmd.getParams)) + val model = estimator.fit(dataset).asInstanceOf[Model[_]] + val id = mlCache.register(model) + proto.MlCommandResult + .newBuilder() + .setOperatorInfo( + proto.MlCommandResult.MlOperatorInfo + .newBuilder() + .setObjRef(proto.ObjectRef.newBuilder().setId(id))) + .build() + + case proto.MlCommand.CommandCase.FETCH => + val helper = AttributeHelper( + sessionHolder, + mlCommand.getFetch.getObjRef.getId, + mlCommand.getFetch.getMethodsList.asScala.toArray) + val attrResult = helper.getAttribute + attrResult match { + case s: Summary => + val id = mlCache.register(s) + proto.MlCommandResult.newBuilder().setSummary(id).build() + case _ => + val param = Serializer.serializeParam(attrResult) + proto.MlCommandResult.newBuilder().setParam(param).build() + } + + case proto.MlCommand.CommandCase.DELETE => + val objId = mlCommand.getDelete.getObjRef.getId + var result = false + if (!objId.contains(".")) { + mlCache.remove(objId) + result = true + } + proto.MlCommandResult + .newBuilder() + .setParam( + proto.Param + .newBuilder() + .setLiteral(LiteralValueProtoConverter.toLiteralProto(result)) + .build()) + .build() + + case proto.MlCommand.CommandCase.WRITE => + mlCommand.getWrite.getTypeCase match { + case proto.MlCommand.Write.TypeCase.OBJ_REF => // save a model + val objId = mlCommand.getWrite.getObjRef.getId + val model = mlCache.get(objId).asInstanceOf[Model[_]] + val copiedModel = model.copy(ParamMap.empty).asInstanceOf[Model[_]] + MLUtils.setInstanceParams(copiedModel, mlCommand.getWrite.getParams) + + copiedModel match { + case m: MLWritable => MLUtils.write(m, mlCommand.getWrite) + case other => throw MlUnsupportedException(s"$other is not writable") + } + + // save an estimator/evaluator/transformer + case proto.MlCommand.Write.TypeCase.OPERATOR => + val writer = mlCommand.getWrite + if (writer.getOperator.getType == proto.MlOperator.OperatorType.ESTIMATOR) { + val estimator = MLUtils.getEstimator(writer.getOperator, Some(writer.getParams)) + estimator match { + case m: MLWritable => MLUtils.write(m, mlCommand.getWrite) + case other => throw MlUnsupportedException(s"Estimator $other is not writable") + } + } else { + throw MlUnsupportedException(s"${writer.getOperator.getName} not supported") + } + + case other => throw MlUnsupportedException(s"$other not supported") + } + proto.MlCommandResult.newBuilder().build() + + case proto.MlCommand.CommandCase.READ => + val operator = mlCommand.getRead.getOperator + val name = operator.getName + val path = mlCommand.getRead.getPath + + if (operator.getType == proto.MlOperator.OperatorType.MODEL) { + val model = MLUtils.load(name, path).asInstanceOf[Model[_]] + val id = mlCache.register(model) + proto.MlCommandResult + .newBuilder() + .setOperatorInfo( + proto.MlCommandResult.MlOperatorInfo + .newBuilder() + .setObjRef(proto.ObjectRef.newBuilder().setId(id)) + .setUid(model.uid) + .setParams(Serializer.serializeParams(model))) + .build() + + } else if (operator.getType == proto.MlOperator.OperatorType.ESTIMATOR) { + val estimator = MLUtils.load(name, path).asInstanceOf[Estimator[_]] + proto.MlCommandResult + .newBuilder() + .setOperatorInfo( + proto.MlCommandResult.MlOperatorInfo + .newBuilder() + .setName(name) + .setUid(estimator.uid) + .setParams(Serializer.serializeParams(estimator))) + .build() + } else { + throw MlUnsupportedException(s"${operator.getType} not supported") + } + + case other => throw MlUnsupportedException(s"$other not supported") + } + } + + def transformMLRelation(relation: proto.MlRelation, sessionHolder: SessionHolder): DataFrame = { + relation.getMlTypeCase match { + // Ml transform + case proto.MlRelation.MlTypeCase.TRANSFORM => + relation.getTransform.getOperatorCase match { + // transform for a new ML transformer + case proto.MlRelation.Transform.OperatorCase.TRANSFORMER => + val transformProto = relation.getTransform + assert( + transformProto.getTransformer.getType == + proto.MlOperator.OperatorType.TRANSFORMER) + val dataset = MLUtils.parseRelationProto(transformProto.getInput, sessionHolder) + val transformer = MLUtils.getTransformer(transformProto) + transformer.transform(dataset) + + // transform on a cached model + case proto.MlRelation.Transform.OperatorCase.OBJ_REF => + val helper = + ModelAttributeHelper( + sessionHolder, + relation.getTransform.getObjRef.getId, + Array.empty) + helper.transform(relation.getTransform) + + case other => throw new IllegalArgumentException(s"$other not supported") + } + + // Get the attribute from a cached object which could be a model or summary + case proto.MlRelation.MlTypeCase.FETCH => + val helper = AttributeHelper( + sessionHolder, + relation.getFetch.getObjRef.getId, + relation.getFetch.getMethodsList.asScala.toArray) + helper.getAttribute.asInstanceOf[DataFrame] + + case other => throw MlUnsupportedException(s"$other not supported") + } + } + +} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLUtils.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLUtils.scala new file mode 100644 index 0000000000000..72c86401eb631 --- /dev/null +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLUtils.scala @@ -0,0 +1,353 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connect.ml + +import java.util.ServiceLoader + +import scala.collection.immutable.HashSet +import scala.jdk.CollectionConverters._ + +import org.apache.commons.lang3.reflect.MethodUtils.invokeMethod + +import org.apache.spark.connect.proto +import org.apache.spark.ml.{Estimator, Transformer} +import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector, Vectors} +import org.apache.spark.ml.param.Params +import org.apache.spark.ml.util.MLWritable +import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.connect.common.LiteralValueProtoConverter +import org.apache.spark.sql.connect.planner.SparkConnectPlanner +import org.apache.spark.sql.connect.service.SessionHolder +import org.apache.spark.util.{SparkClassUtils, Utils} + +private[ml] object MLUtils { + + /** + * Load the registered ML operators via ServiceLoader + * + * @param mlCls + * the operator class + * @return + * a Map with name and class + */ + private def loadOperators(mlCls: Class[_]): Map[String, Class[_]] = { + val loader = Utils.getContextOrSparkClassLoader + val serviceLoader = ServiceLoader.load(mlCls, loader) + val providers = serviceLoader.asScala.toList + providers.map(est => est.getClass.getName -> est.getClass).toMap + } + + private lazy val estimators = loadOperators(classOf[Estimator[_]]) + + private lazy val transformers = loadOperators(classOf[Transformer]) + + def deserializeVector(vector: proto.Vector): Vector = { + if (vector.hasDense) { + val values = vector.getDense.getValueList.asScala.map(_.toDouble).toArray + Vectors.dense(values) + } else { + val size = vector.getSparse.getSize + val indices = vector.getSparse.getIndexList.asScala.map(_.toInt).toArray + val values = vector.getSparse.getValueList.asScala.map(_.toDouble).toArray + Vectors.sparse(size, indices, values) + } + } + + def deserializeMatrix(matrix: proto.Matrix): Matrix = { + if (matrix.hasDense) { + val values = matrix.getDense.getValueList.asScala.map(_.toDouble).toArray + Matrices.dense(matrix.getDense.getNumRows, matrix.getDense.getNumCols, values) + } else { + val sparse = matrix.getSparse + val colPtrs = sparse.getColptrList.asScala.map(_.toInt).toArray + val rowIndices = sparse.getRowIndexList.asScala.map(_.toInt).toArray + val values = sparse.getValueList.asScala.map(_.toDouble).toArray + Matrices.sparse(sparse.getNumRows, sparse.getNumCols, colPtrs, rowIndices, values) + } + } + + /** + * Set the parameters to the ML instance + * + * @param instance + * an ML operator + * @param params + * the parameters of the ML operator + */ + def setInstanceParams(instance: Params, params: proto.MlParams): Unit = { + params.getParamsMap.asScala.foreach { case (name, paramProto) => + val p = instance.getParam(name) + val value = if (paramProto.hasLiteral) { + reconcileParam( + p.paramValueClassTag.runtimeClass, + LiteralValueProtoConverter.toCatalystValue(paramProto.getLiteral)) + } else if (paramProto.hasVector) { + deserializeVector(paramProto.getVector) + } else if (paramProto.hasMatrix) { + deserializeMatrix(paramProto.getMatrix) + } else { + throw MlUnsupportedException(s"Unsupported parameter type for ${name}") + } + instance.set(p, value) + } + } + + /** + * Convert the array from Object[] to Array[_] + * @param elementType + * the element type of the array + * @param array + * to be reconciled + * @return + * the reconciled array + */ + private def reconcileArray(elementType: Class[_], array: Array[_]): Array[_] = { + if (elementType == classOf[Byte]) { + array.map(_.asInstanceOf[Byte]) + } else if (elementType == classOf[Short]) { + array.map(_.asInstanceOf[Short]) + } else if (elementType == classOf[Int]) { + array.map(_.asInstanceOf[Int]) + } else if (elementType == classOf[Long]) { + array.map(_.asInstanceOf[Long]) + } else if (elementType == classOf[Float]) { + array.map(_.asInstanceOf[Float]) + } else if (elementType == classOf[Double]) { + array.map(_.asInstanceOf[Double]) + } else if (elementType == classOf[String]) { + array.map(_.asInstanceOf[String]) + } else { + throw MlUnsupportedException( + s"array element type unsupported, " + + s"found ${elementType.getName}") + } + } + + /** + * Reconcile the parameter value given the provided parameter type. Currently, support + * byte/short/int/long/float/double/string and array. Note that, array of array is not supported + * yet. + */ + private def reconcileParam(paramType: Class[_], value: Any): Any = { + // Some cases the param type might be mismatched with the value type. + // Because in python side we only have int / float type for numeric params. + // e.g.: + // param type is Int but client sends a Long type. + // param type is Long but client sends a Int type. + // param type is Float but client sends a Double type. + // param type is Array[Int] but client sends a Array[Long] type. + // param type is Array[Float] but client sends a Array[Double] type. + // param type is Array[Array[Int]] but client sends a Array[Array[Long]] type. + // param type is Array[Array[Float]] but client sends a Array[Array[Double]] type. + if (paramType == classOf[Byte]) { + value.asInstanceOf[java.lang.Number].byteValue() + } else if (paramType == classOf[Short]) { + value.asInstanceOf[java.lang.Number].shortValue() + } else if (paramType == classOf[Int]) { + value.asInstanceOf[java.lang.Number].intValue() + } else if (paramType == classOf[Long]) { + value.asInstanceOf[java.lang.Number].longValue() + } else if (paramType == classOf[Float]) { + value.asInstanceOf[java.lang.Number].floatValue() + } else if (paramType == classOf[Double]) { + value.asInstanceOf[java.lang.Number].doubleValue() + } else if (paramType == classOf[Boolean]) { + value.asInstanceOf[Boolean] + } else if (paramType == classOf[String]) { + value.asInstanceOf[String] + } else if (paramType.isArray) { + val compType = paramType.getComponentType + if (compType.isArray) { + throw MlUnsupportedException(s"Array of array unsupported") + } else { + val array = value.asInstanceOf[Array[_]].map { e => + reconcileParam(compType, e) + } + reconcileArray(compType, array) + } + } else { + throw MlUnsupportedException(s"Unsupported parameter type, found ${paramType.getName}") + } + } + + def parseRelationProto(relation: proto.Relation, sessionHolder: SessionHolder): DataFrame = { + val planner = new SparkConnectPlanner(sessionHolder) + val plan = planner.transformRelation(relation) + Dataset.ofRows(sessionHolder.session, plan) + } + + /** + * Get the instance according to the provided proto information. + * + * @param name + * The name of the instance (either estimator or transformer). + * @param uid + * The unique identifier for the instance. + * @param instanceMap + * A map of instance names to constructors. + * @param params + * Optional parameters for the instance. + * @tparam T + * The type of the instance (Estimator or Transformer). + * @return + * The instance of the requested type. + * @throws MlUnsupportedException + * If the instance is not supported. + */ + private def getInstance[T]( + name: String, + uid: String, + instanceMap: Map[String, Class[_]], + params: Option[proto.MlParams]): T = { + if (instanceMap.isEmpty || !instanceMap.contains(name)) { + throw MlUnsupportedException(s"Unsupported ML operator, found $name") + } + + val instance = instanceMap(name) + .getConstructor(classOf[String]) + .newInstance(uid) + .asInstanceOf[T] + + // Set parameters for the instance if they are provided + params.foreach(p => MLUtils.setInstanceParams(instance.asInstanceOf[Params], p)) + instance + } + + /** + * Get the Estimator instance according to the proto information + * + * @param operator + * MlOperator information + * @param params + * The optional parameters of the estimator + * @return + * the estimator + */ + def getEstimator(operator: proto.MlOperator, params: Option[proto.MlParams]): Estimator[_] = { + val name = operator.getName + val uid = operator.getUid + getInstance[Estimator[_]](name, uid, estimators, params) + } + + /** + * Get the transformer instance according to the transform proto + * + * @param transformProto + * transform proto + * @return + * a transformer + */ + def getTransformer(transformProto: proto.MlRelation.Transform): Transformer = { + val name = transformProto.getTransformer.getName + val uid = transformProto.getTransformer.getUid + val params = transformProto.getParams + getInstance[Transformer](name, uid, transformers, Some(params)) + } + + /** + * Call "load: function on the ML operator given the operator name + * + * @param className + * the ML operator name + * @param path + * the path to be loaded + * @return + * the ML instance + */ + def load(className: String, path: String): Object = { + val loadedMethod = SparkClassUtils.classForName(className).getMethod("load", classOf[String]) + loadedMethod.invoke(null, path) + } + + // Since we're using reflection way to get the attribute, in order not to + // leave a security hole, we define an allowed attribute list that can be accessed. + // The attributes could be retrieved from the corresponding python class + private lazy val ALLOWED_ATTRIBUTES = HashSet( + "toString", + "numFeatures", + "predict", // PredictionModel + "numClasses", + "predictRaw", // ClassificationModel + "predictProbability", // ProbabilisticClassificationModel + "coefficients", + "intercept", + "coefficientMatrix", + "interceptVector", // LogisticRegressionModel + "summary", + "hasSummary", + "evaluate", // LogisticRegressionModel + "predictions", + "predictionCol", + "labelCol", + "weightCol", + "labels", // _ClassificationSummary + "truePositiveRateByLabel", + "falsePositiveRateByLabel", // _ClassificationSummary + "precisionByLabel", + "recallByLabel", + "fMeasureByLabel", + "accuracy", // _ClassificationSummary + "weightedTruePositiveRate", + "weightedFalsePositiveRate", // _ClassificationSummary + "weightedRecall", + "weightedPrecision", + "weightedFMeasure", // _ClassificationSummary + "scoreCol", + "roc", + "areaUnderROC", + "pr", + "fMeasureByThreshold", // _BinaryClassificationSummary + "precisionByThreshold", + "recallByThreshold", // _BinaryClassificationSummary + "probabilityCol", + "featuresCol", // LogisticRegressionSummary + "objectiveHistory", + "totalIterations" // _TrainingSummary + ) + + def invokeMethodAllowed(obj: Object, methodName: String): Object = { + if (!ALLOWED_ATTRIBUTES.contains(methodName)) { + throw MLAttributeNotAllowedException(methodName) + } + invokeMethod(obj, methodName) + } + + def invokeMethodAllowed( + obj: Object, + methodName: String, + args: Array[Object], + parameterTypes: Array[Class[_]]): Object = { + if (!ALLOWED_ATTRIBUTES.contains(methodName)) { + throw MLAttributeNotAllowedException(methodName) + } + invokeMethod(obj, methodName, args, parameterTypes) + } + + def write(instance: MLWritable, writeProto: proto.MlCommand.Write): Unit = { + val writer = if (writeProto.getShouldOverwrite) { + instance.write.overwrite() + } else { + instance.write + } + val path = writeProto.getPath + val options = writeProto.getOptionsMap + options.forEach((k, v) => writer.option(k, v)) + writer.save(path) + } + +} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/Serializer.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/Serializer.scala new file mode 100644 index 0000000000000..ad6735997f834 --- /dev/null +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/Serializer.scala @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connect.ml + +import org.apache.spark.connect.proto +import org.apache.spark.ml.linalg.{DenseMatrix, DenseVector, SparseMatrix, SparseVector} +import org.apache.spark.ml.param.Params +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.connect.common.LiteralValueProtoConverter +import org.apache.spark.sql.connect.service.SessionHolder + +private[ml] object Serializer { + + /** + * Serialize the ML parameters, currently support Vector/Matrix and literals + * @param data + * the value of parameter + * @return + * proto.Param + */ + def serializeParam(data: Any): proto.Param = { + data match { + case v: DenseVector => + val denseBuilder = proto.Vector.Dense.newBuilder() + v.values.foreach(denseBuilder.addValue) + proto.Param + .newBuilder() + .setVector(proto.Vector.newBuilder().setDense(denseBuilder)) + .build() + case v: SparseVector => + val sparseBuilder = proto.Vector.Sparse.newBuilder().setSize(v.size) + v.indices.foreach(sparseBuilder.addIndex) + v.values.foreach(sparseBuilder.addValue) + proto.Param + .newBuilder() + .setVector(proto.Vector.newBuilder().setSparse(sparseBuilder)) + .build() + case v: DenseMatrix => + val denseBuilder = proto.Matrix.Dense.newBuilder() + v.values.foreach(denseBuilder.addValue) + denseBuilder.setNumCols(v.numCols) + denseBuilder.setNumRows(v.numRows) + denseBuilder.setIsTransposed(v.isTransposed) + proto.Param + .newBuilder() + .setMatrix(proto.Matrix.newBuilder().setDense(denseBuilder)) + .build() + case v: SparseMatrix => + val sparseBuilder = proto.Matrix.Sparse + .newBuilder() + .setNumCols(v.numCols) + .setNumRows(v.numRows) + v.values.foreach(sparseBuilder.addValue) + v.colPtrs.foreach(sparseBuilder.addColptr) + v.rowIndices.foreach(sparseBuilder.addRowIndex) + proto.Param + .newBuilder() + .setMatrix(proto.Matrix.newBuilder().setSparse(sparseBuilder)) + .build() + case _: Byte | _: Short | _: Int | _: Long | _: Float | _: Double | _: Boolean | _: String | + _: Array[_] => + proto.Param + .newBuilder() + .setLiteral(LiteralValueProtoConverter.toLiteralProto(data)) + .build() + + case other => throw MlUnsupportedException(s"$other not supported") + } + } + + def deserializeMethodArguments( + args: Array[proto.Fetch.Method.Args], + sessionHolder: SessionHolder): Array[(Object, Class[_])] = { + args.map { arg => + if (arg.hasParam) { + val param = arg.getParam + if (param.hasLiteral) { + param.getLiteral.getLiteralTypeCase match { + case proto.Expression.Literal.LiteralTypeCase.INTEGER => + (param.getLiteral.getInteger.asInstanceOf[Object], classOf[Int]) + case proto.Expression.Literal.LiteralTypeCase.FLOAT => + (param.getLiteral.getFloat.toDouble.asInstanceOf[Object], classOf[Double]) + case proto.Expression.Literal.LiteralTypeCase.STRING => + (param.getLiteral.getString, classOf[String]) + case proto.Expression.Literal.LiteralTypeCase.DOUBLE => + (param.getLiteral.getDouble.asInstanceOf[Object], classOf[Double]) + case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => + (param.getLiteral.getBoolean.asInstanceOf[Object], classOf[Boolean]) + case other => + throw MlUnsupportedException(s"$other not supported") + } + } else if (param.hasVector) { + val vector = MLUtils.deserializeVector(param.getVector) + val vectorType = if (param.getVector.hasDense) { + classOf[DenseVector] + } else { + classOf[SparseVector] + } + (vector, vectorType) + } else if (param.hasMatrix) { + val matrix = MLUtils.deserializeMatrix(param.getMatrix) + val matrixType = if (param.getMatrix.hasDense) { + classOf[DenseMatrix] + } else { + classOf[SparseMatrix] + } + (matrix, matrixType) + } else { + throw MlUnsupportedException(s"$param not supported") + } + } else if (arg.hasInput) { + (MLUtils.parseRelationProto(arg.getInput, sessionHolder), classOf[Dataset[_]]) + } else { + throw MlUnsupportedException(s"$arg not supported") + } + } + } + + /** + * Serialize an instance of "Params" which could be estimator/model/evaluator ... + * @param instance + * of Params + * @return + * proto.MlParams + */ + def serializeParams(instance: Params): proto.MlParams = { + val builder = proto.MlParams.newBuilder() + instance.params.foreach { param => + if (instance.isSet(param)) { + val v = serializeParam(instance.get(param).get) + builder.putParams(param.name, v) + } + } + builder.build() + } +} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index c0b4384af8b6d..94a1ab1618086 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -45,7 +45,7 @@ import org.apache.spark.internal.LogKeys.{DATAFRAME_ID, SESSION_ID} import org.apache.spark.resource.{ExecutorResourceRequest, ResourceProfile, TaskResourceProfile, TaskResourceRequest} import org.apache.spark.sql.{Column, Dataset, Encoders, ForeachWriter, Observation, RelationalGroupedDataset, Row, SparkSession} import org.apache.spark.sql.catalyst.{expressions, AliasIdentifier, FunctionIdentifier, QueryPlanningTracker} -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, GlobalTempView, LazyExpression, LocalTempView, MultiAlias, NameParameterizedQuery, PosParameterizedQuery, UnresolvedAlias, UnresolvedAttribute, UnresolvedDataFrameStar, UnresolvedDeserializer, UnresolvedExtractValue, UnresolvedFunction, UnresolvedPlanId, UnresolvedRegex, UnresolvedRelation, UnresolvedStar, UnresolvedSubqueryColumnAliases, UnresolvedTableValuedFunction, UnresolvedTranspose} +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, GlobalTempView, LazyExpression, LocalTempView, MultiAlias, NameParameterizedQuery, PosParameterizedQuery, UnresolvedAlias, UnresolvedAttribute, UnresolvedDataFrameStar, UnresolvedDeserializer, UnresolvedExtractValue, UnresolvedFunction, UnresolvedPlanId, UnresolvedRegex, UnresolvedRelation, UnresolvedStar, UnresolvedStarWithColumns, UnresolvedStarWithColumnsRenames, UnresolvedSubqueryColumnAliases, UnresolvedTableValuedFunction, UnresolvedTranspose} import org.apache.spark.sql.catalyst.encoders.{encoderFor, AgnosticEncoder, ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.UnboundRowEncoder import org.apache.spark.sql.catalyst.expressions._ @@ -61,6 +61,7 @@ import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CharVarcharUtils} import org.apache.spark.sql.classic.ClassicConversions._ import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, ForeachWriterPacket, InvalidCommandInput, InvalidPlanInput, LiteralValueProtoConverter, StorageLevelProtoConverter, StreamingListenerPacket, UdfPacket} import org.apache.spark.sql.connect.config.Connect.CONNECT_GRPC_ARROW_MAX_BATCH_SIZE +import org.apache.spark.sql.connect.ml.MLHandler import org.apache.spark.sql.connect.plugin.SparkConnectPluginRegistry import org.apache.spark.sql.connect.service.{ExecuteHolder, SessionHolder, SparkConnectService} import org.apache.spark.sql.connect.utils.MetricGenerator @@ -223,6 +224,10 @@ class SparkConnectPlanner( // Catalog API (internal-only) case proto.Relation.RelTypeCase.CATALOG => transformCatalog(rel.getCatalog) + // ML Relation + case proto.Relation.RelTypeCase.ML_RELATION => + MLHandler.transformMLRelation(rel.getMlRelation, sessionHolder).logicalPlan + // Handle plugins for Spark Connect Relation types. case proto.Relation.RelTypeCase.EXTENSION => transformRelationPlugin(rel.getExtension) @@ -845,9 +850,10 @@ class SparkConnectPlanner( kEncoder: ExpressionEncoder[_], vEncoder: ExpressionEncoder[_], analyzed: LogicalPlan, - dataAttributes: Seq[Attribute], + analyzedData: LogicalPlan, groupingAttributes: Seq[Attribute], sortOrder: Seq[SortOrder]) { + val dataAttributes: Seq[Attribute] = analyzedData.output val valueDeserializer: Expression = UnresolvedDeserializer(vEncoder.deserializer, dataAttributes) } @@ -900,7 +906,7 @@ class SparkConnectPlanner( dummyFunc.outEnc, dummyFunc.inEnc, qe.analyzed, - analyzed.output, + analyzed, aliasedGroupings, sortOrder) } @@ -924,7 +930,7 @@ class SparkConnectPlanner( kEnc, vEnc, withGroupingKeyAnalyzed, - analyzed.output, + analyzed, withGroupingKey.newColumns, sortOrder) } @@ -1064,25 +1070,21 @@ class SparkConnectPlanner( } private def transformWithColumnsRenamed(rel: proto.WithColumnsRenamed): LogicalPlan = { - if (rel.getRenamesCount > 0) { - val (colNames, newColNames) = rel.getRenamesList.asScala.toSeq.map { rename => + val (colNames, newColNames) = if (rel.getRenamesCount > 0) { + rel.getRenamesList.asScala.toSeq.map { rename => (rename.getColName, rename.getNewColName) }.unzip - Dataset - .ofRows(session, transformRelation(rel.getInput)) - .withColumnsRenamed(colNames, newColNames) - .logicalPlan } else { // for backward compatibility - Dataset - .ofRows(session, transformRelation(rel.getInput)) - .withColumnsRenamed(rel.getRenameColumnsMapMap) - .logicalPlan + rel.getRenameColumnsMapMap.asScala.toSeq.unzip } + Project( + Seq(UnresolvedStarWithColumnsRenames(existingNames = colNames, newNames = newColNames)), + transformRelation(rel.getInput)) } private def transformWithColumns(rel: proto.WithColumns): LogicalPlan = { - val (colNames, cols, metadata) = + val (colNames, exprs, metadata) = rel.getAliasesList.asScala.toSeq.map { alias => if (alias.getNameCount != 1) { throw InvalidPlanInput(s"""WithColumns require column name only contains one name part, @@ -1095,13 +1097,16 @@ class SparkConnectPlanner( Metadata.empty } - (alias.getName(0), Column(transformExpression(alias.getExpr)), metadata) + (alias.getName(0), transformExpression(alias.getExpr), metadata) }.unzip3 - Dataset - .ofRows(session, transformRelation(rel.getInput)) - .withColumns(colNames, cols, metadata) - .logicalPlan + Project( + Seq( + UnresolvedStarWithColumns( + colNames = colNames, + exprs = exprs, + explicitMetadata = Some(metadata))), + transformRelation(rel.getInput)) } private def transformWithWatermark(rel: proto.WithWatermark): LogicalPlan = { @@ -1489,11 +1494,19 @@ class SparkConnectPlanner( logical.OneRowRelation() } + val logicalPlan = + if (rel.getExpressionsList.asScala.toSeq.exists( + _.getExprTypeCase == proto.Expression.ExprTypeCase.TYPED_AGGREGATE_EXPRESSION)) { + session.sessionState.executePlan(baseRel).analyzed + } else { + baseRel + } + val projection = rel.getExpressionsList.asScala.toSeq - .map(transformExpression(_, Some(baseRel))) + .map(transformExpression(_, Some(logicalPlan))) .map(toNamedExpression) - logical.Project(projectList = projection, child = baseRel) + logical.Project(projectList = projection, child = logicalPlan) } /** @@ -2241,7 +2254,7 @@ class SparkConnectPlanner( val keyColumn = TypedAggUtils.aggKeyColumn(ds.kEncoder, ds.groupingAttributes) val namedColumns = rel.getAggregateExpressionsList.asScala.toSeq - .map(expr => transformExpressionWithTypedReduceExpression(expr, input)) + .map(expr => transformExpressionWithTypedReduceExpression(expr, ds.analyzedData)) .map(toNamedExpression) logical.Aggregate(ds.groupingAttributes, keyColumn +: namedColumns, ds.analyzed) } @@ -2252,9 +2265,17 @@ class SparkConnectPlanner( } val input = transformRelation(rel.getInput) + val logicalPlan = + if (rel.getAggregateExpressionsList.asScala.toSeq.exists( + _.getExprTypeCase == proto.Expression.ExprTypeCase.TYPED_AGGREGATE_EXPRESSION)) { + session.sessionState.executePlan(input).analyzed + } else { + input + } + val groupingExprs = rel.getGroupingExpressionsList.asScala.toSeq.map(transformExpression) val aggExprs = rel.getAggregateExpressionsList.asScala.toSeq - .map(expr => transformExpressionWithTypedReduceExpression(expr, input)) + .map(expr => transformExpressionWithTypedReduceExpression(expr, logicalPlan)) val aliasedAgg = (groupingExprs ++ aggExprs).map(toNamedExpression) rel.getGroupType match { @@ -2262,19 +2283,19 @@ class SparkConnectPlanner( logical.Aggregate( groupingExpressions = groupingExprs, aggregateExpressions = aliasedAgg, - child = input) + child = logicalPlan) case proto.Aggregate.GroupType.GROUP_TYPE_ROLLUP => logical.Aggregate( groupingExpressions = Seq(Rollup(groupingExprs.map(Seq(_)))), aggregateExpressions = aliasedAgg, - child = input) + child = logicalPlan) case proto.Aggregate.GroupType.GROUP_TYPE_CUBE => logical.Aggregate( groupingExpressions = Seq(Cube(groupingExprs.map(Seq(_)))), aggregateExpressions = aliasedAgg, - child = input) + child = logicalPlan) case proto.Aggregate.GroupType.GROUP_TYPE_PIVOT => if (!rel.hasPivot) { @@ -2286,7 +2307,7 @@ class SparkConnectPlanner( rel.getPivot.getValuesList.asScala.toSeq.map(transformLiteral) } else { RelationalGroupedDataset - .collectPivotValues(Dataset.ofRows(session, input), Column(pivotExpr)) + .collectPivotValues(Dataset.ofRows(session, logicalPlan), Column(pivotExpr)) .map(expressions.Literal.apply) } logical.Pivot( @@ -2294,7 +2315,7 @@ class SparkConnectPlanner( pivotColumn = pivotExpr, pivotValues = valueExprs, aggregates = aggExprs, - child = input) + child = logicalPlan) case proto.Aggregate.GroupType.GROUP_TYPE_GROUPING_SETS => val groupingSetsExprs = rel.getGroupingSetsList.asScala.toSeq.map { getGroupingSets => @@ -2306,7 +2327,7 @@ class SparkConnectPlanner( groupingSets = groupingSetsExprs, userGivenGroupByExprs = groupingExprs)), aggregateExpressions = aliasedAgg, - child = input) + child = logicalPlan) case other => throw InvalidPlanInput(s"Unknown Group Type $other") } @@ -2444,11 +2465,27 @@ class SparkConnectPlanner( handleRemoveCachedRemoteRelationCommand(command.getRemoveCachedRemoteRelationCommand) case proto.Command.CommandTypeCase.MERGE_INTO_TABLE_COMMAND => handleMergeIntoTableCommand(command.getMergeIntoTableCommand) + case proto.Command.CommandTypeCase.ML_COMMAND => + handleMlCommand(command.getMlCommand, responseObserver) case _ => throw new UnsupportedOperationException(s"$command not supported.") } } + private def handleMlCommand( + command: proto.MlCommand, + responseObserver: StreamObserver[proto.ExecutePlanResponse]): Unit = { + val result = MLHandler.handleMlCommand(sessionHolder, command) + executeHolder.eventsManager.postFinished() + responseObserver.onNext( + proto.ExecutePlanResponse + .newBuilder() + .setSessionId(sessionId) + .setServerSideSessionId(sessionHolder.serverSessionId) + .setMlCommandResult(result) + .build()) + } + private def handleSqlCommand( command: SqlCommand, responseObserver: StreamObserver[ExecutePlanResponse]): Unit = { diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala index 36f3bcd1e6cd7..5b56b7079a897 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connect.common.InvalidPlanInput import org.apache.spark.sql.connect.config.Connect +import org.apache.spark.sql.connect.ml.MLCache import org.apache.spark.sql.connect.planner.PythonStreamingQueryListener import org.apache.spark.sql.connect.planner.StreamingForeachBatchHelper import org.apache.spark.sql.connect.service.ExecuteKey @@ -111,6 +112,9 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio private[spark] lazy val dataFrameCache: ConcurrentMap[String, DataFrame] = new ConcurrentHashMap() + // ML model cache + private[connect] lazy val mlCache = new MLCache() + // Mapping from id to StreamingQueryListener. Used for methods like removeListener() in // StreamingQueryManager. private lazy val listenerCache: ConcurrentMap[String, StreamingQueryListener] = @@ -322,6 +326,8 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio // remove all executions and no new executions will be added in the meanwhile. SparkConnectService.executionManager.removeAllExecutionsForSession(this.key) + mlCache.clear() + eventManager.postClosed() } diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ml/MLSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ml/MLSuite.scala new file mode 100644 index 0000000000000..bea7072b2034e --- /dev/null +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ml/MLSuite.scala @@ -0,0 +1,390 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connect.ml + +import java.io.File + +import org.apache.spark.SparkFunSuite +import org.apache.spark.connect.proto +import org.apache.spark.ml.classification.LogisticRegressionModel +import org.apache.spark.ml.linalg.{Vectors, VectorUDT} +import org.apache.spark.ml.param._ +import org.apache.spark.ml.util.Identifiable +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.UnsafeProjection +import org.apache.spark.sql.catalyst.types.DataTypeUtils +import org.apache.spark.sql.connect.SparkConnectTestUtils +import org.apache.spark.sql.connect.planner.SparkConnectPlanTest +import org.apache.spark.sql.types.{FloatType, Metadata, StructField, StructType} +import org.apache.spark.util.Utils + +trait FakeArrayParams extends Params { + final val arrayString: StringArrayParam = + new StringArrayParam(this, "arrayString", "array string") + + final def getArrayString: Array[String] = $(arrayString) + + final val arrayDouble: DoubleArrayParam = + new DoubleArrayParam(this, "arrayDouble", "array double") + + final def getArrayDouble: Array[Double] = $(arrayDouble) + + final val arrayInt: IntArrayParam = new IntArrayParam(this, "arrayInt", "array int") + + final def getArrayInt: Array[Int] = $(arrayInt) + + final val int: IntParam = new IntParam(this, "int", "int") + + final def getInt: Int = $(int) + + final val float: FloatParam = new FloatParam(this, "float", "float") + + final def getFloat: Float = $(float) + + final val boolean: BooleanParam = new BooleanParam(this, "boolean", "boolean") + + final def getBoolean: Boolean = $(boolean) + + final val double: DoubleParam = new DoubleParam(this, "double", "double") + + final def getDouble: Double = $(double) +} + +class FakedML(override val uid: String) extends FakeArrayParams { + def this() = this(Identifiable.randomUID("FakedML")) + + override def copy(extra: ParamMap): Params = this +} + +class MLSuite extends SparkFunSuite with SparkConnectPlanTest { + + def createLocalRelationProto: proto.Relation = { + val udt = new VectorUDT() + val rows = Seq( + InternalRow(1.0f, udt.serialize(Vectors.dense(Array(1.0, 2.0)))), + InternalRow(1.0f, udt.serialize(Vectors.dense(Array(2.0, -1.0)))), + InternalRow(0.0f, udt.serialize(Vectors.dense(Array(-3.0, -2.0)))), + InternalRow(0.0f, udt.serialize(Vectors.dense(Array(-1.0, -2.0))))) + + val schema = StructType( + Seq( + StructField("label", FloatType), + StructField("features", new VectorUDT(), false, Metadata.empty))) + + val inputRows = rows.map { row => + val proj = UnsafeProjection.create(schema) + proj(row).copy() + } + createLocalRelationProto(DataTypeUtils.toAttributes(schema), inputRows, "UTC", Some(schema)) + } + + test("reconcileParam") { + val fakedML = new FakedML + val params = proto.MlParams + .newBuilder() + .putParams( + "boolean", + proto.Param + .newBuilder() + .setLiteral(proto.Expression.Literal.newBuilder().setBoolean(true)) + .build()) + .putParams( + "double", + proto.Param + .newBuilder() + .setLiteral(proto.Expression.Literal.newBuilder().setDouble(1.0)) + .build()) + .putParams( + "int", + proto.Param + .newBuilder() + .setLiteral(proto.Expression.Literal.newBuilder().setInteger(10)) + .build()) + .putParams( + "float", + proto.Param + .newBuilder() + .setLiteral(proto.Expression.Literal.newBuilder().setFloat(10.0f)) + .build()) + .putParams( + "arrayString", + proto.Param + .newBuilder() + .setLiteral( + proto.Expression.Literal + .newBuilder() + .setArray( + proto.Expression.Literal.Array + .newBuilder() + .setElementType(proto.DataType + .newBuilder() + .setString(proto.DataType.String.getDefaultInstance) + .build()) + .addElements(proto.Expression.Literal.newBuilder().setString("hello")) + .addElements(proto.Expression.Literal.newBuilder().setString("world")) + .build()) + .build()) + .build()) + .putParams( + "arrayInt", + proto.Param + .newBuilder() + .setLiteral( + proto.Expression.Literal + .newBuilder() + .setArray( + proto.Expression.Literal.Array + .newBuilder() + .setElementType(proto.DataType + .newBuilder() + .setInteger(proto.DataType.Integer.getDefaultInstance) + .build()) + .addElements(proto.Expression.Literal.newBuilder().setInteger(1)) + .addElements(proto.Expression.Literal.newBuilder().setInteger(2)) + .build()) + .build()) + .build()) + .putParams( + "arrayDouble", + proto.Param + .newBuilder() + .setLiteral( + proto.Expression.Literal + .newBuilder() + .setArray( + proto.Expression.Literal.Array + .newBuilder() + .setElementType(proto.DataType + .newBuilder() + .setDouble(proto.DataType.Double.getDefaultInstance) + .build()) + .addElements(proto.Expression.Literal.newBuilder().setDouble(11.0)) + .addElements(proto.Expression.Literal.newBuilder().setDouble(12.0)) + .build()) + .build()) + .build()) + .build() + MLUtils.setInstanceParams(fakedML, params) + assert(fakedML.getInt === 10) + assert(fakedML.getFloat === 10.0) + assert(fakedML.getArrayInt === Array(1, 2)) + assert(fakedML.getArrayDouble === Array(11.0, 12.0)) + assert(fakedML.getArrayString === Array("hello", "world")) + assert(fakedML.getBoolean === true) + assert(fakedML.getDouble === 1.0) + } + + test("LogisticRegression works") { + val sessionHolder = SparkConnectTestUtils.createDummySessionHolder(spark) + + def verifyModel(modelId: String, hasSummary: Boolean = false): Unit = { + val model = sessionHolder.mlCache.get(modelId) + // Model is cached + assert(model != null) + assert(model.isInstanceOf[LogisticRegressionModel]) + val lrModel = model.asInstanceOf[LogisticRegressionModel] + assert(lrModel.getMaxIter === 2) + + // Fetch double attribute + val interceptCommand = proto.MlCommand + .newBuilder() + .setFetch( + proto.Fetch + .newBuilder() + .setObjRef(proto.ObjectRef.newBuilder().setId(modelId)) + .addMethods(proto.Fetch.Method.newBuilder().setMethod("intercept"))) + .build() + val interceptResult = MLHandler.handleMlCommand(sessionHolder, interceptCommand) + assert(interceptResult.getParam.getLiteral.getDouble === lrModel.intercept) + + // Fetch Vector attribute + val coefficientsCommand = proto.MlCommand + .newBuilder() + .setFetch( + proto.Fetch + .newBuilder() + .setObjRef(proto.ObjectRef.newBuilder().setId(modelId)) + .addMethods(proto.Fetch.Method.newBuilder().setMethod("coefficients"))) + .build() + val coefficientsResult = MLHandler.handleMlCommand(sessionHolder, coefficientsCommand) + val deserializedCoefficients = + MLUtils.deserializeVector(coefficientsResult.getParam.getVector) + assert(deserializedCoefficients === lrModel.coefficients) + + // Fetch Matrix attribute + val coefficientsMatrixCommand = proto.MlCommand + .newBuilder() + .setFetch( + proto.Fetch + .newBuilder() + .setObjRef(proto.ObjectRef.newBuilder().setId(modelId)) + .addMethods(proto.Fetch.Method.newBuilder().setMethod("coefficientMatrix"))) + .build() + val coefficientsMatrixResult = + MLHandler.handleMlCommand(sessionHolder, coefficientsMatrixCommand) + val deserializedCoefficientsMatrix = + MLUtils.deserializeMatrix(coefficientsMatrixResult.getParam.getMatrix) + assert(lrModel.coefficientMatrix === deserializedCoefficientsMatrix) + + // Predict with sparse vector + val sparseVector = Vectors.dense(Array(0.0, 2.0)).toSparse + val predictCommand = proto.MlCommand + .newBuilder() + .setFetch( + proto.Fetch + .newBuilder() + .setObjRef(proto.ObjectRef.newBuilder().setId(modelId)) + .addMethods( + proto.Fetch.Method + .newBuilder() + .setMethod("predict") + .addArgs(proto.Fetch.Method.Args + .newBuilder() + .setParam(Serializer.serializeParam(sparseVector))))) + .build() + val predictResult = MLHandler.handleMlCommand(sessionHolder, predictCommand) + val predictValue = predictResult.getParam.getLiteral.getDouble + assert(lrModel.predict(sparseVector) === predictValue) + + // The loaded model doesn't have summary + if (hasSummary) { + // Fetch summary attribute + val accuracyCommand = proto.MlCommand + .newBuilder() + .setFetch( + proto.Fetch + .newBuilder() + .setObjRef(proto.ObjectRef.newBuilder().setId(modelId)) + .addMethods(proto.Fetch.Method.newBuilder().setMethod("summary")) + .addMethods(proto.Fetch.Method.newBuilder().setMethod("accuracy"))) + .build() + val accuracyResult = MLHandler.handleMlCommand(sessionHolder, accuracyCommand) + assert(lrModel.summary.accuracy === accuracyResult.getParam.getLiteral.getDouble) + + val weightedFMeasureCommand = proto.MlCommand + .newBuilder() + .setFetch( + proto.Fetch + .newBuilder() + .setObjRef(proto.ObjectRef.newBuilder().setId(modelId)) + .addMethods(proto.Fetch.Method.newBuilder().setMethod("summary")) + .addMethods( + proto.Fetch.Method + .newBuilder() + .setMethod("weightedFMeasure") + .addArgs(proto.Fetch.Method.Args + .newBuilder() + .setParam(Serializer.serializeParam(2.5))))) + .build() + val weightedFMeasureResult = + MLHandler.handleMlCommand(sessionHolder, weightedFMeasureCommand) + assert( + lrModel.summary.weightedFMeasure(2.5) === + weightedFMeasureResult.getParam.getLiteral.getDouble) + } + } + + try { + val fitCommand = proto.MlCommand + .newBuilder() + .setFit( + proto.MlCommand.Fit + .newBuilder() + .setDataset(createLocalRelationProto) + .setEstimator( + proto.MlOperator + .newBuilder() + .setName("org.apache.spark.ml.classification.LogisticRegression") + .setUid("LogisticRegression") + .setType(proto.MlOperator.OperatorType.ESTIMATOR)) + .setParams( + proto.MlParams + .newBuilder() + .putParams( + "maxIter", + proto.Param + .newBuilder() + .setLiteral(proto.Expression.Literal + .newBuilder() + .setInteger(2)) + .build()))) + .build() + val fitResult = MLHandler.handleMlCommand(sessionHolder, fitCommand) + val modelId = fitResult.getOperatorInfo.getObjRef.getId + + verifyModel(modelId, true) + + // read/write + val tempDir = Utils.createTempDir(namePrefix = this.getClass.getName) + try { + val path = new File(tempDir, Identifiable.randomUID("LogisticRegression")).getPath + val writeCmd = proto.MlCommand + .newBuilder() + .setWrite( + proto.MlCommand.Write + .newBuilder() + .setPath(path) + .setObjRef(proto.ObjectRef.newBuilder().setId(modelId))) + .build() + MLHandler.handleMlCommand(sessionHolder, writeCmd) + + val readCmd = proto.MlCommand + .newBuilder() + .setRead( + proto.MlCommand.Read + .newBuilder() + .setOperator( + proto.MlOperator + .newBuilder() + .setName("org.apache.spark.ml.classification.LogisticRegressionModel") + .setType(proto.MlOperator.OperatorType.MODEL)) + .setPath(path)) + .build() + + val readResult = MLHandler.handleMlCommand(sessionHolder, readCmd) + verifyModel(readResult.getOperatorInfo.getObjRef.getId) + + } finally { + Utils.deleteRecursively(tempDir) + } + + } finally { + sessionHolder.mlCache.clear() + } + } + + test("Exception: Unsupported ML operator") { + intercept[MlUnsupportedException] { + val sessionHolder = SparkConnectTestUtils.createDummySessionHolder(spark) + val command = proto.MlCommand + .newBuilder() + .setFit( + proto.MlCommand.Fit + .newBuilder() + .setDataset(createLocalRelationProto) + .setEstimator( + proto.MlOperator + .newBuilder() + .setName("org.apache.spark.ml.NotExistingML") + .setUid("FakedUid") + .setType(proto.MlOperator.OperatorType.ESTIMATOR))) + .build() + MLHandler.handleMlCommand(sessionHolder, command) + } + } +} diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala index aaeb5d9fe509a..55c492f511049 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala @@ -89,7 +89,8 @@ trait SparkConnectPlanTest extends SharedSparkSession { def createLocalRelationProto( attrs: Seq[AttributeReference], data: Seq[InternalRow], - timeZoneId: String = "UTC"): proto.Relation = { + timeZoneId: String = "UTC", + schema: Option[StructType] = None): proto.Relation = { val localRelationBuilder = proto.LocalRelation.newBuilder() val bytes = ArrowConverters @@ -103,6 +104,7 @@ trait SparkConnectPlanTest extends SharedSparkSession { .next() localRelationBuilder.setData(ByteString.copyFrom(bytes)) + schema.foreach(s => localRelationBuilder.setSchema(s.json)) proto.Relation.newBuilder().setLocalRelation(localRelationBuilder.build()).build() } } @@ -504,26 +506,27 @@ class SparkConnectPlannerSuite extends SparkFunSuite with SparkConnectPlanTest { } test("Test duplicated names in WithColumns") { - intercept[AnalysisException] { - transform( - proto.Relation - .newBuilder() - .setWithColumns( - proto.WithColumns - .newBuilder() - .setInput(readRel) - .addAliases(proto.Expression.Alias + val logical = transform( + proto.Relation + .newBuilder() + .setWithColumns( + proto.WithColumns + .newBuilder() + .setInput(readRel) + .addAliases( + proto.Expression.Alias .newBuilder() .addName("test") .setExpr(proto.Expression.newBuilder .setLiteral(proto.Expression.Literal.newBuilder.setInteger(32)))) - .addAliases(proto.Expression.Alias - .newBuilder() - .addName("test") - .setExpr(proto.Expression.newBuilder - .setLiteral(proto.Expression.Literal.newBuilder.setInteger(32))))) - .build()) - } + .addAliases(proto.Expression.Alias + .newBuilder() + .addName("test") + .setExpr(proto.Expression.newBuilder + .setLiteral(proto.Expression.Literal.newBuilder.setInteger(32))))) + .build()) + + intercept[AnalysisException](Dataset.ofRows(spark, logical)) } test("Test multi nameparts for column names in WithColumns") { diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java index 0464fe8159898..42d39457330c1 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java @@ -82,6 +82,8 @@ public InternalRow copy() { row.setInt(i, getInt(i)); } else if (dt instanceof TimestampType) { row.setLong(i, getLong(i)); + } else if (dt instanceof TimestampNTZType) { + row.setLong(i, getLong(i)); } else if (dt instanceof StructType) { row.update(i, getStruct(i, ((StructType) dt).fields().length).copy()); } else if (dt instanceof ArrayType) { @@ -191,6 +193,8 @@ public Object get(int ordinal, DataType dataType) { return getInt(ordinal); } else if (dataType instanceof TimestampType) { return getLong(ordinal); + } else if (dataType instanceof TimestampNTZType) { + return getLong(ordinal); } else if (dataType instanceof ArrayType) { return getArray(ordinal); } else if (dataType instanceof StructType structType) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index e4e782a50e3d4..e41521cba533a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1275,29 +1275,14 @@ class Dataset[T] private[sql]( require(colNames.size == cols.size, s"The size of column names: ${colNames.size} isn't equal to " + s"the size of columns: ${cols.size}") - SchemaUtils.checkColumnNameDuplication( - colNames, - sparkSession.sessionState.conf.caseSensitiveAnalysis) - - val resolver = sparkSession.sessionState.analyzer.resolver - val output = queryExecution.analyzed.output - - val columnSeq = colNames.zip(cols) - - val replacedAndExistingColumns = output.map { field => - columnSeq.find { case (colName, _) => - resolver(field.name, colName) - } match { - case Some((colName: String, col: Column)) => col.as(colName) - case _ => Column(field) - } + withPlan { + Project( + Seq( + UnresolvedStarWithColumns( + colNames = colNames, + exprs = cols.map(_.expr))), + logicalPlan) } - - val newColumns = columnSeq.filter { case (colName, col) => - !output.exists(f => resolver(f.name, colName)) - }.map { case (colName, col) => col.as(colName) } - - select(replacedAndExistingColumns ++ newColumns : _*) } /** @inheritdoc */ @@ -1324,26 +1309,13 @@ class Dataset[T] private[sql]( require(colNames.size == newColNames.size, s"The size of existing column names: ${colNames.size} isn't equal to " + s"the size of new column names: ${newColNames.size}") - - val resolver = sparkSession.sessionState.analyzer.resolver - val output: Seq[NamedExpression] = queryExecution.analyzed.output - var shouldRename = false - - val projectList = colNames.zip(newColNames).foldLeft(output) { - case (attrs, (existingName, newName)) => - attrs.map(attr => - if (resolver(attr.name, existingName)) { - shouldRename = true - Alias(attr, newName)() - } else { - attr - } - ) - } - if (shouldRename) { - withPlan(Project(projectList, logicalPlan)) - } else { - toDF() + withPlan { + Project( + Seq( + UnresolvedStarWithColumnsRenames( + existingNames = colNames, + newNames = newColNames)), + logicalPlan) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index ae06e82335b12..2deccb845fea2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -291,7 +291,8 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with newMap } catch { - case e: SparkException if e.getCondition.contains("CANNOT_LOAD_STATE_STORE") => + case e: SparkException + if Option(e.getCondition).exists(_.contains("CANNOT_LOAD_STATE_STORE")) => throw e case e: OutOfMemoryError => throw QueryExecutionErrors.notEnoughMemoryToLoadStore( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index 60652367f3351..9fc48a60d7c6a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -439,7 +439,8 @@ private[sql] class RocksDBStateStoreProvider new RocksDBStateStore(version) } catch { - case e: SparkException if e.getCondition.contains("CANNOT_LOAD_STATE_STORE") => + case e: SparkException + if Option(e.getCondition).exists(_.contains("CANNOT_LOAD_STATE_STORE")) => throw e case e: OutOfMemoryError => throw QueryExecutionErrors.notEnoughMemoryToLoadStore( @@ -462,7 +463,8 @@ private[sql] class RocksDBStateStoreProvider new RocksDBStateStore(version) } catch { - case e: SparkException if e.getCondition.contains("CANNOT_LOAD_STATE_STORE") => + case e: SparkException + if Option(e.getCondition).exists(_.contains("CANNOT_LOAD_STATE_STORE")) => throw e case e: OutOfMemoryError => throw QueryExecutionErrors.notEnoughMemoryToLoadStore( diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out new file mode 100644 index 0000000000000..b3c10e929f297 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-udf.sql.out @@ -0,0 +1,575 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE FUNCTION foo1a0() RETURNS INT RETURN 1 +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo1a0`" + } +} + + +-- !query +SELECT foo1a0() +-- !query analysis +Project [spark_catalog.default.foo1a0() AS spark_catalog.default.foo1a0()#x] ++- Project + +- OneRowRelation + + +-- !query +SELECT foo1a0(1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "0", + "functionName" : "`spark_catalog`.`default`.`foo1a0`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "foo1a0(1)" + } ] +} + + +-- !query +CREATE FUNCTION foo1a1(a INT) RETURNS INT RETURN 1 +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo1a1`" + } +} + + +-- !query +SELECT foo1a1(1) +-- !query analysis +Project [spark_catalog.default.foo1a1(a#x) AS spark_catalog.default.foo1a1(1)#x] ++- Project [cast(1 as int) AS a#x] + +- OneRowRelation + + +-- !query +SELECT foo1a1(1, 2) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`spark_catalog`.`default`.`foo1a1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "foo1a1(1, 2)" + } ] +} + + +-- !query +CREATE FUNCTION foo1a2(a INT, b INT, c INT, d INT) RETURNS INT RETURN 1 +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo1a2`" + } +} + + +-- !query +SELECT foo1a2(1, 2, 3, 4) +-- !query analysis +Project [spark_catalog.default.foo1a2(a#x, b#x, c#x, d#x) AS spark_catalog.default.foo1a2(1, 2, 3, 4)#x] ++- Project [cast(1 as int) AS a#x, cast(2 as int) AS b#x, cast(3 as int) AS c#x, cast(4 as int) AS d#x] + +- OneRowRelation + + +-- !query +CREATE FUNCTION foo2_1a(a INT) RETURNS INT RETURN a +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_1a`" + } +} + + +-- !query +SELECT foo2_1a(5) +-- !query analysis +Project [spark_catalog.default.foo2_1a(a#x) AS spark_catalog.default.foo2_1a(5)#x] ++- Project [cast(5 as int) AS a#x] + +- OneRowRelation + + +-- !query +CREATE FUNCTION foo2_1b(a INT, b INT) RETURNS INT RETURN a + b +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_1b`" + } +} + + +-- !query +SELECT foo2_1b(5, 6) +-- !query analysis +Project [spark_catalog.default.foo2_1b(a#x, b#x) AS spark_catalog.default.foo2_1b(5, 6)#x] ++- Project [cast(5 as int) AS a#x, cast(6 as int) AS b#x] + +- OneRowRelation + + +-- !query +CREATE FUNCTION foo2_1c(a INT, b INT) RETURNS INT RETURN 10 * (a + b) + 100 * (a -b) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_1c`" + } +} + + +-- !query +SELECT foo2_1c(5, 6) +-- !query analysis +Project [spark_catalog.default.foo2_1c(a#x, b#x) AS spark_catalog.default.foo2_1c(5, 6)#x] ++- Project [cast(5 as int) AS a#x, cast(6 as int) AS b#x] + +- OneRowRelation + + +-- !query +CREATE FUNCTION foo2_1d(a INT, b INT) RETURNS INT RETURN ABS(a) - LENGTH(CAST(b AS VARCHAR(10))) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_1d`" + } +} + + +-- !query +SELECT foo2_1d(-5, 6) +-- !query analysis +Project [spark_catalog.default.foo2_1d(a#x, b#x) AS spark_catalog.default.foo2_1d(-5, 6)#x] ++- Project [cast(-5 as int) AS a#x, cast(6 as int) AS b#x] + +- OneRowRelation + + +-- !query +CREATE FUNCTION foo2_2a(a INT) RETURNS INT RETURN SELECT a +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_2a`" + } +} + + +-- !query +SELECT foo2_2a(5) +-- !query analysis +Project [spark_catalog.default.foo2_2a(a#x) AS spark_catalog.default.foo2_2a(5)#x] ++- Project [cast(5 as int) AS a#x] + +- OneRowRelation + + +-- !query +CREATE FUNCTION foo2_2b(a INT) RETURNS INT RETURN 1 + (SELECT a) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_2b`" + } +} + + +-- !query +SELECT foo2_2b(5) +-- !query analysis +Project [spark_catalog.default.foo2_2b(a#x) AS spark_catalog.default.foo2_2b(5)#x] +: +- Project [outer(a#x)] +: +- OneRowRelation ++- Project [cast(5 as int) AS a#x] + +- OneRowRelation + + +-- !query +CREATE FUNCTION foo2_2c(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT a)) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 21, + "stopIndex" : 21, + "fragment" : "a" + } ] +} + + +-- !query +CREATE FUNCTION foo2_2d(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT (SELECT (SELECT a)))) +-- !query analysis +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 37, + "stopIndex" : 37, + "fragment" : "a" + } ] +} + + +-- !query +CREATE FUNCTION foo2_2e(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) WHERE c1 = 2 +UNION ALL +SELECT a + 1 FROM (VALUES 1) AS V(c1) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_2e`" + } +} + + +-- !query +CREATE FUNCTION foo2_2f(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) +EXCEPT +SELECT a + 1 FROM (VALUES 1) AS V(a) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_2f`" + } +} + + +-- !query +CREATE FUNCTION foo2_2g(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) +INTERSECT +SELECT a FROM (VALUES 1) AS V(a) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_2g`" + } +} + + +-- !query +DROP TABLE IF EXISTS t1 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1 + + +-- !query +DROP TABLE IF EXISTS t2 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t2 + + +-- !query +DROP TABLE IF EXISTS ts +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.ts + + +-- !query +DROP TABLE IF EXISTS tm +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tm + + +-- !query +DROP TABLE IF EXISTS ta +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.ta + + +-- !query +DROP TABLE IF EXISTS V1 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.V1 + + +-- !query +DROP TABLE IF EXISTS V2 +-- !query analysis +DropTable true, false ++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.V2 + + +-- !query +DROP VIEW IF EXISTS t1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t1`, true, true, false + + +-- !query +DROP VIEW IF EXISTS t2 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`t2`, true, true, false + + +-- !query +DROP VIEW IF EXISTS ts +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`ts`, true, true, false + + +-- !query +DROP VIEW IF EXISTS tm +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`tm`, true, true, false + + +-- !query +DROP VIEW IF EXISTS ta +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`ta`, true, true, false + + +-- !query +DROP VIEW IF EXISTS V1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`V1`, true, true, false + + +-- !query +DROP VIEW IF EXISTS V2 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`V2`, true, true, false + + +-- !query +CREATE FUNCTION foo2_3(a INT, b INT) RETURNS INT RETURN a + b +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_3`" + } +} + + +-- !query +CREATE VIEW V1(c1, c2) AS VALUES (1, 2), (3, 4), (5, 6) +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`V1`, [(c1,None), (c2,None)], VALUES (1, 2), (3, 4), (5, 6), false, false, PersistedView, COMPENSATION, true + +- LocalRelation [col1#x, col2#x] + + +-- !query +CREATE VIEW V2(c1, c2) AS VALUES (-1, -2), (-3, -4), (-5, -6) +-- !query analysis +CreateViewCommand `spark_catalog`.`default`.`V2`, [(c1,None), (c2,None)], VALUES (-1, -2), (-3, -4), (-5, -6), false, false, PersistedView, COMPENSATION, true + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT foo2_3(c1, c2), foo2_3(c2, 1), foo2_3(c1, c2) - foo2_3(c2, c1 - 1) FROM V1 ORDER BY 1, 2, 3 +-- !query analysis +Sort [spark_catalog.default.foo2_3(c1, c2)#x ASC NULLS FIRST, spark_catalog.default.foo2_3(c2, 1)#x ASC NULLS FIRST, (spark_catalog.default.foo2_3(c1, c2) - spark_catalog.default.foo2_3(c2, (c1 - 1)))#x ASC NULLS FIRST], true ++- Project [spark_catalog.default.foo2_3(a#x, b#x) AS spark_catalog.default.foo2_3(c1, c2)#x, spark_catalog.default.foo2_3(a#x, b#x) AS spark_catalog.default.foo2_3(c2, 1)#x, (spark_catalog.default.foo2_3(a#x, b#x) - spark_catalog.default.foo2_3(a#x, b#x)) AS (spark_catalog.default.foo2_3(c1, c2) - spark_catalog.default.foo2_3(c2, (c1 - 1)))#x] + +- Project [c1#x, c2#x, cast(c1#x as int) AS a#x, cast(c2#x as int) AS b#x, cast(c2#x as int) AS a#x, cast(1 as int) AS b#x, cast(c1#x as int) AS a#x, cast(c2#x as int) AS b#x, cast(c2#x as int) AS a#x, cast((c1#x - 1) as int) AS b#x] + +- SubqueryAlias spark_catalog.default.v1 + +- View (`spark_catalog`.`default`.`v1`, [c1#x, c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT * FROM V1 WHERE foo2_3(c1, 0) = c1 AND foo2_3(c1, c2) < 8 +-- !query analysis +Project [c1#x, c2#x] ++- Project [c1#x, c2#x] + +- Filter ((spark_catalog.default.foo2_3(a#x, b#x) = c1#x) AND (spark_catalog.default.foo2_3(a#x, b#x) < 8)) + +- Project [c1#x, c2#x, cast(c1#x as int) AS a#x, cast(0 as int) AS b#x, cast(c1#x as int) AS a#x, cast(c2#x as int) AS b#x] + +- SubqueryAlias spark_catalog.default.v1 + +- View (`spark_catalog`.`default`.`v1`, [c1#x, c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT foo2_3(SUM(c1), SUM(c2)), SUM(c1) + SUM(c2), SUM(foo2_3(c1, c2) + foo2_3(c2, c1) - foo2_3(c2, c1)) +FROM V1 +-- !query analysis +Project [spark_catalog.default.foo2_3(a#x, b#x) AS spark_catalog.default.foo2_3(sum(c1), sum(c2))#x, (sum(c1) + sum(c2))#xL, sum(((spark_catalog.default.foo2_3(c1, c2) + spark_catalog.default.foo2_3(c2, c1)) - spark_catalog.default.foo2_3(c2, c1)))#xL] ++- Project [sum(c1)#xL, sum(c2)#xL, (sum(c1) + sum(c2))#xL, sum(((spark_catalog.default.foo2_3(c1, c2) + spark_catalog.default.foo2_3(c2, c1)) - spark_catalog.default.foo2_3(c2, c1)))#xL, cast(sum(c1)#xL as int) AS a#x, cast(sum(c2)#xL as int) AS b#x] + +- Aggregate [sum(c1#x) AS sum(c1)#xL, sum(c2#x) AS sum(c2)#xL, (sum(c1#x) + sum(c2#x)) AS (sum(c1) + sum(c2))#xL, sum(((spark_catalog.default.foo2_3(a#x, b#x) + spark_catalog.default.foo2_3(a#x, b#x)) - spark_catalog.default.foo2_3(a#x, b#x))) AS sum(((spark_catalog.default.foo2_3(c1, c2) + spark_catalog.default.foo2_3(c2, c1)) - spark_catalog.default.foo2_3(c2, c1)))#xL] + +- Project [c1#x, c2#x, cast(c1#x as int) AS a#x, cast(c2#x as int) AS b#x, cast(c2#x as int) AS a#x, cast(c1#x as int) AS b#x, cast(c2#x as int) AS a#x, cast(c1#x as int) AS b#x] + +- SubqueryAlias spark_catalog.default.v1 + +- View (`spark_catalog`.`default`.`v1`, [c1#x, c2#x]) + +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +CREATE FUNCTION foo2_4a(a ARRAY) RETURNS STRING RETURN +SELECT array_sort(a, (i, j) -> rank[i] - rank[j])[0] FROM (SELECT MAP('a', 1, 'b', 2) rank) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_4a`" + } +} + + +-- !query +SELECT foo2_4a(ARRAY('a', 'b')) +-- !query analysis +Project [spark_catalog.default.foo2_4a(a#x) AS spark_catalog.default.foo2_4a(array(a, b))#x] +: +- Project [array_sort(outer(a#x), lambdafunction((rank#x[lambda i#x] - rank#x[lambda j#x]), lambda i#x, lambda j#x, false), false)[0] AS array_sort(outer(foo2_4a.a), lambdafunction((rank[namedlambdavariable()] - rank[namedlambdavariable()]), namedlambdavariable(), namedlambdavariable()))[0]#x] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Project [map(a, 1, b, 2) AS rank#x] +: +- OneRowRelation ++- Project [cast(array(a, b) as array) AS a#x] + +- OneRowRelation + + +-- !query +CREATE FUNCTION foo2_4b(m MAP, k STRING) RETURNS STRING RETURN +SELECT v || ' ' || v FROM (SELECT upper(m[k]) AS v) +-- !query analysis +org.apache.spark.sql.catalyst.analysis.FunctionAlreadyExistsException +{ + "errorClass" : "ROUTINE_ALREADY_EXISTS", + "sqlState" : "42723", + "messageParameters" : { + "existingRoutineType" : "routine", + "newRoutineType" : "routine", + "routineName" : "`default`.`foo2_4b`" + } +} + + +-- !query +SELECT foo2_4b(map('a', 'hello', 'b', 'world'), 'a') +-- !query analysis +Project [spark_catalog.default.foo2_4b(m#x, k#x) AS spark_catalog.default.foo2_4b(map(a, hello, b, world), a)#x] +: +- Project [concat(concat(v#x, ), v#x) AS concat(concat(v, ), v)#x] +: +- SubqueryAlias __auto_generated_subquery_name +: +- Project [upper(outer(m#x)[outer(k#x)]) AS v#x] +: +- OneRowRelation ++- Project [cast(map(a, hello, b, world) as map) AS m#x, cast(a as string) AS k#x] + +- OneRowRelation + + +-- !query +DROP VIEW V2 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`V2`, false, true, false + + +-- !query +DROP VIEW V1 +-- !query analysis +DropTableCommand `spark_catalog`.`default`.`V1`, false, true, false diff --git a/sql/core/src/test/resources/sql-tests/inputs/sql-udf.sql b/sql/core/src/test/resources/sql-tests/inputs/sql-udf.sql new file mode 100644 index 0000000000000..34cb41d726766 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/sql-udf.sql @@ -0,0 +1,122 @@ +-- test cases for SQL User Defined Functions + +-- 1. CREATE FUNCTION +-- 1.1 Parameter +-- 1.1.a A scalar function with various numbers of parameter +-- Expect success +CREATE FUNCTION foo1a0() RETURNS INT RETURN 1; +-- Expect: 1 +SELECT foo1a0(); +-- Expect failure +SELECT foo1a0(1); + +CREATE FUNCTION foo1a1(a INT) RETURNS INT RETURN 1; +-- Expect: 1 +SELECT foo1a1(1); +-- Expect failure +SELECT foo1a1(1, 2); + +CREATE FUNCTION foo1a2(a INT, b INT, c INT, d INT) RETURNS INT RETURN 1; +-- Expect: 1 +SELECT foo1a2(1, 2, 3, 4); + +------------------------------- +-- 2. Scalar SQL UDF +-- 2.1 deterministic simple expressions +CREATE FUNCTION foo2_1a(a INT) RETURNS INT RETURN a; +SELECT foo2_1a(5); + +CREATE FUNCTION foo2_1b(a INT, b INT) RETURNS INT RETURN a + b; +SELECT foo2_1b(5, 6); + +CREATE FUNCTION foo2_1c(a INT, b INT) RETURNS INT RETURN 10 * (a + b) + 100 * (a -b); +SELECT foo2_1c(5, 6); + +CREATE FUNCTION foo2_1d(a INT, b INT) RETURNS INT RETURN ABS(a) - LENGTH(CAST(b AS VARCHAR(10))); +SELECT foo2_1d(-5, 6); + +-- 2.2 deterministic complex expression with subqueries +-- 2.2.1 Nested Scalar subqueries +CREATE FUNCTION foo2_2a(a INT) RETURNS INT RETURN SELECT a; +SELECT foo2_2a(5); + +CREATE FUNCTION foo2_2b(a INT) RETURNS INT RETURN 1 + (SELECT a); +SELECT foo2_2b(5); + +-- Expect error: deep correlation is not yet supported +CREATE FUNCTION foo2_2c(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT a)); +-- SELECT foo2_2c(5); + +-- Expect error: deep correlation is not yet supported +CREATE FUNCTION foo2_2d(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT (SELECT (SELECT a)))); +-- SELECT foo2_2d(5); + +-- 2.2.2 Set operations +-- Expect error: correlated scalar subquery must be aggregated. +CREATE FUNCTION foo2_2e(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) WHERE c1 = 2 +UNION ALL +SELECT a + 1 FROM (VALUES 1) AS V(c1); +-- SELECT foo2_2e(5); + +-- Expect error: correlated scalar subquery must be aggregated. +CREATE FUNCTION foo2_2f(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) +EXCEPT +SELECT a + 1 FROM (VALUES 1) AS V(a); +-- SELECT foo2_2f(5); + +-- Expect error: correlated scalar subquery must be aggregated. +CREATE FUNCTION foo2_2g(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) +INTERSECT +SELECT a FROM (VALUES 1) AS V(a); +-- SELECT foo2_2g(5); + +-- Prepare by dropping views or tables if they already exist. +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; +DROP TABLE IF EXISTS ts; +DROP TABLE IF EXISTS tm; +DROP TABLE IF EXISTS ta; +DROP TABLE IF EXISTS V1; +DROP TABLE IF EXISTS V2; +DROP VIEW IF EXISTS t1; +DROP VIEW IF EXISTS t2; +DROP VIEW IF EXISTS ts; +DROP VIEW IF EXISTS tm; +DROP VIEW IF EXISTS ta; +DROP VIEW IF EXISTS V1; +DROP VIEW IF EXISTS V2; + +-- 2.3 Calling Scalar UDF from various places +CREATE FUNCTION foo2_3(a INT, b INT) RETURNS INT RETURN a + b; +CREATE VIEW V1(c1, c2) AS VALUES (1, 2), (3, 4), (5, 6); +CREATE VIEW V2(c1, c2) AS VALUES (-1, -2), (-3, -4), (-5, -6); + +-- 2.3.1 Multiple times in the select list +SELECT foo2_3(c1, c2), foo2_3(c2, 1), foo2_3(c1, c2) - foo2_3(c2, c1 - 1) FROM V1 ORDER BY 1, 2, 3; + +-- 2.3.2 In the WHERE clause +SELECT * FROM V1 WHERE foo2_3(c1, 0) = c1 AND foo2_3(c1, c2) < 8; + +-- 2.3.3 Different places around an aggregate +SELECT foo2_3(SUM(c1), SUM(c2)), SUM(c1) + SUM(c2), SUM(foo2_3(c1, c2) + foo2_3(c2, c1) - foo2_3(c2, c1)) +FROM V1; + +-- 2.4 Scalar UDF with complex one row relation subquery +-- 2.4.1 higher order functions +CREATE FUNCTION foo2_4a(a ARRAY) RETURNS STRING RETURN +SELECT array_sort(a, (i, j) -> rank[i] - rank[j])[0] FROM (SELECT MAP('a', 1, 'b', 2) rank); + +SELECT foo2_4a(ARRAY('a', 'b')); + +-- 2.4.2 built-in functions +CREATE FUNCTION foo2_4b(m MAP, k STRING) RETURNS STRING RETURN +SELECT v || ' ' || v FROM (SELECT upper(m[k]) AS v); + +SELECT foo2_4b(map('a', 'hello', 'b', 'world'), 'a'); + +-- Clean up +DROP VIEW V2; +DROP VIEW V1; diff --git a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out index af1bb75aef88d..93c1444777594 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out @@ -128,6 +128,7 @@ View Schema Mode: BINDING Schema: root |-- e: integer (nullable = true) + showdb show_t1 false Catalog: spark_catalog Database: showdb Table: show_t1 @@ -145,6 +146,7 @@ Schema: root |-- c: string (nullable = true) |-- d: string (nullable = true) + showdb show_t2 false Catalog: spark_catalog Database: showdb Table: show_t2 diff --git a/sql/core/src/test/resources/sql-tests/results/sql-udf.sql.out b/sql/core/src/test/resources/sql-tests/results/sql-udf.sql.out new file mode 100644 index 0000000000000..9f7af7c644871 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/sql-udf.sql.out @@ -0,0 +1,484 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +CREATE FUNCTION foo1a0() RETURNS INT RETURN 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1a0() +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT foo1a0(1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "1", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "0", + "functionName" : "`spark_catalog`.`default`.`foo1a0`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 16, + "fragment" : "foo1a0(1)" + } ] +} + + +-- !query +CREATE FUNCTION foo1a1(a INT) RETURNS INT RETURN 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1a1(1) +-- !query schema +struct +-- !query output +1 + + +-- !query +SELECT foo1a1(1, 2) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "WRONG_NUM_ARGS.WITHOUT_SUGGESTION", + "sqlState" : "42605", + "messageParameters" : { + "actualNum" : "2", + "docroot" : "https://spark.apache.org/docs/latest", + "expectedNum" : "1", + "functionName" : "`spark_catalog`.`default`.`foo1a1`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 8, + "stopIndex" : 19, + "fragment" : "foo1a1(1, 2)" + } ] +} + + +-- !query +CREATE FUNCTION foo1a2(a INT, b INT, c INT, d INT) RETURNS INT RETURN 1 +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo1a2(1, 2, 3, 4) +-- !query schema +struct +-- !query output +1 + + +-- !query +CREATE FUNCTION foo2_1a(a INT) RETURNS INT RETURN a +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_1a(5) +-- !query schema +struct +-- !query output +5 + + +-- !query +CREATE FUNCTION foo2_1b(a INT, b INT) RETURNS INT RETURN a + b +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_1b(5, 6) +-- !query schema +struct +-- !query output +11 + + +-- !query +CREATE FUNCTION foo2_1c(a INT, b INT) RETURNS INT RETURN 10 * (a + b) + 100 * (a -b) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_1c(5, 6) +-- !query schema +struct +-- !query output +10 + + +-- !query +CREATE FUNCTION foo2_1d(a INT, b INT) RETURNS INT RETURN ABS(a) - LENGTH(CAST(b AS VARCHAR(10))) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_1d(-5, 6) +-- !query schema +struct +-- !query output +4 + + +-- !query +CREATE FUNCTION foo2_2a(a INT) RETURNS INT RETURN SELECT a +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_2a(5) +-- !query schema +struct +-- !query output +5 + + +-- !query +CREATE FUNCTION foo2_2b(a INT) RETURNS INT RETURN 1 + (SELECT a) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_2b(5) +-- !query schema +struct +-- !query output +6 + + +-- !query +CREATE FUNCTION foo2_2c(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT a)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 21, + "stopIndex" : 21, + "fragment" : "a" + } ] +} + + +-- !query +CREATE FUNCTION foo2_2d(a INT) RETURNS INT RETURN 1 + (SELECT (SELECT (SELECT (SELECT a)))) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.ExtendedAnalysisException +{ + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", + "messageParameters" : { + "objectName" : "`a`" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 37, + "stopIndex" : 37, + "fragment" : "a" + } ] +} + + +-- !query +CREATE FUNCTION foo2_2e(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) WHERE c1 = 2 +UNION ALL +SELECT a + 1 FROM (VALUES 1) AS V(c1) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo2_2f(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) +EXCEPT +SELECT a + 1 FROM (VALUES 1) AS V(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo2_2g(a INT) RETURNS INT RETURN +SELECT a FROM (VALUES 1) AS V(c1) +INTERSECT +SELECT a FROM (VALUES 1) AS V(a) +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS ts +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS tm +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS ta +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS V1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP TABLE IF EXISTS V2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS t1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS t2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS ts +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS tm +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS ta +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS V1 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW IF EXISTS V2 +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE FUNCTION foo2_3(a INT, b INT) RETURNS INT RETURN a + b +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW V1(c1, c2) AS VALUES (1, 2), (3, 4), (5, 6) +-- !query schema +struct<> +-- !query output + + + +-- !query +CREATE VIEW V2(c1, c2) AS VALUES (-1, -2), (-3, -4), (-5, -6) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_3(c1, c2), foo2_3(c2, 1), foo2_3(c1, c2) - foo2_3(c2, c1 - 1) FROM V1 ORDER BY 1, 2, 3 +-- !query schema +struct +-- !query output +3 3 1 +7 5 1 +11 7 1 + + +-- !query +SELECT * FROM V1 WHERE foo2_3(c1, 0) = c1 AND foo2_3(c1, c2) < 8 +-- !query schema +struct +-- !query output +1 2 +3 4 + + +-- !query +SELECT foo2_3(SUM(c1), SUM(c2)), SUM(c1) + SUM(c2), SUM(foo2_3(c1, c2) + foo2_3(c2, c1) - foo2_3(c2, c1)) +FROM V1 +-- !query schema +struct +-- !query output +21 21 21 + + +-- !query +CREATE FUNCTION foo2_4a(a ARRAY) RETURNS STRING RETURN +SELECT array_sort(a, (i, j) -> rank[i] - rank[j])[0] FROM (SELECT MAP('a', 1, 'b', 2) rank) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_4a(ARRAY('a', 'b')) +-- !query schema +struct +-- !query output +a + + +-- !query +CREATE FUNCTION foo2_4b(m MAP, k STRING) RETURNS STRING RETURN +SELECT v || ' ' || v FROM (SELECT upper(m[k]) AS v) +-- !query schema +struct<> +-- !query output + + + +-- !query +SELECT foo2_4b(map('a', 'hello', 'b', 'world'), 'a') +-- !query schema +struct +-- !query output +HELLO HELLO + + +-- !query +DROP VIEW V2 +-- !query schema +struct<> +-- !query output + + + +-- !query +DROP VIEW V1 +-- !query schema +struct<> +-- !query output + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala index 579756038cb52..47ebd387e89a3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala @@ -761,7 +761,8 @@ trait CharVarcharTestSuite extends QueryTest with SQLTestUtils { test("implicitly cast char/varchar into atomics") { Seq("char", "varchar").foreach { typ => - withSQLConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key -> "true") { + withSQLConf(SQLConf.PRESERVE_CHAR_VARCHAR_TYPE_INFO.key -> "true", + SQLConf.ANSI_ENABLED.key -> "true") { checkAnswer(sql( s""" |SELECT diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala index fdfb909d9ba73..621d468454d40 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala @@ -777,9 +777,51 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { val t1 = table1() checkAnswer( - t1.withColumn("scalar", spark.range(1).select($"c1".outer() + $"c2".outer()).scalar()), - t1.withColumn("scalar", $"c1" + $"c2") - ) + t1.withColumn( + "scalar", + spark + .range(1) + .select($"c1".outer() + $"c2".outer()) + .scalar()), + t1.select($"*", ($"c1" + $"c2").as("scalar"))) + + checkAnswer( + t1.withColumn( + "scalar", + spark + .range(1) + .withColumn("c1", $"c1".outer()) + .select($"c1" + $"c2".outer()) + .scalar()), + t1.select($"*", ($"c1" + $"c2").as("scalar"))) + + checkAnswer( + t1.withColumn( + "scalar", + spark + .range(1) + .select($"c1".outer().as("c1")) + .withColumn("c2", $"c2".outer()) + .select($"c1" + $"c2") + .scalar()), + t1.select($"*", ($"c1" + $"c2").as("scalar"))) + } + } + + test("subquery in withColumnsRenamed") { + withView("t1") { + val t1 = table1() + + checkAnswer( + t1.withColumn( + "scalar", + spark + .range(1) + .select($"c1".outer().as("c1"), $"c2".outer().as("c2")) + .withColumnsRenamed(Map("c1" -> "x", "c2" -> "y")) + .select($"x" + $"y") + .scalar()), + t1.select($"*", ($"c1".as("x") + $"c2".as("y")).as("scalar"))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 317a88edf8e95..71d55b007aa17 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -430,21 +430,23 @@ class DataFrameSuite extends QueryTest test("repartition by MapType") { Seq("int", "long", "float", "double", "decimal(10, 2)", "string", "varchar(6)").foreach { dt => - val df = spark.range(20) - .withColumn("c1", - when(col("id") % 3 === 1, typedLit(Map(1 -> 1))) - .when(col("id") % 3 === 2, typedLit(Map(1 -> 1, 2 -> 2))) - .otherwise(typedLit(Map(2 -> 2, 1 -> 1))).cast(s"map<$dt, $dt>")) - .withColumn("c2", typedLit(Map(1 -> null)).cast(s"map<$dt, $dt>")) - .withColumn("c3", lit(null).cast(s"map<$dt, $dt>")) - - assertPartitionNumber(df.repartition(4, col("c1")), 2) - assertPartitionNumber(df.repartition(4, col("c2")), 1) - assertPartitionNumber(df.repartition(4, col("c3")), 1) - assertPartitionNumber(df.repartition(4, col("c1"), col("c2")), 2) - assertPartitionNumber(df.repartition(4, col("c1"), col("c3")), 2) - assertPartitionNumber(df.repartition(4, col("c1"), col("c2"), col("c3")), 2) - assertPartitionNumber(df.repartition(4, col("c2"), col("c3")), 2) + withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") { + val df = spark.range(20) + .withColumn("c1", + when(col("id") % 3 === 1, typedLit(Map(1 -> 1))) + .when(col("id") % 3 === 2, typedLit(Map(1 -> 1, 2 -> 2))) + .otherwise(typedLit(Map(2 -> 2, 1 -> 1))).cast(s"map<$dt, $dt>")) + .withColumn("c2", typedLit(Map(1 -> null)).cast(s"map<$dt, $dt>")) + .withColumn("c3", lit(null).cast(s"map<$dt, $dt>")) + + assertPartitionNumber(df.repartition(4, col("c1")), 2) + assertPartitionNumber(df.repartition(4, col("c2")), 1) + assertPartitionNumber(df.repartition(4, col("c3")), 1) + assertPartitionNumber(df.repartition(4, col("c1"), col("c2")), 2) + assertPartitionNumber(df.repartition(4, col("c1"), col("c3")), 2) + assertPartitionNumber(df.repartition(4, col("c1"), col("c2"), col("c3")), 2) + assertPartitionNumber(df.repartition(4, col("c2"), col("c3")), 2) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/LogQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/LogQuerySuite.scala index 873337e7a4242..861b0bf0f3945 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/LogQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/LogQuerySuite.scala @@ -33,12 +33,18 @@ class LogQuerySuite extends QueryTest with SharedSparkSession with Logging { new File(pwd + "/target/LogQuerySuite.log") } + override def beforeAll(): Unit = { + super.beforeAll() + Logging.enableStructuredLogging() + } + override def afterAll(): Unit = { super.afterAll() // Clear the log file if (logFile.exists()) { logFile.delete() } + Logging.disableStructuredLogging() } private def createTempView(viewName: String): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLFunctionSuite.scala new file mode 100644 index 0000000000000..4da3b9ab1d06b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLFunctionSuite.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.test.SharedSparkSession + +/** + * Test suite for SQL user-defined functions (UDFs). + */ +class SQLFunctionSuite extends QueryTest with SharedSparkSession { + import testImplicits._ + + protected override def beforeAll(): Unit = { + super.beforeAll() + Seq((0, 1), (1, 2)).toDF("a", "b").createOrReplaceTempView("t") + } + + test("SQL scalar function") { + withUserDefinedFunction("area" -> false) { + sql( + """ + |CREATE FUNCTION area(width DOUBLE, height DOUBLE) + |RETURNS DOUBLE + |RETURN width * height + |""".stripMargin) + checkAnswer(sql("SELECT area(1, 2)"), Row(2)) + checkAnswer(sql("SELECT area(a, b) FROM t"), Seq(Row(0), Row(2))) + } + } + + test("SQL scalar function with subquery in the function body") { + withUserDefinedFunction("foo" -> false) { + withTable("tbl") { + sql("CREATE TABLE tbl AS SELECT * FROM VALUES (1, 2), (1, 3), (2, 3) t(a, b)") + sql( + """ + |CREATE FUNCTION foo(x INT) RETURNS INT + |RETURN SELECT SUM(b) FROM tbl WHERE x = a; + |""".stripMargin) + checkAnswer(sql("SELECT foo(1)"), Row(5)) + checkAnswer(sql("SELECT foo(a) FROM t"), Seq(Row(null), Row(5))) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala index 0cc4f7bf2548e..0edbfd10d8cde 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala @@ -274,6 +274,19 @@ class ColumnVectorSuite extends SparkFunSuite with SQLHelper { } } + testVectors("mutable ColumnarRow with TimestampNTZType", 10, TimestampNTZType) { testVector => + val mutableRow = new MutableColumnarRow(Array(testVector)) + (0 until 10).foreach { i => + mutableRow.rowId = i + mutableRow.setLong(0, 10 - i) + } + (0 until 10).foreach { i => + mutableRow.rowId = i + assert(mutableRow.get(0, TimestampNTZType) === (10 - i)) + assert(mutableRow.copy().get(0, TimestampNTZType) === (10 - i)) + } + } + val arrayType: ArrayType = ArrayType(IntegerType, containsNull = true) testVectors("array", 10, arrayType) { testVector => @@ -384,18 +397,24 @@ class ColumnVectorSuite extends SparkFunSuite with SQLHelper { } val structType: StructType = new StructType().add("int", IntegerType).add("double", DoubleType) + .add("ts", TimestampNTZType) testVectors("struct", 10, structType) { testVector => val c1 = testVector.getChild(0) val c2 = testVector.getChild(1) + val c3 = testVector.getChild(2) c1.putInt(0, 123) c2.putDouble(0, 3.45) + c3.putLong(0, 1000L) c1.putInt(1, 456) c2.putDouble(1, 5.67) + c3.putLong(1, 2000L) assert(testVector.getStruct(0).get(0, IntegerType) === 123) assert(testVector.getStruct(0).get(1, DoubleType) === 3.45) + assert(testVector.getStruct(0).get(2, TimestampNTZType) === 1000L) assert(testVector.getStruct(1).get(0, IntegerType) === 456) assert(testVector.getStruct(1).get(1, DoubleType) === 5.67) + assert(testVector.getStruct(1).get(2, TimestampNTZType) === 2000L) } testVectors("SPARK-44805: getInts with dictionary", 3, IntegerType) { testVector => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala index c00f00ceaa355..a7af22a0554e9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala @@ -79,7 +79,8 @@ class ExpressionInfoSuite extends SparkFunSuite with SharedSparkSession { assert(info.getSource === "built-in") val validSources = Seq( - "built-in", "hive", "python_udf", "scala_udf", "java_udf", "python_udtf", "internal") + "built-in", "hive", "python_udf", "scala_udf", "java_udf", "python_udtf", "internal", + "sql_udf") validSources.foreach { source => val info = new ExpressionInfo( "testClass", null, "testName", null, "", "", "", "", "", "", source) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/vectorized/ArrowColumnVectorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/vectorized/ArrowColumnVectorSuite.scala index 436cea50ad972..9180ce1aee198 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/vectorized/ArrowColumnVectorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/vectorized/ArrowColumnVectorSuite.scala @@ -515,4 +515,28 @@ class ArrowColumnVectorSuite extends SparkFunSuite { columnVector.close() allocator.close() } + + test("struct with TimestampNTZType") { + val allocator = ArrowUtils.rootAllocator.newChildAllocator("struct", 0, Long.MaxValue) + val schema = new StructType().add("ts", TimestampNTZType) + val vector = ArrowUtils.toArrowField("struct", schema, nullable = true, null) + .createVector(allocator).asInstanceOf[StructVector] + vector.allocateNew() + val timestampVector = vector.getChildByOrdinal(0).asInstanceOf[TimeStampMicroVector] + + vector.setIndexDefined(0) + timestampVector.setSafe(0, 1000L) + + timestampVector.setValueCount(1) + vector.setValueCount(1) + + val columnVector = new ArrowColumnVector(vector) + assert(columnVector.dataType === schema) + + val row0 = columnVector.getStruct(0) + assert(row0.get(0, TimestampNTZType) === 1000L) + + columnVector.close() + allocator.close() + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/HiveCharVarcharTestSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveCharVarcharTestSuite.scala similarity index 98% rename from sql/hive/src/test/scala/org/apache/spark/sql/HiveCharVarcharTestSuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveCharVarcharTestSuite.scala index c12d727e59740..90cb5501ee6f6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/HiveCharVarcharTestSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveCharVarcharTestSuite.scala @@ -15,8 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql +package org.apache.spark.sql.hive +import org.apache.spark.sql.{CharVarcharTestSuite, Row} import org.apache.spark.sql.execution.command.CharVarcharDDLTestBase import org.apache.spark.sql.hive.test.TestHiveSingleton