Repository: spark
Updated Branches:
  refs/heads/master ef43a5fe5 -> dcf407de6


[SPARK-15375][SQL][STREAMING] Add ConsoleSink to structure streaming

## What changes were proposed in this pull request?

Add ConsoleSink to structure streaming, user could use it to display dataframes 
on the console (useful for debugging and demostrating), similar to the 
functionality of `DStream#print`, to use it:

```
    val query = result.write
      .format("console")
      .trigger(ProcessingTime("2 seconds"))
      .startStream()
```

## How was this patch tested?

local verified.

Not sure it is suitable to add into structure streaming, please review and help 
to comment, thanks a lot.

Author: jerryshao <ss...@hortonworks.com>

Closes #13162 from jerryshao/SPARK-15375.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/dcf407de
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/dcf407de
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/dcf407de

Branch: refs/heads/master
Commit: dcf407de676ce99462a62f17cd98c9df56f057df
Parents: ef43a5f
Author: jerryshao <ss...@hortonworks.com>
Authored: Thu May 19 17:42:59 2016 -0700
Committer: Reynold Xin <r...@databricks.com>
Committed: Thu May 19 17:42:59 2016 -0700

----------------------------------------------------------------------
 ....apache.spark.sql.sources.DataSourceRegister |  1 +
 .../spark/sql/execution/streaming/console.scala | 60 ++++++++++++++++++++
 .../streaming/DataFrameReaderWriterSuite.scala  | 15 +++++
 3 files changed, 76 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/dcf407de/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
 
b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
index 226d59d..ef92557 100644
--- 
a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
+++ 
b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -3,3 +3,4 @@ org.apache.spark.sql.execution.datasources.jdbc.DefaultSource
 org.apache.spark.sql.execution.datasources.json.DefaultSource
 org.apache.spark.sql.execution.datasources.parquet.DefaultSource
 org.apache.spark.sql.execution.datasources.text.DefaultSource
+org.apache.spark.sql.execution.streaming.ConsoleSinkProvider

http://git-wip-us.apache.org/repos/asf/spark/blob/dcf407de/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
new file mode 100644
index 0000000..f11a3fb
--- /dev/null
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala
@@ -0,0 +1,60 @@
+/*
+ * 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.streaming
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{DataFrame, SQLContext}
+import org.apache.spark.sql.sources.{DataSourceRegister, StreamSinkProvider}
+
+class ConsoleSink(options: Map[String, String]) extends Sink with Logging {
+  // Number of rows to display, by default 20 rows
+  private val numRowsToShow = options.get("numRows").map(_.toInt).getOrElse(20)
+
+  // Truncate the displayed data if it is too long, by default it is true
+  private val isTruncated = 
options.get("truncate").map(_.toBoolean).getOrElse(true)
+
+  // Track the batch id
+  private var lastBatchId = -1L
+
+  override def addBatch(batchId: Long, data: DataFrame): Unit = synchronized {
+    val batchIdStr = if (batchId <= lastBatchId) {
+      s"Rerun batch: $batchId"
+    } else {
+      lastBatchId = batchId
+      s"Batch: $batchId"
+    }
+
+    // scalastyle:off println
+    println("-------------------------------------------")
+    println(batchIdStr)
+    println("-------------------------------------------")
+    // scalastyle:off println
+    data.show(numRowsToShow, isTruncated)
+  }
+}
+
+class ConsoleSinkProvider extends StreamSinkProvider with DataSourceRegister {
+  def createSink(
+      sqlContext: SQLContext,
+      parameters: Map[String, String],
+      partitionColumns: Seq[String]): Sink = {
+    new ConsoleSink(parameters)
+  }
+
+  def shortName(): String = "console"
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/dcf407de/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala
index cb53b2b..ff53505 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala
@@ -524,4 +524,19 @@ class DataFrameReaderWriterSuite extends StreamTest with 
SharedSQLContext with B
     val e = intercept[AnalysisException](w.csv("non_exist_path"))
     assert(e.getMessage == "csv() can only be called on non-continuous 
queries;")
   }
+
+  test("ConsoleSink can be correctly loaded") {
+    LastOptions.clear()
+    val df = spark.read
+      .format("org.apache.spark.sql.streaming.test")
+      .stream()
+
+    val cq = df.write
+      .format("console")
+      .option("checkpointLocation", newMetadataDir)
+      .trigger(ProcessingTime(2.seconds))
+      .startStream()
+
+    cq.awaitTermination(2000L)
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to