aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorjerryshao <sshao@hortonworks.com>2016-05-19 17:42:59 -0700
committerReynold Xin <rxin@databricks.com>2016-05-19 17:42:59 -0700
commitdcf407de676ce99462a62f17cd98c9df56f057df (patch)
treefee1a0777f8c77f61ff6899f9d3b391d30678e71
parentef43a5fe51614eecce2d144cc13b33004a47533a (diff)
downloadspark-dcf407de676ce99462a62f17cd98c9df56f057df.tar.gz
spark-dcf407de676ce99462a62f17cd98c9df56f057df.tar.bz2
spark-dcf407de676ce99462a62f17cd98c9df56f057df.zip
[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 <sshao@hortonworks.com> Closes #13162 from jerryshao/SPARK-15375.
-rw-r--r--sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister1
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala60
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/streaming/DataFrameReaderWriterSuite.scala15
3 files changed, 76 insertions, 0 deletions
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 226d59d0ea..ef9255794b 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
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 0000000000..f11a3fb969
--- /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"
+}
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 cb53b2b1aa..ff53505549 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)
+ }
}