Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
60 commits
Select commit Hold shift + click to select a range
01e4cdf
Merge remote-tracking branch 'upstream/master'
gatorsmile Nov 13, 2015
6835704
Merge remote-tracking branch 'upstream/master'
gatorsmile Nov 14, 2015
9180687
Merge remote-tracking branch 'upstream/master'
gatorsmile Nov 14, 2015
b38a21e
SPARK-11633
gatorsmile Nov 17, 2015
d2b84af
Merge remote-tracking branch 'upstream/master' into joinMakeCopy
gatorsmile Nov 17, 2015
fda8025
Merge remote-tracking branch 'upstream/master'
gatorspark Nov 17, 2015
ac0dccd
Merge branch 'master' of https://github.com/gatorsmile/spark
gatorspark Nov 17, 2015
6e0018b
Merge remote-tracking branch 'upstream/master'
Nov 20, 2015
0546772
converge
gatorsmile Nov 20, 2015
b37a64f
converge
gatorsmile Nov 20, 2015
c2a872c
Merge remote-tracking branch 'upstream/master'
gatorsmile Jan 6, 2016
ab6dbd7
Merge remote-tracking branch 'upstream/master'
gatorsmile Jan 6, 2016
4276356
Merge remote-tracking branch 'upstream/master'
gatorsmile Jan 6, 2016
2dab708
Merge remote-tracking branch 'upstream/master'
gatorsmile Jan 7, 2016
0458770
Merge remote-tracking branch 'upstream/master'
gatorsmile Jan 8, 2016
1debdfa
Merge remote-tracking branch 'upstream/master'
gatorsmile Jan 9, 2016
763706d
Merge remote-tracking branch 'upstream/master'
gatorsmile Jan 14, 2016
4de6ec1
Merge remote-tracking branch 'upstream/master'
gatorsmile Jan 18, 2016
9422a4f
Merge remote-tracking branch 'upstream/master'
gatorsmile Jan 19, 2016
52bdf48
Merge remote-tracking branch 'upstream/master'
gatorsmile Jan 20, 2016
1e95df3
Merge remote-tracking branch 'upstream/master'
gatorsmile Jan 23, 2016
fab24cf
Merge remote-tracking branch 'upstream/master'
gatorsmile Feb 1, 2016
8b2e33b
Merge remote-tracking branch 'upstream/master'
gatorsmile Feb 5, 2016
2ee1876
Merge remote-tracking branch 'upstream/master'
gatorsmile Feb 11, 2016
b9f0090
Merge remote-tracking branch 'upstream/master'
gatorsmile Feb 12, 2016
ade6f7e
Merge remote-tracking branch 'upstream/master'
gatorsmile Feb 15, 2016
9fd63d2
Merge remote-tracking branch 'upstream/master'
gatorsmile Feb 19, 2016
5199d49
Merge remote-tracking branch 'upstream/master'
gatorsmile Feb 22, 2016
404214c
Merge remote-tracking branch 'upstream/master'
gatorsmile Feb 23, 2016
c001dd9
Merge remote-tracking branch 'upstream/master'
gatorsmile Feb 25, 2016
59daa48
Merge remote-tracking branch 'upstream/master'
gatorsmile Mar 5, 2016
41d5f64
Merge remote-tracking branch 'upstream/master'
gatorsmile Mar 7, 2016
472a6e3
Merge remote-tracking branch 'upstream/master'
gatorsmile Mar 10, 2016
0fba10a
Merge remote-tracking branch 'upstream/master'
gatorsmile Mar 12, 2016
cbf73b3
Merge remote-tracking branch 'upstream/master'
gatorsmile Mar 21, 2016
c08f561
Merge remote-tracking branch 'upstream/master'
gatorsmile Mar 22, 2016
474df88
Merge remote-tracking branch 'upstream/master'
gatorsmile Mar 22, 2016
3d9828d
Merge remote-tracking branch 'upstream/master'
gatorsmile Mar 24, 2016
72d2361
Merge remote-tracking branch 'upstream/master'
gatorsmile Mar 26, 2016
07afea5
Merge remote-tracking branch 'upstream/master'
gatorsmile Mar 29, 2016
8bf2007
Merge remote-tracking branch 'upstream/master'
gatorsmile Mar 30, 2016
87a165b
Merge remote-tracking branch 'upstream/master'
gatorsmile Mar 31, 2016
b9359cd
Merge remote-tracking branch 'upstream/master'
gatorsmile Apr 1, 2016
65bd090
Merge remote-tracking branch 'upstream/master'
gatorsmile Apr 5, 2016
babf2da
Merge remote-tracking branch 'upstream/master'
gatorsmile Apr 5, 2016
9e09469
Merge remote-tracking branch 'upstream/master'
gatorsmile Apr 6, 2016
50a8e4a
Merge remote-tracking branch 'upstream/master'
gatorsmile Apr 6, 2016
34e44da
fix 1
gatorsmile Apr 7, 2016
c040e1d
Merge remote-tracking branch 'upstream/master' into alterView
gatorsmile Apr 10, 2016
c8811bc
correct compilation errors
gatorsmile Apr 10, 2016
f3337fa
Merge remote-tracking branch 'upstream/master'
gatorsmile Apr 10, 2016
66cb2c1
Merge remote-tracking branch 'upstream/master' into alterView
gatorsmile Apr 12, 2016
c55ab12
impl
gatorsmile Apr 12, 2016
09cc36d
Merge remote-tracking branch 'upstream/master'
gatorsmile Apr 12, 2016
ed5a7d0
added more test cases.
gatorsmile Apr 12, 2016
cd20636
Merge remote-tracking branch 'upstream/master' into alterView
gatorsmile Apr 12, 2016
83a1915
Merge remote-tracking branch 'upstream/master'
gatorsmile Apr 14, 2016
fdc653f
Merge branch 'alterView' into alterViewNew
gatorsmile Apr 14, 2016
e82aa47
address comments.
gatorsmile Apr 14, 2016
db7f931
address comments.
gatorsmile Apr 14, 2016
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -393,7 +393,8 @@ class SparkSqlAstBuilder extends AstBuilder {
override def visitRenameTable(ctx: RenameTableContext): LogicalPlan = withOrigin(ctx) {
AlterTableRename(
visitTableIdentifier(ctx.from),
visitTableIdentifier(ctx.to))
visitTableIdentifier(ctx.to),
ctx.VIEW != null)
}

