Skip to content

Commit

Permalink
[SPARK-48906][SQL] Introduce SHOW COLLATIONS LIKE ... syntax to sho…
Browse files Browse the repository at this point in the history
…w all collations

### What changes were proposed in this pull request?
The pr aims to introduce `SHOW COLLATIONS LIKE ...` syntax to `show all collations`.

### Why are the changes needed?
End-users will be able to obtain `collations` currently supported by the spark through SQL.
Other databases, such as `MySQL`, also have similar syntax,
ref: https://dev.mysql.com/doc/refman/9.0/en/show-collation.html
<img width="958" alt="image" src="https://github.com/user-attachments/assets/1d5106b3-f8b8-42c5-b3ad-0f35c61ad5e2">

postgresql: https://database.guide/how-to-return-a-list-of-available-collations-in-postgresql/

### Does this PR introduce _any_ user-facing change?
Yes, end-users will be able to obtain `collation` currently supported by the spark through commands similar to the following

|name|provider|version|binaryEquality|binaryOrdering|lowercaseEquality|
| --------- | ----------- | ----------- | ----------- | ----------- | ----------- |

```
spark-sql (default)> SHOW COLLATIONS;
UTF8_BINARY	spark	1.0	true	true	false
UTF8_LCASE	spark	1.0	false	false	true
ff_Adlm	icu	153.120.0.0	false	false	false
ff_Adlm_CI	icu	153.120.0.0	false	false	false
ff_Adlm_AI	icu	153.120.0.0	false	false	false
ff_Adlm_CI_AI	icu	153.120.0.0	false	false	false
...

spark-sql (default)> SHOW COLLATIONS LIKE '*UTF8_BINARY*';
UTF8_BINARY	spark	1.0	true	true	false
Time taken: 0.043 seconds, Fetched 1 row(s)
```

<img width="513" alt="image" src="https://github.com/user-attachments/assets/d5765e32-718d-4236-857d-d508f5473329">

### How was this patch tested?
Add new UT.

### Was this patch authored or co-authored using generative AI tooling?
No.

Closes #47364 from panbingkun/show_collation_syntax.

Authored-by: panbingkun <[email protected]>
Signed-off-by: Max Gekk <[email protected]>
  • Loading branch information
panbingkun authored and MaxGekk committed Sep 12, 2024
1 parent b466f32 commit 0f4d289
Show file tree
Hide file tree
Showing 12 changed files with 278 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,14 @@
import java.util.function.Function;
import java.util.function.BiFunction;
import java.util.function.ToLongFunction;
import java.util.stream.Stream;

import com.ibm.icu.text.CollationKey;
import com.ibm.icu.text.Collator;
import com.ibm.icu.text.RuleBasedCollator;
import com.ibm.icu.text.StringSearch;
import com.ibm.icu.util.ULocale;
import com.ibm.icu.text.CollationKey;
import com.ibm.icu.text.Collator;
import com.ibm.icu.util.VersionInfo;

import org.apache.spark.SparkException;
import org.apache.spark.unsafe.types.UTF8String;
Expand Down Expand Up @@ -88,6 +90,17 @@ public Optional<String> getVersion() {
}
}

public record CollationMeta(
String catalog,
String schema,
String collationName,
String language,
String country,
String icuVersion,
String padAttribute,
boolean accentSensitivity,
boolean caseSensitivity) { }

/**
* Entry encapsulating all information about a collation.
*/
Expand Down Expand Up @@ -342,6 +355,23 @@ private static int collationNameToId(String collationName) throws SparkException
}

protected abstract Collation buildCollation();

protected abstract CollationMeta buildCollationMeta();

static List<CollationIdentifier> listCollations() {
return Stream.concat(
CollationSpecUTF8.listCollations().stream(),
CollationSpecICU.listCollations().stream()).toList();
}

static CollationMeta loadCollationMeta(CollationIdentifier collationIdentifier) {
CollationMeta collationSpecUTF8 =
CollationSpecUTF8.loadCollationMeta(collationIdentifier);
if (collationSpecUTF8 == null) {
return CollationSpecICU.loadCollationMeta(collationIdentifier);
}
return collationSpecUTF8;
}
}

