You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@seatunnel.apache.org by GitBox <gi...@apache.org> on 2022/04/14 01:23:01 UTC

[GitHub] [incubator-seatunnel] ruanwenjun commented on a diff in pull request #1695: [Feature][Connectors] add source plugin: seatunnel-connector-spark-webhook

ruanwenjun commented on code in PR #1695:
URL: https://github.com/apache/incubator-seatunnel/pull/1695#discussion_r850007727


##########
seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-webhook/src/main/scala/org/apache/seatunnel/spark/webhook/source/HttpData.scala:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.seatunnel.spark.webhook.source
+
+import java.sql.Timestamp

Review Comment:
   It might be better to use `java.util.Date`, since we don't use jdbc api.



##########
docs/en/connector/source/Webhook.md:
##########
@@ -0,0 +1,42 @@
+# Webhook
+
+## Description
+
+Provide http interface to push data.
+
+:::tip
+
+Engine Supported and plugin name
+
+* [x] Spark: Webhook
+* [ ] Flink
+
+:::
+
+## Options
+
+| name | type   | required | default value |
+| ---- | ------ | -------- | ------------- |
+| port | int    | no       | 9999          |
+| path | string | no       | /             |
+
+### port[int]
+
+Port for push requests, default 9999.
+
+### path[string]
+
+Push request path, default "/".
+
+### common options [string]
+
+Source plugin common parameters, please refer to [Source Plugin](common-options.mdx) for details.
+
+## Example
+
+```bash
+Webhook {
+       result_table_name= "request_body"
+   }

Review Comment:
   Please format.
   ```suggestion
   Webhook {
       result_table_name= "request_body"
   }
   ```



##########
docs/en/connector/source/Webhook.md:
##########
@@ -0,0 +1,42 @@
+# Webhook
+
+## Description
+
+Provide http interface to push data.

Review Comment:
   It's better to announce this plugin can only handle post request.



##########
seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-webhook/src/main/scala/org/apache/seatunnel/spark/webhook/source/Webhook.scala:
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.seatunnel.spark.webhook.source
+
+import org.apache.seatunnel.spark.SparkEnvironment
+import org.apache.seatunnel.spark.stream.SparkStreamingSource
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{Dataset, Row, SparkSession, SQLContext}
+import org.apache.spark.streaming.dstream.DStream
+
+class Webhook extends SparkStreamingSource[String] {
+
+  override def start(env: SparkEnvironment, handler: Dataset[Row] => Unit): Unit = {
+    var spark = env.getSparkSession
+    // Create HTTP Server and start streaming
+    implicit val sqlContext: SQLContext = spark.sqlContext
+
+    var port = if (config.hasPath("port")) config.getInt("port") else 9999
+    var baseUrl = if (config.hasPath("path")) config.getString("path") else "/"
+
+    val query = new JettyServerStream(port, baseUrl)
+      .toDF
+      .writeStream
+      .foreachBatch((batch, batchId) => {
+        handler(batch)
+      })
+      .start()
+
+    query.awaitTermination()
+  }
+
+  override def rdd2dataset(sparkSession: SparkSession, rdd: RDD[String]): Dataset[Row] = { null }
+
+  override def getData(env: SparkEnvironment): DStream[String] = { null }

Review Comment:
   You need to implement this method, otherwise the data received by http datasource will not be processed by downstream plugin



##########
seatunnel-connectors/seatunnel-connectors-spark/seatunnel-connector-spark-webhook/src/main/scala/org/apache/seatunnel/spark/webhook/source/JettyServerStream.scala:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.seatunnel.spark.webhook.source
+
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.{DataFrame, Encoder, Encoders, SQLContext}
+import org.spark_project.jetty.server.Server
+import org.spark_project.jetty.servlet.{ServletContextHandler, ServletHolder}
+
+class JettyServerStream(port: Int = 9999, baseUrl: String = "/") {

Review Comment:
   Is there needed to provide close method to close the `Server`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@seatunnel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org