Skip to content

Commit

Permalink
fix comment missing issue for thrift server
Browse files Browse the repository at this point in the history
  • Loading branch information
bomeng committed Mar 29, 2017
1 parent c622a87 commit 69f2172
Show file tree
Hide file tree
Showing 2 changed files with 14 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -292,7 +292,7 @@ object SparkExecuteStatementOperation {
def getTableSchema(structType: StructType): TableSchema = {
val schema = structType.map { field =>
val attrTypeString = if (field.dataType == NullType) "void" else field.dataType.catalogString
new FieldSchema(field.name, attrTypeString, "")
new FieldSchema(field.name, attrTypeString, field.getComment.getOrElse(""))
}
new TableSchema(schema.asJava)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.sql.hive.thriftserver

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.types.{NullType, StructField, StructType}
import org.apache.spark.sql.types.{IntegerType, NullType, StringType, StructField, StructType}

class SparkExecuteStatementOperationSuite extends SparkFunSuite {
test("SPARK-17112 `select null` via JDBC triggers IllegalArgumentException in ThriftServer") {
Expand All @@ -30,4 +30,16 @@ class SparkExecuteStatementOperationSuite extends SparkFunSuite {
assert(columns.get(0).getType() == org.apache.hive.service.cli.Type.NULL_TYPE)
assert(columns.get(1).getType() == org.apache.hive.service.cli.Type.NULL_TYPE)
}

test("SPARK-20146 Comment should be preserved") {
val field1 = StructField("column1", StringType).withComment("comment 1")
val field2 = StructField("column2", IntegerType)
val tableSchema = StructType(Seq(field1, field2))
val columns = SparkExecuteStatementOperation.getTableSchema(tableSchema).getColumnDescriptors()
assert(columns.size() == 2)
assert(columns.get(0).getType() == org.apache.hive.service.cli.Type.STRING_TYPE)
assert(columns.get(0).getComment() == "comment 1")
assert(columns.get(1).getType() == org.apache.hive.service.cli.Type.INT_TYPE)
assert(columns.get(1).getComment() == "")
}
}

0 comments on commit 69f2172

Please sign in to comment.