ruanwenjun commented on code in PR #14833:
URL: 
https://github.com/apache/dolphinscheduler/pull/14833#discussion_r1310271296


##########
dolphinscheduler-master/src/main/resources/application.yaml:
##########
@@ -83,6 +83,9 @@ registry:
     block-until-connected: 600ms
     digest: ~
 
+listener:

Review Comment:
   This config will make confusion.



##########
dolphinscheduler-listener/dolphinscheduler-listener-plugin/dolphinscheduler-listener-kafka/src/main/java/org/apache/dolphinscheduler/listener/KafkaListener.java:
##########
@@ -0,0 +1,214 @@
+/*
+ * 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.dolphinscheduler.listener;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.listener.event.ServerDownListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskCreateListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskEndListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskFailListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskRemoveListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskStartListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskUpdateListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowCreateListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowEndListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowFailListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowRemoveListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowStartListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowUpdateListenerEvent;
+import org.apache.dolphinscheduler.listener.plugin.ListenerPlugin;
+import org.apache.dolphinscheduler.spi.params.base.PluginParams;
+import org.apache.dolphinscheduler.spi.params.base.Validate;
+import org.apache.dolphinscheduler.spi.params.input.InputParam;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.serialization.StringSerializer;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class KafkaListener implements ListenerPlugin {
+
+    private final Map<String, KafkaProducer<String, String>> kafkaProducers = 
new HashMap<>();
+
+    @Override
+    public String name() {
+        return "KafkaListener";
+    }
+
+    @Override
+    public List<PluginParams> params() {
+        List<PluginParams> paramsList = new ArrayList<>();
+        InputParam hostParam = InputParam.newBuilder("servers", 
"bootstrap.servers")
+                .setPlaceholder("please input servers")
+                .addValidate(Validate.newBuilder()
+                        .setRequired(true)
+                        .build())
+                .build();
+        InputParam topicParam = InputParam.newBuilder("topic", "topic")
+                .setPlaceholder("please input topic")
+                .addValidate(Validate.newBuilder()
+                        .setRequired(true)
+                        .build())
+                .build();
+        InputParam usernameParam = InputParam.newBuilder("username", 
"username")
+                .setPlaceholder("please input username")
+                .addValidate(Validate.newBuilder()
+                        .setRequired(false)
+                        .build())
+                .build();
+        InputParam passwordParam = InputParam.newBuilder("password", 
"password")
+                .setPlaceholder("please input password")
+                .addValidate(Validate.newBuilder()
+                        .setRequired(false)
+                        .build())
+                .build();
+        paramsList.add(hostParam);
+        paramsList.add(topicParam);
+        paramsList.add(usernameParam);
+        paramsList.add(passwordParam);
+        return paramsList;
+    }
+
+    @Override
+    public void onServerDown(ServerDownListenerEvent serverDownListenerEvent) {
+        sendEvent(serverDownListenerEvent.getListenerInstanceParams(), 
ServerDownListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(serverDownListenerEvent));
+    }
+
+    @Override
+    public void onWorkflowAdded(WorkflowCreateListenerEvent 
workflowCreateEvent) {
+        sendEvent(workflowCreateEvent.getListenerInstanceParams(), 
WorkflowCreateListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(workflowCreateEvent));
+    }
+
+    @Override
+    public void onWorkflowUpdate(WorkflowUpdateListenerEvent 
workflowUpdateEvent) {
+        sendEvent(workflowUpdateEvent.getListenerInstanceParams(), 
WorkflowUpdateListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(workflowUpdateEvent));
+    }
+
+    @Override
+    public void onWorkflowRemoved(WorkflowRemoveListenerEvent 
workflowRemovedEvent) {
+        sendEvent(workflowRemovedEvent.getListenerInstanceParams(), 
WorkflowRemoveListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(workflowRemovedEvent));
+    }
+
+    @Override
+    public void onWorkflowStart(WorkflowStartListenerEvent workflowStartEvent) 
{
+        sendEvent(workflowStartEvent.getListenerInstanceParams(), 
WorkflowStartListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(workflowStartEvent));
+
+    }
+
+    @Override
+    public void onWorkflowEnd(WorkflowEndListenerEvent workflowEndEvent) {
+        sendEvent(workflowEndEvent.getListenerInstanceParams(), 
WorkflowEndListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(workflowEndEvent));
+    }
+
+    @Override
+    public void onWorkflowFail(WorkflowFailListenerEvent workflowErrorEvent) {
+        sendEvent(workflowErrorEvent.getListenerInstanceParams(), 
WorkflowFailListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(workflowErrorEvent));
+    }
+
+    @Override
+    public void onTaskAdded(TaskCreateListenerEvent taskAddedEvent) {
+        sendEvent(taskAddedEvent.getListenerInstanceParams(), 
TaskCreateListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(taskAddedEvent));
+    }
+
+    @Override
+    public void onTaskUpdate(TaskUpdateListenerEvent taskUpdateEvent) {
+        sendEvent(taskUpdateEvent.getListenerInstanceParams(), 
TaskUpdateListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(taskUpdateEvent));
+    }
+
+    @Override
+    public void onTaskRemoved(TaskRemoveListenerEvent taskRemovedEvent) {
+        sendEvent(taskRemovedEvent.getListenerInstanceParams(), 
TaskRemoveListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(taskRemovedEvent));
+    }
+
+    @Override
+    public void onTaskStart(TaskStartListenerEvent taskStartEvent) {
+        sendEvent(taskStartEvent.getListenerInstanceParams(), 
TaskStartListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(taskStartEvent));
+    }
+
+    @Override
+    public void onTaskEnd(TaskEndListenerEvent taskEndEvent) {
+        sendEvent(taskEndEvent.getListenerInstanceParams(), 
TaskEndListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(taskEndEvent));
+    }
+
+    @Override
+    public void onTaskFail(TaskFailListenerEvent taskErrorEvent) {
+        sendEvent(taskErrorEvent.getListenerInstanceParams(), 
TaskFailListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(taskErrorEvent));
+    }
+
+    private void sendEvent(Map<String, String> listenerInstanceParams, String 
key, String value) {
+        String uniqueId = uniqueId(listenerInstanceParams);
+        if (!kafkaProducers.containsKey(uniqueId)) {
+            String kafkaBroker = listenerInstanceParams.get("servers");
+            String username = listenerInstanceParams.get("username");
+            String password = listenerInstanceParams.get("password");
+            Map<String, Object> configurations = new HashMap<>();
+            // TODO: when use username/password, throws exception: Unable to 
find LoginModule class:
+            // org.apache.kafka.common.security.plain.PlainLoginModule
+            configurations.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, 
kafkaBroker);
+            configurations.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, 
StringSerializer.class);
+            configurations.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, 
StringSerializer.class);
+            if (StringUtils.isNotEmpty(username) && 
StringUtils.isNotEmpty(password)) {
+                configurations.put("sasl.jaas.config", String.format(
+                        
"org.apache.kafka.common.security.plain.PlainLoginModule required username='%s' 
password='%s';",
+                        username, password));
+                configurations.put("security.protocol", "SASL_PLAINTEXT");
+                configurations.put("sasl.mechanism", "PLAIN");
+            }
+            KafkaProducer<String, String> producer = new 
KafkaProducer<>(configurations);
+            kafkaProducers.put(uniqueId, producer);
+
+        }
+        KafkaProducer<String, String> producer = kafkaProducers.get(uniqueId);
+        String topic = listenerInstanceParams.get("topic");
+        producer.send(new ProducerRecord<>(topic, key, value), 
(recordMetadata, e) -> {
+            if (e != null) {
+                throw new RuntimeException(e);
+            }
+        });
+    }
+
+    private String uniqueId(Map<String, String> listenerInstanceParams) {
+        String kafkaBroker = listenerInstanceParams.get("servers");
+        String topic = listenerInstanceParams.get("topic");
+        String username = listenerInstanceParams.getOrDefault("username", 
"foo");
+        String password = listenerInstanceParams.getOrDefault("password", 
"foo");

Review Comment:
   We don't need to add default username and password.



##########
dolphinscheduler-listener/dolphinscheduler-listener-service/src/main/java/org/apache/dolphinscheduler/listener/service/jdbc/mapper/ListenerEventMapper.java:
##########
@@ -0,0 +1,46 @@
+/*
+ *
+ *  * Licensed to 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. Apache Software Foundation (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.dolphinscheduler.listener.service.jdbc.mapper;
+
+import org.apache.dolphinscheduler.listener.service.jdbc.JdbcListenerEvent;
+
+import org.apache.ibatis.annotations.Insert;
+import org.apache.ibatis.annotations.Param;
+
+import java.util.List;
+
+import com.baomidou.mybatisplus.core.mapper.BaseMapper;
+
+public interface ListenerEventMapper extends BaseMapper<JdbcListenerEvent> {
+
+    @Insert({"<script>",
+            "        insert into t_ds_listener_event ( content, post_status, 
event_type, log, plugin_instance_id, create_time, update_time)",
+            "        values",
+            "        <foreach collection='jdbcListenerEvents' 
item='jdbcListenerEvent' separator=','>",
+            "            
(#{jdbcListenerEvent.content},#{jdbcListenerEvent.postStatus}," +
+                    "            
#{jdbcListenerEvent.eventType},#{jdbcListenerEvent.log},#{jdbcListenerEvent.pluginInstanceId},
 #{jdbcListenerEvent.createTime}, #{jdbcListenerEvent.updateTime})"
+                    +
+                    "        </foreach>",
+            "</script>"})

Review Comment:
   Please don't use `@Annotation` and xml at the same time.



##########
dolphinscheduler-listener/dolphinscheduler-listener-plugin/dolphinscheduler-listener-kafka/src/main/java/org/apache/dolphinscheduler/listener/KafkaListener.java:
##########
@@ -0,0 +1,214 @@
+/*
+ * 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.dolphinscheduler.listener;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.listener.event.ServerDownListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskCreateListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskEndListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskFailListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskRemoveListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskStartListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskUpdateListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowCreateListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowEndListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowFailListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowRemoveListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowStartListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowUpdateListenerEvent;
+import org.apache.dolphinscheduler.listener.plugin.ListenerPlugin;
+import org.apache.dolphinscheduler.spi.params.base.PluginParams;
+import org.apache.dolphinscheduler.spi.params.base.Validate;
+import org.apache.dolphinscheduler.spi.params.input.InputParam;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.serialization.StringSerializer;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class KafkaListener implements ListenerPlugin {
+
+    private final Map<String, KafkaProducer<String, String>> kafkaProducers = 
new HashMap<>();
+
+    @Override
+    public String name() {
+        return "KafkaListener";
+    }
+
+    @Override
+    public List<PluginParams> params() {
+        List<PluginParams> paramsList = new ArrayList<>();
+        InputParam hostParam = InputParam.newBuilder("servers", 
"bootstrap.servers")
+                .setPlaceholder("please input servers")
+                .addValidate(Validate.newBuilder()
+                        .setRequired(true)
+                        .build())
+                .build();
+        InputParam topicParam = InputParam.newBuilder("topic", "topic")
+                .setPlaceholder("please input topic")
+                .addValidate(Validate.newBuilder()
+                        .setRequired(true)
+                        .build())
+                .build();
+        InputParam usernameParam = InputParam.newBuilder("username", 
"username")
+                .setPlaceholder("please input username")
+                .addValidate(Validate.newBuilder()
+                        .setRequired(false)
+                        .build())
+                .build();
+        InputParam passwordParam = InputParam.newBuilder("password", 
"password")
+                .setPlaceholder("please input password")
+                .addValidate(Validate.newBuilder()
+                        .setRequired(false)
+                        .build())
+                .build();
+        paramsList.add(hostParam);
+        paramsList.add(topicParam);
+        paramsList.add(usernameParam);
+        paramsList.add(passwordParam);
+        return paramsList;
+    }
+
+    @Override
+    public void onServerDown(ServerDownListenerEvent serverDownListenerEvent) {
+        sendEvent(serverDownListenerEvent.getListenerInstanceParams(), 
ServerDownListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(serverDownListenerEvent));
+    }
+
+    @Override
+    public void onWorkflowAdded(WorkflowCreateListenerEvent 
workflowCreateEvent) {
+        sendEvent(workflowCreateEvent.getListenerInstanceParams(), 
WorkflowCreateListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(workflowCreateEvent));
+    }
+
+    @Override
+    public void onWorkflowUpdate(WorkflowUpdateListenerEvent 
workflowUpdateEvent) {
+        sendEvent(workflowUpdateEvent.getListenerInstanceParams(), 
WorkflowUpdateListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(workflowUpdateEvent));
+    }
+
+    @Override
+    public void onWorkflowRemoved(WorkflowRemoveListenerEvent 
workflowRemovedEvent) {
+        sendEvent(workflowRemovedEvent.getListenerInstanceParams(), 
WorkflowRemoveListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(workflowRemovedEvent));
+    }
+
+    @Override
+    public void onWorkflowStart(WorkflowStartListenerEvent workflowStartEvent) 
{
+        sendEvent(workflowStartEvent.getListenerInstanceParams(), 
WorkflowStartListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(workflowStartEvent));
+
+    }
+
+    @Override
+    public void onWorkflowEnd(WorkflowEndListenerEvent workflowEndEvent) {
+        sendEvent(workflowEndEvent.getListenerInstanceParams(), 
WorkflowEndListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(workflowEndEvent));
+    }
+
+    @Override
+    public void onWorkflowFail(WorkflowFailListenerEvent workflowErrorEvent) {
+        sendEvent(workflowErrorEvent.getListenerInstanceParams(), 
WorkflowFailListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(workflowErrorEvent));
+    }
+
+    @Override
+    public void onTaskAdded(TaskCreateListenerEvent taskAddedEvent) {
+        sendEvent(taskAddedEvent.getListenerInstanceParams(), 
TaskCreateListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(taskAddedEvent));
+    }
+
+    @Override
+    public void onTaskUpdate(TaskUpdateListenerEvent taskUpdateEvent) {
+        sendEvent(taskUpdateEvent.getListenerInstanceParams(), 
TaskUpdateListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(taskUpdateEvent));
+    }
+
+    @Override
+    public void onTaskRemoved(TaskRemoveListenerEvent taskRemovedEvent) {
+        sendEvent(taskRemovedEvent.getListenerInstanceParams(), 
TaskRemoveListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(taskRemovedEvent));
+    }
+
+    @Override
+    public void onTaskStart(TaskStartListenerEvent taskStartEvent) {
+        sendEvent(taskStartEvent.getListenerInstanceParams(), 
TaskStartListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(taskStartEvent));
+    }
+
+    @Override
+    public void onTaskEnd(TaskEndListenerEvent taskEndEvent) {
+        sendEvent(taskEndEvent.getListenerInstanceParams(), 
TaskEndListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(taskEndEvent));
+    }
+
+    @Override
+    public void onTaskFail(TaskFailListenerEvent taskErrorEvent) {
+        sendEvent(taskErrorEvent.getListenerInstanceParams(), 
TaskFailListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(taskErrorEvent));
+    }
+
+    private void sendEvent(Map<String, String> listenerInstanceParams, String 
key, String value) {
+        String uniqueId = uniqueId(listenerInstanceParams);
+        if (!kafkaProducers.containsKey(uniqueId)) {
+            String kafkaBroker = listenerInstanceParams.get("servers");
+            String username = listenerInstanceParams.get("username");
+            String password = listenerInstanceParams.get("password");
+            Map<String, Object> configurations = new HashMap<>();
+            // TODO: when use username/password, throws exception: Unable to 
find LoginModule class:
+            // org.apache.kafka.common.security.plain.PlainLoginModule
+            configurations.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, 
kafkaBroker);
+            configurations.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, 
StringSerializer.class);
+            configurations.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, 
StringSerializer.class);
+            if (StringUtils.isNotEmpty(username) && 
StringUtils.isNotEmpty(password)) {
+                configurations.put("sasl.jaas.config", String.format(
+                        
"org.apache.kafka.common.security.plain.PlainLoginModule required username='%s' 
password='%s';",
+                        username, password));
+                configurations.put("security.protocol", "SASL_PLAINTEXT");
+                configurations.put("sasl.mechanism", "PLAIN");

Review Comment:
   Why don't expose this config,



##########
dolphinscheduler-listener/dolphinscheduler-listener-plugin/dolphinscheduler-listener-kafka/src/main/java/org/apache/dolphinscheduler/listener/KafkaListener.java:
##########
@@ -0,0 +1,214 @@
+/*
+ * 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.dolphinscheduler.listener;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.listener.event.ServerDownListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskCreateListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskEndListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskFailListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskRemoveListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskStartListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskUpdateListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowCreateListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowEndListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowFailListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowRemoveListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowStartListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowUpdateListenerEvent;
+import org.apache.dolphinscheduler.listener.plugin.ListenerPlugin;
+import org.apache.dolphinscheduler.spi.params.base.PluginParams;
+import org.apache.dolphinscheduler.spi.params.base.Validate;
+import org.apache.dolphinscheduler.spi.params.input.InputParam;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.serialization.StringSerializer;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class KafkaListener implements ListenerPlugin {
+
+    private final Map<String, KafkaProducer<String, String>> kafkaProducers = 
new HashMap<>();
+
+    @Override
+    public String name() {
+        return "KafkaListener";
+    }
+
+    @Override
+    public List<PluginParams> params() {
+        List<PluginParams> paramsList = new ArrayList<>();
+        InputParam hostParam = InputParam.newBuilder("servers", 
"bootstrap.servers")
+                .setPlaceholder("please input servers")
+                .addValidate(Validate.newBuilder()
+                        .setRequired(true)
+                        .build())
+                .build();
+        InputParam topicParam = InputParam.newBuilder("topic", "topic")
+                .setPlaceholder("please input topic")
+                .addValidate(Validate.newBuilder()
+                        .setRequired(true)
+                        .build())
+                .build();
+        InputParam usernameParam = InputParam.newBuilder("username", 
"username")
+                .setPlaceholder("please input username")
+                .addValidate(Validate.newBuilder()
+                        .setRequired(false)
+                        .build())
+                .build();
+        InputParam passwordParam = InputParam.newBuilder("password", 
"password")
+                .setPlaceholder("please input password")
+                .addValidate(Validate.newBuilder()
+                        .setRequired(false)
+                        .build())
+                .build();
+        paramsList.add(hostParam);
+        paramsList.add(topicParam);
+        paramsList.add(usernameParam);
+        paramsList.add(passwordParam);
+        return paramsList;
+    }
+
+    @Override
+    public void onServerDown(ServerDownListenerEvent serverDownListenerEvent) {
+        sendEvent(serverDownListenerEvent.getListenerInstanceParams(), 
ServerDownListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(serverDownListenerEvent));
+    }
+
+    @Override
+    public void onWorkflowAdded(WorkflowCreateListenerEvent 
workflowCreateEvent) {
+        sendEvent(workflowCreateEvent.getListenerInstanceParams(), 
WorkflowCreateListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(workflowCreateEvent));
+    }
+
+    @Override
+    public void onWorkflowUpdate(WorkflowUpdateListenerEvent 
workflowUpdateEvent) {
+        sendEvent(workflowUpdateEvent.getListenerInstanceParams(), 
WorkflowUpdateListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(workflowUpdateEvent));
+    }
+
+    @Override
+    public void onWorkflowRemoved(WorkflowRemoveListenerEvent 
workflowRemovedEvent) {
+        sendEvent(workflowRemovedEvent.getListenerInstanceParams(), 
WorkflowRemoveListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(workflowRemovedEvent));
+    }
+
+    @Override
+    public void onWorkflowStart(WorkflowStartListenerEvent workflowStartEvent) 
{
+        sendEvent(workflowStartEvent.getListenerInstanceParams(), 
WorkflowStartListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(workflowStartEvent));
+
+    }
+
+    @Override
+    public void onWorkflowEnd(WorkflowEndListenerEvent workflowEndEvent) {
+        sendEvent(workflowEndEvent.getListenerInstanceParams(), 
WorkflowEndListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(workflowEndEvent));
+    }
+
+    @Override
+    public void onWorkflowFail(WorkflowFailListenerEvent workflowErrorEvent) {
+        sendEvent(workflowErrorEvent.getListenerInstanceParams(), 
WorkflowFailListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(workflowErrorEvent));
+    }
+
+    @Override
+    public void onTaskAdded(TaskCreateListenerEvent taskAddedEvent) {
+        sendEvent(taskAddedEvent.getListenerInstanceParams(), 
TaskCreateListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(taskAddedEvent));
+    }
+
+    @Override
+    public void onTaskUpdate(TaskUpdateListenerEvent taskUpdateEvent) {
+        sendEvent(taskUpdateEvent.getListenerInstanceParams(), 
TaskUpdateListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(taskUpdateEvent));
+    }
+
+    @Override
+    public void onTaskRemoved(TaskRemoveListenerEvent taskRemovedEvent) {
+        sendEvent(taskRemovedEvent.getListenerInstanceParams(), 
TaskRemoveListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(taskRemovedEvent));
+    }
+
+    @Override
+    public void onTaskStart(TaskStartListenerEvent taskStartEvent) {
+        sendEvent(taskStartEvent.getListenerInstanceParams(), 
TaskStartListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(taskStartEvent));
+    }
+
+    @Override
+    public void onTaskEnd(TaskEndListenerEvent taskEndEvent) {
+        sendEvent(taskEndEvent.getListenerInstanceParams(), 
TaskEndListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(taskEndEvent));
+    }
+
+    @Override
+    public void onTaskFail(TaskFailListenerEvent taskErrorEvent) {
+        sendEvent(taskErrorEvent.getListenerInstanceParams(), 
TaskFailListenerEvent.class.getSimpleName(),
+                JSONUtils.toJsonString(taskErrorEvent));
+    }
+
+    private void sendEvent(Map<String, String> listenerInstanceParams, String 
key, String value) {
+        String uniqueId = uniqueId(listenerInstanceParams);
+        if (!kafkaProducers.containsKey(uniqueId)) {
+            String kafkaBroker = listenerInstanceParams.get("servers");
+            String username = listenerInstanceParams.get("username");
+            String password = listenerInstanceParams.get("password");
+            Map<String, Object> configurations = new HashMap<>();
+            // TODO: when use username/password, throws exception: Unable to 
find LoginModule class:
+            // org.apache.kafka.common.security.plain.PlainLoginModule
+            configurations.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, 
kafkaBroker);
+            configurations.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, 
StringSerializer.class);
+            configurations.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, 
StringSerializer.class);
+            if (StringUtils.isNotEmpty(username) && 
StringUtils.isNotEmpty(password)) {
+                configurations.put("sasl.jaas.config", String.format(
+                        
"org.apache.kafka.common.security.plain.PlainLoginModule required username='%s' 
password='%s';",
+                        username, password));
+                configurations.put("security.protocol", "SASL_PLAINTEXT");
+                configurations.put("sasl.mechanism", "PLAIN");
+            }
+            KafkaProducer<String, String> producer = new 
KafkaProducer<>(configurations);
+            kafkaProducers.put(uniqueId, producer);
+
+        }
+        KafkaProducer<String, String> producer = kafkaProducers.get(uniqueId);
+        String topic = listenerInstanceParams.get("topic");
+        producer.send(new ProducerRecord<>(topic, key, value), 
(recordMetadata, e) -> {

Review Comment:
   You said you need to `ensures that messages are neither lost nor unordered` 
but this is asyn send, how can you ensure the message doesn't lost.



##########
dolphinscheduler-listener/dolphinscheduler-listener-service/src/main/java/org/apache/dolphinscheduler/listener/service/ListenerEventPublishService.java:
##########
@@ -0,0 +1,273 @@
+/*
+ *
+ *  * Licensed to 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. Apache Software Foundation (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.dolphinscheduler.listener.service;
+
+import org.apache.dolphinscheduler.dao.entity.ProcessDefinition;
+import org.apache.dolphinscheduler.dao.entity.ProcessInstance;
+import org.apache.dolphinscheduler.dao.entity.ProcessTaskRelationLog;
+import org.apache.dolphinscheduler.dao.entity.Project;
+import org.apache.dolphinscheduler.dao.entity.ProjectUser;
+import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
+import org.apache.dolphinscheduler.dao.entity.TaskDefinitionLog;
+import org.apache.dolphinscheduler.dao.entity.TaskInstance;
+import org.apache.dolphinscheduler.listener.enums.ListenerEventType;
+import org.apache.dolphinscheduler.listener.event.ListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskCreateListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskEndListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskFailListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskRemoveListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskStartListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskUpdateListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowCreateListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowEndListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowFailListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowRemoveListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowStartListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowUpdateListenerEvent;
+
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import javax.annotation.PostConstruct;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+@Component
+@Slf4j
+public class ListenerEventPublishService {
+
+    private final BlockingQueue<ListenerEvent> listenerEventQueue = new 
LinkedBlockingQueue<>();
+
+    @Autowired
+    private ListenerEventProducer producer;
+
+    /**
+     * create a daemon thread to process the listener event queue
+     */
+    @PostConstruct
+    private void init() {
+        Thread thread = new Thread(this::doPublish);
+        thread.setDaemon(true);
+        thread.setName("Listener-Event-Produce-Thread");
+        thread.start();
+    }
+
+    public void publish(ListenerEvent listenerEvent) {
+        if (!listenerEventQueue.offer(listenerEvent)) {
+            log.error("Publish listener event failed, message:{}", 
listenerEvent);
+        }
+    }

