spark git commit: [SPARK-15538][SPARK-15539][SQL] Truncate table fixes round 2

2016-05-26 Thread andrewor14
Repository: spark
Updated Branches:
  refs/heads/branch-2.0 d3cd579d7 -> c99ad9215


[SPARK-15538][SPARK-15539][SQL] Truncate table fixes round 2

## What changes were proposed in this pull request?

Two more changes:
(1) Fix truncate table for data source tables (only for cases without 
`PARTITION`)
(2) Disallow truncating external tables or views

## How was this patch tested?

`DDLSuite`

Author: Andrew Or 

Closes #13315 from andrewor14/truncate-table.


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

Branch: refs/heads/branch-2.0
Commit: c99ad9215cea7f0f983ce06e9cbdbcfa7bd092bf
Parents: d3cd579
Author: Andrew Or 
Authored: Thu May 26 19:01:41 2016 -0700
Committer: Andrew Or 
Committed: Thu May 26 19:02:28 2016 -0700

--
 .../spark/sql/execution/command/tables.scala| 78 +---
 .../spark/sql/execution/command/DDLSuite.scala  | 34 +
 2 files changed, 86 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/c99ad921/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
index bef4c92..e34beec 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
@@ -285,41 +285,67 @@ case class TruncateTableCommand(
 tableName: TableIdentifier,
 partitionSpec: Option[TablePartitionSpec]) extends RunnableCommand {
 
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-val catalog = sparkSession.sessionState.catalog
+  override def run(spark: SparkSession): Seq[Row] = {
+val catalog = spark.sessionState.catalog
 if (!catalog.tableExists(tableName)) {
   throw new AnalysisException(s"Table '$tableName' in TRUNCATE TABLE does 
not exist.")
-} else if (catalog.isTemporaryTable(tableName)) {
+}
+if (catalog.isTemporaryTable(tableName)) {
   throw new AnalysisException(
 s"Operation not allowed: TRUNCATE TABLE on temporary tables: 
'$tableName'")
-} else {
-  val locations = if (partitionSpec.isDefined) {
-catalog.listPartitions(tableName, 
partitionSpec).map(_.storage.locationUri)
+}
+val table = catalog.getTableMetadata(tableName)
+if (table.tableType == CatalogTableType.EXTERNAL) {
+  throw new AnalysisException(
+s"Operation not allowed: TRUNCATE TABLE on external tables: 
'$tableName'")
+}
+if (table.tableType == CatalogTableType.VIEW) {
+  throw new AnalysisException(
+s"Operation not allowed: TRUNCATE TABLE on views: '$tableName'")
+}
+val isDatasourceTable = DDLUtils.isDatasourceTable(table)
+if (isDatasourceTable && partitionSpec.isDefined) {
+  throw new AnalysisException(
+s"Operation not allowed: TRUNCATE TABLE ... PARTITION is not supported 
" +
+s"for tables created using the data sources API: '$tableName'")
+}
+if (table.partitionColumnNames.isEmpty && partitionSpec.isDefined) {
+  throw new AnalysisException(
+s"Operation not allowed: TRUNCATE TABLE ... PARTITION is not supported 
" +
+s"for tables that are not partitioned: '$tableName'")
+}
+val locations =
+  if (isDatasourceTable || table.partitionColumnNames.isEmpty) {
+Seq(table.storage.locationUri)
   } else {
-val table = catalog.getTableMetadata(tableName)
-if (table.partitionColumnNames.nonEmpty) {
-  catalog.listPartitions(tableName).map(_.storage.locationUri)
-} else {
-  Seq(table.storage.locationUri)
-}
+catalog.listPartitions(tableName, 
partitionSpec).map(_.storage.locationUri)
   }
-  val hadoopConf = sparkSession.sessionState.newHadoopConf()
-  locations.foreach { location =>
-if (location.isDefined) {
-  val path = new Path(location.get)
-  try {
-val fs = path.getFileSystem(hadoopConf)
-fs.delete(path, true)
-fs.mkdirs(path)
-  } catch {
-case NonFatal(e) =>
-  throw new AnalysisException(
-s"Failed to truncate table '$tableName' when removing data of 
the path: $path " +
-  s"because of ${e.toString}")
-  }
+val hadoopConf = spark.sessionState.newHadoopConf()
+locations.foreach { location =>
+  if 

spark git commit: [SPARK-15538][SPARK-15539][SQL] Truncate table fixes round 2

2016-05-26 Thread andrewor14
Repository: spark
Updated Branches:
  refs/heads/master 3ac2363d7 -> 008a5377d


[SPARK-15538][SPARK-15539][SQL] Truncate table fixes round 2

## What changes were proposed in this pull request?

Two more changes:
(1) Fix truncate table for data source tables (only for cases without 
`PARTITION`)
(2) Disallow truncating external tables or views

## How was this patch tested?

`DDLSuite`

Author: Andrew Or 

Closes #13315 from andrewor14/truncate-table.


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

Branch: refs/heads/master
Commit: 008a5377d57ce6692eca4a41539fb27978b58e01
Parents: 3ac2363
Author: Andrew Or 
Authored: Thu May 26 19:01:41 2016 -0700
Committer: Andrew Or 
Committed: Thu May 26 19:01:41 2016 -0700

--
 .../spark/sql/execution/command/tables.scala| 78 +---
 .../spark/sql/execution/command/DDLSuite.scala  | 34 +
 2 files changed, 86 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/spark/blob/008a5377/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
--
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
index bef4c92..e34beec 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
@@ -285,41 +285,67 @@ case class TruncateTableCommand(
 tableName: TableIdentifier,
 partitionSpec: Option[TablePartitionSpec]) extends RunnableCommand {
 
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-val catalog = sparkSession.sessionState.catalog
+  override def run(spark: SparkSession): Seq[Row] = {
+val catalog = spark.sessionState.catalog
 if (!catalog.tableExists(tableName)) {
   throw new AnalysisException(s"Table '$tableName' in TRUNCATE TABLE does 
not exist.")
-} else if (catalog.isTemporaryTable(tableName)) {
+}
+if (catalog.isTemporaryTable(tableName)) {
   throw new AnalysisException(
 s"Operation not allowed: TRUNCATE TABLE on temporary tables: 
'$tableName'")
-} else {
-  val locations = if (partitionSpec.isDefined) {
-catalog.listPartitions(tableName, 
partitionSpec).map(_.storage.locationUri)
+}
+val table = catalog.getTableMetadata(tableName)
+if (table.tableType == CatalogTableType.EXTERNAL) {
+  throw new AnalysisException(
+s"Operation not allowed: TRUNCATE TABLE on external tables: 
'$tableName'")
+}
+if (table.tableType == CatalogTableType.VIEW) {
+  throw new AnalysisException(
+s"Operation not allowed: TRUNCATE TABLE on views: '$tableName'")
+}
+val isDatasourceTable = DDLUtils.isDatasourceTable(table)
+if (isDatasourceTable && partitionSpec.isDefined) {
+  throw new AnalysisException(
+s"Operation not allowed: TRUNCATE TABLE ... PARTITION is not supported 
" +
+s"for tables created using the data sources API: '$tableName'")
+}
+if (table.partitionColumnNames.isEmpty && partitionSpec.isDefined) {
+  throw new AnalysisException(
+s"Operation not allowed: TRUNCATE TABLE ... PARTITION is not supported 
" +
+s"for tables that are not partitioned: '$tableName'")
+}
+val locations =
+  if (isDatasourceTable || table.partitionColumnNames.isEmpty) {
+Seq(table.storage.locationUri)
   } else {
-val table = catalog.getTableMetadata(tableName)
-if (table.partitionColumnNames.nonEmpty) {
-  catalog.listPartitions(tableName).map(_.storage.locationUri)
-} else {
-  Seq(table.storage.locationUri)
-}
+catalog.listPartitions(tableName, 
partitionSpec).map(_.storage.locationUri)
   }
-  val hadoopConf = sparkSession.sessionState.newHadoopConf()
-  locations.foreach { location =>
-if (location.isDefined) {
-  val path = new Path(location.get)
-  try {
-val fs = path.getFileSystem(hadoopConf)
-fs.delete(path, true)
-fs.mkdirs(path)
-  } catch {
-case NonFatal(e) =>
-  throw new AnalysisException(
-s"Failed to truncate table '$tableName' when removing data of 
the path: $path " +
-  s"because of ${e.toString}")
-  }
+val hadoopConf = spark.sessionState.newHadoopConf()
+locations.foreach { location =>
+  if