Skip to content

[SPARK-17631] [SQL] Add HttpStreamSink for structured streaming. #15197

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
Show file tree
Hide file tree
Changes from all commits
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
6 changes: 6 additions & 0 deletions sql/core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -126,6 +126,12 @@
<groupId>org.apache.xbean</groupId>
<artifactId>xbean-asm5-shaded</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.sparkjava</groupId>
<artifactId>spark-core</artifactId>
<version>2.5</version>
<scope>test</scope>
</dependency>
</dependencies>
<build>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -5,3 +5,4 @@ org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
org.apache.spark.sql.execution.datasources.text.TextFileFormat
org.apache.spark.sql.execution.streaming.ConsoleSinkProvider
org.apache.spark.sql.execution.streaming.TextSocketSourceProvider
org.apache.spark.sql.execution.streaming.HttpStreamSink
Original file line number Diff line number Diff line change
@@ -0,0 +1,97 @@
/*
* 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 java.io.{BufferedReader, InputStreamReader, PrintWriter}
import java.net.{URL, URLConnection}

import org.apache.spark.internal.Logging
import org.apache.spark.sql._
import org.apache.spark.sql.sources.{DataSourceRegister, StreamSinkProvider}
import org.apache.spark.sql.streaming.OutputMode
import org.apache.spark.sql.types.{StringType, StructType}


class HttpStreamSink extends StreamSinkProvider with DataSourceRegister{
override def createSink(
sqlContext: SQLContext,
parameters: Map[String, String],
partitionColumns: Seq[String],
outputMode: OutputMode): Sink = {
if (!parameters.contains("url")) {
throw new AnalysisException("Http url should be set: .option(\"url\", \"...\").")
}
new HttpSink(parameters)
}

override def shortName(): String = "http"
}

/**
* A sink that outputs streaming query results through sending http post request. Each [[Row]]
* in batch will be post to a http url.
* Each [[Row]] in batch must only have one single column, and the column type should be
* [[StringType]].
*/
class HttpSink(options: Map[String, String]) extends Sink with Logging {
override def addBatch(batchId: Long, data: DataFrame): Unit = synchronized {
verifySchema(data.schema)
data.collect().foreach(dataSeq => {
post(dataSeq.get(0).toString)
})
}
private def verifySchema(schema: StructType): Unit = {
if (schema.size != 1) {
throw new AnalysisException(
s"Http data sink supports only a single column, and you have ${schema.size} columns.")
}
val tpe = schema(0).dataType
if (tpe != StringType) {
throw new AnalysisException(
s"Http data sink supports only a string column, but you have ${tpe.simpleString}.")
}
}
private def post(data: String): Unit = {
val url: URL = new URL(options.get("url").get)
val connection: URLConnection = url.openConnection
connection.setDoInput(true)
connection.setDoOutput(true)
val writer = new PrintWriter(connection.getOutputStream)
try {
writer.print(data)
writer.flush()
} catch {
case cause: Throwable => logError("Post http request error: ", cause)
} finally {
writer.close()
}
val reader = new BufferedReader(new InputStreamReader(connection.getInputStream))
try {
val it = reader.lines().iterator()
var lines: String = ""
while (it.hasNext()) {
lines += it.next()
}
logTrace(s"Http request post result: ${lines}.")
} catch {
case cause: Throwable => logError("Read http result error: ", cause)
} finally {
reader.close()
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
package org.apache.spark.sql.execution.streaming

import spark.{Response, Request, Route, Spark}

import org.apache.spark.sql.Row
import org.apache.spark.sql.streaming.StreamTest
import org.apache.spark.sql.test.SharedSQLContext
import org.scalatest.BeforeAndAfter

class HttpStreamSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAfter{
import testImplicits._
after {
sqlContext.streams.active.foreach(_.stop())
}
test("http sink"){
var output: String = ""
Spark.port(3775)
Spark.get("/welcome/:vistor", new Route{
override def handle(req: Request, resp: Response) : Object = {
val name: String = req.params(":vistor")
output = name
return s"welcome $name"
}
})
val input = MemoryStream[String]
val query = input.toDF().writeStream
.outputMode("complete")
.format("http")
.option("url", "http://localhost:3775/welcome")
.start()
input.addData("Jerry")
CheckAnswer(Row(output))
query.awaitTermination()
}
}