Review Comment:
   If the downstream producer crash, this will cause master OOM, this is not 
reasonable, DolphinScheduler is use to schedule and trigger workflow, why 
`event` handle will affect its available?



##########
dolphinscheduler-listener/dolphinscheduler-listener-plugin/dolphinscheduler-listener-logger/src/main/java/org/apache/dolphinscheduler/listener/LoggerListener.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * 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.dolphinscheduler.listener;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.listener.event.ServerDownListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskCreateListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskEndListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskFailListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskRemoveListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskStartListenerEvent;
+import org.apache.dolphinscheduler.listener.event.TaskUpdateListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowCreateListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowEndListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowFailListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowRemoveListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowStartListenerEvent;
+import org.apache.dolphinscheduler.listener.event.WorkflowUpdateListenerEvent;
+import org.apache.dolphinscheduler.listener.plugin.ListenerPlugin;
+import org.apache.dolphinscheduler.spi.params.base.PluginParams;
+import org.apache.dolphinscheduler.spi.params.base.Validate;
+import org.apache.dolphinscheduler.spi.params.input.InputParam;
+
+import java.io.BufferedWriter;
+import java.io.FileWriter;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class LoggerListener implements ListenerPlugin {
+
+    @Override
+    public String name() {
+        return "LoggerListener";
+    }
+
+    @Override
+    public List<PluginParams> params() {
+        List<PluginParams> paramsList = new ArrayList<>();
+        InputParam param1 = InputParam.newBuilder("logFile", "log_file")
+                .setPlaceholder("please input log file")
+                .addValidate(Validate.newBuilder()
+                        .setRequired(true)
+                        .build())
+                .build();
+        paramsList.add(param1);
+        return paramsList;
+    }
+
+    @Override
+    public void onServerDown(ServerDownListenerEvent serverDownListenerEvent) {
+        printLogIntoFile(serverDownListenerEvent.getListenerInstanceParams(),
+                JSONUtils.toJsonString(serverDownListenerEvent));
+    }
+
+    @Override
+    public void onWorkflowAdded(WorkflowCreateListenerEvent 
workflowCreateEvent) {
+        printLogIntoFile(workflowCreateEvent.getListenerInstanceParams(), 
JSONUtils.toJsonString(workflowCreateEvent));
+    }
+
+    @Override
+    public void onWorkflowUpdate(WorkflowUpdateListenerEvent 
workflowUpdateEvent) {
+        printLogIntoFile(workflowUpdateEvent.getListenerInstanceParams(), 
JSONUtils.toJsonString(workflowUpdateEvent));
+    }
+
+    @Override
+    public void onWorkflowRemoved(WorkflowRemoveListenerEvent 
workflowRemovedEvent) {
+        printLogIntoFile(workflowRemovedEvent.getListenerInstanceParams(),
+                JSONUtils.toJsonString(workflowRemovedEvent));
+    }
+
+    @Override
+    public void onWorkflowStart(WorkflowStartListenerEvent workflowStartEvent) 
{
+        printLogIntoFile(workflowStartEvent.getListenerInstanceParams(), 
JSONUtils.toJsonString(workflowStartEvent));
+    }
+
+    @Override
+    public void onWorkflowEnd(WorkflowEndListenerEvent workflowEndEvent) {
+        printLogIntoFile(workflowEndEvent.getListenerInstanceParams(), 
JSONUtils.toJsonString(workflowEndEvent));
+    }
+
+    @Override
+    public void onWorkflowFail(WorkflowFailListenerEvent workflowErrorEvent) {
+        printLogIntoFile(workflowErrorEvent.getListenerInstanceParams(), 
JSONUtils.toJsonString(workflowErrorEvent));
+    }
+
+    @Override
+    public void onTaskAdded(TaskCreateListenerEvent taskAddedEvent) {
+        printLogIntoFile(taskAddedEvent.getListenerInstanceParams(), 
JSONUtils.toJsonString(taskAddedEvent));
+    }
+
+    @Override
+    public void onTaskUpdate(TaskUpdateListenerEvent taskUpdateEvent) {
+        printLogIntoFile(taskUpdateEvent.getListenerInstanceParams(), 
JSONUtils.toJsonString(taskUpdateEvent));
+    }
+
+    @Override
+    public void onTaskRemoved(TaskRemoveListenerEvent taskRemovedEvent) {
+        printLogIntoFile(taskRemovedEvent.getListenerInstanceParams(), 
JSONUtils.toJsonString(taskRemovedEvent));
+    }
+
+    @Override
+    public void onTaskStart(TaskStartListenerEvent taskStartEvent) {
+        printLogIntoFile(taskStartEvent.getListenerInstanceParams(), 
JSONUtils.toJsonString(taskStartEvent));
+    }
+
+    @Override
+    public void onTaskEnd(TaskEndListenerEvent taskEndEvent) {
+        printLogIntoFile(taskEndEvent.getListenerInstanceParams(), 
JSONUtils.toJsonString(taskEndEvent));
+    }
+
+    @Override
+    public void onTaskFail(TaskFailListenerEvent taskErrorEvent) {
+        printLogIntoFile(taskErrorEvent.getListenerInstanceParams(), 
JSONUtils.toJsonString(taskErrorEvent));
+    }
+
+    private void printLogIntoFile(Map<String, String> listenerInstanceParams, 
String content) {
+        String logFile = listenerInstanceParams.get("logFile");
+        try (BufferedWriter writer = new BufferedWriter(new 
FileWriter(logFile, true))) {
+            writer.write(content);
+            writer.newLine();
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }

Review Comment:
   Have you test the performance? AFAIK, this is very slow.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to