Skip to content

Commit

Permalink
solve compatability issue
Browse files Browse the repository at this point in the history
  • Loading branch information
zhzhan committed Oct 7, 2014
1 parent f7912a9 commit 20f6cf7
Show file tree
Hide file tree
Showing 3 changed files with 17 additions and 6 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -293,7 +293,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {

proc match {
case driver: Driver =>
val results = new JArrayList[String]
val results = HiveShim.createDriverResultsArray
val response: CommandProcessorResponse = driver.run(cmd)
// Throw an exception if there is an error in query processing.
if (response.getResponseCode != 0) {
Expand All @@ -303,7 +303,12 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
driver.setMaxRows(maxRows)
driver.getResults(results)
driver.close()
results
results.map { r =>
r match {
case s: String => s
case o => o.toString
}
}
case _ =>
sessionState.out.println(tokens(0) + " " + cmd_1)
Seq(proc.run(cmd_1).getResponseCode.toString)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.spark.sql.hive

import java.net.URI
import java.util.{ArrayList => JArrayList}
import java.util.Properties
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
Expand Down Expand Up @@ -49,14 +50,16 @@ private[hive] object HiveShim {
new TableDesc(serdeClass, inputFormatClass, outputFormatClass, properties)
}

def createDriverResultsArray = new JArrayList[String]

def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE

def createDefaultDBIfNeeded(context: HiveContext) ={ }
def createDefaultDBIfNeeded(context: HiveContext) = { }

/** The string used to denote an empty comments field in the schema. */
def getEmptyCommentsFieldValue = "None"

def getCommandProcessor(cmd: Array[String], conf: HiveConf) = {
def getCommandProcessor(cmd: Array[String], conf: HiveConf) = {
CommandProcessorFactory.get(cmd(0), conf)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.spark.sql.hive

import java.util.{ArrayList => JArrayList}
import java.util.Properties
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
Expand Down Expand Up @@ -53,17 +54,19 @@ private[hive] object HiveShim {
new TableDesc(inputFormatClass, outputFormatClass, properties)
}

def createDriverResultsArray = new JArrayList[Object]

def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE

def createDefaultDBIfNeeded(context: HiveContext) ={
def createDefaultDBIfNeeded(context: HiveContext) = {
context.runSqlHive("CREATE DATABASE default")
context.runSqlHive("USE default")
}

/* The string used to denote an empty comments field in the schema. */
def getEmptyCommentsFieldValue = ""

def getCommandProcessor(cmd: Array[String], conf: HiveConf) = {
def getCommandProcessor(cmd: Array[String], conf: HiveConf) = {
CommandProcessorFactory.get(cmd, conf)
}

Expand Down

0 comments on commit 20f6cf7

Please sign in to comment.