/**
Expand All @@ -409,7 +410,8 @@ class SparkSqlAstBuilder extends AstBuilder {
ctx: SetTablePropertiesContext): LogicalPlan = withOrigin(ctx) {
AlterTableSetProperties(
visitTableIdentifier(ctx.tableIdentifier),
visitTablePropertyList(ctx.tablePropertyList))
visitTablePropertyList(ctx.tablePropertyList),
ctx.VIEW != null)
}

/**
Expand All @@ -426,7 +428,8 @@ class SparkSqlAstBuilder extends AstBuilder {
AlterTableUnsetProperties(
visitTableIdentifier(ctx.tableIdentifier),
visitTablePropertyList(ctx.tablePropertyList).keys.toSeq,
ctx.EXISTS != null)
ctx.EXISTS != null,
ctx.VIEW != null)
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import org.apache.spark.internal.Logging
import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogTable}
import org.apache.spark.sql.catalyst.catalog.{CatalogTablePartition, CatalogTableType}
import org.apache.spark.sql.catalyst.catalog.{CatalogTablePartition, CatalogTableType, SessionCatalog}
import org.apache.spark.sql.catalyst.catalog.ExternalCatalog.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.types._
Expand Down Expand Up @@ -235,11 +235,13 @@ case class DropTable(
*/
case class AlterTableSetProperties(
tableName: TableIdentifier,
properties: Map[String, String])
properties: Map[String, String],
isView: Boolean)
extends RunnableCommand {

override def run(sqlContext: SQLContext): Seq[Row] = {
val catalog = sqlContext.sessionState.catalog
DDLUtils.verifyAlterTableType(catalog, tableName, isView)
val table = catalog.getTableMetadata(tableName)
val newProperties = table.properties ++ properties
if (DDLUtils.isDatasourceTable(newProperties)) {
Expand All @@ -265,11 +267,13 @@ case class AlterTableSetProperties(
case class AlterTableUnsetProperties(
tableName: TableIdentifier,
propKeys: Seq[String],
ifExists: Boolean)
ifExists: Boolean,
isView: Boolean)
extends RunnableCommand {

override def run(sqlContext: SQLContext): Seq[Row] = {
val catalog = sqlContext.sessionState.catalog
DDLUtils.verifyAlterTableType(catalog, tableName, isView)
val table = catalog.getTableMetadata(tableName)
if (DDLUtils.isDatasourceTable(table)) {
throw new AnalysisException(
Expand Down Expand Up @@ -513,5 +517,24 @@ private object DDLUtils {
def isDatasourceTable(table: CatalogTable): Boolean = {
isDatasourceTable(table.properties)
}

/**
* If the command ALTER VIEW is to alter a table or ALTER TABLE is to alter a view,
* issue an exception [[AnalysisException]].
*/
def verifyAlterTableType(
catalog: SessionCatalog,
tableIdentifier: TableIdentifier,
isView: Boolean): Unit = {
catalog.getTableMetadataOption(tableIdentifier).map(_.tableType match {
case CatalogTableType.VIRTUAL_VIEW if !isView =>
throw new AnalysisException(
"Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead")
case o if o != CatalogTableType.VIRTUAL_VIEW && isView =>
throw new AnalysisException(
s"Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead")
case _ =>
})
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -67,11 +67,13 @@ case class CreateTable(table: CatalogTable, ifNotExists: Boolean) extends Runnab
*/
case class AlterTableRename(
oldName: TableIdentifier,
newName: TableIdentifier)
newName: TableIdentifier,
isView: Boolean)
extends RunnableCommand {

override def run(sqlContext: SQLContext): Seq[Row] = {
val catalog = sqlContext.sessionState.catalog
DDLUtils.verifyAlterTableType(catalog, oldName, isView)
catalog.invalidateTable(oldName)
catalog.renameTable(oldName, newName)
Seq.empty[Row]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -214,10 +214,12 @@ class DDLCommandSuite extends PlanTest {
val parsed_view = parser.parsePlan(sql_view)
val expected_table = AlterTableRename(
TableIdentifier("table_name", None),
TableIdentifier("new_table_name", None))
TableIdentifier("new_table_name", None),
isView = false)
val expected_view = AlterTableRename(
TableIdentifier("table_name", None),
TableIdentifier("new_table_name", None))
TableIdentifier("new_table_name", None),
isView = true)
comparePlans(parsed_table, expected_table)
comparePlans(parsed_view, expected_view)
}
Expand All @@ -244,14 +246,14 @@ class DDLCommandSuite extends PlanTest {

val tableIdent = TableIdentifier("table_name", None)
val expected1_table = AlterTableSetProperties(
tableIdent, Map("test" -> "test", "comment" -> "new_comment"))
tableIdent, Map("test" -> "test", "comment" -> "new_comment"), isView = false)
val expected2_table = AlterTableUnsetProperties(
tableIdent, Seq("comment", "test"), ifExists = false)
tableIdent, Seq("comment", "test"), ifExists = false, isView = false)
val expected3_table = AlterTableUnsetProperties(
tableIdent, Seq("comment", "test"), ifExists = true)
val expected1_view = expected1_table
val expected2_view = expected2_table
val expected3_view = expected3_table
tableIdent, Seq("comment", "test"), ifExists = true, isView = false)
val expected1_view = expected1_table.copy(isView = true)
val expected2_view = expected2_table.copy(isView = true)
val expected3_view = expected3_table.copy(isView = true)

comparePlans(parsed1_table, expected1_table)
comparePlans(parsed2_table, expected2_table)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -131,6 +131,118 @@ class HiveDDLSuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
}
}

test("alter views - rename") {
val tabName = "tab1"
withTable(tabName) {
sqlContext.range(10).write.saveAsTable(tabName)
val oldViewName = "view1"
val newViewName = "view2"
withView(oldViewName, newViewName) {
val catalog = hiveContext.sessionState.catalog
sql(s"CREATE VIEW $oldViewName AS SELECT * FROM $tabName")

assert(catalog.tableExists(TableIdentifier(oldViewName)))
assert(!catalog.tableExists(TableIdentifier(newViewName)))
sql(s"ALTER VIEW $oldViewName RENAME TO $newViewName")
assert(!catalog.tableExists(TableIdentifier(oldViewName)))
assert(catalog.tableExists(TableIdentifier(newViewName)))
}
}
}

test("alter views - set/unset tblproperties") {
val tabName = "tab1"
withTable(tabName) {
sqlContext.range(10).write.saveAsTable(tabName)
val viewName = "view1"
withView(viewName) {
val catalog = hiveContext.sessionState.catalog
sql(s"CREATE VIEW $viewName AS SELECT * FROM $tabName")

assert(catalog.getTableMetadata(TableIdentifier(viewName))
.properties.filter(_._1 != "transient_lastDdlTime") == Map())
sql(s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'an')")
assert(catalog.getTableMetadata(TableIdentifier(viewName))
.properties.filter(_._1 != "transient_lastDdlTime") == Map("p" -> "an"))

// no exception or message will be issued if we set it again
sql(s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'an')")
assert(catalog.getTableMetadata(TableIdentifier(viewName))
.properties.filter(_._1 != "transient_lastDdlTime") == Map("p" -> "an"))

// the value will be updated if we set the same key to a different value
sql(s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'b')")
assert(catalog.getTableMetadata(TableIdentifier(viewName))
.properties.filter(_._1 != "transient_lastDdlTime") == Map("p" -> "b"))

sql(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')")
assert(catalog.getTableMetadata(TableIdentifier(viewName))
.properties.filter(_._1 != "transient_lastDdlTime") == Map())

val message = intercept[AnalysisException] {
sql(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')")
}.getMessage
assert(message.contains(
"attempted to unset non-existent property 'p' in table '`view1`'"))
}
}
}

test("alter views and alter table - misuse") {
val tabName = "tab1"
withTable(tabName) {
sqlContext.range(10).write.saveAsTable(tabName)
val oldViewName = "view1"
val newViewName = "view2"
withView(oldViewName, newViewName) {
val catalog = hiveContext.sessionState.catalog
sql(s"CREATE VIEW $oldViewName AS SELECT * FROM $tabName")

assert(catalog.tableExists(TableIdentifier(tabName)))
assert(catalog.tableExists(TableIdentifier(oldViewName)))

var message = intercept[AnalysisException] {
sql(s"ALTER VIEW $tabName RENAME TO $newViewName")
}.getMessage
assert(message.contains(
"Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead"))

message = intercept[AnalysisException] {
sql(s"ALTER VIEW $tabName SET TBLPROPERTIES ('p' = 'an')")
}.getMessage
assert(message.contains(
"Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead"))

message = intercept[AnalysisException] {
sql(s"ALTER VIEW $tabName UNSET TBLPROPERTIES ('p')")
}.getMessage
assert(message.contains(
"Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead"))

message = intercept[AnalysisException] {
sql(s"ALTER TABLE $oldViewName RENAME TO $newViewName")
}.getMessage
assert(message.contains(
"Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead"))

message = intercept[AnalysisException] {
sql(s"ALTER TABLE $oldViewName SET TBLPROPERTIES ('p' = 'an')")
}.getMessage
assert(message.contains(
"Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead"))

message = intercept[AnalysisException] {
sql(s"ALTER TABLE $oldViewName UNSET TBLPROPERTIES ('p')")
}.getMessage
assert(message.contains(
"Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead"))

assert(catalog.tableExists(TableIdentifier(tabName)))
assert(catalog.tableExists(TableIdentifier(oldViewName)))
}
}
}

test("drop table using drop view") {
withTable("tab1") {
sql("CREATE TABLE tab1(c1 int)")
Expand Down