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