Skip to content

Commit 33a2b0a

Browse files
committed
compile fix accordng to new changes of inernalrow
1 parent 7dc9272 commit 33a2b0a

File tree

3 files changed

+5
-1
lines changed

3 files changed

+5
-1
lines changed

sql/hive/src/main/scala/org/apache/spark/sql/hive/SaveAsHiveFile.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@ import org.apache.spark.rdd.RDD
2929
import org.apache.spark.sql.catalyst.InternalRow
3030
import org.apache.spark.util.SerializableJobConf
3131
import org.apache.spark.sql.types.StructType
32+
import org.apache.spark.sql.types.DataType
3233
import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc}
3334
import org.apache.spark.{Logging, SparkContext, TaskContext}
3435

@@ -46,6 +47,7 @@ private[hive] trait SaveAsHiveFile extends HiveInspectors with Logging {
4647
sparkContext: SparkContext,
4748
rdd: RDD[InternalRow],
4849
schema: StructType,
50+
dataTypes: Array[DataType],
4951
valueClass: Class[_],
5052
fileSinkConf: FileSinkDesc,
5153
conf: SerializableJobConf,
@@ -84,7 +86,7 @@ private[hive] trait SaveAsHiveFile extends HiveInspectors with Logging {
8486
iterator.foreach { row =>
8587
var i = 0
8688
while (i < fieldOIs.length) {
87-
outputData(i) = if (row.isNullAt(i)) null else wrappers(i)(row(i))
89+
outputData(i) = if (row.isNullAt(i)) null else wrappers(i)(row.get(i, dataTypes(i)))
8890
i += 1
8991
}
9092

sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -124,6 +124,7 @@ case class InsertIntoHiveTable(
124124
sc.sparkContext,
125125
child.execute(),
126126
table.schema,
127+
child.output.map(_.dataType).toArray,
127128
outputClass,
128129
fileSinkConf,
129130
jobConfSer,

sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/WriteToDirectory.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -94,6 +94,7 @@ case class WriteToDirectory(
9494
hiveContext.sparkContext,
9595
child.execute(),
9696
StructType.fromAttributes(output),
97+
child.output.map(_.dataType).toArray,
9798
outputClass,
9899
fileSinkConf,
99100
jobConfSer,

0 commit comments

Comments
 (0)