You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@linkis.apache.org by pe...@apache.org on 2023/05/26 03:22:29 UTC
[linkis] branch dev-1.4.0 updated: spark etl support kafka (#4563)
This is an automated email from the ASF dual-hosted git repository.
peacewong pushed a commit to branch dev-1.4.0
in repository https://gitbox.apache.org/repos/asf/linkis.git
The following commit(s) were added to refs/heads/dev-1.4.0 by this push:
new 42c52b2cc spark etl support kafka (#4563)
42c52b2cc is described below
commit 42c52b2cc6a7c3c611890885739989d51b92b6a5
Author: ChengJie1053 <18...@163.com>
AuthorDate: Fri May 26 11:22:21 2023 +0800
spark etl support kafka (#4563)
---
.../spark/datacalc/sink/KafkaSinkConfig.java | 70 ++++++++++++
.../spark/datacalc/source/KafkaSourceConfig.java | 55 ++++++++++
.../spark/datacalc/util/PluginUtil.java | 2 +
.../spark/datacalc/sink/KafkaSink.scala | 47 ++++++++
.../spark/datacalc/source/KafkaSource.scala | 42 ++++++++
.../spark/datacalc/TestKafkaCala.scala | 118 +++++++++++++++++++++
6 files changed, 334 insertions(+)
diff --git a/linkis-engineconn-plugins/spark/src/main/java/org/apache/linkis/engineplugin/spark/datacalc/sink/KafkaSinkConfig.java b/linkis-engineconn-plugins/spark/src/main/java/org/apache/linkis/engineplugin/spark/datacalc/sink/KafkaSinkConfig.java
new file mode 100644
index 000000000..ed3e94a27
--- /dev/null
+++ b/linkis-engineconn-plugins/spark/src/main/java/org/apache/linkis/engineplugin/spark/datacalc/sink/KafkaSinkConfig.java
@@ -0,0 +1,70 @@
+/*
+ * 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.linkis.engineplugin.spark.datacalc.sink;
+
+import org.apache.linkis.engineplugin.spark.datacalc.model.SinkConfig;
+
+import javax.validation.constraints.NotBlank;
+import javax.validation.constraints.Pattern;
+
+public class KafkaSinkConfig extends SinkConfig {
+
+ @NotBlank private String servers;
+
+ @NotBlank private String topic;
+
+ private String checkpointLocation = "./ck";
+
+ @NotBlank
+ @Pattern(
+ regexp = "^(batch|stream)$",
+ message = "Unknown mode: {saveMode}. Accepted modes are 'batch', 'stream'.")
+ private String mode = "stream";
+
+ public String getCheckpointLocation() {
+ return checkpointLocation;
+ }
+
+ public void setCheckpointLocation(String checkpointLocation) {
+ this.checkpointLocation = checkpointLocation;
+ }
+
+ public String getMode() {
+ return mode;
+ }
+
+ public void setMode(String mode) {
+ this.mode = mode;
+ }
+
+ public String getServers() {
+ return servers;
+ }
+
+ public void setServers(String servers) {
+ this.servers = servers;
+ }
+
+ public String getTopic() {
+ return topic;
+ }
+
+ public void setTopic(String topic) {
+ this.topic = topic;
+ }
+}
diff --git a/linkis-engineconn-plugins/spark/src/main/java/org/apache/linkis/engineplugin/spark/datacalc/source/KafkaSourceConfig.java b/linkis-engineconn-plugins/spark/src/main/java/org/apache/linkis/engineplugin/spark/datacalc/source/KafkaSourceConfig.java
new file mode 100644
index 000000000..489565e9c
--- /dev/null
+++ b/linkis-engineconn-plugins/spark/src/main/java/org/apache/linkis/engineplugin/spark/datacalc/source/KafkaSourceConfig.java
@@ -0,0 +1,55 @@
+/*
+ * 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.linkis.engineplugin.spark.datacalc.source;
+
+import org.apache.linkis.engineplugin.spark.datacalc.model.SourceConfig;
+
+import javax.validation.constraints.NotBlank;
+
+public class KafkaSourceConfig extends SourceConfig {
+
+ @NotBlank private String servers;
+
+ @NotBlank private String topic;
+
+ private String consumeMode = "earliest";
+
+ public String getServers() {
+ return servers;
+ }
+
+ public void setServers(String servers) {
+ this.servers = servers;
+ }
+
+ public String getTopic() {
+ return topic;
+ }
+
+ public void setTopic(String topic) {
+ this.topic = topic;
+ }
+
+ public String getConsumeMode() {
+ return consumeMode;
+ }
+
+ public void setConsumeMode(String consumeMode) {
+ this.consumeMode = consumeMode;
+ }
+}
diff --git a/linkis-engineconn-plugins/spark/src/main/java/org/apache/linkis/engineplugin/spark/datacalc/util/PluginUtil.java b/linkis-engineconn-plugins/spark/src/main/java/org/apache/linkis/engineplugin/spark/datacalc/util/PluginUtil.java
index 259117f91..28ae3a4c3 100644
--- a/linkis-engineconn-plugins/spark/src/main/java/org/apache/linkis/engineplugin/spark/datacalc/util/PluginUtil.java
+++ b/linkis-engineconn-plugins/spark/src/main/java/org/apache/linkis/engineplugin/spark/datacalc/util/PluginUtil.java
@@ -50,6 +50,7 @@ public class PluginUtil {
classMap.put("mongo", MongoSource.class);
classMap.put("elasticsearch", ElasticsearchSource.class);
classMap.put("solr", SolrSource.class);
+ classMap.put("kafka", KafkaSource.class);
return classMap;
}
@@ -71,6 +72,7 @@ public class PluginUtil {
classMap.put("mongo", MongoSink.class);
classMap.put("elasticsearch", ElasticsearchSink.class);
classMap.put("solr", SolrSink.class);
+ classMap.put("kafka", KafkaSink.class);
return classMap;
}
diff --git a/linkis-engineconn-plugins/spark/src/main/scala/org/apache/linkis/engineplugin/spark/datacalc/sink/KafkaSink.scala b/linkis-engineconn-plugins/spark/src/main/scala/org/apache/linkis/engineplugin/spark/datacalc/sink/KafkaSink.scala
new file mode 100644
index 000000000..7c495e1d5
--- /dev/null
+++ b/linkis-engineconn-plugins/spark/src/main/scala/org/apache/linkis/engineplugin/spark/datacalc/sink/KafkaSink.scala
@@ -0,0 +1,47 @@
+/*
+ * 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.linkis.engineplugin.spark.datacalc.sink
+
+import org.apache.linkis.common.utils.Logging
+import org.apache.linkis.engineplugin.spark.datacalc.api.DataCalcSink
+
+import org.apache.spark.sql.{Dataset, Row, SparkSession}
+
+import scala.collection.JavaConverters._
+
+class KafkaSink extends DataCalcSink[KafkaSinkConfig] with Logging {
+
+ def output(spark: SparkSession, ds: Dataset[Row]): Unit = {
+ var options = Map("kafka.bootstrap.servers" -> config.getServers, "topic" -> config.getTopic)
+
+ if (config.getOptions != null && !config.getOptions.isEmpty) {
+ options = config.getOptions.asScala.toMap ++ options
+ }
+ logger.info(s"Load data to kafka servers: ${config.getServers}, topic: ${config.getTopic}")
+
+ config.getMode match {
+ case "batch" =>
+ ds.selectExpr("to_json(struct(*)) AS value").write.format("kafka").options(options).save()
+ case "stream" =>
+ options = Map("checkpointLocation" -> config.getCheckpointLocation) ++ options
+ ds.writeStream.format("kafka").options(options).start().awaitTermination()
+ case _ =>
+ }
+ }
+
+}
diff --git a/linkis-engineconn-plugins/spark/src/main/scala/org/apache/linkis/engineplugin/spark/datacalc/source/KafkaSource.scala b/linkis-engineconn-plugins/spark/src/main/scala/org/apache/linkis/engineplugin/spark/datacalc/source/KafkaSource.scala
new file mode 100644
index 000000000..519b1bfdd
--- /dev/null
+++ b/linkis-engineconn-plugins/spark/src/main/scala/org/apache/linkis/engineplugin/spark/datacalc/source/KafkaSource.scala
@@ -0,0 +1,42 @@
+/*
+ * 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.linkis.engineplugin.spark.datacalc.source
+
+import org.apache.linkis.common.utils.Logging
+import org.apache.linkis.engineplugin.spark.datacalc.api.DataCalcSource
+
+import org.apache.spark.sql.{Dataset, Row, SparkSession}
+
+class KafkaSource extends DataCalcSource[KafkaSourceConfig] with Logging {
+
+ override def getData(spark: SparkSession): Dataset[Row] = {
+ val reader = spark.readStream.format("kafka")
+ if (config.getOptions != null && !config.getOptions.isEmpty) {
+ reader.options(config.getOptions)
+ }
+
+ logger.info(s"Load data from kafka servers: ${config.getServers}, topic: ${config.getTopic}")
+
+ reader
+ .option("kafka.bootstrap.servers", config.getServers)
+ .option("subscribe", config.getTopic)
+ .option("startingOffsets", config.getConsumeMode)
+ .load()
+ }
+
+}
diff --git a/linkis-engineconn-plugins/spark/src/test/scala/org/apache/linkis/engineplugin/spark/datacalc/TestKafkaCala.scala b/linkis-engineconn-plugins/spark/src/test/scala/org/apache/linkis/engineplugin/spark/datacalc/TestKafkaCala.scala
new file mode 100644
index 000000000..2a8620b4c
--- /dev/null
+++ b/linkis-engineconn-plugins/spark/src/test/scala/org/apache/linkis/engineplugin/spark/datacalc/TestKafkaCala.scala
@@ -0,0 +1,118 @@
+/*
+ * 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.linkis.engineplugin.spark.datacalc
+
+import org.apache.linkis.common.io.FsPath
+import org.apache.linkis.engineplugin.spark.datacalc.model.DataCalcGroupData
+
+import org.junit.jupiter.api.{Assertions, Test};
+
+class TestKafkaCala {
+
+ val filePath = this.getClass.getResource("/").getFile
+
+ @Test
+ def testKafkaWrite: Unit = {
+ // skip os: windows
+ if (!FsPath.WINDOWS) {
+ val data = DataCalcGroupData.getData(kafkaWriteConfigJson.replace("{filePath}", filePath))
+ Assertions.assertTrue(data != null)
+
+ val (sources, transforms, sinks) = DataCalcExecution.getPlugins(data)
+ Assertions.assertTrue(sources != null)
+ Assertions.assertTrue(transforms != null)
+ Assertions.assertTrue(sinks != null)
+ }
+ }
+
+ @Test
+ def testKafkaReader: Unit = {
+ // skip os: windows
+ if (!FsPath.WINDOWS) {
+ val data = DataCalcGroupData.getData(kafkaReaderConfigJson)
+ Assertions.assertTrue(data != null)
+
+ val (sources, transforms, sinks) = DataCalcExecution.getPlugins(data)
+ Assertions.assertTrue(sources != null)
+ Assertions.assertTrue(transforms != null)
+ Assertions.assertTrue(sinks != null)
+ }
+ }
+
+ val kafkaWriteConfigJson =
+ """
+ |{
+ | "sources": [
+ | {
+ | "name": "file",
+ | "type": "source",
+ | "config": {
+ | "resultTable": "T1654611700631",
+ | "path": "file://{filePath}/etltest.dolphin",
+ | "serializer": "csv",
+ | "options": {
+ | "header":"true",
+ | "delimiter":";"
+ | },
+ | "columnNames": ["name", "age"]
+ | }
+ | }
+ | ],
+ | "sinks": [
+ | {
+ | "name": "kafka",
+ | "config": {
+ | "sourceTable": "T1654611700631",
+ | "servers": "localhost:9092",
+ | "mode": "batch",
+ | "topic": "test121212"
+ | }
+ | }
+ | ]
+ |}
+ |""".stripMargin
+
+ val kafkaReaderConfigJson =
+ """
+ |{
+ | "sources": [
+ | {
+ | "name": "kafka",
+ | "type": "source",
+ | "config": {
+ | "resultTable": "T1654611700631",
+ | "servers": "localhost:9092",
+ | "topic": "test121212"
+ | }
+ | }
+ | ],
+ | "sinks": [
+ | {
+ | "name": "kafka",
+ | "config": {
+ | "sourceTable": "T1654611700631",
+ | "servers": "localhost:9092",
+ | "mode": "stream",
+ | "topic": "test55555"
+ | }
+ | }
+ | ]
+ |}
+ |""".stripMargin
+
+}
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@linkis.apache.org
For additional commands, e-mail: commits-help@linkis.apache.org