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
Next Next commit
1. Replace trait HttpDataFormat by verifying that output dataFrame mu…
…st hava a single string column. 2. Add HttpStreamSinkSuite to test
  • Loading branch information
zhangxinyu1 committed Sep 26, 2016
commit d046e85fb6d0d68259abdf123032c304926d159f
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 @@ -18,45 +18,65 @@
package org.apache.spark.sql.execution.streaming

import java.io.{BufferedReader, InputStreamReader, PrintWriter}
import java.net.{UnknownHostException, URL, URLConnection}
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.util.Utils
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 {
val dataFormat: HttpDataFormat = {
val className = options.getOrElse("format.class",
"org.apache.spark.sql.execution.streaming.HttpDataToStringDefault")
createObject[HttpDataFormat](className)
}
verifySchema(data.schema)
data.collect().foreach(dataSeq => {
post(dataFormat.format(dataSeq.toSeq))
post(dataSeq.get(0).toString)
})
}

private def createObject[T<:AnyRef](className: String, args: AnyRef*): T = {
val klass = Utils.classForName(className).asInstanceOf[Class[T]]
val constructor = klass.getConstructor(args.map(_.getClass): _*)
constructor.newInstance(args: _*)
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(param: String): Unit = {
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(param)
writer.print(data)
writer.flush()
} catch {
case cause: Throwable => {
logError("Post http request error: ", cause)
}
case cause: Throwable => logError("Post http request error: ", cause)
} finally {
writer.close()
}
Expand All @@ -67,39 +87,11 @@ class HttpSink(options: Map[String, String]) extends Sink with Logging {
while (it.hasNext()) {
lines += it.next()
}
logTrace("Http request post result: " + lines)
logTrace(s"Http request post result: ${lines}.")
} catch {
case cause: Throwable => {
logError("Read http result error: ", cause)
}
case cause: Throwable => logError("Read http result error: ", cause)
} finally {
reader.close()
}
}
}

trait HttpDataFormat{
def format(data: Seq[Any]): String
}

class HttpDataToStringDefault extends HttpDataFormat {
def format(data: Seq[Any]) : String = {
return data.mkString(", ")
}
}

class HttpStreamSink extends StreamSinkProvider with DataSourceRegister{
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)
}

def shortName(): String = "http"
}

Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
package org.apache.spark.sql.execution.streaming

import java.util

import org.apache.spark.sql.Row
import org.apache.spark.sql.types.{StructField, StructType, StringType}
import spark.{Route, Spark, Request, Response}
import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema
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()
}
}