You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by gv...@apache.org on 2018/06/08 11:40:39 UTC
[14/50] [abbrv] carbondata git commit: [CARBONDATA-2227] Added
support to show partition details in describe formatted
[CARBONDATA-2227] Added support to show partition details in describe formatted
Added Detailed information in describe formatted command like partition location and partition values.
Example Usage: Descsribe formatted partition(partition_col_name=partition_value)
This closes #2033
Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/604902b9
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/604902b9
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/604902b9
Branch: refs/heads/spark-2.3
Commit: 604902b9a52ec613c1ec025b4dc33657b179895e
Parents: 2ebfab1
Author: kunal642 <ku...@gmail.com>
Authored: Mon Mar 5 20:33:06 2018 +0530
Committer: manishgupta88 <to...@gmail.com>
Committed: Tue May 22 11:37:07 2018 +0530
----------------------------------------------------------------------
.../describeTable/TestDescribeTable.scala | 20 +++++++++++++++
.../partition/TestDDLForPartitionTable.scala | 2 +-
.../table/CarbonDescribeFormattedCommand.scala | 26 +++++++++++++++++---
.../sql/execution/strategy/DDLStrategy.scala | 1 +
4 files changed, 44 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/carbondata/blob/604902b9/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/describeTable/TestDescribeTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/describeTable/TestDescribeTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/describeTable/TestDescribeTable.scala
index fe01598..1e333ee 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/describeTable/TestDescribeTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/describeTable/TestDescribeTable.scala
@@ -28,6 +28,7 @@ class TestDescribeTable extends QueryTest with BeforeAndAfterAll {
override def beforeAll: Unit = {
sql("DROP TABLE IF EXISTS Desc1")
sql("DROP TABLE IF EXISTS Desc2")
+ sql("drop table if exists a")
sql("CREATE TABLE Desc1(Dec1Col1 String, Dec1Col2 String, Dec1Col3 int, Dec1Col4 double) stored by 'carbondata'")
sql("DESC Desc1")
sql("DROP TABLE Desc1")
@@ -56,9 +57,28 @@ class TestDescribeTable extends QueryTest with BeforeAndAfterAll {
assert(sql("desc formatted desc1").count() == 20)
}
+ test("test describe formatted for partition table") {
+ sql("create table a(a string) partitioned by (b int) stored by 'carbondata'")
+ sql("insert into a values('a',1)")
+ sql("insert into a values('a',2)")
+ val desc = sql("describe formatted a").collect()
+ assert(desc(desc.indexWhere(_.get(0).toString.contains("#Partition")) + 2).get(0).toString.contains("b"))
+ val descPar = sql("describe formatted a partition(b=1)").collect
+ descPar.find(_.get(0).toString.contains("Partition Value:")) match {
+ case Some(row) => assert(row.get(1).toString.contains("1"))
+ case None => fail("Partition Value not found in describe formatted")
+ }
+ descPar.find(_.get(0).toString.contains("Location:")) match {
+ case Some(row) => assert(row.get(1).toString.contains("target/warehouse/a/b=1"))
+ case None => fail("Partition Location not found in describe formatted")
+ }
+ assert(descPar.exists(_.toString().contains("Partition Parameters:")))
+ }
+
override def afterAll: Unit = {
sql("DROP TABLE Desc1")
sql("DROP TABLE Desc2")
+ sql("drop table if exists a")
}
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/604902b9/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
index 2cbafa8..cafd465 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
@@ -145,7 +145,7 @@ class TestDDLForPartitionTable extends QueryTest with BeforeAndAfterAll {
sql(
"""create table des(a int, b string) partitioned by (c string) stored by 'carbondata'
|tblproperties ('partition_type'='list','list_info'='1,2')""".stripMargin)
- checkExistence(sql("describe formatted des"), true, "Partition Columns")
+ checkExistence(sql("describe formatted des"), true, "Partition Type")
sql("drop table if exists des")
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/604902b9/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
index ce03959..69bb91f 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
@@ -21,6 +21,7 @@ import scala.collection.JavaConverters._
import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.command.MetadataCommand
@@ -35,6 +36,7 @@ import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
private[sql] case class CarbonDescribeFormattedCommand(
child: SparkPlan,
override val output: Seq[Attribute],
+ partitionSpec: TablePartitionSpec,
tblIdentifier: TableIdentifier)
extends MetadataCommand {
@@ -148,12 +150,28 @@ private[sql] case class CarbonDescribeFormattedCommand(
.map(column => column).mkString(","), ""))
if (carbonTable.getPartitionInfo(carbonTable.getTableName) != null) {
results ++=
- Seq(("Partition Columns", carbonTable.getPartitionInfo(carbonTable.getTableName)
- .getColumnSchemaList.asScala.map(_.getColumnName).mkString(","), ""))
- results ++=
- Seq(("Partition Type", carbonTable.getPartitionInfo(carbonTable.getTableName)
+ Seq(("#Partition Information", "", ""),
+ ("#col_name", "data_type", "comment"))
+ results ++= carbonTable.getPartitionInfo(carbonTable.getTableName)
+ .getColumnSchemaList.asScala.map {
+ col => (col.getColumnName, col.getDataType.getName, "NULL")
+ }
+ results ++= Seq(("Partition Type", carbonTable.getPartitionInfo(carbonTable.getTableName)
.getPartitionType.toString, ""))
}
+ if (partitionSpec.nonEmpty) {
+ val partitions = sparkSession.sessionState.catalog.getPartition(tblIdentifier, partitionSpec)
+ results ++=
+ Seq(("", "", ""),
+ ("##Detailed Partition Information", "", ""),
+ ("Partition Value:", partitions.spec.values.mkString("[", ",", "]"), ""),
+ ("Database:", tblIdentifier.database.getOrElse(sparkSession.catalog.currentDatabase), ""),
+ ("Table:", tblIdentifier.table, ""))
+ if (partitions.storage.locationUri.isDefined) {
+ results ++= Seq(("Location:", partitions.storage.locationUri.get.toString, ""))
+ }
+ results ++= Seq(("Partition Parameters:", partitions.parameters.mkString(", "), ""))
+ }
results.map {
case (name, dataType, null) =>
Row(f"$name%-36s", f"$dataType%-80s", null)
http://git-wip-us.apache.org/repos/asf/carbondata/blob/604902b9/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
index ef4d05c..468121b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
@@ -179,6 +179,7 @@ class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
CarbonDescribeFormattedCommand(
resultPlan,
plan.output,
+ partitionSpec,
identifier)) :: Nil
} else {
Nil