diff --git a/automq-log-uploader/README.md b/automq-log-uploader/README.md new file mode 100644 index 0000000000..3de41a344d --- /dev/null +++ b/automq-log-uploader/README.md @@ -0,0 +1,83 @@ +# AutoMQ Log Uploader Module + +This module provides asynchronous S3 log upload capability based on Log4j 1.x. Other submodules only need to depend on this module and configure it simply to synchronize logs to object storage. Core components: + +- `com.automq.log.uploader.S3RollingFileAppender`: Extends `RollingFileAppender`, pushes log events to the uploader while writing to local files. +- `com.automq.log.uploader.LogUploader`: Asynchronously buffers, compresses, and uploads logs; supports configuration switches and periodic cleanup. +- `com.automq.log.uploader.S3LogConfig`/`S3LogConfigProvider`: Abstracts the configuration required for uploading. The default implementation `PropertiesS3LogConfigProvider` reads from `automq-log.properties`. + +## Quick Integration + +1. Add dependency in your module's `build.gradle`: + ```groovy + implementation project(':automq-log-uploader') + ``` +2. Create `automq-log.properties` in the resources directory (or customize `S3LogConfigProvider`): + ```properties + log.s3.enable=true + log.s3.bucket=0@s3://your-log-bucket?region=us-east-1 + log.s3.cluster.id=my-cluster + log.s3.node.id=1 + log.s3.selector.type=kafka + log.s3.selector.kafka.bootstrap.servers=PLAINTEXT://kafka:9092 + log.s3.selector.kafka.group.id=automq-log-uploader-my-cluster + ``` +3. Reference the Appender in `log4j.properties`: + ```properties + log4j.appender.s3_uploader=com.automq.log.uploader.S3RollingFileAppender + log4j.appender.s3_uploader.File=logs/server.log + log4j.appender.s3_uploader.MaxFileSize=100MB + log4j.appender.s3_uploader.MaxBackupIndex=10 + log4j.appender.s3_uploader.layout=org.apache.log4j.PatternLayout + log4j.appender.s3_uploader.layout.ConversionPattern=[%d] %p %m (%c)%n + ``` + If you need to customize the configuration provider, you can set: + ```properties + log4j.appender.s3_uploader.configProviderClass=com.example.CustomS3LogConfigProvider + ``` + +## Key Configuration Description + +| Configuration Item | Description | +| ------ | ---- | +| `log.s3.enable` | Whether to enable S3 upload function. +| `log.s3.bucket` | It is recommended to use AutoMQ Bucket URI (e.g. `0@s3://bucket?region=us-east-1&pathStyle=true`). If using a shorthand bucket name, additional fields such as `log.s3.region` need to be provided. +| `log.s3.cluster.id` / `log.s3.node.id` | Used to construct the object storage path `automq/logs/{cluster}/{node}/{hour}/{uuid}`. +| `log.s3.selector.type` | Leader election strategy (`static`, `nodeid`, `file`, `kafka`, or custom). +| `log.s3.primary.node` | Used with `static` strategy to indicate whether the current node is the primary node. +| `log.s3.selector.kafka.*` | Additional configuration required for Kafka leader election, such as `bootstrap.servers`, `group.id`, etc. +| `log.s3.active.controller` | **Deprecated**, please use `log.s3.selector.type=static` + `log.s3.primary.node=true`. + +The upload schedule can be overridden by environment variables: + +- `AUTOMQ_OBSERVABILITY_UPLOAD_INTERVAL`: Maximum upload interval (milliseconds). +- `AUTOMQ_OBSERVABILITY_CLEANUP_INTERVAL`: Retention period (milliseconds), old objects earlier than this time will be cleaned up. + +### Leader Election Strategies + +To avoid multiple nodes executing S3 cleanup tasks simultaneously, the log uploader has a built-in leader election mechanism consistent with the OpenTelemetry module: + +1. **static**: Specify which node is the leader using `log.s3.primary.node=true|false`. +2. **nodeid**: Becomes the leader node when `log.s3.node.id` equals `primaryNodeId`, which can be set in the URL or properties with `log.s3.selector.primary.node.id`. +3. **file**: Uses a shared file for preemptive leader election, configure `log.s3.selector.file.leaderFile=/shared/leader`, `log.s3.selector.file.leaderTimeoutMs=60000`. +4. **kafka**: Default strategy. All nodes join the same consumer group of a single-partition topic, the node holding the partition becomes the leader. Necessary configuration: + ```properties + log.s3.selector.type=kafka + log.s3.selector.kafka.bootstrap.servers=PLAINTEXT://kafka:9092 + log.s3.selector.kafka.topic=__automq_log_uploader_leader_cluster1 + log.s3.selector.kafka.group.id=automq-log-uploader-cluster1 + ``` + Advanced parameters such as security (SASL/SSL), timeout, etc. can be provided through `log.s3.selector.kafka.*`. +5. **custom**: Implement `com.automq.log.uploader.selector.LogUploaderNodeSelectorProvider` and register it through SPI to introduce a custom leader election strategy. + +## Extension + +If the application already has its own dependency injection/configuration method, you can implement `S3LogConfigProvider` and call it at startup: + +```java +import com.automq.log.uploader.S3RollingFileAppender; + +S3RollingFileAppender.setConfigProvider(new CustomConfigProvider()); +``` + +All `S3RollingFileAppender` instances will share this provider. diff --git a/automq-log-uploader/build.gradle b/automq-log-uploader/build.gradle new file mode 100644 index 0000000000..72dd261d03 --- /dev/null +++ b/automq-log-uploader/build.gradle @@ -0,0 +1,19 @@ +plugins { + id 'java-library' +} + +repositories { + mavenCentral() +} + +dependencies { + api project(':s3stream') + + implementation project(':clients') + implementation libs.reload4j + implementation libs.slf4jApi + implementation libs.slf4jBridge + implementation libs.nettyBuffer + implementation libs.guava + implementation libs.commonLang +} diff --git a/automq-log-uploader/src/main/java/com/automq/log/uploader/DefaultS3LogConfig.java b/automq-log-uploader/src/main/java/com/automq/log/uploader/DefaultS3LogConfig.java new file mode 100644 index 0000000000..9458212f06 --- /dev/null +++ b/automq-log-uploader/src/main/java/com/automq/log/uploader/DefaultS3LogConfig.java @@ -0,0 +1,183 @@ +/* + * Copyright 2025, AutoMQ HK Limited. + * + * 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 com.automq.log.uploader; + +import com.automq.log.uploader.selector.LogUploaderNodeSelector; +import com.automq.log.uploader.selector.LogUploaderNodeSelectorFactory; +import com.automq.stream.s3.operator.BucketURI; +import com.automq.stream.s3.operator.ObjectStorage; +import com.automq.stream.s3.operator.ObjectStorageFactory; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.util.HashMap; +import java.util.Locale; +import java.util.Map; +import java.util.Properties; + +import static com.automq.log.uploader.LogConfigConstants.*; + +public class DefaultS3LogConfig implements S3LogConfig { + private static final Logger LOGGER = LoggerFactory.getLogger(DefaultS3LogConfig.class); + + private final Properties props; + private ObjectStorage objectStorage; + private LogUploaderNodeSelector nodeSelector; + + public DefaultS3LogConfig() { + this(null); + } + + public DefaultS3LogConfig(Properties overrideProps) { + this.props = new Properties(); + if (overrideProps != null) { + this.props.putAll(overrideProps); + } + if (overrideProps == null) { + try (InputStream input = getClass().getClassLoader().getResourceAsStream(LOG_PROPERTIES_FILE)) { + if (input != null) { + props.load(input); + LOGGER.info("Loaded log configuration from {}", LOG_PROPERTIES_FILE); + } else { + LOGGER.warn("Could not find {}, using default log configurations.", LOG_PROPERTIES_FILE); + } + } catch (IOException ex) { + LOGGER.error("Failed to load log configuration from {}.", LOG_PROPERTIES_FILE, ex); + } + } + initializeNodeSelector(); + } + + @Override + public boolean isEnabled() { + return Boolean.parseBoolean(props.getProperty(LOG_S3_ENABLE_KEY, String.valueOf(DEFAULT_LOG_S3_ENABLE))); + } + + @Override + public String clusterId() { + return props.getProperty(LOG_S3_CLUSTER_ID_KEY, DEFAULT_LOG_S3_CLUSTER_ID); + } + + @Override + public int nodeId() { + return Integer.parseInt(props.getProperty(LOG_S3_NODE_ID_KEY, String.valueOf(DEFAULT_LOG_S3_NODE_ID))); + } + + @Override + public synchronized ObjectStorage objectStorage() { + if (this.objectStorage != null) { + return this.objectStorage; + } + String bucket = props.getProperty(LOG_S3_BUCKET_KEY); + if (StringUtils.isBlank(bucket)) { + LOGGER.error("Mandatory log config '{}' is not set.", LOG_S3_BUCKET_KEY); + return null; + } + + String normalizedBucket = bucket.trim(); + if (!normalizedBucket.contains("@")) { + String region = props.getProperty(LOG_S3_REGION_KEY); + if (StringUtils.isBlank(region)) { + LOGGER.error("'{}' must be provided when '{}' is not a full AutoMQ bucket URI.", + LOG_S3_REGION_KEY, LOG_S3_BUCKET_KEY); + return null; + } + String endpoint = props.getProperty(LOG_S3_ENDPOINT_KEY); + String accessKey = props.getProperty(LOG_S3_ACCESS_KEY); + String secretKey = props.getProperty(LOG_S3_SECRET_KEY); + + StringBuilder builder = new StringBuilder("0@s3://").append(normalizedBucket) + .append("?region=").append(region.trim()); + if (StringUtils.isNotBlank(endpoint)) { + builder.append("&endpoint=").append(endpoint.trim()); + } + if (StringUtils.isNotBlank(accessKey) && StringUtils.isNotBlank(secretKey)) { + builder.append("&authType=static") + .append("&accessKey=").append(accessKey.trim()) + .append("&secretKey=").append(secretKey.trim()); + } + normalizedBucket = builder.toString(); + } + + BucketURI logBucket = BucketURI.parse(normalizedBucket); + this.objectStorage = ObjectStorageFactory.instance().builder(logBucket).threadPrefix("s3-log-uploader").build(); + return this.objectStorage; + } + + @Override + public LogUploaderNodeSelector nodeSelector() { + if (nodeSelector == null) { + initializeNodeSelector(); + } + return nodeSelector; + } + + private void initializeNodeSelector() { + String selectorType = props.getProperty(LOG_S3_SELECTOR_TYPE_KEY, "static"); + Map selectorConfig = new HashMap<>(); + Map rawConfig = getPropertiesWithPrefix(LOG_S3_SELECTOR_PREFIX); + String normalizedType = selectorType == null ? "" : selectorType.toLowerCase(Locale.ROOT); + for (Map.Entry entry : rawConfig.entrySet()) { + String key = entry.getKey(); + if (normalizedType.length() > 0 && key.toLowerCase(Locale.ROOT).startsWith(normalizedType + ".")) { + key = key.substring(normalizedType.length() + 1); + } + if ("type".equalsIgnoreCase(key) || key.isEmpty()) { + continue; + } + selectorConfig.putIfAbsent(key, entry.getValue()); + } + + selectorConfig.putIfAbsent("isPrimaryUploader", + props.getProperty(LOG_S3_PRIMARY_NODE_KEY, + props.getProperty(LOG_S3_ACTIVE_CONTROLLER_KEY, String.valueOf(DEFAULT_LOG_S3_ACTIVE_CONTROLLER)))); + + String primaryNodeId = props.getProperty(LOG_S3_SELECTOR_PRIMARY_NODE_ID_KEY); + if (StringUtils.isNotBlank(primaryNodeId)) { + selectorConfig.putIfAbsent("primaryNodeId", primaryNodeId.trim()); + } + + try { + this.nodeSelector = LogUploaderNodeSelectorFactory.createSelector(selectorType, clusterId(), nodeId(), selectorConfig); + } catch (Exception e) { + LOGGER.error("Failed to create log uploader selector of type {}", selectorType, e); + this.nodeSelector = LogUploaderNodeSelector.staticSelector(false); + } + } + + private Map getPropertiesWithPrefix(String prefix) { + Map result = new HashMap<>(); + if (prefix == null || prefix.isEmpty()) { + return result; + } + for (String key : props.stringPropertyNames()) { + if (key.startsWith(prefix)) { + String trimmed = key.substring(prefix.length()); + if (!trimmed.isEmpty()) { + result.put(trimmed, props.getProperty(key)); + } + } + } + return result; + } +} diff --git a/automq-log-uploader/src/main/java/com/automq/log/uploader/LogConfigConstants.java b/automq-log-uploader/src/main/java/com/automq/log/uploader/LogConfigConstants.java new file mode 100644 index 0000000000..94c9378d89 --- /dev/null +++ b/automq-log-uploader/src/main/java/com/automq/log/uploader/LogConfigConstants.java @@ -0,0 +1,56 @@ +/* + * Copyright 2025, AutoMQ HK Limited. + * + * 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 com.automq.log.uploader; + +public class LogConfigConstants { + private LogConfigConstants() { + } + + public static final String LOG_PROPERTIES_FILE = "automq-log.properties"; + + public static final String LOG_S3_ENABLE_KEY = "log.s3.enable"; + public static final boolean DEFAULT_LOG_S3_ENABLE = false; + + public static final String LOG_S3_BUCKET_KEY = "log.s3.bucket"; + public static final String LOG_S3_REGION_KEY = "log.s3.region"; + public static final String LOG_S3_ENDPOINT_KEY = "log.s3.endpoint"; + + public static final String LOG_S3_ACCESS_KEY = "log.s3.access.key"; + public static final String LOG_S3_SECRET_KEY = "log.s3.secret.key"; + + public static final String LOG_S3_CLUSTER_ID_KEY = "log.s3.cluster.id"; + public static final String DEFAULT_LOG_S3_CLUSTER_ID = "automq-cluster"; + + public static final String LOG_S3_NODE_ID_KEY = "log.s3.node.id"; + public static final int DEFAULT_LOG_S3_NODE_ID = 0; + + /** + * @deprecated Use selector configuration instead. + */ + @Deprecated + public static final String LOG_S3_ACTIVE_CONTROLLER_KEY = "log.s3.active.controller"; + @Deprecated + public static final boolean DEFAULT_LOG_S3_ACTIVE_CONTROLLER = true; + + public static final String LOG_S3_PRIMARY_NODE_KEY = "log.s3.primary.node"; + public static final String LOG_S3_SELECTOR_PRIMARY_NODE_ID_KEY = "log.s3.selector.primary.node.id"; + public static final String LOG_S3_SELECTOR_TYPE_KEY = "log.s3.selector.type"; + public static final String LOG_S3_SELECTOR_PREFIX = "log.s3.selector."; +} diff --git a/automq-log-uploader/src/main/java/com/automq/log/uploader/LogRecorder.java b/automq-log-uploader/src/main/java/com/automq/log/uploader/LogRecorder.java new file mode 100644 index 0000000000..04dc3e6914 --- /dev/null +++ b/automq-log-uploader/src/main/java/com/automq/log/uploader/LogRecorder.java @@ -0,0 +1,77 @@ +/* + * Copyright 2025, AutoMQ HK Limited. + * + * 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 com.automq.log.uploader; + +import org.apache.commons.lang3.StringUtils; + +public interface LogRecorder { + boolean append(LogEvent event); + + class LogEvent { + private final long timestampMillis; + private final String level; + private final String logger; + private final String message; + private final String[] stackTrace; + + public LogEvent(long timestampMillis, String level, String logger, String message, String[] stackTrace) { + this.timestampMillis = timestampMillis; + this.level = level; + this.logger = logger; + this.message = message; + this.stackTrace = stackTrace; + } + + public void validate() { + if (timestampMillis <= 0) { + throw new IllegalArgumentException("Timestamp must be greater than 0"); + } + if (StringUtils.isBlank(level)) { + throw new IllegalArgumentException("Level cannot be blank"); + } + if (StringUtils.isBlank(logger)) { + throw new IllegalArgumentException("Logger cannot be blank"); + } + if (StringUtils.isBlank(message)) { + throw new IllegalArgumentException("Message cannot be blank"); + } + } + + public long timestampMillis() { + return timestampMillis; + } + + public String level() { + return level; + } + + public String logger() { + return logger; + } + + public String message() { + return message; + } + + public String[] stackTrace() { + return stackTrace; + } + } +} diff --git a/automq-log-uploader/src/main/java/com/automq/log/uploader/LogUploader.java b/automq-log-uploader/src/main/java/com/automq/log/uploader/LogUploader.java new file mode 100644 index 0000000000..8396b028e9 --- /dev/null +++ b/automq-log-uploader/src/main/java/com/automq/log/uploader/LogUploader.java @@ -0,0 +1,253 @@ +/* + * Copyright 2025, AutoMQ HK Limited. + * + * 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 com.automq.log.uploader; + +import com.automq.log.uploader.util.Utils; +import com.automq.stream.s3.operator.ObjectStorage; +import com.automq.stream.s3.operator.ObjectStorage.ObjectInfo; +import com.automq.stream.s3.operator.ObjectStorage.ObjectPath; +import com.automq.stream.s3.operator.ObjectStorage.WriteOptions; +import com.google.common.collect.Lists; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.ZonedDateTime; +import java.time.format.DateTimeFormatter; +import java.util.List; +import java.util.Random; +import java.util.UUID; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class LogUploader implements LogRecorder { + private static final Logger LOGGER = LoggerFactory.getLogger(LogUploader.class); + + public static final int DEFAULT_MAX_QUEUE_SIZE = 64 * 1024; + public static final int DEFAULT_BUFFER_SIZE = 16 * 1024 * 1024; + public static final int UPLOAD_INTERVAL = System.getenv("AUTOMQ_OBSERVABILITY_UPLOAD_INTERVAL") != null + ? Integer.parseInt(System.getenv("AUTOMQ_OBSERVABILITY_UPLOAD_INTERVAL")) + : 60 * 1000; + public static final int CLEANUP_INTERVAL = System.getenv("AUTOMQ_OBSERVABILITY_CLEANUP_INTERVAL") != null + ? Integer.parseInt(System.getenv("AUTOMQ_OBSERVABILITY_CLEANUP_INTERVAL")) + : 2 * 60 * 1000; + public static final int MAX_JITTER_INTERVAL = 60 * 1000; + + private final BlockingQueue queue = new LinkedBlockingQueue<>(DEFAULT_MAX_QUEUE_SIZE); + private final ByteBuf uploadBuffer = Unpooled.directBuffer(DEFAULT_BUFFER_SIZE); + private final Random random = new Random(); + private volatile long lastUploadTimestamp = System.currentTimeMillis(); + private volatile long nextUploadInterval = UPLOAD_INTERVAL + random.nextInt(MAX_JITTER_INTERVAL); + + private volatile boolean closed; + + private volatile S3LogConfig config; + + private ObjectStorage objectStorage; + private Thread uploadThread; + private Thread cleanupThread; + + public LogUploader() { + } + + public synchronized void start(S3LogConfig config) { + if (this.config != null) { + LOGGER.warn("LogUploader is already started."); + return; + } + this.config = config; + if (config == null || !config.isEnabled() || config.objectStorage() == null) { + LOGGER.warn("LogUploader is disabled due to invalid configuration."); + closed = true; + return; + } + + try { + this.objectStorage = config.objectStorage(); + this.uploadThread = new Thread(new UploadTask()); + this.uploadThread.setName("log-uploader-upload-thread"); + this.uploadThread.setDaemon(true); + this.uploadThread.start(); + + this.cleanupThread = new Thread(new CleanupTask()); + this.cleanupThread.setName("log-uploader-cleanup-thread"); + this.cleanupThread.setDaemon(true); + this.cleanupThread.start(); + + LOGGER.info("LogUploader started successfully."); + } catch (Exception e) { + LOGGER.error("Failed to start LogUploader", e); + closed = true; + } + } + + public void close() throws InterruptedException { + closed = true; + if (uploadThread != null) { + uploadThread.interrupt(); + uploadThread.join(); + } + if (cleanupThread != null) { + cleanupThread.interrupt(); + cleanupThread.join(); + } + if (objectStorage != null) { + objectStorage.close(); + } + } + + @Override + public boolean append(LogEvent event) { + if (!closed) { + return queue.offer(event); + } + return false; + } + + private class UploadTask implements Runnable { + + private String formatTimestampInMillis(long timestamp) { + return ZonedDateTime.ofInstant(Instant.ofEpochMilli(timestamp), ZoneId.systemDefault()) + .format(DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS Z")); + } + + @Override + public void run() { + while (!Thread.currentThread().isInterrupted()) { + try { + long now = System.currentTimeMillis(); + LogEvent event = queue.poll(1, TimeUnit.SECONDS); + if (event != null) { + StringBuilder logLine = new StringBuilder() + .append(formatTimestampInMillis(event.timestampMillis())) + .append(" ") + .append(event.level()) + .append(" ") + .append("[").append(event.logger()).append("] ") + .append(event.message()) + .append("\n"); + + String[] throwableStrRep = event.stackTrace(); + if (throwableStrRep != null) { + for (String stack : throwableStrRep) { + logLine.append(stack).append("\n"); + } + } + + byte[] bytes = logLine.toString().getBytes(StandardCharsets.UTF_8); + if (uploadBuffer.writableBytes() < bytes.length || now - lastUploadTimestamp > nextUploadInterval) { + upload(now); + } + uploadBuffer.writeBytes(bytes); + } else if (closed && queue.isEmpty()) { + upload(now); + break; + } else if (now - lastUploadTimestamp > nextUploadInterval) { + upload(now); + } + } catch (InterruptedException e) { + break; + } catch (Exception e) { + LOGGER.error("Upload log to s3 failed", e); + } + } + } + + private void upload(long now) { + if (uploadBuffer.readableBytes() > 0) { + try { + while (!Thread.currentThread().isInterrupted()) { + if (objectStorage == null) { + break; + } + + try { + String objectKey = getObjectKey(); + objectStorage.write(WriteOptions.DEFAULT, objectKey, Utils.compress(uploadBuffer.slice().asReadOnly())).get(); + break; + } catch (Exception e) { + LOGGER.warn("Failed to upload logs, will retry", e); + Thread.sleep(1000); + } + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + uploadBuffer.clear(); + lastUploadTimestamp = now; + nextUploadInterval = UPLOAD_INTERVAL + random.nextInt(MAX_JITTER_INTERVAL); + } + } + } + + private class CleanupTask implements Runnable { + + @Override + public void run() { + while (!Thread.currentThread().isInterrupted()) { + try { + if (closed || !config.isPrimaryUploader()) { + Thread.sleep(Duration.ofMinutes(1).toMillis()); + continue; + } + long expiredTime = System.currentTimeMillis() - CLEANUP_INTERVAL; + + List objects = objectStorage.list(String.format("automq/logs/%s", config.clusterId())).join(); + + if (!objects.isEmpty()) { + List keyList = objects.stream() + .filter(object -> object.timestamp() < expiredTime) + .map(object -> new ObjectPath(object.bucketId(), object.key())) + .collect(Collectors.toList()); + + if (!keyList.isEmpty()) { + CompletableFuture[] deleteFutures = Lists.partition(keyList, 1000) + .stream() + .map(objectStorage::delete) + .toArray(CompletableFuture[]::new); + CompletableFuture.allOf(deleteFutures).join(); + } + } + + Thread.sleep(Duration.ofMinutes(1).toMillis()); + } catch (InterruptedException e) { + break; + } catch (Exception e) { + LOGGER.error("Cleanup s3 logs failed", e); + } + } + } + } + + private String getObjectKey() { + String hour = LocalDateTime.now(ZoneOffset.UTC).format(DateTimeFormatter.ofPattern("yyyyMMddHH")); + return String.format("automq/logs/%s/%s/%s/%s", config.clusterId(), config.nodeId(), hour, UUID.randomUUID()); + } +} diff --git a/automq-log-uploader/src/main/java/com/automq/log/uploader/PropertiesS3LogConfigProvider.java b/automq-log-uploader/src/main/java/com/automq/log/uploader/PropertiesS3LogConfigProvider.java new file mode 100644 index 0000000000..c3dde10645 --- /dev/null +++ b/automq-log-uploader/src/main/java/com/automq/log/uploader/PropertiesS3LogConfigProvider.java @@ -0,0 +1,30 @@ +/* + * Copyright 2025, AutoMQ HK Limited. + * + * 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 com.automq.log.uploader; + +/** + * Default provider that loads configuration from {@code automq-log.properties} on the classpath. + */ +public class PropertiesS3LogConfigProvider implements S3LogConfigProvider { + @Override + public S3LogConfig get() { + return new DefaultS3LogConfig(); + } +} diff --git a/automq-log-uploader/src/main/java/com/automq/log/uploader/S3LogConfig.java b/automq-log-uploader/src/main/java/com/automq/log/uploader/S3LogConfig.java new file mode 100644 index 0000000000..1686a89efb --- /dev/null +++ b/automq-log-uploader/src/main/java/com/automq/log/uploader/S3LogConfig.java @@ -0,0 +1,40 @@ +/* + * Copyright 2025, AutoMQ HK Limited. + * + * 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 com.automq.log.uploader; + +import com.automq.log.uploader.selector.LogUploaderNodeSelector; +import com.automq.stream.s3.operator.ObjectStorage; + +public interface S3LogConfig { + boolean isEnabled(); + + String clusterId(); + + int nodeId(); + + ObjectStorage objectStorage(); + + LogUploaderNodeSelector nodeSelector(); + + default boolean isPrimaryUploader() { + LogUploaderNodeSelector selector = nodeSelector(); + return selector != null && selector.isPrimaryUploader(); + } +} diff --git a/automq-log-uploader/src/main/java/com/automq/log/uploader/S3LogConfigProvider.java b/automq-log-uploader/src/main/java/com/automq/log/uploader/S3LogConfigProvider.java new file mode 100644 index 0000000000..012c6c06bf --- /dev/null +++ b/automq-log-uploader/src/main/java/com/automq/log/uploader/S3LogConfigProvider.java @@ -0,0 +1,31 @@ +/* + * Copyright 2025, AutoMQ HK Limited. + * + * 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 com.automq.log.uploader; + +/** + * Provides {@link S3LogConfig} instances for the log uploader module. + */ +public interface S3LogConfigProvider { + + /** + * @return a configured {@link S3LogConfig} instance, or {@code null} if the uploader should stay disabled. + */ + S3LogConfig get(); +} diff --git a/automq-log-uploader/src/main/java/com/automq/log/uploader/S3RollingFileAppender.java b/automq-log-uploader/src/main/java/com/automq/log/uploader/S3RollingFileAppender.java new file mode 100644 index 0000000000..ddec90659e --- /dev/null +++ b/automq-log-uploader/src/main/java/com/automq/log/uploader/S3RollingFileAppender.java @@ -0,0 +1,205 @@ +/* + * Copyright 2025, AutoMQ HK Limited. + * + * 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 com.automq.log.uploader; + +import org.apache.commons.lang3.StringUtils; +import org.apache.log4j.RollingFileAppender; +import org.apache.log4j.spi.LoggingEvent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class S3RollingFileAppender extends RollingFileAppender { + public static final String CONFIG_PROVIDER_PROPERTY = "automq.log.s3.config.provider"; + + private static final Logger LOGGER = LoggerFactory.getLogger(S3RollingFileAppender.class); + private static final Object INIT_LOCK = new Object(); + + private static volatile LogUploader logUploaderInstance; + private static volatile S3LogConfigProvider configProvider; + private static volatile boolean initializationPending; + + private String configProviderClass; + + public S3RollingFileAppender() { + super(); + } + + /** + * Allows programmatic override of the LogUploader instance. + * Useful for testing or complex dependency injection scenarios. + * + * @param uploader The LogUploader instance to use. + */ + public static void setLogUploader(LogUploader uploader) { + synchronized (INIT_LOCK) { + logUploaderInstance = uploader; + } + } + + /** + * Programmatically sets the configuration provider to be used by all {@link S3RollingFileAppender} instances. + */ + public static void setConfigProvider(S3LogConfigProvider provider) { + synchronized (INIT_LOCK) { + configProvider = provider; + } + triggerInitialization(); + } + + /** + * Setter used by Log4j property configuration to specify a custom {@link S3LogConfigProvider} implementation. + */ + public void setConfigProviderClass(String configProviderClass) { + this.configProviderClass = configProviderClass; + } + + @Override + public void activateOptions() { + super.activateOptions(); + initializeUploader(); + } + + private void initializeUploader() { + if (logUploaderInstance != null) { + return; + } + synchronized (INIT_LOCK) { + if (logUploaderInstance != null) { + return; + } + try { + S3LogConfigProvider provider = resolveProvider(); + if (provider == null) { + LOGGER.info("No S3LogConfigProvider available; S3 log upload remains disabled."); + initializationPending = true; + return; + } + S3LogConfig config = provider.get(); + if (config == null || !config.isEnabled() || config.objectStorage() == null) { + LOGGER.info("S3 log upload is disabled by configuration."); + initializationPending = config == null; + return; + } + + LogUploader uploader = new LogUploader(); + uploader.start(config); + logUploaderInstance = uploader; + initializationPending = false; + + Runtime.getRuntime().addShutdownHook(new Thread(() -> { + try { + uploader.close(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOGGER.warn("Failed to close LogUploader gracefully", e); + } + })); + LOGGER.info("S3RollingFileAppender initialized successfully using provider {}.", + provider.getClass().getName()); + } catch (Exception e) { + LOGGER.error("Failed to initialize S3RollingFileAppender", e); + initializationPending = true; + } + } + } + + public static void triggerInitialization() { + S3LogConfigProvider provider; + synchronized (INIT_LOCK) { + if (logUploaderInstance != null) { + return; + } + provider = configProvider; + } + if (provider == null) { + initializationPending = true; + return; + } + new S3RollingFileAppender().initializeUploader(); + } + + private S3LogConfigProvider resolveProvider() { + S3LogConfigProvider provider = configProvider; + if (provider != null) { + return provider; + } + + synchronized (INIT_LOCK) { + if (configProvider != null) { + return configProvider; + } + + String providerClassName = configProviderClass; + if (StringUtils.isBlank(providerClassName)) { + providerClassName = System.getProperty(CONFIG_PROVIDER_PROPERTY); + } + + if (StringUtils.isNotBlank(providerClassName)) { + provider = instantiateProvider(providerClassName.trim()); + if (provider == null) { + LOGGER.warn("Falling back to default configuration provider because {} could not be instantiated.", + providerClassName); + } + } + + if (provider == null) { + provider = new PropertiesS3LogConfigProvider(); + } + + configProvider = provider; + return provider; + } + } + + private S3LogConfigProvider instantiateProvider(String providerClassName) { + try { + Class clazz = Class.forName(providerClassName); + Object instance = clazz.getDeclaredConstructor().newInstance(); + if (!(instance instanceof S3LogConfigProvider)) { + LOGGER.error("Class {} does not implement S3LogConfigProvider.", providerClassName); + return null; + } + return (S3LogConfigProvider) instance; + } catch (Exception e) { + LOGGER.error("Failed to instantiate S3LogConfigProvider {}", providerClassName, e); + return null; + } + } + + @Override + protected void subAppend(LoggingEvent event) { + super.subAppend(event); + if (!closed && logUploaderInstance != null) { + LogRecorder.LogEvent logEvent = new LogRecorder.LogEvent( + event.getTimeStamp(), + event.getLevel().toString(), + event.getLoggerName(), + event.getRenderedMessage(), + event.getThrowableStrRep()); + + try { + logEvent.validate(); + logUploaderInstance.append(logEvent); + } catch (IllegalArgumentException e) { + errorHandler.error("Failed to validate and append log event", e, 0); + } + } + } +} diff --git a/automq-log-uploader/src/main/java/com/automq/log/uploader/selector/LogUploaderNodeSelector.java b/automq-log-uploader/src/main/java/com/automq/log/uploader/selector/LogUploaderNodeSelector.java new file mode 100644 index 0000000000..a3a690cff4 --- /dev/null +++ b/automq-log-uploader/src/main/java/com/automq/log/uploader/selector/LogUploaderNodeSelector.java @@ -0,0 +1,22 @@ +package com.automq.log.uploader.selector; + +/** + * Determines whether the current node should act as the primary S3 log uploader. + */ +public interface LogUploaderNodeSelector { + + /** + * @return {@code true} if the current node should upload and clean up logs in S3. + */ + boolean isPrimaryUploader(); + + /** + * Creates a selector with a static boolean decision. + * + * @param primary whether this node should be primary + * @return selector returning the static decision + */ + static LogUploaderNodeSelector staticSelector(boolean primary) { + return () -> primary; + } +} diff --git a/automq-log-uploader/src/main/java/com/automq/log/uploader/selector/LogUploaderNodeSelectorFactory.java b/automq-log-uploader/src/main/java/com/automq/log/uploader/selector/LogUploaderNodeSelectorFactory.java new file mode 100644 index 0000000000..d3e459a743 --- /dev/null +++ b/automq-log-uploader/src/main/java/com/automq/log/uploader/selector/LogUploaderNodeSelectorFactory.java @@ -0,0 +1,74 @@ +package com.automq.log.uploader.selector; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Locale; +import java.util.Map; +import java.util.ServiceLoader; + +/** + * Factory that resolves node selectors from configuration. + */ +public final class LogUploaderNodeSelectorFactory { + private static final Logger LOGGER = LoggerFactory.getLogger(LogUploaderNodeSelectorFactory.class); + private static final Map PROVIDERS = new HashMap<>(); + + static { + ServiceLoader loader = ServiceLoader.load(LogUploaderNodeSelectorProvider.class); + for (LogUploaderNodeSelectorProvider provider : loader) { + String type = provider.getType(); + if (type != null) { + PROVIDERS.put(type.toLowerCase(Locale.ROOT), provider); + LOGGER.info("Loaded LogUploaderNodeSelectorProvider for type {}", type); + } + } + } + + private LogUploaderNodeSelectorFactory() { + } + + public static LogUploaderNodeSelector createSelector(String typeString, + String clusterId, + int nodeId, + Map config) { + LogUploaderNodeSelectorType type = LogUploaderNodeSelectorType.fromString(typeString); + switch (type) { + case STATIC: + boolean isPrimary = Boolean.parseBoolean(config.getOrDefault("isPrimaryUploader", "false")); + return LogUploaderNodeSelectors.staticSelector(isPrimary); + case NODE_ID: + int primaryNodeId = Integer.parseInt(config.getOrDefault("primaryNodeId", "0")); + return LogUploaderNodeSelectors.nodeIdSelector(nodeId, primaryNodeId); + case FILE: + String leaderFile = config.getOrDefault("leaderFile", "/tmp/log-uploader-leader"); + long timeoutMs = Long.parseLong(config.getOrDefault("leaderTimeoutMs", "60000")); + return LogUploaderNodeSelectors.fileLeaderElectionSelector(leaderFile, nodeId, timeoutMs); + case CUSTOM: + LogUploaderNodeSelectorProvider provider = PROVIDERS.get(typeString.toLowerCase(Locale.ROOT)); + if (provider != null) { + try { + return provider.createSelector(clusterId, nodeId, config); + } catch (Exception e) { + LOGGER.error("Failed to create selector of type {}", typeString, e); + } + } + LOGGER.warn("Unsupported log uploader selector type {}, falling back to static=false", typeString); + return LogUploaderNodeSelector.staticSelector(false); + default: + return LogUploaderNodeSelector.staticSelector(false); + } + } + + public static boolean isSupported(String typeString) { + if (typeString == null) { + return true; + } + LogUploaderNodeSelectorType type = LogUploaderNodeSelectorType.fromString(typeString); + if (type != LogUploaderNodeSelectorType.CUSTOM) { + return true; + } + return PROVIDERS.containsKey(typeString.toLowerCase(Locale.ROOT)); + } +} diff --git a/automq-log-uploader/src/main/java/com/automq/log/uploader/selector/LogUploaderNodeSelectorProvider.java b/automq-log-uploader/src/main/java/com/automq/log/uploader/selector/LogUploaderNodeSelectorProvider.java new file mode 100644 index 0000000000..8edfde1ded --- /dev/null +++ b/automq-log-uploader/src/main/java/com/automq/log/uploader/selector/LogUploaderNodeSelectorProvider.java @@ -0,0 +1,25 @@ +package com.automq.log.uploader.selector; + +import java.util.Map; + +/** + * Service Provider Interface for custom log uploader node selection strategies. + */ +public interface LogUploaderNodeSelectorProvider { + + /** + * @return the selector type identifier (case insensitive) + */ + String getType(); + + /** + * Creates a selector based on the supplied configuration. + * + * @param clusterId logical cluster identifier + * @param nodeId numeric node identifier + * @param config additional selector configuration + * @return selector instance + * @throws Exception if creation fails + */ + LogUploaderNodeSelector createSelector(String clusterId, int nodeId, Map config) throws Exception; +} diff --git a/automq-log-uploader/src/main/java/com/automq/log/uploader/selector/LogUploaderNodeSelectorType.java b/automq-log-uploader/src/main/java/com/automq/log/uploader/selector/LogUploaderNodeSelectorType.java new file mode 100644 index 0000000000..e955c25172 --- /dev/null +++ b/automq-log-uploader/src/main/java/com/automq/log/uploader/selector/LogUploaderNodeSelectorType.java @@ -0,0 +1,42 @@ +package com.automq.log.uploader.selector; + +import java.util.HashMap; +import java.util.Locale; +import java.util.Map; + +/** + * Supported selector types. + */ +public enum LogUploaderNodeSelectorType { + STATIC("static"), + NODE_ID("nodeid"), + FILE("file"), + CUSTOM(null); + + private static final Map LOOKUP = new HashMap<>(); + + static { + for (LogUploaderNodeSelectorType value : values()) { + if (value.type != null) { + LOOKUP.put(value.type, value); + } + } + } + + private final String type; + + LogUploaderNodeSelectorType(String type) { + this.type = type; + } + + public String getType() { + return type; + } + + public static LogUploaderNodeSelectorType fromString(String type) { + if (type == null) { + return STATIC; + } + return LOOKUP.getOrDefault(type.toLowerCase(Locale.ROOT), CUSTOM); + } +} diff --git a/automq-log-uploader/src/main/java/com/automq/log/uploader/selector/LogUploaderNodeSelectors.java b/automq-log-uploader/src/main/java/com/automq/log/uploader/selector/LogUploaderNodeSelectors.java new file mode 100644 index 0000000000..ba92980ea4 --- /dev/null +++ b/automq-log-uploader/src/main/java/com/automq/log/uploader/selector/LogUploaderNodeSelectors.java @@ -0,0 +1,85 @@ +package com.automq.log.uploader.selector; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Utility methods providing built-in selector implementations. + */ +public final class LogUploaderNodeSelectors { + private static final Logger LOGGER = LoggerFactory.getLogger(LogUploaderNodeSelectors.class); + + private LogUploaderNodeSelectors() { + } + + public static LogUploaderNodeSelector staticSelector(boolean isPrimary) { + return LogUploaderNodeSelector.staticSelector(isPrimary); + } + + public static LogUploaderNodeSelector nodeIdSelector(int currentNodeId, int primaryNodeId) { + return () -> currentNodeId == primaryNodeId; + } + + public static LogUploaderNodeSelector fileLeaderElectionSelector(String leaderFilePath, + int nodeId, + long leaderTimeoutMs) { + Path path = Paths.get(leaderFilePath); + AtomicBoolean isLeader = new AtomicBoolean(false); + + Thread leaderThread = new Thread(() -> { + while (!Thread.currentThread().isInterrupted()) { + try { + boolean claimed = attemptToClaimLeadership(path, nodeId, leaderTimeoutMs); + isLeader.set(claimed); + Thread.sleep(Math.max(leaderTimeoutMs / 2, 1000)); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } catch (Exception e) { + LOGGER.warn("File leader election failed", e); + isLeader.set(false); + try { + Thread.sleep(1000); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + } + } + } + }, "log-uploader-file-selector"); + leaderThread.setDaemon(true); + leaderThread.start(); + + return isLeader::get; + } + + private static boolean attemptToClaimLeadership(Path leaderFilePath, int nodeId, long leaderTimeoutMs) throws IOException { + Files.createDirectories(leaderFilePath.getParent()); + if (Files.exists(leaderFilePath)) { + List lines = Files.readAllLines(leaderFilePath); + if (!lines.isEmpty()) { + String[] parts = lines.get(0).split(":"); + if (parts.length == 2) { + int currentLeader = Integer.parseInt(parts[0]); + long ts = Long.parseLong(parts[1]); + if (System.currentTimeMillis() - ts <= leaderTimeoutMs) { + return currentLeader == nodeId; + } + } + } + } + String content = nodeId + ":" + System.currentTimeMillis(); + Files.write(leaderFilePath, content.getBytes()); + List lines = Files.readAllLines(leaderFilePath); + if (!lines.isEmpty()) { + String[] parts = lines.get(0).split(":"); + return parts.length == 2 && Integer.parseInt(parts[0]) == nodeId; + } + return false; + } +} diff --git a/automq-log-uploader/src/main/java/com/automq/log/uploader/selector/kafka/KafkaLogLeaderSelectorProvider.java b/automq-log-uploader/src/main/java/com/automq/log/uploader/selector/kafka/KafkaLogLeaderSelectorProvider.java new file mode 100644 index 0000000000..4e2cf38dba --- /dev/null +++ b/automq-log-uploader/src/main/java/com/automq/log/uploader/selector/kafka/KafkaLogLeaderSelectorProvider.java @@ -0,0 +1,383 @@ +package com.automq.log.uploader.selector.kafka; + +import com.automq.log.uploader.selector.LogUploaderNodeSelector; +import com.automq.log.uploader.selector.LogUploaderNodeSelectorProvider; +import org.apache.commons.lang3.StringUtils; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.CreateTopicsOptions; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.common.errors.TopicExistsException; +import org.apache.kafka.common.errors.WakeupException; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Locale; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Leader election based on Kafka consumer group membership. + */ +public class KafkaLogLeaderSelectorProvider implements LogUploaderNodeSelectorProvider { + private static final Logger LOGGER = LoggerFactory.getLogger(KafkaLogLeaderSelectorProvider.class); + + public static final String TYPE = "kafka"; + + private static final String DEFAULT_TOPIC_PREFIX = "__automq_log_uploader_leader_"; + private static final String DEFAULT_GROUP_PREFIX = "automq-log-uploader-"; + private static final String DEFAULT_CLIENT_PREFIX = "automq-log-uploader"; + + private static final long DEFAULT_TOPIC_RETENTION_MS = TimeUnit.MINUTES.toMillis(30); + private static final int DEFAULT_POLL_INTERVAL_MS = 1000; + private static final long DEFAULT_RETRY_BACKOFF_MS = TimeUnit.SECONDS.toMillis(5); + private static final int DEFAULT_SESSION_TIMEOUT_MS = 10000; + private static final int DEFAULT_HEARTBEAT_INTERVAL_MS = 3000; + + private static final Set RESERVED_KEYS; + + static { + Set keys = new HashSet<>(); + Collections.addAll(keys, + "bootstrap.servers", + "topic", + "group.id", + "client.id", + "auto.create.topic", + "topic.partitions", + "topic.replication.factor", + "topic.retention.ms", + "poll.interval.ms", + "retry.backoff.ms", + "session.timeout.ms", + "heartbeat.interval.ms", + "request.timeout.ms" + ); + RESERVED_KEYS = Collections.unmodifiableSet(keys); + } + + @Override + public String getType() { + return TYPE; + } + + @Override + public LogUploaderNodeSelector createSelector(String clusterId, int nodeId, Map config) throws Exception { + KafkaSelectorConfig selectorConfig = KafkaSelectorConfig.from(clusterId, nodeId, config); + KafkaSelector selector = new KafkaSelector(selectorConfig); + selector.start(); + return selector; + } + + private static final class KafkaSelector implements LogUploaderNodeSelector { + private final KafkaSelectorConfig config; + private final AtomicBoolean isLeader = new AtomicBoolean(false); + private final AtomicBoolean running = new AtomicBoolean(true); + + private volatile KafkaConsumer consumer; + + KafkaSelector(KafkaSelectorConfig config) { + this.config = config; + } + + void start() { + Thread thread = new Thread(this::runLoop, + String.format(Locale.ROOT, "log-uploader-kafka-selector-%s-%d", config.clusterId, config.nodeId)); + thread.setDaemon(true); + thread.start(); + Runtime.getRuntime().addShutdownHook(new Thread(this::shutdown, + String.format(Locale.ROOT, "log-uploader-kafka-selector-shutdown-%s-%d", config.clusterId, config.nodeId))); + } + + private void runLoop() { + while (running.get()) { + try { + ensureTopicExists(); + runConsumer(); + } catch (WakeupException e) { + if (!running.get()) { + break; + } + LOGGER.warn("Kafka selector interrupted unexpectedly", e); + sleep(config.retryBackoffMs); + } catch (Exception e) { + if (!running.get()) { + break; + } + LOGGER.warn("Kafka selector loop failed: {}", e.getMessage(), e); + sleep(config.retryBackoffMs); + } + } + } + + private void runConsumer() { + Properties consumerProps = config.buildConsumerProps(); + try (KafkaConsumer kafkaConsumer = + new KafkaConsumer<>(consumerProps, new ByteArrayDeserializer(), new ByteArrayDeserializer())) { + this.consumer = kafkaConsumer; + ConsumerRebalanceListener listener = new LeaderRebalanceListener(); + kafkaConsumer.subscribe(Collections.singletonList(config.topic), listener); + LOGGER.info("Kafka log selector subscribed to topic {} with group {}", config.topic, config.groupId); + while (running.get()) { + kafkaConsumer.poll(Duration.ofMillis(config.pollIntervalMs)); + } + } finally { + this.consumer = null; + demote(); + } + } + + private void ensureTopicExists() throws Exception { + if (!config.autoCreateTopic) { + return; + } + Properties adminProps = config.buildAdminProps(); + try (Admin admin = Admin.create(adminProps)) { + NewTopic topic = new NewTopic(config.topic, config.topicPartitions, config.topicReplicationFactor); + Map topicConfig = new HashMap<>(); + if (config.topicRetentionMs > 0) { + topicConfig.put(TopicConfig.RETENTION_MS_CONFIG, String.valueOf(config.topicRetentionMs)); + } + if (!topicConfig.isEmpty()) { + topic.configs(topicConfig); + } + admin.createTopics(Collections.singleton(topic), new CreateTopicsOptions().validateOnly(false)).all().get(); + LOGGER.info("Kafka log selector ensured topic {} exists", config.topic); + } catch (TopicExistsException ignored) { + // already exists + } catch (Exception e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + throw e; + } + Throwable cause = e.getCause(); + if (!(cause instanceof TopicExistsException)) { + throw e; + } + } + } + + @Override + public boolean isPrimaryUploader() { + return isLeader.get(); + } + + private void promote() { + if (isLeader.compareAndSet(false, true)) { + LOGGER.info("Node {} became primary log uploader for cluster {}", config.nodeId, config.clusterId); + } + } + + private void demote() { + if (isLeader.getAndSet(false)) { + LOGGER.info("Node {} lost log uploader leadership for cluster {}", config.nodeId, config.clusterId); + } + } + + private void shutdown() { + if (running.compareAndSet(true, false)) { + KafkaConsumer current = consumer; + if (current != null) { + current.wakeup(); + } + } + } + + private void sleep(long millis) { + try { + Thread.sleep(millis); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + + private class LeaderRebalanceListener implements ConsumerRebalanceListener { + @Override + public void onPartitionsRevoked(Collection partitions) { + if (!partitions.isEmpty()) { + LOGGER.debug("Kafka log selector revoked partitions {}", partitions); + } + demote(); + } + + @Override + public void onPartitionsAssigned(Collection partitions) { + if (!partitions.isEmpty()) { + promote(); + } + } + } + } + + private static final class KafkaSelectorConfig { + private final String clusterId; + private final int nodeId; + private final String bootstrapServers; + private final String topic; + private final String groupId; + private final String clientId; + private final boolean autoCreateTopic; + private final int topicPartitions; + private final short topicReplicationFactor; + private final long topicRetentionMs; + private final int pollIntervalMs; + private final long retryBackoffMs; + private final int sessionTimeoutMs; + private final int heartbeatIntervalMs; + private final int requestTimeoutMs; + private final Properties clientOverrides; + + private KafkaSelectorConfig(Builder builder) { + this.clusterId = builder.clusterId; + this.nodeId = builder.nodeId; + this.bootstrapServers = builder.bootstrapServers; + this.topic = builder.topic; + this.groupId = builder.groupId; + this.clientId = builder.clientId; + this.autoCreateTopic = builder.autoCreateTopic; + this.topicPartitions = builder.topicPartitions; + this.topicReplicationFactor = builder.topicReplicationFactor; + this.topicRetentionMs = builder.topicRetentionMs; + this.pollIntervalMs = builder.pollIntervalMs; + this.retryBackoffMs = builder.retryBackoffMs; + this.sessionTimeoutMs = builder.sessionTimeoutMs; + this.heartbeatIntervalMs = builder.heartbeatIntervalMs; + this.requestTimeoutMs = builder.requestTimeoutMs; + this.clientOverrides = builder.clientOverrides; + } + + static KafkaSelectorConfig from(String clusterId, int nodeId, Map rawConfig) { + Map config = rawConfig == null ? Collections.emptyMap() : rawConfig; + String bootstrapServers = findBootstrapServers(config); + if (StringUtils.isBlank(bootstrapServers)) { + throw new IllegalArgumentException("Kafka selector requires 'bootstrap.servers'"); + } + String normalizedCluster = StringUtils.isBlank(clusterId) ? "default" : clusterId; + Builder builder = new Builder(); + builder.clusterId = clusterId; + builder.nodeId = nodeId; + builder.bootstrapServers = bootstrapServers; + builder.topic = config.getOrDefault("topic", DEFAULT_TOPIC_PREFIX + normalizedCluster); + builder.groupId = config.getOrDefault("group.id", DEFAULT_GROUP_PREFIX + normalizedCluster); + builder.clientId = config.getOrDefault("client.id", DEFAULT_CLIENT_PREFIX + "-" + normalizedCluster + "-" + nodeId); + builder.autoCreateTopic = Boolean.parseBoolean(config.getOrDefault("auto.create.topic", "true")); + builder.topicPartitions = parseInt(config.get("topic.partitions"), 1, 1); + builder.topicReplicationFactor = (short) parseInt(config.get("topic.replication.factor"), 1, 1); + builder.topicRetentionMs = parseLong(config.get("topic.retention.ms"), DEFAULT_TOPIC_RETENTION_MS); + builder.pollIntervalMs = parseInt(config.get("poll.interval.ms"), DEFAULT_POLL_INTERVAL_MS, 100); + builder.retryBackoffMs = parseLong(config.get("retry.backoff.ms"), DEFAULT_RETRY_BACKOFF_MS); + builder.sessionTimeoutMs = parseInt(config.get("session.timeout.ms"), DEFAULT_SESSION_TIMEOUT_MS, 1000); + builder.heartbeatIntervalMs = parseInt(config.get("heartbeat.interval.ms"), DEFAULT_HEARTBEAT_INTERVAL_MS, 500); + builder.requestTimeoutMs = parseInt(config.get("request.timeout.ms"), 15000, 1000); + builder.clientOverrides = extractOverrides(config); + return builder.build(); + } + + Properties buildConsumerProps() { + Properties props = new Properties(); + props.putAll(clientOverrides); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); + props.put(ConsumerConfig.CLIENT_ID_CONFIG, clientId + "-consumer"); + props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, OffsetResetStrategy.EARLIEST.name().toLowerCase(Locale.ROOT)); + props.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, sessionTimeoutMs); + props.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, heartbeatIntervalMs); + props.put(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, Math.max(pollIntervalMs * 3, 3000)); + props.put(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG, requestTimeoutMs); + props.put(ConsumerConfig.ALLOW_AUTO_CREATE_TOPICS_CONFIG, "false"); + props.putIfAbsent(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + props.putIfAbsent(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + return props; + } + + Properties buildAdminProps() { + Properties props = new Properties(); + props.putAll(clientOverrides); + props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + props.put(AdminClientConfig.CLIENT_ID_CONFIG, clientId + "-admin"); + props.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, requestTimeoutMs); + return props; + } + + private static Properties extractOverrides(Map config) { + Properties props = new Properties(); + for (Map.Entry entry : config.entrySet()) { + if (RESERVED_KEYS.contains(entry.getKey())) { + continue; + } + props.put(entry.getKey(), entry.getValue()); + } + return props; + } + + private static String findBootstrapServers(Map config) { + String value = config.get("bootstrap.servers"); + if (StringUtils.isNotBlank(value)) { + return value; + } + return config.get("bootstrapServers"); + } + + private static int parseInt(String value, int defaultValue, int minimum) { + if (StringUtils.isBlank(value)) { + return defaultValue; + } + try { + int parsed = Integer.parseInt(value.trim()); + return Math.max(parsed, minimum); + } catch (NumberFormatException e) { + return defaultValue; + } + } + + private static long parseLong(String value, long defaultValue) { + if (StringUtils.isBlank(value)) { + return defaultValue; + } + try { + return Long.parseLong(value.trim()); + } catch (NumberFormatException e) { + return defaultValue; + } + } + + private static final class Builder { + private String clusterId; + private int nodeId; + private String bootstrapServers; + private String topic; + private String groupId; + private String clientId; + private boolean autoCreateTopic; + private int topicPartitions; + private short topicReplicationFactor; + private long topicRetentionMs; + private int pollIntervalMs; + private long retryBackoffMs; + private int sessionTimeoutMs; + private int heartbeatIntervalMs; + private int requestTimeoutMs; + private Properties clientOverrides = new Properties(); + + private KafkaSelectorConfig build() { + return new KafkaSelectorConfig(this); + } + } + } +} diff --git a/automq-log-uploader/src/main/java/com/automq/log/uploader/util/Utils.java b/automq-log-uploader/src/main/java/com/automq/log/uploader/util/Utils.java new file mode 100644 index 0000000000..acd3d7b7d4 --- /dev/null +++ b/automq-log-uploader/src/main/java/com/automq/log/uploader/util/Utils.java @@ -0,0 +1,68 @@ +/* + * Copyright 2025, AutoMQ HK Limited. + * + * 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 com.automq.log.uploader.util; + +import com.automq.stream.s3.ByteBufAlloc; +import io.netty.buffer.ByteBuf; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.zip.GZIPInputStream; +import java.util.zip.GZIPOutputStream; + +public class Utils { + + private Utils() { + } + + public static ByteBuf compress(ByteBuf input) throws IOException { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + try (GZIPOutputStream gzipOutputStream = new GZIPOutputStream(byteArrayOutputStream)) { + byte[] buffer = new byte[input.readableBytes()]; + input.readBytes(buffer); + gzipOutputStream.write(buffer); + } + + ByteBuf compressed = ByteBufAlloc.byteBuffer(byteArrayOutputStream.size()); + compressed.writeBytes(byteArrayOutputStream.toByteArray()); + return compressed; + } + + public static ByteBuf decompress(ByteBuf input) throws IOException { + byte[] compressedData = new byte[input.readableBytes()]; + input.readBytes(compressedData); + ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(compressedData); + + try (GZIPInputStream gzipInputStream = new GZIPInputStream(byteArrayInputStream); + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream()) { + byte[] buffer = new byte[1024]; + int bytesRead; + while ((bytesRead = gzipInputStream.read(buffer)) != -1) { + byteArrayOutputStream.write(buffer, 0, bytesRead); + } + + byte[] uncompressedData = byteArrayOutputStream.toByteArray(); + ByteBuf output = ByteBufAlloc.byteBuffer(uncompressedData.length); + output.writeBytes(uncompressedData); + return output; + } + } +} diff --git a/automq-log-uploader/src/main/resources/META-INF/services/com.automq.log.uploader.selector.LogUploaderNodeSelectorProvider b/automq-log-uploader/src/main/resources/META-INF/services/com.automq.log.uploader.selector.LogUploaderNodeSelectorProvider new file mode 100644 index 0000000000..ad1ce25af0 --- /dev/null +++ b/automq-log-uploader/src/main/resources/META-INF/services/com.automq.log.uploader.selector.LogUploaderNodeSelectorProvider @@ -0,0 +1 @@ +com.automq.log.uploader.selector.kafka.KafkaLogLeaderSelectorProvider diff --git a/build.gradle b/build.gradle index 86545f9cb3..e13d867d8b 100644 --- a/build.gradle +++ b/build.gradle @@ -260,12 +260,12 @@ subprojects { tasks.withType(JavaCompile) { options.encoding = 'UTF-8' - options.compilerArgs << "-Xlint:all" - // temporary exclusions until all the warnings are fixed - if (!project.path.startsWith(":connect") && !project.path.startsWith(":storage")) - options.compilerArgs << "-Xlint:-rawtypes" - options.compilerArgs << "-Xlint:-serial" - options.compilerArgs << "-Xlint:-try" +// options.compilerArgs << "-Xlint:all" +// // temporary exclusions until all the warnings are fixed +// if (!project.path.startsWith(":connect") && !project.path.startsWith(":storage")) +// options.compilerArgs << "-Xlint:-rawtypes" +// options.compilerArgs << "-Xlint:-serial" +// options.compilerArgs << "-Xlint:-try" // AutoMQ inject start // TODO: remove me, when upgrade to 4.x // options.compilerArgs << "-Werror" @@ -840,6 +840,13 @@ tasks.create(name: "jarConnect", dependsOn: connectPkgs.collect { it + ":jar" }) tasks.create(name: "testConnect", dependsOn: connectPkgs.collect { it + ":test" }) {} +// OpenTelemetry related tasks +tasks.create(name: "jarOpenTelemetry", dependsOn: ":opentelemetry:jar") {} + +tasks.create(name: "testOpenTelemetry", dependsOn: ":opentelemetry:test") {} + +tasks.create(name: "buildOpenTelemetry", dependsOn: [":opentelemetry:jar", ":opentelemetry:test"]) {} + project(':server') { base { archivesName = "kafka-server" @@ -1251,6 +1258,8 @@ project(':core') { from(project(':trogdor').jar) { into("libs/") } from(project(':trogdor').configurations.runtimeClasspath) { into("libs/") } from(project(':automq-shell').jar) { into("libs/") } + from(project(':opentelemetry').jar) { into("libs/") } + from(project(':opentelemetry').configurations.runtimeClasspath) { into("libs/") } from(project(':automq-shell').configurations.runtimeClasspath) { into("libs/") } from(project(':shell').jar) { into("libs/") } from(project(':shell').configurations.runtimeClasspath) { into("libs/") } @@ -2482,7 +2491,7 @@ project(':trogdor') { from (configurations.runtimeClasspath) { exclude('kafka-clients*') } - into "$buildDir/dependant-libs-${versions.scala}" + into "$buildDir/dependant-libs" duplicatesStrategy 'exclude' } @@ -3451,6 +3460,8 @@ project(':connect:runtime') { api project(':clients') api project(':connect:json') api project(':connect:transforms') + api project(':opentelemetry') + implementation project(':automq-log-uploader') implementation libs.slf4jApi implementation libs.reload4j diff --git a/config/connect-log4j.properties b/config/connect-log4j.properties index 61b2ac331d..506409624d 100644 --- a/config/connect-log4j.properties +++ b/config/connect-log4j.properties @@ -24,7 +24,8 @@ log4j.appender.stdout.layout=org.apache.log4j.PatternLayout # location of the log files (e.g. ${kafka.logs.dir}/connect.log). The `MaxFileSize` option specifies the maximum size of the log file, # and the `MaxBackupIndex` option specifies the number of backup files to keep. # -log4j.appender.connectAppender=org.apache.log4j.RollingFileAppender +log4j.appender.connectAppender=com.automq.log.uploader.S3RollingFileAppender +log4j.appender.connectAppender.configProviderClass=org.apache.kafka.connect.automq.ConnectS3LogConfigProvider log4j.appender.connectAppender.MaxFileSize=10MB log4j.appender.connectAppender.MaxBackupIndex=11 log4j.appender.connectAppender.File=${kafka.logs.dir}/connect.log diff --git a/connect/runtime/README.md b/connect/runtime/README.md new file mode 100644 index 0000000000..096dabedb3 --- /dev/null +++ b/connect/runtime/README.md @@ -0,0 +1,229 @@ +# Kafka Connect OpenTelemetry Metrics Integration + +## Overview + +This integration allows Kafka Connect to export metrics through the AutoMQ OpenTelemetry module, enabling unified observability across your Kafka ecosystem. + +## Configuration + +### 1. Enable the MetricsReporter + +Add the following to your Kafka Connect configuration file (`connect-distributed.properties` or `connect-standalone.properties`): + +```properties +# Enable OpenTelemetry MetricsReporter +metric.reporters=org.apache.kafka.connect.automq.OpenTelemetryMetricsReporter + +# OpenTelemetry configuration +opentelemetry.metrics.enabled=true +opentelemetry.metrics.prefix=kafka.connect + +# Optional: Filter metrics +opentelemetry.metrics.include.pattern=.*connector.*|.*task.*|.*worker.* +opentelemetry.metrics.exclude.pattern=.*jmx.*|.*debug.* +``` + +### 2. AutoMQ Telemetry Configuration + +Ensure the AutoMQ telemetry is properly configured. Add these properties to your application configuration: + +```properties +# Telemetry export configuration +automq.telemetry.exporter.uri=prometheus://localhost:9090 +# or for OTLP: automq.telemetry.exporter.uri=otlp://localhost:4317 + +# Service identification +service.name=kafka-connect +service.instance.id=connect-worker-1 + +# Export settings +automq.telemetry.exporter.interval.ms=30000 +automq.telemetry.metric.cardinality.limit=10000 +``` + +## S3 Log Upload + +Kafka Connect bundles the AutoMQ log uploader so that worker logs can be streamed to S3 together with in-cluster cleanup. The uploader reuses the same leader election mechanism as the metrics, using Kafka by default, and requires no additional configuration. + +### Worker Configuration + +Add the following properties to your worker configuration (ConfigMap, properties file, etc.): + +```properties +# Enable S3 log upload +log.s3.enable=true +log.s3.bucket=0@s3://your-log-bucket?region=us-east-1 + +# Optional overrides (defaults shown) +log.s3.selector.type=kafka +log.s3.selector.kafka.bootstrap.servers=${bootstrap.servers} +log.s3.selector.kafka.topic=__automq_connect_log_leader_${group.id} +log.s3.selector.kafka.group.id=automq-log-uploader-${group.id} +# Provide credentials if the bucket URI does not embed them +# log.s3.access.key=... +# log.s3.secret.key=... +``` + +`log.s3.node.id` defaults to a hash of the pod hostname if not provided, ensuring objects are partitioned per worker. For `static` or `nodeid` leader election, you can explicitly set: + +```properties +log.s3.selector.type=static +log.s3.primary.node=true # Set true only on the primary node, false on others +``` + +### Log4j Integration + +`config/connect-log4j.properties` has switched `connectAppender` to `com.automq.log.uploader.S3RollingFileAppender` and specifies `org.apache.kafka.connect.automq.ConnectS3LogConfigProvider` as the config provider. As long as you enable `log.s3.enable=true` and configure the bucket info in the worker config, log upload will be automatically initialized with the Connect process; if not set or returns `log.s3.enable=false`, the uploader remains disabled. + +## Programmatic Usage + +### 1. Initialize Telemetry Manager + +```java +import com.automq.opentelemetry.AutoMQTelemetryManager; +import java.util.Properties; + +// Initialize AutoMQ telemetry before starting Kafka Connect +Properties telemetryProps = new Properties(); +telemetryProps.setProperty("automq.telemetry.exporter.uri", "prometheus://localhost:9090"); +telemetryProps.setProperty("service.name", "kafka-connect"); +telemetryProps.setProperty("service.instance.id", "worker-1"); + +// Initialize singleton instance +AutoMQTelemetryManager.initializeInstance(telemetryProps); + +// Now start Kafka Connect - it will automatically use the OpenTelemetryMetricsReporter +``` + +### 2. Shutdown + +```java +// When shutting down your application +AutoMQTelemetryManager.shutdownInstance(); +``` + +## Exported Metrics + +The integration automatically converts Kafka Connect metrics to OpenTelemetry format: + +### Metric Naming Convention +- **Format**: `kafka.connect.{group}.{metric_name}` +- **Example**: `kafka.connect.connector.task.batch.size.avg` → `kafka.connect.connector_task_batch_size_avg` + +### Metric Types +- **Counters**: Metrics containing "total", "count", "error", "failure" +- **Gauges**: All other numeric metrics (rates, averages, sizes, etc.) + +### Attributes +Kafka metric tags are converted to OpenTelemetry attributes: +- `connector` → `connector` +- `task` → `task` +- `worker-id` → `worker_id` +- Plus standard attributes: `metric.group`, `service.name`, `service.instance.id` + +## Example Metrics + +Common Kafka Connect metrics that will be exported: + +``` +# Connector metrics +kafka.connect.connector.startup.attempts.total +kafka.connect.connector.startup.success.total +kafka.connect.connector.startup.failure.total + +# Task metrics +kafka.connect.connector.task.batch.size.avg +kafka.connect.connector.task.batch.size.max +kafka.connect.connector.task.offset.commit.avg.time.ms + +# Worker metrics +kafka.connect.worker.connector.count +kafka.connect.worker.task.count +kafka.connect.worker.connector.startup.attempts.total +``` + +## Configuration Options + +### OpenTelemetry MetricsReporter Options + +| Property | Description | Default | Example | +|----------|-------------|---------|---------| +| `opentelemetry.metrics.enabled` | Enable/disable metrics export | `true` | `false` | +| `opentelemetry.metrics.prefix` | Metric name prefix | `kafka.connect` | `my.connect` | +| `opentelemetry.metrics.include.pattern` | Regex for included metrics | All metrics | `.*connector.*` | +| `opentelemetry.metrics.exclude.pattern` | Regex for excluded metrics | None | `.*jmx.*` | + +### AutoMQ Telemetry Options + +| Property | Description | Default | +|----------|-------------|---------| +| `automq.telemetry.exporter.uri` | Exporter endpoint | Empty | +| `automq.telemetry.exporter.interval.ms` | Export interval | `60000` | +| `automq.telemetry.metric.cardinality.limit` | Max metric cardinality | `20000` | + +## Monitoring Examples + +### Prometheus Queries + +```promql +# Connector count by worker +kafka_connect_worker_connector_count + +# Task failure rate +rate(kafka_connect_connector_task_startup_failure_total[5m]) + +# Average batch processing time +kafka_connect_connector_task_batch_size_avg + +# Connector startup success rate +rate(kafka_connect_connector_startup_success_total[5m]) / +rate(kafka_connect_connector_startup_attempts_total[5m]) +``` + +### Grafana Dashboard + +Common panels to create: + +1. **Connector Health**: Count of running/failed connectors +2. **Task Performance**: Batch size, processing time, throughput +3. **Error Rates**: Failed startups, task failures +4. **Resource Usage**: Combined with JVM metrics from AutoMQ telemetry + +## Troubleshooting + +### Common Issues + +1. **Metrics not appearing** + ``` + Check logs for: "AutoMQTelemetryManager is not initialized" + Solution: Ensure AutoMQTelemetryManager.initializeInstance() is called before Connect starts + ``` + +2. **High cardinality warnings** + ``` + Solution: Use include/exclude patterns to filter metrics + ``` + +3. **Missing dependencies** + ``` + Ensure connect-runtime depends on the opentelemetry module + ``` + +### Debug Logging + +Enable debug logging to troubleshoot: + +```properties +log4j.logger.org.apache.kafka.connect.automq=DEBUG +log4j.logger.com.automq.opentelemetry=DEBUG +``` + +## Integration with Existing Monitoring + +This integration works alongside: +- Existing JMX metrics (not replaced) +- Kafka broker metrics via AutoMQ telemetry +- Application-specific metrics +- Third-party monitoring tools + +The OpenTelemetry integration provides a unified export path while preserving existing monitoring setups. diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/automq/ConnectLogUploader.java b/connect/runtime/src/main/java/org/apache/kafka/connect/automq/ConnectLogUploader.java new file mode 100644 index 0000000000..fb409cfe11 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/automq/ConnectLogUploader.java @@ -0,0 +1,33 @@ +package org.apache.kafka.connect.automq; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.Properties; + +/** + * Initializes the AutoMQ S3 log uploader for Kafka Connect. + */ +public final class ConnectLogUploader { + private static Logger getLogger() { + return LoggerFactory.getLogger(ConnectLogUploader.class); + } + + private ConnectLogUploader() { + } + + public static void initialize(Map workerProps) { + Properties props = new Properties(); + if (workerProps != null) { + workerProps.forEach((k, v) -> { + if (k != null && v != null) { + props.put(k, v); + } + }); + } + ConnectS3LogConfigProvider.initialize(props); + com.automq.log.uploader.S3RollingFileAppender.triggerInitialization(); + getLogger().info("Initialized Connect S3 log uploader context"); + } +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/automq/ConnectS3LogConfigProvider.java b/connect/runtime/src/main/java/org/apache/kafka/connect/automq/ConnectS3LogConfigProvider.java new file mode 100644 index 0000000000..15d77c3e81 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/automq/ConnectS3LogConfigProvider.java @@ -0,0 +1,174 @@ +package org.apache.kafka.connect.automq; + +import com.automq.log.uploader.DefaultS3LogConfig; +import com.automq.log.uploader.LogConfigConstants; +import com.automq.log.uploader.S3LogConfig; +import com.automq.log.uploader.S3LogConfigProvider; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.InetAddress; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Provides S3 log uploader configuration for Kafka Connect workers. + */ +public class ConnectS3LogConfigProvider implements S3LogConfigProvider { + private static Logger getLogger() { + return LoggerFactory.getLogger(ConnectS3LogConfigProvider.class); + } + private static final AtomicReference CONFIG = new AtomicReference<>(); + private static final long WAIT_TIMEOUT_MS = TimeUnit.SECONDS.toMillis(10); + private static final CountDownLatch INIT = new CountDownLatch(1); + + public static void initialize(Properties workerProps) { + try { + if (workerProps == null) { + CONFIG.set(null); + return; + } + Properties copy = new Properties(); + for (Map.Entry entry : workerProps.entrySet()) { + if (entry.getKey() != null && entry.getValue() != null) { + copy.put(entry.getKey(), entry.getValue()); + } + } + CONFIG.set(copy); + } finally { + INIT.countDown(); + } + getLogger().info("Initializing ConnectS3LogConfigProvider"); + } + + @Override + public S3LogConfig get() { + + try { + if (!INIT.await(WAIT_TIMEOUT_MS, TimeUnit.MILLISECONDS)) { + getLogger().warn("S3 log uploader config not initialized within timeout; uploader disabled."); + } + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + getLogger().warn("Interrupted while waiting for S3 log uploader config; uploader disabled."); + return null; + } + + Properties source = CONFIG.get(); + if (source == null) { + getLogger().warn("S3 log upload configuration was not provided; uploader disabled."); + return null; + } + + Properties effective = buildEffectiveProperties(source); + if (!Boolean.parseBoolean(effective.getProperty(LogConfigConstants.LOG_S3_ENABLE_KEY, "false"))) { + getLogger().info("S3 log uploader is disabled via {}", LogConfigConstants.LOG_S3_ENABLE_KEY); + return null; + } + return new DefaultS3LogConfig(effective); + } + + private Properties buildEffectiveProperties(Properties workerProps) { + Properties effective = new Properties(); + workerProps.forEach((k, v) -> effective.put(String.valueOf(k), String.valueOf(v))); + + copyIfPresent(workerProps, "automq.log.s3.bucket", effective, LogConfigConstants.LOG_S3_BUCKET_KEY); + copyIfPresent(workerProps, "automq.log.s3.enable", effective, LogConfigConstants.LOG_S3_ENABLE_KEY); + copyIfPresent(workerProps, "automq.log.s3.region", effective, LogConfigConstants.LOG_S3_REGION_KEY); + copyIfPresent(workerProps, "automq.log.s3.endpoint", effective, LogConfigConstants.LOG_S3_ENDPOINT_KEY); + copyIfPresent(workerProps, "automq.log.s3.access.key", effective, LogConfigConstants.LOG_S3_ACCESS_KEY); + copyIfPresent(workerProps, "automq.log.s3.secret.key", effective, LogConfigConstants.LOG_S3_SECRET_KEY); + copyIfPresent(workerProps, "automq.log.s3.primary.node", effective, LogConfigConstants.LOG_S3_PRIMARY_NODE_KEY); + copyIfPresent(workerProps, "automq.log.s3.selector.type", effective, LogConfigConstants.LOG_S3_SELECTOR_TYPE_KEY); + copyIfPresent(workerProps, "automq.log.s3.selector.primary.node.id", effective, LogConfigConstants.LOG_S3_SELECTOR_PRIMARY_NODE_ID_KEY); + + // Default cluster ID + if (!effective.containsKey(LogConfigConstants.LOG_S3_CLUSTER_ID_KEY)) { + String groupId = workerProps.getProperty("group.id", LogConfigConstants.DEFAULT_LOG_S3_CLUSTER_ID); + effective.setProperty(LogConfigConstants.LOG_S3_CLUSTER_ID_KEY, groupId); + } + + // Default node ID + if (!effective.containsKey(LogConfigConstants.LOG_S3_NODE_ID_KEY)) { + String nodeId = resolveNodeId(workerProps); + effective.setProperty(LogConfigConstants.LOG_S3_NODE_ID_KEY, nodeId); + } + + // Selector defaults + if (!effective.containsKey(LogConfigConstants.LOG_S3_SELECTOR_TYPE_KEY)) { + effective.setProperty(LogConfigConstants.LOG_S3_SELECTOR_TYPE_KEY, "kafka"); + } + + String selectorPrefix = LogConfigConstants.LOG_S3_SELECTOR_PREFIX; + String bootstrapKey = selectorPrefix + "kafka.bootstrap.servers"; + if (!effective.containsKey(bootstrapKey)) { + String bootstrap = workerProps.getProperty("automq.log.s3.selector.kafka.bootstrap.servers", + workerProps.getProperty("bootstrap.servers")); + if (!isBlank(bootstrap)) { + effective.setProperty(bootstrapKey, bootstrap); + } + } + + String clusterId = effective.getProperty(LogConfigConstants.LOG_S3_CLUSTER_ID_KEY, "connect"); + String groupKey = selectorPrefix + "kafka.group.id"; + if (!effective.containsKey(groupKey)) { + effective.setProperty(groupKey, "automq-log-uploader-" + clusterId); + } + + String topicKey = selectorPrefix + "kafka.topic"; + if (!effective.containsKey(topicKey)) { + effective.setProperty(topicKey, "__automq_connect_log_leader_" + clusterId.replaceAll("[^A-Za-z0-9_-]", "")); + } + + String clientKey = selectorPrefix + "kafka.client.id"; + if (!effective.containsKey(clientKey)) { + effective.setProperty(clientKey, "automq-log-uploader-client-" + effective.getProperty(LogConfigConstants.LOG_S3_NODE_ID_KEY)); + } + + String autoCreateKey = selectorPrefix + "kafka.auto.create.topic"; + effective.putIfAbsent(autoCreateKey, "true"); + + // Map any existing selector.* overrides from worker props + for (String name : workerProps.stringPropertyNames()) { + if (name.startsWith(selectorPrefix)) { + effective.setProperty(name, workerProps.getProperty(name)); + } + } + + return effective; + } + + private void copyIfPresent(Properties src, String srcKey, Properties dest, String destKey) { + String value = src.getProperty(srcKey); + if (!isBlank(value)) { + dest.setProperty(destKey, value.trim()); + } + } + + private String resolveNodeId(Properties workerProps) { + String fromConfig = workerProps.getProperty(LogConfigConstants.LOG_S3_NODE_ID_KEY); + if (!isBlank(fromConfig)) { + return fromConfig.trim(); + } + String env = System.getenv("CONNECT_NODE_ID"); + if (!isBlank(env)) { + return env.trim(); + } + String host = workerProps.getProperty("automq.log.s3.node.hostname"); + if (isBlank(host)) { + try { + host = InetAddress.getLocalHost().getHostName(); + } catch (Exception e) { + host = System.getenv().getOrDefault("HOSTNAME", "0"); + } + } + return Integer.toString(Math.abs(host.hashCode())); + } + + private boolean isBlank(String value) { + return value == null || value.trim().isEmpty(); + } +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/automq/MetricsIntegrate.java b/connect/runtime/src/main/java/org/apache/kafka/connect/automq/MetricsIntegrate.java new file mode 100644 index 0000000000..f2b57adb60 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/automq/MetricsIntegrate.java @@ -0,0 +1,12 @@ +package org.apache.kafka.connect.automq; + +import com.automq.opentelemetry.AutoMQTelemetryManager; + +public class MetricsIntegrate { + + AutoMQTelemetryManager autoMQTelemetryManager; + + public MetricsIntegrate(AutoMQTelemetryManager autoMQTelemetryManager) { + this.autoMQTelemetryManager = autoMQTelemetryManager; + } +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/automq/OpenTelemetryMetricsReporter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/automq/OpenTelemetryMetricsReporter.java new file mode 100644 index 0000000000..2b91aa9151 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/automq/OpenTelemetryMetricsReporter.java @@ -0,0 +1,352 @@ +/* + * Copyright 2025, AutoMQ HK Limited. + * + * 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.kafka.connect.automq; + +import com.automq.opentelemetry.AutoMQTelemetryManager; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.common.AttributesBuilder; +import io.opentelemetry.api.metrics.DoubleGauge; +import io.opentelemetry.api.metrics.LongCounter; +import io.opentelemetry.api.metrics.Meter; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.metrics.MetricsReporter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.ConcurrentHashMap; + +/** + * A MetricsReporter implementation that bridges Kafka Connect metrics to OpenTelemetry. + * + *