private static class CollationSpecUTF8 extends CollationSpec {
Expand All @@ -364,6 +394,9 @@ 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).collationId;
private static final int UTF8_LCASE_COLLATION_ID =
Expand Down Expand Up @@ -406,7 +439,7 @@ private static CollationSpecUTF8 fromCollationId(int collationId) {
protected Collation buildCollation() {
if (collationId == UTF8_BINARY_COLLATION_ID) {
return new Collation(
"UTF8_BINARY",
UTF8_BINARY_COLLATION_NAME,
PROVIDER_SPARK,
null,
UTF8String::binaryCompare,
Expand All @@ -417,7 +450,7 @@ protected Collation buildCollation() {
/* supportsLowercaseEquality = */ false);
} else {
return new Collation(
"UTF8_LCASE",
UTF8_LCASE_COLLATION_NAME,
PROVIDER_SPARK,
null,
CollationAwareUTF8String::compareLowerCase,
Expand All @@ -428,6 +461,52 @@ protected Collation buildCollation() {
/* supportsLowercaseEquality = */ true);
}
}

@Override
protected CollationMeta buildCollationMeta() {
if (collationId == UTF8_BINARY_COLLATION_ID) {
return new CollationMeta(
CATALOG,
SCHEMA,
UTF8_BINARY_COLLATION_NAME,
/* language = */ null,
/* country = */ null,
/* icuVersion = */ null,
COLLATION_PAD_ATTRIBUTE,
/* accentSensitivity = */ true,
/* caseSensitivity = */ true);
} else {
return new CollationMeta(
CATALOG,
SCHEMA,
UTF8_LCASE_COLLATION_NAME,
/* language = */ null,
/* country = */ null,
/* icuVersion = */ null,
COLLATION_PAD_ATTRIBUTE,
/* accentSensitivity = */ true,
/* caseSensitivity = */ false);
}
}

static List<CollationIdentifier> listCollations() {
CollationIdentifier UTF8_BINARY_COLLATION_IDENT =
new CollationIdentifier(PROVIDER_SPARK, UTF8_BINARY_COLLATION_NAME, "1.0");
CollationIdentifier UTF8_LCASE_COLLATION_IDENT =
new CollationIdentifier(PROVIDER_SPARK, UTF8_LCASE_COLLATION_NAME, "1.0");
return Arrays.asList(UTF8_BINARY_COLLATION_IDENT, UTF8_LCASE_COLLATION_IDENT);
}

static CollationMeta loadCollationMeta(CollationIdentifier collationIdentifier) {
try {
int collationId = CollationSpecUTF8.collationNameToId(
collationIdentifier.name, collationIdentifier.name.toUpperCase());
return CollationSpecUTF8.fromCollationId(collationId).buildCollationMeta();
} catch (SparkException ignored) {
// ignore
return null;
}
}
}

private static class CollationSpecICU extends CollationSpec {
Expand Down Expand Up @@ -684,6 +763,20 @@ protected Collation buildCollation() {
/* supportsLowercaseEquality = */ false);
}

@Override
protected CollationMeta buildCollationMeta() {
return new CollationMeta(
CATALOG,
SCHEMA,
collationName(),
ICULocaleMap.get(locale).getDisplayLanguage(),
ICULocaleMap.get(locale).getDisplayCountry(),
VersionInfo.ICU_VERSION.toString(),
COLLATION_PAD_ATTRIBUTE,
caseSensitivity == CaseSensitivity.CS,
accentSensitivity == AccentSensitivity.AS);
}

/**
* Compute normalized collation name. Components of collation name are given in order:
* - Locale name
Expand All @@ -704,6 +797,37 @@ private String collationName() {
}
return builder.toString();
}

private static List<String> allCollationNames() {
List<String> collationNames = new ArrayList<>();
for (String locale: ICULocaleToId.keySet()) {
// CaseSensitivity.CS + AccentSensitivity.AS
collationNames.add(locale);
// CaseSensitivity.CS + AccentSensitivity.AI
collationNames.add(locale + "_AI");
// CaseSensitivity.CI + AccentSensitivity.AS
collationNames.add(locale + "_CI");
// CaseSensitivity.CI + AccentSensitivity.AI
collationNames.add(locale + "_CI_AI");
}
return collationNames.stream().sorted().toList();
}

static List<CollationIdentifier> listCollations() {
return allCollationNames().stream().map(name ->
new CollationIdentifier(PROVIDER_ICU, name, VersionInfo.ICU_VERSION.toString())).toList();
}

static CollationMeta loadCollationMeta(CollationIdentifier collationIdentifier) {
try {
int collationId = CollationSpecICU.collationNameToId(
collationIdentifier.name, collationIdentifier.name.toUpperCase());
return CollationSpecICU.fromCollationId(collationId).buildCollationMeta();
} catch (SparkException ignored) {
// ignore
return null;
}
}
}

/**
Expand All @@ -730,9 +854,12 @@ public CollationIdentifier identifier() {
}
}

public static final String CATALOG = "SYSTEM";
public static final String SCHEMA = "BUILTIN";
public static final String PROVIDER_SPARK = "spark";
public static final String PROVIDER_ICU = "icu";
public static final List<String> SUPPORTED_PROVIDERS = List.of(PROVIDER_SPARK, PROVIDER_ICU);
public static final String COLLATION_PAD_ATTRIBUTE = "NO_PAD";

public static final int UTF8_BINARY_COLLATION_ID =
Collation.CollationSpecUTF8.UTF8_BINARY_COLLATION_ID;
Expand Down Expand Up @@ -923,4 +1050,12 @@ public static String getClosestSuggestionsOnInvalidName(

return String.join(", ", suggestions);
}

public static List<CollationIdentifier> listCollations() {
return Collation.CollationSpec.listCollations();
}

public static CollationMeta loadCollationMeta(CollationIdentifier collationIdentifier) {
return Collation.CollationSpec.loadCollationMeta(collationIdentifier);
}
}
1 change: 1 addition & 0 deletions docs/sql-ref-ansi-compliance.md
Original file line number Diff line number Diff line change
Expand Up @@ -442,6 +442,7 @@ Below is a list of all the keywords in Spark SQL.
|CODEGEN|non-reserved|non-reserved|non-reserved|
|COLLATE|reserved|non-reserved|reserved|
|COLLATION|reserved|non-reserved|reserved|
|COLLATIONS|reserved|non-reserved|reserved|
|COLLECTION|non-reserved|non-reserved|non-reserved|
|COLUMN|reserved|non-reserved|reserved|
|COLUMNS|non-reserved|non-reserved|non-reserved|
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -162,6 +162,7 @@ CLUSTERED: 'CLUSTERED';
CODEGEN: 'CODEGEN';
COLLATE: 'COLLATE';
COLLATION: 'COLLATION';
COLLATIONS: 'COLLATIONS';
COLLECTION: 'COLLECTION';
COLUMN: 'COLUMN';
COLUMNS: 'COLUMNS';
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -260,6 +260,7 @@ statement
| SHOW PARTITIONS identifierReference partitionSpec? #showPartitions
| SHOW identifier? FUNCTIONS ((FROM | IN) ns=identifierReference)?
(LIKE? (legacy=multipartIdentifier | pattern=stringLit))? #showFunctions
| SHOW COLLATIONS (LIKE? pattern=stringLit)? #showCollations
| SHOW CREATE TABLE identifierReference (AS SERDE)? #showCreateTable
| SHOW CURRENT namespace #showCurrentNamespace
| SHOW CATALOGS (LIKE? pattern=stringLit)? #showCatalogs
Expand Down Expand Up @@ -1837,6 +1838,7 @@ nonReserved
| CODEGEN
| COLLATE
| COLLATION
| COLLATIONS
| COLLECTION
| COLUMN
| COLUMNS
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import java.util.concurrent.TimeUnit
import javax.annotation.concurrent.GuardedBy

import scala.collection.mutable
import scala.jdk.CollectionConverters.CollectionHasAsScala
import scala.util.{Failure, Success, Try}

import com.google.common.cache.{Cache, CacheBuilder}
Expand All @@ -39,7 +40,8 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, Expression, Expre
import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 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.catalyst.util.{CharVarcharUtils, CollationFactory, StringUtils}
import org.apache.spark.sql.catalyst.util.CollationFactory.CollationMeta
import org.apache.spark.sql.connector.catalog.CatalogManager
import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME
import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors}
Expand Down Expand Up @@ -1899,6 +1901,17 @@ class SessionCatalog(
.filter(isTemporaryFunction)
}

/**
* List all built-in collations with the given pattern.
*/
def listCollations(pattern: Option[String]): Seq[CollationMeta] = {
val collationIdentifiers = CollationFactory.listCollations().asScala.toSeq
val filteredCollationNames = StringUtils.filterPattern(
collationIdentifiers.map(_.getName), pattern.getOrElse("*")).toSet
collationIdentifiers.filter(ident => filteredCollationNames.contains(ident.getName)).map(
CollationFactory.loadCollationMeta)
}

// -----------------
// | Other methods |
// -----------------
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ CLOSE
COALESCE
COLLATE
COLLATION
COLLATIONS
COLLECT
COLUMN
COMMIT
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1096,4 +1096,16 @@ class SparkSqlAstBuilder extends AstBuilder {
withIdentClause(ctx.identifierReference(), UnresolvedNamespace(_)),
cleanedProperties)
}

/**
* Create a [[ShowCollationsCommand]] command.
* Expected format:
* {{{
* SHOW COLLATIONS (LIKE? pattern=stringLit)?;
* }}}
*/
override def visitShowCollations(ctx: ShowCollationsContext): LogicalPlan = withOrigin(ctx) {
val pattern = Option(ctx.pattern).map(x => string(visitStringLit(x)))
ShowCollationsCommand(pattern)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* 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.command

import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.catalyst.util.CollationFactory.CollationMeta
import org.apache.spark.sql.types.StringType

/**
* A command for `SHOW COLLATIONS`.
*
* The syntax of this command is:
* {{{
* SHOW COLLATIONS (LIKE? pattern=stringLit)?;
* }}}
*/
case class ShowCollationsCommand(pattern: Option[String]) extends LeafRunnableCommand {

override val output: Seq[Attribute] = Seq(
AttributeReference("COLLATION_CATALOG", StringType, nullable = false)(),
AttributeReference("COLLATION_SCHEMA", StringType, nullable = false)(),
AttributeReference("COLLATION_NAME", StringType, nullable = false)(),
AttributeReference("LANGUAGE", StringType)(),
AttributeReference("COUNTRY", StringType)(),
AttributeReference("ACCENT_SENSITIVITY", StringType, nullable = false)(),
AttributeReference("CASE_SENSITIVITY", StringType, nullable = false)(),
AttributeReference("PAD_ATTRIBUTE", StringType, nullable = false)(),
AttributeReference("ICU_VERSION", StringType)())

override def run(sparkSession: SparkSession): Seq[Row] = {
val systemCollations: Seq[CollationMeta] =
sparkSession.sessionState.catalog.listCollations(pattern)

systemCollations.map(m => Row(
m.catalog,
m.schema,
m.collationName,
m.language,
m.country,
if (m.accentSensitivity) "ACCENT_SENSITIVE" else "ACCENT_INSENSITIVE",
if (m.caseSensitivity) "CASE_SENSITIVE" else "CASE_INSENSITIVE",
m.padAttribute,
m.icuVersion
))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ CLUSTERED false
CODEGEN false
COLLATE true
COLLATION true
COLLATIONS true
COLLECTION false
COLUMN true
COLUMNS false
Expand Down Expand Up @@ -381,6 +382,7 @@ CAST
CHECK
COLLATE
COLLATION
COLLATIONS
COLUMN
CONSTRAINT
CREATE
Expand Down
Loading

0 comments on commit 0f4d289

Please sign in to comment.