aloyszhang commented on code in PR #9805:
URL: https://github.com/apache/inlong/pull/9805#discussion_r1522633837


##########
inlong-agent/agent-plugins/src/main/java/org/apache/inlong/agent/plugin/sources/PulsarSource.java:
##########
@@ -0,0 +1,312 @@
+/*
+ * 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.inlong.agent.plugin.sources;
+
+import org.apache.inlong.agent.common.AgentThreadFactory;
+import org.apache.inlong.agent.conf.InstanceProfile;
+import org.apache.inlong.agent.conf.TaskProfile;
+import org.apache.inlong.agent.constant.CycleUnitType;
+import org.apache.inlong.agent.core.task.MemoryManager;
+import org.apache.inlong.agent.except.FileException;
+import org.apache.inlong.agent.message.DefaultMessage;
+import org.apache.inlong.agent.metrics.audit.AuditUtils;
+import org.apache.inlong.agent.plugin.Message;
+import org.apache.inlong.agent.plugin.file.Reader;
+import org.apache.inlong.agent.plugin.sources.file.AbstractSource;
+import org.apache.inlong.agent.utils.AgentUtils;
+
+import lombok.AllArgsConstructor;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.ObjectUtils;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.Schema;
+import org.apache.pulsar.client.api.SubscriptionInitialPosition;
+import org.apache.pulsar.client.api.SubscriptionType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static 
org.apache.inlong.agent.constant.CommonConstants.DEFAULT_PROXY_PACKAGE_MAX_SIZE;
+import static org.apache.inlong.agent.constant.CommonConstants.PROXY_KEY_DATA;
+import static 
org.apache.inlong.agent.constant.CommonConstants.PROXY_KEY_STREAM_ID;
+import static 
org.apache.inlong.agent.constant.CommonConstants.PROXY_PACKAGE_MAX_SIZE;
+import static 
org.apache.inlong.agent.constant.CommonConstants.PROXY_SEND_PARTITION_KEY;
+import static 
org.apache.inlong.agent.constant.FetcherConstants.AGENT_GLOBAL_READER_QUEUE_PERMIT;
+import static 
org.apache.inlong.agent.constant.FetcherConstants.AGENT_GLOBAL_READER_SOURCE_PERMIT;
+import static org.apache.inlong.agent.constant.TaskConstants.OFFSET;
+import static org.apache.inlong.agent.constant.TaskConstants.TASK_CYCLE_UNIT;
+import static 
org.apache.inlong.agent.constant.TaskConstants.TASK_PULSAR_RESET_TIME;
+import static 
org.apache.inlong.agent.constant.TaskConstants.TASK_PULSAR_SCAN_STARTUP_MODE;
+import static 
org.apache.inlong.agent.constant.TaskConstants.TASK_PULSAR_SERVICE_URL;
+
+public class PulsarSource extends AbstractSource {
+
+    @Data
+    @AllArgsConstructor
+    @NoArgsConstructor
+    private class SourceData {
+
+        private byte[] data;
+        private Long offset;
+    }
+
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(PulsarSource.class);
+    private static final ThreadPoolExecutor EXECUTOR_SERVICE = new 
ThreadPoolExecutor(
+            0, Integer.MAX_VALUE,
+            1L, TimeUnit.SECONDS,
+            new SynchronousQueue<>(),
+            new AgentThreadFactory("pulsar-source"));
+    private BlockingQueue<SourceData> queue;
+    public InstanceProfile profile;
+    private int maxPackSize;
+    private String taskId;
+    private String instanceId;
+    private String topic;
+    private String serviceUrl;
+    private String scanStartupMode;
+    private PulsarClient pulsarClient;
+    private Long timestamp;
+    Map<Integer, Long> partitionOffsets = new HashMap<>();
+    private volatile boolean running = false;
+    private volatile boolean runnable = true;
+    private volatile AtomicLong emptyCount = new AtomicLong(0);
+
+    private final Integer CACHE_QUEUE_SIZE = 100000;
+    private final Integer READ_WAIT_TIMEOUT_MS = 10;
+    private final Integer EMPTY_CHECK_COUNT_AT_LEAST = 5 * 60;
+    private final Integer BATCH_TOTAL_LEN = 1024 * 1024;
+
+    private final Integer CORE_THREAD_PRINT_INTERVAL_MS = 1000;
+    private boolean isRealTime = false;
+    private boolean isRestoreFromDB = false;
+
+    public PulsarSource() {
+    }
+
+    @Override
+    public void init(InstanceProfile profile) {
+        try {
+            LOGGER.info("PulsarSource init: {}", profile.toJsonStr());
+            this.profile = profile;
+            super.init(profile);
+            String cycleUnit = profile.get(TASK_CYCLE_UNIT);
+            if (cycleUnit.compareToIgnoreCase(CycleUnitType.REAL_TIME) == 0) {
+                isRealTime = true;
+                cycleUnit = CycleUnitType.HOUR;
+            }
+            queue = new LinkedBlockingQueue<>(CACHE_QUEUE_SIZE);
+            maxPackSize = profile.getInt(PROXY_PACKAGE_MAX_SIZE, 
DEFAULT_PROXY_PACKAGE_MAX_SIZE);
+            taskId = profile.getTaskId();
+            instanceId = profile.getInstanceId();
+            topic = profile.getInstanceId();
+            serviceUrl = profile.get(TASK_PULSAR_SERVICE_URL);
+            scanStartupMode = profile.get(TASK_PULSAR_SCAN_STARTUP_MODE);
+            timestamp = profile.getLong(TASK_PULSAR_RESET_TIME, 0);
+            pulsarClient = 
PulsarClient.builder().serviceUrl(serviceUrl).build();
+
+            EXECUTOR_SERVICE.execute(run());
+        } catch (Exception ex) {
+            stopRunning();
+            throw new FileException("error init stream for " + topic, ex);
+        }
+    }
+
+    private Runnable run() {
+        return () -> {
+            AgentThreadFactory.nameThread("pulsar-source-" + taskId + "-" + 
instanceId);
+            running = true;
+            try {
+                try (Consumer<byte[]> consumer = 
pulsarClient.newConsumer(Schema.BYTES)
+                        .topic(topic)
+                        .subscriptionName("inlong-agent-" + taskId)
+                        
.subscriptionInitialPosition(SubscriptionInitialPosition.valueOf(scanStartupMode))
+                        .subscriptionType(SubscriptionType.Exclusive)
+                        .subscribe()) {
+
+                    if (!isRestoreFromDB && timestamp != 0L) {
+                        consumer.seek(timestamp);
+                        LOGGER.info("Reset consume from {}", timestamp);
+                    } else {
+                        LOGGER.info("Skip to reset consume");
+                    }
+
+                    doRun(consumer);
+                }
+            } catch (Throwable e) {
+                LOGGER.error("do run error maybe pulsar client is configured 
incorrectly: ", e);
+            }
+            running = false;
+        };
+    }
+
+    private void doRun(Consumer<byte[]> consumer) throws PulsarClientException 
{
+        long lastPrintTime = 0;
+        while (isRunnable()) {
+            boolean suc = waitForPermit(AGENT_GLOBAL_READER_SOURCE_PERMIT, 
BATCH_TOTAL_LEN);
+            if (!suc) {
+                break;
+            }
+            org.apache.pulsar.client.api.Message<byte[]> message = 
consumer.receive(0, TimeUnit.MILLISECONDS);

Review Comment:
   If the timeout parameter is 0 and the topic has no messages backlog, 
`message` will be returned as null, then loop the next round immediately, this 
may cause high CPU load.



-- 
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