From a26e3ed5e414d0a350cfe65dd511b154868b9f1d Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 10 Apr 2017 20:11:56 +0100 Subject: [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String toLowerCase "Turkish locale bug" causes Spark problems ## What changes were proposed in this pull request? Add Locale.ROOT to internal calls to String `toLowerCase`, `toUpperCase`, to avoid inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem"). The change looks large but it is just adding `Locale.ROOT` (the locale with no country or language specified) to every call to these methods. ## How was this patch tested? Existing tests. Author: Sean Owen Closes #17527 from srowen/SPARK-20156. --- .../apache/spark/sql/catalyst/analysis/ResolveHints.scala | 4 +++- .../spark/sql/catalyst/catalog/ExternalCatalogUtils.scala | 7 +++++-- .../apache/spark/sql/catalyst/catalog/SessionCatalog.scala | 3 ++- .../spark/sql/catalyst/catalog/functionResources.scala | 4 +++- .../apache/spark/sql/catalyst/expressions/Expression.scala | 4 +++- .../spark/sql/catalyst/expressions/mathExpressions.scala | 6 ++++-- .../spark/sql/catalyst/expressions/regexpExpressions.scala | 3 ++- .../spark/sql/catalyst/expressions/windowExpressions.scala | 4 +++- .../org/apache/spark/sql/catalyst/json/JacksonParser.scala | 5 +++-- .../org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 12 ++++++++---- .../org/apache/spark/sql/catalyst/plans/joinTypes.scala | 4 +++- .../spark/sql/catalyst/streaming/InternalOutputModes.scala | 4 +++- .../apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala | 9 ++++++--- .../apache/spark/sql/catalyst/util/CompressionCodecs.scala | 4 +++- .../org/apache/spark/sql/catalyst/util/DateTimeUtils.scala | 4 ++-- .../scala/org/apache/spark/sql/catalyst/util/ParseMode.scala | 4 +++- .../apache/spark/sql/catalyst/util/StringKeyHashMap.scala | 4 +++- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 6 +++--- .../src/main/scala/org/apache/spark/sql/types/DataType.scala | 8 ++++++-- .../main/scala/org/apache/spark/sql/types/DecimalType.scala | 4 +++- .../org/apache/spark/sql/streaming/JavaOutputModeSuite.java | 6 ++++-- .../apache/spark/sql/catalyst/analysis/AnalysisTest.scala | 5 ++++- .../sql/catalyst/analysis/UnsupportedOperationsSuite.scala | 7 ++++--- .../spark/sql/catalyst/expressions/ScalaUDFSuite.scala | 4 +++- .../sql/catalyst/streaming/InternalOutputModesSuite.scala | 4 +++- 25 files changed, 89 insertions(+), 40 deletions(-) (limited to 'sql/catalyst') diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala index f8004ca300..c4827b81e8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.analysis +import java.util.Locale + import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.CurrentOrigin @@ -83,7 +85,7 @@ object ResolveHints { } def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case h: Hint if BROADCAST_HINT_NAMES.contains(h.name.toUpperCase) => + case h: Hint if BROADCAST_HINT_NAMES.contains(h.name.toUpperCase(Locale.ROOT)) => applyBroadcastHint(h.child, h.parameters.toSet) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala index 254eedfe77..3ca9e6a8da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.catalog import java.net.URI +import java.util.Locale import org.apache.hadoop.fs.Path import org.apache.hadoop.util.Shell @@ -167,8 +168,10 @@ object CatalogUtils { */ def maskCredentials(options: Map[String, String]): Map[String, String] = { options.map { - case (key, _) if key.toLowerCase == "password" => (key, "###") - case (key, value) if key.toLowerCase == "url" && value.toLowerCase.contains("password") => + case (key, _) if key.toLowerCase(Locale.ROOT) == "password" => (key, "###") + case (key, value) + if key.toLowerCase(Locale.ROOT) == "url" && + value.toLowerCase(Locale.ROOT).contains("password") => (key, "###") case o => o } 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 6f8c6ee2f0..faedf5f91c 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.catalog import java.net.URI +import java.util.Locale import javax.annotation.concurrent.GuardedBy import scala.collection.mutable @@ -1098,7 +1099,7 @@ class SessionCatalog( name.database.isEmpty && functionRegistry.functionExists(name.funcName) && !FunctionRegistry.builtin.functionExists(name.funcName) && - !hiveFunctions.contains(name.funcName.toLowerCase) + !hiveFunctions.contains(name.funcName.toLowerCase(Locale.ROOT)) } protected def failFunctionLookup(name: String): Nothing = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala index 8e46b962ff..67bf2d06c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.catalog +import java.util.Locale + import org.apache.spark.sql.AnalysisException /** A trait that represents the type of a resourced needed by a function. */ @@ -33,7 +35,7 @@ object ArchiveResource extends FunctionResourceType("archive") object FunctionResourceType { def fromString(resourceType: String): FunctionResourceType = { - resourceType.toLowerCase match { + resourceType.toLowerCase(Locale.ROOT) match { case "jar" => JarResource case "file" => FileResource case "archive" => ArchiveResource diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 1db26d9c41..b847ef7bfa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import java.util.Locale + import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen._ @@ -184,7 +186,7 @@ abstract class Expression extends TreeNode[Expression] { * Returns a user-facing string representation of this expression's name. * This should usually match the name of the function in SQL. */ - def prettyName: String = nodeName.toLowerCase + def prettyName: String = nodeName.toLowerCase(Locale.ROOT) protected def flatArguments: Iterator[Any] = productIterator.flatMap { case t: Traversable[_] => t diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala index dea5f85cb0..c4d47ab208 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.{lang => jl} +import java.util.Locale import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} @@ -68,7 +69,7 @@ abstract class UnaryMathExpression(val f: Double => Double, name: String) } // name of function in java.lang.Math - def funcName: String = name.toLowerCase + def funcName: String = name.toLowerCase(Locale.ROOT) override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { defineCodeGen(ctx, ev, c => s"java.lang.Math.${funcName}($c)") @@ -124,7 +125,8 @@ abstract class BinaryMathExpression(f: (Double, Double) => Double, name: String) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.${name.toLowerCase}($c1, $c2)") + defineCodeGen(ctx, ev, (c1, c2) => + s"java.lang.Math.${name.toLowerCase(Locale.ROOT)}($c1, $c2)") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala index b23da537be..49b7797113 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions +import java.util.Locale import java.util.regex.{MatchResult, Pattern} import org.apache.commons.lang3.StringEscapeUtils @@ -60,7 +61,7 @@ abstract class StringRegexExpression extends BinaryExpression } } - override def sql: String = s"${left.sql} ${prettyName.toUpperCase} ${right.sql}" + override def sql: String = s"${left.sql} ${prettyName.toUpperCase(Locale.ROOT)} ${right.sql}" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala index b2a3888ff7..37190429fc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import java.util.Locale + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, UnresolvedException} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} @@ -631,7 +633,7 @@ abstract class RankLike extends AggregateWindowFunction { override val updateExpressions = increaseRank +: increaseRowNumber +: children override val evaluateExpression: Expression = rank - override def sql: String = s"${prettyName.toUpperCase}()" + override def sql: String = s"${prettyName.toUpperCase(Locale.ROOT)}()" def withOrder(order: Seq[Expression]): RankLike } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index fdb7d88d5b..ff6c93ae98 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.json import java.io.ByteArrayOutputStream +import java.util.Locale import scala.collection.mutable.ArrayBuffer import scala.util.Try @@ -126,7 +127,7 @@ class JacksonParser( case VALUE_STRING => // Special case handling for NaN and Infinity. val value = parser.getText - val lowerCaseValue = value.toLowerCase + val lowerCaseValue = value.toLowerCase(Locale.ROOT) if (lowerCaseValue.equals("nan") || lowerCaseValue.equals("infinity") || lowerCaseValue.equals("-infinity") || @@ -146,7 +147,7 @@ class JacksonParser( case VALUE_STRING => // Special case handling for NaN and Infinity. val value = parser.getText - val lowerCaseValue = value.toLowerCase + val lowerCaseValue = value.toLowerCase(Locale.ROOT) if (lowerCaseValue.equals("nan") || lowerCaseValue.equals("infinity") || lowerCaseValue.equals("-infinity") || diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index c372551538..e1db1ef5b8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.parser import java.sql.{Date, Timestamp} +import java.util.Locale import javax.xml.bind.DatatypeConverter import scala.collection.JavaConverters._ @@ -1047,7 +1048,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { val name = ctx.qualifiedName.getText val isDistinct = Option(ctx.setQuantifier()).exists(_.DISTINCT != null) val arguments = ctx.namedExpression().asScala.map(expression) match { - case Seq(UnresolvedStar(None)) if name.toLowerCase == "count" && !isDistinct => + case Seq(UnresolvedStar(None)) + if name.toLowerCase(Locale.ROOT) == "count" && !isDistinct => // Transform COUNT(*) into COUNT(1). Seq(Literal(1)) case expressions => @@ -1271,7 +1273,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { */ override def visitTypeConstructor(ctx: TypeConstructorContext): Literal = withOrigin(ctx) { val value = string(ctx.STRING) - val valueType = ctx.identifier.getText.toUpperCase + val valueType = ctx.identifier.getText.toUpperCase(Locale.ROOT) try { valueType match { case "DATE" => @@ -1427,7 +1429,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { import ctx._ val s = value.getText try { - val interval = (unit.getText.toLowerCase, Option(to).map(_.getText.toLowerCase)) match { + val unitText = unit.getText.toLowerCase(Locale.ROOT) + val interval = (unitText, Option(to).map(_.getText.toLowerCase(Locale.ROOT))) match { case (u, None) if u.endsWith("s") => // Handle plural forms, e.g: yearS/monthS/weekS/dayS/hourS/minuteS/hourS/... CalendarInterval.fromSingleUnitString(u.substring(0, u.length - 1), s) @@ -1465,7 +1468,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { * Resolve/create a primitive type. */ override def visitPrimitiveDataType(ctx: PrimitiveDataTypeContext): DataType = withOrigin(ctx) { - (ctx.identifier.getText.toLowerCase, ctx.INTEGER_VALUE().asScala.toList) match { + val dataType = ctx.identifier.getText.toLowerCase(Locale.ROOT) + (dataType, ctx.INTEGER_VALUE().asScala.toList) match { case ("boolean", Nil) => BooleanType case ("tinyint" | "byte", Nil) => ByteType case ("smallint" | "short", Nil) => ShortType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala index 818f4e5ed2..90d11d6d91 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala @@ -17,10 +17,12 @@ package org.apache.spark.sql.catalyst.plans +import java.util.Locale + import org.apache.spark.sql.catalyst.expressions.Attribute object JoinType { - def apply(typ: String): JoinType = typ.toLowerCase.replace("_", "") match { + def apply(typ: String): JoinType = typ.toLowerCase(Locale.ROOT).replace("_", "") match { case "inner" => Inner case "outer" | "full" | "fullouter" => FullOuter case "leftouter" | "left" => LeftOuter diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModes.scala index bdf2baf736..3cd6970ebe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModes.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.streaming +import java.util.Locale + import org.apache.spark.sql.streaming.OutputMode /** @@ -47,7 +49,7 @@ private[sql] object InternalOutputModes { def apply(outputMode: String): OutputMode = { - outputMode.toLowerCase match { + outputMode.toLowerCase(Locale.ROOT) match { case "append" => OutputMode.Append case "complete" => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala index 66dd093bbb..bb2c5926ae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CaseInsensitiveMap.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.util +import java.util.Locale + /** * Builds a map in which keys are case insensitive. Input map can be accessed for cases where * case-sensitive information is required. The primary constructor is marked private to avoid @@ -26,11 +28,12 @@ package org.apache.spark.sql.catalyst.util class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends Map[String, T] with Serializable { - val keyLowerCasedMap = originalMap.map(kv => kv.copy(_1 = kv._1.toLowerCase)) + val keyLowerCasedMap = originalMap.map(kv => kv.copy(_1 = kv._1.toLowerCase(Locale.ROOT))) - override def get(k: String): Option[T] = keyLowerCasedMap.get(k.toLowerCase) + override def get(k: String): Option[T] = keyLowerCasedMap.get(k.toLowerCase(Locale.ROOT)) - override def contains(k: String): Boolean = keyLowerCasedMap.contains(k.toLowerCase) + override def contains(k: String): Boolean = + keyLowerCasedMap.contains(k.toLowerCase(Locale.ROOT)) override def +[B1 >: T](kv: (String, B1)): Map[String, B1] = { new CaseInsensitiveMap(originalMap + kv) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CompressionCodecs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CompressionCodecs.scala index 435fba9d88..1377a03d93 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CompressionCodecs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CompressionCodecs.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.util +import java.util.Locale + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress._ @@ -38,7 +40,7 @@ object CompressionCodecs { * If it is already a class name, just return it. */ def getCodecClassName(name: String): String = { - val codecName = shortCompressionCodecNames.getOrElse(name.toLowerCase, name) + val codecName = shortCompressionCodecNames.getOrElse(name.toLowerCase(Locale.ROOT), name) try { // Validate the codec name if (codecName != null) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index f614965520..eb6aad5b2d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -894,7 +894,7 @@ object DateTimeUtils { * (Because 1970-01-01 is Thursday). */ def getDayOfWeekFromString(string: UTF8String): Int = { - val dowString = string.toString.toUpperCase + val dowString = string.toString.toUpperCase(Locale.ROOT) dowString match { case "SU" | "SUN" | "SUNDAY" => 3 case "MO" | "MON" | "MONDAY" => 4 @@ -951,7 +951,7 @@ object DateTimeUtils { if (format == null) { TRUNC_INVALID } else { - format.toString.toUpperCase match { + format.toString.toUpperCase(Locale.ROOT) match { case "YEAR" | "YYYY" | "YY" => TRUNC_TO_YEAR case "MON" | "MONTH" | "MM" => TRUNC_TO_MONTH case _ => TRUNC_INVALID diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParseMode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParseMode.scala index 4565dbde88..2beb875d17 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParseMode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ParseMode.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.util +import java.util.Locale + import org.apache.spark.internal.Logging sealed trait ParseMode { @@ -45,7 +47,7 @@ object ParseMode extends Logging { /** * Returns the parse mode from the given string. */ - def fromString(mode: String): ParseMode = mode.toUpperCase match { + def fromString(mode: String): ParseMode = mode.toUpperCase(Locale.ROOT) match { case PermissiveMode.name => PermissiveMode case DropMalformedMode.name => DropMalformedMode case FailFastMode.name => FailFastMode diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringKeyHashMap.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringKeyHashMap.scala index a7ac613683..812d5ded4b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringKeyHashMap.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringKeyHashMap.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.util +import java.util.Locale + /** * Build a map with String type of key, and it also supports either key case * sensitive or insensitive. @@ -25,7 +27,7 @@ object StringKeyHashMap { def apply[T](caseSensitive: Boolean): StringKeyHashMap[T] = if (caseSensitive) { new StringKeyHashMap[T](identity) } else { - new StringKeyHashMap[T](_.toLowerCase) + new StringKeyHashMap[T](_.toLowerCase(Locale.ROOT)) } } 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 640c0f189c..6b0f495033 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.internal -import java.util.{NoSuchElementException, Properties, TimeZone} +import java.util.{Locale, NoSuchElementException, Properties, TimeZone} import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ @@ -243,7 +243,7 @@ object SQLConf { .doc("Sets the compression codec use when writing Parquet files. Acceptable values include: " + "uncompressed, snappy, gzip, lzo.") .stringConf - .transform(_.toLowerCase()) + .transform(_.toLowerCase(Locale.ROOT)) .checkValues(Set("uncompressed", "snappy", "gzip", "lzo")) .createWithDefault("snappy") @@ -324,7 +324,7 @@ object SQLConf { "properties) and NEVER_INFER (fallback to using the case-insensitive metastore schema " + "instead of inferring).") .stringConf - .transform(_.toUpperCase()) + .transform(_.toUpperCase(Locale.ROOT)) .checkValues(HiveCaseSensitiveInferenceMode.values.map(_.toString)) .createWithDefault(HiveCaseSensitiveInferenceMode.INFER_AND_SAVE.toString) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala index 26871259c6..520aff5e2b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.types +import java.util.Locale + import org.json4s._ import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ @@ -49,7 +51,9 @@ abstract class DataType extends AbstractDataType { /** Name of the type used in JSON serialization. */ def typeName: String = { - this.getClass.getSimpleName.stripSuffix("$").stripSuffix("Type").stripSuffix("UDT").toLowerCase + this.getClass.getSimpleName + .stripSuffix("$").stripSuffix("Type").stripSuffix("UDT") + .toLowerCase(Locale.ROOT) } private[sql] def jsonValue: JValue = typeName @@ -69,7 +73,7 @@ abstract class DataType extends AbstractDataType { /** Readable string representation for the type with truncation */ private[sql] def simpleString(maxNumberFields: Int): String = simpleString - def sql: String = simpleString.toUpperCase + def sql: String = simpleString.toUpperCase(Locale.ROOT) /** * Check if `this` and `other` are the same data type when ignoring nullability diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala index 4dc06fc9cf..5c4bc5e33c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.types +import java.util.Locale + import scala.reflect.runtime.universe.typeTag import org.apache.spark.annotation.InterfaceStability @@ -65,7 +67,7 @@ case class DecimalType(precision: Int, scale: Int) extends FractionalType { override def toString: String = s"DecimalType($precision,$scale)" - override def sql: String = typeName.toUpperCase + override def sql: String = typeName.toUpperCase(Locale.ROOT) /** * Returns whether this DecimalType is wider than `other`. If yes, it means `other` diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/streaming/JavaOutputModeSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/streaming/JavaOutputModeSuite.java index e0a54fe30a..d8845e0c83 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/streaming/JavaOutputModeSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/streaming/JavaOutputModeSuite.java @@ -17,6 +17,8 @@ package org.apache.spark.sql.streaming; +import java.util.Locale; + import org.junit.Test; public class JavaOutputModeSuite { @@ -24,8 +26,8 @@ public class JavaOutputModeSuite { @Test public void testOutputModes() { OutputMode o1 = OutputMode.Append(); - assert(o1.toString().toLowerCase().contains("append")); + assert(o1.toString().toLowerCase(Locale.ROOT).contains("append")); OutputMode o2 = OutputMode.Complete(); - assert (o2.toString().toLowerCase().contains("complete")); + assert (o2.toString().toLowerCase(Locale.ROOT).contains("complete")); } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index 1be25ec06c..82015b1e06 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.analysis +import java.util.Locale + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.plans.PlanTest @@ -79,7 +81,8 @@ trait AnalysisTest extends PlanTest { analyzer.checkAnalysis(analyzer.execute(inputPlan)) } - if (!expectedErrors.map(_.toLowerCase).forall(e.getMessage.toLowerCase.contains)) { + if (!expectedErrors.map(_.toLowerCase(Locale.ROOT)).forall( + e.getMessage.toLowerCase(Locale.ROOT).contains)) { fail( s"""Exception message should contain the following substrings: | diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index 8f0a0c0d99..c39e372c27 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -17,19 +17,20 @@ package org.apache.spark.sql.catalyst.analysis +import java.util.Locale + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Literal, NamedExpression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, NamedExpression} import org.apache.spark.sql.catalyst.expressions.aggregate.Count import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{FlatMapGroupsWithState, _} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.{IntegerType, LongType, MetadataBuilder} -import org.apache.spark.unsafe.types.CalendarInterval /** A dummy command for testing unsupported operations. */ case class DummyCommand() extends Command @@ -696,7 +697,7 @@ class UnsupportedOperationsSuite extends SparkFunSuite { testBody } expectedMsgs.foreach { m => - if (!e.getMessage.toLowerCase.contains(m.toLowerCase)) { + if (!e.getMessage.toLowerCase(Locale.ROOT).contains(m.toLowerCase(Locale.ROOT))) { fail(s"Exception message should contain: '$m', " + s"actual exception message:\n\t'${e.getMessage}'") } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala index 7e45028653..13bd363c8b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import java.util.Locale + import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.sql.types.{IntegerType, StringType} @@ -32,7 +34,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { test("better error message for NPE") { val udf = ScalaUDF( - (s: String) => s.toLowerCase, + (s: String) => s.toLowerCase(Locale.ROOT), StringType, Literal.create(null, StringType) :: Nil) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModesSuite.scala index 201dac35ed..3159b541dc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/streaming/InternalOutputModesSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.streaming +import java.util.Locale + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.streaming.OutputMode @@ -40,7 +42,7 @@ class InternalOutputModesSuite extends SparkFunSuite { val acceptedModes = Seq("append", "update", "complete") val e = intercept[IllegalArgumentException](InternalOutputModes(outputMode)) (Seq("output mode", "unknown", outputMode) ++ acceptedModes).foreach { s => - assert(e.getMessage.toLowerCase.contains(s.toLowerCase)) + assert(e.getMessage.toLowerCase(Locale.ROOT).contains(s.toLowerCase(Locale.ROOT))) } } testMode("Xyz") -- cgit v1.2.3