Skip to content

Commit

Permalink
resolve conflicts
Browse files Browse the repository at this point in the history
  • Loading branch information
itholic committed Oct 14, 2024
2 parents b3b5015 + eeb044e commit 5a12501
Show file tree
Hide file tree
Showing 12 changed files with 51 additions and 45 deletions.
28 changes: 12 additions & 16 deletions common/utils/src/main/resources/error/error-conditions.json
Original file line number Diff line number Diff line change
Expand Up @@ -2599,6 +2599,13 @@
},
"sqlState" : "42K0K"
},
"INVALID_JAVA_IDENTIFIER_AS_FIELD_NAME" : {
"message" : [
"<fieldName> is not a valid identifier of Java and cannot be used as field name",
"<walkedTypePath>."
],
"sqlState" : "46121"
},
"INVALID_JOIN_TYPE_FOR_JOINWITH" : {
"message" : [
"Invalid join type in joinWith: <joinType>."
Expand Down Expand Up @@ -5186,6 +5193,11 @@
"message" : [
"TRIM specifier in the collation."
]
},
"UPDATE_COLUMN_NULLABILITY" : {
"message" : [
"Update column nullability for MySQL and MS SQL Server."
]
}
},
"sqlState" : "0A000"
Expand Down Expand Up @@ -6131,11 +6143,6 @@
"<className> is not a valid Spark SQL Data Source."
]
},
"_LEGACY_ERROR_TEMP_1136" : {
"message" : [
"Cannot save interval data type into external storage."
]
},
"_LEGACY_ERROR_TEMP_1137" : {
"message" : [
"Unable to resolve <name> given [<outputStr>]."
Expand Down Expand Up @@ -7202,12 +7209,6 @@
"cannot have circular references in class, but got the circular reference of class <t>."
]
},
"_LEGACY_ERROR_TEMP_2140" : {
"message" : [
"`<fieldName>` is not a valid identifier of Java and cannot be used as field name",
"<walkedTypePath>."
]
},
"_LEGACY_ERROR_TEMP_2144" : {
"message" : [
"Unable to find constructor for <tpe>. This could happen if <tpe> is an interface, or a trait without companion object constructor."
Expand Down Expand Up @@ -7696,11 +7697,6 @@
"comment on table is not supported."
]
},
"_LEGACY_ERROR_TEMP_2271" : {
"message" : [
"UpdateColumnNullability is not supported."
]
},
"_LEGACY_ERROR_TEMP_2272" : {
"message" : [
"Rename column is only supported for MySQL version 8.0 and above."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1673,8 +1673,12 @@ abstract class AvroSuite
exception = intercept[AnalysisException] {
sql("select interval 1 days").write.format("avro").mode("overwrite").save(tempDir)
},
condition = "_LEGACY_ERROR_TEMP_1136",
parameters = Map.empty
condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE",
parameters = Map(
"format" -> "Avro",
"columnName" -> "`INTERVAL '1 days'`",
"columnType" -> "\"INTERVAL\""
)
)
checkError(
exception = intercept[AnalysisException] {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,7 +115,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JD
exception = intercept[SparkSQLFeatureNotSupportedException] {
sql(s"ALTER TABLE $tbl ALTER COLUMN ID DROP NOT NULL")
},
condition = "_LEGACY_ERROR_TEMP_2271")
condition = "UNSUPPORTED_FEATURE.UPDATE_COLUMN_NULLABILITY")
}

test("SPARK-47440: SQLServer does not support boolean expression in binary comparison") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,7 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest
exception = intercept[SparkSQLFeatureNotSupportedException] {
sql(s"ALTER TABLE $tbl ALTER COLUMN ID DROP NOT NULL")
},
condition = "_LEGACY_ERROR_TEMP_2271")
condition = "UNSUPPORTED_FEATURE.UPDATE_COLUMN_NULLABILITY")
}

override def testCreateTableWithProperty(tbl: String): Unit = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ class SparkThrowableSuite extends SparkFunSuite {
}}}
*/
private val regenerateCommand = "SPARK_GENERATE_GOLDEN_FILES=1 build/sbt " +
"\"core/testOnly *SparkThrowableSuite -- -t \\\"Error classes match with document\\\"\""
"\"core/testOnly *SparkThrowableSuite -- -t \\\"Error conditions are correctly formatted\\\"\""

private val errorJsonFilePath = getWorkspaceFilePath(
"common", "utils", "src", "main", "resources", "error", "error-conditions.json")
Expand Down
3 changes: 1 addition & 2 deletions python/pyspark/pandas/internal.py
Original file line number Diff line number Diff line change
Expand Up @@ -902,11 +902,10 @@ def attach_default_index(

@staticmethod
def attach_sequence_column(sdf: PySparkDataFrame, column_name: str) -> PySparkDataFrame:
scols = [scol_for(sdf, column) for column in sdf.columns]
sequential_index = (
F.row_number().over(Window.orderBy(F.monotonically_increasing_id())).cast("long") - 1
)
return sdf.select(sequential_index.alias(column_name), *scols)
return sdf.select(sequential_index.alias(column_name), "*")

@staticmethod
def attach_distributed_column(sdf: PySparkDataFrame, column_name: str) -> PySparkDataFrame:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -195,9 +195,9 @@ private[sql] trait ExecutionErrors extends DataTypeErrorsBase {
fieldName: String,
walkedTypePath: WalkedTypePath): SparkUnsupportedOperationException = {
new SparkUnsupportedOperationException(
errorClass = "_LEGACY_ERROR_TEMP_2140",
errorClass = "INVALID_JAVA_IDENTIFIER_AS_FIELD_NAME",
messageParameters =
Map("fieldName" -> fieldName, "walkedTypePath" -> walkedTypePath.toString))
Map("fieldName" -> toSQLId(fieldName), "walkedTypePath" -> walkedTypePath.toString))
}

