diff --git a/pom.xml b/pom.xml
index 70cb9729ff6d..bb87ab106960 100644
--- a/pom.xml
+++ b/pom.xml
@@ -127,7 +127,7 @@
0.94.6
1.4.0
3.4.5
- 0.12.0
+ 0.13.1
1.4.3
1.2.3
8.1.14.v20131031
@@ -428,7 +428,7 @@
org.apache.derby
derby
- 10.4.2.0
+ 10.10.1.1
com.codahale.metrics
diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml
index 124fc107cb8a..2570a9fbdc66 100644
--- a/sql/hive-thriftserver/pom.xml
+++ b/sql/hive-thriftserver/pom.xml
@@ -42,17 +42,17 @@
${project.version}
- org.spark-project.hive
+ org.apache.hive
hive-cli
${hive.version}
- org.spark-project.hive
+ org.apache.hive
hive-jdbc
${hive.version}
- org.spark-project.hive
+ org.apache.hive
hive-beeline
${hive.version}
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 7ba4564602ec..4960eab06965 100755
--- 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
@@ -116,7 +116,7 @@ private[hive] object SparkSQLCLIDriver {
}
}
- if (!sessionState.isRemoteMode && !ShimLoader.getHadoopShims.usesJobShell()) {
+ if (!sessionState.isRemoteMode) {
// Hadoop-20 and above - we need to augment classpath using hiveconf
// components.
// See also: code in ExecDriver.java
@@ -258,7 +258,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
} else {
var ret = 0
val hconf = conf.asInstanceOf[HiveConf]
- val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hconf)
+ val proc: CommandProcessor = CommandProcessorFactory.get(Array(tokens(0)), hconf)
if (proc != null) {
if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor]) {
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala
index 42cbf363b274..4e518938d625 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala
@@ -44,12 +44,12 @@ private[hive] class SparkSQLCLIService(hiveContext: HiveContext)
val sparkSqlSessionManager = new SparkSQLSessionManager(hiveContext)
setSuperField(this, "sessionManager", sparkSqlSessionManager)
addService(sparkSqlSessionManager)
+ val authentication = "hive.server2.authentication"
try {
- HiveAuthFactory.loginFromKeytab(hiveConf)
- val serverUserName = ShimLoader.getHadoopShims
- .getShortUserName(ShimLoader.getHadoopShims.getUGIForConf(hiveConf))
- setSuperField(this, "serverUserName", serverUserName)
+ if(hiveConf.get(authentication)!="NONE") {
+ HiveAuthFactory.loginFromKeytab(hiveConf)
+ }
} catch {
case e @ (_: IOException | _: LoginException) =>
throw new ServiceException("Unable to login to kerberos with given principal/keytab", e)
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
index 7463df1f47d4..207a681e891f 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
@@ -74,11 +74,11 @@ private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveCo
override def getSchema: Schema = tableSchema
- override def getResults(res: JArrayList[String]): Boolean = {
+ override def getResults(res: java.util.List[_]): Boolean = {
if (hiveResponse == null) {
false
} else {
- res.addAll(hiveResponse)
+ res.asInstanceOf[java.util.ArrayList[String]].addAll(hiveResponse)
hiveResponse = null
true
}
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala
index bd3f68d92d8c..5b11af6ddf9d 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala
@@ -17,9 +17,17 @@
package org.apache.spark.sql.hive.thriftserver.server
+import java.security.PrivilegedExceptionAction
import java.sql.Timestamp
+import java.util.concurrent.Future
import java.util.{Map => JMap}
+import org.apache.hadoop.hive.conf.HiveConf
+import org.apache.hadoop.hive.ql.metadata.Hive
+import org.apache.hadoop.hive.ql.session.SessionState
+import org.apache.hadoop.hive.shims.ShimLoader
+import org.apache.hadoop.security.UserGroupInformation
+
import scala.collection.JavaConversions._
import scala.collection.mutable.{ArrayBuffer, Map}
import scala.math.{random, round}
@@ -54,102 +62,95 @@ private[thriftserver] class SparkSQLOperationManager(hiveContext: HiveContext)
confOverlay: JMap[String, String],
async: Boolean): ExecuteStatementOperation = synchronized {
- val operation = new ExecuteStatementOperation(parentSession, statement, confOverlay) {
+ val operation = new ExecuteStatementOperation(parentSession, statement, confOverlay, true) {
private var result: SchemaRDD = _
private var iter: Iterator[SparkRow] = _
private var dataTypes: Array[DataType] = _
+ private def runInternal(cmd: String) = {
+ try {
+ result = hiveContext.sql(cmd)
+ logDebug(result.queryExecution.toString())
+ val groupId = round(random * 1000000).toString
+ hiveContext.sparkContext.setJobGroup(groupId, statement)
+ iter = {
+ val resultRdd = result.queryExecution.toRdd
+ val useIncrementalCollect =
+ hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean
+ if (useIncrementalCollect) {
+ resultRdd.toLocalIterator
+ } else {
+ resultRdd.collect().iterator
+ }
+ }
+ dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray
+ } catch {
+ // Actually do need to catch Throwable as some failures don't inherit from Exception and
+ // HiveServer will silently swallow them.
+ case e: Throwable =>
+ logError("Error executing query:",e)
+ throw new HiveSQLException(e.toString)
+ }
+ }
+
def close(): Unit = {
// RDDs will be cleaned automatically upon garbage collection.
logDebug("CLOSING")
}
def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = {
+ validateDefaultFetchOrientation(order)
+ assertState(OperationState.FINISHED)
+ setHasResultSet(true)
+ val reultRowSet: RowSet = RowSetFactory.create(getResultSetSchema, getProtocolVersion)
if (!iter.hasNext) {
- new RowSet()
+ reultRowSet
} else {
// maxRowsL here typically maps to java.sql.Statement.getFetchSize, which is an int
val maxRows = maxRowsL.toInt
var curRow = 0
- var rowSet = new ArrayBuffer[Row](maxRows.min(1024))
-
while (curRow < maxRows && iter.hasNext) {
val sparkRow = iter.next()
- val row = new Row()
+ val row = ArrayBuffer[Any]()
var curCol = 0
-
while (curCol < sparkRow.length) {
- if (sparkRow.isNullAt(curCol)) {
- addNullColumnValue(sparkRow, row, curCol)
- } else {
- addNonNullColumnValue(sparkRow, row, curCol)
+ dataTypes(curCol) match {
+ case StringType =>
+ row += sparkRow.get(curCol).asInstanceOf[String]
+ case IntegerType =>
+ row += sparkRow.getInt(curCol)
+ case BooleanType =>
+ row += sparkRow.getBoolean(curCol)
+ case DoubleType =>
+ row += sparkRow.getDouble(curCol)
+ case FloatType =>
+ row += sparkRow.getFloat(curCol)
+ case DecimalType =>
+ row += sparkRow.get(curCol).asInstanceOf[BigDecimal].bigDecimal
+ case LongType =>
+ row += sparkRow.getLong(curCol)
+ case ByteType =>
+ row += sparkRow.getByte(curCol)
+ case ShortType =>
+ row += sparkRow.getShort(curCol)
+ case TimestampType =>
+ row += sparkRow.get(curCol).asInstanceOf[Timestamp]
+ case BinaryType =>
+ row += sparkRow.get(curCol).asInstanceOf[String]
+ case _: ArrayType =>
+ row += sparkRow.get(curCol).asInstanceOf[String]
+ case _: StructType =>
+ row += sparkRow.get(curCol).asInstanceOf[String]
+ case _: MapType =>
+ row += sparkRow.get(curCol).asInstanceOf[String]
}
curCol += 1
}
- rowSet += row
+ reultRowSet.addRow(row.toArray.asInstanceOf[Array[Object]])
+ row.clear()
curRow += 1
}
- new RowSet(rowSet, 0)
- }
- }
-
- def addNonNullColumnValue(from: SparkRow, to: Row, ordinal: Int) {
- dataTypes(ordinal) match {
- case StringType =>
- to.addString(from(ordinal).asInstanceOf[String])
- case IntegerType =>
- to.addColumnValue(ColumnValue.intValue(from.getInt(ordinal)))
- case BooleanType =>
- to.addColumnValue(ColumnValue.booleanValue(from.getBoolean(ordinal)))
- case DoubleType =>
- to.addColumnValue(ColumnValue.doubleValue(from.getDouble(ordinal)))
- case FloatType =>
- to.addColumnValue(ColumnValue.floatValue(from.getFloat(ordinal)))
- case DecimalType =>
- val hiveDecimal = from.get(ordinal).asInstanceOf[BigDecimal].bigDecimal
- to.addColumnValue(ColumnValue.stringValue(new HiveDecimal(hiveDecimal)))
- case LongType =>
- to.addColumnValue(ColumnValue.longValue(from.getLong(ordinal)))
- case ByteType =>
- to.addColumnValue(ColumnValue.byteValue(from.getByte(ordinal)))
- case ShortType =>
- to.addColumnValue(ColumnValue.intValue(from.getShort(ordinal)))
- case TimestampType =>
- to.addColumnValue(
- ColumnValue.timestampValue(from.get(ordinal).asInstanceOf[Timestamp]))
- case BinaryType | _: ArrayType | _: StructType | _: MapType =>
- val hiveString = result
- .queryExecution
- .asInstanceOf[HiveContext#QueryExecution]
- .toHiveString((from.get(ordinal), dataTypes(ordinal)))
- to.addColumnValue(ColumnValue.stringValue(hiveString))
- }
- }
-
- def addNullColumnValue(from: SparkRow, to: Row, ordinal: Int) {
- dataTypes(ordinal) match {
- case StringType =>
- to.addString(null)
- case IntegerType =>
- to.addColumnValue(ColumnValue.intValue(null))
- case BooleanType =>
- to.addColumnValue(ColumnValue.booleanValue(null))
- case DoubleType =>
- to.addColumnValue(ColumnValue.doubleValue(null))
- case FloatType =>
- to.addColumnValue(ColumnValue.floatValue(null))
- case DecimalType =>
- to.addColumnValue(ColumnValue.stringValue(null: HiveDecimal))
- case LongType =>
- to.addColumnValue(ColumnValue.longValue(null))
- case ByteType =>
- to.addColumnValue(ColumnValue.byteValue(null))
- case ShortType =>
- to.addColumnValue(ColumnValue.intValue(null))
- case TimestampType =>
- to.addColumnValue(ColumnValue.timestampValue(null))
- case BinaryType | _: ArrayType | _: StructType | _: MapType =>
- to.addColumnValue(ColumnValue.stringValue(null: String))
+ reultRowSet
}
}
@@ -165,44 +166,83 @@ private[thriftserver] class SparkSQLOperationManager(hiveContext: HiveContext)
}
}
+ private def getConfigForOperation: HiveConf = {
+ var sqlOperationConf: HiveConf = getParentSession.getHiveConf
+ if (!getConfOverlay.isEmpty || shouldRunAsync) {
+ sqlOperationConf = new HiveConf(sqlOperationConf)
+ import scala.collection.JavaConversions._
+ for (confEntry <- getConfOverlay.entrySet) {
+ try {
+ sqlOperationConf.verifyAndSet(confEntry.getKey, confEntry.getValue)
+ }
+ catch {
+ case e: IllegalArgumentException => {
+ throw new HiveSQLException("Error applying statement specific settings", e)
+ }
+ }
+ }
+ }
+ return sqlOperationConf
+ }
+
def run(): Unit = {
logInfo(s"Running query '$statement'")
+ val opConfig: HiveConf = getConfigForOperation
setState(OperationState.RUNNING)
- try {
- result = hiveContext.sql(statement)
- logDebug(result.queryExecution.toString())
- result.queryExecution.logical match {
- case SetCommand(Some(key), Some(value)) if (key == SQLConf.THRIFTSERVER_POOL) =>
- sessionToActivePool(parentSession) = value
- logInfo(s"Setting spark.scheduler.pool=$value for future statements in this session.")
- case _ =>
- }
+ setHasResultSet(true)
- val groupId = round(random * 1000000).toString
- hiveContext.sparkContext.setJobGroup(groupId, statement)
- sessionToActivePool.get(parentSession).foreach { pool =>
- hiveContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool)
- }
- iter = {
- val resultRdd = result.queryExecution.toRdd
- val useIncrementalCollect =
- hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean
- if (useIncrementalCollect) {
- resultRdd.toLocalIterator
- } else {
- resultRdd.collect().iterator
+ if (!shouldRunAsync) {
+ runInternal(statement)
+ setState(OperationState.FINISHED)
+ } else {
+ val parentSessionState = SessionState.get
+ val sessionHive: Hive = Hive.get
+ val currentUGI: UserGroupInformation = ShimLoader.getHadoopShims.getUGIForConf(opConfig)
+
+ val backgroundOperation: Runnable = new Runnable {
+ def run {
+ val doAsAction: PrivilegedExceptionAction[AnyRef] =
+ new PrivilegedExceptionAction[AnyRef] {
+ def run: AnyRef = {
+ Hive.set(sessionHive)
+ SessionState.setCurrentSessionState(parentSessionState)
+ try {
+ runInternal(statement)
+ }
+ catch {
+ case e: HiveSQLException => {
+ setOperationException(e)
+ logError("Error running hive query: ", e)
+ }
+ }
+ return null
+ }
+ }
+ try {
+ ShimLoader.getHadoopShims.doAs(currentUGI, doAsAction)
+ }
+ catch {
+ case e: Exception => {
+ setOperationException(new HiveSQLException(e))
+ logError("Error running hive query as user : " + currentUGI.getShortUserName, e)
+ }
+ }
+ setState(OperationState.FINISHED)
}
}
- dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray
- setHasResultSet(true)
- } catch {
- // Actually do need to catch Throwable as some failures don't inherit from Exception and
- // HiveServer will silently swallow them.
- case e: Throwable =>
- logError("Error executing query:",e)
- throw new HiveSQLException(e.toString)
+
+ try {
+ val backgroundHandle: Future[_] = getParentSession.getSessionManager.
+ submitBackgroundOperation(backgroundOperation)
+ setBackgroundHandle(backgroundHandle)
+ } catch {
+ // Actually do need to catch Throwable as some failures don't inherit from Exception and
+ // HiveServer will silently swallow them.
+ case e: Throwable =>
+ logError("Error executing query:",e)
+ throw new HiveSQLException(e.toString)
+ }
}
- setState(OperationState.FINISHED)
}
}
diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
index 556c984ad392..f83b96a058ad 100644
--- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
+++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
@@ -324,8 +324,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"create_struct_table",
"cross_join",
"ct_case_insensitive",
- "database_location",
- "database_properties",
+// "database_location",
+// "database_properties",
"decimal_1",
"decimal_4",
"decimal_join",
@@ -691,7 +691,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"sort_merge_join_desc_7",
"stats0",
"stats_aggregator_error_1",
- "stats_empty_partition",
+// "stats_empty_partition", catylyst result not match hive result, they get different COLUMN_STATS_ACCURATE of Partition Parameters
"stats_publisher_error_1",
"subq2",
"tablename_with_select",
diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml
index 9d7a02bf7b0b..b90d444bf9d4 100644
--- a/sql/hive/pom.xml
+++ b/sql/hive/pom.xml
@@ -36,11 +36,6 @@
-
- com.twitter
- parquet-hive-bundle
- 1.5.0
-
org.apache.spark
spark-core_${scala.binary.version}
@@ -52,7 +47,7 @@
${project.version}
- org.spark-project.hive
+ org.apache.hive
hive-metastore
${hive.version}
@@ -62,7 +57,7 @@
3.1
- org.spark-project.hive
+ org.apache.hive
hive-exec
${hive.version}
@@ -77,7 +72,7 @@
jackson-mapper-asl
- org.spark-project.hive
+ org.apache.hive
hive-serde
${hive.version}
diff --git a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
index ab7862f4f9e0..a7d62305800f 100644
--- a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
@@ -28,7 +28,16 @@ import org.apache.hadoop.hive.ql.plan.FileSinkDesc
import org.apache.hadoop.mapred._
import org.apache.hadoop.io.Writable
-import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter}
+import org.apache.spark.{SparkHadoopWriter, SerializableWritable, Logging}
+import org.apache.hadoop.hive.ql.plan.TableDesc
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter
+import org.apache.hadoop.hive.ql.io.HiveOutputFormat
+import org.apache.hadoop.hive.ql.io.HivePassThroughOutputFormat
+import org.apache.hadoop.util.ReflectionUtils
+import org.apache.hadoop.hive.ql.exec.Utilities
+import org.apache.hadoop.io.compress.CompressionCodec
+import org.apache.hadoop.io.SequenceFile.CompressionType
+import org.apache.hadoop.hive.ql.io.HiveFileFormatUtils
/**
* Internal helper class that saves an RDD using a Hive OutputFormat.
@@ -36,7 +45,10 @@ import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter}
*/
private[hive] class SparkHiveHadoopWriter(
@transient jobConf: JobConf,
- fileSinkConf: FileSinkDesc)
+ isCompressed: Boolean,
+ tableDesc: TableDesc,
+ compressCodec: String,
+ compressType: String)
extends Logging
with SparkHadoopMapRedUtil
with Serializable {
@@ -50,8 +62,8 @@ private[hive] class SparkHiveHadoopWriter(
private var jID: SerializableWritable[JobID] = null
private var taID: SerializableWritable[TaskAttemptID] = null
- @transient private var writer: FileSinkOperator.RecordWriter = null
- @transient private var format: HiveOutputFormat[AnyRef, Writable] = null
+ @transient private var writer: RecordWriter = null
+ @transient private var format: HiveOutputFormat[_, _] = null
@transient private var committer: OutputCommitter = null
@transient private var jobContext: JobContext = null
@transient private var taskContext: TaskAttemptContext = null
@@ -74,23 +86,40 @@ private[hive] class SparkHiveHadoopWriter(
val numfmt = NumberFormat.getInstance()
numfmt.setMinimumIntegerDigits(5)
numfmt.setGroupingUsed(false)
+ val jc = conf.value
+
+ getOutputCommitter().setupTask(getTaskContext())
+ var hiveOutputFormat: HiveOutputFormat[_, _] = tableDesc.getOutputFileFormatClass.newInstance.
+ asInstanceOf[HiveOutputFormat[_, _]]
+
+ format = hiveOutputFormat
val extension = Utilities.getFileExtension(
- conf.value,
- fileSinkConf.getCompressed,
+ jc,
+ isCompressed,
getOutputFormat())
val outputName = "part-" + numfmt.format(splitID) + extension
- val path = FileOutputFormat.getTaskOutputPath(conf.value, outputName)
-
- getOutputCommitter().setupTask(getTaskContext())
- writer = HiveFileFormatUtils.getHiveRecordWriter(
- conf.value,
- fileSinkConf.getTableInfo,
- conf.value.getOutputValueClass.asInstanceOf[Class[Writable]],
- fileSinkConf,
- path,
- null)
+ val path = FileOutputFormat.getTaskOutputPath(jc, outputName)
+
+ var jc_output: JobConf = jc
+ if (isCompressed) {
+ jc_output = new JobConf(jc)
+ val codecStr: String = compressCodec
+ if (codecStr != null && !(codecStr.trim == "")) {
+ val codec: Class[_ <: CompressionCodec] =
+ Class.forName(codecStr).asInstanceOf[Class[_ <: CompressionCodec]]
+ FileOutputFormat.setOutputCompressorClass(jc_output, codec)
+ }
+ val typ: String = compressType
+ if (typ != null && !(typ.trim == "")) {
+ val style: CompressionType = CompressionType.valueOf(typ)
+ SequenceFileOutputFormat.setOutputCompressionType(jc, style)
+ }
+ }
+ writer = HiveFileFormatUtils.getRecordWriter(jc_output, hiveOutputFormat,
+ jc.getOutputValueClass.asInstanceOf[Class[Writable]],
+ isCompressed, tableDesc.getProperties, path, null)
}
def write(value: Writable) {
@@ -132,10 +161,10 @@ private[hive] class SparkHiveHadoopWriter(
// ********* Private Functions *********
- private def getOutputFormat(): HiveOutputFormat[AnyRef,Writable] = {
+ private def getOutputFormat(): HiveOutputFormat[_,_] = {
if (format == null) {
format = conf.value.getOutputFormat()
- .asInstanceOf[HiveOutputFormat[AnyRef,Writable]]
+ .asInstanceOf[HiveOutputFormat[_, _]]
}
format
}
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 3e1a7b71528e..5795d6ac0a02 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
@@ -32,7 +32,6 @@ import org.apache.hadoop.hive.ql.Driver
import org.apache.hadoop.hive.ql.metadata.Table
import org.apache.hadoop.hive.ql.processors._
import org.apache.hadoop.hive.ql.session.SessionState
-import org.apache.hadoop.hive.ql.stats.StatsSetupConst
import org.apache.hadoop.hive.serde2.io.TimestampWritable
import org.apache.spark.SparkContext
@@ -46,6 +45,7 @@ import org.apache.spark.sql.execution.ExtractPythonUdfs
import org.apache.spark.sql.execution.QueryExecutionException
import org.apache.spark.sql.execution.{Command => PhysicalCommand}
import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand
+import org.apache.hadoop.hive.common.StatsSetupConst
/**
* DEPRECATED: Use HiveContext instead.
@@ -95,7 +95,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
if (dialect == "sql") {
super.sql(sqlText)
} else if (dialect == "hiveql") {
- new SchemaRDD(this, HiveQl.parseSql(sqlText))
+ new SchemaRDD(this, HiveQl.parseSql(sqlText, hiveconf))
} else {
sys.error(s"Unsupported SQL dialect: $dialect. Try 'sql' or 'hiveql'")
}
@@ -103,7 +103,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
@deprecated("hiveql() is deprecated as the sql function now parses using HiveQL by default. " +
s"The SQL dialect for parsing can be set using ${SQLConf.DIALECT}", "1.1")
- def hiveql(hqlQuery: String): SchemaRDD = new SchemaRDD(this, HiveQl.parseSql(hqlQuery))
+ def hiveql(hqlQuery: String): SchemaRDD = new SchemaRDD(this, HiveQl.parseSql(hqlQuery, hiveconf))
@deprecated("hql() is deprecated as the sql function now parses using HiveQL by default. " +
s"The SQL dialect for parsing can be set using ${SQLConf.DIALECT}", "1.1")
@@ -284,7 +284,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
val cmd_trimmed: String = cmd.trim()
val tokens: Array[String] = cmd_trimmed.split("\\s+")
val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim()
- val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hiveconf)
+ val proc: CommandProcessor = CommandProcessorFactory.get(Array(tokens(0)), hiveconf)
SessionState.start(sessionState)
@@ -292,17 +292,22 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
case driver: Driver =>
driver.init()
- val results = new JArrayList[String]
+ val results = new JArrayList[Object]
val response: CommandProcessorResponse = driver.run(cmd)
// Throw an exception if there is an error in query processing.
if (response.getResponseCode != 0) {
- driver.destroy()
+ driver.close()
throw new QueryExecutionException(response.getErrorMessage)
}
driver.setMaxRows(maxRows)
driver.getResults(results)
- driver.destroy()
- results
+ driver.close()
+ results.map { r =>
+ r match {
+ case s: String => s
+ case a: Array[Object] => a(0).asInstanceOf[String]
+ }
+ }
case _ =>
sessionState.out.println(tokens(0) + " " + cmd_1)
Seq(proc.run(cmd_1).getResponseCode.toString)
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
index fa889ec104c6..844333a1ab2a 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
@@ -145,7 +145,7 @@ private[hive] trait HiveInspectors {
case l: Long => l: java.lang.Long
case l: Short => l: java.lang.Short
case l: Byte => l: java.lang.Byte
- case b: BigDecimal => new HiveDecimal(b.underlying())
+ case b: BigDecimal => HiveDecimal.create(b.underlying())
case b: Array[Byte] => b
case t: java.sql.Timestamp => t
case s: Seq[_] => seqAsJavaList(s.map(wrap))
@@ -213,6 +213,8 @@ private[hive] trait HiveInspectors {
case _: JavaHiveDecimalObjectInspector => DecimalType
case _: WritableTimestampObjectInspector => TimestampType
case _: JavaTimestampObjectInspector => TimestampType
+ case _: WritableVoidObjectInspector => NullType
+ case _: JavaVoidObjectInspector => NullType
}
implicit class typeInfoConversions(dt: DataType) {
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 9a0b9b46ac4e..c24e18fadfce 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
@@ -23,7 +23,6 @@ import org.apache.hadoop.hive.metastore.api.{FieldSchema, StorageDescriptor, Ser
import org.apache.hadoop.hive.metastore.api.{Table => TTable, Partition => TPartition}
import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table}
import org.apache.hadoop.hive.ql.plan.TableDesc
-import org.apache.hadoop.hive.ql.stats.StatsSetupConst
import org.apache.hadoop.hive.serde2.Deserializer
import org.apache.spark.annotation.DeveloperApi
@@ -38,6 +37,7 @@ import org.apache.spark.sql.catalyst.types._
import org.apache.spark.sql.columnar.InMemoryRelation
import org.apache.spark.sql.hive.execution.HiveTableScan
import org.apache.spark.util.Utils
+import org.apache.hadoop.hive.common.StatsSetupConst
/* Implicit conversions */
import scala.collection.JavaConversions._
@@ -59,7 +59,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with
val table = client.getTable(databaseName, tblName)
val partitions: Seq[Partition] =
if (table.isPartitioned) {
- client.getAllPartitionsForPruner(table).toSeq
+ client.getAllPartitionsOf(table).toSeq
} else {
Nil
}
@@ -192,7 +192,7 @@ object HiveMetastoreTypes extends RegexParsers {
"bigint" ^^^ LongType |
"binary" ^^^ BinaryType |
"boolean" ^^^ BooleanType |
- "decimal" ^^^ DecimalType |
+ "decimal(10,0)" ^^^ DecimalType |
"timestamp" ^^^ TimestampType |
"varchar\\((\\d+)\\)".r ^^^ StringType
@@ -284,10 +284,6 @@ private[hive] case class MetastoreRelation
)
val tableDesc = new TableDesc(
- Class.forName(
- hiveQlTable.getSerializationLib,
- true,
- Utils.getContextOrSparkClassLoader).asInstanceOf[Class[Deserializer]],
hiveQlTable.getInputFormatClass,
// The class of table should be org.apache.hadoop.hive.ql.metadata.Table because
// getOutputFormatClass will use HiveFileFormatUtils.getOutputFormatSubstitute to
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 0aa6292c0184..0b5bf8fa8401 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
@@ -27,6 +27,8 @@ import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.types._
+import org.apache.hadoop.hive.conf.HiveConf
+import org.apache.hadoop.hive.ql.Context
/* Implicit conversions */
import scala.collection.JavaConversions._
@@ -212,10 +214,10 @@ private[hive] object HiveQl {
/**
* Returns the AST for the given SQL string.
*/
- def getAst(sql: String): ASTNode = ParseUtils.findRootNonNullToken((new ParseDriver).parse(sql))
-
+ def getAst(sql: String, hiveConf: HiveConf): ASTNode = ParseUtils.findRootNonNullToken(
+ (new ParseDriver).parse(sql, new Context(hiveConf)))
/** Returns a LogicalPlan for a given HiveQL string. */
- def parseSql(sql: String): LogicalPlan = {
+ def parseSql(sql: String, hiveConf: HiveConf): LogicalPlan = {
try {
if (sql.trim.toLowerCase.startsWith("set")) {
// Split in two parts since we treat the part before the first "="
@@ -233,7 +235,7 @@ private[hive] object HiveQl {
case Seq(tableName) =>
CacheCommand(tableName, true)
case Seq(tableName, _, select @ _*) =>
- CacheTableAsSelectCommand(tableName, createPlan(select.mkString(" ").trim))
+ CacheTableAsSelectCommand(tableName, createPlan(select.mkString(" ").trim, hiveConf))
}
} else if (sql.trim.toLowerCase.startsWith("uncache table")) {
CacheCommand(sql.trim.drop(14).trim, false)
@@ -248,21 +250,21 @@ private[hive] object HiveQl {
} else if (sql.trim.startsWith("!")) {
ShellCommand(sql.drop(1))
} else {
- createPlan(sql)
+ createPlan(sql, hiveConf)
}
} catch {
case e: Exception => throw new ParseException(sql, e)
case e: NotImplementedError => sys.error(
s"""
|Unsupported language features in query: $sql
- |${dumpTree(getAst(sql))}
+ |${dumpTree(getAst(sql, hiveConf))}
""".stripMargin)
}
}
/** Creates LogicalPlan for a given HiveQL string. */
- def createPlan(sql: String) = {
- val tree = getAst(sql)
+ def createPlan(sql: String, hiveConf: HiveConf) = {
+ val tree = getAst(sql, hiveConf)
if (nativeCommands contains tree.getText) {
NativeCommand(sql)
} else {
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 84fafcde63d0..95799134a9f7 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
@@ -140,7 +140,7 @@ class HadoopTableReader(
filterOpt: Option[PathFilter]): RDD[Row] = {
val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializer) =>
val partDesc = Utilities.getPartitionDesc(partition)
- val partPath = partition.getPartitionPath
+ val partPath = partition.getDataLocation
val inputPathStr = applyFilterIfNeeded(partPath, filterOpt)
val ifc = partDesc.getInputFileFormatClass
.asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]]
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
index 70fb15259e7d..97a54ca79684 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
@@ -140,7 +140,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
val describedTable = "DESCRIBE (\\w+)".r
protected[hive] class HiveQLQueryExecution(hql: String) extends this.QueryExecution {
- lazy val logical = HiveQl.parseSql(hql)
+ lazy val logical = HiveQl.parseSql(hql, hiveconf)
def hiveExec() = runSqlHive(hql)
override def toString = hql + "\n" + super.toString
}
@@ -415,7 +415,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
// Lots of tests fail if we do not change the partition whitelist from the default.
runSqlHive("set hive.metastore.partition.name.whitelist.pattern=.*")
configure()
-
+ runSqlHive("create database if not exists default")
runSqlHive("USE default")
// Just loading src makes a lot of tests pass. This is because some tests do something like
@@ -430,7 +430,6 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
// At this point there is really no reason to continue, but the test framework traps exits.
// So instead we just pause forever so that at least the developer can see where things
// started to go wrong.
- Thread.sleep(100000)
}
}
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala
index a201d2349a2e..966649f387e1 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala
@@ -34,7 +34,7 @@ class JavaHiveContext(sparkContext: JavaSparkContext) extends JavaSQLContext(spa
if (sqlContext.dialect == "sql") {
super.sql(sqlText)
} else if (sqlContext.dialect == "hiveql") {
- new JavaSchemaRDD(sqlContext, HiveQl.parseSql(sqlText))
+ new JavaSchemaRDD(sqlContext, HiveQl.parseSql(sqlText, sqlContext.hiveconf))
} else {
sys.error(s"Unsupported SQL dialect: ${sqlContext.dialect}. Try 'sql' or 'hiveql'")
}
@@ -45,5 +45,5 @@ class JavaHiveContext(sparkContext: JavaSparkContext) extends JavaSQLContext(spa
*/
@Deprecated
def hql(hqlQuery: String): JavaSchemaRDD =
- new JavaSchemaRDD(sqlContext, HiveQl.parseSql(hqlQuery))
+ new JavaSchemaRDD(sqlContext, HiveQl.parseSql(hqlQuery, sqlContext.hiveconf))
}
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 317801001c7a..bed72be37cb0 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
@@ -43,7 +43,7 @@ case class DescribeHiveTableCommand(
// Strings with the format like Hive. It is used for result comparison in our unit tests.
lazy val hiveString: Seq[String] = sideEffectResult.map {
case Row(name: String, dataType: String, comment) =>
- Seq(name, dataType, Option(comment.asInstanceOf[String]).getOrElse("None"))
+ Seq(name, dataType, Option(comment.asInstanceOf[String]).getOrElse(""))
.map(s => String.format(s"%-20s", s))
.mkString("\t")
}
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 577ca928b43b..e5472cbfb922 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
@@ -77,8 +77,10 @@ case class HiveTableScan(
attributes.map(a =>
relation.attributes.indexWhere(_.name == a.name): Integer).filter(index => index >= 0)
- ColumnProjectionUtils.appendReadColumnIDs(hiveConf, neededColumnIDs)
- ColumnProjectionUtils.appendReadColumnNames(hiveConf, attributes.map(_.name))
+ if(neededColumnIDs.size != 0) {
+ ColumnProjectionUtils.appendReadColumns(
+ hiveConf, neededColumnIDs, attributes.map(_.name).toSeq)
+ }
// Specifies types and object inspectors of columns to be scanned.
val structOI = ObjectInspectorUtils
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 a284a91a91e3..be8806964567 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
@@ -34,7 +34,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveVarcharOb
import org.apache.hadoop.io.Writable
import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf}
-import org.apache.spark.{SparkException, TaskContext}
+import org.apache.spark.{SerializableWritable, SparkException, TaskContext}
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.expressions.Row
@@ -76,7 +76,7 @@ case class InsertIntoHiveTable(
new HiveVarchar(s, s.size)
case (bd: BigDecimal, oi: JavaHiveDecimalObjectInspector) =>
- new HiveDecimal(bd.underlying())
+ HiveDecimal.create(bd.underlying())
case (row: Row, oi: StandardStructObjectInspector) =>
val struct = oi.create()
@@ -129,11 +129,16 @@ case class InsertIntoHiveTable(
conf.setOutputCommitter(classOf[FileOutputCommitter])
FileOutputFormat.setOutputPath(
conf,
- SparkHiveHadoopWriter.createPathFromString(fileSinkConf.getDirName, conf))
+ SparkHiveHadoopWriter.createPathFromString(fileSinkConf.getDirName.toString, conf))
log.debug("Saving as hadoop file of type " + valueClass.getSimpleName)
-
- val writer = new SparkHiveHadoopWriter(conf, fileSinkConf)
+ val broadcastConf = sc.sparkContext.broadcast(new SerializableWritable(conf))
+ val compressed = fileSinkConf.getCompressed
+ val tableDesc = fileSinkConf.getTableInfo
+ val compressCodec = fileSinkConf.getCompressCodec
+ val compressType = fileSinkConf.getCompressType
+ val writer = new SparkHiveHadoopWriter(broadcastConf.value.value, compressed, tableDesc,
+ compressCodec, compressType)
writer.preSetup()
def writeToFile(context: TaskContext, iter: Iterator[Writable]) {
@@ -176,13 +181,13 @@ case class InsertIntoHiveTable(
// instances within the closure, since Serializer is not serializable while TableDesc is.
val tableDesc = table.tableDesc
val tableLocation = table.hiveQlTable.getDataLocation
- val tmpLocation = hiveContext.getExternalTmpFileURI(tableLocation)
- val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false)
+ val tmpLocation = hiveContext.getExternalTmpPath(tableLocation.toUri)
+ @transient val fileSinkConf = new FileSinkDesc(tmpLocation, tableDesc, false)
val rdd = childRdd.mapPartitions { iter =>
- val serializer = newSerializer(fileSinkConf.getTableInfo)
+ val serializer = newSerializer(tableDesc)
val standardOI = ObjectInspectorUtils
.getStandardObjectInspector(
- fileSinkConf.getTableInfo.getDeserializer.getObjectInspector,
+ tableDesc.getDeserializer.getObjectInspector,
ObjectInspectorCopyOption.JAVA)
.asInstanceOf[StructObjectInspector]
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 732e4976f684..41af9b6d9e0b 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
@@ -32,6 +32,9 @@ import org.apache.spark.sql.catalyst.analysis
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.types._
import org.apache.spark.util.Utils.getContextOrSparkClassLoader
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory
+import org.apache.hadoop.io.IntWritable
/* Implicit conversions */
import scala.collection.JavaConversions._
@@ -122,8 +125,16 @@ private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq
type UDFType = GenericUDF
@transient
- protected lazy val argumentInspectors = children.map(_.dataType).map(toInspector)
-
+ protected lazy val argumentInspectors = {
+ if( functionClassName == "org.apache.hadoop.hive.ql.udf.generic.GenericUDFRound" &&
+ children.size == 2) {
+ Seq(toInspector(children(0).dataType),
+ PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector(
+ TypeInfoFactory.intTypeInfo, new IntWritable(children(1).eval(null).asInstanceOf[Int])))
+ } else {
+ children.map(_.dataType).map(toInspector)
+ }
+ }
@transient
protected lazy val returnInspector = function.initialize(argumentInspectors.toArray)
diff --git a/sql/hive/src/test/resources/data/files/TestSerDe.jar b/sql/hive/src/test/resources/data/files/TestSerDe.jar
index f29def6f8c97..7535ad562397 100644
Binary files a/sql/hive/src/test/resources/data/files/TestSerDe.jar and b/sql/hive/src/test/resources/data/files/TestSerDe.jar differ
diff --git a/sql/hive/src/test/resources/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591 b/sql/hive/src/test/resources/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591
index 62a8ea1d115f..641f41144c4c 100644
--- a/sql/hive/src/test/resources/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591
+++ b/sql/hive/src/test/resources/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591
@@ -1,10 +1,10 @@
-a int None
-b int None
-insertdate string None
+a int
+b int
+insertdate string
# Partition Information
# col_name data_type comment
-insertdate string None
+insertdate string
-Detailed Table Information Table(tableName:alter2, dbName:default, owner:marmbrus, createTime:1389728471, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1389728471}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter2, dbName:default, owner:root, createTime:1411193036, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse2779974130068756517/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1411193036}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591 b/sql/hive/src/test/resources/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591
index 72621add45bb..931f61ea5538 100644
--- a/sql/hive/src/test/resources/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591
+++ b/sql/hive/src/test/resources/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591
@@ -1,10 +1,10 @@
-a int None
-b int None
-insertdate string None
+a int
+b int
+insertdate string
# Partition Information
# col_name data_type comment
-insertdate string None
+insertdate string
-Detailed Table Information Table(tableName:alter2, dbName:default, owner:marmbrus, createTime:1389728473, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1389728473}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter2, dbName:default, owner:root, createTime:1411193038, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse2779974130068756517/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1411193038}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591 b/sql/hive/src/test/resources/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591
index 72621add45bb..931f61ea5538 100644
--- a/sql/hive/src/test/resources/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591
+++ b/sql/hive/src/test/resources/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591
@@ -1,10 +1,10 @@
-a int None
-b int None
-insertdate string None
+a int
+b int
+insertdate string
# Partition Information
# col_name data_type comment
-insertdate string None
+insertdate string
-Detailed Table Information Table(tableName:alter2, dbName:default, owner:marmbrus, createTime:1389728473, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1389728473}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter2, dbName:default, owner:root, createTime:1411193038, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse2779974130068756517/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1411193038}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter2-15-75a213649242c2410ea6846f08c91d75 b/sql/hive/src/test/resources/golden/alter2-15-75a213649242c2410ea6846f08c91d75
index 15e6bc882315..30f78a2dc1fb 100644
--- a/sql/hive/src/test/resources/golden/alter2-15-75a213649242c2410ea6846f08c91d75
+++ b/sql/hive/src/test/resources/golden/alter2-15-75a213649242c2410ea6846f08c91d75
@@ -1 +1 @@
-insertdate=2008-01-01
\ No newline at end of file
+insertdate=2008-01-01
diff --git a/sql/hive/src/test/resources/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591 b/sql/hive/src/test/resources/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591
index 72621add45bb..931f61ea5538 100644
--- a/sql/hive/src/test/resources/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591
+++ b/sql/hive/src/test/resources/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591
@@ -1,10 +1,10 @@
-a int None
-b int None
-insertdate string None
+a int
+b int
+insertdate string
# Partition Information
# col_name data_type comment
-insertdate string None
+insertdate string
-Detailed Table Information Table(tableName:alter2, dbName:default, owner:marmbrus, createTime:1389728473, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1389728473}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter2, dbName:default, owner:root, createTime:1411193038, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse2779974130068756517/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1411193038}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter2-18-75a213649242c2410ea6846f08c91d75 b/sql/hive/src/test/resources/golden/alter2-18-75a213649242c2410ea6846f08c91d75
index d31318a383fc..81b4ae118d12 100644
--- a/sql/hive/src/test/resources/golden/alter2-18-75a213649242c2410ea6846f08c91d75
+++ b/sql/hive/src/test/resources/golden/alter2-18-75a213649242c2410ea6846f08c91d75
@@ -1,2 +1,2 @@
insertdate=2008-01-01
-insertdate=2008-01-02
\ No newline at end of file
+insertdate=2008-01-02
diff --git a/sql/hive/src/test/resources/golden/alter2-20-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter2-20-9c36cac1372650b703400c60dd29042c
index ecafeaea5f61..85c1918f4656 100644
--- a/sql/hive/src/test/resources/golden/alter2-20-9c36cac1372650b703400c60dd29042c
+++ b/sql/hive/src/test/resources/golden/alter2-20-9c36cac1372650b703400c60dd29042c
@@ -1,2 +1,2 @@
src
-srcpart
\ No newline at end of file
+srcpart
diff --git a/sql/hive/src/test/resources/golden/alter2-25-4ef75e12575453225738ea167c4617e5 b/sql/hive/src/test/resources/golden/alter2-25-4ef75e12575453225738ea167c4617e5
index 83184977e9da..8467d5b395a3 100644
--- a/sql/hive/src/test/resources/golden/alter2-25-4ef75e12575453225738ea167c4617e5
+++ b/sql/hive/src/test/resources/golden/alter2-25-4ef75e12575453225738ea167c4617e5
@@ -1,10 +1,10 @@
-a int None
-b int None
-insertdate string None
+a int
+b int
+insertdate string
# Partition Information
# col_name data_type comment
-insertdate string None
+insertdate string
-Detailed Table Information Table(tableName:alter2, dbName:alter2_db, owner:marmbrus, createTime:1389728475, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1389728475}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter2, dbName:alter2_db, owner:root, createTime:1411193040, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse2779974130068756517/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1411193040}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter2-28-4ef75e12575453225738ea167c4617e5 b/sql/hive/src/test/resources/golden/alter2-28-4ef75e12575453225738ea167c4617e5
index 83184977e9da..8467d5b395a3 100644
--- a/sql/hive/src/test/resources/golden/alter2-28-4ef75e12575453225738ea167c4617e5
+++ b/sql/hive/src/test/resources/golden/alter2-28-4ef75e12575453225738ea167c4617e5
@@ -1,10 +1,10 @@
-a int None
-b int None
-insertdate string None
+a int
+b int
+insertdate string
# Partition Information
# col_name data_type comment
-insertdate string None
+insertdate string
-Detailed Table Information Table(tableName:alter2, dbName:alter2_db, owner:marmbrus, createTime:1389728475, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1389728475}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter2, dbName:alter2_db, owner:root, createTime:1411193040, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse2779974130068756517/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1411193040}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter2-29-63f13c364546ddce5d2176c6604a948f b/sql/hive/src/test/resources/golden/alter2-29-63f13c364546ddce5d2176c6604a948f
index 15e6bc882315..30f78a2dc1fb 100644
--- a/sql/hive/src/test/resources/golden/alter2-29-63f13c364546ddce5d2176c6604a948f
+++ b/sql/hive/src/test/resources/golden/alter2-29-63f13c364546ddce5d2176c6604a948f
@@ -1 +1 @@
-insertdate=2008-01-01
\ No newline at end of file
+insertdate=2008-01-01
diff --git a/sql/hive/src/test/resources/golden/alter2-31-4ef75e12575453225738ea167c4617e5 b/sql/hive/src/test/resources/golden/alter2-31-4ef75e12575453225738ea167c4617e5
index 83184977e9da..8467d5b395a3 100644
--- a/sql/hive/src/test/resources/golden/alter2-31-4ef75e12575453225738ea167c4617e5
+++ b/sql/hive/src/test/resources/golden/alter2-31-4ef75e12575453225738ea167c4617e5
@@ -1,10 +1,10 @@
-a int None
-b int None
-insertdate string None
+a int
+b int
+insertdate string
# Partition Information
# col_name data_type comment
-insertdate string None
+insertdate string
-Detailed Table Information Table(tableName:alter2, dbName:alter2_db, owner:marmbrus, createTime:1389728475, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1389728475}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter2, dbName:alter2_db, owner:root, createTime:1411193040, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse2779974130068756517/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1411193040}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter2-32-63f13c364546ddce5d2176c6604a948f b/sql/hive/src/test/resources/golden/alter2-32-63f13c364546ddce5d2176c6604a948f
index d31318a383fc..81b4ae118d12 100644
--- a/sql/hive/src/test/resources/golden/alter2-32-63f13c364546ddce5d2176c6604a948f
+++ b/sql/hive/src/test/resources/golden/alter2-32-63f13c364546ddce5d2176c6604a948f
@@ -1,2 +1,2 @@
insertdate=2008-01-01
-insertdate=2008-01-02
\ No newline at end of file
+insertdate=2008-01-02
diff --git a/sql/hive/src/test/resources/golden/alter2-35-4ef75e12575453225738ea167c4617e5 b/sql/hive/src/test/resources/golden/alter2-35-4ef75e12575453225738ea167c4617e5
index 6e30936a881b..01073d12ba30 100644
--- a/sql/hive/src/test/resources/golden/alter2-35-4ef75e12575453225738ea167c4617e5
+++ b/sql/hive/src/test/resources/golden/alter2-35-4ef75e12575453225738ea167c4617e5
@@ -1,10 +1,10 @@
-a int None
-b int None
-insertdate string None
+a int
+b int
+insertdate string
# Partition Information
# col_name data_type comment
-insertdate string None
+insertdate string
-Detailed Table Information Table(tableName:alter2, dbName:alter2_db, owner:marmbrus, createTime:1389728477, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1389728477}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter2, dbName:alter2_db, owner:root, createTime:1411193041, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse2779974130068756517/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1411193041}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter2-38-4ef75e12575453225738ea167c4617e5 b/sql/hive/src/test/resources/golden/alter2-38-4ef75e12575453225738ea167c4617e5
index 6e30936a881b..01073d12ba30 100644
--- a/sql/hive/src/test/resources/golden/alter2-38-4ef75e12575453225738ea167c4617e5
+++ b/sql/hive/src/test/resources/golden/alter2-38-4ef75e12575453225738ea167c4617e5
@@ -1,10 +1,10 @@
-a int None
-b int None
-insertdate string None
+a int
+b int
+insertdate string
# Partition Information
# col_name data_type comment
-insertdate string None
+insertdate string
-Detailed Table Information Table(tableName:alter2, dbName:alter2_db, owner:marmbrus, createTime:1389728477, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1389728477}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter2, dbName:alter2_db, owner:root, createTime:1411193041, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse2779974130068756517/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1411193041}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter2-39-63f13c364546ddce5d2176c6604a948f b/sql/hive/src/test/resources/golden/alter2-39-63f13c364546ddce5d2176c6604a948f
index 15e6bc882315..30f78a2dc1fb 100644
--- a/sql/hive/src/test/resources/golden/alter2-39-63f13c364546ddce5d2176c6604a948f
+++ b/sql/hive/src/test/resources/golden/alter2-39-63f13c364546ddce5d2176c6604a948f
@@ -1 +1 @@
-insertdate=2008-01-01
\ No newline at end of file
+insertdate=2008-01-01
diff --git a/sql/hive/src/test/resources/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591 b/sql/hive/src/test/resources/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591
index 62a8ea1d115f..641f41144c4c 100644
--- a/sql/hive/src/test/resources/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591
+++ b/sql/hive/src/test/resources/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591
@@ -1,10 +1,10 @@
-a int None
-b int None
-insertdate string None
+a int
+b int
+insertdate string
# Partition Information
# col_name data_type comment
-insertdate string None
+insertdate string
-Detailed Table Information Table(tableName:alter2, dbName:default, owner:marmbrus, createTime:1389728471, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1389728471}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter2, dbName:default, owner:root, createTime:1411193036, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse2779974130068756517/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1411193036}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter2-41-4ef75e12575453225738ea167c4617e5 b/sql/hive/src/test/resources/golden/alter2-41-4ef75e12575453225738ea167c4617e5
index 6e30936a881b..01073d12ba30 100644
--- a/sql/hive/src/test/resources/golden/alter2-41-4ef75e12575453225738ea167c4617e5
+++ b/sql/hive/src/test/resources/golden/alter2-41-4ef75e12575453225738ea167c4617e5
@@ -1,10 +1,10 @@
-a int None
-b int None
-insertdate string None
+a int
+b int
+insertdate string
# Partition Information
# col_name data_type comment
-insertdate string None
+insertdate string
-Detailed Table Information Table(tableName:alter2, dbName:alter2_db, owner:marmbrus, createTime:1389728477, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1389728477}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter2, dbName:alter2_db, owner:root, createTime:1411193041, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse2779974130068756517/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1411193041}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter2-42-63f13c364546ddce5d2176c6604a948f b/sql/hive/src/test/resources/golden/alter2-42-63f13c364546ddce5d2176c6604a948f
index d31318a383fc..81b4ae118d12 100644
--- a/sql/hive/src/test/resources/golden/alter2-42-63f13c364546ddce5d2176c6604a948f
+++ b/sql/hive/src/test/resources/golden/alter2-42-63f13c364546ddce5d2176c6604a948f
@@ -1,2 +1,2 @@
insertdate=2008-01-01
-insertdate=2008-01-02
\ No newline at end of file
+insertdate=2008-01-02
diff --git a/sql/hive/src/test/resources/golden/alter2-5-75a213649242c2410ea6846f08c91d75 b/sql/hive/src/test/resources/golden/alter2-5-75a213649242c2410ea6846f08c91d75
index 15e6bc882315..30f78a2dc1fb 100644
--- a/sql/hive/src/test/resources/golden/alter2-5-75a213649242c2410ea6846f08c91d75
+++ b/sql/hive/src/test/resources/golden/alter2-5-75a213649242c2410ea6846f08c91d75
@@ -1 +1 @@
-insertdate=2008-01-01
\ No newline at end of file
+insertdate=2008-01-01
diff --git a/sql/hive/src/test/resources/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591 b/sql/hive/src/test/resources/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591
index 62a8ea1d115f..641f41144c4c 100644
--- a/sql/hive/src/test/resources/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591
+++ b/sql/hive/src/test/resources/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591
@@ -1,10 +1,10 @@
-a int None
-b int None
-insertdate string None
+a int
+b int
+insertdate string
# Partition Information
# col_name data_type comment
-insertdate string None
+insertdate string
-Detailed Table Information Table(tableName:alter2, dbName:default, owner:marmbrus, createTime:1389728471, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1389728471}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter2, dbName:default, owner:root, createTime:1411193036, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse2779974130068756517/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1411193036}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter2-8-75a213649242c2410ea6846f08c91d75 b/sql/hive/src/test/resources/golden/alter2-8-75a213649242c2410ea6846f08c91d75
index d31318a383fc..81b4ae118d12 100644
--- a/sql/hive/src/test/resources/golden/alter2-8-75a213649242c2410ea6846f08c91d75
+++ b/sql/hive/src/test/resources/golden/alter2-8-75a213649242c2410ea6846f08c91d75
@@ -1,2 +1,2 @@
insertdate=2008-01-01
-insertdate=2008-01-02
\ No newline at end of file
+insertdate=2008-01-02
diff --git a/sql/hive/src/test/resources/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506 b/sql/hive/src/test/resources/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506
index ba1746da5ce6..aaac441e2b1a 100644
--- a/sql/hive/src/test/resources/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506
+++ b/sql/hive/src/test/resources/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506
@@ -1,11 +1,11 @@
-col1 string None
-pcol1 string None
-pcol2 string None
+col1 string
+pcol1 string
+pcol2 string
# Partition Information
# col_name data_type comment
-pcol1 string None
-pcol2 string None
+pcol1 string
+pcol2 string
-Detailed Table Information Table(tableName:alter3_like_renamed, dbName:default, owner:marmbrus, createTime:1389728495, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter3_like_renamed, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], parameters:{numPartitions=1, numFiles=1, last_modified_by=marmbrus, last_modified_time=1389728524, transient_lastDdlTime=1389728524, numRows=6, totalSize=171, rawDataSize=6}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter3_like_renamed, dbName:default, owner:root, createTime:1411176341, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/alter3_like_renamed, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], parameters:{last_modified_by=root, last_modified_time=1411176352, transient_lastDdlTime=1411176352}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter3-16-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter3-16-9c36cac1372650b703400c60dd29042c
index ecafeaea5f61..85c1918f4656 100644
--- a/sql/hive/src/test/resources/golden/alter3-16-9c36cac1372650b703400c60dd29042c
+++ b/sql/hive/src/test/resources/golden/alter3-16-9c36cac1372650b703400c60dd29042c
@@ -1,2 +1,2 @@
src
-srcpart
\ No newline at end of file
+srcpart
diff --git a/sql/hive/src/test/resources/golden/alter3-25-568a59760e5d3241b63d65cce595face b/sql/hive/src/test/resources/golden/alter3-25-568a59760e5d3241b63d65cce595face
index 8e609740f34a..d1f5bd707fcb 100644
--- a/sql/hive/src/test/resources/golden/alter3-25-568a59760e5d3241b63d65cce595face
+++ b/sql/hive/src/test/resources/golden/alter3-25-568a59760e5d3241b63d65cce595face
@@ -3,4 +3,4 @@
3 test_part: test_part:
4 test_part: test_part:
5 test_part: test_part:
-6 test_part: test_part:
\ No newline at end of file
+6 test_part: test_part:
diff --git a/sql/hive/src/test/resources/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87 b/sql/hive/src/test/resources/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87
index 0e12e0e16633..55c142440d97 100644
--- a/sql/hive/src/test/resources/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87
+++ b/sql/hive/src/test/resources/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87
@@ -1,11 +1,11 @@
-col1 string None
-pcol1 string None
-pcol2 string None
+col1 string
+pcol1 string
+pcol2 string
# Partition Information
# col_name data_type comment
-pcol1 string None
-pcol2 string None
+pcol1 string
+pcol2 string
-Detailed Table Information Table(tableName:alter3_renamed, dbName:alter3_db, owner:marmbrus, createTime:1389728526, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter3_db.db/alter3_renamed, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], parameters:{numPartitions=1, numFiles=1, last_modified_by=marmbrus, last_modified_time=1389728544, transient_lastDdlTime=1389728544, numRows=6, totalSize=171, rawDataSize=6}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter3_renamed, dbName:alter3_db, owner:root, createTime:1411176353, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/alter3_db.db/alter3_renamed, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], parameters:{last_modified_by=root, last_modified_time=1411176358, transient_lastDdlTime=1411176358}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773 b/sql/hive/src/test/resources/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773
index 4acb920388ce..c8cd1f56cf31 100644
--- a/sql/hive/src/test/resources/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773
+++ b/sql/hive/src/test/resources/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773
@@ -1,11 +1,11 @@
-col1 string None
-pcol1 string None
-pcol2 string None
+col1 string
+pcol1 string
+pcol2 string
# Partition Information
# col_name data_type comment
-pcol1 string None
-pcol2 string None
+pcol1 string
+pcol2 string
-Detailed Partition Information Partition(values:[test_part:, test_part:], dbName:alter3_db, tableName:alter3_renamed, createTime:1389728543, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter3_db.db/alter3_renamed/pcol1=test_part%3A/pcol2=test_part%3A, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1389728544, numRows=6, totalSize=171, rawDataSize=6})
\ No newline at end of file
+Detailed Partition Information Partition(values:[test_part:, test_part:], dbName:alter3_db, tableName:alter3_renamed, createTime:1411176358, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/alter3_db.db/alter3_renamed/pcol1=test_part%3A/pcol2=test_part%3A, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1411176358, COLUMN_STATS_ACCURATE=true, totalSize=171, numRows=6, rawDataSize=6})
diff --git a/sql/hive/src/test/resources/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c b/sql/hive/src/test/resources/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c
index 8e609740f34a..d1f5bd707fcb 100644
--- a/sql/hive/src/test/resources/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c
+++ b/sql/hive/src/test/resources/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c
@@ -3,4 +3,4 @@
3 test_part: test_part:
4 test_part: test_part:
5 test_part: test_part:
-6 test_part: test_part:
\ No newline at end of file
+6 test_part: test_part:
diff --git a/sql/hive/src/test/resources/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65 b/sql/hive/src/test/resources/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65
index 12b038204bef..d1b31c40b416 100644
--- a/sql/hive/src/test/resources/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65
+++ b/sql/hive/src/test/resources/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65
@@ -1,11 +1,11 @@
-col1 string None
-pcol1 string None
-pcol2 string None
+col1 string
+pcol1 string
+pcol2 string
# Partition Information
# col_name data_type comment
-pcol1 string None
-pcol2 string None
+pcol1 string
+pcol2 string
-Detailed Table Information Table(tableName:alter3_like_renamed, dbName:alter3_db, owner:marmbrus, createTime:1389728527, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter3_db.db/alter3_like_renamed, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], parameters:{numPartitions=1, numFiles=1, last_modified_by=marmbrus, last_modified_time=1389728560, transient_lastDdlTime=1389728560, numRows=6, totalSize=171, rawDataSize=6}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter3_like_renamed, dbName:alter3_db, owner:root, createTime:1411176353, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/alter3_db.db/alter3_like_renamed, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], parameters:{last_modified_by=root, last_modified_time=1411176365, transient_lastDdlTime=1411176365}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97 b/sql/hive/src/test/resources/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97
index 8e609740f34a..d1f5bd707fcb 100644
--- a/sql/hive/src/test/resources/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97
+++ b/sql/hive/src/test/resources/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97
@@ -3,4 +3,4 @@
3 test_part: test_part:
4 test_part: test_part:
5 test_part: test_part:
-6 test_part: test_part:
\ No newline at end of file
+6 test_part: test_part:
diff --git a/sql/hive/src/test/resources/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8 b/sql/hive/src/test/resources/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8
index 0c47fb0ac64a..2fa534a77148 100644
--- a/sql/hive/src/test/resources/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8
+++ b/sql/hive/src/test/resources/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8
@@ -1,11 +1,11 @@
-col1 string None
-pcol1 string None
-pcol2 string None
+col1 string
+pcol1 string
+pcol2 string
# Partition Information
# col_name data_type comment
-pcol1 string None
-pcol2 string None
+pcol1 string
+pcol2 string
-Detailed Table Information Table(tableName:alter3_renamed, dbName:default, owner:marmbrus, createTime:1389728495, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter3_renamed, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], parameters:{numPartitions=1, numFiles=1, last_modified_by=marmbrus, last_modified_time=1389728509, transient_lastDdlTime=1389728509, numRows=6, totalSize=171, rawDataSize=6}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter3_renamed, dbName:default, owner:root, createTime:1411176341, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/alter3_renamed, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], parameters:{last_modified_by=root, last_modified_time=1411176346, transient_lastDdlTime=1411176346}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43 b/sql/hive/src/test/resources/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43
index 1165de3a92fd..79e20045eac8 100644
--- a/sql/hive/src/test/resources/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43
+++ b/sql/hive/src/test/resources/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43
@@ -1,11 +1,11 @@
-col1 string None
-pcol1 string None
-pcol2 string None
+col1 string
+pcol1 string
+pcol2 string
# Partition Information
# col_name data_type comment
-pcol1 string None
-pcol2 string None
+pcol1 string
+pcol2 string
-Detailed Partition Information Partition(values:[test_part:, test_part:], dbName:default, tableName:alter3_renamed, createTime:1389728508, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter3_renamed/pcol1=test_part%3A/pcol2=test_part%3A, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1389728508, numRows=6, totalSize=171, rawDataSize=6})
\ No newline at end of file
+Detailed Partition Information Partition(values:[test_part:, test_part:], dbName:default, tableName:alter3_renamed, createTime:1411176346, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/alter3_renamed/pcol1=test_part%3A/pcol2=test_part%3A, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1411176346, COLUMN_STATS_ACCURATE=true, totalSize=171, numRows=6, rawDataSize=6})
diff --git a/sql/hive/src/test/resources/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2 b/sql/hive/src/test/resources/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2
index 8e609740f34a..d1f5bd707fcb 100644
--- a/sql/hive/src/test/resources/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2
+++ b/sql/hive/src/test/resources/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2
@@ -3,4 +3,4 @@
3 test_part: test_part:
4 test_part: test_part:
5 test_part: test_part:
-6 test_part: test_part:
\ No newline at end of file
+6 test_part: test_part:
diff --git a/sql/hive/src/test/resources/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7 b/sql/hive/src/test/resources/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7
index 8d2aebeb4d29..bb7c88b73573 100644
--- a/sql/hive/src/test/resources/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7
+++ b/sql/hive/src/test/resources/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7
@@ -1,4 +1,4 @@
-key int None
-value string None
+key int
+value string
-Detailed Table Information Table(tableName:set_bucketing_test, dbName:default, owner:marmbrus, createTime:1389735344, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5432448755546164619/set_bucketing_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:10, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[key], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389735344}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:set_bucketing_test, dbName:default, owner:root, createTime:1411193054, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse2779974130068756517/set_bucketing_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:10, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[key], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1411193054}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7 b/sql/hive/src/test/resources/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7
index cf67a009ff29..18f9d60ddf3d 100644
--- a/sql/hive/src/test/resources/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7
+++ b/sql/hive/src/test/resources/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7
@@ -1,4 +1,4 @@
-key int None
-value string None
+key int
+value string
-Detailed Table Information Table(tableName:set_bucketing_test, dbName:alter4_db, owner:marmbrus, createTime:1389735347, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5432448755546164619/alter4_db.db/set_bucketing_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:10, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[key], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389735347}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:set_bucketing_test, dbName:alter4_db, owner:root, createTime:1411193054, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse2779974130068756517/alter4_db.db/set_bucketing_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:10, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[key], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1411193054}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7 b/sql/hive/src/test/resources/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7
index a6375189a7a2..2f4b9e6bdd02 100644
--- a/sql/hive/src/test/resources/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7
+++ b/sql/hive/src/test/resources/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7
@@ -1,4 +1,4 @@
-key int None
-value string None
+key int
+value string
-Detailed Table Information Table(tableName:set_bucketing_test, dbName:alter4_db, owner:marmbrus, createTime:1389735347, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5432448755546164619/alter4_db.db/set_bucketing_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1389735348, transient_lastDdlTime=1389735348}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:set_bucketing_test, dbName:alter4_db, owner:root, createTime:1411193054, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse2779974130068756517/alter4_db.db/set_bucketing_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=root, last_modified_time=1411193054, transient_lastDdlTime=1411193054, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20 b/sql/hive/src/test/resources/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20
index 331d858ce9b1..4ad96d51599f 100644
--- a/sql/hive/src/test/resources/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20
+++ b/sql/hive/src/test/resources/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20
@@ -1 +1 @@
-default
\ No newline at end of file
+default
diff --git a/sql/hive/src/test/resources/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7 b/sql/hive/src/test/resources/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7
index 6b79a37a85f0..90651b50d732 100644
--- a/sql/hive/src/test/resources/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7
+++ b/sql/hive/src/test/resources/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7
@@ -1,4 +1,4 @@
-key int None
-value string None
+key int
+value string
-Detailed Table Information Table(tableName:set_bucketing_test, dbName:default, owner:marmbrus, createTime:1389735344, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5432448755546164619/set_bucketing_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1389735345, transient_lastDdlTime=1389735345}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:set_bucketing_test, dbName:default, owner:root, createTime:1411193054, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse2779974130068756517/set_bucketing_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=root, last_modified_time=1411193054, transient_lastDdlTime=1411193054, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter4-5-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter4-5-9c36cac1372650b703400c60dd29042c
index ecafeaea5f61..85c1918f4656 100644
--- a/sql/hive/src/test/resources/golden/alter4-5-9c36cac1372650b703400c60dd29042c
+++ b/sql/hive/src/test/resources/golden/alter4-5-9c36cac1372650b703400c60dd29042c
@@ -1,2 +1,2 @@
src
-srcpart
\ No newline at end of file
+srcpart
diff --git a/sql/hive/src/test/resources/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf b/sql/hive/src/test/resources/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf
index e180d4c53ae7..42a5d5014be8 100644
--- a/sql/hive/src/test/resources/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf
+++ b/sql/hive/src/test/resources/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf
@@ -1,9 +1,9 @@
-col1 string None
-dt string None
+col1 string
+dt string
# Partition Information
# col_name data_type comment
-dt string None
+dt string
-Detailed Partition Information Partition(values:[a], dbName:alter5_db, tableName:alter5, createTime:1390897177, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/alter5_db.db/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1390897177})
+Detailed Partition Information Partition(values:[a], dbName:alter5_db, tableName:alter5, createTime:1411141192, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse1046274063941765216/alter5_db.db/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1411141192})
diff --git a/sql/hive/src/test/resources/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf b/sql/hive/src/test/resources/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf
index 3d4ba2636ab6..c8378a592637 100644
--- a/sql/hive/src/test/resources/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf
+++ b/sql/hive/src/test/resources/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf
@@ -1,9 +1,9 @@
-col1 string None
-dt string None
+col1 string
+dt string
# Partition Information
# col_name data_type comment
-dt string None
+dt string
-Detailed Partition Information Partition(values:[a], dbName:alter5_db, tableName:alter5, createTime:1390897177, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/alter5_db.db/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1390897186, numRows=6, totalSize=12, rawDataSize=6})
+Detailed Partition Information Partition(values:[a], dbName:alter5_db, tableName:alter5, createTime:1411141192, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse1046274063941765216/alter5_db.db/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1411141198, COLUMN_STATS_ACCURATE=true, totalSize=12, numRows=6, rawDataSize=6})
diff --git a/sql/hive/src/test/resources/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf b/sql/hive/src/test/resources/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf
index 6669b628fc22..091bd8494d7a 100644
--- a/sql/hive/src/test/resources/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf
+++ b/sql/hive/src/test/resources/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf
@@ -1,9 +1,9 @@
-col1 string None
-dt string None
+col1 string
+dt string
# Partition Information
# col_name data_type comment
-dt string None
+dt string
-Detailed Partition Information Partition(values:[a], dbName:default, tableName:alter5, createTime:1390897166, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1390897166})
+Detailed Partition Information Partition(values:[a], dbName:default, tableName:alter5, createTime:1411141186, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse1046274063941765216/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1411141186})
diff --git a/sql/hive/src/test/resources/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf b/sql/hive/src/test/resources/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf
index e866ae0fa765..2bb337a32c69 100644
--- a/sql/hive/src/test/resources/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf
+++ b/sql/hive/src/test/resources/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf
@@ -1,9 +1,9 @@
-col1 string None
-dt string None
+col1 string
+dt string
# Partition Information
# col_name data_type comment
-dt string None
+dt string
-Detailed Partition Information Partition(values:[a], dbName:default, tableName:alter5, createTime:1390897166, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1390897176, numRows=6, totalSize=12, rawDataSize=6})
+Detailed Partition Information Partition(values:[a], dbName:default, tableName:alter5, createTime:1411141186, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse1046274063941765216/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1411141192, COLUMN_STATS_ACCURATE=true, totalSize=12, numRows=6, rawDataSize=6})
diff --git a/sql/hive/src/test/resources/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31 b/sql/hive/src/test/resources/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31
index 2860d5c6b4cf..5dabc616579a 100644
--- a/sql/hive/src/test/resources/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31
+++ b/sql/hive/src/test/resources/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31
@@ -1,5 +1,5 @@
-key int None
+key int
_bucketname string
_offsets array
-Detailed Table Information Table(tableName:default__src_src_index_8__, dbName:default, owner:null, createTime:1389733869, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2023038695216118221/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389733869}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:default__src_src_index_8__, dbName:default, owner:root, createTime:1411193060, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/tmp/sparkHiveWarehouse2779974130068756517/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1411193060}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 b/sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31
index 2860d5c6b4cf..5dabc616579a 100644
--- a/sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31
+++ b/sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31
@@ -1,5 +1,5 @@
-key int None
+key int
_bucketname string
_offsets array
-Detailed Table Information Table(tableName:default__src_src_index_8__, dbName:default, owner:null, createTime:1389733869, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2023038695216118221/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389733869}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:default__src_src_index_8__, dbName:default, owner:root, createTime:1411193060, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/tmp/sparkHiveWarehouse2779974130068756517/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1411193060}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d b/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d
index ecafeaea5f61..85c1918f4656 100644
--- a/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d
+++ b/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d
@@ -1,2 +1,2 @@
src
-srcpart
\ No newline at end of file
+srcpart
diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 b/sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30
index 618c1d01b726..7b3d44b1d985 100644
--- a/sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30
+++ b/sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30
@@ -1,12 +1,12 @@
-key int from deserializer
-value string from deserializer
-ds string None
-ts string None
+key int
+value string
+ds string
+ts string
# Partition Information
# col_name data_type comment
-ds string None
-ts string None
+ds string
+ts string
-Detailed Partition Information Partition(values:[2012-01-03, 2012-01-03+14:46:31], dbName:default, tableName:src_rc_merge_test_part, createTime:1389728902, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/src_rc_merge_test_part/ds=2012-01-03/ts=2012-01-03+14%3A46%3A31, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1389728902})
\ No newline at end of file
+Detailed Partition Information Partition(values:[2012-01-03, 2012-01-03+14:46:31], dbName:default, tableName:src_rc_merge_test_part, createTime:1411176376, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:ts, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/src_rc_merge_test_part/ds=2012-01-03/ts=2012-01-03+14%3A46%3A31, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1411176376})
diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8
index db182e444d31..45b0151b37d6 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8
+++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8
@@ -1,4 +1,4 @@
-key int None
-value string None
+key int
+value string
-Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1388805891}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:root, createTime:1411176383, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/alter_partition_format_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1411176383}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321
index 81e23f0bc195..33c1cdea7eb3 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321
+++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321
@@ -1,10 +1,10 @@
-key int from deserializer
-value string from deserializer
-ds string None
+key int
+value string
+ds string
# Partition Information
# col_name data_type comment
-ds string None
+ds string
-Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1388805891, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388805891, transient_lastDdlTime=1388805891})
\ No newline at end of file
+Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1411176383, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=root, last_modified_time=1411176384, transient_lastDdlTime=1411176384, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1})
diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321
index 6dc1f3ca2c18..8a75b3d58aa5 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321
+++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321
@@ -1,10 +1,10 @@
-key int from deserializer
-value string from deserializer
-ds string None
+key int
+value string
+ds string
# Partition Information
# col_name data_type comment
-ds string None
+ds string
-Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1388805891, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388805891, transient_lastDdlTime=1388805891})
\ No newline at end of file
+Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1411176383, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/test/test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=root, last_modified_time=1411176384, transient_lastDdlTime=1411176384, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1})
diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8
index 4b754043d63a..acd74d368cb6 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8
+++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8
@@ -1,10 +1,10 @@
-key int None
-value string None
-ds string None
+key int
+value string
+ds string
# Partition Information
# col_name data_type comment
-ds string None
+ds string
-Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1388805891}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:root, createTime:1411176383, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/alter_partition_format_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1411176383}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8
index f44c28ee3676..a2083c38bdb0 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8
+++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8
@@ -1,10 +1,10 @@
-key int from deserializer
-value string from deserializer
-ds string None
+key int
+value string
+ds string
# Partition Information
# col_name data_type comment
-ds string None
+ds string
-Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388805892, transient_lastDdlTime=1388805892}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:root, createTime:1411176383, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=root, last_modified_time=1411176384, transient_lastDdlTime=1411176384}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8
index e739ad4992ec..90afa01ed14e 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8
+++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8
@@ -1,10 +1,10 @@
-key int from deserializer
-value string from deserializer
-ds string None
+key int
+value string
+ds string
# Partition Information
# col_name data_type comment
-ds string None
+ds string
-Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388805892, transient_lastDdlTime=1388805892}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:root, createTime:1411176383, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=root, last_modified_time=1411176384, transient_lastDdlTime=1411176384}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8
index 092e5ed6e8a4..91e52f813065 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8
+++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8
@@ -1,4 +1,4 @@
-key int from deserializer
-value string from deserializer
+key int
+value string
-Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388805891, transient_lastDdlTime=1388805891}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:root, createTime:1411176383, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=root, last_modified_time=1411176383, transient_lastDdlTime=1411176383, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8
index 197e67d09bf4..7a56e81126af 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8
+++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8
@@ -1,4 +1,4 @@
-key int from deserializer
-value string from deserializer
+key int
+value string
-Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388805891, transient_lastDdlTime=1388805891}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:root, createTime:1411176383, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=root, last_modified_time=1411176383, transient_lastDdlTime=1411176383, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321
index 29b5b693b858..3771c962c08f 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321
+++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321
@@ -1,10 +1,10 @@
-key int None
-value string None
-ds string None
+key int
+value string
+ds string
# Partition Information
# col_name data_type comment
-ds string None
+ds string
-Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1388805891, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388805891})
\ No newline at end of file
+Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1411176383, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1411176383})
diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 b/sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879
index ccd6518a50f7..f92deb2abf18 100644
--- a/sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879
+++ b/sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879
@@ -1,5 +1,5 @@
-id int None
-query string None
-name string None
+id int
+query string
+name string
-Detailed Table Information Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1388805893, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1388805893}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:test_table, dbName:default, owner:root, createTime:1411176392, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1411176392}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8
index d135e450e6e8..51bcca13cbf3 100644
--- a/sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8
+++ b/sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8
@@ -1,11 +1,11 @@
-id int None
-query string None
-name string None
-dt string None
+id int
+query string
+name string
+dt string
# Partition Information
# col_name data_type comment
-dt string None
+dt string
-Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388805893})
\ No newline at end of file
+Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1411176392, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1411176392})
diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8
index d135e450e6e8..51bcca13cbf3 100644
--- a/sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8
+++ b/sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8
@@ -1,11 +1,11 @@
-id int None
-query string None
-name string None
-dt string None
+id int
+query string
+name string
+dt string
# Partition Information
# col_name data_type comment
-dt string None
+dt string
-Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388805893})
\ No newline at end of file
+Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1411176392, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1411176392})
diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8
index c3fac88f91a3..d4c04792618f 100644
--- a/sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8
+++ b/sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8
@@ -1,11 +1,11 @@
-id int None
-query string None
-name string None
-dt string None
+id int
+query string
+name string
+dt string
# Partition Information
# col_name data_type comment
-dt string None
+dt string
-Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388805893, transient_lastDdlTime=1388805893})
\ No newline at end of file
+Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1411176392, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=root, last_modified_time=1411176393, transient_lastDdlTime=1411176393, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1})
diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8
index 6c8f91de7cde..8036b5b9a88e 100644
--- a/sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8
+++ b/sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8
@@ -1,11 +1,11 @@
-id int None
-query string None
-name string None
-dt string None
+id int
+query string
+name string
+dt string
# Partition Information
# col_name data_type comment
-dt string None
+dt string
-Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1, field.delim=,}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388805893, transient_lastDdlTime=1388805893})
\ No newline at end of file
+Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1411176392, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1, field.delim=,}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=root, last_modified_time=1411176393, transient_lastDdlTime=1411176393, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1})
diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 b/sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879
index 37a5b2cc47ba..33bd59d1b5f5 100644
--- a/sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879
+++ b/sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879
@@ -1,5 +1,5 @@
-id int from deserializer
-query string from deserializer
-name string from deserializer
+id int
+query string
+name string
-Detailed Table Information Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1388805893, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:from deserializer), FieldSchema(name:query, type:string, comment:from deserializer), FieldSchema(name:name, type:string, comment:from deserializer)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388805893, transient_lastDdlTime=1388805893}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:test_table, dbName:default, owner:root, createTime:1411176392, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=root, last_modified_time=1411176392, transient_lastDdlTime=1411176392, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 b/sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879
index 0348dd15fd4f..ed1222ae50ec 100644
--- a/sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879
+++ b/sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879
@@ -1,5 +1,5 @@
-id int from deserializer
-query string from deserializer
-name string from deserializer
+id int
+query string
+name string
-Detailed Table Information Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1388805893, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:from deserializer), FieldSchema(name:query, type:string, comment:from deserializer), FieldSchema(name:name, type:string, comment:from deserializer)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1, field.delim=,}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388805893, transient_lastDdlTime=1388805893}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
\ No newline at end of file
+Detailed Table Information Table(tableName:test_table, dbName:default, owner:root, createTime:1411176392, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1, field.delim=,}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=root, last_modified_time=1411176392, transient_lastDdlTime=1411176392, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8
index d135e450e6e8..51bcca13cbf3 100644
--- a/sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8
+++ b/sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8
@@ -1,11 +1,11 @@
-id int None
-query string None
-name string None
-dt string None
+id int
+query string
+name string
+dt string
# Partition Information
# col_name data_type comment
-dt string None
+dt string
-Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388805893})
\ No newline at end of file
+Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1411176392, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/tmp/sparkHiveWarehouse6393134224728886583/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1411176392})
diff --git a/sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b b/sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b
index a99747531cef..cf72972a19a3 100644
--- a/sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b
+++ b/sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b
@@ -1,20 +1,20 @@
# col_name data_type comment
-key int None
-value string None
-ds string None
-hr string None
+key int
+value string
+ds string
+hr string
# Detailed Table Information
Database: default
-Owner: marmbrus
-CreateTime: Fri Feb 07 14:47:52 PST 2014
+Owner: root
+CreateTime: Fri Sep 19 08:40:29 PDT 2014
LastAccessTime: UNKNOWN
Protect Mode: None
Retention: 0
Table Type: VIRTUAL_VIEW
Table Parameters:
- transient_lastDdlTime 1391813272
+ transient_lastDdlTime 1411141229
# Storage Information
SerDe Library: null
diff --git a/sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b b/sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b
index 499c73127d89..daa8bd1d7aa4 100644
--- a/sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b
+++ b/sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b
@@ -1,17 +1,17 @@
# col_name data_type comment
-value string None
+value string
# Detailed Table Information
Database: default
-Owner: marmbrus
-CreateTime: Fri Feb 07 14:47:52 PST 2014
+Owner: root
+CreateTime: Fri Sep 19 08:40:29 PDT 2014
LastAccessTime: UNKNOWN
Protect Mode: None
Retention: 0
Table Type: VIRTUAL_VIEW
Table Parameters:
- transient_lastDdlTime 1391813272
+ transient_lastDdlTime 1411141229
# Storage Information
SerDe Library: null
diff --git a/sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b b/sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b
index a5fba77abdf0..ba39ab225627 100644
--- a/sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b
+++ b/sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b
@@ -1,18 +1,18 @@
# col_name data_type comment
-key int None
-value string None
+key int
+value string
# Detailed Table Information
Database: default
-Owner: marmbrus
-CreateTime: Fri Feb 07 14:47:52 PST 2014
+Owner: root
+CreateTime: Fri Sep 19 08:40:29 PDT 2014
LastAccessTime: UNKNOWN
Protect Mode: None
Retention: 0
Table Type: VIRTUAL_VIEW
Table Parameters:
- transient_lastDdlTime 1391813272
+ transient_lastDdlTime 1411141229
# Storage Information
SerDe Library: null
diff --git a/sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 b/sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59
index c615f0148ccd..d35bf9093ca9 100644
--- a/sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59
+++ b/sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59
@@ -19,9 +19,9 @@ my_stringset struct<> from deserializer
my_enumset struct<> from deserializer
my_structset struct<> from deserializer
optionals struct<> from deserializer
-b string None
+b string
# Partition Information
# col_name data_type comment
-b string None
\ No newline at end of file
+b string
diff --git a/sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc b/sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc
index 83f572c42492..90f9bd0430a4 100644
--- a/sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc
+++ b/sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc
@@ -1,2 +1,2 @@
-key int None
-value string None
\ No newline at end of file
+key int
+value string
diff --git a/sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c b/sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c
index 83f572c42492..90f9bd0430a4 100644
--- a/sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c
+++ b/sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c
@@ -1,2 +1,2 @@
-key int None
-value string None
\ No newline at end of file
+key int
+value string
diff --git a/sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 b/sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5
index 83f572c42492..90f9bd0430a4 100644
--- a/sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5
+++ b/sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5
@@ -1,2 +1,2 @@
-key int None
-value string None
\ No newline at end of file
+key int
+value string
diff --git a/sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 b/sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705
index 3e2a912824ab..eb53a4788a5e 100644
--- a/sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705
+++ b/sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705
@@ -1,19 +1,19 @@
# col_name data_type comment
-a string None
-b string None
+a string
+b string
# Detailed Table Information
Database: default
-Owner: marmbrus
-CreateTime: Fri Feb 07 14:52:37 PST 2014
+Owner: root
+CreateTime: Fri Sep 19 18:30:09 PDT 2014
LastAccessTime: UNKNOWN
Protect Mode: None
Retention: 0
-Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table2
+Location: file:/tmp/sparkHiveWarehouse6393134224728886583/table2
Table Type: MANAGED_TABLE
Table Parameters:
- transient_lastDdlTime 1391813557
+ transient_lastDdlTime 1411176609
# Storage Information
SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
diff --git a/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 b/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3
index 115d12fb70c8..26b598948875 100644
--- a/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3
+++ b/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3
@@ -1,2 +1,2 @@
-a string None
-b string None
+a string
+b string
diff --git a/sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 b/sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7
index 61d34badb1a2..b6822c01776a 100644
--- a/sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7
+++ b/sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7
@@ -1,20 +1,20 @@
# col_name data_type comment
-a string None
-b string None
+a string
+b string
# Detailed Table Information
Database: default
-Owner: marmbrus
-CreateTime: Fri Feb 07 14:52:37 PST 2014
+Owner: root
+CreateTime: Fri Sep 19 18:30:09 PDT 2014
LastAccessTime: UNKNOWN
Protect Mode: None
Retention: 0
-Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table3
+Location: file:/tmp/sparkHiveWarehouse6393134224728886583/table3
Table Type: EXTERNAL_TABLE
Table Parameters:
EXTERNAL TRUE
- transient_lastDdlTime 1391813557
+ transient_lastDdlTime 1411176609
# Storage Information
SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
diff --git a/sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d b/sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d
index 4c2967215fe6..362614202d30 100644
--- a/sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d
+++ b/sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d
@@ -1,25 +1,25 @@
# col_name data_type comment
-key int None
-value string None
+key int
+value string
# Partition Information
# col_name data_type comment
-ds string None
-hr string None
+ds string
+hr string
# Detailed Table Information
Database: default
-Owner: marmbrus
-CreateTime: Fri Feb 07 14:53:16 PST 2014
+Owner: root
+CreateTime: Fri Sep 19 18:30:34 PDT 2014
LastAccessTime: UNKNOWN
Protect Mode: None
Retention: 0
-Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table1
+Location: file:/tmp/sparkHiveWarehouse6393134224728886583/table1
Table Type: MANAGED_TABLE
Table Parameters:
- transient_lastDdlTime 1391813596
+ transient_lastDdlTime 1411176634
# Storage Information
SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
diff --git a/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 b/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084
index 115d12fb70c8..26b598948875 100644
--- a/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084
+++ b/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084
@@ -1,2 +1,2 @@
-a string None
-b string None
+a string
+b string
diff --git a/sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af b/sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af
index 02dee147bca4..0ddae33d3acf 100644
--- a/sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af
+++ b/sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af
@@ -1,19 +1,19 @@
# col_name data_type comment
-a string None
-b string None
+a string
+b string
# Detailed Table Information
Database: default
-Owner: marmbrus
-CreateTime: Fri Feb 07 14:52:37 PST 2014
+Owner: root
+CreateTime: Fri Sep 19 18:30:09 PDT 2014
LastAccessTime: UNKNOWN
Protect Mode: None
Retention: 0
-Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table1
+Location: file:/tmp/sparkHiveWarehouse6393134224728886583/table1
Table Type: MANAGED_TABLE
Table Parameters:
- transient_lastDdlTime 1391813557
+ transient_lastDdlTime 1411176609
# Storage Information
SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
diff --git a/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a b/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a
index 115d12fb70c8..26b598948875 100644
--- a/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a
+++ b/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a
@@ -1,2 +1,2 @@
-a string None
-b string None
+a string
+b string
diff --git a/sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 b/sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084
index c49ad26c04d6..6485e594f54f 100644
--- a/sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084
+++ b/sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084
@@ -1,4 +1,4 @@
-a string None
-b array None
-c array