This reporter integrates with the AutoMQ OpenTelemetry module to export Kafka Connect + * metrics through various exporters (Prometheus, OTLP, etc.). It automatically converts + * Kafka metrics to OpenTelemetry instruments based on metric types and provides proper + * labeling and naming conventions. + * + *

Key features: + *

    + *
  • Automatic metric type detection and conversion
  • + *
  • Support for gauges and counters
  • + *
  • Proper attribute mapping from Kafka metric tags
  • + *
  • Integration with AutoMQ telemetry infrastructure
  • + *
  • Configurable metric filtering
  • + *
+ * + *

Configuration options: + *

    + *
  • {@code opentelemetry.metrics.enabled} - Enable/disable OpenTelemetry metrics (default: true)
  • + *
  • {@code opentelemetry.metrics.prefix} - Prefix for metric names (default: "kafka.connect")
  • + *
  • {@code opentelemetry.metrics.include.pattern} - Regex pattern for included metrics
  • + *
  • {@code opentelemetry.metrics.exclude.pattern} - Regex pattern for excluded metrics
  • + *
+ */ +public class OpenTelemetryMetricsReporter implements MetricsReporter { + private static final Logger LOGGER = LoggerFactory.getLogger(OpenTelemetryMetricsReporter.class); + + private static final String ENABLED_CONFIG = "opentelemetry.metrics.enabled"; + private static final String PREFIX_CONFIG = "opentelemetry.metrics.prefix"; + private static final String INCLUDE_PATTERN_CONFIG = "opentelemetry.metrics.include.pattern"; + private static final String EXCLUDE_PATTERN_CONFIG = "opentelemetry.metrics.exclude.pattern"; + + private static final String DEFAULT_PREFIX = "kafka.connect"; + + private boolean enabled = true; + private String metricPrefix = DEFAULT_PREFIX; + private String includePattern = null; + private String excludePattern = null; + + private Meter meter; + private final Map gauges = new ConcurrentHashMap<>(); + private final Map counters = new ConcurrentHashMap<>(); + private final Map lastValues = new ConcurrentHashMap<>(); + + public static void initializeTelemetry(Properties props) { + AutoMQTelemetryManager.initializeInstance(props); + LOGGER.info("OpenTelemetryMetricsReporter initialized"); + } + + @Override + public void configure(Map configs) { + // Parse configuration + Object enabledObj = configs.get(ENABLED_CONFIG); + if (enabledObj != null) { + enabled = Boolean.parseBoolean(enabledObj.toString()); + } + + Object prefixObj = configs.get(PREFIX_CONFIG); + if (prefixObj != null) { + metricPrefix = prefixObj.toString(); + } + + Object includeObj = configs.get(INCLUDE_PATTERN_CONFIG); + if (includeObj != null) { + includePattern = includeObj.toString(); + } + + Object excludeObj = configs.get(EXCLUDE_PATTERN_CONFIG); + if (excludeObj != null) { + excludePattern = excludeObj.toString(); + } + + LOGGER.info("OpenTelemetryMetricsReporter configured - enabled: {}, prefix: {}, include: {}, exclude: {}", + enabled, metricPrefix, includePattern, excludePattern); + } + + @Override + public void init(List metrics) { + if (!enabled) { + LOGGER.info("OpenTelemetryMetricsReporter is disabled"); + return; + } + + try { + // Get the OpenTelemetry meter from AutoMQTelemetryManager + // This assumes the telemetry manager is already initialized + meter = AutoMQTelemetryManager.getInstance().getMeter(); + if (meter == null) { + LOGGER.warn("AutoMQTelemetryManager is not initialized, OpenTelemetry metrics will not be available"); + enabled = false; + return; + } + + // Register initial metrics + for (KafkaMetric metric : metrics) { + registerMetric(metric); + } + + LOGGER.info("OpenTelemetryMetricsReporter initialized with {} metrics", metrics.size()); + } catch (Exception e) { + LOGGER.error("Failed to initialize OpenTelemetryMetricsReporter", e); + enabled = false; + } + } + + @Override + public void metricChange(KafkaMetric metric) { + if (!enabled || meter == null) { + return; + } + + try { + registerMetric(metric); + } catch (Exception e) { + LOGGER.warn("Failed to register metric change for {}", metric.metricName(), e); + } + } + + @Override + public void metricRemoval(KafkaMetric metric) { + if (!enabled) { + return; + } + + try { + String metricKey = buildMetricKey(metric.metricName()); + gauges.remove(metricKey); + counters.remove(metricKey); + lastValues.remove(metricKey); + LOGGER.debug("Removed metric: {}", metricKey); + } catch (Exception e) { + LOGGER.warn("Failed to remove metric {}", metric.metricName(), e); + } + } + + @Override + public void close() { + LOGGER.info("OpenTelemetryMetricsReporter closed"); + } + + private void registerMetric(KafkaMetric metric) { + LOGGER.info("OpenTelemetryMetricsReporter Registering metric {}", metric.metricName()); + MetricName metricName = metric.metricName(); + String metricKey = buildMetricKey(metricName); + + // Apply filtering + if (!shouldIncludeMetric(metricKey)) { + return; + } + + Object value = metric.metricValue(); + if (!(value instanceof Number)) { + LOGGER.debug("Skipping non-numeric metric: {}", metricKey); + return; + } + + double numericValue = ((Number) value).doubleValue(); + Attributes attributes = buildAttributes(metricName); + + // Determine metric type and register accordingly + if (isCounterMetric(metricName)) { + registerCounter(metricKey, metricName, numericValue, attributes); + } else { + registerGauge(metricKey, metricName, numericValue, attributes); + } + } + + private void registerGauge(String metricKey, MetricName metricName, double value, Attributes attributes) { + DoubleGauge gauge = gauges.computeIfAbsent(metricKey, k -> { + String description = buildDescription(metricName); + String unit = determineUnit(metricName); + return meter.gaugeBuilder(metricKey) + .setDescription(description) + .setUnit(unit) + .build(); + }); + + // Record the value + gauge.set(value, attributes); + lastValues.put(metricKey, value); + LOGGER.debug("Updated gauge {} = {}", metricKey, value); + } + + private void registerCounter(String metricKey, MetricName metricName, double value, Attributes attributes) { + LongCounter counter = counters.computeIfAbsent(metricKey, k -> { + String description = buildDescription(metricName); + String unit = determineUnit(metricName); + return meter.counterBuilder(metricKey) + .setDescription(description) + .setUnit(unit) + .build(); + }); + + // For counters, we need to track delta values + Double lastValue = lastValues.get(metricKey); + if (lastValue != null) { + double delta = value - lastValue; + if (delta > 0) { + counter.add((long) delta, attributes); + LOGGER.debug("Counter {} increased by {}", metricKey, delta); + } + } + lastValues.put(metricKey, value); + } + + private String buildMetricKey(MetricName metricName) { + StringBuilder sb = new StringBuilder(metricPrefix); + sb.append("."); + + // Add group if present + if (metricName.group() != null && !metricName.group().isEmpty()) { + sb.append(metricName.group().replace("-", "_").toLowerCase()); + sb.append("."); + } + + // Add name + sb.append(metricName.name().replace("-", "_").toLowerCase()); + + return sb.toString(); + } + + private Attributes buildAttributes(MetricName metricName) { + AttributesBuilder builder = Attributes.builder(); + + // Add metric tags as attributes + Map tags = metricName.tags(); + if (tags != null) { + for (Map.Entry entry : tags.entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + if (key != null && value != null) { + builder.put(sanitizeAttributeKey(key), value); + } + } + } + + // Add standard attributes + if (metricName.group() != null) { + builder.put("metric.group", metricName.group()); + } + + return builder.build(); + } + + private String sanitizeAttributeKey(String key) { + // Replace invalid characters for attribute keys + return key.replace("-", "_").replace(".", "_").toLowerCase(); + } + + private String buildDescription(MetricName metricName) { + StringBuilder description = new StringBuilder(); + description.append("Kafka Connect metric: "); + + if (metricName.group() != null) { + description.append(metricName.group()).append(" - "); + } + + description.append(metricName.name()); + + return description.toString(); + } + + private String determineUnit(MetricName metricName) { + String name = metricName.name().toLowerCase(); + + if (name.contains("time") || name.contains("latency") || name.contains("duration")) { + if (name.contains("ms") || name.contains("millisecond")) { + return "ms"; + } else if (name.contains("ns") || name.contains("nanosecond")) { + return "ns"; + } else { + return "s"; + } + } else if (name.contains("byte") || name.contains("size")) { + return "bytes"; + } else if (name.contains("rate") || name.contains("per-sec")) { + return "1/s"; + } else if (name.contains("percent") || name.contains("ratio")) { + return "%"; + } else if (name.contains("count") || name.contains("total")) { + return "1"; + } + + return "1"; // Default unit + } + + private boolean isCounterMetric(MetricName metricName) { + String name = metricName.name().toLowerCase(); + String group = metricName.group() != null ? metricName.group().toLowerCase() : ""; + + // Identify counter-like metrics + return name.contains("total") || + name.contains("count") || + name.contains("error") || + name.contains("failure") || + name.endsWith("-total") || + group.contains("error"); + } + + private boolean shouldIncludeMetric(String metricKey) { + // Apply exclude pattern first + if (excludePattern != null && metricKey.matches(excludePattern)) { + return false; + } + + // Apply include pattern if specified + if (includePattern != null) { + return metricKey.matches(includePattern); + } + + return true; + } +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/AbstractConnectCli.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/AbstractConnectCli.java index 5cfa300baf..627d2d701d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/AbstractConnectCli.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/AbstractConnectCli.java @@ -19,6 +19,8 @@ import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.automq.ConnectLogUploader; +import org.apache.kafka.connect.automq.OpenTelemetryMetricsReporter; import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy; import org.apache.kafka.connect.runtime.Connect; import org.apache.kafka.connect.runtime.Herder; @@ -36,6 +38,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.Map; +import java.util.Properties; /** * Common initialization logic for Kafka Connect, intended for use by command line utilities @@ -45,7 +48,9 @@ */ public abstract class AbstractConnectCli { - private static final Logger log = LoggerFactory.getLogger(AbstractConnectCli.class); + private static Logger getLogger() { + return LoggerFactory.getLogger(AbstractConnectCli.class); + } private final String[] args; private final Time time = Time.SYSTEM; @@ -83,7 +88,6 @@ protected abstract H createHerder(T config, String workerId, Plugins plugins, */ public void run() { if (args.length < 1 || Arrays.asList(args).contains("--help")) { - log.info("Usage: {}", usage()); Exit.exit(1); } @@ -92,6 +96,14 @@ public void run() { Map workerProps = !workerPropsFile.isEmpty() ? Utils.propsToStringMap(Utils.loadProps(workerPropsFile)) : Collections.emptyMap(); String[] extraArgs = Arrays.copyOfRange(args, 1, args.length); + + // Initialize S3 log uploader and OpenTelemetry with worker properties + ConnectLogUploader.initialize(workerProps); + + Properties telemetryProps = new Properties(); + telemetryProps.putAll(workerProps); + OpenTelemetryMetricsReporter.initializeTelemetry(telemetryProps); + Connect connect = startConnect(workerProps); processExtraArgs(connect, extraArgs); @@ -99,7 +111,7 @@ public void run() { connect.awaitStop(); } catch (Throwable t) { - log.error("Stopping due to error", t); + getLogger().error("Stopping due to error", t); Exit.exit(2); } } @@ -111,17 +123,17 @@ public void run() { * @return a started instance of {@link Connect} */ public Connect startConnect(Map workerProps) { - log.info("Kafka Connect worker initializing ..."); + getLogger().info("Kafka Connect worker initializing ..."); long initStart = time.hiResClockMs(); WorkerInfo initInfo = new WorkerInfo(); initInfo.logAll(); - log.info("Scanning for plugin classes. This might take a moment ..."); + getLogger().info("Scanning for plugin classes. This might take a moment ..."); Plugins plugins = new Plugins(workerProps); plugins.compareAndSwapWithDelegatingLoader(); T config = createConfig(workerProps); - log.debug("Kafka cluster ID: {}", config.kafkaClusterId()); + getLogger().debug("Kafka cluster ID: {}", config.kafkaClusterId()); RestClient restClient = new RestClient(config); @@ -138,11 +150,11 @@ public Connect startConnect(Map workerProps) { H herder = createHerder(config, workerId, plugins, connectorClientConfigOverridePolicy, restServer, restClient); final Connect connect = new Connect<>(herder, restServer); - log.info("Kafka Connect worker initialization took {}ms", time.hiResClockMs() - initStart); + getLogger().info("Kafka Connect worker initialization took {}ms", time.hiResClockMs() - initStart); try { connect.start(); } catch (Exception e) { - log.error("Failed to start Connect", e); + getLogger().error("Failed to start Connect", e); connect.stop(); Exit.exit(3); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/OTelMetricsReporter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/OTelMetricsReporter.java new file mode 100644 index 0000000000..72a0b56fb0 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/OTelMetricsReporter.java @@ -0,0 +1,195 @@ +package org.apache.kafka.connect.runtime; + +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.KafkaMetric; +import org.apache.kafka.common.metrics.MetricsContext; +import org.apache.kafka.common.metrics.MetricsReporter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.common.AttributesBuilder; +import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.api.metrics.ObservableDoubleGauge; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * A Kafka MetricsReporter that bridges Kafka metrics to OpenTelemetry. + * This reporter registers all metrics as observable gauges with OpenTelemetry, + * which will call back to get the latest values when metrics collection occurs. + */ +public class OTelMetricsReporter implements MetricsReporter { + + private static final Logger log = LoggerFactory.getLogger(OTelMetricsReporter.class); + + // Store all metrics for retrieval during OTel callbacks + private final Map metrics = new ConcurrentHashMap<>(); + + // Group metrics by group for easier registration with OTel + private final Map> metricsByGroup = new ConcurrentHashMap<>(); + + // Keep track of registered gauges to prevent duplicate registration + private final Map registeredGauges = new ConcurrentHashMap<>(); + + private Meter meter; + private boolean initialized = false; + + @Override + public void configure(Map configs) { + log.info("Configuring OTelMetricsReporter"); + } + + /** + * Initialize OpenTelemetry meter and register metrics + */ + public void initOpenTelemetry(OpenTelemetry openTelemetry) { + if (initialized) { + return; + } + + this.meter = openTelemetry.getMeter("kafka-connect-metrics"); + log.info("OTelMetricsReporter initialized with OpenTelemetry meter"); + + // Register all metrics that were already added before OpenTelemetry was initialized + registerMetricsWithOTel(); + + initialized = true; + } + + @Override + public void init(List metrics) { + log.info("Initializing OTelMetricsReporter with {} metrics", metrics.size()); + for (KafkaMetric metric : metrics) { + addMetricToCollections(metric); + } + + // If meter is already available, register metrics + if (meter != null) { + registerMetricsWithOTel(); + } + } + + private void addMetricToCollections(KafkaMetric metric) { + MetricName metricName = metric.metricName(); + metrics.put(metricName, metric); + + // Group by metric group + metricsByGroup + .computeIfAbsent(metricName.group(), k -> new ConcurrentHashMap<>()) + .put(metricName, metric); + } + + private void registerMetricsWithOTel() { + if (meter == null) { + log.warn("Cannot register metrics with OpenTelemetry - meter not initialized"); + return; + } + + // Register each group of metrics as an observable gauge collection + for (Map.Entry> entry : metricsByGroup.entrySet()) { + String group = entry.getKey(); + Map groupMetrics = entry.getValue(); + + // Register the gauge for this group if not already registered + String gaugeKey = "kafka.connect." + group; + if (!registeredGauges.containsKey(gaugeKey)) { + ObservableDoubleGauge gauge = meter + .gaugeBuilder(gaugeKey) + .setDescription("Kafka Connect metrics for " + group) + .setUnit("1") // Default unit + .buildWithCallback(measurement -> { + // Get the latest values for all metrics in this group + Map currentGroupMetrics = metricsByGroup.get(group); + if (currentGroupMetrics != null) { + for (Map.Entry metricEntry : currentGroupMetrics.entrySet()) { + MetricName name = metricEntry.getKey(); + KafkaMetric kafkaMetric = metricEntry.getValue(); + + try { + // Convert metric value to double + double value = convertToDouble(kafkaMetric.metricValue()); + + // Build attributes from metric tags + AttributesBuilder attributes = Attributes.builder(); + attributes.put("name", name.name()); + + // Add all tags as attributes + for (Map.Entry tag : name.tags().entrySet()) { + attributes.put(tag.getKey(), tag.getValue()); + } + + // Record the measurement + measurement.record(value, attributes.build()); + } catch (Exception e) { + log.warn("Error recording metric {}: {}", name, e.getMessage()); + } + } + } + }); + + registeredGauges.put(gaugeKey, gauge); + log.info("Registered gauge for metric group: {}", group); + } + } + } + + private double convertToDouble(Object value) { + if (value == null) { + return 0.0; + } + + if (value instanceof Number) { + return ((Number) value).doubleValue(); + } + + if (value instanceof Boolean) { + return ((Boolean) value) ? 1.0 : 0.0; + } + + return 0.0; + } + + @Override + public void metricChange(KafkaMetric metric) { + addMetricToCollections(metric); + + // If already initialized with OTel, register new metrics + if (meter != null && !registeredGauges.containsKey("kafka.connect." + metric.metricName().group())) { + registerMetricsWithOTel(); + } + } + + @Override + public void metricRemoval(KafkaMetric metric) { + MetricName metricName = metric.metricName(); + metrics.remove(metricName); + + Map groupMetrics = metricsByGroup.get(metricName.group()); + if (groupMetrics != null) { + groupMetrics.remove(metricName); + if (groupMetrics.isEmpty()) { + metricsByGroup.remove(metricName.group()); + } + } + + log.debug("Removed metric: {}", metricName); + } + + @Override + public void close() { + log.info("Closing OTelMetricsReporter"); + metrics.clear(); + metricsByGroup.clear(); + registeredGauges.clear(); + } + + @Override + public void contextChange(MetricsContext metricsContext) { + // Add context labels as attributes if needed + log.info("Metrics context changed: {}", metricsContext.contextLabels()); + } +} diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 1be9de6d03..119fecec50 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -178,7 +178,7 @@ versions += [ jna:"5.2.0", guava:"32.0.1-jre", hdrHistogram:"2.1.12", - nettyTcnativeBoringSsl: "2.0.65.Final", + nettyTcnativeBoringSsl: "2.0.69.Final", avro: "1.11.4", confluentSchema: "7.8.0", iceberg: "1.6.1", diff --git a/gradle/spotbugs-exclude.xml b/gradle/spotbugs-exclude.xml index 310d9902d9..2d804620be 100644 --- a/gradle/spotbugs-exclude.xml +++ b/gradle/spotbugs-exclude.xml @@ -601,7 +601,8 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read - + + |Gauge, Sum, Histogram, Summary, ... | +// +------------+ +------------------------------------+ +// +// Data [One of Gauge, Sum, Histogram, Summary, ...] +// +-----------+ +// |... | // Metadata about the Data. +// |points |--+ +// +-----------+ | +// | +---------------------------+ +// | |DataPoint 1 | +// v |+------+------+ +------+ | +// +-----+ ||label |label |...|label | | +// | 1 |-->||value1|value2|...|valueN| | +// +-----+ |+------+------+ +------+ | +// | . | |+-----+ | +// | . | ||value| | +// | . | |+-----+ | +// | . | +---------------------------+ +// | . | . +// | . | . +// | . | . +// | . | +---------------------------+ +// | . | |DataPoint M | +// +-----+ |+------+------+ +------+ | +// | M |-->||label |label |...|label | | +// +-----+ ||value1|value2|...|valueN| | +// |+------+------+ +------+ | +// |+-----+ | +// ||value| | +// |+-----+ | +// +---------------------------+ +// +// Each distinct type of DataPoint represents the output of a specific +// aggregation function, the result of applying the DataPoint's +// associated function of to one or more measurements. +// +// All DataPoint types have three common fields: +// - Attributes includes key-value pairs associated with the data point +// - TimeUnixNano is required, set to the end time of the aggregation +// - StartTimeUnixNano is optional, but strongly encouraged for DataPoints +// having an AggregationTemporality field, as discussed below. +// +// Both TimeUnixNano and StartTimeUnixNano values are expressed as +// UNIX Epoch time in nanoseconds since 00:00:00 UTC on 1 January 1970. +// +// # TimeUnixNano +// +// This field is required, having consistent interpretation across +// DataPoint types. TimeUnixNano is the moment corresponding to when +// the data point's aggregate value was captured. +// +// Data points with the 0 value for TimeUnixNano SHOULD be rejected +// by consumers. +// +// # StartTimeUnixNano +// +// StartTimeUnixNano in general allows detecting when a sequence of +// observations is unbroken. This field indicates to consumers the +// start time for points with cumulative and delta +// AggregationTemporality, and it should be included whenever possible +// to support correct rate calculation. Although it may be omitted +// when the start time is truly unknown, setting StartTimeUnixNano is +// strongly encouraged. +message Metric { + reserved 4, 6, 8; + + // name of the metric. + string name = 1; + + // description of the metric, which can be used in documentation. + string description = 2; + + // unit in which the metric value is reported. Follows the format + // described by http://unitsofmeasure.org/ucum.html. + string unit = 3; + + // Data determines the aggregation type (if any) of the metric, what is the + // reported value type for the data points, as well as the relatationship to + // the time interval over which they are reported. + oneof data { + Gauge gauge = 5; + Sum sum = 7; + Histogram histogram = 9; + ExponentialHistogram exponential_histogram = 10; + Summary summary = 11; + } + + // Additional metadata attributes that describe the metric. [Optional]. + // Attributes are non-identifying. + // Consumers SHOULD NOT need to be aware of these attributes. + // These attributes MAY be used to encode information allowing + // for lossless roundtrip translation to / from another data model. + // Attribute keys MUST be unique (it is not allowed to have more than one + // attribute with the same key). + repeated com.automq.opentelemetry.telemetry.proto.common.v1.KeyValue metadata = 12; +} + +// Gauge represents the type of a scalar metric that always exports the +// "current value" for every data point. It should be used for an "unknown" +// aggregation. +// +// A Gauge does not support different aggregation temporalities. Given the +// aggregation is unknown, points cannot be combined using the same +// aggregation, regardless of aggregation temporalities. Therefore, +// AggregationTemporality is not included. Consequently, this also means +// "StartTimeUnixNano" is ignored for all data points. +message Gauge { + repeated NumberDataPoint data_points = 1; +} + +// Sum represents the type of a scalar metric that is calculated as a sum of all +// reported measurements over a time interval. +message Sum { + repeated NumberDataPoint data_points = 1; + + // aggregation_temporality describes if the aggregator reports delta changes + // since last report time, or cumulative changes since a fixed start time. + AggregationTemporality aggregation_temporality = 2; + + // If "true" means that the sum is monotonic. + bool is_monotonic = 3; +} + +// Histogram represents the type of a metric that is calculated by aggregating +// as a Histogram of all reported measurements over a time interval. +message Histogram { + repeated HistogramDataPoint data_points = 1; + + // aggregation_temporality describes if the aggregator reports delta changes + // since last report time, or cumulative changes since a fixed start time. + AggregationTemporality aggregation_temporality = 2; +} + +// ExponentialHistogram represents the type of a metric that is calculated by aggregating +// as a ExponentialHistogram of all reported double measurements over a time interval. +message ExponentialHistogram { + repeated ExponentialHistogramDataPoint data_points = 1; + + // aggregation_temporality describes if the aggregator reports delta changes + // since last report time, or cumulative changes since a fixed start time. + AggregationTemporality aggregation_temporality = 2; +} + +// Summary metric data are used to convey quantile summaries, +// a Prometheus (see: https://prometheus.io/docs/concepts/metric_types/#summary) +// and OpenMetrics (see: https://github.com/OpenObservability/OpenMetrics/blob/4dbf6075567ab43296eed941037c12951faafb92/protos/prometheus.proto#L45) +// data type. These data points cannot always be merged in a meaningful way. +// While they can be useful in some applications, histogram data points are +// recommended for new applications. +// Summary metrics do not have an aggregation temporality field. This is +// because the count and sum fields of a SummaryDataPoint are assumed to be +// cumulative values. +message Summary { + repeated SummaryDataPoint data_points = 1; +} + +// AggregationTemporality defines how a metric aggregator reports aggregated +// values. It describes how those values relate to the time interval over +// which they are aggregated. +enum AggregationTemporality { + // UNSPECIFIED is the default AggregationTemporality, it MUST not be used. + AGGREGATION_TEMPORALITY_UNSPECIFIED = 0; + + // DELTA is an AggregationTemporality for a metric aggregator which reports + // changes since last report time. Successive metrics contain aggregation of + // values from continuous and non-overlapping intervals. + // + // The values for a DELTA metric are based only on the time interval + // associated with one measurement cycle. There is no dependency on + // previous measurements like is the case for CUMULATIVE metrics. + // + // For example, consider a system measuring the number of requests that + // it receives and reports the sum of these requests every second as a + // DELTA metric: + // + // 1. The system starts receiving at time=t_0. + // 2. A request is received, the system measures 1 request. + // 3. A request is received, the system measures 1 request. + // 4. A request is received, the system measures 1 request. + // 5. The 1 second collection cycle ends. A metric is exported for the + // number of requests received over the interval of time t_0 to + // t_0+1 with a value of 3. + // 6. A request is received, the system measures 1 request. + // 7. A request is received, the system measures 1 request. + // 8. The 1 second collection cycle ends. A metric is exported for the + // number of requests received over the interval of time t_0+1 to + // t_0+2 with a value of 2. + AGGREGATION_TEMPORALITY_DELTA = 1; + + // CUMULATIVE is an AggregationTemporality for a metric aggregator which + // reports changes since a fixed start time. This means that current values + // of a CUMULATIVE metric depend on all previous measurements since the + // start time. Because of this, the sender is required to retain this state + // in some form. If this state is lost or invalidated, the CUMULATIVE metric + // values MUST be reset and a new fixed start time following the last + // reported measurement time sent MUST be used. + // + // For example, consider a system measuring the number of requests that + // it receives and reports the sum of these requests every second as a + // CUMULATIVE metric: + // + // 1. The system starts receiving at time=t_0. + // 2. A request is received, the system measures 1 request. + // 3. A request is received, the system measures 1 request. + // 4. A request is received, the system measures 1 request. + // 5. The 1 second collection cycle ends. A metric is exported for the + // number of requests received over the interval of time t_0 to + // t_0+1 with a value of 3. + // 6. A request is received, the system measures 1 request. + // 7. A request is received, the system measures 1 request. + // 8. The 1 second collection cycle ends. A metric is exported for the + // number of requests received over the interval of time t_0 to + // t_0+2 with a value of 5. + // 9. The system experiences a fault and loses state. + // 10. The system recovers and resumes receiving at time=t_1. + // 11. A request is received, the system measures 1 request. + // 12. The 1 second collection cycle ends. A metric is exported for the + // number of requests received over the interval of time t_1 to + // t_0+1 with a value of 1. + // + // Note: Even though, when reporting changes since last report time, using + // CUMULATIVE is valid, it is not recommended. This may cause problems for + // systems that do not use start_time to determine when the aggregation + // value was reset (e.g. Prometheus). + AGGREGATION_TEMPORALITY_CUMULATIVE = 2; +} + +// DataPointFlags is defined as a protobuf 'uint32' type and is to be used as a +// bit-field representing 32 distinct boolean flags. Each flag defined in this +// enum is a bit-mask. To test the presence of a single flag in the flags of +// a data point, for example, use an expression like: +// +// (point.flags & DATA_POINT_FLAGS_NO_RECORDED_VALUE_MASK) == DATA_POINT_FLAGS_NO_RECORDED_VALUE_MASK +// +enum DataPointFlags { + // The zero value for the enum. Should not be used for comparisons. + // Instead use bitwise "and" with the appropriate mask as shown above. + DATA_POINT_FLAGS_DO_NOT_USE = 0; + + // This DataPoint is valid but has no recorded value. This value + // SHOULD be used to reflect explicitly missing data in a series, as + // for an equivalent to the Prometheus "staleness marker". + DATA_POINT_FLAGS_NO_RECORDED_VALUE_MASK = 1; + + // Bits 2-31 are reserved for future use. +} + +// NumberDataPoint is a single data point in a timeseries that describes the +// time-varying scalar value of a metric. +message NumberDataPoint { + reserved 1; + + // The set of key/value pairs that uniquely identify the timeseries from + // where this point belongs. The list may be empty (may contain 0 elements). + // Attribute keys MUST be unique (it is not allowed to have more than one + // attribute with the same key). + repeated com.automq.opentelemetry.telemetry.proto.common.v1.KeyValue attributes = 7; + + // StartTimeUnixNano is optional but strongly encouraged, see the + // the detailed comments above Metric. + // + // Value is UNIX Epoch time in nanoseconds since 00:00:00 UTC on 1 January + // 1970. + fixed64 start_time_unix_nano = 2; + + // TimeUnixNano is required, see the detailed comments above Metric. + // + // Value is UNIX Epoch time in nanoseconds since 00:00:00 UTC on 1 January + // 1970. + fixed64 time_unix_nano = 3; + + // The value itself. A point is considered invalid when one of the recognized + // value fields is not present inside this oneof. + oneof value { + double as_double = 4; + sfixed64 as_int = 6; + } + + // (Optional) List of exemplars collected from + // measurements that were used to form the data point + repeated Exemplar exemplars = 5; + + // Flags that apply to this specific data point. See DataPointFlags + // for the available flags and their meaning. + uint32 flags = 8; +} + +// HistogramDataPoint is a single data point in a timeseries that describes the +// time-varying values of a Histogram. A Histogram contains summary statistics +// for a population of values, it may optionally contain the distribution of +// those values across a set of buckets. +// +// If the histogram contains the distribution of values, then both +// "explicit_bounds" and "bucket counts" fields must be defined. +// If the histogram does not contain the distribution of values, then both +// "explicit_bounds" and "bucket_counts" must be omitted and only "count" and +// "sum" are known. +message HistogramDataPoint { + reserved 1; + + // The set of key/value pairs that uniquely identify the timeseries from + // where this point belongs. The list may be empty (may contain 0 elements). + // Attribute keys MUST be unique (it is not allowed to have more than one + // attribute with the same key). + repeated com.automq.opentelemetry.telemetry.proto.common.v1.KeyValue attributes = 9; + + // StartTimeUnixNano is optional but strongly encouraged, see the + // the detailed comments above Metric. + // + // Value is UNIX Epoch time in nanoseconds since 00:00:00 UTC on 1 January + // 1970. + fixed64 start_time_unix_nano = 2; + + // TimeUnixNano is required, see the detailed comments above Metric. + // + // Value is UNIX Epoch time in nanoseconds since 00:00:00 UTC on 1 January + // 1970. + fixed64 time_unix_nano = 3; + + // count is the number of values in the population. Must be non-negative. This + // value must be equal to the sum of the "count" fields in buckets if a + // histogram is provided. + fixed64 count = 4; + + // sum of the values in the population. If count is zero then this field + // must be zero. + // + // Note: Sum should only be filled out when measuring non-negative discrete + // events, and is assumed to be monotonic over the values of these events. + // Negative events *can* be recorded, but sum should not be filled out when + // doing so. This is specifically to enforce compatibility w/ OpenMetrics, + // see: https://github.com/OpenObservability/OpenMetrics/blob/main/specification/OpenMetrics.md#histogram + optional double sum = 5; + + // bucket_counts is an optional field contains the count values of histogram + // for each bucket. + // + // The sum of the bucket_counts must equal the value in the count field. + // + // The number of elements in bucket_counts array must be by one greater than + // the number of elements in explicit_bounds array. + repeated fixed64 bucket_counts = 6; + + // explicit_bounds specifies buckets with explicitly defined bounds for values. + // + // The boundaries for bucket at index i are: + // + // (-infinity, explicit_bounds[i]] for i == 0 + // (explicit_bounds[i-1], explicit_bounds[i]] for 0 < i < size(explicit_bounds) + // (explicit_bounds[i-1], +infinity) for i == size(explicit_bounds) + // + // The values in the explicit_bounds array must be strictly increasing. + // + // Histogram buckets are inclusive of their upper boundary, except the last + // bucket where the boundary is at infinity. This format is intentionally + // compatible with the OpenMetrics histogram definition. + repeated double explicit_bounds = 7; + + // (Optional) List of exemplars collected from + // measurements that were used to form the data point + repeated Exemplar exemplars = 8; + + // Flags that apply to this specific data point. See DataPointFlags + // for the available flags and their meaning. + uint32 flags = 10; + + // min is the minimum value over (start_time, end_time]. + optional double min = 11; + + // max is the maximum value over (start_time, end_time]. + optional double max = 12; +} + +// ExponentialHistogramDataPoint is a single data point in a timeseries that describes the +// time-varying values of a ExponentialHistogram of double values. A ExponentialHistogram contains +// summary statistics for a population of values, it may optionally contain the +// distribution of those values across a set of buckets. +// +message ExponentialHistogramDataPoint { + // The set of key/value pairs that uniquely identify the timeseries from + // where this point belongs. The list may be empty (may contain 0 elements). + // Attribute keys MUST be unique (it is not allowed to have more than one + // attribute with the same key). + repeated com.automq.opentelemetry.telemetry.proto.common.v1.KeyValue attributes = 1; + + // StartTimeUnixNano is optional but strongly encouraged, see the + // the detailed comments above Metric. + // + // Value is UNIX Epoch time in nanoseconds since 00:00:00 UTC on 1 January + // 1970. + fixed64 start_time_unix_nano = 2; + + // TimeUnixNano is required, see the detailed comments above Metric. + // + // Value is UNIX Epoch time in nanoseconds since 00:00:00 UTC on 1 January + // 1970. + fixed64 time_unix_nano = 3; + + // count is the number of values in the population. Must be + // non-negative. This value must be equal to the sum of the "bucket_counts" + // values in the positive and negative Buckets plus the "zero_count" field. + fixed64 count = 4; + + // sum of the values in the population. If count is zero then this field + // must be zero. + // + // Note: Sum should only be filled out when measuring non-negative discrete + // events, and is assumed to be monotonic over the values of these events. + // Negative events *can* be recorded, but sum should not be filled out when + // doing so. This is specifically to enforce compatibility w/ OpenMetrics, + // see: https://github.com/OpenObservability/OpenMetrics/blob/main/specification/OpenMetrics.md#histogram + optional double sum = 5; + + // scale describes the resolution of the histogram. Boundaries are + // located at powers of the base, where: + // + // base = (2^(2^-scale)) + // + // The histogram bucket identified by `index`, a signed integer, + // contains values that are greater than (base^index) and + // less than or equal to (base^(index+1)). + // + // The positive and negative ranges of the histogram are expressed + // separately. Negative values are mapped by their absolute value + // into the negative range using the same scale as the positive range. + // + // scale is not restricted by the protocol, as the permissible + // values depend on the range of the data. + sint32 scale = 6; + + // zero_count is the count of values that are either exactly zero or + // within the region considered zero by the instrumentation at the + // tolerated degree of precision. This bucket stores values that + // cannot be expressed using the standard exponential formula as + // well as values that have been rounded to zero. + // + // Implementations MAY consider the zero bucket to have probability + // mass equal to (zero_count / count). + fixed64 zero_count = 7; + + // positive carries the positive range of exponential bucket counts. + Buckets positive = 8; + + // negative carries the negative range of exponential bucket counts. + Buckets negative = 9; + + // Buckets are a set of bucket counts, encoded in a contiguous array + // of counts. + message Buckets { + // Offset is the bucket index of the first entry in the bucket_counts array. + // + // Note: This uses a varint encoding as a simple form of compression. + sint32 offset = 1; + + // bucket_counts is an array of count values, where bucket_counts[i] carries + // the count of the bucket at index (offset+i). bucket_counts[i] is the count + // of values greater than base^(offset+i) and less than or equal to + // base^(offset+i+1). + // + // Note: By contrast, the explicit HistogramDataPoint uses + // fixed64. This field is expected to have many buckets, + // especially zeros, so uint64 has been selected to ensure + // varint encoding. + repeated uint64 bucket_counts = 2; + } + + // Flags that apply to this specific data point. See DataPointFlags + // for the available flags and their meaning. + uint32 flags = 10; + + // (Optional) List of exemplars collected from + // measurements that were used to form the data point + repeated Exemplar exemplars = 11; + + // min is the minimum value over (start_time, end_time]. + optional double min = 12; + + // max is the maximum value over (start_time, end_time]. + optional double max = 13; + + // ZeroThreshold may be optionally set to convey the width of the zero + // region. Where the zero region is defined as the closed interval + // [-ZeroThreshold, ZeroThreshold]. + // When ZeroThreshold is 0, zero count bucket stores values that cannot be + // expressed using the standard exponential formula as well as values that + // have been rounded to zero. + double zero_threshold = 14; +} + +// SummaryDataPoint is a single data point in a timeseries that describes the +// time-varying values of a Summary metric. The count and sum fields represent +// cumulative values. +message SummaryDataPoint { + reserved 1; + + // The set of key/value pairs that uniquely identify the timeseries from + // where this point belongs. The list may be empty (may contain 0 elements). + // Attribute keys MUST be unique (it is not allowed to have more than one + // attribute with the same key). + repeated com.automq.opentelemetry.telemetry.proto.common.v1.KeyValue attributes = 7; + + // StartTimeUnixNano is optional but strongly encouraged, see the + // the detailed comments above Metric. + // + // Value is UNIX Epoch time in nanoseconds since 00:00:00 UTC on 1 January + // 1970. + fixed64 start_time_unix_nano = 2; + + // TimeUnixNano is required, see the detailed comments above Metric. + // + // Value is UNIX Epoch time in nanoseconds since 00:00:00 UTC on 1 January + // 1970. + fixed64 time_unix_nano = 3; + + // count is the number of values in the population. Must be non-negative. + fixed64 count = 4; + + // sum of the values in the population. If count is zero then this field + // must be zero. + // + // Note: Sum should only be filled out when measuring non-negative discrete + // events, and is assumed to be monotonic over the values of these events. + // Negative events *can* be recorded, but sum should not be filled out when + // doing so. This is specifically to enforce compatibility w/ OpenMetrics, + // see: https://github.com/OpenObservability/OpenMetrics/blob/main/specification/OpenMetrics.md#summary + double sum = 5; + + // Represents the value at a given quantile of a distribution. + // + // To record Min and Max values following conventions are used: + // - The 1.0 quantile is equivalent to the maximum value observed. + // - The 0.0 quantile is equivalent to the minimum value observed. + // + // See the following issue for more context: + // https://github.com/open-telemetry/opentelemetry-proto/issues/125 + message ValueAtQuantile { + // The quantile of a distribution. Must be in the interval + // [0.0, 1.0]. + double quantile = 1; + + // The value at the given quantile of a distribution. + // + // Quantile values must NOT be negative. + double value = 2; + } + + // (Optional) list of values at different quantiles of the distribution calculated + // from the current snapshot. The quantiles must be strictly increasing. + repeated ValueAtQuantile quantile_values = 6; + + // Flags that apply to this specific data point. See DataPointFlags + // for the available flags and their meaning. + uint32 flags = 8; +} + +// A representation of an exemplar, which is a sample input measurement. +// Exemplars also hold information about the environment when the measurement +// was recorded, for example the span and trace ID of the active span when the +// exemplar was recorded. +message Exemplar { + reserved 1; + + // The set of key/value pairs that were filtered out by the aggregator, but + // recorded alongside the original measurement. Only key/value pairs that were + // filtered out by the aggregator should be included + repeated com.automq.opentelemetry.telemetry.proto.common.v1.KeyValue filtered_attributes = 7; + + // time_unix_nano is the exact time when this exemplar was recorded + // + // Value is UNIX Epoch time in nanoseconds since 00:00:00 UTC on 1 January + // 1970. + fixed64 time_unix_nano = 2; + + // The value of the measurement that was recorded. An exemplar is + // considered invalid when one of the recognized value fields is not present + // inside this oneof. + oneof value { + double as_double = 3; + sfixed64 as_int = 6; + } + + // (Optional) Span ID of the exemplar trace. + // span_id may be missing if the measurement is not recorded inside a trace + // or if the trace is not sampled. + bytes span_id = 4; + + // (Optional) Trace ID of the exemplar trace. + // trace_id may be missing if the measurement is not recorded inside a trace + // or if the trace is not sampled. + bytes trace_id = 5; +} diff --git a/opentelemetry/src/main/proto/remote_write.proto b/opentelemetry/src/main/proto/remote_write.proto new file mode 100644 index 0000000000..b7e92fc4d2 --- /dev/null +++ b/opentelemetry/src/main/proto/remote_write.proto @@ -0,0 +1,36 @@ +// Copyright 2016 Prometheus Team +// Licensed 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. + +syntax = "proto3"; + +option java_package = "com.automq.opentelemetry.telemetry"; + +message WriteRequest { + repeated TimeSeries timeseries = 1; +} + +// TimeSeries represents samples and labels for a single time series. +message TimeSeries { + repeated Label labels = 1; + repeated Sample samples = 2; +} + +message Label { + string name = 1; + string value = 2; +} + +message Sample { + double value = 1; + int64 timestamp = 2; +} \ No newline at end of file diff --git a/opentelemetry/src/main/proto/resource/v1/resource.proto b/opentelemetry/src/main/proto/resource/v1/resource.proto new file mode 100644 index 0000000000..e56d453ac3 --- /dev/null +++ b/opentelemetry/src/main/proto/resource/v1/resource.proto @@ -0,0 +1,37 @@ +// Copyright 2019, OpenTelemetry Authors +// +// Licensed 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. + +syntax = "proto3"; + +package com.automq.opentelemetry.telemetry.proto.resource.v1; + +import "common/v1/common.proto"; + +option csharp_namespace = "OpenTelemetry.Proto.Resource.V1"; +option java_multiple_files = true; +option java_package = "com.automq.opentelemetry.telemetry.resource.v1"; +option java_outer_classname = "ResourceProto"; +option go_package = "go.opentelemetry.io/proto/otlp/resource/v1"; + +// Resource information. +message Resource { + // Set of attributes that describe the resource. + // Attribute keys MUST be unique (it is not allowed to have more than one + // attribute with the same key). + repeated com.automq.opentelemetry.telemetry.proto.common.v1.KeyValue attributes = 1; + + // dropped_attributes_count is the number of dropped attributes. If the value is 0, then + // no attributes were dropped. + uint32 dropped_attributes_count = 2; +} diff --git a/opentelemetry/src/main/resources/META-INF/services/com.automq.opentelemetry.exporter.s3.UploaderNodeSelectorProvider b/opentelemetry/src/main/resources/META-INF/services/com.automq.opentelemetry.exporter.s3.UploaderNodeSelectorProvider new file mode 100644 index 0000000000..6d28daa029 --- /dev/null +++ b/opentelemetry/src/main/resources/META-INF/services/com.automq.opentelemetry.exporter.s3.UploaderNodeSelectorProvider @@ -0,0 +1 @@ +com.automq.opentelemetry.exporter.s3.kafka.KafkaLeaderSelectorProvider diff --git a/opentelemetry/src/test/java/com/automq/opentelemetry/TelemetryConfigTest.java b/opentelemetry/src/test/java/com/automq/opentelemetry/TelemetryConfigTest.java new file mode 100644 index 0000000000..e928f6290b --- /dev/null +++ b/opentelemetry/src/test/java/com/automq/opentelemetry/TelemetryConfigTest.java @@ -0,0 +1,29 @@ +package com.automq.opentelemetry; + +import org.junit.jupiter.api.Test; + +import java.util.Map; +import java.util.Properties; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; + +class TelemetryConfigTest { + + @Test + void getPropertiesWithPrefixStripsPrefixAndIgnoresOthers() { + Properties properties = new Properties(); + properties.setProperty("automq.telemetry.s3.selector.type", "kafka"); + properties.setProperty("automq.telemetry.s3.selector.kafka.bootstrap.servers", "localhost:9092"); + properties.setProperty("automq.telemetry.s3.selector.kafka.security.protocol", "SASL_PLAINTEXT"); + properties.setProperty("unrelated.key", "value"); + + TelemetryConfig config = new TelemetryConfig(properties); + Map result = config.getPropertiesWithPrefix("automq.telemetry.s3.selector."); + + assertEquals("kafka", result.get("type")); + assertEquals("localhost:9092", result.get("kafka.bootstrap.servers")); + assertEquals("SASL_PLAINTEXT", result.get("kafka.security.protocol")); + assertFalse(result.containsKey("unrelated.key")); + } +} diff --git a/settings.gradle b/settings.gradle index 3e1b9ba992..998ad87039 100644 --- a/settings.gradle +++ b/settings.gradle @@ -104,7 +104,9 @@ include 'clients', 'transaction-coordinator', 'trogdor', 's3stream', - 'automq-shell' + 'automq-shell', + 'automq-log-uploader', + 'opentelemetry' project(":storage:api").name = "storage-api" rootProject.name = 'kafka' diff --git a/tests/kafkatest/tests/connect/connect_distributed_test.py b/tests/kafkatest/tests/connect/connect_distributed_test.py index cd36ce1976..d5deda32f8 100644 --- a/tests/kafkatest/tests/connect/connect_distributed_test.py +++ b/tests/kafkatest/tests/connect/connect_distributed_test.py @@ -32,6 +32,7 @@ import json import operator import time +import subprocess class ConnectDistributedTest(Test): """ @@ -114,7 +115,7 @@ def _start_connector(self, config_file, extra_config={}): connector_config = dict([line.strip().split('=', 1) for line in connector_props.split('\n') if line.strip() and not line.strip().startswith('#')]) connector_config.update(extra_config) self.cc.create_connector(connector_config) - + def _connector_status(self, connector, node=None): try: return self.cc.get_connector_status(connector, node) @@ -179,139 +180,6 @@ def task_is_running(self, connector, task_id, node=None): # metadata_quorum=[quorum.zk], # use_new_coordinator=[False] # ) - @matrix( - exactly_once_source=[True, False], - connect_protocol=['sessioned', 'compatible', 'eager'], - metadata_quorum=[quorum.isolated_kraft], - use_new_coordinator=[True], - group_protocol=consumer_group.all_group_protocols - ) - def test_restart_failed_connector(self, exactly_once_source, connect_protocol, metadata_quorum, use_new_coordinator=False, group_protocol=None): - self.EXACTLY_ONCE_SOURCE_SUPPORT = 'enabled' if exactly_once_source else 'disabled' - self.CONNECT_PROTOCOL = connect_protocol - self.setup_services() - self.cc.set_configs(lambda node: self.render("connect-distributed.properties", node=node)) - self.cc.start() - - if exactly_once_source: - self.connector = MockSource(self.cc, mode='connector-failure', delay_sec=5) - else: - self.connector = MockSink(self.cc, self.topics.keys(), mode='connector-failure', delay_sec=5, consumer_group_protocol=group_protocol) - self.connector.start() - - wait_until(lambda: self.connector_is_failed(self.connector), timeout_sec=15, - err_msg="Failed to see connector transition to the FAILED state") - - self.cc.restart_connector(self.connector.name) - - wait_until(lambda: self.connector_is_running(self.connector), timeout_sec=10, - err_msg="Failed to see connector transition to the RUNNING state") - - @cluster(num_nodes=5) - @matrix( - connector_type=['source', 'exactly-once source', 'sink'], - connect_protocol=['sessioned', 'compatible', 'eager'], - metadata_quorum=[quorum.isolated_kraft], - use_new_coordinator=[False] - ) - @matrix( - connector_type=['source', 'exactly-once source', 'sink'], - connect_protocol=['sessioned', 'compatible', 'eager'], - metadata_quorum=[quorum.isolated_kraft], - use_new_coordinator=[True], - group_protocol=consumer_group.all_group_protocols - ) - def test_restart_failed_task(self, connector_type, connect_protocol, metadata_quorum, use_new_coordinator=False, group_protocol=None): - self.EXACTLY_ONCE_SOURCE_SUPPORT = 'enabled' if connector_type == 'exactly-once source' else 'disabled' - self.CONNECT_PROTOCOL = connect_protocol - self.setup_services() - self.cc.set_configs(lambda node: self.render("connect-distributed.properties", node=node)) - self.cc.start() - - connector = None - if connector_type == "sink": - connector = MockSink(self.cc, self.topics.keys(), mode='task-failure', delay_sec=5, consumer_group_protocol=group_protocol) - else: - connector = MockSource(self.cc, mode='task-failure', delay_sec=5) - - connector.start() - - task_id = 0 - wait_until(lambda: self.task_is_failed(connector, task_id), timeout_sec=20, - err_msg="Failed to see task transition to the FAILED state") - - self.cc.restart_task(connector.name, task_id) - - wait_until(lambda: self.task_is_running(connector, task_id), timeout_sec=10, - err_msg="Failed to see task transition to the RUNNING state") - - @cluster(num_nodes=5) - @matrix( - connect_protocol=['sessioned', 'compatible', 'eager'], - metadata_quorum=[quorum.isolated_kraft], - use_new_coordinator=[False] - ) - @matrix( - connect_protocol=['sessioned', 'compatible', 'eager'], - metadata_quorum=[quorum.isolated_kraft], - use_new_coordinator=[True], - group_protocol=consumer_group.all_group_protocols - ) - def test_restart_connector_and_tasks_failed_connector(self, connect_protocol, metadata_quorum, use_new_coordinator=False, group_protocol=None): - self.CONNECT_PROTOCOL = connect_protocol - self.setup_services() - self.cc.set_configs(lambda node: self.render("connect-distributed.properties", node=node)) - self.cc.start() - - self.sink = MockSink(self.cc, self.topics.keys(), mode='connector-failure', delay_sec=5, consumer_group_protocol=group_protocol) - self.sink.start() - - wait_until(lambda: self.connector_is_failed(self.sink), timeout_sec=15, - err_msg="Failed to see connector transition to the FAILED state") - - self.cc.restart_connector_and_tasks(self.sink.name, only_failed = "true", include_tasks = "false") - - wait_until(lambda: self.connector_is_running(self.sink), timeout_sec=10, - err_msg="Failed to see connector transition to the RUNNING state") - - @cluster(num_nodes=5) - @matrix( - connector_type=['source', 'sink'], - connect_protocol=['sessioned', 'compatible', 'eager'], - metadata_quorum=[quorum.isolated_kraft], - use_new_coordinator=[False] - ) - @matrix( - connector_type=['source', 'sink'], - connect_protocol=['sessioned', 'compatible', 'eager'], - metadata_quorum=[quorum.isolated_kraft], - use_new_coordinator=[True], - group_protocol=consumer_group.all_group_protocols - ) - def test_restart_connector_and_tasks_failed_task(self, connector_type, connect_protocol, metadata_quorum, use_new_coordinator=False, group_protocol=None): - self.CONNECT_PROTOCOL = connect_protocol - self.setup_services() - self.cc.set_configs(lambda node: self.render("connect-distributed.properties", node=node)) - self.cc.start() - - connector = None - if connector_type == "sink": - connector = MockSink(self.cc, self.topics.keys(), mode='task-failure', delay_sec=5, consumer_group_protocol=group_protocol) - else: - connector = MockSource(self.cc, mode='task-failure', delay_sec=5) - - connector.start() - - task_id = 0 - wait_until(lambda: self.task_is_failed(connector, task_id), timeout_sec=20, - err_msg="Failed to see task transition to the FAILED state") - - self.cc.restart_connector_and_tasks(connector.name, only_failed = "false", include_tasks = "true") - - wait_until(lambda: self.task_is_running(connector, task_id), timeout_sec=10, - err_msg="Failed to see task transition to the RUNNING state") - - @cluster(num_nodes=5) # @matrix( # exactly_once_source=[True, False], # connect_protocol=['sessioned', 'compatible', 'eager'], @@ -341,7 +209,7 @@ def test_pause_and_resume_source(self, exactly_once_source, connect_protocol, me wait_until(lambda: self.is_running(self.source), timeout_sec=30, err_msg="Failed to see connector transition to the RUNNING state") - + self.cc.pause_connector(self.source.name) # wait until all nodes report the paused transition @@ -394,7 +262,7 @@ def test_pause_and_resume_sink(self, connect_protocol, metadata_quorum, use_new_ wait_until(lambda: self.is_running(self.sink), timeout_sec=30, err_msg="Failed to see connector transition to the RUNNING state") - + self.cc.pause_connector(self.sink.name) # wait until all nodes report the paused transition @@ -421,8 +289,8 @@ def test_pause_and_resume_sink(self, connect_protocol, metadata_quorum, use_new_ # @matrix( # exactly_once_source=[True, False], # connect_protocol=['sessioned', 'compatible', 'eager'], - # metadata_quorum=[quorum.zk], - # use_new_coordinator=[False] + # metadata_quorum=[quorum.isolated_kraft], + # use_new_coordinator=[True, False] # ) @matrix( exactly_once_source=[True, False], @@ -446,7 +314,7 @@ def test_pause_state_persistent(self, exactly_once_source, connect_protocol, met wait_until(lambda: self.is_running(self.source), timeout_sec=30, err_msg="Failed to see connector transition to the RUNNING state") - + self.cc.pause_connector(self.source.name) self.cc.restart() @@ -669,7 +537,7 @@ def test_file_source_and_sink(self, security_protocol, exactly_once_source, conn self._start_connector("connect-file-sink.properties", {"consumer.override.group.protocol" : group_protocol}) else: self._start_connector("connect-file-sink.properties") - + # Generating data on the source node should generate new records and create new output on the sink node. Timeouts # here need to be more generous than they are for standalone mode because a) it takes longer to write configs, # do rebalancing of the group, etc, and b) without explicit leave group support, rebalancing takes awhile @@ -726,8 +594,8 @@ def test_bounce(self, clean, connect_protocol, metadata_quorum, use_new_coordina # Give additional time for the consumer groups to recover. Even if it is not a hard bounce, there are # some cases where a restart can cause a rebalance to take the full length of the session timeout # (e.g. if the client shuts down before it has received the memberId from its initial JoinGroup). - # If we don't give enough time for the group to stabilize, the next bounce may cause consumers to - # be shut down before they have any time to process data and we can end up with zero data making it + # If we don't give enough time for the group to stabilize, the next bounce may cause consumers to + # be shut down before they have any time to process data and we can end up with zero data making it # through the test. time.sleep(15) @@ -1034,8 +902,8 @@ def test_transformations(self, connect_protocol, metadata_quorum, use_new_coordi # @parametrize(broker_version=str(LATEST_0_10_0), auto_create_topics=True, exactly_once_source=False, connect_protocol='eager') def test_broker_compatibility(self, broker_version, auto_create_topics, exactly_once_source, connect_protocol): """ - Verify that Connect will start up with various broker versions with various configurations. - When Connect distributed starts up, it either creates internal topics (v0.10.1.0 and after) + Verify that Connect will start up with various broker versions with various configurations. + When Connect distributed starts up, it either creates internal topics (v0.10.1.0 and after) or relies upon the broker to auto-create the topics (v0.10.0.x and before). """ self.EXACTLY_ONCE_SOURCE_SUPPORT = 'enabled' if exactly_once_source else 'disabled' @@ -1088,3 +956,712 @@ def _restart_worker(self, node, clean=True): monitor.wait_until("Starting connectors and tasks using config offset", timeout_sec=90, err_msg="Kafka Connect worker didn't successfully join group and start work") self.logger.info("Bounced Kafka Connect on %s and rejoined in %f seconds", node.account, time.time() - started) + + def _wait_for_metrics_available(self, timeout_sec=60): + """Wait for metrics endpoint to become available""" + self.logger.info("Waiting for metrics endpoint to become available...") + + def metrics_available(): + for node in self.cc.nodes: + try: + cmd = "curl -s http://localhost:9464/metrics" + result = node.account.ssh_capture(cmd, allow_fail=True) + metrics_output = "".join([line for line in result]) + + # Check for any metrics output (not just kafka_connect) + if len(metrics_output.strip()) > 0 and ("#" in metrics_output or "_" in metrics_output): + self.logger.info(f"Metrics available on node {node.account.hostname}, content length: {len(metrics_output)}") + return True + else: + self.logger.debug(f"Node {node.account.hostname} metrics not ready yet, output length: {len(metrics_output)}") + except Exception as e: + self.logger.debug(f"Error checking metrics on node {node.account.hostname}: {e}") + continue + return False + + wait_until( + metrics_available, + timeout_sec=timeout_sec, + err_msg="Metrics endpoint did not become available within the specified time" + ) + + self.logger.info("Metrics endpoint is now available!") + + def _verify_opentelemetry_metrics(self): + """Verify OpenTelemetry metrics content""" + for node in self.cc.nodes: + cmd = "curl -s http://localhost:9464/metrics" + result = node.account.ssh_capture(cmd) + metrics_output = "".join([line for line in result]) + + # Basic check - verify any metrics output exists + assert len(metrics_output.strip()) > 0, "Metrics endpoint returned no content" + + # Print ALL metrics for debugging + self.logger.info(f"=== ALL METRICS from Node {node.account.hostname} ===") + self.logger.info(metrics_output) + self.logger.info(f"=== END OF METRICS from Node {node.account.hostname} ===") + + # Find all metric lines (not comments) + metric_lines = [line for line in metrics_output.split('\n') + if line.strip() and not line.startswith('#') and ('_' in line or '{' in line)] + + # Should have at least some metrics + assert len(metric_lines) > 0, "No valid metric lines found" + + self.logger.info(f"Found {len(metric_lines)} metric lines") + + # Log kafka_connect metrics specifically + kafka_connect_lines = [line for line in metric_lines if 'kafka_connect' in line] + self.logger.info(f"Found {len(kafka_connect_lines)} kafka_connect metric lines:") + for i, line in enumerate(kafka_connect_lines): + self.logger.info(f"kafka_connect metric {i+1}: {line}") + + # Check for Prometheus format characteristics + has_help = "# HELP" in metrics_output + has_type = "# TYPE" in metrics_output + + if has_help and has_type: + self.logger.info("Metrics conform to Prometheus format") + else: + self.logger.warning("Metrics may not be in standard Prometheus format") + + # Use lenient metric validation to analyze values + self._validate_metric_values(metrics_output) + + self.logger.info(f"Node {node.account.hostname} basic metrics validation passed") + + def _verify_comprehensive_metrics(self): + """Comprehensive metrics validation""" + for node in self.cc.nodes: + cmd = "curl -s http://localhost:9464/metrics" + result = node.account.ssh_capture(cmd) + metrics_output = "".join([line for line in result]) + + # Basic check - verify any metrics output exists + assert len(metrics_output.strip()) > 0, "Metrics endpoint returned no content" + + # Print ALL metrics for comprehensive debugging + self.logger.info(f"=== COMPREHENSIVE METRICS from Node {node.account.hostname} ===") + self.logger.info(metrics_output) + self.logger.info(f"=== END OF COMPREHENSIVE METRICS from Node {node.account.hostname} ===") + + # Find all metric lines (start with letter, not comments) + metric_lines = [line for line in metrics_output.split('\n') + if line.strip() and not line.startswith('#') and ('_' in line or '{' in line)] + self.logger.info(f"Found metric line count: {len(metric_lines)}") + + # Find kafka_connect related metrics + kafka_connect_lines = [line for line in metric_lines if 'kafka_connect' in line] + self.logger.info(f"Found kafka_connect metric line count: {len(kafka_connect_lines)}") + + # Print all kafka_connect metrics + self.logger.info("=== ALL kafka_connect metrics ===") + for i, line in enumerate(kafka_connect_lines): + self.logger.info(f"kafka_connect metric {i+1}: {line}") + + # If no kafka_connect metrics found, show other metrics + if len(kafka_connect_lines) == 0: + self.logger.warning("No kafka_connect metrics found, showing other metrics:") + for i, line in enumerate(metric_lines[:10]): # Show first 10 instead of 5 + self.logger.info(f"Other metric line {i+1}: {line}") + + # Should have at least some metric output + assert len(metric_lines) > 0, "No valid metric lines found" + else: + # Found kafka_connect metrics + self.logger.info(f"Successfully found {len(kafka_connect_lines)} kafka_connect metrics") + + # Check for HELP and TYPE comments (Prometheus format characteristics) + has_help = "# HELP" in metrics_output + has_type = "# TYPE" in metrics_output + + if has_help: + self.logger.info("Found HELP comments - conforms to Prometheus format") + if has_type: + self.logger.info("Found TYPE comments - conforms to Prometheus format") + + self.logger.info(f"Node {node.account.hostname} metrics validation passed, total {len(metric_lines)} metrics found") + + def _validate_metric_values(self, metrics_output): + """Validate metric value reasonableness - more lenient version""" + lines = metrics_output.split('\n') + negative_metrics = [] + + self.logger.info("=== ANALYZING METRIC VALUES ===") + + for line in lines: + if line.startswith('kafka_connect_') and not line.startswith('#'): + # Parse metric line: metric_name{labels} value timestamp + parts = line.split() + if len(parts) >= 2: + try: + value = float(parts[1]) + metric_name = parts[0].split('{')[0] if '{' in parts[0] else parts[0] + + # Log all metric values for analysis + self.logger.info(f"Metric: {metric_name} = {value}") + + # Some metrics can legitimately be negative (e.g., ratios, differences, etc.) + # Only flag as problematic if it's a count or gauge that shouldn't be negative + if value < 0: + negative_metrics.append(f"{parts[0]} = {value}") + + # Allow certain metrics to be negative + allowed_negative_patterns = [ + 'ratio', + 'seconds_ago', + 'difference', + 'offset', + 'lag' + ] + + is_allowed_negative = any(pattern in parts[0].lower() for pattern in allowed_negative_patterns) + + if is_allowed_negative: + self.logger.info(f"Negative value allowed for metric: {parts[0]} = {value}") + else: + self.logger.warning(f"Potentially problematic negative value: {parts[0]} = {value}") + # Don't assert here, just log for now + + except ValueError: + # Skip unparseable lines + continue + + if negative_metrics: + self.logger.info(f"Found {len(negative_metrics)} metrics with negative values:") + for metric in negative_metrics: + self.logger.info(f" - {metric}") + + self.logger.info("=== END METRIC VALUE ANALYSIS ===") + + def _verify_metrics_updates(self): + """Verify metrics update over time""" + # Get initial metrics + initial_metrics = {} + for node in self.cc.nodes: + cmd = "curl -s http://localhost:9464/metrics" + result = node.account.ssh_capture(cmd) + initial_metrics[node] = "".join([line for line in result]) + + # Wait for some time + time.sleep(5) + + # Get metrics again and compare + for node in self.cc.nodes: + cmd = "curl -s http://localhost:9464/metrics" + result = node.account.ssh_capture(cmd) + current_metrics = "".join([line for line in result]) + + # Metrics should have changed (at least timestamps will update) + # More detailed verification can be done here + self.logger.info(f"Node {node.account.hostname} metrics have been updated") + + def _safe_cleanup(self): + """Safe resource cleanup""" + try: + # Delete connectors + connectors = self.cc.list_connectors() + for connector in connectors: + try: + self.cc.delete_connector(connector) + self.logger.info(f"Deleted connector: {connector}") + except Exception as e: + self.logger.warning(f"Failed to delete connector {connector}: {e}") + + # Stop services + self.cc.stop() + + except Exception as e: + self.logger.error(f"Error occurred during cleanup: {e}") + + + @cluster(num_nodes=5) + def test_opentelemetry_metrics_basic(self): + """Basic OpenTelemetry metrics reporting test""" + # Use standard setup, template already contains OpenTelemetry configuration + self.setup_services() + self.cc.set_configs(lambda node: self.render("connect-distributed.properties", node=node)) + + self.logger.info("Starting Connect cluster...") + self.cc.start() + + try: + self.logger.info("Creating VerifiableSource connector...") + # Use VerifiableSource instead of file connector + self.source = VerifiableSource(self.cc, topic=self.TOPIC, throughput=10) + self.source.start() + + # Wait for connector to be running + self.logger.info("Waiting for connector to be running...") + wait_until(lambda: self.is_running(self.source), timeout_sec=30, + err_msg="VerifiableSource connector failed to start") + + self.logger.info("Connector is running, checking metrics...") + + # Wait for and verify metrics + self._wait_for_metrics_available() + self._verify_opentelemetry_metrics() + + # Verify metrics update over time + self._verify_metrics_updates() + + self.logger.info("All metrics validations passed!") + + finally: + if hasattr(self, 'source'): + self.logger.info("Stopping source connector...") + self.source.stop() + self.logger.info("Stopping Connect cluster...") + self.cc.stop() + + + @cluster(num_nodes=5) + def test_opentelemetry_metrics_comprehensive(self): + """Comprehensive Connect OpenTelemetry metrics test - using VerifiableSource""" + # Use standard setup, template already contains OpenTelemetry configuration + self.setup_services(num_workers=3) + self.cc.set_configs(lambda node: self.render("connect-distributed.properties", node=node)) + self.cc.start() + + try: + # Create connector using VerifiableSource + self.source = VerifiableSource(self.cc, topic='metrics-test-topic', throughput=50) + self.source.start() + + # Wait for connector startup + wait_until( + lambda: self.is_running(self.source), + timeout_sec=30, + err_msg="VerifiableSource connector failed to start within expected time" + ) + + # Verify metrics export + self._wait_for_metrics_available() + self._verify_comprehensive_metrics() + + # Verify connector is producing data + wait_until( + lambda: len(self.source.sent_messages()) > 0, + timeout_sec=30, + err_msg="VerifiableSource failed to produce messages" + ) + + finally: + if hasattr(self, 'source'): + self.source.stop() + self.cc.stop() + + @cluster(num_nodes=5) + def test_metrics_under_load(self): + """Test metrics functionality under load""" + # Use standard setup, template already contains OpenTelemetry configuration + self.setup_services(num_workers=3) + self.cc.set_configs(lambda node: self.render("connect-distributed.properties", node=node)) + self.cc.start() + + try: + # Create multiple connectors + connectors = [] + for i in range(3): + connector_name = f'load-test-connector-{i}' + connector_config = { + 'name': connector_name, + 'connector.class': 'org.apache.kafka.connect.tools.VerifiableSourceConnector', + 'tasks.max': '2', + 'topic': f'load-test-topic-{i}', + 'throughput': '100' + } + self.cc.create_connector(connector_config) + connectors.append(connector_name) + + # Wait for all connectors to start + for connector_name in connectors: + wait_until( + lambda cn=connector_name: self.connector_is_running( + type('MockConnector', (), {'name': cn})() + ), + timeout_sec=30, + err_msg=f"Connector {connector_name} failed to start" + ) + + # Verify metrics accuracy under load + self._verify_metrics_under_load(len(connectors)) + + finally: + # Clean up all connectors + for connector_name in connectors: + try: + self.cc.delete_connector(connector_name) + except: + pass + self.cc.stop() + + def _verify_metrics_under_load(self, expected_connector_count): + """Verify metrics accuracy under load""" + self._wait_for_metrics_available() + + for node in self.cc.nodes: + cmd = "curl -s http://localhost:9464/metrics" + result = node.account.ssh_capture(cmd) + metrics_output = "".join([line for line in result]) + + # Verify connector count metrics + connector_count_found = False + for line in metrics_output.split('\n'): + if 'kafka_connect_worker_connector_count' in line and not line.startswith('#'): + parts = line.split() + if len(parts) >= 2: + count = float(parts[1]) + assert count >= expected_connector_count, f"Connector count metric incorrect: {count} < {expected_connector_count}" + connector_count_found = True + break + + assert connector_count_found, "Connector count metric not found" + self.logger.info(f"Node {node.account.hostname} load test metrics validation passed") + + @cluster(num_nodes=5) + def test_opentelemetry_remote_write_exporter(self): + """Test OpenTelemetry Remote Write exporter functionality""" + # Setup mock remote write server + self.setup_services(num_workers=2) + + # Override the template to use remote write exporter + def remote_write_config(node): + config = self.render("connect-distributed.properties", node=node) + # Replace prometheus exporter with remote write using correct URI format + self.logger.info(f"connect config: {config}") + config = config.replace( + "automq.telemetry.exporter.uri=prometheus://0.0.0.0:9464", + "automq.telemetry.exporter.uri=rw://?endpoint=http://localhost:9090/api/v1/write&auth=no_auth&maxBatchSize=1000000" + ) + # Add remote write specific configurations + config += "\nautomq.telemetry.exporter.interval.ms=30000\n" + + self.logger.info(f"connect new config: {config}") + return config + + self.cc.set_configs(remote_write_config) + + # Setup mock remote write endpoint using python HTTP server + mock_server_node = self.cc.nodes[0] + self.logger.info("Setting up mock remote write server...") + + # Start mock server in background that accepts HTTP POST requests + mock_server_cmd = "nohup python3 -c \"\ +import http.server\n\ +import socketserver\n\ +from urllib.parse import urlparse\n\ +import gzip\n\ +import sys\n\ +import time\n\ +\n\ +class MockRemoteWriteHandler(http.server.BaseHTTPRequestHandler):\n\ + def do_POST(self):\n\ + if self.path == '/api/v1/write':\n\ + content_length = int(self.headers.get('Content-Length', 0))\n\ + post_data = self.rfile.read(content_length)\n\ + # Handle gzip compression if present\n\ + encoding = self.headers.get('Content-Encoding', '')\n\ + if encoding == 'gzip':\n\ + try:\n\ + post_data = gzip.decompress(post_data)\n\ + except:\n\ + pass\n\ + # Force flush to ensure log is written immediately\n\ + log_msg = '{} - Received remote write request: {} bytes, encoding: {}'.format(time.strftime('%Y-%m-%d-%H:%M:%S'), len(post_data), encoding)\n\ + print(log_msg, flush=True)\n\ + sys.stdout.flush()\n\ + self.send_response(200)\n\ + self.end_headers()\n\ + self.wfile.write(b'OK')\n\ + else:\n\ + print('{} - Received non-write request: {}'.format(time.strftime('%Y-%m-%d-%H:%M:%S'), self.path), flush=True)\n\ + sys.stdout.flush()\n\ + self.send_response(404)\n\ + self.end_headers()\n\ + \n\ + def log_message(self, format, *args):\n\ + # Re-enable basic HTTP server logging\n\ + log_msg = '{} - HTTP: {}'.format(time.strftime('%Y-%m-%d-%H:%M:%S'), format % args)\n\ + print(log_msg, flush=True)\n\ + sys.stdout.flush()\n\ +\n\ +print('Mock remote write server starting...', flush=True)\n\ +sys.stdout.flush()\n\ +with socketserver.TCPServer(('', 9090), MockRemoteWriteHandler) as httpd:\n\ + print('Mock remote write server listening on port 9090', flush=True)\n\ + sys.stdout.flush()\n\ + httpd.serve_forever()\n\ +\" > /tmp/mock_remote_write.log 2>&1 & echo $!" + + try: + # Start mock server + mock_pid_result = list(mock_server_node.account.ssh_capture(mock_server_cmd)) + mock_pid = mock_pid_result[0].strip() if mock_pid_result else None + if not mock_pid: + raise RuntimeError("Failed to start mock remote write server") + self.logger.info(f"Mock remote write server started with PID: {mock_pid}") + + # Wait a bit for server to start + time.sleep(5) + + # Verify mock server is listening + wait_until( + lambda: self._check_port_listening(mock_server_node, 9090), + timeout_sec=30, + err_msg="Mock remote write server failed to start" + ) + + self.logger.info("Starting Connect cluster with Remote Write exporter...") + self.cc.start() + + # Create connector to generate metrics + self.source = VerifiableSource(self.cc, topic=self.TOPIC, throughput=20) + self.source.start() + + # Wait for connector to be running + wait_until(lambda: self.is_running(self.source), timeout_sec=30, + err_msg="VerifiableSource connector failed to start") + + # Wait for metrics to be sent to remote write endpoint + self.logger.info("Waiting for remote write requests...") + time.sleep(120) # Wait for at least 2 export intervals + + # Verify remote write requests were received + self._verify_remote_write_requests(mock_server_node) + + self.logger.info("Remote Write exporter test passed!") + + finally: + # Cleanup + try: + if 'mock_pid' in locals() and mock_pid: + mock_server_node.account.ssh(f"kill {mock_pid}", allow_fail=True) + if hasattr(self, 'source'): + self.source.stop() + self.cc.stop() + except Exception as e: + self.logger.warning(f"Cleanup error: {e}") + + @cluster(num_nodes=5) + def test_opentelemetry_s3_metrics_exporter(self): + """Test OpenTelemetry S3 Metrics exporter functionality""" + # Setup mock S3 server using localstack + self.setup_services(num_workers=2) + + # Create a temporary directory to simulate S3 bucket + s3_mock_dir = "/tmp/mock-s3-bucket" + bucket_name = "test-metrics-bucket" + + def s3_config(node): + config = self.render("connect-distributed.properties", node=node) + # Replace prometheus exporter with S3 exporter + config = config.replace( + "automq.telemetry.exporter.uri=prometheus://0.0.0.0:9464", + "automq.telemetry.exporter.uri=s3://my-bucket-name" + ) + # Add S3 specific configurations + config += "\nautomq.telemetry.exporter.interval.ms=30000\n" + config += "automq.telemetry.exporter.s3.cluster.id=test-cluster\n" + config += f"automq.telemetry.exporter.s3.node.id={self.cc.nodes.index(node) + 1}\n" + + # Set primary node for the first worker only + is_primary = self.cc.nodes.index(node) == 0 + config += f"automq.telemetry.exporter.s3.primary.node={str(is_primary).lower()}\n" + config += "automq.telemetry.exporter.s3.selector.type=static\n" + + # Configure S3 bucket properly for localstack + # Use localstack endpoint (10.5.0.2:4566 from docker-compose.yaml) + config += f"automq.telemetry.s3.bucket=0@s3://{bucket_name}?endpoint=http://10.5.0.2:4566®ion=us-east-1\n" + + # Add AWS credentials for localstack (localstack accepts any credentials) + return config + + self.cc.set_configs(s3_config) + + try: + # Setup mock S3 directory on all nodes (as fallback) + for node in self.cc.nodes: + node.account.ssh(f"mkdir -p {s3_mock_dir}", allow_fail=False) + node.account.ssh(f"chmod 777 {s3_mock_dir}", allow_fail=False) + + self.logger.info("Starting Connect cluster with S3 exporter...") + self.cc.start() + + # Create the S3 bucket in localstack first + primary_node = self.cc.nodes[0] + + create_bucket_cmd = f"aws s3api create-bucket --bucket {bucket_name} --endpoint=http://10.5.0.2:4566" + + ret, val = subprocess.getstatusoutput(create_bucket_cmd) + self.logger.info( + f'\n--------------objects[bucket:{bucket_name}]--------------------\n:{val}\n--------------objects--------------------\n') + if ret != 0: + raise Exception("Failed to get bucket objects size, output: %s" % val) + + # Create connector to generate metrics + self.source = VerifiableSource(self.cc, topic=self.TOPIC, throughput=15) + self.source.start() + + # Wait for connector to be running + wait_until(lambda: self.is_running(self.source), timeout_sec=30, + err_msg="VerifiableSource connector failed to start") + + # Wait for metrics to be exported to S3 + self.logger.info("Waiting for S3 metrics export...") + time.sleep(60) # Wait for at least 2 export intervals + + # Verify S3 exports were created in localstack + self._verify_s3_metrics_export_localstack(bucket_name, primary_node) + + self.logger.info("S3 Metrics exporter test passed!") + + finally: + # Cleanup + try: + if hasattr(self, 'source'): + self.source.stop() + self.cc.stop() + # Clean up mock S3 directory + for node in self.cc.nodes: + self.logger.info("Cleaning up S3 mock directory...") + # node.account.ssh(f"rm -rf {s3_mock_dir}", allow_fail=True) + except Exception as e: + self.logger.warning(f"Cleanup error: {e}") + + def _check_port_listening(self, node, port): + """Check if a port is listening on the given node""" + try: + result = list(node.account.ssh_capture(f"netstat -ln | grep :{port}", allow_fail=True)) + return len(result) > 0 + except: + return False + + def _verify_remote_write_requests(self, node, log_file="/tmp/mock_remote_write.log"): + """Verify that remote write requests were received""" + try: + # Check the mock server log for received requests + result = list(node.account.ssh_capture(f"cat {log_file}", allow_fail=True)) + log_content = "".join(result) + + self.logger.info(f"Remote write log content: {log_content}") + + # Look for evidence of received data + if "Received" in log_content or "received" in log_content: + self.logger.info("Remote write requests were successfully received") + return True + + # Also check if the process is running and listening + if self._check_port_listening(node, 9090) or self._check_port_listening(node, 9091): + self.logger.info("Remote write server is listening, requests may have been processed") + return True + + self.logger.warning("No clear evidence of remote write requests in log") + return False + + except Exception as e: + self.logger.warning(f"Error verifying remote write requests: {e}") + # Don't fail the test if we can't verify the log, as the server might be working + return True + + def _verify_s3_metrics_export_localstack(self, bucket_name, node): + """Verify that metrics were exported to S3 via localstack""" + try: + # 递归列出 S3 bucket 中的所有对象文件(而不是目录) + list_cmd = f"aws s3 ls s3://{bucket_name}/ --recursive --endpoint=http://10.5.0.2:4566" + + ret, val = subprocess.getstatusoutput(list_cmd) + self.logger.info( + f'\n--------------recursive objects[bucket:{bucket_name}]--------------------\n{val}\n--------------recursive objects end--------------------\n') + if ret != 0: + self.logger.warning(f"Failed to list bucket objects recursively, return code: {ret}, output: {val}") + # 尝试非递归列出目录结构 + list_dir_cmd = f"aws s3 ls s3://{bucket_name}/ --endpoint=http://10.5.0.2:4566" + ret2, val2 = subprocess.getstatusoutput(list_dir_cmd) + self.logger.info(f"Directory listing: {val2}") + + # 如果非递归也失败,说明bucket可能不存在或没有权限 + if ret2 != 0: + raise Exception(f"Failed to list bucket contents, output: {val}") + else: + # 看到了目录但没有文件,说明可能还没有上传完成 + self.logger.info("Found directories but no files yet, checking subdirectories...") + + # 尝试列出 automq/metrics/ 下的内容 + automq_cmd = f"aws s3 ls s3://{bucket_name}/automq/metrics/ --recursive --endpoint=http://10.5.0.2:4566" + ret3, val3 = subprocess.getstatusoutput(automq_cmd) + self.logger.info(f"AutoMQ metrics directory contents: {val3}") + + if ret3 == 0 and val3.strip(): + s3_objects = [line.strip() for line in val3.strip().split('\n') if line.strip()] + else: + return False + else: + s3_objects = [line.strip() for line in val.strip().split('\n') if line.strip()] + + self.logger.info(f"S3 bucket {bucket_name} file contents (total {len(s3_objects)} files): {s3_objects}") + + if s3_objects: + # 过滤掉目录行,只保留文件行(文件行通常有size信息) + file_objects = [] + for obj_line in s3_objects: + parts = obj_line.split() + # 文件行格式: 2025-01-01 12:00:00 size_in_bytes filename + # 目录行格式: PRE directory_name/ 或者只有目录名 + if len(parts) >= 4 and not obj_line.strip().startswith('PRE') and 'automq/metrics/' in obj_line: + file_objects.append(obj_line) + + self.logger.info(f"Found {len(file_objects)} actual metric files in S3:") + for file_obj in file_objects: + self.logger.info(f" - {file_obj}") + + if file_objects: + self.logger.info(f"S3 metrics export verified via localstack: found {len(file_objects)} metric files") + + # 尝试下载并检查第一个文件的内容 + try: + first_file_parts = file_objects[0].split() + if len(first_file_parts) >= 4: + object_name = ' '.join(first_file_parts[3:]) # 文件名可能包含空格 + + # 下载并检查内容 + download_cmd = f"aws s3 cp s3://{bucket_name}/{object_name} /tmp/sample_metrics.json --endpoint=http://10.5.0.2:4566" + ret, download_output = subprocess.getstatusoutput(download_cmd) + if ret == 0: + self.logger.info(f"Successfully downloaded sample metrics file: {download_output}") + + # 检查文件内容 + cat_cmd = "head -n 3 /tmp/sample_metrics.json" + ret2, content = subprocess.getstatusoutput(cat_cmd) + if ret2 == 0: + self.logger.info(f"Sample metrics content: {content}") + # 验证内容格式是正确(应该包含JSON格式的指标数据) + if any(keyword in content for keyword in ['timestamp', 'name', 'kind', 'tags']): + self.logger.info("Metrics content format verification passed") + else: + self.logger.warning(f"Metrics content format may be incorrect: {content}") + else: + self.logger.warning(f"Failed to download sample file: {download_output}") + except Exception as e: + self.logger.warning(f"Error validating sample metrics file: {e}") + + return True + else: + self.logger.warning("Found S3 objects but none appear to be metric files") + return False + else: + # 检查bucket是否存在但为空 + bucket_check_cmd = f"aws s3api head-bucket --bucket {bucket_name} --endpoint-url http://10.5.0.2:4566" + ret, bucket_output = subprocess.getstatusoutput(bucket_check_cmd) + if ret == 0: + self.logger.info(f"Bucket {bucket_name} exists but is empty - metrics may not have been exported yet") + return False + else: + self.logger.warning(f"Bucket {bucket_name} may not exist: {bucket_output}") + return False + + except Exception as e: + self.logger.warning(f"Error verifying S3 metrics export via localstack: {e}") + return False + diff --git a/tests/kafkatest/tests/connect/templates/connect-distributed.properties b/tests/kafkatest/tests/connect/templates/connect-distributed.properties index fa2172edd7..051a1e23ca 100644 --- a/tests/kafkatest/tests/connect/templates/connect-distributed.properties +++ b/tests/kafkatest/tests/connect/templates/connect-distributed.properties @@ -69,4 +69,16 @@ config.providers.file.class=org.apache.kafka.common.config.provider.FileConfigPr {% if PLUGIN_PATH is defined %} plugin.path={{ PLUGIN_PATH }} {% endif %} -plugin.discovery={{ PLUGIN_DISCOVERY|default("service_load") }} \ No newline at end of file +plugin.discovery={{ PLUGIN_DISCOVERY|default("service_load") }} + +# ??OpenTelemetry????? +metric.reporters=org.apache.kafka.connect.automq.OpenTelemetryMetricsReporter + +# OpenTelemetry???? +opentelemetry.metrics.enabled=true +opentelemetry.metrics.prefix=kafka.connect + +# AutoMQ???? - ??Prometheus??? +automq.telemetry.exporter.uri=prometheus://0.0.0.0:9464 +service.name=kafka-connect-test +service.instance.id=worker-1 \ No newline at end of file