diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala
index 481a5a4f212b2888853696f2d06dce597dd6a473..a01809c1fc5e2820b4b441c2da90f9bf8e29f4f8 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala
@@ -50,7 +50,7 @@ case class SetCommand(key: Option[String], value: Option[String]) extends Comman
  * Returned by a parser when the users only wants to see what query plan would be executed, without
  * actually performing the execution.
  */
-case class ExplainCommand(plan: LogicalPlan) extends Command {
+case class ExplainCommand(plan: LogicalPlan, extended: Boolean = false) extends Command {
   override def output =
     Seq(AttributeReference("plan", StringType, nullable = false)())
 }
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index af9f7c62a1d25623342c00bc7324b278537b03c3..8a9f4deb6a19ef88357ab70fb09d8a14ebc82ce8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -408,10 +408,18 @@ class SQLContext(@transient val sparkContext: SparkContext)
     protected def stringOrError[A](f: => A): String =
       try f.toString catch { case e: Throwable => e.toString }
 
-    def simpleString: String = stringOrError(executedPlan)
+    def simpleString: String = 
+      s"""== Physical Plan ==
+         |${stringOrError(executedPlan)}
+      """
 
     override def toString: String =
-      s"""== Logical Plan ==
+      // TODO previously will output RDD details by run (${stringOrError(toRdd.toDebugString)})
+      // however, the `toRdd` will cause the real execution, which is not what we want.
+      // We need to think about how to avoid the side effect.
+      s"""== Parsed Logical Plan ==
+         |${stringOrError(logical)}
+         |== Analyzed Logical Plan ==
          |${stringOrError(analyzed)}
          |== Optimized Logical Plan ==
          |${stringOrError(optimizedPlan)}
@@ -419,7 +427,6 @@ class SQLContext(@transient val sparkContext: SparkContext)
          |${stringOrError(executedPlan)}
          |Code Generation: ${executedPlan.codegenEnabled}
          |== RDD ==
-         |${stringOrError(toRdd.toDebugString)}
       """.stripMargin.trim
   }
 
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index 517b77804ae2ca1dc52689561eb38c9efa05d0ed..8dacb84c8a17e0754986b686ab4b15322df09380 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -301,8 +301,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
     def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
       case logical.SetCommand(key, value) =>
         Seq(execution.SetCommand(key, value, plan.output)(context))
-      case logical.ExplainCommand(logicalPlan) =>
-        Seq(execution.ExplainCommand(logicalPlan, plan.output)(context))
+      case logical.ExplainCommand(logicalPlan, extended) =>
+        Seq(execution.ExplainCommand(logicalPlan, plan.output, extended)(context))
       case logical.CacheCommand(tableName, cache) =>
         Seq(execution.CacheCommand(tableName, cache)(context))
       case _ => Nil
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
index 38f37564f178807e3b5eb64dda0750d189b93008..031b695169cea67d31de492129fc7466ffa2ce62 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
@@ -108,15 +108,19 @@ case class SetCommand(
  */
 @DeveloperApi
 case class ExplainCommand(
-    logicalPlan: LogicalPlan, output: Seq[Attribute])(
+    logicalPlan: LogicalPlan, output: Seq[Attribute], extended: Boolean)(
     @transient context: SQLContext)
   extends LeafNode with Command {
 
   // Run through the optimizer to generate the physical plan.
   override protected[sql] lazy val sideEffectResult: Seq[String] = try {
-    "Physical execution plan:" +: context.executePlan(logicalPlan).executedPlan.toString.split("\n")
+    // TODO in Hive, the "extended" ExplainCommand prints the AST as well, and detailed properties.
+    val queryExecution = context.executePlan(logicalPlan)
+    val outputString = if (extended) queryExecution.toString else queryExecution.simpleString
+
+    outputString.split("\n")
   } catch { case cause: TreeNodeException[_] =>
-    "Error occurred during query planning: " +: cause.getMessage.split("\n")
+    ("Error occurred during query planning: \n" + cause.getMessage).split("\n")
   }
 
   def execute(): RDD[Row] = {
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 29baefe714c64b38ee7edc88a7ebdc20e3cfe539..d9b2bc7348ad2fd5fb5269cc3da87a8e1e9dfe44 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
@@ -424,7 +424,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
       logical match {
         case _: NativeCommand => "<Native command: executed by Hive>"
         case _: SetCommand => "<SET command: executed by Hive, and noted by SQLContext>"
-        case _ => executedPlan.toString
+        case _ => super.simpleString
       }
   }
 }
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 1d9ba1b24a7a4e73a75962b37cd5f084bbc670df..5da6e8df03aeec463f28b97a6b45a51c02265e4c 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
@@ -409,10 +409,9 @@ private[hive] object HiveQl {
       ExplainCommand(NoRelation)
     case Token("TOK_EXPLAIN", explainArgs) =>
       // Ignore FORMATTED if present.
-      val Some(query) :: _ :: _ :: Nil =
+      val Some(query) :: _ :: extended :: Nil =
         getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs)
-      // TODO: support EXTENDED?
-      ExplainCommand(nodeToPlan(query))
+      ExplainCommand(nodeToPlan(query), extended != None)
 
     case Token("TOK_DESCTABLE", describeArgs) =>
       // Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..4ed58f4be116783ed72a12694ad8a2e7323d1029
--- /dev/null
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive.execution
+
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.hive.test.TestHive
+import org.apache.spark.sql.hive.test.TestHive._
+import org.apache.spark.sql.Row
+
+/**
+ * A set of tests that validates support for Hive Explain command.
+ */
+class HiveExplainSuite extends QueryTest {
+  private def check(sqlCmd: String, exists: Boolean, keywords: String*) {
+    val outputs = sql(sqlCmd).collect().map(_.getString(0)).mkString
+    for (key <- keywords) {
+      if (exists) {
+        assert(outputs.contains(key), s"Failed for $sqlCmd ($key doens't exist in result)")
+      } else {
+        assert(!outputs.contains(key), s"Failed for $sqlCmd ($key existed in the result)")
+      }
+    }
+  }
+
+  test("explain extended command") {
+    check(" explain   select * from src where key=123 ", true,
+          "== Physical Plan ==")
+    check(" explain   select * from src where key=123 ", false,
+          "== Parsed Logical Plan ==",
+          "== Analyzed Logical Plan ==", 
+          "== Optimized Logical Plan ==")
+    check(" explain   extended select * from src where key=123 ", true,
+          "== Parsed Logical Plan ==", 
+          "== Analyzed Logical Plan ==", 
+          "== Optimized Logical Plan ==", 
+          "== Physical Plan ==", 
+          "Code Generation", "== RDD ==")
+  }
+}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
index 26e4ec6e6dcce177e0abea887f021c54e5c4bc9a..6d925e56e6838c31db3ae1be6bd2be632de931e3 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
@@ -327,7 +327,7 @@ class HiveQuerySuite extends HiveComparisonTest {
 
   def isExplanation(result: SchemaRDD) = {
     val explanation = result.select('plan).collect().map { case Row(plan: String) => plan }
-    explanation.size > 1 && explanation.head.startsWith("Physical execution plan")
+    explanation.exists(_ == "== Physical Plan ==")
   }
 
   test("SPARK-1704: Explain commands as a SchemaRDD") {