Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Address
  • Loading branch information
zsxwing committed Feb 15, 2017
commit 20eae3d06e521570d1e36af1b738d1cba8c0e5e7
Original file line number Diff line number Diff line change
Expand Up @@ -86,18 +86,18 @@ case class ExecutedCommandExec(cmd: RunnableCommand) extends SparkPlan {
* }}}
*
* @param logicalPlan plan to explain
* @param output output schema
* @param extended whether to do extended explain or not
* @param codegen whether to output generated code from whole-stage codegen or not
*/
case class ExplainCommand(
logicalPlan: LogicalPlan,
override val output: Seq[Attribute] =
Seq(AttributeReference("plan", StringType, nullable = true)()),
extended: Boolean = false,
codegen: Boolean = false)
extends RunnableCommand {

override val output: Seq[Attribute] =
Seq(AttributeReference("plan", StringType, nullable = true)())

// Run through the optimizer to generate the physical plan.
override def run(sparkSession: SparkSession): Seq[Row] = try {
val queryExecution =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -291,6 +291,8 @@ class StreamSuite extends StreamTest {
.map(_.getString(0))
.mkString("\n")
assert(explainString.contains("StateStoreRestore"))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would also check that this doesn't have a LocalTableScan but has a StreamingRelation

assert(explainString.contains("StreamingRelation"))
assert(!explainString.contains("LocalTableScan"))

// Test StreamingQuery.display
val q = df.writeStream.queryName("memory_explain").outputMode("complete").format("memory")
Expand Down