def primaryConstructorNotFoundError(cls: Class[_]): SparkRuntimeException = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1691,12 +1691,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
messageParameters = Map("className" -> className))
}

def cannotSaveIntervalIntoExternalStorageError(): Throwable = {
new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_1136",
messageParameters = Map.empty)
}

def cannotResolveAttributeError(name: String, outputStr: String): Throwable = {
new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_1137",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2275,7 +2275,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE

def unsupportedUpdateColumnNullabilityError(): SparkSQLFeatureNotSupportedException = {
new SparkSQLFeatureNotSupportedException(
errorClass = "_LEGACY_ERROR_TEMP_2271",
errorClass = "UNSUPPORTED_FEATURE.UPDATE_COLUMN_NULLABILITY",
messageParameters = Map.empty)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -514,7 +514,8 @@ case class DataSource(
dataSource.createRelation(
sparkSession.sqlContext, mode, caseInsensitiveOptions, Dataset.ofRows(sparkSession, data))
case format: FileFormat =>
disallowWritingIntervals(outputColumns.map(_.dataType), forbidAnsiIntervals = false)
disallowWritingIntervals(
outputColumns.toStructType.asNullable, format.toString, forbidAnsiIntervals = false)
val cmd = planForWritingFileFormat(format, mode, data)
val qe = sparkSession.sessionState.executePlan(cmd)
qe.assertCommandExecuted()
Expand All @@ -539,7 +540,7 @@ case class DataSource(
}
SaveIntoDataSourceCommand(data, dataSource, caseInsensitiveOptions, mode)
case format: FileFormat =>
disallowWritingIntervals(data.schema.map(_.dataType), forbidAnsiIntervals = false)
disallowWritingIntervals(data.schema, format.toString, forbidAnsiIntervals = false)
DataSource.validateSchema(format.toString, data.schema, sparkSession.sessionState.conf)
planForWritingFileFormat(format, mode, data)
case _ => throw SparkException.internalError(
Expand All @@ -566,12 +567,15 @@ case class DataSource(
}

private def disallowWritingIntervals(
dataTypes: Seq[DataType],
outputColumns: Seq[StructField],
format: String,
forbidAnsiIntervals: Boolean): Unit = {
dataTypes.foreach(
TypeUtils.invokeOnceForInterval(_, forbidAnsiIntervals) {
throw QueryCompilationErrors.cannotSaveIntervalIntoExternalStorageError()
})
outputColumns.foreach { field =>
TypeUtils.invokeOnceForInterval(field.dataType, forbidAnsiIntervals) {
throw QueryCompilationErrors.dataTypeUnsupportedByDataSourceError(
format, field
)}
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -513,14 +513,23 @@ class FileBasedDataSourceSuite extends QueryTest
withSQLConf(
SQLConf.USE_V1_SOURCE_LIST.key -> useV1List,
SQLConf.LEGACY_INTERVAL_ENABLED.key -> "true") {
val formatMapping = Map(
"csv" -> "CSV",
"json" -> "JSON",
"parquet" -> "Parquet",
"orc" -> "ORC"
)
// write path
Seq("csv", "json", "parquet", "orc").foreach { format =>
checkError(
exception = intercept[AnalysisException] {
sql("select interval 1 days").write.format(format).mode("overwrite").save(tempDir)
},
condition = "_LEGACY_ERROR_TEMP_1136",
parameters = Map.empty
condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE",
parameters = Map(
"format" -> formatMapping(format),
"columnName" -> "`INTERVAL '1 days'`",
"columnType" -> "\"INTERVAL\"")
)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -163,9 +163,9 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSparkSession
exception = intercept[SparkUnsupportedOperationException] {
Seq(InvalidInJava(1)).toDS()
},
condition = "_LEGACY_ERROR_TEMP_2140",
condition = "INVALID_JAVA_IDENTIFIER_AS_FIELD_NAME",
parameters = Map(
"fieldName" -> "abstract",
"fieldName" -> "`abstract`",
"walkedTypePath" -> "- root class: \"org.apache.spark.sql.InvalidInJava\""))
}

Expand All @@ -174,9 +174,9 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSparkSession
exception = intercept[SparkUnsupportedOperationException] {
Seq(InvalidInJava2(1)).toDS()
},
condition = "_LEGACY_ERROR_TEMP_2140",
condition = "INVALID_JAVA_IDENTIFIER_AS_FIELD_NAME",
parameters = Map(
"fieldName" -> "0",
"fieldName" -> "`0`",
"walkedTypePath" ->
"- root class: \"org.apache.spark.sql.ScalaReflectionRelationSuite.InvalidInJava2\""))
}
Expand Down

0 comments on commit 5a12501

Please sign in to comment.