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 9368435a63c359896cd4943b2acd2d7eb00de27c..691b476fff8d6405e6d431df843b59d59e4f100f 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
@@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.encoders.Encoder
 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, ExplainCommand, FileRelation, LogicalRDD, QueryExecution, 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
@@ -116,7 +116,8 @@ private[sql] object DataFrame {
 @Experimental
 class DataFrame private[sql](
     @transient val sqlContext: SQLContext,
-    @DeveloperApi @transient val queryExecution: QueryExecution) extends Serializable {
+    @DeveloperApi @transient val queryExecution: QueryExecution)
+  extends Queryable with Serializable {
 
   // Note for Spark contributors: if adding or updating any action in `DataFrame`, please make sure
   // you wrap it with `withNewExecutionId` if this actions doesn't call other action.
@@ -234,15 +235,6 @@ class DataFrame private[sql](
     sb.toString()
   }
 
-  override def toString: String = {
-    try {
-      schema.map(f => s"${f.name}: ${f.dataType.simpleString}").mkString("[", ", ", "]")
-    } catch {
-      case NonFatal(e) =>
-        s"Invalid tree; ${e.getMessage}:\n$queryExecution"
-    }
-  }
-
   /**
    * Returns the object itself.
    * @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 6d2968e2881f829ff82601b0fc107add71e7acb6..a7e5ab19bf846f963d8de8fa8164075c0db270fd 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
@@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.encoders._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.Inner
 import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.execution.QueryExecution
+import org.apache.spark.sql.execution.{Queryable, QueryExecution}
 import org.apache.spark.sql.types.StructType
 
 /**
@@ -62,7 +62,7 @@ import org.apache.spark.sql.types.StructType
 class Dataset[T] private[sql](
     @transient val sqlContext: SQLContext,
     @transient val queryExecution: QueryExecution,
-    unresolvedEncoder: Encoder[T]) extends Serializable {
+    unresolvedEncoder: Encoder[T]) extends Queryable with Serializable {
 
   /** The encoder for this [[Dataset]] that has been resolved to its output schema. */
   private[sql] implicit val encoder: ExpressionEncoder[T] = unresolvedEncoder match {
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
new file mode 100644
index 0000000000000000000000000000000000000000..9ca383896a09b77c17a264c6d75f1c898246c4fd
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Queryable.scala
@@ -0,0 +1,37 @@
+/*
+ * 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.execution
+
+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
+  def queryExecution: QueryExecution
+
+  override def toString: String = {
+    try {
+      schema.map(f => s"${f.name}: ${f.dataType.simpleString}").mkString("[", ", ", "]")
+    } catch {
+      case NonFatal(e) =>
+        s"Invalid tree; ${e.getMessage}:\n$queryExecution"
+    }
+  }
+}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
index aea5a700d0204114e15e0a25f6e2bd0fb115bdaa..621148528714f3ed490dc2124f6830eec353489f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
@@ -313,4 +313,9 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
     val joined = ds1.joinWith(ds2, $"a.value" === $"b.value")
     checkAnswer(joined, ("2", 2))
   }
+
+  test("toString") {
+    val ds = Seq((1, 2)).toDS()
+    assert(ds.toString == "[_1: int, _2: int]")
+  }
 }