@@ -69,6 +69,7 @@ import org.apache.spark.sql.internal.SQLConf
69
69
import org .apache .spark .sql .types ._
70
70
import org .apache .spark .sql .util .CaseInsensitiveStringMap
71
71
72
+
72
73
/**
73
74
* Analysis rules for Delta. Currently, these rules enable schema enforcement / evolution with
74
75
* INSERT INTO.
@@ -930,6 +931,18 @@ class DeltaAnalysis(session: SparkSession)
930
931
// Keep the type from the query, the target schema will be updated to widen the existing
931
932
// type to match it.
932
933
attr
934
+ case (s : MapType , t : MapType ) if s != t =>
935
+ // Handle only specific cases where the value type of the MapType is a StructType
936
+ // This could be revisited and expanded in the future when needs for more
937
+ // nested complex operations are desired vis-a-vis ALTER TABLE COLUMN operations
938
+ // for deep nested fields
939
+ (s.valueType, t.valueType) match {
940
+ case (structS : StructType , structT : StructType ) if structS != structT =>
941
+ addCastsToMaps(tblName, attr, s, t, allowTypeWidening)
942
+ case _ =>
943
+ // Default for all other MapType cases
944
+ getCastFunction(attr, targetAttr.dataType, targetAttr.name)
945
+ }
933
946
case _ =>
934
947
getCastFunction(attr, targetAttr.dataType, targetAttr.name)
935
948
}
@@ -1049,6 +1062,7 @@ class DeltaAnalysis(session: SparkSession)
1049
1062
/**
1050
1063
* Recursively casts structs in case it contains null types.
1051
1064
* TODO: Support other complex types like MapType and ArrayType
1065
+ * The case mapType that calls addCastsToMaps addresses the MapType todo
1052
1066
*/
1053
1067
private def addCastsToStructs (
1054
1068
tableName : String ,
@@ -1067,6 +1081,8 @@ class DeltaAnalysis(session: SparkSession)
1067
1081
val subField = Alias (GetStructField (parent, i, Option (name)), target(i).name)(
1068
1082
explicitMetadata = Option (metadata))
1069
1083
addCastsToStructs(tableName, subField, nested, t, allowTypeWidening)
1084
+ // We could also handle maptype within struct here but there is restriction
1085
+ // on deep nexted operations that may result in maxIteration error
1070
1086
case o =>
1071
1087
val field = parent.qualifiedName + " ." + name
1072
1088
val targetName = parent.qualifiedName + " ." + target(i).name
@@ -1124,6 +1140,48 @@ class DeltaAnalysis(session: SparkSession)
1124
1140
DeltaViewHelper .stripTempViewForMerge(plan, conf)
1125
1141
}
1126
1142
1143
+ /**
1144
+ * Recursively casts maps in case it contains null types.
1145
+ */
1146
+ private def addCastsToMaps (
1147
+ tableName : String ,
1148
+ parent : NamedExpression ,
1149
+ sourceMapType : MapType ,
1150
+ targetMapType : MapType ,
1151
+ allowTypeWidening : Boolean ): Expression = {
1152
+ // First get keys from the map
1153
+ val keysExpr = MapKeys (parent)
1154
+
1155
+ // Create a transformation for the values
1156
+ val transformLambdaFunc = {
1157
+ val elementVar = NamedLambdaVariable (
1158
+ " elementVar" , sourceMapType.valueType, sourceMapType.valueContainsNull)
1159
+ val castedValue = sourceMapType.valueType match {
1160
+ case structType : StructType =>
1161
+ // Handle StructType values
1162
+ addCastsToStructs(
1163
+ tableName,
1164
+ elementVar,
1165
+ structType,
1166
+ targetMapType.valueType.asInstanceOf [StructType ],
1167
+ allowTypeWidening
1168
+ )
1169
+ case _ =>
1170
+ // Not expected to get here: see addCastsToColumn
1171
+ throw new IllegalArgumentException (
1172
+ s " Target type must be a StructType " )
1173
+ }
1174
+
1175
+ LambdaFunction (castedValue, Seq (elementVar))
1176
+ }
1177
+
1178
+ val transformedValues = ArrayTransform (
1179
+ MapValues (parent), transformLambdaFunc)
1180
+
1181
+ // Create new map from keys and transformed values
1182
+ MapFromArrays (keysExpr, transformedValues)
1183
+ }
1184
+
1127
1185
/**
1128
1186
* Verify the input plan for a SINGLE streaming query with the following:
1129
1187
* 1. Schema location must be under checkpoint location, if not lifted by flag
0 commit comments