File tree Expand file tree Collapse file tree 3 files changed +12
-6
lines changed
src/main/scala/org/apache/spark/sql/hive
v0.12.0/src/main/scala/org/apache/spark/sql/hive
v0.13.1/src/main/scala/org/apache/spark/sql/hive Expand file tree Collapse file tree 3 files changed +12
-6
lines changed Original file line number Diff line number Diff line change @@ -304,12 +304,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
304304 driver.setMaxRows(maxRows)
305305 driver.getResults(results)
306306 driver.close()
307- results.map { r =>
308- r match {
309- case s : String => s
310- case o => o.toString
311- }
312- }
307+ HiveShim .processResults(results)
313308 case _ =>
314309 sessionState.out.println(tokens(0 ) + " " + cmd_1)
315310 Seq (proc.run(cmd_1).getResponseCode.toString)
Original file line number Diff line number Diff line change @@ -52,6 +52,8 @@ private[hive] object HiveShim {
5252
5353 def createDriverResultsArray = new JArrayList [String ]
5454
55+ def processResults (results : JArrayList [String ]) = results
56+
5557 def getStatsSetupConstTotalSize = StatsSetupConst .TOTAL_SIZE
5658
5759 def createDefaultDBIfNeeded (context : HiveContext ) = { }
Original file line number Diff line number Diff line change @@ -56,6 +56,15 @@ private[hive] object HiveShim {
5656
5757 def createDriverResultsArray = new JArrayList [Object ]
5858
59+ def processResults (results : JArrayList [Object ]) = {
60+ results.map { r =>
61+ r match {
62+ case s : String => s
63+ case a : Array [Object ] => a(0 ).asInstanceOf [String ]
64+ }
65+ }
66+ }
67+
5968 def getStatsSetupConstTotalSize = StatsSetupConst .TOTAL_SIZE
6069
6170 def createDefaultDBIfNeeded (context : HiveContext ) = {
You can’t perform that action at this time.
0 commit comments