diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
index 5eca1db9525ec745cb52f74a53149da303da0d26..d8319b9a97fcf43e6ec9e4840c45be350b6e9dd7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala
@@ -37,7 +37,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.plans.{Inner, JoinType}
 import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, SqlParser}
-import org.apache.spark.sql.execution.{EvaluatePython, FileRelation, LogicalRDD, QueryExecution, Queryable, SQLExecution}
+import org.apache.spark.sql.execution.{EvaluatePython, ExplainCommand, FileRelation, LogicalRDD, QueryExecution, Queryable, SQLExecution}
 import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, LogicalRelation}
 import org.apache.spark.sql.execution.datasources.json.JacksonGenerator
 import org.apache.spark.sql.sources.HadoopFsRelation
@@ -112,8 +112,8 @@ private[sql] object DataFrame {
  */
 @Experimental
 class DataFrame private[sql](
-    @transient val sqlContext: SQLContext,
-    @DeveloperApi @transient val queryExecution: QueryExecution)
+    @transient override val sqlContext: SQLContext,
+    @DeveloperApi @transient override val queryExecution: QueryExecution)
   extends Queryable with Serializable {
 
   // Note for Spark contributors: if adding or updating any action in `DataFrame`, please make sure
@@ -282,6 +282,35 @@ class DataFrame private[sql](
    */
   def schema: StructType = queryExecution.analyzed.schema
 
+  /**
+   * Prints the schema to the console in a nice tree format.
+   * @group basic
+   * @since 1.3.0
+   */
+  // scalastyle:off println
+  override def printSchema(): Unit = println(schema.treeString)
+  // scalastyle:on println
+
+  /**
+   * Prints the plans (logical and physical) to the console for debugging purposes.
+   * @group basic
+   * @since 1.3.0
+   */
+  override def explain(extended: Boolean): Unit = {
+    val explain = ExplainCommand(queryExecution.logical, extended = extended)
+    sqlContext.executePlan(explain).executedPlan.executeCollect().foreach {
+      // scalastyle:off println
+      r => println(r.getString(0))
+      // scalastyle:on println
+    }
+  }
+
+  /**
+   * Prints the physical plan to the console for debugging purposes.
+   * @since 1.3.0
+   */
+  override def explain(): Unit = explain(extended = false)
+
   /**
    * Returns all column names and their data types as an array.
    * @group basic
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index 97eb5b969280d7cfc181863159b794018ea03e87..da4600133290f0b4d96d41ff0381be3b049b8bc1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -61,8 +61,8 @@ import org.apache.spark.util.Utils
  */
 @Experimental
 class Dataset[T] private[sql](
-    @transient val sqlContext: SQLContext,
-    @transient val queryExecution: QueryExecution,
+    @transient override val sqlContext: SQLContext,
+    @transient override val queryExecution: QueryExecution,
     tEncoder: Encoder[T]) extends Queryable with Serializable {
 
   /**
@@ -85,7 +85,25 @@ class Dataset[T] private[sql](
    * Returns the schema of the encoded form of the objects in this [[Dataset]].
    * @since 1.6.0
    */
-  def schema: StructType = resolvedTEncoder.schema
+  override def schema: StructType = resolvedTEncoder.schema
+
+  /**
+   * Prints the schema of the underlying [[DataFrame]] to the console in a nice tree format.
+   * @since 1.6.0
+   */
+  override def printSchema(): Unit = toDF().printSchema()
+
+  /**
+   * Prints the plans (logical and physical) to the console for debugging purposes.
+   * @since 1.6.0
+   */
+  override def explain(extended: Boolean): Unit = toDF().explain(extended)
+
+  /**
+   * Prints the physical plan to the console for debugging purposes.
+   * @since 1.6.0
+   */
+  override def explain(): Unit = toDF().explain()
 
   /* ************* *
    *  Conversions  *
@@ -152,6 +170,59 @@ class Dataset[T] private[sql](
    */
   def count(): Long = toDF().count()
 
+  /**
+   * Displays the content of this [[Dataset]] in a tabular form. Strings more than 20 characters
+   * will be truncated, and all cells will be aligned right. For example:
+   * {{{
+   *   year  month AVG('Adj Close) MAX('Adj Close)
+   *   1980  12    0.503218        0.595103
+   *   1981  01    0.523289        0.570307
+   *   1982  02    0.436504        0.475256
+   *   1983  03    0.410516        0.442194
+   *   1984  04    0.450090        0.483521
+   * }}}
+   * @param numRows Number of rows to show
+   *
+   * @since 1.6.0
+   */
+  def show(numRows: Int): Unit = show(numRows, truncate = true)
+
+  /**
+   * Displays the top 20 rows of [[DataFrame]] in a tabular form. Strings more than 20 characters
+   * will be truncated, and all cells will be aligned right.
+   *
+   * @since 1.6.0
+   */
+  def show(): Unit = show(20)
+
+  /**
+   * Displays the top 20 rows of [[DataFrame]] in a tabular form.
+   *
+   * @param truncate Whether truncate long strings. If true, strings more than 20 characters will
+   *              be truncated and all cells will be aligned right
+   *
+   * @since 1.6.0
+   */
+  def show(truncate: Boolean): Unit = show(20, truncate)
+
+  /**
+   * Displays the [[DataFrame]] in a tabular form. For example:
+   * {{{
+   *   year  month AVG('Adj Close) MAX('Adj Close)
+   *   1980  12    0.503218        0.595103
+   *   1981  01    0.523289        0.570307
+   *   1982  02    0.436504        0.475256
+   *   1983  03    0.410516        0.442194
+   *   1984  04    0.450090        0.483521
+   * }}}
+   * @param numRows Number of rows to show
+   * @param truncate Whether truncate long strings. If true, strings more than 20 characters will
+   *              be truncated and all cells will be aligned right
+   *
+   * @since 1.6.0
+   */
+  def show(numRows: Int, truncate: Boolean): Unit = toDF().show(numRows, truncate)
+
   /**
     * Returns a new [[Dataset]] that has exactly `numPartitions` partitions.
     * @since 1.6.0
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Queryable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Queryable.scala
index 321e2c783537f665204c32faab9359283df52e4e..f2f5997d1b7c64f79cd4593111bd200d6831151d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Queryable.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Queryable.scala
@@ -17,11 +17,11 @@
 
 package org.apache.spark.sql.execution
 
+import scala.util.control.NonFatal
+
 import org.apache.spark.sql.SQLContext
 import org.apache.spark.sql.types.StructType
 
-import scala.util.control.NonFatal
-
 /** A trait that holds shared code between DataFrames and Datasets. */
 private[sql] trait Queryable {
   def schema: StructType
@@ -37,31 +37,9 @@ private[sql] trait Queryable {
     }
   }
 
-  /**
-   * Prints the schema to the console in a nice tree format.
-   * @group basic
-   * @since 1.3.0
-   */
-  // scalastyle:off println
-  def printSchema(): Unit = println(schema.treeString)
-  // scalastyle:on println
+  def printSchema(): Unit
 
-  /**
-   * Prints the plans (logical and physical) to the console for debugging purposes.
-   * @since 1.3.0
-   */
-  def explain(extended: Boolean): Unit = {
-    val explain = ExplainCommand(queryExecution.logical, extended = extended)
-    sqlContext.executePlan(explain).executedPlan.executeCollect().foreach {
-      // scalastyle:off println
-      r => println(r.getString(0))
-      // scalastyle:on println
-    }
-  }
+  def explain(extended: Boolean): Unit
 
-  /**
-   * Only prints the physical plan to the console for debugging purposes.
-   * @since 1.3.0
-   */
-  def explain(): Unit = explain(extended = false)
+  def explain(): Unit
 }