Repository: spark
Updated Branches:
  refs/heads/branch-1.3 c96458889 -> 0021d2260


[SPARK-6428][SQL] Added explicit type for all public methods for Hive module

Author: Reynold Xin <[email protected]>

Closes #5108 from rxin/hive-public-type and squashes the following commits:

a320328 [Reynold Xin] [SPARK-6428][SQL] Added explicit type for all public 
methods for Hive module.

(cherry picked from commit b6090f902e6ec24923b4dde4aabc9076956521c1)
Signed-off-by: Michael Armbrust <[email protected]>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0021d226
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0021d226
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0021d226

Branch: refs/heads/branch-1.3
Commit: 0021d226061f2fe1896b2fb16e2ae0bdde880ff5
Parents: c964588
Author: Reynold Xin <[email protected]>
Authored: Sat Mar 21 14:30:04 2015 -0700
Committer: Michael Armbrust <[email protected]>
Committed: Sat Mar 21 14:30:14 2015 -0700

----------------------------------------------------------------------
 .../hive/thriftserver/SparkSQLCLIDriver.scala   |  4 +-
 .../org/apache/spark/sql/hive/HiveContext.scala |  6 +--
 .../spark/sql/hive/HiveMetastoreCatalog.scala   | 21 ++++-----
 .../org/apache/spark/sql/hive/HiveQl.scala      | 10 ++---
 .../apache/spark/sql/hive/HiveStrategies.scala  |  6 +--
 .../org/apache/spark/sql/hive/TableReader.scala |  2 +-
 .../hive/execution/CreateTableAsSelect.scala    |  2 +-
 .../execution/DescribeHiveTableCommand.scala    |  2 +-
 .../sql/hive/execution/HiveNativeCommand.scala  |  4 +-
 .../sql/hive/execution/HiveTableScan.scala      |  5 ++-
 .../hive/execution/InsertIntoHiveTable.scala    |  4 +-
 .../hive/execution/ScriptTransformation.scala   |  5 ++-
 .../spark/sql/hive/execution/commands.scala     |  8 ++--
 .../org/apache/spark/sql/hive/hiveUdfs.scala    | 45 ++++++++++++--------
 .../spark/sql/hive/hiveWriterContainers.scala   |  4 +-
 .../apache/spark/sql/hive/test/TestHive.scala   | 13 ++++--
 16 files changed, 79 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/0021d226/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
index 401e97b..6e3c92f 100644
--- 
a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
+++ 
b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala
@@ -194,8 +194,8 @@ private[hive] object SparkSQLCLIDriver {
     val currentDB = ReflectionUtils.invokeStatic(classOf[CliDriver], 
"getFormattedDb",
       classOf[HiveConf] -> conf, classOf[CliSessionState] -> sessionState)
 
-    def promptWithCurrentDB = s"$prompt$currentDB"
-    def continuedPromptWithDBSpaces = continuedPrompt + 
ReflectionUtils.invokeStatic(
+    def promptWithCurrentDB: String = s"$prompt$currentDB"
+    def continuedPromptWithDBSpaces: String = continuedPrompt + 
ReflectionUtils.invokeStatic(
       classOf[CliDriver], "spacesForString", classOf[String] -> currentDB)
 
     var currentPrompt = promptWithCurrentDB

http://git-wip-us.apache.org/repos/asf/spark/blob/0021d226/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index c439dfe..da4cf24 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -187,7 +187,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
 
   // Circular buffer to hold what hive prints to STDOUT and ERR.  Only printed 
when failures occur.
   @transient
-  protected lazy val outputBuffer =  new java.io.OutputStream {
+  protected lazy val outputBuffer = new java.io.OutputStream {
     var pos: Int = 0
     var buffer = new Array[Int](10240)
     def write(i: Int): Unit = {
@@ -195,7 +195,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
       pos = (pos + 1) % buffer.size
     }
 
-    override def toString = {
+    override def toString: String = {
       val (end, start) = buffer.splitAt(pos)
       val input = new java.io.InputStream {
         val iterator = (start ++ end).iterator
@@ -255,7 +255,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
   @transient
   override protected[sql] lazy val functionRegistry =
     new HiveFunctionRegistry with OverrideFunctionRegistry {
-      def caseSensitive = false
+      def caseSensitive: Boolean = false
     }
 
   /* An analyzer that uses the Hive metastore. */

http://git-wip-us.apache.org/repos/asf/spark/blob/0021d226/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index 949a4e5..4c5eb48 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -58,7 +58,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) 
extends Catalog with
   // TODO: Use this everywhere instead of tuples or databaseName, tableName,.
   /** A fully qualified identifier for a table (i.e., database.tableName) */
   case class QualifiedTableName(database: String, name: String) {
-    def toLowerCase = QualifiedTableName(database.toLowerCase, 
name.toLowerCase)
+    def toLowerCase: QualifiedTableName = 
QualifiedTableName(database.toLowerCase, name.toLowerCase)
   }
 
   /** A cache of Spark SQL data source tables that have been accessed. */
@@ -629,7 +629,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) 
extends Catalog with
         castChildOutput(p, table, child)
     }
 
-    def castChildOutput(p: InsertIntoTable, table: MetastoreRelation, child: 
LogicalPlan) = {
+    def castChildOutput(p: InsertIntoTable, table: MetastoreRelation, child: 
LogicalPlan)
+      : LogicalPlan = {
       val childOutputDataTypes = child.output.map(_.dataType)
       val tableOutputDataTypes =
         (table.attributes ++ 
table.partitionKeys).take(child.output.length).map(_.dataType)
@@ -667,7 +668,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) 
extends Catalog with
    */
   override def unregisterTable(tableIdentifier: Seq[String]): Unit = ???
 
-  override def unregisterAllTables() = {}
+  override def unregisterAllTables(): Unit = {}
 }
 
 /**
@@ -682,10 +683,10 @@ private[hive] case class InsertIntoHiveTable(
     overwrite: Boolean)
   extends LogicalPlan {
 
-  override def children = child :: Nil
-  override def output = child.output
+  override def children: Seq[LogicalPlan] = child :: Nil
+  override def output: Seq[Attribute] = child.output
 
-  override lazy val resolved = childrenResolved && 
child.output.zip(table.output).forall {
+  override lazy val resolved: Boolean = childrenResolved && 
child.output.zip(table.output).forall {
     case (childAttr, tableAttr) => 
childAttr.dataType.sameType(tableAttr.dataType)
   }
 }
@@ -704,13 +705,13 @@ private[hive] case class MetastoreRelation
   // org.apache.hadoop.hive.ql.metadata.Partition will cause a 
NotSerializableException
   // which indicates the SerDe we used is not Serializable.
 
-  @transient val hiveQlTable = new Table(table)
+  @transient val hiveQlTable: Table = new Table(table)
 
-  @transient val hiveQlPartitions = partitions.map { p =>
+  @transient val hiveQlPartitions: Seq[Partition] = partitions.map { p =>
     new Partition(hiveQlTable, p)
   }
 
-  @transient override lazy val statistics = Statistics(
+  @transient override lazy val statistics: Statistics = Statistics(
     sizeInBytes = {
       val totalSize = 
hiveQlTable.getParameters.get(HiveShim.getStatsSetupConstTotalSize)
       val rawDataSize = 
hiveQlTable.getParameters.get(HiveShim.getStatsSetupConstRawDataSize)
@@ -754,7 +755,7 @@ private[hive] case class MetastoreRelation
   )
 
   implicit class SchemaAttribute(f: FieldSchema) {
-    def toAttribute = AttributeReference(
+    def toAttribute: AttributeReference = AttributeReference(
       f.getName,
       HiveMetastoreTypes.toDataType(f.getType),
       // Since data can be dumped in randomly with no validation, everything 
is nullable.

http://git-wip-us.apache.org/repos/asf/spark/blob/0021d226/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
index ced99cd..51775eb 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
@@ -196,8 +196,8 @@ private[hive] object HiveQl {
      * Right now this function only checks the name, type, text and children 
of the node
      * for equality.
      */
-    def checkEquals(other: ASTNode) {
-      def check(field: String, f: ASTNode => Any) = if (f(n) != f(other)) {
+    def checkEquals(other: ASTNode): Unit = {
+      def check(field: String, f: ASTNode => Any): Unit = if (f(n) != 
f(other)) {
         sys.error(s"$field does not match for trees. " +
           s"'${f(n)}' != '${f(other)}' left: ${dumpTree(n)}, right: 
${dumpTree(other)}")
       }
@@ -209,7 +209,7 @@ private[hive] object HiveQl {
       val leftChildren = nilIfEmpty(n.getChildren).asInstanceOf[Seq[ASTNode]]
       val rightChildren = 
nilIfEmpty(other.getChildren).asInstanceOf[Seq[ASTNode]]
       leftChildren zip rightChildren foreach {
-        case (l,r) => l checkEquals r
+        case (l, r) => l checkEquals r
       }
     }
   }
@@ -269,7 +269,7 @@ private[hive] object HiveQl {
   }
 
   /** Creates LogicalPlan for a given VIEW */
-  def createPlanForView(view: Table, alias: Option[String]) = alias match {
+  def createPlanForView(view: Table, alias: Option[String]): Subquery = alias 
match {
     // because hive use things like `_c0` to build the expanded text
     // currently we cannot support view from "create view v1(c1) as ..."
     case None => Subquery(view.getTableName, 
createPlan(view.getViewExpandedText))
@@ -323,7 +323,7 @@ private[hive] object HiveQl {
     clauses
   }
 
-  def getClause(clauseName: String, nodeList: Seq[Node]) =
+  def getClause(clauseName: String, nodeList: Seq[Node]): Node =
     getClauseOption(clauseName, nodeList).getOrElse(sys.error(
       s"Expected clause $clauseName missing from 
${nodeList.map(dumpTree(_)).mkString("\n")}"))
 

http://git-wip-us.apache.org/repos/asf/spark/blob/0021d226/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
index e63cea6..5f7e897 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
@@ -58,9 +58,9 @@ private[hive] trait HiveStrategies {
   @Experimental
   object ParquetConversion extends Strategy {
     implicit class LogicalPlanHacks(s: DataFrame) {
-      def lowerCase = DataFrame(s.sqlContext, s.logicalPlan)
+      def lowerCase: DataFrame = DataFrame(s.sqlContext, s.logicalPlan)
 
-      def addPartitioningAttributes(attrs: Seq[Attribute]) = {
+      def addPartitioningAttributes(attrs: Seq[Attribute]): DataFrame = {
         // Don't add the partitioning key if its already present in the data.
         if 
(attrs.map(_.name).toSet.subsetOf(s.logicalPlan.output.map(_.name).toSet)) {
           s
@@ -75,7 +75,7 @@ private[hive] trait HiveStrategies {
     }
 
     implicit class PhysicalPlanHacks(originalPlan: SparkPlan) {
-      def fakeOutput(newOutput: Seq[Attribute]) =
+      def fakeOutput(newOutput: Seq[Attribute]): OutputFaker =
         OutputFaker(
           originalPlan.output.map(a =>
             newOutput.find(a.name.toLowerCase == _.name.toLowerCase)

http://git-wip-us.apache.org/repos/asf/spark/blob/0021d226/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
index f22c9ea..af309c0 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
@@ -175,7 +175,7 @@ class HadoopTableReader(
           relation.partitionKeys.contains(attr)
         }
 
-      def fillPartitionKeys(rawPartValues: Array[String], row: MutableRow) = {
+      def fillPartitionKeys(rawPartValues: Array[String], row: MutableRow): 
Unit = {
         partitionKeyAttrs.foreach { case (attr, ordinal) =>
           val partOrdinal = relation.partitionKeys.indexOf(attr)
           row(ordinal) = Cast(Literal(rawPartValues(partOrdinal)), 
attr.dataType).eval(null)

http://git-wip-us.apache.org/repos/asf/spark/blob/0021d226/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
index a0c91cb..fade9e5 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
@@ -45,7 +45,7 @@ case class CreateTableAsSelect(
     allowExisting: Boolean,
     desc: Option[CreateTableDesc]) extends RunnableCommand {
 
-  override def run(sqlContext: SQLContext) = {
+  override def run(sqlContext: SQLContext): Seq[Row] = {
     val hiveContext = sqlContext.asInstanceOf[HiveContext]
     lazy val metastoreRelation: MetastoreRelation = {
       // Create Hive Table

http://git-wip-us.apache.org/repos/asf/spark/blob/0021d226/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
index d0510aa..6fce69b 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
@@ -37,7 +37,7 @@ case class DescribeHiveTableCommand(
     override val output: Seq[Attribute],
     isExtended: Boolean) extends RunnableCommand {
 
-  override def run(sqlContext: SQLContext) = {
+  override def run(sqlContext: SQLContext): Seq[Row] = {
     // Trying to mimic the format of Hive's output. But not exactly the same.
     var results: Seq[(String, String, String)] = Nil
 

http://git-wip-us.apache.org/repos/asf/spark/blob/0021d226/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
index 9636da2..60a9bb6 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveNativeCommand.scala
@@ -26,9 +26,9 @@ import org.apache.spark.sql.types.StringType
 private[hive]
 case class HiveNativeCommand(sql: String) extends RunnableCommand {
 
-  override def output =
+  override def output: Seq[AttributeReference] =
     Seq(AttributeReference("result", StringType, nullable = false)())
 
-  override def run(sqlContext: SQLContext) =
+  override def run(sqlContext: SQLContext): Seq[Row] =
     sqlContext.asInstanceOf[HiveContext].runSqlHive(sql).map(Row(_))
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0021d226/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
index 5b3cf28..0a5f19e 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.serde2.objectinspector._
 import 
org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils
 
+import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.execution._
 import org.apache.spark.sql.hive._
@@ -128,11 +129,11 @@ case class HiveTableScan(
     }
   }
 
-  override def execute() = if (!relation.hiveQlTable.isPartitioned) {
+  override def execute(): RDD[Row] = if (!relation.hiveQlTable.isPartitioned) {
     hadoopReader.makeRDDForTable(relation.hiveQlTable)
   } else {
     
hadoopReader.makeRDDForPartitionedTable(prunePartitions(relation.hiveQlPartitions))
   }
 
-  override def output = attributes
+  override def output: Seq[Attribute] = attributes
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0021d226/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
index ba5c8e0..da53d30 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
@@ -33,7 +33,7 @@ import org.apache.hadoop.hive.serde2.objectinspector._
 import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, 
JobConf}
 
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.expressions.Row
+import org.apache.spark.sql.catalyst.expressions.{Attribute, Row}
 import org.apache.spark.sql.execution.{UnaryNode, SparkPlan}
 import org.apache.spark.sql.hive._
 import org.apache.spark.sql.hive.{ ShimFileSinkDesc => FileSinkDesc}
@@ -58,7 +58,7 @@ case class InsertIntoHiveTable(
     serializer
   }
 
-  def output = child.output
+  def output: Seq[Attribute] = child.output
 
   def saveAsHiveFile(
       rdd: RDD[Row],

http://git-wip-us.apache.org/repos/asf/spark/blob/0021d226/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
index 0c9aee3..8efed7f 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.serde.serdeConstants
 import org.apache.hadoop.hive.serde2.AbstractSerDe
 import org.apache.hadoop.hive.serde2.objectinspector._
 
+import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema
 import org.apache.spark.sql.execution._
@@ -51,9 +52,9 @@ case class ScriptTransformation(
     ioschema: HiveScriptIOSchema)(@transient sc: HiveContext)
   extends UnaryNode {
 
-  override def otherCopyArgs = sc :: Nil
+  override def otherCopyArgs: Seq[HiveContext] = sc :: Nil
 
-  def execute() = {
+  def execute(): RDD[Row] = {
     child.execute().mapPartitions { iter =>
       val cmd = List("/bin/bash", "-c", script)
       val builder = new ProcessBuilder(cmd)

http://git-wip-us.apache.org/repos/asf/spark/blob/0021d226/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
index 63ad145..4345ffb 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
@@ -38,7 +38,7 @@ import org.apache.spark.sql.types.StructType
 private[hive]
 case class AnalyzeTable(tableName: String) extends RunnableCommand {
 
-  override def run(sqlContext: SQLContext) = {
+  override def run(sqlContext: SQLContext): Seq[Row] = {
     sqlContext.asInstanceOf[HiveContext].analyze(tableName)
     Seq.empty[Row]
   }
@@ -52,7 +52,7 @@ case class DropTable(
     tableName: String,
     ifExists: Boolean) extends RunnableCommand {
 
-  override def run(sqlContext: SQLContext) = {
+  override def run(sqlContext: SQLContext): Seq[Row] = {
     val hiveContext = sqlContext.asInstanceOf[HiveContext]
     val ifExistsClause = if (ifExists) "IF EXISTS " else ""
     try {
@@ -75,7 +75,7 @@ case class DropTable(
 private[hive]
 case class AddJar(path: String) extends RunnableCommand {
 
-  override def run(sqlContext: SQLContext) = {
+  override def run(sqlContext: SQLContext): Seq[Row] = {
     val hiveContext = sqlContext.asInstanceOf[HiveContext]
     hiveContext.runSqlHive(s"ADD JAR $path")
     hiveContext.sparkContext.addJar(path)
@@ -86,7 +86,7 @@ case class AddJar(path: String) extends RunnableCommand {
 private[hive]
 case class AddFile(path: String) extends RunnableCommand {
 
-  override def run(sqlContext: SQLContext) = {
+  override def run(sqlContext: SQLContext): Seq[Row] = {
     val hiveContext = sqlContext.asInstanceOf[HiveContext]
     hiveContext.runSqlHive(s"ADD FILE $path")
     hiveContext.sparkContext.addFile(path)

http://git-wip-us.apache.org/repos/asf/spark/blob/0021d226/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
index 34c21c1..d27e781 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
@@ -45,7 +45,7 @@ import scala.collection.JavaConversions._
 private[hive] abstract class HiveFunctionRegistry
   extends analysis.FunctionRegistry with HiveInspectors {
 
-  def getFunctionInfo(name: String) = FunctionRegistry.getFunctionInfo(name)
+  def getFunctionInfo(name: String): FunctionInfo = 
FunctionRegistry.getFunctionInfo(name)
 
   def lookupFunction(name: String, children: Seq[Expression]): Expression = {
     // We only look it up to see if it exists, but do not include it in the 
HiveUDF since it is
@@ -78,7 +78,7 @@ private[hive] case class HiveSimpleUdf(funcWrapper: 
HiveFunctionWrapper, childre
   type EvaluatedType = Any
   type UDFType = UDF
 
-  def nullable = true
+  override def nullable: Boolean = true
 
   @transient
   lazy val function = funcWrapper.createFunction[UDFType]()
@@ -96,7 +96,7 @@ private[hive] case class HiveSimpleUdf(funcWrapper: 
HiveFunctionWrapper, childre
     udfType != null && udfType.deterministic()
   }
 
-  override def foldable = isUDFDeterministic && children.forall(_.foldable)
+  override def foldable: Boolean = isUDFDeterministic && 
children.forall(_.foldable)
 
   // Create parameter converters
   @transient
@@ -110,7 +110,7 @@ private[hive] case class HiveSimpleUdf(funcWrapper: 
HiveFunctionWrapper, childre
     method.getGenericReturnType(), ObjectInspectorOptions.JAVA)
 
   @transient
-  protected lazy val cached = new Array[AnyRef](children.length)
+  protected lazy val cached: Array[AnyRef] = new Array[AnyRef](children.length)
 
   // TODO: Finish input output types.
   override def eval(input: Row): Any = {
@@ -120,17 +120,19 @@ private[hive] case class HiveSimpleUdf(funcWrapper: 
HiveFunctionWrapper, childre
       returnInspector)
   }
 
-  override def toString = 
s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})"
+  override def toString: String = {
+    s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})"
+  }
 }
 
 // Adapter from Catalyst ExpressionResult to Hive DeferredObject
 private[hive] class DeferredObjectAdapter(oi: ObjectInspector)
   extends DeferredObject with HiveInspectors {
   private var func: () => Any = _
-  def set(func: () => Any) {
+  def set(func: () => Any): Unit = {
     this.func = func
   }
-  override def prepare(i: Int) = {}
+  override def prepare(i: Int): Unit = {}
   override def get(): AnyRef = wrap(func(), oi)
 }
 
@@ -139,7 +141,7 @@ private[hive] case class HiveGenericUdf(funcWrapper: 
HiveFunctionWrapper, childr
   type UDFType = GenericUDF
   type EvaluatedType = Any
 
-  def nullable = true
+  override def nullable: Boolean = true
 
   @transient
   lazy val function = funcWrapper.createFunction[UDFType]()
@@ -158,7 +160,7 @@ private[hive] case class HiveGenericUdf(funcWrapper: 
HiveFunctionWrapper, childr
     (udfType != null && udfType.deterministic())
   }
 
-  override def foldable =
+  override def foldable: Boolean =
     isUDFDeterministic && returnInspector.isInstanceOf[ConstantObjectInspector]
 
   @transient
@@ -182,7 +184,9 @@ private[hive] case class HiveGenericUdf(funcWrapper: 
HiveFunctionWrapper, childr
     unwrap(function.evaluate(deferedObjects), returnInspector)
   }
 
-  override def toString = 
s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})"
+  override def toString: String = {
+    s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})"
+  }
 }
 
 private[hive] case class HiveGenericUdaf(
@@ -209,9 +213,11 @@ private[hive] case class HiveGenericUdaf(
 
   def nullable: Boolean = true
 
-  override def toString = 
s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})"
+  override def toString: String = {
+    s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})"
+  }
 
-  def newInstance() = new HiveUdafFunction(funcWrapper, children, this)
+  def newInstance(): HiveUdafFunction = new HiveUdafFunction(funcWrapper, 
children, this)
 }
 
 /** It is used as a wrapper for the hive functions which uses UDAF interface */
@@ -240,10 +246,11 @@ private[hive] case class HiveUdaf(
 
   def nullable: Boolean = true
 
-  override def toString = 
s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})"
+  override def toString: String = {
+    s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})"
+  }
 
-  def newInstance() =
-    new HiveUdafFunction(funcWrapper, children, this, true)
+  def newInstance(): HiveUdafFunction = new HiveUdafFunction(funcWrapper, 
children, this, true)
 }
 
 /**
@@ -314,21 +321,23 @@ private[hive] case class HiveGenericUdtf(
       collected += unwrap(input, outputInspector).asInstanceOf[Row]
     }
 
-    def collectRows() = {
+    def collectRows(): Seq[Row] = {
       val toCollect = collected
       collected = new ArrayBuffer[Row]
       toCollect
     }
   }
 
-  override def toString = 
s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})"
+  override def toString: String = {
+    s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})"
+  }
 }
 
 /**
  * Resolve Udtfs Alias.
  */
 private[spark] object ResolveUdtfsAlias extends Rule[LogicalPlan] {
-  def apply(plan: LogicalPlan) = plan transform {
+  def apply(plan: LogicalPlan): LogicalPlan = plan transform {
     case p @ Project(projectList, _)
       if projectList.exists(_.isInstanceOf[MultiAlias]) && projectList.size != 
1 =>
       throw new TreeNodeException(p, "only single Generator supported for 
SELECT clause")

http://git-wip-us.apache.org/repos/asf/spark/blob/0021d226/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala
index f136e43..ba2bf67 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala
@@ -222,7 +222,7 @@ private[spark] class 
SparkHiveDynamicPartitionWriterContainer(
         s"/$col=$colString"
       }.mkString
 
-    def newWriter = {
+    def newWriter(): FileSinkOperator.RecordWriter = {
       val newFileSinkDesc = new FileSinkDesc(
         fileSinkConf.getDirName + dynamicPartPath,
         fileSinkConf.getTableInfo,
@@ -246,6 +246,6 @@ private[spark] class 
SparkHiveDynamicPartitionWriterContainer(
         Reporter.NULL)
     }
 
-    writers.getOrElseUpdate(dynamicPartPath, newWriter)
+    writers.getOrElseUpdate(dynamicPartPath, newWriter())
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/0021d226/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
index a2d99f1..3380ef0 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
@@ -153,8 +153,8 @@ class TestHiveContext(sc: SparkContext) extends 
HiveContext(sc) {
 
   protected[hive] class HiveQLQueryExecution(hql: String)
     extends this.QueryExecution(HiveQl.parseSql(hql)) {
-    def hiveExec() = runSqlHive(hql)
-    override def toString = hql + "\n" + super.toString
+    def hiveExec(): Seq[String] = runSqlHive(hql)
+    override def toString: String = hql + "\n" + super.toString
   }
 
   /**
@@ -184,7 +184,9 @@ class TestHiveContext(sc: SparkContext) extends 
HiveContext(sc) {
   case class TestTable(name: String, commands: (()=>Unit)*)
 
   protected[hive] implicit class SqlCmd(sql: String) {
-    def cmd = () => new HiveQLQueryExecution(sql).stringResult(): Unit
+    def cmd: () => Unit = {
+      () => new HiveQLQueryExecution(sql).stringResult(): Unit
+    }
   }
 
   /**
@@ -192,7 +194,10 @@ class TestHiveContext(sc: SparkContext) extends 
HiveContext(sc) {
    * demand when a query are run against it.
    */
   lazy val testTables = new mutable.HashMap[String, TestTable]()
-  def registerTestTable(testTable: TestTable) = testTables += (testTable.name 
-> testTable)
+
+  def registerTestTable(testTable: TestTable): Unit = {
+    testTables += (testTable.name -> testTable)
+  }
 
   // The test tables that are defined in the Hive QTestUtil.
   // /itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to