diff --git a/fe/.idea/vcs.xml b/fe/.idea/vcs.xml index 7b2cdb1cbbd39a..e5158c553f8f33 100644 --- a/fe/.idea/vcs.xml +++ b/fe/.idea/vcs.xml @@ -1,20 +1,4 @@ - - + + + + + - + \ No newline at end of file diff --git a/fe/docs/design/kafka_streaming_job_design.md b/fe/docs/design/kafka_streaming_job_design.md new file mode 100644 index 00000000000000..54d98fac403dbd --- /dev/null +++ b/fe/docs/design/kafka_streaming_job_design.md @@ -0,0 +1,898 @@ +--- +name: Kafka Streaming Job +overview: 实现 Kafka StreamingInsertJob 功能,通过 kafka() TVF 订阅 Kafka 数据并持续写入 Doris 表,支持 exactly-once 语义,每个 partition 独立任务并行执行。 +todos: + - id: modify-trino-table + content: 修改 TrinoConnectorExternalTable 支持暴露 Kafka 隐藏列 (_offset, _partition 等) + status: pending + - id: create-kafka-offset + content: 创建 KafkaPartitionOffset 和 KafkaOffset 类,支持多分区 offset 追踪 + status: pending + - id: create-kafka-props-converter + content: 创建 KafkaPropertiesConverter,实现 Trino Catalog 属性到 KafkaUtil 参数的转换 + status: pending + - id: create-kafka-provider + content: 创建 KafkaSourceOffsetProvider,实现多分区 offset 管理和 SQL 重写逻辑 + status: pending + - id: create-kafka-tvf + content: 创建 kafka() TVF 和 KafkaTableValuedFunction + status: pending + - id: refactor-streaming-job + content: 改造 StreamingInsertJob 支持多任务并行(每个 partition 一个任务) + status: pending + - id: register-provider + content: 在 SourceOffsetProviderFactory 中注册 KafkaSourceOffsetProvider + status: pending + - id: add-tests + content: 添加单元测试 + status: pending +isProject: false +--- + +# Kafka StreamingInsertJob 功能设计方案 + +## 1. 功能概述 + +### 1.1 需求描述 + +通过创建 KafkaStreamingJob,可以订阅 Kafka 数据,并持续通过 INSERT 操作将数据从 Kafka 写入到 Doris 表,支持 **exactly-once** 精准一次消费语义。 + +### 1.2 使用方式 + +#### 基本语法 + +```sql +CREATE JOB my_kafka_job +ON STREAMING +DO +INSERT INTO doris_tbl +SELECT * FROM kafka( + "catalog" = "kafka_catalog", + "database" = "kafka_db", + "table" = "kafka_tbl", + "kafka_default_offsets" = "OFFSET_BEGINNING", + "max_batch_rows" = "100000" +); +``` + +#### 支持列选择 + +除了 `SELECT *` 外,还可以指定具体的列名。列名来自于 Trino Kafka Catalog 中定义的表 schema: + +```sql +-- 选择指定列 +CREATE JOB my_kafka_job +ON STREAMING +DO +INSERT INTO doris_tbl (col1, col2, col3) +SELECT col1, col2, col3 FROM kafka( + "catalog" = "kafka_catalog", + "database" = "kafka_db", + "table" = "kafka_tbl", + "kafka_default_offsets" = "OFFSET_BEGINNING", + "max_batch_rows" = "100000" +); + +-- 支持表达式和列别名 +CREATE JOB my_kafka_job +ON STREAMING +DO +INSERT INTO doris_tbl +SELECT + col1, + col2, + CAST(col3 AS INT) as col3_int, + NOW() as load_time +FROM kafka( + "catalog" = "kafka_catalog", + "database" = "kafka_db", + "table" = "kafka_tbl" +); +``` + +**说明**: +- 可选择的列名取决于 Trino Kafka Connector 解析 Kafka 消息后暴露的列 +- 支持使用 SQL 表达式进行数据转换 +- 隐藏列 `_partition` 和 `_offset` 在修改 TrinoConnectorExternalTable 后也可在 SELECT 中使用 + +### 1.3 前置依赖 + +**重要**: `kafka()` TVF 的实现依赖于 **TrinoConnectorExternalCatalog** 中配置的 Kafka Connector。 + +用户需要先创建一个 Trino Kafka Catalog: + +```sql +CREATE CATALOG kafka_catalog PROPERTIES ( + "type" = "trino-connector", + "trino.connector.name" = "kafka", + "kafka.nodes" = "broker1:9092,broker2:9092", + "kafka.table-names" = "topic1,topic2", + "kafka.hide-internal-columns" = "false" + -- 其他 Kafka connector 配置 +); +``` + +然后才能使用 `kafka()` TVF 引用该 Catalog 中的表。 + +### 1.4 Kafka 参数转换 + +由于 `KafkaUtil` 原本是为 `KafkaRoutineLoadJob` 设计的,其参数格式与 TrinoConnectorExternalCatalog 的属性格式不同,需要进行转换。 + +#### 1.4.1 参数格式对比 + +| 参数 | KafkaUtil 格式 | Trino Catalog 格式 | 说明 | +|------|---------------|-------------------|------| +| Broker 列表 | `brokerList` (String) | `trino.kafka.nodes` | host1:9092,host2:9092 | +| Topic 名称 | `topic` (String) | 表名(从 TVF 参数获取) | 每个 topic 对应一个表 | +| 客户端属性 | `convertedCustomProperties` (Map) | `trino.kafka.*` | 需要去除前缀并转换 | + +#### 1.4.2 Trino Kafka Connector 常用属性 + +```properties +# 在 Doris Catalog 中的配置(带 trino. 前缀) +trino.connector.name=kafka +trino.kafka.nodes=broker1:9092,broker2:9092 +trino.kafka.default-schema=default +trino.kafka.table-names=topic1,topic2 +trino.kafka.hide-internal-columns=false + +# 认证相关(如果启用 SASL) +trino.kafka.config.resources=/path/to/kafka.properties +# 或者直接配置 +trino.kafka.security.protocol=SASL_PLAINTEXT +trino.kafka.sasl.mechanism=PLAIN +trino.kafka.sasl.jaas.config=... +``` + +#### 1.4.3 参数转换逻辑 + +在 `KafkaSourceOffsetProvider` 中实现参数转换: + +```java +/** + * 从 TrinoConnectorExternalCatalog 提取 Kafka 连接参数 + * 转换为 KafkaUtil 所需的格式 + */ +public class KafkaPropertiesConverter { + + private static final String TRINO_PREFIX = "trino."; + private static final String KAFKA_PREFIX = "kafka."; + + /** + * 从 Catalog 属性中提取 broker 列表 + * trino.kafka.nodes -> brokerList + */ + public static String extractBrokerList(Map catalogProps) { + // 优先从 trino.kafka.nodes 获取 + String brokers = catalogProps.get("trino.kafka.nodes"); + if (brokers == null) { + // 兼容:尝试从 kafka.nodes 获取 + brokers = catalogProps.get("kafka.nodes"); + } + if (brokers == null) { + throw new IllegalArgumentException("Missing required property: kafka.nodes"); + } + return brokers; + } + + /** + * 提取并转换 Kafka 客户端属性 + * 用于 KafkaUtil.getLatestOffsets() 等方法 + * + * 转换规则: + * - trino.kafka.xxx -> xxx (去除 trino.kafka. 前缀) + * - 保留 security.protocol, sasl.* 等认证相关属性 + */ + public static Map convertToKafkaClientProperties( + Map catalogProps) { + Map kafkaProps = new HashMap<>(); + + String fullPrefix = TRINO_PREFIX + KAFKA_PREFIX; + + for (Map.Entry entry : catalogProps.entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + + if (key.startsWith(fullPrefix)) { + // trino.kafka.xxx -> xxx + String kafkaKey = key.substring(fullPrefix.length()); + + // 过滤掉 Trino 特有的配置,保留 Kafka 客户端配置 + if (isKafkaClientProperty(kafkaKey)) { + kafkaProps.put(kafkaKey, value); + } + } + } + + return kafkaProps; + } + + /** + * 判断是否为 Kafka 客户端属性(非 Trino 特有属性) + */ + private static boolean isKafkaClientProperty(String key) { + // Trino 特有属性,不需要传递给 Kafka 客户端 + Set trinoOnlyProps = Set.of( + "nodes", // broker 列表,单独处理 + "default-schema", // Trino schema 配置 + "table-names", // Trino 表名列表 + "table-description-dir", + "hide-internal-columns", + "timestamp-upper-bound-force-push-down-enabled" + ); + + return !trinoOnlyProps.contains(key); + } +} +``` + +#### 1.4.4 使用示例 + +```java +// 在 KafkaSourceOffsetProvider 中使用 +public class KafkaSourceOffsetProvider implements SourceOffsetProvider { + + private String brokerList; + private String topic; + private Map kafkaClientProps; + + /** + * 初始化 Kafka 连接参数 + * @param catalog TrinoConnectorExternalCatalog 实例 + * @param tableName Kafka topic 对应的表名 + */ + public void initFromCatalog(TrinoConnectorExternalCatalog catalog, String tableName) { + Map catalogProps = catalog.getCatalogProperty().getProperties(); + + // 提取 broker 列表 + this.brokerList = KafkaPropertiesConverter.extractBrokerList(catalogProps); + + // topic 名称就是表名 + this.topic = tableName; + + // 转换 Kafka 客户端属性 + this.kafkaClientProps = KafkaPropertiesConverter + .convertToKafkaClientProperties(catalogProps); + } + + @Override + public void fetchRemoteMeta(Map properties) throws Exception { + // 获取所有分区 + List partitionIds = KafkaUtil.getAllKafkaPartitions( + brokerList, topic, kafkaClientProps); + + // 获取各分区最新 offset + List> offsets = KafkaUtil.getLatestOffsets( + jobId, UUID.randomUUID(), brokerList, topic, kafkaClientProps, partitionIds); + + // 更新 latestOffsets + this.latestOffsets = offsets.stream() + .collect(Collectors.toMap(p -> p.first, p -> p.second)); + } +} +``` + +## 2. 整体架构 + +### 2.1 核心设计原则 + +为了实现 **exactly-once** 语义: + +1. **每个 Partition 独立任务**: 每个 Kafka partition 对应一个独立的 `StreamingInsertTask` +2. **精确 offset 追踪**: 每个任务完成后,通过事务机制精确记录该 partition 的消费进度 +3. **原子性保证**: 数据写入和 offset 更新在同一个事务中完成 + +### 2.2 架构图 + +```mermaid +flowchart TB + subgraph Job [StreamingInsertJob] + direction TB + Provider[KafkaSourceOffsetProvider] + OffsetMap["KafkaOffset\n(partition -> offset)"] + end + + subgraph Tasks [并行任务] + direction LR + Task0[Task: Partition 0] + Task1[Task: Partition 1] + Task2[Task: Partition N] + end + + subgraph Kafka + P0[Partition 0] + P1[Partition 1] + P2[Partition N] + end + + Job --> Tasks + Task0 --> P0 + Task1 --> P1 + Task2 --> P2 + + Task0 -->|"完成后更新\npartition 0 offset"| OffsetMap + Task1 -->|"完成后更新\npartition 1 offset"| OffsetMap + Task2 -->|"完成后更新\npartition N offset"| OffsetMap +``` + +### 2.3 数据流程 + +```mermaid +sequenceDiagram + participant Job as StreamingInsertJob + participant Provider as KafkaSourceOffsetProvider + participant KafkaUtil + participant Tasks as StreamingInsertTasks + participant TrinoKafka as Trino Kafka Connector + participant Doris as Doris Table + + Job->>Provider: fetchRemoteMeta() + Provider->>KafkaUtil: getLatestOffsets() + KafkaUtil-->>Provider: 返回各 partition 最新 offset + + Job->>Provider: getNextOffsets() 获取各 partition 的 offset 范围 + Provider-->>Job: 返回 List + + loop 每个 partition + Job->>Tasks: 创建 Task (partition_id, start_offset, end_offset) + end + + par 并行执行 + Tasks->>TrinoKafka: INSERT INTO ... SELECT ... WHERE _partition=0 AND _offset >= X AND _offset < Y + TrinoKafka-->>Doris: 写入数据 + Doris-->>Tasks: 返回写入行数 + Tasks->>Job: onTaskSuccess(partition_id, consumed_rows) + Job->>Provider: updatePartitionOffset(partition_id, new_offset) + end + + Job->>Job: 所有 partition 任务完成,创建下一批任务 +``` + +## 3. 详细设计 + +### 3.1 核心组件 + +| 组件 | 位置 | 职责 | + +|------|------|------| + +| `Kafka` (TVF) | `nereids/trees/expressions/functions/table/` | Kafka TVF 定义,解析参数 | + +| `KafkaTableValuedFunction` | `tablefunction/` | Catalog 层 TVF 实现 | + +| `KafkaSourceOffsetProvider` | `job/offset/kafka/` | 多分区 offset 管理 | + +| `KafkaOffset` | `job/offset/kafka/` | 整体 offset 状态(所有分区) | + +| `KafkaPartitionOffset` | `job/offset/kafka/` | 单分区 offset 范围 | + +| `TrinoConnectorExternalTable` | `datasource/trinoconnector/` | 修改以支持暴露隐藏列 | + +| `StreamingInsertJob` | `job/extensions/insert/streaming/` | 改造支持多任务并行 | + +### 3.2 Offset 数据结构 + +#### 3.2.1 KafkaPartitionOffset(单分区 offset) + +```java +// fe-core/src/main/java/org/apache/doris/job/offset/kafka/KafkaPartitionOffset.java +public class KafkaPartitionOffset implements Offset { + private int partitionId; // 分区 ID + private long startOffset; // 本次任务起始 offset(包含) + private long endOffset; // 本次任务结束 offset(不包含) + private long consumedRows; // 实际消费的行数(任务完成后填充) + + @Override + public String showRange() { + return String.format("{partition=%d, range=[%d, %d)}", + partitionId, startOffset, endOffset); + } +} +``` + +#### 3.2.2 KafkaOffset(全局 offset 状态) + +```java +// fe-core/src/main/java/org/apache/doris/job/offset/kafka/KafkaOffset.java +public class KafkaOffset implements Offset { + // 各分区当前已消费的 offset(下次消费的起始位置) + @SerializedName("partitionOffsets") + private Map partitionOffsets; + + // topic 名称 + @SerializedName("topic") + private String topic; + + // 用于持久化和恢复 + @Override + public String toSerializedJson() { + return GsonUtils.GSON.toJson(this); + } + + // 更新单个分区的 offset + public void updatePartitionOffset(int partitionId, long newOffset) { + partitionOffsets.put(partitionId, newOffset); + } +} +``` + +### 3.3 KafkaSourceOffsetProvider + +```java +// fe-core/src/main/java/org/apache/doris/job/offset/kafka/KafkaSourceOffsetProvider.java +public class KafkaSourceOffsetProvider implements SourceOffsetProvider { + + // 当前消费位置(各分区) + private KafkaOffset currentOffset; + + // Kafka 最新位置(各分区) + private Map latestOffsets; + + // Kafka 连接信息(从 TVF 参数或 Catalog 获取) + private String brokerList; + private String topic; + private Map kafkaProps; + + // 每批最大行数 + private long maxBatchRows; + + /** + * 获取各分区下一批要消费的 offset 范围 + * @return 每个分区的 offset 范围列表 + */ + public List getNextOffsets(StreamingJobProperties jobProps) { + List offsets = new ArrayList<>(); + for (Map.Entry entry : currentOffset.getPartitionOffsets().entrySet()) { + int partitionId = entry.getKey(); + long currentPos = entry.getValue(); + long latestPos = latestOffsets.getOrDefault(partitionId, currentPos); + + if (currentPos < latestPos) { + // 计算本次要读取的范围 + long endOffset = Math.min(currentPos + maxBatchRows, latestPos); + offsets.add(new KafkaPartitionOffset(partitionId, currentPos, endOffset)); + } + } + return offsets; + } + + /** + * 重写 SQL:将 kafka() TVF 重写为对 Trino Kafka 表的查询 + * + * 原始: SELECT * FROM kafka("catalog"="xxx", "table"="yyy", ...) + * 重写: SELECT * FROM xxx.default.yyy WHERE _partition = ? AND _offset >= ? AND _offset < ? + */ + @Override + public InsertIntoTableCommand rewriteTvfParams( + InsertIntoTableCommand originCommand, + Offset runningOffset) { + KafkaPartitionOffset partitionOffset = (KafkaPartitionOffset) runningOffset; + + // 构建新的 SQL: + // SELECT * FROM catalog.database.table + // WHERE _partition = {partitionId} + // AND _offset >= {startOffset} + // AND _offset < {endOffset} + + // ... 实现 plan 重写逻辑 + } + + /** + * 通过 KafkaUtil 获取各分区的最新 offset + */ + @Override + public void fetchRemoteMeta(Map properties) throws Exception { + List partitionIds = KafkaUtil.getAllKafkaPartitions( + brokerList, topic, kafkaProps); + + List> offsets = KafkaUtil.getLatestOffsets( + jobId, taskId, brokerList, topic, kafkaProps, partitionIds); + + this.latestOffsets = offsets.stream() + .collect(Collectors.toMap(p -> p.first, p -> p.second)); + } + + /** + * 更新指定分区的 offset + */ + public void updatePartitionOffset(int partitionId, long newOffset) { + currentOffset.updatePartitionOffset(partitionId, newOffset); + } + + @Override + public boolean hasMoreDataToConsume() { + for (Map.Entry entry : currentOffset.getPartitionOffsets().entrySet()) { + int partitionId = entry.getKey(); + long currentPos = entry.getValue(); + long latestPos = latestOffsets.getOrDefault(partitionId, currentPos); + if (currentPos < latestPos) { + return true; + } + } + return false; + } +} +``` + +### 3.4 StreamingInsertJob 多任务改造 + +现有的 `StreamingInsertJob` 只维护一个 `runningStreamTask`,需要改造为支持多任务并行: + +```java +// StreamingInsertJob.java 改造要点 + +public class StreamingInsertJob extends AbstractJob<...> { + + // 原有:单任务 + // AbstractStreamingTask runningStreamTask; + + // 改造后:多任务映射 (partitionId -> task) + Map runningPartitionTasks = new ConcurrentHashMap<>(); + + // 当前批次已完成的分区数 + AtomicInteger completedPartitionCount = new AtomicInteger(0); + + // 当前批次总分区数 + int currentBatchPartitionCount = 0; + + /** + * 创建多个分区任务 + */ + protected List createKafkaPartitionTasks() { + KafkaSourceOffsetProvider provider = (KafkaSourceOffsetProvider) offsetProvider; + List partitionOffsets = provider.getNextOffsets(jobProperties); + + List tasks = new ArrayList<>(); + for (KafkaPartitionOffset partitionOffset : partitionOffsets) { + StreamingInsertTask task = new StreamingInsertTask( + getJobId(), + Env.getCurrentEnv().getNextId(), + getExecuteSql(), + offsetProvider, + getCurrentDbName(), + jobProperties, + originTvfProps, + getCreateUser(), + partitionOffset // 新增:分区 offset 信息 + ); + tasks.add(task); + runningPartitionTasks.put(partitionOffset.getPartitionId(), task); + } + + currentBatchPartitionCount = tasks.size(); + completedPartitionCount.set(0); + return tasks; + } + + /** + * 单个分区任务成功回调 + */ + public void onPartitionTaskSuccess(StreamingInsertTask task, int partitionId, long consumedRows) { + writeLock(); + try { + // 更新该分区的 offset + KafkaSourceOffsetProvider provider = (KafkaSourceOffsetProvider) offsetProvider; + KafkaPartitionOffset partitionOffset = (KafkaPartitionOffset) task.getRunningOffset(); + long newOffset = partitionOffset.getStartOffset() + consumedRows; + provider.updatePartitionOffset(partitionId, newOffset); + + // 从运行中任务移除 + runningPartitionTasks.remove(partitionId); + + // 检查是否所有分区都完成 + int completed = completedPartitionCount.incrementAndGet(); + if (completed >= currentBatchPartitionCount) { + // 所有分区完成,创建下一批任务 + createKafkaPartitionTasks(); + } + } finally { + writeUnlock(); + } + } +} +``` + +### 3.5 StreamingInsertTask 改造 + +```java +// StreamingInsertTask.java 改造要点 + +public class StreamingInsertTask extends AbstractStreamingTask { + + // 新增:分区 offset 信息(Kafka 场景使用) + private KafkaPartitionOffset partitionOffset; + + // 新增构造函数 + public StreamingInsertTask(..., KafkaPartitionOffset partitionOffset) { + // ... + this.partitionOffset = partitionOffset; + } + + @Override + public void before() throws Exception { + // ... + + if (partitionOffset != null) { + // Kafka 场景:使用分区 offset + this.runningOffset = partitionOffset; + } else { + // 原有 S3 场景 + this.runningOffset = offsetProvider.getNextOffset(jobProperties, originTvfProps); + } + + // 重写 TVF 参数 + this.taskCommand = offsetProvider.rewriteTvfParams(baseCommand, runningOffset); + // ... + } + + @Override + public boolean onSuccess() throws JobException { + // ... + + StreamingInsertJob job = (StreamingInsertJob) Env.getCurrentEnv() + .getJobManager().getJob(getJobId()); + + if (partitionOffset != null) { + // Kafka 场景:获取实际消费行数并回调 + long consumedRows = getConsumedRowCount(); + job.onPartitionTaskSuccess(this, partitionOffset.getPartitionId(), consumedRows); + } else { + // 原有场景 + job.onStreamTaskSuccess(this); + } + return true; + } + + /** + * 获取本次任务实际消费的行数 + */ + private long getConsumedRowCount() { + // 从 LoadJob 统计信息中获取 + List loadJobs = Env.getCurrentEnv().getLoadManager() + .queryLoadJobsByJobIds(Arrays.asList(this.getTaskId())); + if (!loadJobs.isEmpty()) { + return loadJobs.get(0).getLoadStatistic().getScannedRows(); + } + return 0; + } +} +``` + +### 3.6 修改 TrinoConnectorExternalTable + +```java +// TrinoConnectorExternalTable.java + +@Override +public Optional initSchema() { + // ... + + for (ColumnHandle columnHandle : columnHandleMap.values()) { + ColumnMetadata columnMetadata = connectorMetadata.getColumnMetadata(...); + + // 修改:增加对 Kafka 隐藏列的支持 + if (columnMetadata.isHidden() && !shouldIncludeHiddenColumns()) { + continue; + } + + // ... 其余逻辑不变 + } +} + +/** + * 判断是否需要包含隐藏列 + * Kafka connector 需要暴露 _partition, _offset 等隐藏列 + */ +private boolean shouldIncludeHiddenColumns() { + String connectorName = catalog.getConnectorName(); + // Kafka connector 需要暴露隐藏列以支持 offset 过滤 + if ("kafka".equalsIgnoreCase(connectorName)) { + return true; + } + // 或者通过 catalog 属性配置 + return "true".equalsIgnoreCase( + catalog.getProperties().get("include_hidden_columns")); +} +``` + +### 3.7 kafka() TVF 实现 + +```java +// fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/Kafka.java + +/** + * kafka() TVF - 用于 StreamingInsertJob 从 Kafka 读取数据 + * + * 注意:此 TVF 依赖于预先配置的 TrinoConnectorExternalCatalog (Kafka Connector) + */ +public class Kafka extends TableValuedFunction { + + public Kafka(Properties properties) { + super("kafka", properties); + } + + @Override + protected TableValuedFunctionIf toCatalogFunction() { + try { + Map arguments = getTVFProperties().getMap(); + return new KafkaTableValuedFunction(arguments); + } catch (Throwable t) { + throw new AnalysisException("Can not build kafka(): " + t.getMessage(), t); + } + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitKafka(this, context); + } +} +``` +```java +// fe-core/src/main/java/org/apache/doris/tablefunction/KafkaTableValuedFunction.java + +/** + * Kafka TVF 实现 + * + * 参数: + * - catalog: Trino Kafka Catalog 名称(必填) + * - database: 数据库名称,默认 "default" + * - table: Kafka topic 对应的表名(必填) + * - kafka_default_offsets: 初始 offset,OFFSET_BEGINNING 或 OFFSET_END + * - max_batch_rows: 每批最大读取行数 + */ +public class KafkaTableValuedFunction extends TableValuedFunctionIf { + + public static final String CATALOG = "catalog"; + public static final String DATABASE = "database"; + public static final String TABLE = "table"; + public static final String KAFKA_DEFAULT_OFFSETS = "kafka_default_offsets"; + public static final String MAX_BATCH_ROWS = "max_batch_rows"; + + private String catalogName; + private String databaseName; + private String tableName; + private String defaultOffsets; + private long maxBatchRows; + + public KafkaTableValuedFunction(Map params) throws AnalysisException { + // 解析参数 + this.catalogName = getRequiredParam(params, CATALOG); + this.databaseName = params.getOrDefault(DATABASE, "default"); + this.tableName = getRequiredParam(params, TABLE); + this.defaultOffsets = params.getOrDefault(KAFKA_DEFAULT_OFFSETS, "OFFSET_END"); + this.maxBatchRows = Long.parseLong(params.getOrDefault(MAX_BATCH_ROWS, "100000")); + + // 验证 Catalog 存在且为 Kafka Connector + validateKafkaCatalog(); + } + + private void validateKafkaCatalog() throws AnalysisException { + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(catalogName); + if (catalog == null) { + throw new AnalysisException("Catalog not found: " + catalogName); + } + if (!(catalog instanceof TrinoConnectorExternalCatalog)) { + throw new AnalysisException("Catalog must be a Trino Connector catalog: " + catalogName); + } + TrinoConnectorExternalCatalog trinoCatalog = (TrinoConnectorExternalCatalog) catalog; + if (!"kafka".equalsIgnoreCase(trinoCatalog.getConnectorName())) { + throw new AnalysisException("Catalog must be a Kafka connector: " + catalogName); + } + } +} +``` + +## 4. 执行示例 + +### 4.1 场景描述 + +- Kafka topic: `my_topic`,3 个分区 +- 初始 offset: OFFSET_BEGINNING +- max_batch_rows: 100000 + +### 4.2 执行过程 + +**第一轮调度**: + +1. 获取各分区最新 offset: `{0: 500000, 1: 300000, 2: 400000}` +2. 当前 offset: `{0: 0, 1: 0, 2: 0}` +3. 创建 3 个任务,每个任务的 SQL: +```sql +-- Task 1 (Partition 0) +INSERT INTO doris_tbl +SELECT * FROM kafka_catalog.default.my_topic +WHERE _partition = 0 AND _offset >= 0 AND _offset < 100000; + +-- Task 2 (Partition 1) +INSERT INTO doris_tbl +SELECT * FROM kafka_catalog.default.my_topic +WHERE _partition = 1 AND _offset >= 0 AND _offset < 100000; + +-- Task 3 (Partition 2) +INSERT INTO doris_tbl +SELECT * FROM kafka_catalog.default.my_topic +WHERE _partition = 2 AND _offset >= 0 AND _offset < 100000; +``` + +4. 任务完成后,假设各任务实际消费行数: `{0: 100000, 1: 100000, 2: 100000}` +5. 更新 offset: `{0: 100000, 1: 100000, 2: 100000}` + +**第二轮调度**: + +1. 当前 offset: `{0: 100000, 1: 100000, 2: 100000}` +2. 创建下一批任务... + +## 5. 开发步骤 + +### 步骤 1: 修改 TrinoConnectorExternalTable + +- 文件: [`TrinoConnectorExternalTable.java`](fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/TrinoConnectorExternalTable.java) +- 添加 `shouldIncludeHiddenColumns()` 方法 +- 修改 `initSchema()` 支持 Kafka 隐藏列 + +### 步骤 2: 创建 Kafka Offset 类 + +- 文件: `fe-core/src/main/java/org/apache/doris/job/offset/kafka/KafkaPartitionOffset.java` +- 文件: `fe-core/src/main/java/org/apache/doris/job/offset/kafka/KafkaOffset.java` + +### 步骤 3: 创建 KafkaSourceOffsetProvider + +- 文件: `fe-core/src/main/java/org/apache/doris/job/offset/kafka/KafkaSourceOffsetProvider.java` +- 实现多分区 offset 管理 +- 实现 `rewriteTvfParams()` 生成带 partition 和 offset 过滤的 SQL + +### 步骤 4: 创建 kafka() TVF + +- 文件: `fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/Kafka.java` +- 文件: `fe-core/src/main/java/org/apache/doris/tablefunction/KafkaTableValuedFunction.java` +- 验证依赖的 Trino Kafka Catalog + +### 步骤 5: 改造 StreamingInsertJob 支持多任务 + +- 文件: [`StreamingInsertJob.java`](fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java) +- 添加 `runningPartitionTasks` 映射 +- 实现 `createKafkaPartitionTasks()` 方法 +- 实现 `onPartitionTaskSuccess()` 回调 + +### 步骤 6: 改造 StreamingInsertTask + +- 文件: [`StreamingInsertTask.java`](fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java) +- 添加 `KafkaPartitionOffset` 支持 +- 修改 `onSuccess()` 回调逻辑 + +### 步骤 7: 注册 Provider + +- 文件: [`SourceOffsetProviderFactory.java`](fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProviderFactory.java) +- 添加 `map.put("kafka", KafkaSourceOffsetProvider.class)` + +### 步骤 8: 添加单元测试 + +- 测试 KafkaOffset 序列化/反序列化 +- 测试多分区 offset 计算逻辑 +- 测试多任务并行执行 + +## 6. 注意事项 + +### 6.1 Exactly-Once 保证 + +- 数据写入和 offset 更新在同一个 Doris 事务中完成 +- 通过 `StreamingTaskTxnCommitAttachment` 将 offset 附加到事务 +- 事务回滚时 offset 不会更新,确保不丢数据 + +### 6.2 依赖关系 + +- `kafka()` TVF 依赖 TrinoConnectorExternalCatalog 的 Kafka Connector +- 用户必须先创建 Trino Kafka Catalog +- Kafka Connector 必须配置 `kafka.hide-internal-columns=false` 以暴露隐藏列 + +### 6.3 性能考虑 + +- 多分区并行执行可以提高吞吐量 +- `max_batch_rows` 应根据实际情况调整 +- 过大的批次可能导致单次任务执行时间过长 + +### 6.4 错误处理(第一期简化) + +- 任何分区任务失败,整个 Job 进入 PAUSED 状态 +- 可通过 `RESUME JOB` 手动恢复 +- 失败的分区会从上次成功的 offset 重新开始 \ No newline at end of file diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinTableValuedFunctions.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinTableValuedFunctions.java index a2fb673b12f220..137082f359853c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinTableValuedFunctions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/BuiltinTableValuedFunctions.java @@ -29,6 +29,7 @@ import org.apache.doris.nereids.trees.expressions.functions.table.HudiMeta; import org.apache.doris.nereids.trees.expressions.functions.table.IcebergMeta; import org.apache.doris.nereids.trees.expressions.functions.table.Jobs; +import org.apache.doris.nereids.trees.expressions.functions.table.Kafka; import org.apache.doris.nereids.trees.expressions.functions.table.Local; import org.apache.doris.nereids.trees.expressions.functions.table.MvInfos; import org.apache.doris.nereids.trees.expressions.functions.table.Numbers; @@ -66,6 +67,7 @@ public class BuiltinTableValuedFunctions implements FunctionHelper { tableValued(HttpStream.class, "http_stream"), tableValued(Numbers.class, "numbers"), tableValued(S3.class, "s3"), + tableValued(Kafka.class, "kafka"), tableValued(MvInfos.class, "mv_infos"), tableValued(Partitions.class, "partitions"), tableValued(Jobs.class, "jobs"), diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/TrinoConnectorExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/TrinoConnectorExternalCatalog.java index 3627aff636db77..5434a4ae3a85ff 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/TrinoConnectorExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/TrinoConnectorExternalCatalog.java @@ -17,6 +17,7 @@ package org.apache.doris.datasource.trinoconnector; +import com.google.common.base.Preconditions; import org.apache.doris.common.DdlException; import org.apache.doris.datasource.CatalogProperty; import org.apache.doris.datasource.ExternalCatalog; @@ -79,6 +80,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.io.IOException; import java.time.ZoneId; import java.util.ArrayList; import java.util.Arrays; @@ -112,6 +114,19 @@ public TrinoConnectorExternalCatalog(long catalogId, String name, String resourc super(catalogId, name, Type.TRINO_CONNECTOR, comment); Objects.requireNonNull(name, "catalogName is null"); catalogProperty = new CatalogProperty(resource, props); + initCatalogName(); + } + + private void initCatalogName() { + String connectorNameString = catalogProperty.getProperties().get("trino.connector.name"); + Objects.requireNonNull(connectorNameString, "connectorName is null"); + if (connectorNameString.indexOf('-') >= 0) { + String deprecatedConnectorName = connectorNameString; + connectorNameString = connectorNameString.replace('-', '_'); + LOG.warn("You are using the deprecated connector name '{}'. The correct connector name is '{}'", + deprecatedConnectorName, connectorNameString); + } + this.connectorName = new ConnectorName(connectorNameString); } @Override @@ -199,17 +214,11 @@ private ConnectorServicesProvider createConnectorServicesProvider() { } Map trinoConnectorProperties = new HashMap<>(); trinoConnectorProperties.putAll(trinoProperties); + // remove connector.name, which is not part of trino property String connectorNameString = trinoConnectorProperties.remove("connector.name"); Objects.requireNonNull(connectorNameString, "connectorName is null"); - if (connectorNameString.indexOf('-') >= 0) { - String deprecatedConnectorName = connectorNameString; - connectorNameString = connectorNameString.replace('-', '_'); - LOG.warn("You are using the deprecated connector name '{}'. The correct connector name is '{}'", - deprecatedConnectorName, connectorNameString); - } - - this.connectorName = new ConnectorName(connectorNameString); - + // connectorName is already init in constructor + Preconditions.checkNotNull(this.connectorName); // 2. create CatalogFactory LazyCatalogFactory catalogFactory = new LazyCatalogFactory(); NoOpTransactionManager noOpTransactionManager = new NoOpTransactionManager(); @@ -327,4 +336,10 @@ public CatalogHandle getTrinoCatalogHandle() { public Session getTrinoSession() { return trinoSession; } + + @Override + public void gsonPostProcess() throws IOException { + super.gsonPostProcess(); + initCatalogName(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/TrinoConnectorExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/TrinoConnectorExternalTable.java index 20e82d0b53735b..4b523b712d2017 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/TrinoConnectorExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/TrinoConnectorExternalTable.java @@ -125,10 +125,13 @@ public Optional initSchema() { // 5. Get ColumnMetadata ImmutableMap.Builder columnMetadataMapBuilder = ImmutableMap.builder(); List columns = Lists.newArrayListWithCapacity(columnHandleMap.size()); + boolean includeHiddenColumns = shouldIncludeHiddenColumns(trinoConnectorCatalog); for (ColumnHandle columnHandle : columnHandleMap.values()) { ColumnMetadata columnMetadata = connectorMetadata.getColumnMetadata(connectorSession, connectorTableHandle.get(), columnHandle); - if (columnMetadata.isHidden()) { + // Skip hidden columns unless explicitly configured to include them + // Kafka connector needs hidden columns (_partition, _offset, etc.) for offset filtering + if (columnMetadata.isHidden() && !includeHiddenColumns) { continue; } columnMetadataMapBuilder.put(columnMetadata.getName(), columnMetadata); @@ -229,6 +232,37 @@ private int getMaxDatetimePrecision(int precision) { return Math.min(precision, 6); } + /** + * Determine whether hidden columns should be included in the schema. + * + * For Kafka connector, hidden columns like _partition, _offset, _timestamp are needed + * for offset filtering in streaming jobs. + * + * @param trinoConnectorCatalog the Trino connector catalog + * @return true if hidden columns should be included, false otherwise + */ + private boolean shouldIncludeHiddenColumns(TrinoConnectorExternalCatalog trinoConnectorCatalog) { + // Get connector name to check if it's Kafka + String connectorNameStr = trinoConnectorCatalog.getConnectorName() != null + ? trinoConnectorCatalog.getConnectorName().toString() : ""; + Map props = trinoConnectorCatalog.getCatalogProperty().getProperties(); + // Kafka connector needs hidden columns for offset filtering + if ("kafka".equalsIgnoreCase(connectorNameStr)) { + if (props.containsKey("trino.kafka.hide-internal-columns")) { + String hideInternal = props.get("trino.kafka.hide-internal-columns"); + if ("false".equalsIgnoreCase(hideInternal)) { + return true; + } else { + return false; + } + } else { + return false; + } + } else { + return false; + } + } + public ConnectorTableHandle getConnectorTableHandle() { makeSureInitialized(); Optional schemaCacheValue = getSchemaCacheValue(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/source/TrinoConnectorSource.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/source/TrinoConnectorSource.java index 20dcf996595a48..6ed0fd89e92a16 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/source/TrinoConnectorSource.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/trinoconnector/source/TrinoConnectorSource.java @@ -18,9 +18,12 @@ package org.apache.doris.datasource.trinoconnector.source; import org.apache.doris.analysis.TupleDescriptor; +import org.apache.doris.catalog.FunctionGenTable; +import org.apache.doris.catalog.TableIf; import org.apache.doris.common.UserException; import org.apache.doris.datasource.trinoconnector.TrinoConnectorExternalCatalog; import org.apache.doris.datasource.trinoconnector.TrinoConnectorExternalTable; +import org.apache.doris.tablefunction.KafkaTableValuedFunction; import org.apache.doris.thrift.TFileAttributes; import io.trino.Session; @@ -45,7 +48,12 @@ public class TrinoConnectorSource { public TrinoConnectorSource(TupleDescriptor desc) { this.desc = desc; - this.trinoConnectorExtTable = (TrinoConnectorExternalTable) desc.getTable(); + TableIf table = desc.getTable(); + if (table instanceof FunctionGenTable) { + this.trinoConnectorExtTable = ((KafkaTableValuedFunction)((FunctionGenTable) table).getTvf()).getTrinoTable(); + } else { + this.trinoConnectorExtTable = (TrinoConnectorExternalTable) desc.getTable(); + } this.trinoConnectorExternalCatalog = (TrinoConnectorExternalCatalog) trinoConnectorExtTable.getCatalog(); this.catalogHandle = trinoConnectorExternalCatalog.getTrinoCatalogHandle(); this.trinoConnectorTableHandle = trinoConnectorExtTable.getConnectorTableHandle(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java index 4dade2b4ec8b11..7c55e61816bbdb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertJob.java @@ -51,6 +51,8 @@ import org.apache.doris.job.offset.SourceOffsetProvider; import org.apache.doris.job.offset.SourceOffsetProviderFactory; import org.apache.doris.job.offset.jdbc.JdbcSourceOffsetProvider; +import org.apache.doris.job.offset.kafka.KafkaPartitionOffset; +import org.apache.doris.job.offset.kafka.KafkaSourceOffsetProvider; import org.apache.doris.job.util.StreamingJobUtils; import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.load.loadv2.LoadStatistic; @@ -93,6 +95,7 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -128,6 +131,20 @@ public class StreamingInsertJob extends AbstractJob originTvfProps; @Getter AbstractStreamingTask runningStreamTask; + + /** + * Running Kafka partition tasks (partitionId -> task). + * Used for Kafka streaming jobs where each partition has its own task. + */ + private Map runningKafkaPartitionTasks = new ConcurrentHashMap<>(); + + /** + * Active partitions that have running pipelines. + * A partition is active when it has a task running or pending. + * A partition becomes idle when it has no more data to consume. + */ + private java.util.Set activePartitions = ConcurrentHashMap.newKeySet(); + SourceOffsetProvider offsetProvider; @Getter @Setter @@ -280,6 +297,14 @@ private void initInsertJob() { this.tvfType = currentTvf.getFunctionName(); this.originTvfProps = currentTvf.getProperties().getMap(); this.offsetProvider = SourceOffsetProviderFactory.createSourceOffsetProvider(currentTvf.getFunctionName()); + + // For Kafka jobs, initialize the KafkaSourceOffsetProvider with TVF properties + if (offsetProvider instanceof KafkaSourceOffsetProvider) { + KafkaSourceOffsetProvider kafkaProvider = (KafkaSourceOffsetProvider) offsetProvider; + kafkaProvider.setJobId(getJobId()); + kafkaProvider.initFromTvfProperties(originTvfProps); + } + // validate offset props if (jobProperties.getOffsetProperty() != null) { Offset offset = validateOffset(jobProperties.getOffsetProperty()); @@ -304,9 +329,9 @@ private void writeLock() { } private void writeUnlock() { - if (lock.writeLock().isHeldByCurrentThread()) { - lock.writeLock().unlock(); - } + // if (lock.writeLock().isHeldByCurrentThread()) { + lock.writeLock().unlock(); + //} } private UnboundTVFRelation getCurrentTvf() { @@ -406,11 +431,20 @@ public void resetFailureInfo(FailureReason reason) { public void cancelAllTasks(boolean needWaitCancelComplete) throws JobException { lock.writeLock().lock(); try { - if (runningStreamTask == null) { - return; + // Cancel single running task + if (runningStreamTask != null) { + runningStreamTask.cancel(needWaitCancelComplete); + canceledTaskCount.incrementAndGet(); + } + + // Cancel all Kafka partition tasks + if (!runningKafkaPartitionTasks.isEmpty()) { + for (AbstractStreamingTask task : runningKafkaPartitionTasks.values()) { + task.cancel(needWaitCancelComplete); + canceledTaskCount.incrementAndGet(); + } + runningKafkaPartitionTasks.clear(); } - runningStreamTask.cancel(needWaitCancelComplete); - canceledTaskCount.incrementAndGet(); } finally { lock.writeLock().unlock(); } @@ -449,6 +483,19 @@ public List createTasks(TaskType taskType, Map kafkaTasks = createKafkaPartitionTasks(); + if (!kafkaTasks.isEmpty()) { + // Return the first task for backward compatibility + // All tasks are registered and scheduled + this.runningStreamTask = kafkaTasks.get(0); + return this.runningStreamTask; + } + return null; + } + + // For other sources (S3, JDBC, etc.), create a single task if (tvfType != null) { this.runningStreamTask = createStreamingInsertTask(); } else { @@ -479,6 +526,222 @@ protected AbstractStreamingTask createStreamingInsertTask() { getExecuteSql(), offsetProvider, getCurrentDbName(), jobProperties, originTvfProps, getCreateUser()); } + + /** + * Create multiple Kafka partition tasks for parallel execution (initial creation). + * Each task handles exactly one partition for exactly-once semantics. + * Called during PENDING state to start all partition pipelines. + * + * @return list of created tasks, or empty list if no data to consume + */ + protected List createKafkaPartitionTasks() { + List tasks = new ArrayList<>(); + + if (!(offsetProvider instanceof KafkaSourceOffsetProvider)) { + log.warn("createKafkaPartitionTasks called but offsetProvider is not KafkaSourceOffsetProvider"); + return tasks; + } + + KafkaSourceOffsetProvider kafkaProvider = (KafkaSourceOffsetProvider) offsetProvider; + List partitionOffsets = kafkaProvider.getNextPartitionOffsets(jobProperties); + + if (partitionOffsets.isEmpty()) { + log.info("No Kafka partitions to consume for job {}", getJobId()); + return tasks; + } + + if (originTvfProps == null) { + this.originTvfProps = getCurrentTvf().getProperties().getMap(); + } + + // Clear previous state + runningKafkaPartitionTasks.clear(); + activePartitions.clear(); + + for (KafkaPartitionOffset partitionOffset : partitionOffsets) { + StreamingInsertTask task = createTaskForPartition(partitionOffset); + if (task != null) { + tasks.add(task); + } + } + + log.info("Created {} Kafka partition tasks for job {}, active partitions: {}", + tasks.size(), getJobId(), activePartitions); + + return tasks; + } + + /** + * Create a task for a specific partition offset. + * + * @param partitionOffset the partition offset range + * @return the created task, or null if failed + */ + private StreamingInsertTask createTaskForPartition(KafkaPartitionOffset partitionOffset) { + int partitionId = partitionOffset.getPartitionId(); + + StreamingInsertTask task = new StreamingInsertTask( + getJobId(), + Env.getCurrentEnv().getNextId(), + getExecuteSql(), + offsetProvider, + getCurrentDbName(), + jobProperties, + originTvfProps, + getCreateUser(), + partitionOffset + ); + + Env.getCurrentEnv().getJobManager().getStreamingTaskManager().registerTask(task); + task.setStatus(TaskStatus.PENDING); + runningKafkaPartitionTasks.put(partitionId, task); + activePartitions.add(partitionId); + recordTasks(task); + + log.info("Created Kafka partition task {} for job {}, partition {}, offset range [{}, {})", + task.getTaskId(), getJobId(), partitionId, + partitionOffset.getStartOffset(), partitionOffset.getEndOffset()); + + return task; + } + + /** + * Create the next task for a specific partition after the previous task completes. + * This is the core of the independent pipeline model - each partition creates + * its own next task without waiting for other partitions. + * + * @param partitionId the Kafka partition ID + * @return the created task, or null if no more data for this partition + */ + protected StreamingInsertTask createNextTaskForPartition(int partitionId) { + if (!(offsetProvider instanceof KafkaSourceOffsetProvider)) { + log.warn("createNextTaskForPartition called but offsetProvider is not KafkaSourceOffsetProvider"); + return null; + } + + KafkaSourceOffsetProvider kafkaProvider = (KafkaSourceOffsetProvider) offsetProvider; + + // Get the next offset range for this partition + KafkaPartitionOffset nextOffset = kafkaProvider.getNextPartitionOffset(partitionId, jobProperties); + + if (nextOffset == null || nextOffset.isEmpty()) { + // No more data for this partition, mark it as idle + activePartitions.remove(partitionId); + log.info("Partition {} has no more data, stopping its pipeline. Active partitions: {}", + partitionId, activePartitions); + return null; + } + + if (originTvfProps == null) { + this.originTvfProps = getCurrentTvf().getProperties().getMap(); + } + + return createTaskForPartition(nextOffset); + } + + /** + * Callback when a Kafka partition task completes successfully. + * Updates the partition's offset and immediately creates the next task for this partition. + * + * This implements the independent pipeline model where each partition runs its own + * continuous pipeline without waiting for other partitions. + * + * @param task the completed task + * @param partitionId the Kafka partition ID + * @param consumedRows number of rows consumed by this task + */ + public void onKafkaPartitionTaskSuccess(StreamingInsertTask task, int partitionId, long consumedRows) { + writeLock(); + try { + resetFailureInfo(null); + succeedTaskCount.incrementAndGet(); + + // Remove from running tasks + Env.getCurrentEnv().getJobManager().getStreamingTaskManager().removeRunningTask(task); + runningKafkaPartitionTasks.remove(partitionId); + + // Update partition offset + if (offsetProvider instanceof KafkaSourceOffsetProvider) { + KafkaSourceOffsetProvider kafkaProvider = (KafkaSourceOffsetProvider) offsetProvider; + KafkaPartitionOffset partitionOffset = task.getKafkaPartitionOffset(); + long newOffset = partitionOffset.getStartOffset() + consumedRows; + kafkaProvider.updatePartitionOffset(partitionId, newOffset); + + log.info("Kafka partition {} task {} success, consumed {} rows, new offset: {}", + partitionId, task.getTaskId(), consumedRows, newOffset); + } + + // Immediately create the next task for this partition (independent pipeline) + StreamingInsertTask nextTask = createNextTaskForPartition(partitionId); + if (nextTask != null) { + log.info("Created next task {} for partition {} (independent pipeline)", + nextTask.getTaskId(), partitionId); + } else { + log.info("Partition {} pipeline stopped (no more data), active partitions: {}", + partitionId, activePartitions); + } + } finally { + writeUnlock(); + } + } + + /** + * Restart idle Kafka partitions that have new data available. + * Called by StreamingJobSchedulerTask during RUNNING state to check for + * partitions that stopped due to no data but now have new messages. + */ + public void restartIdleKafkaPartitions() { + if (!(offsetProvider instanceof KafkaSourceOffsetProvider)) { + return; + } + + KafkaSourceOffsetProvider kafkaProvider = (KafkaSourceOffsetProvider) offsetProvider; + java.util.Set allPartitions = kafkaProvider.getAllPartitionIds(); + + int restartedCount = 0; + for (Integer partitionId : allPartitions) { + // Skip partitions that are already active + if (activePartitions.contains(partitionId)) { + continue; + } + + // Check if this idle partition now has new data + if (kafkaProvider.hasMoreDataForPartition(partitionId)) { + writeLock(); + try { + // Double-check after acquiring lock + if (!activePartitions.contains(partitionId)) { + StreamingInsertTask task = createNextTaskForPartition(partitionId); + if (task != null) { + restartedCount++; + log.info("Restarted idle partition {} with new task {}", + partitionId, task.getTaskId()); + } + } + } finally { + writeUnlock(); + } + } + } + + if (restartedCount > 0) { + log.info("Restarted {} idle Kafka partitions for job {}", restartedCount, getJobId()); + } + } + + /** + * Get the set of currently active partitions. + */ + public java.util.Set getActivePartitions() { + return java.util.Collections.unmodifiableSet(activePartitions); + } + + /** + * Check if this is a Kafka streaming job. + */ + public boolean isKafkaStreamingJob() { + return "kafka".equalsIgnoreCase(tvfType); + } public void recordTasks(AbstractStreamingTask task) { if (Config.max_streaming_task_show_count < 1) { @@ -511,6 +774,13 @@ protected void fetchMeta() throws JobException { if (originTvfProps == null) { this.originTvfProps = getCurrentTvf().getProperties().getMap(); } + if (offsetProvider instanceof KafkaSourceOffsetProvider && originTvfProps != null) { + KafkaSourceOffsetProvider kafkaProvider = (KafkaSourceOffsetProvider) offsetProvider; + if (!kafkaProvider.isInitialized()) { + kafkaProvider.setJobId(getJobId()); + kafkaProvider.initFromTvfProperties(originTvfProps); + } + } offsetProvider.fetchRemoteMeta(originTvfProps); } else { offsetProvider.fetchRemoteMeta(new HashMap<>()); @@ -542,12 +812,27 @@ public boolean needScheduleTask() { } public void clearRunningStreamTask(JobStatus newJobStatus) { + boolean needWait = !JobStatus.STOPPED.equals(newJobStatus); + + // Clear single running task if (runningStreamTask != null) { log.info("clear running streaming insert task for job {}, task {}, status {} ", getJobId(), runningStreamTask.getTaskId(), runningStreamTask.getStatus()); - runningStreamTask.cancel(JobStatus.STOPPED.equals(newJobStatus) ? false : true); + runningStreamTask.cancel(needWait); runningStreamTask.closeOrReleaseResources(); } + + // Clear all Kafka partition tasks + if (!runningKafkaPartitionTasks.isEmpty()) { + for (AbstractStreamingTask task : runningKafkaPartitionTasks.values()) { + log.info("clear running Kafka partition task for job {}, task {}, status {} ", + getJobId(), task.getTaskId(), task.getStatus()); + task.cancel(needWait); + task.closeOrReleaseResources(); + } + runningKafkaPartitionTasks.clear(); + activePartitions.clear(); + } } public boolean hasMoreDataToConsume() { @@ -570,6 +855,7 @@ public void onTaskSuccess(StreamingJobSchedulerTask task) throws JobException { } public void onStreamTaskFail(AbstractStreamingTask task) throws JobException { + writeLock(); try { failedTaskCount.incrementAndGet(); Env.getCurrentEnv().getJobManager().getStreamingTaskManager().removeRunningTask(task); @@ -581,6 +867,7 @@ public void onStreamTaskFail(AbstractStreamingTask task) throws JobException { } public void onStreamTaskSuccess(AbstractStreamingTask task) { + writeLock(); try { resetFailureInfo(null); succeedTaskCount.incrementAndGet(); @@ -602,7 +889,12 @@ private void updateJobStatisticAndOffset(StreamingTaskTxnCommitAttachment attach this.jobStatistic.setLoadBytes(this.jobStatistic.getLoadBytes() + attachment.getLoadBytes()); this.jobStatistic.setFileNumber(this.jobStatistic.getFileNumber() + attachment.getNumFiles()); this.jobStatistic.setFileSize(this.jobStatistic.getFileSize() + attachment.getFileBytes()); - offsetProvider.updateOffset(offsetProvider.deserializeOffset(attachment.getOffset())); + log.info("[KAFKA_OFFSET_DEBUG] updateJobStatisticAndOffset: jobId={}, about to deserialize offset from attachment", + getJobId()); + Offset deserializedOffset = offsetProvider.deserializeOffset(attachment.getOffset()); + log.info("[KAFKA_OFFSET_DEBUG] updateJobStatisticAndOffset: jobId={}, deserialized offset type={}, calling updateOffset", + getJobId(), deserializedOffset != null ? deserializedOffset.getClass().getSimpleName() : "null"); + offsetProvider.updateOffset(deserializedOffset); } private void updateCloudJobStatisticAndOffset(StreamingTaskTxnCommitAttachment attachment) { @@ -653,6 +945,9 @@ public void onReplayCreate() throws JobException { * @param replayJob */ public void replayOnUpdated(StreamingInsertJob replayJob) { + log.info("[KAFKA_OFFSET_DEBUG] replayOnUpdated started: jobId={}, replayJob.offsetProviderPersist={}", + getJobId(), + replayJob.getOffsetProviderPersist() != null ? "has value (length=" + replayJob.getOffsetProviderPersist().length() + ")" : "null"); if (!JobStatus.RUNNING.equals(replayJob.getJobStatus())) { // No need to restore in the running state, as scheduling relies on pending states. // insert TVF does not persist the running state. @@ -670,7 +965,12 @@ public void replayOnUpdated(StreamingInsertJob replayJob) { log.error("replay modify streaming insert job properties failed, job id: {}", getJobId(), e); } if (replayJob.getOffsetProviderPersist() != null) { + log.info("[KAFKA_OFFSET_DEBUG] replayOnUpdated: jobId={}, setting offsetProviderPersist from replayJob", + getJobId()); setOffsetProviderPersist(replayJob.getOffsetProviderPersist()); + } else { + log.info("[KAFKA_OFFSET_DEBUG] replayOnUpdated: jobId={}, replayJob.offsetProviderPersist is null, NOT setting", + getJobId()); } if (replayJob.getNonTxnJobStatistic() != null) { setNonTxnJobStatistic(replayJob.getNonTxnJobStatistic()); @@ -679,6 +979,7 @@ public void replayOnUpdated(StreamingInsertJob replayJob) { setSucceedTaskCount(replayJob.getSucceedTaskCount()); setFailedTaskCount(replayJob.getFailedTaskCount()); setCanceledTaskCount(replayJob.getCanceledTaskCount()); + log.info("[KAFKA_OFFSET_DEBUG] replayOnUpdated completed: jobId={}", getJobId()); } /** @@ -923,33 +1224,87 @@ public long getId() { @Override public void beforeCommitted(TransactionState txnState) throws TransactionException { - boolean shouldReleaseLock = false; + // boolean shouldReleaseLock = false; + boolean shouldReleaseLock = true; writeLock(); try { - if (runningStreamTask.getIsCanceled().get()) { + // For Kafka streaming jobs with multiple partition tasks, + // offset is managed in onKafkaPartitionTaskSuccess() instead. + // Here we need to find the correct task from runningKafkaPartitionTasks. + AbstractStreamingTask currentTask = runningStreamTask; + // for kafka job, runningStreamTask is meaning less, find task in runningKafkaPartitionTasks + if (isKafkaStreamingJob() && !runningKafkaPartitionTasks.isEmpty()) { + // Try to find a running task from Kafka partition tasks + // This handles the case where runningStreamTask is not set + for (AbstractStreamingTask task : runningKafkaPartitionTasks.values()) { + if (task != null && !task.getIsCanceled().get() + && task.getLabelName().equals(txnState.getLabel())) { + currentTask = task; + break; + } + } + } + + if (currentTask == null) { + log.warn("No running task found for job {} in beforeCommitted, skipping", getJobId()); + return; + } + + if (currentTask.getIsCanceled().get()) { log.info("streaming insert job {} task {} is canceled, skip beforeCommitted", - getJobId(), runningStreamTask.getTaskId()); + getJobId(), currentTask.getTaskId()); return; } ArrayList taskIds = new ArrayList<>(); - taskIds.add(runningStreamTask.getTaskId()); + taskIds.add(currentTask.getTaskId()); // todo: Check whether the taskid of runningtask is consistent with the taskid associated with txn List loadJobs = Env.getCurrentEnv().getLoadManager().queryLoadJobsByJobIds(taskIds); if (loadJobs.size() == 0) { - throw new TransactionException("load job not found, insert job id is " + runningStreamTask.getTaskId()); + throw new TransactionException("load job not found, insert job id is " + currentTask.getTaskId()); } LoadJob loadJob = loadJobs.get(0); LoadStatistic loadStatistic = loadJob.getLoadStatistic(); + + // CRITICAL FIX: For KafkaPartitionOffset, we need to update consumedRows with actual scanned rows + // Otherwise consumedRows stays at 0 (initialized in constructor) and offset won't advance + Offset runningOffset = currentTask.getRunningOffset(); + String offsetJson; + + if (runningOffset instanceof KafkaPartitionOffset) { + KafkaPartitionOffset kafkaOffset = (KafkaPartitionOffset) runningOffset; + // Create a new KafkaPartitionOffset with the actual consumed rows + KafkaPartitionOffset updatedOffset = new KafkaPartitionOffset( + kafkaOffset.getPartitionId(), + kafkaOffset.getStartOffset(), + kafkaOffset.getEndOffset()); + updatedOffset.setConsumedRows(loadStatistic.getScannedRows()); + offsetJson = updatedOffset.toSerializedJson(); + + log.info("[KAFKA_OFFSET_DEBUG] <<>> beforeCommitted: jobId={}, taskId={}, partition={}, startOffset={}, endOffset={}, scannedRows={}, UPDATED consumedRows={} <<>>", + getJobId(), currentTask.getTaskId(), + kafkaOffset.getPartitionId(), + kafkaOffset.getStartOffset(), + kafkaOffset.getEndOffset(), + loadStatistic.getScannedRows(), + updatedOffset.getConsumedRows()); + } else { + offsetJson = runningOffset.toSerializedJson(); + log.info("[KAFKA_OFFSET_DEBUG] beforeCommitted: jobId={}, taskId={}, offsetType={}, offsetJson='{}'", + getJobId(), currentTask.getTaskId(), + runningOffset.getClass().getSimpleName(), + offsetJson); + } + txnState.setTxnCommitAttachment(new StreamingTaskTxnCommitAttachment( getJobId(), - runningStreamTask.getTaskId(), + currentTask.getTaskId(), loadStatistic.getScannedRows(), loadStatistic.getLoadBytes(), loadStatistic.getFileNumber(), loadStatistic.getTotalFileSizeB(), - runningStreamTask.getRunningOffset().toSerializedJson())); + offsetJson)); } finally { if (shouldReleaseLock) { writeUnlock(); @@ -966,7 +1321,12 @@ public void afterCommitted(TransactionState txnState, boolean txnOperated) throw Preconditions.checkNotNull(txnState.getTxnCommitAttachment(), txnState); StreamingTaskTxnCommitAttachment attachment = (StreamingTaskTxnCommitAttachment) txnState.getTxnCommitAttachment(); + log.info("[KAFKA_OFFSET_DEBUG] afterCommitted: jobId={}, taskId={}, attachmentOffset='{}'", + getJobId(), attachment.getTaskId(), attachment.getOffset()); updateJobStatisticAndOffset(attachment); + // Persist offset to ensure recovery after FE restart + persistOffsetProviderIfNeed(); + log.info("[KAFKA_OFFSET_DEBUG] afterCommitted completed, persistOffsetProviderIfNeed called"); } @Override @@ -976,6 +1336,8 @@ public void replayOnCommitted(TransactionState txnState) { (StreamingTaskTxnCommitAttachment) txnState.getTxnCommitAttachment(); updateJobStatisticAndOffset(attachment); succeedTaskCount.incrementAndGet(); + // Persist offset during replay to keep offsetProviderPersist in sync + persistOffsetProviderIfNeed(); } public long getDbId() { @@ -1040,9 +1402,14 @@ public void replayOnVisible(TransactionState txnState) { @Override public void gsonPostProcess() throws IOException { + log.info("[KAFKA_OFFSET_DEBUG] gsonPostProcess started: jobId={}, tvfType={}, offsetProviderPersist={}", + getJobId(), tvfType, + offsetProviderPersist != null ? "has value (length=" + offsetProviderPersist.length() + ")" : "null"); if (offsetProvider == null) { if (tvfType != null) { offsetProvider = SourceOffsetProviderFactory.createSourceOffsetProvider(tvfType); + log.info("[KAFKA_OFFSET_DEBUG] gsonPostProcess: created offsetProvider of type {}", + offsetProvider.getClass().getSimpleName()); } else { offsetProvider = new JdbcSourceOffsetProvider(getJobId(), dataSourceType, sourceProperties); } @@ -1069,6 +1436,17 @@ public void gsonPostProcess() throws IOException { if (null == lock) { this.lock = new ReentrantReadWriteLock(true); } + + // Initialize Kafka partition task map + if (null == runningKafkaPartitionTasks) { + runningKafkaPartitionTasks = new ConcurrentHashMap<>(); + } + + // Initialize active partitions set + if (null == activePartitions) { + activePartitions = ConcurrentHashMap.newKeySet(); + } + log.info("[KAFKA_OFFSET_DEBUG] gsonPostProcess completed: jobId={}", getJobId()); } /** @@ -1125,10 +1503,18 @@ public void commitOffset(CommitOffsetRequest offsetRequest) throws JobException } private void persistOffsetProviderIfNeed() { - // only for jdbc + log.info("[KAFKA_OFFSET_DEBUG] persistOffsetProviderIfNeed called: jobId={}", getJobId()); + // Persist offset for Kafka and JDBC sources this.offsetProviderPersist = offsetProvider.getPersistInfo(); + log.info("[KAFKA_OFFSET_DEBUG] persistOffsetProviderIfNeed: jobId={}, offsetProviderPersist={}", + getJobId(), offsetProviderPersist != null ? "has value (length=" + offsetProviderPersist.length() + ")" : "null"); if (this.offsetProviderPersist != null) { + log.info("[KAFKA_OFFSET_DEBUG] persistOffsetProviderIfNeed: jobId={}, calling logUpdateOperation to persist", + getJobId()); logUpdateOperation(); + } else { + log.info("[KAFKA_OFFSET_DEBUG] persistOffsetProviderIfNeed: jobId={}, offsetProviderPersist is null, NOT persisting", + getJobId()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java index 635c639256c276..3a382941b184f7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingInsertTask.java @@ -26,7 +26,9 @@ import org.apache.doris.job.common.TaskStatus; import org.apache.doris.job.exception.JobException; import org.apache.doris.job.extensions.insert.InsertTask; +import org.apache.doris.job.offset.Offset; import org.apache.doris.job.offset.SourceOffsetProvider; +import org.apache.doris.job.offset.kafka.KafkaPartitionOffset; import org.apache.doris.load.loadv2.LoadJob; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.glue.LogicalPlanAdapter; @@ -58,6 +60,12 @@ public class StreamingInsertTask extends AbstractStreamingTask { private StreamingJobProperties jobProperties; private Map originTvfProps; SourceOffsetProvider offsetProvider; + + /** + * Kafka partition offset for this task (only for Kafka streaming jobs). + * When set, this task handles a specific partition with the given offset range. + */ + private KafkaPartitionOffset kafkaPartitionOffset; public StreamingInsertTask(long jobId, long taskId, @@ -73,6 +81,29 @@ public StreamingInsertTask(long jobId, this.offsetProvider = offsetProvider; this.jobProperties = jobProperties; this.originTvfProps = originTvfProps; + this.kafkaPartitionOffset = null; + } + + /** + * Constructor for Kafka partition-specific tasks. + * Each task handles exactly one Kafka partition for exactly-once semantics. + */ + public StreamingInsertTask(long jobId, + long taskId, + String sql, + SourceOffsetProvider offsetProvider, + String currentDb, + StreamingJobProperties jobProperties, + Map originTvfProps, + UserIdentity userIdentity, + KafkaPartitionOffset kafkaPartitionOffset) { + super(jobId, taskId, userIdentity); + this.sql = sql; + this.currentDb = currentDb; + this.offsetProvider = offsetProvider; + this.jobProperties = jobProperties; + this.originTvfProps = originTvfProps; + this.kafkaPartitionOffset = kafkaPartitionOffset; } @Override @@ -88,8 +119,16 @@ public void before() throws Exception { StatementContext statementContext = new StatementContext(); ctx.setStatementContext(statementContext); - this.runningOffset = offsetProvider.getNextOffset(jobProperties, originTvfProps); - log.info("streaming insert task {} get running offset: {}", taskId, runningOffset.toString()); + // For Kafka partition tasks, use the pre-assigned partition offset + // For other sources (S3, etc.), get the next offset from the provider + if (kafkaPartitionOffset != null) { + this.runningOffset = kafkaPartitionOffset; + log.info("streaming insert task {} using kafka partition offset: {}", taskId, runningOffset.toString()); + } else { + this.runningOffset = offsetProvider.getNextOffset(jobProperties, originTvfProps); + log.info("streaming insert task {} get running offset: {}", taskId, runningOffset.toString()); + } + InsertIntoTableCommand baseCommand = (InsertIntoTableCommand) new NereidsParser().parseSingle(sql); baseCommand.setJobId(getTaskId()); StmtExecutor baseStmtExecutor = @@ -143,9 +182,34 @@ public boolean onSuccess() throws JobException { } StreamingInsertJob streamingInsertJob = (StreamingInsertJob) job; - streamingInsertJob.onStreamTaskSuccess(this); + + // For Kafka partition tasks, use the partition-specific callback + if (kafkaPartitionOffset != null) { + long consumedRows = getConsumedRowCount(); + streamingInsertJob.onKafkaPartitionTaskSuccess(this, + kafkaPartitionOffset.getPartitionId(), consumedRows); + } else { + // Standard callback for non-Kafka sources + streamingInsertJob.onStreamTaskSuccess(this); + } return true; } + + /** + * Get the number of rows consumed by this task. + * Used for updating Kafka partition offsets after task completion. + */ + public long getConsumedRowCount() { + List loadJobs = Env.getCurrentEnv().getLoadManager() + .queryLoadJobsByJobIds(Arrays.asList(this.getTaskId())); + if (!loadJobs.isEmpty()) { + LoadJob loadJob = loadJobs.get(0); + if (loadJob.getLoadStatistic() != null) { + return loadJob.getLoadStatistic().getScannedRows(); + } + } + return 0; + } @Override protected void onFail(String errMsg) throws JobException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java index 942a25812dbb37..5844a1bbc3d923 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/insert/streaming/StreamingJobSchedulerTask.java @@ -66,14 +66,28 @@ private void handlePendingState() throws JobException { } } streamingInsertJob.replayOffsetProviderIfNeed(); + // For Kafka jobs: fetch remote metadata to initialize partition offsets + // before creating the first batch of tasks + if (streamingInsertJob.isKafkaStreamingJob()) { + streamingInsertJob.fetchMeta(); + } + streamingInsertJob.createStreamingTask(); streamingInsertJob.updateJobStatus(JobStatus.RUNNING); streamingInsertJob.setAutoResumeCount(0); } private void handleRunningState() throws JobException { + // 1. Process timeout tasks (for StreamingMultiTblTask) streamingInsertJob.processTimeoutTasks(); + + // 2. Fetch remote metadata (updates latest offsets for all partitions) streamingInsertJob.fetchMeta(); + + // 3. For Kafka jobs: restart idle partitions that now have new data + if (streamingInsertJob.isKafkaStreamingJob()) { + streamingInsertJob.restartIdleKafkaPartitions(); + } } private void autoResumeHandler() throws JobException { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProviderFactory.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProviderFactory.java index adc49249eb1c8b..0e83f368f511c0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProviderFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/SourceOffsetProviderFactory.java @@ -18,6 +18,7 @@ package org.apache.doris.job.offset; import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.offset.kafka.KafkaSourceOffsetProvider; import org.apache.doris.job.offset.s3.S3SourceOffsetProvider; import lombok.extern.log4j.Log4j2; @@ -31,6 +32,7 @@ public class SourceOffsetProviderFactory { static { map.put("s3", S3SourceOffsetProvider.class); + map.put("kafka", KafkaSourceOffsetProvider.class); } public static SourceOffsetProvider createSourceOffsetProvider(String sourceType) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/kafka/KafkaOffset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/kafka/KafkaOffset.java new file mode 100644 index 00000000000000..8004cbdd24dafa --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/kafka/KafkaOffset.java @@ -0,0 +1,152 @@ +// 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.doris.job.offset.kafka; + +import org.apache.doris.job.offset.Offset; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.gson.annotations.SerializedName; +import lombok.Getter; +import lombok.Setter; + +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Represents the overall Kafka offset state for all partitions of a topic. + * + * This class tracks the current consumption position for each partition + * and is used for persistence and recovery of streaming job state. + */ +@Getter +@Setter +public class KafkaOffset implements Offset { + + /** + * Map of partition ID to current offset (next offset to consume) + */ + @SerializedName("partitionOffsets") + private Map partitionOffsets; + + /** + * The Kafka topic name + */ + @SerializedName("topic") + private String topic; + + /** + * The catalog name for the Trino Kafka connector + */ + @SerializedName("catalogName") + private String catalogName; + + /** + * The database name (schema) in the Trino Kafka connector + */ + @SerializedName("databaseName") + private String databaseName; + + public KafkaOffset() { + this.partitionOffsets = new HashMap<>(); + } + + public KafkaOffset(String topic, String catalogName, String databaseName) { + this.topic = topic; + this.catalogName = catalogName; + this.databaseName = databaseName; + this.partitionOffsets = new HashMap<>(); + } + + /** + * Update the offset for a specific partition. + * + * @param partitionId the partition ID + * @param newOffset the new offset value + */ + public void updatePartitionOffset(int partitionId, long newOffset) { + partitionOffsets.put(partitionId, newOffset); + } + + /** + * Get the current offset for a specific partition. + * + * @param partitionId the partition ID + * @return the current offset, or 0 if not found + */ + public long getPartitionOffset(int partitionId) { + return partitionOffsets.getOrDefault(partitionId, 0L); + } + + /** + * Initialize offsets for a list of partitions. + * + * @param partitionIds list of partition IDs + * @param initialOffset the initial offset value for each partition + */ + public void initializePartitions(Iterable partitionIds, long initialOffset) { + for (Integer partitionId : partitionIds) { + if (!partitionOffsets.containsKey(partitionId)) { + partitionOffsets.put(partitionId, initialOffset); + } + } + } + + @Override + public String toSerializedJson() { + return GsonUtils.GSON.toJson(this); + } + + @Override + public boolean isEmpty() { + return partitionOffsets == null || partitionOffsets.isEmpty(); + } + + @Override + public boolean isValidOffset() { + return topic != null && !topic.isEmpty() && partitionOffsets != null; + } + + @Override + public String showRange() { + if (partitionOffsets == null || partitionOffsets.isEmpty()) { + return "{}"; + } + String offsetsStr = partitionOffsets.entrySet().stream() + .map(e -> String.format("p%d=%d", e.getKey(), e.getValue())) + .collect(Collectors.joining(", ")); + return String.format("{topic=%s, offsets={%s}}", topic, offsetsStr); + } + + @Override + public String toString() { + return String.format("KafkaOffset{topic='%s', catalog='%s', database='%s', partitionOffsets=%s}", + topic, catalogName, databaseName, partitionOffsets); + } + + /** + * Create a deep copy of this KafkaOffset. + * + * @return a new KafkaOffset instance with the same values + */ + public KafkaOffset copy() { + KafkaOffset copy = new KafkaOffset(topic, catalogName, databaseName); + copy.partitionOffsets = new HashMap<>(this.partitionOffsets); + return copy; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/kafka/KafkaPartitionOffset.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/kafka/KafkaPartitionOffset.java new file mode 100644 index 00000000000000..4ce79d24d1de93 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/kafka/KafkaPartitionOffset.java @@ -0,0 +1,105 @@ +// 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.doris.job.offset.kafka; + +import org.apache.doris.job.offset.Offset; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.gson.annotations.SerializedName; +import lombok.Getter; +import lombok.Setter; + +/** + * Represents the offset range for a single Kafka partition in a streaming task. + * + * Each StreamingInsertTask processes exactly one partition to ensure + * exactly-once semantics and precise offset tracking. + */ +@Getter +@Setter +public class KafkaPartitionOffset implements Offset { + + /** + * The Kafka partition ID + */ + @SerializedName("partitionId") + private int partitionId; + + /** + * The starting offset for this task (inclusive) + */ + @SerializedName("startOffset") + private long startOffset; + + /** + * The ending offset for this task (exclusive) + */ + @SerializedName("endOffset") + private long endOffset; + + /** + * The actual number of rows consumed (populated after task completion) + */ + @SerializedName("consumedRows") + private long consumedRows; + + public KafkaPartitionOffset() { + } + + public KafkaPartitionOffset(int partitionId, long startOffset, long endOffset) { + this.partitionId = partitionId; + this.startOffset = startOffset; + this.endOffset = endOffset; + this.consumedRows = 0; + } + + @Override + public String toSerializedJson() { + return GsonUtils.GSON.toJson(this); + } + + @Override + public boolean isEmpty() { + return startOffset >= endOffset; + } + + @Override + public boolean isValidOffset() { + return startOffset >= 0 && endOffset > startOffset; + } + + @Override + public String showRange() { + return String.format("{partition=%d, range=[%d, %d)}", partitionId, startOffset, endOffset); + } + + @Override + public String toString() { + return String.format("{partitionId=%d, startOffset=%d, endOffset=%d, consumedRows=%d}", + partitionId, startOffset, endOffset, consumedRows); + } + + /** + * Calculate the expected number of rows for this partition offset range. + * Note: This is the maximum possible rows, actual consumed rows may be less + * if there are gaps in the offset sequence. + */ + public long getExpectedRows() { + return endOffset - startOffset; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/kafka/KafkaPropertiesConverter.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/kafka/KafkaPropertiesConverter.java new file mode 100644 index 00000000000000..9c2867d3a417ba --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/kafka/KafkaPropertiesConverter.java @@ -0,0 +1,192 @@ +// 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.doris.job.offset.kafka; + +import com.google.common.collect.ImmutableSet; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +/** + * Utility class to convert Trino Kafka connector properties to KafkaUtil parameters. + * + * KafkaUtil was originally designed for KafkaRoutineLoadJob and expects parameters + * in a specific format. This converter bridges the gap between TrinoConnectorExternalCatalog + * properties and KafkaUtil expectations. + * + * Property format conversion: + * - Trino Catalog: trino.kafka.nodes=broker1:9092 + * - KafkaUtil: brokerList=broker1:9092 + */ +public class KafkaPropertiesConverter { + + private static final String TRINO_PREFIX = "trino."; + private static final String KAFKA_PREFIX = "kafka."; + + /** + * Trino-specific properties that should not be passed to the Kafka client. + * These are configuration options used only by Trino's Kafka connector, + * not by the underlying Kafka client. + */ + private static final Set TRINO_ONLY_PROPERTIES = ImmutableSet.of( + "nodes", // Broker list, handled separately + "default-schema", // Trino schema configuration + "table-names", // Trino table names list + "table-description-dir", // Trino table description directory + "hide-internal-columns", // Trino column visibility + "timestamp-upper-bound-force-push-down-enabled", + "internal-column-prefix", // Trino internal column prefix + "messages-per-split" // Trino split configuration + ); + + private KafkaPropertiesConverter() { + // Utility class, no instantiation + } + + /** + * Extract the Kafka broker list from catalog properties. + * + * Looks for the broker list in the following order: + * 1. trino.kafka.nodes + * 2. kafka.nodes + * + * @param catalogProps the catalog properties + * @return the broker list string (e.g., "broker1:9092,broker2:9092") + * @throws IllegalArgumentException if no broker list is found + */ + public static String extractBrokerList(Map catalogProps) { + // Try trino.kafka.nodes first + String brokers = catalogProps.get("trino.kafka.nodes"); + if (brokers != null && !brokers.isEmpty()) { + return brokers; + } + + // Fallback to kafka.nodes (without trino prefix) + brokers = catalogProps.get("kafka.nodes"); + if (brokers != null && !brokers.isEmpty()) { + return brokers; + } + + throw new IllegalArgumentException( + "Missing required property: kafka.nodes or trino.kafka.nodes. " + + "Please configure the Kafka broker list in the Trino Kafka catalog."); + } + + /** + * Extract and convert Kafka client properties from catalog properties. + * + * This method performs the following conversions: + * - Strips the "trino.kafka." prefix from property keys + * - Filters out Trino-specific properties + * - Retains Kafka client properties (security, SASL, etc.) + * + * @param catalogProps the catalog properties + * @return a map of Kafka client properties ready for KafkaUtil + */ + public static Map convertToKafkaClientProperties(Map catalogProps) { + Map kafkaProps = new HashMap<>(); + + String fullPrefix = TRINO_PREFIX + KAFKA_PREFIX; // "trino.kafka." + + for (Map.Entry entry : catalogProps.entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + + if (key.startsWith(fullPrefix)) { + // Remove "trino.kafka." prefix to get the Kafka property name + String kafkaKey = key.substring(fullPrefix.length()); + + // Only include properties that are valid Kafka client properties + if (isKafkaClientProperty(kafkaKey)) { + kafkaProps.put(kafkaKey, value); + } + } + } + + return kafkaProps; + } + + /** + * Check if a property key is a valid Kafka client property. + * + * Returns true for: + * - Security properties (security.protocol, ssl.*, sasl.*) + * - Connection properties (request.timeout.ms, etc.) + * - Any property not in the TRINO_ONLY_PROPERTIES set + * + * @param key the property key (without "trino.kafka." prefix) + * @return true if this is a valid Kafka client property + */ + private static boolean isKafkaClientProperty(String key) { + return !TRINO_ONLY_PROPERTIES.contains(key); + } + + /** + * Convert Trino security protocol format to Kafka client format if needed. + * + * Trino may use different property names for security configuration. + * This method normalizes them to the Kafka client expected format. + * + * @param kafkaProps the Kafka properties map (will be modified in place) + */ + public static void normalizeSecurityProperties(Map kafkaProps) { + // Normalize security protocol property names if needed + // Trino uses "security.protocol", Kafka client also uses "security.protocol" + // This method is reserved for future compatibility needs + + // If any SASL properties are present, ensure security protocol is set + boolean hasSaslConfig = kafkaProps.keySet().stream() + .anyMatch(k -> k.startsWith("sasl.")); + if (hasSaslConfig && !kafkaProps.containsKey("security.protocol")) { + // Default to SASL_PLAINTEXT if SASL config present but no protocol specified + kafkaProps.putIfAbsent("security.protocol", "SASL_PLAINTEXT"); + } + } + + /** + * Validate that all required properties are present for Kafka connection. + * + * @param brokerList the broker list + * @param topic the topic name + * @param kafkaProps additional Kafka properties + * @throws IllegalArgumentException if required properties are missing + */ + public static void validateKafkaProperties(String brokerList, String topic, + Map kafkaProps) { + if (brokerList == null || brokerList.isEmpty()) { + throw new IllegalArgumentException("Kafka broker list is required"); + } + if (topic == null || topic.isEmpty()) { + throw new IllegalArgumentException("Kafka topic is required"); + } + + // Check for valid security configuration + String securityProtocol = kafkaProps.get("security.protocol"); + if (securityProtocol != null) { + String upperProtocol = securityProtocol.toUpperCase(); + if (upperProtocol.contains("SASL")) { + // SASL authentication requires mechanism configuration + if (!kafkaProps.containsKey("sasl.mechanism")) { + throw new IllegalArgumentException( + "SASL authentication requires 'sasl.mechanism' property"); + } + } + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/offset/kafka/KafkaSourceOffsetProvider.java b/fe/fe-core/src/main/java/org/apache/doris/job/offset/kafka/KafkaSourceOffsetProvider.java new file mode 100644 index 00000000000000..df5a38c5bffa1d --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/job/offset/kafka/KafkaSourceOffsetProvider.java @@ -0,0 +1,767 @@ +// 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.doris.job.offset.kafka; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.Pair; +import org.apache.doris.common.UserException; +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.kafka.KafkaUtil; +import org.apache.doris.datasource.trinoconnector.TrinoConnectorExternalCatalog; +import org.apache.doris.job.exception.JobException; +import org.apache.doris.job.extensions.insert.streaming.StreamingInsertJob; +import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; +import org.apache.doris.job.offset.Offset; +import org.apache.doris.job.offset.SourceOffsetProvider; +import org.apache.doris.nereids.analyzer.UnboundRelation; +import org.apache.doris.nereids.analyzer.UnboundSlot; +import org.apache.doris.nereids.analyzer.UnboundTVFRelation; +import org.apache.doris.nereids.trees.expressions.And; +import org.apache.doris.nereids.trees.expressions.EqualTo; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.GreaterThanEqual; +import org.apache.doris.nereids.trees.expressions.LessThan; +import org.apache.doris.nereids.trees.expressions.literal.BigIntLiteral; +import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.persist.gson.GsonUtils; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import lombok.Getter; +import lombok.Setter; +import lombok.extern.log4j.Log4j2; +import org.apache.commons.lang3.StringUtils; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.stream.Collectors; + +/** + * Offset provider for Kafka streaming jobs. + * + * This provider manages offset tracking for each Kafka partition and supports + * exactly-once semantics by generating partition-specific tasks. + */ +@Log4j2 +@Getter +@Setter +public class KafkaSourceOffsetProvider implements SourceOffsetProvider { + + // Kafka TVF parameter names + public static final String PARAM_CATALOG = "catalog"; + public static final String PARAM_DATABASE = "database"; + public static final String PARAM_TABLE = "table"; + public static final String PARAM_DEFAULT_OFFSETS = "kafka_default_offsets"; + public static final String PARAM_MAX_BATCH_ROWS = "max_batch_rows"; + + // Offset constants + public static final String OFFSET_BEGINNING = "OFFSET_BEGINNING"; + public static final String OFFSET_END = "OFFSET_END"; + public static final long OFFSET_BEGINNING_VAL = -2L; + public static final long OFFSET_END_VAL = -1L; + + // Kafka hidden column names + private static final String PARTITION_COLUMN = "_partition_id"; + private static final String OFFSET_COLUMN = "_partition_offset"; + + // Current offset state for all partitions + private KafkaOffset currentOffset; + + // Latest available offsets for each partition (from Kafka) + private Map latestOffsets = new HashMap<>(); + + // Kafka connection parameters + private String brokerList; + private String topic; + private Map kafkaClientProps = new HashMap<>(); + + // Catalog information + private String catalogName; + private String databaseName; + private String tableName; + + // Job configuration + private long maxBatchRows = 100000L; + private String defaultOffsetsConfig = OFFSET_END; + + // Job ID for tracking + private long jobId; + + private boolean isInitialized = false; + + public boolean isInitialized() { + log.info("[KAFKA_OFFSET_DEBUG] <<>> isInitialized() called: returning {} <<>>", isInitialized); + return isInitialized; + } + + @Override + public String getSourceType() { + return "kafka"; + } + + /** + * Initialize the provider from TVF properties. + * This should be called when the job is first created. + */ + public void initFromTvfProperties(Map tvfProps) throws UserException { + log.info("[KAFKA_OFFSET_DEBUG] <<>> initFromTvfProperties called: jobId={}, isInitialized={}, currentOffset={}, partition 0 offset = {} <<>>", + jobId, + isInitialized, + currentOffset != null ? currentOffset.showRange() : "null", + currentOffset != null && currentOffset.getPartitionOffsets() != null ? currentOffset.getPartitionOffsets().get(0) : "null"); + + if (isInitialized) { + log.info("[KAFKA_OFFSET_DEBUG] <<>> initFromTvfProperties: EARLY RETURN because isInitialized=true <<>>"); + return; + } + this.catalogName = getRequiredProperty(tvfProps, PARAM_CATALOG); + this.databaseName = tvfProps.getOrDefault(PARAM_DATABASE, "default"); + this.tableName = getRequiredProperty(tvfProps, PARAM_TABLE); + this.defaultOffsetsConfig = tvfProps.getOrDefault(PARAM_DEFAULT_OFFSETS, OFFSET_END); + + String maxBatchRowsStr = tvfProps.get(PARAM_MAX_BATCH_ROWS); + if (maxBatchRowsStr != null) { + this.maxBatchRows = Long.parseLong(maxBatchRowsStr); + } + + // Get the Trino Kafka catalog and extract connection parameters + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(catalogName); + if (catalog == null) { + throw new UserException("Catalog not found: " + catalogName); + } + if (!(catalog instanceof TrinoConnectorExternalCatalog)) { + throw new UserException("Catalog must be a Trino Connector catalog: " + catalogName); + } + + TrinoConnectorExternalCatalog trinoCatalog = (TrinoConnectorExternalCatalog) catalog; + if (!"kafka".equalsIgnoreCase(trinoCatalog.getConnectorName().toString())) { + throw new UserException("Catalog must be a Kafka connector: " + catalogName); + } + + // Extract Kafka connection parameters from catalog properties + Map catalogProps = trinoCatalog.getCatalogProperty().getProperties(); + this.brokerList = KafkaPropertiesConverter.extractBrokerList(catalogProps); + this.topic = tableName; // In Trino Kafka, table name is the topic name + this.kafkaClientProps = KafkaPropertiesConverter.convertToKafkaClientProperties(catalogProps); + + // Initialize current offset only if not already restored from persistence + // This prevents overwriting the offset restored by replayIfNeed() + if (this.currentOffset == null || !this.currentOffset.isValidOffset()) { + this.currentOffset = new KafkaOffset(topic, catalogName, databaseName); + log.info("[KAFKA_OFFSET_DEBUG] initFromTvfProperties: created new empty currentOffset"); + } else { + // Log partition 0 offset before updating metadata + log.info("[KAFKA_OFFSET_DEBUG] initFromTvfProperties: BEFORE metadata update, partition 0 offset = {}", + this.currentOffset.getPartitionOffsets() != null ? this.currentOffset.getPartitionOffsets().get(0) : "null"); + + // Update metadata fields if they're missing (can happen after deserialization) + if (this.currentOffset.getTopic() == null) { + this.currentOffset.setTopic(topic); + } + if (this.currentOffset.getCatalogName() == null) { + this.currentOffset.setCatalogName(catalogName); + } + if (this.currentOffset.getDatabaseName() == null) { + this.currentOffset.setDatabaseName(databaseName); + } + + // Log partition 0 offset after updating metadata + log.info("[KAFKA_OFFSET_DEBUG] initFromTvfProperties: AFTER metadata update, partition 0 offset = {}", + this.currentOffset.getPartitionOffsets() != null ? this.currentOffset.getPartitionOffsets().get(0) : "null"); + log.info("[KAFKA_OFFSET_DEBUG] initFromTvfProperties: preserved restored currentOffset, updated metadata if needed"); + } + + log.info("[KAFKA_OFFSET_DEBUG] initFromTvfProperties completed: jobId={}, catalog={}, topic={}, brokers={}, currentOffset={}, partition 0 offset = {}", + jobId, catalogName, topic, brokerList, + currentOffset != null ? currentOffset.showRange() : "null", + currentOffset != null && currentOffset.getPartitionOffsets() != null ? currentOffset.getPartitionOffsets().get(0) : "null"); + this.isInitialized = true; + } + + /** + * Get the next offset for a single partition (standard interface). + * For Kafka, this returns the KafkaOffset which contains all partition offsets. + */ + @Override + public Offset getNextOffset(StreamingJobProperties jobProps, Map properties) { + // Return the current offset state + // The actual partition-level offsets are managed through getNextPartitionOffsets() + return currentOffset; + } + + /** + * Get the next batch of partition offsets for creating parallel tasks. + * Each KafkaPartitionOffset represents a single partition's work unit. + */ + public List getNextPartitionOffsets(StreamingJobProperties jobProps) { + List offsets = new ArrayList<>(); + + if (currentOffset == null || currentOffset.getPartitionOffsets() == null) { + return offsets; + } + + for (Map.Entry entry : currentOffset.getPartitionOffsets().entrySet()) { + int partitionId = entry.getKey(); + KafkaPartitionOffset partitionOffset = getNextPartitionOffset(partitionId, jobProps); + if (partitionOffset != null) { + offsets.add(partitionOffset); + } + } + + return offsets; + } + + /** + * Get the next offset range for a single partition. + * Returns null if there is no more data to consume for this partition. + * + * @param partitionId the Kafka partition ID + * @param jobProps job properties (not used currently but kept for API consistency) + * @return the next offset range, or null if no data available + */ + public KafkaPartitionOffset getNextPartitionOffset(int partitionId, StreamingJobProperties jobProps) { + if (currentOffset == null || currentOffset.getPartitionOffsets() == null) { + if (partitionId == 0) { + log.info("[KAFKA_OFFSET_DEBUG] <<>> getNextPartitionOffset: partition 0 - currentOffset is null or empty <<>>"); + } + return null; + } + + long currentPos = currentOffset.getPartitionOffset(partitionId); + long latestPos = latestOffsets.getOrDefault(partitionId, currentPos); + + if (partitionId == 0) { + log.info("[KAFKA_OFFSET_DEBUG] <<>> getNextPartitionOffset: partition 0 - currentPos={}, latestPos={}, currentOffset.showRange()={} <<>>", + currentPos, latestPos, currentOffset.showRange()); + } + + // No more data if current >= latest + if (currentPos >= latestPos) { + log.info("Partition {} has no more data: current={}, latest={}", + partitionId, currentPos, latestPos); + return null; + } + + // Calculate the end offset for this batch + long endOffset = Math.min(currentPos + maxBatchRows, latestPos); + + log.info("Partition {} offset range: [{}, {}), latest: {}", + partitionId, currentPos, endOffset, latestPos); + + KafkaPartitionOffset result = new KafkaPartitionOffset(partitionId, currentPos, endOffset); + + if (partitionId == 0) { + log.info("[KAFKA_OFFSET_DEBUG] <<>> getNextPartitionOffset: partition 0 - created task with startOffset={}, endOffset={} <<>>", + result.getStartOffset(), result.getEndOffset()); + } + + return result; + } + + /** + * Check if a specific partition has more data to consume. + * + * @param partitionId the Kafka partition ID + * @return true if the partition has unconsumed data + */ + public boolean hasMoreDataForPartition(int partitionId) { + if (currentOffset == null || currentOffset.getPartitionOffsets() == null) { + return false; + } + + long currentPos = currentOffset.getPartitionOffset(partitionId); + long latestPos = latestOffsets.getOrDefault(partitionId, currentPos); + + return currentPos < latestPos; + } + + /** + * Get all partition IDs that have been initialized. + * + * @return set of partition IDs + */ + public java.util.Set getAllPartitionIds() { + if (currentOffset == null || currentOffset.getPartitionOffsets() == null) { + return java.util.Collections.emptySet(); + } + return currentOffset.getPartitionOffsets().keySet(); + } + + @Override + public String getShowCurrentOffset() { + if (currentOffset != null) { + return currentOffset.showRange(); + } + return "{}"; + } + + @Override + public String getShowMaxOffset() { + if (latestOffsets != null && !latestOffsets.isEmpty()) { + String offsetsStr = latestOffsets.entrySet().stream() + .map(e -> String.format("p%d=%d", e.getKey(), e.getValue())) + .collect(Collectors.joining(", ")); + return "{" + offsetsStr + "}"; + } + return "{}"; + } + + /** + * Rewrite the TVF-based INSERT command to a direct table query with offset filtering. + * + * Original: INSERT INTO target SELECT * FROM kafka(...) + * Rewritten: INSERT INTO target SELECT * FROM catalog.db.table + * WHERE _partition = X AND _offset >= Y AND _offset < Z + */ + @Override + public InsertIntoTableCommand rewriteTvfParams(InsertIntoTableCommand originCommand, Offset runningOffset) { + KafkaPartitionOffset partitionOffset = (KafkaPartitionOffset) runningOffset; + + // Rewrite the plan tree + Plan rewritePlan = originCommand.getParsedPlan().get().rewriteUp(plan -> { + if (plan instanceof LogicalProject) { + LogicalProject project = (LogicalProject) plan; + Plan child = project.child(); + + // Check if the child is a TVF relation (possibly wrapped in a filter) + if (child instanceof UnboundTVFRelation) { + UnboundTVFRelation tvf = (UnboundTVFRelation) child; + if ("kafka".equalsIgnoreCase(tvf.getFunctionName())) { + // Create table reference and filter + Plan tableWithFilter = createKafkaTableWithFilter(tvf, partitionOffset); + return project.withChildren(ImmutableList.of(tableWithFilter)); + } + } + } else if (plan instanceof UnboundTVFRelation) { + UnboundTVFRelation tvf = (UnboundTVFRelation) plan; + if ("kafka".equalsIgnoreCase(tvf.getFunctionName())) { + return createKafkaTableWithFilter(tvf, partitionOffset); + } + } + return plan; + }); + + InsertIntoTableCommand newCommand = new InsertIntoTableCommand( + (LogicalPlan) rewritePlan, + Optional.empty(), Optional.empty(), Optional.empty(), true, Optional.empty()); + newCommand.setJobId(originCommand.getJobId()); + return newCommand; + } + + /** + * Create a Kafka table reference with partition and offset filter. + */ + private Plan createKafkaTableWithFilter(UnboundTVFRelation tvf, KafkaPartitionOffset partitionOffset) { + // Create UnboundRelation for the Trino Kafka table: catalog.database.table + List tableParts = ImmutableList.of(catalogName, databaseName, tableName); + UnboundRelation tableRelation = new UnboundRelation( + tvf.getRelationId(), + tableParts + ); + + // Create filter expressions: + // _partition = partitionId AND _offset >= startOffset AND _offset < endOffset + Expression partitionFilter = new EqualTo( + new UnboundSlot(PARTITION_COLUMN), + new IntegerLiteral(partitionOffset.getPartitionId()) + ); + + Expression offsetGte = new GreaterThanEqual( + new UnboundSlot(OFFSET_COLUMN), + new BigIntLiteral(partitionOffset.getStartOffset()) + ); + + Expression offsetLt = new LessThan( + new UnboundSlot(OFFSET_COLUMN), + new BigIntLiteral(partitionOffset.getEndOffset()) + ); + + // Combine all conditions with AND + Expression filterExpr = new And(partitionFilter, new And(offsetGte, offsetLt)); + + log.info("create offset filter for partition {}: [{}, {})", + partitionOffset.getPartitionId(), partitionOffset.getStartOffset(), partitionOffset.getEndOffset()); + // Create LogicalFilter with the table relation as child + return new LogicalFilter<>(ImmutableSet.of(filterExpr), tableRelation); + } + + /** + * Update the offset for a specific partition after task completion. + */ + public void updatePartitionOffset(int partitionId, long newOffset) { + if (currentOffset != null) { + long oldOffset = currentOffset.getPartitionOffset(partitionId); + currentOffset.updatePartitionOffset(partitionId, newOffset); + + if (partitionId == 0) { + // Get stack trace for partition 0 updates + StackTraceElement[] stackTrace = Thread.currentThread().getStackTrace(); + StringBuilder sb = new StringBuilder(); + for (int i = 2; i < Math.min(10, stackTrace.length); i++) { + sb.append("\n at ").append(stackTrace[i]); + } + + log.info("[KAFKA_OFFSET_DEBUG] <<>> updatePartitionOffset: partition 0 updated from {} to {}, REGRESSION={} <<>> CALL STACK:{}", + oldOffset, newOffset, newOffset < oldOffset ? "YES!!!" : "no", sb.toString()); + } else { + log.info("[KAFKA_OFFSET_DEBUG] updatePartitionOffset: partition {} updated from {} to {}", + partitionId, oldOffset, newOffset); + } + } else { + log.warn("[KAFKA_OFFSET_DEBUG] updatePartitionOffset: currentOffset is null, cannot update partition {}", partitionId); + } + } + + @Override + public void updateOffset(Offset offset) { + if (offset instanceof KafkaOffset) { + KafkaOffset newKafkaOffset = (KafkaOffset) offset; + + // Log the incoming KafkaOffset details for partition 0 + if (newKafkaOffset.getPartitionOffsets() != null && newKafkaOffset.getPartitionOffsets().containsKey(0)) { + log.info("[KAFKA_OFFSET_DEBUG] updateOffset with KafkaOffset: partition 0 new value = {}, topic={}, currentOffset partition 0 = {}", + newKafkaOffset.getPartitionOffsets().get(0), + newKafkaOffset.getTopic(), + this.currentOffset != null && this.currentOffset.getPartitionOffsets() != null + ? this.currentOffset.getPartitionOffsets().get(0) : "null"); + } + + // If current offset exists and the new one doesn't have metadata, preserve it + if (this.currentOffset != null && newKafkaOffset.getTopic() == null) { + log.info("[KAFKA_OFFSET_DEBUG] updateOffset: new KafkaOffset missing metadata, preserving from current"); + // Create a new KafkaOffset with preserved metadata + KafkaOffset preservedOffset = new KafkaOffset( + this.currentOffset.getTopic(), + this.currentOffset.getCatalogName(), + this.currentOffset.getDatabaseName()); + if (newKafkaOffset.getPartitionOffsets() != null) { + preservedOffset.setPartitionOffsets(new HashMap<>(newKafkaOffset.getPartitionOffsets())); + } + this.currentOffset = preservedOffset; + } else { + this.currentOffset = newKafkaOffset; + } + log.info("[KAFKA_OFFSET_DEBUG] updateOffset with KafkaOffset: jobId={}, offset={}", + jobId, currentOffset != null ? currentOffset.showRange() : "null"); + } else if (offset instanceof KafkaPartitionOffset) { + // Update single partition offset + KafkaPartitionOffset partitionOffset = (KafkaPartitionOffset) offset; + if (this.currentOffset == null) { + log.warn("[KAFKA_OFFSET_DEBUG] updateOffset: currentOffset is null, cannot update partition offset"); + return; + } + long newOffset = partitionOffset.getStartOffset() + partitionOffset.getConsumedRows(); + + // Critical check: detect offset regression + if (partitionOffset.getPartitionId() == 0) { + long currentP0 = this.currentOffset.getPartitionOffset(0); + log.info("[KAFKA_OFFSET_DEBUG] updateOffset with KafkaPartitionOffset: <<>> partition 0: startOffset={}, consumedRows={}, calculated newOffset={}, current={}, REGRESSION={}", + partitionOffset.getStartOffset(), + partitionOffset.getConsumedRows(), + newOffset, + currentP0, + newOffset < currentP0 ? "YES!!!" : "no"); + + // Get stack trace to find who is calling this + StackTraceElement[] stackTrace = Thread.currentThread().getStackTrace(); + StringBuilder sb = new StringBuilder(); + for (int i = 2; i < Math.min(8, stackTrace.length); i++) { + sb.append("\n at ").append(stackTrace[i]); + } + log.info("[KAFKA_OFFSET_DEBUG] updateOffset CALL STACK for partition 0:{}", sb.toString()); + } + + updatePartitionOffset(partitionOffset.getPartitionId(), newOffset); + log.info("[KAFKA_OFFSET_DEBUG] updateOffset with KafkaPartitionOffset: jobId={}, partition={}, newOffset={}, currentOffset={}", + jobId, partitionOffset.getPartitionId(), newOffset, + currentOffset != null ? currentOffset.showRange() : "null"); + } + } + + /** + * Fetch latest offsets from Kafka for all partitions. + */ + @Override + public void fetchRemoteMeta(Map properties) throws Exception { + log.info("[KAFKA_OFFSET_DEBUG] <<>> fetchRemoteMeta started: jobId={}, topic={}, isInitialized={}, currentOffset={} <<>>", + jobId, topic, isInitialized, currentOffset != null ? currentOffset.showRange() : "null"); + + // Log current partition offsets before fetching + if (currentOffset != null && currentOffset.getPartitionOffsets() != null) { + log.info("[KAFKA_OFFSET_DEBUG] <<>> fetchRemoteMeta: BEFORE fetch, partition 0 offset = {}, currentOffset.isEmpty() = {} <<>>", + currentOffset.getPartitionOffsets().get(0), + currentOffset.isEmpty()); + } + + try { + // Get all partitions for the topic + List partitionIds = KafkaUtil.getAllKafkaPartitions( + brokerList, topic, kafkaClientProps); + + log.info("[KAFKA_OFFSET_DEBUG] Fetched {} partitions for topic {}", partitionIds.size(), topic); + + // Initialize partition offsets if this is the first run + boolean wasEmpty = currentOffset.isEmpty(); + log.info("[KAFKA_OFFSET_DEBUG] <<>> fetchRemoteMeta: currentOffset.isEmpty() = {} <<>>", wasEmpty); + + if (wasEmpty) { + log.info("[KAFKA_OFFSET_DEBUG] <<>> fetchRemoteMeta: currentOffset is empty, calling initializePartitionOffsets() <<>>"); + initializePartitionOffsets(partitionIds); + log.info("[KAFKA_OFFSET_DEBUG] <<>> fetchRemoteMeta: AFTER initializePartitionOffsets, partition 0 offset = {} <<>>", + currentOffset.getPartitionOffsets().get(0)); + } else { + log.info("[KAFKA_OFFSET_DEBUG] <<>> fetchRemoteMeta: currentOffset is NOT empty, skipping initialization, partition 0 offset = {} <<>>", + currentOffset.getPartitionOffsets().get(0)); + } + + // Get latest offsets for all partitions + List> offsets = KafkaUtil.getLatestOffsets( + jobId, UUID.randomUUID(), brokerList, topic, kafkaClientProps, partitionIds); + + // Update latest offsets map + latestOffsets.clear(); + for (Pair offset : offsets) { + latestOffsets.put(offset.first, offset.second); + } + + log.info("[KAFKA_OFFSET_DEBUG] fetchRemoteMeta completed: latestOffsets={}, currentOffset={}, partition 0: current={}, latest={}", + latestOffsets, + currentOffset != null ? currentOffset.showRange() : "null", + currentOffset != null ? currentOffset.getPartitionOffsets().get(0) : "null", + latestOffsets.get(0)); + + } catch (Exception e) { + log.warn("[KAFKA_OFFSET_DEBUG] Failed to fetch Kafka metadata for topic {}", topic, e); + throw e; + } + } + + /** + * Initialize offsets for all partitions based on the default offset configuration. + */ + private void initializePartitionOffsets(List partitionIds) throws Exception { + long initialOffset; + + if (OFFSET_BEGINNING.equalsIgnoreCase(defaultOffsetsConfig)) { + // Start from the beginning (offset 0 for each partition) + // We need to fetch the actual beginning offsets + List> beginningOffsets = getBeginningOffsets(partitionIds); + for (Pair offset : beginningOffsets) { + currentOffset.updatePartitionOffset(offset.first, offset.second); + } + } else if (OFFSET_END.equalsIgnoreCase(defaultOffsetsConfig)) { + // Start from the end (latest offset for each partition) + List> endOffsets = KafkaUtil.getLatestOffsets( + jobId, UUID.randomUUID(), brokerList, topic, kafkaClientProps, partitionIds); + for (Pair offset : endOffsets) { + currentOffset.updatePartitionOffset(offset.first, offset.second); + } + } else { + // Try to parse as a specific offset value + try { + initialOffset = Long.parseLong(defaultOffsetsConfig); + for (Integer partitionId : partitionIds) { + currentOffset.updatePartitionOffset(partitionId, initialOffset); + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException( + "Invalid kafka_default_offsets value: " + defaultOffsetsConfig + + ". Expected: OFFSET_BEGINNING, OFFSET_END, or a numeric value"); + } + } + + log.info("Initialized partition offsets: {}", currentOffset.getPartitionOffsets()); + } + + /** + * Get beginning offsets (earliest available) for partitions. + */ + private List> getBeginningOffsets(List partitionIds) throws Exception { + // Use timestamp -2 to get earliest offsets + List> timestampOffsets = partitionIds.stream() + .map(p -> Pair.of(p, OFFSET_BEGINNING_VAL)) + .collect(Collectors.toList()); + + return KafkaUtil.getRealOffsets(brokerList, topic, kafkaClientProps, timestampOffsets); + } + + @Override + public boolean hasMoreDataToConsume() { + if (currentOffset == null || currentOffset.isEmpty()) { + return true; // Need to initialize + } + + for (Map.Entry entry : currentOffset.getPartitionOffsets().entrySet()) { + int partitionId = entry.getKey(); + long currentPos = entry.getValue(); + long latestPos = latestOffsets.getOrDefault(partitionId, currentPos); + + if (currentPos < latestPos) { + return true; + } + } + + return false; + } + + @Override + public Offset deserializeOffset(String offset) { + log.info("[KAFKA_OFFSET_DEBUG] deserializeOffset input: jobId={}, offsetJson='{}'", + jobId, offset); + + // Try to detect which type of offset this is by checking for specific fields + // KafkaPartitionOffset has: partitionId, startOffset, endOffset + // KafkaOffset has: partitionOffsets, topic, catalogName + Offset result = null; + + try { + if (offset.contains("\"partitionId\"")) { + // This is a KafkaPartitionOffset from a task + KafkaPartitionOffset partitionOffset = GsonUtils.GSON.fromJson(offset, KafkaPartitionOffset.class); + log.info("[KAFKA_OFFSET_DEBUG] deserializeOffset: detected KafkaPartitionOffset, partition={}, range=[{}, {})", + partitionOffset.getPartitionId(), + partitionOffset.getStartOffset(), + partitionOffset.getEndOffset()); + result = partitionOffset; + } else { + // This is a KafkaOffset (job-level offset) + KafkaOffset kafkaOffset = GsonUtils.GSON.fromJson(offset, KafkaOffset.class); + log.info("[KAFKA_OFFSET_DEBUG] deserializeOffset: detected KafkaOffset, topic={}, partitions={}", + kafkaOffset.getTopic(), + kafkaOffset.getPartitionOffsets() != null ? kafkaOffset.getPartitionOffsets().keySet() : "null"); + result = kafkaOffset; + } + } catch (Exception e) { + log.warn("[KAFKA_OFFSET_DEBUG] deserializeOffset failed to parse offset JSON", e); + // Fallback to KafkaOffset + result = GsonUtils.GSON.fromJson(offset, KafkaOffset.class); + } + + log.info("[KAFKA_OFFSET_DEBUG] deserializeOffset output: jobId={}, resultType={}, result={}", + jobId, + result != null ? result.getClass().getSimpleName() : "null", + result != null ? result.showRange() : "null"); + return result; + } + + @Override + public Offset deserializeOffsetProperty(String offset) { + if (StringUtils.isBlank(offset)) { + return null; + } + + try { + // Try to parse as KafkaOffset JSON + return GsonUtils.GSON.fromJson(offset, KafkaOffset.class); + } catch (Exception e) { + log.warn("Failed to deserialize Kafka offset: {}", offset, e); + return null; + } + } + + @Override + public String getPersistInfo() { + String persistInfo = null; + if (currentOffset != null) { + persistInfo = currentOffset.toSerializedJson(); + // Log each partition's offset explicitly for debugging + if (currentOffset.getPartitionOffsets() != null) { + for (Map.Entry entry : currentOffset.getPartitionOffsets().entrySet()) { + log.info("[KAFKA_OFFSET_DEBUG] getPersistInfo: partition {} has offset {}", + entry.getKey(), entry.getValue()); + } + } + } + log.info("[KAFKA_OFFSET_DEBUG] getPersistInfo called: jobId={}, currentOffset={}, persistInfo='{}'", + jobId, + currentOffset != null ? currentOffset.showRange() : "null", + persistInfo != null ? persistInfo : "null"); + return persistInfo; + } + + /** + * Replay offset from persistence when FE restarts. + * This method is called during job initialization to restore the offset state. + */ + @Override + public void replayIfNeed(StreamingInsertJob job) throws JobException { + String offsetProviderPersist = job.getOffsetProviderPersist(); + log.info("[KAFKA_OFFSET_DEBUG] <<>> replayIfNeed called: jobId={}, isInitialized={}, offsetProviderPersist={} <<>>", + job.getJobId(), + isInitialized, + offsetProviderPersist != null ? "has value (length=" + offsetProviderPersist.length() + ")" : "null"); + + if (offsetProviderPersist != null && !offsetProviderPersist.isEmpty()) { + try { + log.info("[KAFKA_OFFSET_DEBUG] replayIfNeed: raw JSON='{}'", offsetProviderPersist); + KafkaOffset restoredOffset = GsonUtils.GSON.fromJson(offsetProviderPersist, KafkaOffset.class); + if (restoredOffset != null && restoredOffset.isValidOffset()) { + // Restore the offset state + this.currentOffset = restoredOffset; + + // DO NOT set isInitialized = true here! + // We still need initFromTvfProperties() to initialize brokerList, topic, etc. + // initFromTvfProperties() will check if currentOffset is valid and preserve it + + log.info("[KAFKA_OFFSET_DEBUG] <<>> replayIfNeed: successfully restored offset (isInitialized stays false), topic={}, partitions={}, DETAILED_OFFSETS={} <<>>", + restoredOffset.getTopic(), + restoredOffset.getPartitionOffsets().keySet(), + restoredOffset.getPartitionOffsets()); + + // Log each partition's offset explicitly + for (Map.Entry entry : restoredOffset.getPartitionOffsets().entrySet()) { + if (entry.getKey() == 0) { + log.info("[KAFKA_OFFSET_DEBUG] <<>> replayIfNeed: partition 0 restored to offset {} <<>>", + entry.getValue()); + } else { + log.info("[KAFKA_OFFSET_DEBUG] replayIfNeed: partition {} restored to offset {}", + entry.getKey(), entry.getValue()); + } + } + } else { + log.warn("[KAFKA_OFFSET_DEBUG] <<>> replayIfNeed: restored offset is invalid, will use default offset <<>>"); + } + } catch (Exception e) { + log.warn("[KAFKA_OFFSET_DEBUG] replayIfNeed: failed to restore offset, will use default offset", e); + // Don't throw exception, let the job continue with default offset initialization + } + } else { + log.info("[KAFKA_OFFSET_DEBUG] replayIfNeed: no persisted offset to restore"); + } + } + + /** + * Get a required property value, throwing an exception if not found. + */ + private String getRequiredProperty(Map props, String key) throws UserException { + String value = props.get(key); + if (value == null || value.isEmpty()) { + throw new UserException("Missing required property: " + key); + } + return value; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/Kafka.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/Kafka.java new file mode 100644 index 00000000000000..1c6ceff9667187 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/Kafka.java @@ -0,0 +1,73 @@ +// 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.doris.nereids.trees.expressions.functions.table; + +import org.apache.doris.catalog.FunctionSignature; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.trees.expressions.Properties; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; +import org.apache.doris.nereids.types.coercion.AnyDataType; +import org.apache.doris.tablefunction.KafkaTableValuedFunction; +import org.apache.doris.tablefunction.TableValuedFunctionIf; + +import java.util.Map; + +/** + * kafka() TVF - Used for StreamingInsertJob to read data from Kafka. + * + * This TVF depends on a pre-configured TrinoConnectorExternalCatalog (Kafka Connector). + * + * Usage: + *
+ * SELECT * FROM kafka(
+ *     "catalog" = "kafka_catalog",
+ *     "database" = "default",
+ *     "table" = "my_topic"
+ * );
+ * 
+ * + * Note: This TVF is primarily designed for use with CREATE JOB ... ON STREAMING. + * When used in a streaming job, the TVF will be rewritten by KafkaSourceOffsetProvider + * to reference the actual Trino Kafka table with partition and offset filtering. + */ +public class Kafka extends TableValuedFunction { + + public Kafka(Properties properties) { + super("kafka", properties); + } + + @Override + public FunctionSignature customSignature() { + return FunctionSignature.of(AnyDataType.INSTANCE_WITHOUT_INDEX, getArgumentsTypes()); + } + + @Override + protected TableValuedFunctionIf toCatalogFunction() { + try { + Map arguments = getTVFProperties().getMap(); + return new KafkaTableValuedFunction(arguments); + } catch (Throwable t) { + throw new AnalysisException("Can not build kafka(): " + t.getMessage(), t); + } + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitKafka(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableValuedFunctionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableValuedFunctionVisitor.java index ee1ccd76478bd0..87ee5f26d51ef8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableValuedFunctionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/TableValuedFunctionVisitor.java @@ -29,6 +29,7 @@ import org.apache.doris.nereids.trees.expressions.functions.table.HudiMeta; import org.apache.doris.nereids.trees.expressions.functions.table.IcebergMeta; import org.apache.doris.nereids.trees.expressions.functions.table.Jobs; +import org.apache.doris.nereids.trees.expressions.functions.table.Kafka; import org.apache.doris.nereids.trees.expressions.functions.table.Local; import org.apache.doris.nereids.trees.expressions.functions.table.MvInfos; import org.apache.doris.nereids.trees.expressions.functions.table.Numbers; @@ -132,4 +133,8 @@ default R visitS3(S3 s3, C context) { default R visitQuery(Query query, C context) { return visitTableValuedFunction(query, context); } + + default R visitKafka(Kafka kafka, C context) { + return visitTableValuedFunction(kafka, context); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java index 49749918d4c548..232f0a95518b4c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateJobInfo.java @@ -332,7 +332,7 @@ private AbstractJob analyzeAndCreateStreamingInsertJob(String sql, String curren if (logicalPlan instanceof InsertIntoTableCommand) { InsertIntoTableCommand insertIntoTableCommand = (InsertIntoTableCommand) logicalPlan; try { - insertIntoTableCommand.initPlan(ConnectContext.get(), ConnectContext.get().getExecutor(), false); + // insertIntoTableCommand.initPlan(ConnectContext.get(), ConnectContext.get().getExecutor(), false); return new StreamingInsertJob(labelNameOptional.get(), JobStatus.PENDING, currentDbName, diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/KafkaTableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/KafkaTableValuedFunction.java new file mode 100644 index 00000000000000..4c6a16e5e13bbd --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/KafkaTableValuedFunction.java @@ -0,0 +1,138 @@ +// 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.doris.tablefunction; + +import org.apache.doris.analysis.TupleDescriptor; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.JdbcTable; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.jdbc.JdbcExternalCatalog; +import org.apache.doris.datasource.jdbc.source.JdbcScanNode; +import org.apache.doris.datasource.trinoconnector.TrinoConnectorExternalCatalog; +import org.apache.doris.datasource.trinoconnector.TrinoConnectorExternalDatabase; +import org.apache.doris.datasource.trinoconnector.TrinoConnectorExternalTable; +import org.apache.doris.datasource.trinoconnector.source.TrinoConnectorScanNode; +import org.apache.doris.planner.PlanNodeId; +import org.apache.doris.planner.ScanNode; +import org.apache.doris.qe.SessionVariable; + +import com.google.common.base.Joiner; +import lombok.Getter; + +import java.util.List; +import java.util.Map; + +/** + * Kafka Table-Valued Function implementation. + * + * This TVF provides a way to read from Kafka topics via a Trino Kafka Connector catalog. + * It is designed primarily for use with streaming jobs (CREATE JOB ... ON STREAMING). + * + * Parameters: + * - catalog: Name of the Trino Kafka Catalog (required) + * - database: Database/schema name, default "default" + * - table: Kafka topic name (required) + * - kafka_default_offsets: Initial offset position, "OFFSET_BEGINNING" or "OFFSET_END" + * - max_batch_rows: Maximum rows per batch + * + * Note: When used in a streaming job, the KafkaSourceOffsetProvider will rewrite + * this TVF to a direct table reference with partition and offset filtering. + * Therefore, getScanNode() is not implemented as it won't be called directly. + */ +@Getter +public class KafkaTableValuedFunction extends TableValuedFunctionIf { + + public static final String NAME = "kafka"; + + // Parameter names + public static final String PARAM_CATALOG = "catalog"; + public static final String PARAM_DATABASE = "database"; + public static final String PARAM_TABLE = "table"; + + private final String catalogName; + private final String databaseName; + private final String tableName; + private final Map properties; + + private TrinoConnectorExternalCatalog trinoCatalog; + private TrinoConnectorExternalDatabase trinoDatabase; + @Getter + private TrinoConnectorExternalTable trinoTable; + + public KafkaTableValuedFunction(Map params) throws AnalysisException { + this.properties = params; + + // Parse required parameters + this.catalogName = getRequiredParam(params, PARAM_CATALOG); + this.databaseName = getRequiredParam(params, PARAM_DATABASE); + this.tableName = getRequiredParam(params, PARAM_TABLE); + + // Validate the catalog exists and is a Kafka connector + validateKafkaCatalog(); + } + + /** + * Validate that the specified catalog exists and is a Trino Kafka connector. + */ + private void validateKafkaCatalog() throws AnalysisException { + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalogOrAnalysisException(catalogName); + if (!(catalog instanceof TrinoConnectorExternalCatalog)) { + throw new AnalysisException( + "Catalog '" + catalogName + "' must be a Trino Connector catalog. " + + "Please create a Trino Kafka catalog first."); + } + + this.trinoCatalog = (TrinoConnectorExternalCatalog) catalog; + String connectorName = trinoCatalog.getConnectorName() != null + ? trinoCatalog.getConnectorName().toString() : ""; + + if (!"kafka".equalsIgnoreCase(connectorName)) { + throw new AnalysisException( + "Catalog '" + catalogName + "' must be a Kafka connector, but found: " + connectorName); + } + + this.trinoDatabase = (TrinoConnectorExternalDatabase) catalog.getDbOrAnalysisException(this.databaseName); + this.trinoTable = this.trinoDatabase.getTableOrAnalysisException(this.tableName); + } + + private String getRequiredParam(Map params, String key) throws AnalysisException { + String value = params.get(key); + if (value == null || value.isEmpty()) { + throw new AnalysisException("Missing required parameter: " + key); + } + return value; + } + + @Override + public String getTableName() { + return Joiner.on(".").join(this.catalogName, this.databaseName, this.tableName); + } + + @Override + public List getTableColumns() throws AnalysisException { + return trinoTable.getFullSchema(); + } + + @Override + public ScanNode getScanNode(PlanNodeId id, TupleDescriptor desc, SessionVariable sv) { + return new TrinoConnectorScanNode(id, desc, false, sv); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java index 7621ca58587e29..9899e807135ee4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/TableValuedFunctionIf.java @@ -106,6 +106,8 @@ public static TableValuedFunctionIf getTableFunction(String funcName, Map partitionOffsets = new HashMap<>(); + partitionOffsets.put(0, 100L); + partitionOffsets.put(1, 200L); + partitionOffsets.put(2, 300L); + + KafkaOffset offset = new KafkaOffset("test_topic", "kafka_catalog", "default"); + offset.setPartitionOffsets(partitionOffsets); + + // Serialize + String json = offset.toSerializedJson(); + Assert.assertNotNull(json); + Assert.assertTrue(json.contains("test_topic")); + + // Deserialize + KafkaOffset deserializedOffset = GsonUtils.GSON.fromJson(json, KafkaOffset.class); + Assert.assertEquals("test_topic", deserializedOffset.getTopic()); + Assert.assertEquals("kafka_catalog", deserializedOffset.getCatalogName()); + Assert.assertEquals("default", deserializedOffset.getDatabaseName()); + Assert.assertEquals(3, deserializedOffset.getPartitionOffsets().size()); + Assert.assertEquals(Long.valueOf(100L), deserializedOffset.getPartitionOffsets().get(0)); + Assert.assertEquals(Long.valueOf(200L), deserializedOffset.getPartitionOffsets().get(1)); + Assert.assertEquals(Long.valueOf(300L), deserializedOffset.getPartitionOffsets().get(2)); + } + + @Test + public void testKafkaOffsetUpdatePartition() { + KafkaOffset offset = new KafkaOffset("test_topic", "catalog", "db"); + Map partitionOffsets = new HashMap<>(); + partitionOffsets.put(0, 0L); + offset.setPartitionOffsets(partitionOffsets); + + // Update single partition + offset.updatePartitionOffset(0, 1000L); + Assert.assertEquals(Long.valueOf(1000L), offset.getPartitionOffsets().get(0)); + + // Add new partition + offset.updatePartitionOffset(1, 500L); + Assert.assertEquals(2, offset.getPartitionOffsets().size()); + Assert.assertEquals(Long.valueOf(500L), offset.getPartitionOffsets().get(1)); + } + + @Test + public void testKafkaOffsetGetPartitionOffset() { + KafkaOffset offset = new KafkaOffset("topic", "catalog", "db"); + Map partitionOffsets = new HashMap<>(); + partitionOffsets.put(0, 100L); + offset.setPartitionOffsets(partitionOffsets); + + // Existing partition + Assert.assertEquals(100L, offset.getPartitionOffset(0)); + + // Non-existing partition (should return 0) + Assert.assertEquals(0L, offset.getPartitionOffset(99)); + } + + @Test + public void testKafkaOffsetIsEmpty() { + KafkaOffset offset = new KafkaOffset(); + Assert.assertTrue(offset.isEmpty()); + + offset = new KafkaOffset("topic", "catalog", "db"); + Assert.assertTrue(offset.isEmpty()); + + offset.updatePartitionOffset(0, 100L); + Assert.assertFalse(offset.isEmpty()); + } + + @Test + public void testKafkaOffsetIsValidOffset() { + KafkaOffset offset = new KafkaOffset(); + Assert.assertFalse(offset.isValidOffset()); + + offset = new KafkaOffset("topic", "catalog", "db"); + Assert.assertTrue(offset.isValidOffset()); + } + + @Test + public void testKafkaOffsetCopy() { + KafkaOffset offset = new KafkaOffset("topic", "catalog", "db"); + offset.updatePartitionOffset(0, 100L); + offset.updatePartitionOffset(1, 200L); + + KafkaOffset copy = offset.copy(); + + Assert.assertEquals(offset.getTopic(), copy.getTopic()); + Assert.assertEquals(offset.getPartitionOffsets().size(), copy.getPartitionOffsets().size()); + + // Modify original should not affect copy + offset.updatePartitionOffset(0, 999L); + Assert.assertEquals(Long.valueOf(100L), copy.getPartitionOffsets().get(0)); + } + + @Test + public void testKafkaPartitionOffsetShowRange() { + KafkaPartitionOffset partitionOffset = new KafkaPartitionOffset(0, 100L, 200L); + String range = partitionOffset.showRange(); + Assert.assertTrue(range.contains("partition=0")); + Assert.assertTrue(range.contains("100")); + Assert.assertTrue(range.contains("200")); + } + + @Test + public void testKafkaPartitionOffsetIsEmpty() { + // Empty when startOffset >= endOffset + KafkaPartitionOffset empty = new KafkaPartitionOffset(0, 100L, 100L); + Assert.assertTrue(empty.isEmpty()); + + KafkaPartitionOffset notEmpty = new KafkaPartitionOffset(0, 100L, 200L); + Assert.assertFalse(notEmpty.isEmpty()); + } + + @Test + public void testKafkaPartitionOffsetIsValidOffset() { + // Valid offset + KafkaPartitionOffset valid = new KafkaPartitionOffset(0, 100L, 200L); + Assert.assertTrue(valid.isValidOffset()); + + // Invalid - startOffset < 0 + KafkaPartitionOffset invalid1 = new KafkaPartitionOffset(0, -1L, 100L); + Assert.assertFalse(invalid1.isValidOffset()); + + // Invalid - endOffset <= startOffset + KafkaPartitionOffset invalid2 = new KafkaPartitionOffset(0, 100L, 100L); + Assert.assertFalse(invalid2.isValidOffset()); + } + + @Test + public void testKafkaPartitionOffsetGetExpectedRows() { + KafkaPartitionOffset offset = new KafkaPartitionOffset(0, 100L, 200L); + Assert.assertEquals(100L, offset.getExpectedRows()); + } + + @Test + public void testKafkaPartitionOffsetSerialization() { + KafkaPartitionOffset offset = new KafkaPartitionOffset(5, 1000L, 2000L); + offset.setConsumedRows(800L); + + String json = offset.toSerializedJson(); + Assert.assertNotNull(json); + + KafkaPartitionOffset deserialized = GsonUtils.GSON.fromJson(json, KafkaPartitionOffset.class); + Assert.assertEquals(5, deserialized.getPartitionId()); + Assert.assertEquals(1000L, deserialized.getStartOffset()); + Assert.assertEquals(2000L, deserialized.getEndOffset()); + Assert.assertEquals(800L, deserialized.getConsumedRows()); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/job/offset/kafka/KafkaPropertiesConverterTest.java b/fe/fe-core/src/test/java/org/apache/doris/job/offset/kafka/KafkaPropertiesConverterTest.java new file mode 100644 index 00000000000000..94449f7508cb4e --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/job/offset/kafka/KafkaPropertiesConverterTest.java @@ -0,0 +1,162 @@ +// 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.doris.job.offset.kafka; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +public class KafkaPropertiesConverterTest { + + @Test + public void testExtractBrokerList() { + Map props = new HashMap<>(); + props.put("trino.kafka.nodes", "broker1:9092,broker2:9092"); + + String brokerList = KafkaPropertiesConverter.extractBrokerList(props); + Assert.assertEquals("broker1:9092,broker2:9092", brokerList); + } + + @Test + public void testExtractBrokerListWithoutPrefix() { + Map props = new HashMap<>(); + props.put("kafka.nodes", "broker1:9092"); + + String brokerList = KafkaPropertiesConverter.extractBrokerList(props); + Assert.assertEquals("broker1:9092", brokerList); + } + + @Test + public void testExtractBrokerListPriorityTrinoPrefix() { + // trino.kafka.nodes should take priority over kafka.nodes + Map props = new HashMap<>(); + props.put("trino.kafka.nodes", "broker1:9092"); + props.put("kafka.nodes", "broker2:9092"); + + String brokerList = KafkaPropertiesConverter.extractBrokerList(props); + Assert.assertEquals("broker1:9092", brokerList); + } + + @Test(expected = IllegalArgumentException.class) + public void testExtractBrokerListMissing() { + Map props = new HashMap<>(); + KafkaPropertiesConverter.extractBrokerList(props); + } + + @Test + public void testConvertToKafkaClientProperties() { + Map props = new HashMap<>(); + // Trino-specific properties (should be filtered out) + props.put("trino.kafka.nodes", "broker1:9092"); + props.put("trino.kafka.default-schema", "default"); + props.put("trino.kafka.table-names", "topic1,topic2"); + props.put("trino.kafka.hide-internal-columns", "false"); + + // Kafka client properties (should be retained) + props.put("trino.kafka.security.protocol", "SASL_PLAINTEXT"); + props.put("trino.kafka.sasl.mechanism", "PLAIN"); + props.put("trino.kafka.request.timeout.ms", "30000"); + + Map kafkaProps = KafkaPropertiesConverter.convertToKafkaClientProperties(props); + + // Verify Trino-specific properties are filtered out + Assert.assertFalse(kafkaProps.containsKey("nodes")); + Assert.assertFalse(kafkaProps.containsKey("default-schema")); + Assert.assertFalse(kafkaProps.containsKey("table-names")); + Assert.assertFalse(kafkaProps.containsKey("hide-internal-columns")); + + // Verify Kafka client properties are retained + Assert.assertEquals("SASL_PLAINTEXT", kafkaProps.get("security.protocol")); + Assert.assertEquals("PLAIN", kafkaProps.get("sasl.mechanism")); + Assert.assertEquals("30000", kafkaProps.get("request.timeout.ms")); + } + + @Test + public void testConvertEmptyProperties() { + Map props = new HashMap<>(); + Map kafkaProps = KafkaPropertiesConverter.convertToKafkaClientProperties(props); + Assert.assertTrue(kafkaProps.isEmpty()); + } + + @Test + public void testConvertPropertiesWithoutKafkaPrefix() { + Map props = new HashMap<>(); + props.put("other.property", "value"); + props.put("trino.other.property", "value2"); + + Map kafkaProps = KafkaPropertiesConverter.convertToKafkaClientProperties(props); + Assert.assertTrue(kafkaProps.isEmpty()); + } + + @Test + public void testNormalizeSecurityProperties() { + Map kafkaProps = new HashMap<>(); + kafkaProps.put("sasl.mechanism", "PLAIN"); + kafkaProps.put("sasl.jaas.config", "..."); + + KafkaPropertiesConverter.normalizeSecurityProperties(kafkaProps); + + // When SASL config is present, security.protocol should be defaulted + Assert.assertEquals("SASL_PLAINTEXT", kafkaProps.get("security.protocol")); + } + + @Test + public void testNormalizeSecurityPropertiesExistingProtocol() { + Map kafkaProps = new HashMap<>(); + kafkaProps.put("sasl.mechanism", "PLAIN"); + kafkaProps.put("security.protocol", "SASL_SSL"); + + KafkaPropertiesConverter.normalizeSecurityProperties(kafkaProps); + + // Should not override existing protocol + Assert.assertEquals("SASL_SSL", kafkaProps.get("security.protocol")); + } + + @Test + public void testValidateKafkaPropertiesSuccess() { + Map kafkaProps = new HashMap<>(); + kafkaProps.put("security.protocol", "SASL_PLAINTEXT"); + kafkaProps.put("sasl.mechanism", "PLAIN"); + + // Should not throw + KafkaPropertiesConverter.validateKafkaProperties("broker1:9092", "topic", kafkaProps); + } + + @Test(expected = IllegalArgumentException.class) + public void testValidateKafkaPropertiesNullBroker() { + Map kafkaProps = new HashMap<>(); + KafkaPropertiesConverter.validateKafkaProperties(null, "topic", kafkaProps); + } + + @Test(expected = IllegalArgumentException.class) + public void testValidateKafkaPropertiesEmptyTopic() { + Map kafkaProps = new HashMap<>(); + KafkaPropertiesConverter.validateKafkaProperties("broker:9092", "", kafkaProps); + } + + @Test(expected = IllegalArgumentException.class) + public void testValidateKafkaPropertiesMissingSaslMechanism() { + Map kafkaProps = new HashMap<>(); + kafkaProps.put("security.protocol", "SASL_PLAINTEXT"); + // Missing sasl.mechanism + + KafkaPropertiesConverter.validateKafkaProperties("broker:9092", "topic", kafkaProps); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/job/offset/kafka/KafkaSourceOffsetProviderTest.java b/fe/fe-core/src/test/java/org/apache/doris/job/offset/kafka/KafkaSourceOffsetProviderTest.java new file mode 100644 index 00000000000000..75ae5718a60127 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/job/offset/kafka/KafkaSourceOffsetProviderTest.java @@ -0,0 +1,364 @@ +// 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.doris.job.offset.kafka; + +import org.apache.doris.job.extensions.insert.streaming.StreamingJobProperties; +import org.apache.doris.job.offset.Offset; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class KafkaSourceOffsetProviderTest { + + private KafkaSourceOffsetProvider provider; + + @Before + public void setUp() { + provider = new KafkaSourceOffsetProvider(); + } + + @Test + public void testGetSourceType() { + Assert.assertEquals("kafka", provider.getSourceType()); + } + + @Test + public void testGetNextOffsetsNoNewData() { + // Set current offset equal to latest - no new data + KafkaOffset currentOffset = new KafkaOffset("topic", "catalog", "db"); + currentOffset.updatePartitionOffset(0, 100L); + provider.setCurrentOffset(currentOffset); + + Map latestOffsets = new HashMap<>(); + latestOffsets.put(0, 100L); // Same as current + provider.setLatestOffsets(latestOffsets); + provider.setMaxBatchRows(100L); + + StreamingJobProperties jobProps = new StreamingJobProperties(new HashMap<>()); + List nextOffsets = provider.getNextPartitionOffsets(jobProps); + + Assert.assertTrue(nextOffsets.isEmpty()); + } + + @Test + public void testGetNextOffsetsWithNewData() { + // Set up current offset + KafkaOffset currentOffset = new KafkaOffset("topic", "catalog", "db"); + currentOffset.updatePartitionOffset(0, 100L); + currentOffset.updatePartitionOffset(1, 200L); + currentOffset.updatePartitionOffset(2, 0L); + provider.setCurrentOffset(currentOffset); + + // Set up latest offsets + Map latestOffsets = new HashMap<>(); + latestOffsets.put(0, 500L); // Has new data + latestOffsets.put(1, 200L); // No new data + latestOffsets.put(2, 1000L); // Has new data + provider.setLatestOffsets(latestOffsets); + provider.setMaxBatchRows(100L); + + StreamingJobProperties jobProps = new StreamingJobProperties(new HashMap<>()); + List nextOffsets = provider.getNextPartitionOffsets(jobProps); + + // Should return 2 partitions (0 and 2 have new data) + Assert.assertEquals(2, nextOffsets.size()); + + // Verify offset range calculation + for (KafkaPartitionOffset offset : nextOffsets) { + if (offset.getPartitionId() == 0) { + Assert.assertEquals(100L, offset.getStartOffset()); + Assert.assertEquals(200L, offset.getEndOffset()); // min(100+100, 500) + } else if (offset.getPartitionId() == 2) { + Assert.assertEquals(0L, offset.getStartOffset()); + Assert.assertEquals(100L, offset.getEndOffset()); // min(0+100, 1000) + } + } + } + + @Test + public void testGetNextOffsetsLimitedByLatest() { + // Test case where latest offset is less than current + maxBatchRows + KafkaOffset currentOffset = new KafkaOffset("topic", "catalog", "db"); + currentOffset.updatePartitionOffset(0, 900L); + provider.setCurrentOffset(currentOffset); + + Map latestOffsets = new HashMap<>(); + latestOffsets.put(0, 950L); // Only 50 messages available + provider.setLatestOffsets(latestOffsets); + provider.setMaxBatchRows(100L); + + StreamingJobProperties jobProps = new StreamingJobProperties(new HashMap<>()); + List nextOffsets = provider.getNextPartitionOffsets(jobProps); + + Assert.assertEquals(1, nextOffsets.size()); + KafkaPartitionOffset offset = nextOffsets.get(0); + Assert.assertEquals(900L, offset.getStartOffset()); + Assert.assertEquals(950L, offset.getEndOffset()); // Limited by latest + } + + @Test + public void testHasMoreDataToConsumeTrue() { + KafkaOffset currentOffset = new KafkaOffset("topic", "catalog", "db"); + currentOffset.updatePartitionOffset(0, 100L); + provider.setCurrentOffset(currentOffset); + + Map latestOffsets = new HashMap<>(); + latestOffsets.put(0, 200L); + provider.setLatestOffsets(latestOffsets); + + Assert.assertTrue(provider.hasMoreDataToConsume()); + } + + @Test + public void testHasMoreDataToConsumeFalse() { + KafkaOffset currentOffset = new KafkaOffset("topic", "catalog", "db"); + currentOffset.updatePartitionOffset(0, 100L); + provider.setCurrentOffset(currentOffset); + + Map latestOffsets = new HashMap<>(); + latestOffsets.put(0, 100L); // Same as current + provider.setLatestOffsets(latestOffsets); + + Assert.assertFalse(provider.hasMoreDataToConsume()); + } + + @Test + public void testHasMoreDataToConsumeEmptyOffset() { + // Empty current offset should return true (needs initialization) + Assert.assertTrue(provider.hasMoreDataToConsume()); + } + + @Test + public void testUpdatePartitionOffset() { + KafkaOffset currentOffset = new KafkaOffset("topic", "catalog", "db"); + currentOffset.updatePartitionOffset(0, 0L); + provider.setCurrentOffset(currentOffset); + + // Simulate task completion with 50 rows consumed + provider.updatePartitionOffset(0, 50L); + + Assert.assertEquals(Long.valueOf(50L), + provider.getCurrentOffset().getPartitionOffsets().get(0)); + } + + @Test + public void testDeserializeOffset() { + String json = "{\"partitionOffsets\":{\"0\":100,\"1\":200},\"topic\":\"test\",\"catalogName\":\"catalog\",\"databaseName\":\"db\"}"; + Offset offset = provider.deserializeOffset(json); + + Assert.assertTrue(offset instanceof KafkaOffset); + KafkaOffset kafkaOffset = (KafkaOffset) offset; + Assert.assertEquals("test", kafkaOffset.getTopic()); + Assert.assertEquals(Long.valueOf(100L), kafkaOffset.getPartitionOffsets().get(0)); + Assert.assertEquals(Long.valueOf(200L), kafkaOffset.getPartitionOffsets().get(1)); + } + + @Test + public void testGetShowCurrentOffset() { + KafkaOffset currentOffset = new KafkaOffset("topic", "catalog", "db"); + currentOffset.updatePartitionOffset(0, 100L); + currentOffset.updatePartitionOffset(1, 200L); + provider.setCurrentOffset(currentOffset); + + String show = provider.getShowCurrentOffset(); + Assert.assertNotNull(show); + Assert.assertTrue(show.contains("topic")); + } + + @Test + public void testGetShowMaxOffset() { + Map latestOffsets = new HashMap<>(); + latestOffsets.put(0, 1000L); + latestOffsets.put(1, 2000L); + provider.setLatestOffsets(latestOffsets); + + String show = provider.getShowMaxOffset(); + Assert.assertNotNull(show); + Assert.assertTrue(show.contains("p0=1000") || show.contains("p1=2000")); + } + + @Test + public void testGetPersistInfo() { + KafkaOffset currentOffset = new KafkaOffset("topic", "catalog", "db"); + currentOffset.updatePartitionOffset(0, 100L); + provider.setCurrentOffset(currentOffset); + + String persistInfo = provider.getPersistInfo(); + Assert.assertNotNull(persistInfo); + Assert.assertTrue(persistInfo.contains("topic")); + Assert.assertTrue(persistInfo.contains("100")); + } + + // ============ Tests for Independent Pipeline Model ============ + + @Test + public void testGetNextPartitionOffsetSinglePartitionWithData() { + // Set up current offset + KafkaOffset currentOffset = new KafkaOffset("topic", "catalog", "db"); + currentOffset.updatePartitionOffset(0, 100L); + currentOffset.updatePartitionOffset(1, 200L); + provider.setCurrentOffset(currentOffset); + + // Set up latest offsets + Map latestOffsets = new HashMap<>(); + latestOffsets.put(0, 500L); + latestOffsets.put(1, 200L); // No new data + provider.setLatestOffsets(latestOffsets); + provider.setMaxBatchRows(100L); + + StreamingJobProperties jobProps = new StreamingJobProperties(new HashMap<>()); + + // Partition 0 has data + KafkaPartitionOffset offset0 = provider.getNextPartitionOffset(0, jobProps); + Assert.assertNotNull(offset0); + Assert.assertEquals(0, offset0.getPartitionId()); + Assert.assertEquals(100L, offset0.getStartOffset()); + Assert.assertEquals(200L, offset0.getEndOffset()); + + // Partition 1 has no data + KafkaPartitionOffset offset1 = provider.getNextPartitionOffset(1, jobProps); + Assert.assertNull(offset1); + } + + @Test + public void testGetNextPartitionOffsetLimitedByLatest() { + KafkaOffset currentOffset = new KafkaOffset("topic", "catalog", "db"); + currentOffset.updatePartitionOffset(0, 900L); + provider.setCurrentOffset(currentOffset); + + Map latestOffsets = new HashMap<>(); + latestOffsets.put(0, 930L); // Only 30 messages available + provider.setLatestOffsets(latestOffsets); + provider.setMaxBatchRows(100L); + + StreamingJobProperties jobProps = new StreamingJobProperties(new HashMap<>()); + KafkaPartitionOffset offset = provider.getNextPartitionOffset(0, jobProps); + + Assert.assertNotNull(offset); + Assert.assertEquals(900L, offset.getStartOffset()); + Assert.assertEquals(930L, offset.getEndOffset()); // Limited by latest, not maxBatchRows + } + + @Test + public void testHasMoreDataForPartitionTrue() { + KafkaOffset currentOffset = new KafkaOffset("topic", "catalog", "db"); + currentOffset.updatePartitionOffset(0, 100L); + currentOffset.updatePartitionOffset(1, 200L); + provider.setCurrentOffset(currentOffset); + + Map latestOffsets = new HashMap<>(); + latestOffsets.put(0, 500L); // Has data + latestOffsets.put(1, 200L); // No data + provider.setLatestOffsets(latestOffsets); + + Assert.assertTrue(provider.hasMoreDataForPartition(0)); + Assert.assertFalse(provider.hasMoreDataForPartition(1)); + } + + @Test + public void testHasMoreDataForPartitionUnknownPartition() { + KafkaOffset currentOffset = new KafkaOffset("topic", "catalog", "db"); + currentOffset.updatePartitionOffset(0, 100L); + provider.setCurrentOffset(currentOffset); + + Map latestOffsets = new HashMap<>(); + latestOffsets.put(0, 500L); + provider.setLatestOffsets(latestOffsets); + + // Partition 99 doesn't exist + Assert.assertFalse(provider.hasMoreDataForPartition(99)); + } + + @Test + public void testGetAllPartitionIds() { + KafkaOffset currentOffset = new KafkaOffset("topic", "catalog", "db"); + currentOffset.updatePartitionOffset(0, 100L); + currentOffset.updatePartitionOffset(1, 200L); + currentOffset.updatePartitionOffset(2, 300L); + provider.setCurrentOffset(currentOffset); + + java.util.Set partitionIds = provider.getAllPartitionIds(); + + Assert.assertEquals(3, partitionIds.size()); + Assert.assertTrue(partitionIds.contains(0)); + Assert.assertTrue(partitionIds.contains(1)); + Assert.assertTrue(partitionIds.contains(2)); + } + + @Test + public void testGetAllPartitionIdsEmpty() { + // No current offset set + java.util.Set partitionIds = provider.getAllPartitionIds(); + Assert.assertTrue(partitionIds.isEmpty()); + } + + @Test + public void testIndependentPipelineScenario() { + // Simulates the independent pipeline model: + // 1. Create initial tasks for all partitions + // 2. Partition 0 completes, gets new task immediately + // 3. Partition 1 has no more data, becomes idle + // 4. Later, partition 1 gets new data and restarts + + KafkaOffset currentOffset = new KafkaOffset("topic", "catalog", "db"); + currentOffset.updatePartitionOffset(0, 0L); + currentOffset.updatePartitionOffset(1, 0L); + provider.setCurrentOffset(currentOffset); + + Map latestOffsets = new HashMap<>(); + latestOffsets.put(0, 200L); + latestOffsets.put(1, 50L); + provider.setLatestOffsets(latestOffsets); + provider.setMaxBatchRows(100L); + + StreamingJobProperties jobProps = new StreamingJobProperties(new HashMap<>()); + + // Step 1: Get initial offsets + List initialOffsets = provider.getNextPartitionOffsets(jobProps); + Assert.assertEquals(2, initialOffsets.size()); + + // Step 2: Partition 0 completes (consumed 100 rows), gets next task + provider.updatePartitionOffset(0, 100L); + KafkaPartitionOffset nextOffset0 = provider.getNextPartitionOffset(0, jobProps); + Assert.assertNotNull(nextOffset0); + Assert.assertEquals(100L, nextOffset0.getStartOffset()); + Assert.assertEquals(200L, nextOffset0.getEndOffset()); + + // Step 3: Partition 1 completes (consumed 50 rows), no more data + provider.updatePartitionOffset(1, 50L); + KafkaPartitionOffset nextOffset1 = provider.getNextPartitionOffset(1, jobProps); + Assert.assertNull(nextOffset1); // No more data + Assert.assertFalse(provider.hasMoreDataForPartition(1)); + + // Step 4: New data arrives for partition 1 + latestOffsets.put(1, 150L); + provider.setLatestOffsets(latestOffsets); + Assert.assertTrue(provider.hasMoreDataForPartition(1)); + + // Step 5: Partition 1 restarts + KafkaPartitionOffset restartOffset1 = provider.getNextPartitionOffset(1, jobProps); + Assert.assertNotNull(restartOffset1); + Assert.assertEquals(50L, restartOffset1.getStartOffset()); + Assert.assertEquals(150L, restartOffset1.getEndOffset()); + } +} diff --git a/regression-test/suites/load_p0/kafka_streaming/test_kafka_streaming_basic.groovy b/regression-test/suites/load_p0/kafka_streaming/test_kafka_streaming_basic.groovy new file mode 100644 index 00000000000000..c8eb4e01b82a12 --- /dev/null +++ b/regression-test/suites/load_p0/kafka_streaming/test_kafka_streaming_basic.groovy @@ -0,0 +1,100 @@ +// 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. + +suite("test_kafka_streaming_basic", "p0,nonConcurrent") { + + // Test configuration + String enabled = context.config.otherConfigs.get("enableKafkaTest") + String kafka_port = context.config.otherConfigs.get("kafka_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + def kafka_broker = "${externalEnvIp}:${kafka_port}" + + if (enabled != null && enabled.equalsIgnoreCase("true")) { + + def catalogName = "test_kafka_catalog" + def tableName = "kafka_streaming_target" + def topicName = "test_kafka_streaming_topic" + def jobName = "test_kafka_streaming_job" + def dbName = context.config.getDbNameByFile(context.file) + + try { + // 1. Drop existing objects if any + sql "DROP JOB IF EXISTS ${jobName}" + sql "DROP TABLE IF EXISTS ${tableName}" + sql "DROP CATALOG IF EXISTS ${catalogName}" + + // 2. Create Trino Kafka Catalog + sql """ + CREATE CATALOG IF NOT EXISTS ${catalogName} PROPERTIES ( + "type" = "trino-connector", + "trino.connector.name" = "kafka", + "trino.kafka.nodes" = "${kafka_broker}", + "trino.kafka.table-names" = "${topicName}", + "trino.kafka.hide-internal-columns" = "false" + ) + """ + + // 3. Create target table + sql """ + CREATE TABLE IF NOT EXISTS ${tableName} ( + id INT, + name VARCHAR(100), + value INT + ) + DISTRIBUTED BY HASH(id) BUCKETS 3 + PROPERTIES ("replication_num" = "1") + """ + + // 4. Create Kafka Streaming Job + sql """ + CREATE JOB ${jobName} + ON STREAMING + DO + INSERT INTO ${tableName} + SELECT * FROM kafka( + "catalog" = "${catalogName}", + "database" = "default", + "table" = "${topicName}", + "kafka_default_offsets" = "OFFSET_BEGINNING", + "max_batch_rows" = "50" + ) + """ + + // 5. Verify job was created + def result = sql "SHOW JOB ${jobName}" + logger.info("Job created: ${result}") + assertTrue(result.size() > 0) + + // 6. Check job status + def jobStatus = sql "SHOW JOB STATUS ${jobName}" + logger.info("Job status: ${jobStatus}") + + } finally { + // Cleanup + try { + sql "STOP JOB ${jobName}" + } catch (Exception e) { + // Ignore + } + sql "DROP JOB IF EXISTS ${jobName}" + sql "DROP TABLE IF EXISTS ${tableName}" + sql "DROP CATALOG IF EXISTS ${catalogName}" + } + } else { + logger.info("Kafka test is disabled, skipping test_kafka_streaming_basic") + } +} diff --git a/regression-test/suites/load_p0/kafka_streaming/test_kafka_streaming_exactly_once.groovy b/regression-test/suites/load_p0/kafka_streaming/test_kafka_streaming_exactly_once.groovy new file mode 100644 index 00000000000000..36fd4800b6da1f --- /dev/null +++ b/regression-test/suites/load_p0/kafka_streaming/test_kafka_streaming_exactly_once.groovy @@ -0,0 +1,102 @@ +// 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. + +suite("test_kafka_streaming_exactly_once", "p0,nonConcurrent") { + + // Test configuration + String enabled = context.config.otherConfigs.get("enableKafkaTest") + String kafka_port = context.config.otherConfigs.get("kafka_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + def kafka_broker = "${externalEnvIp}:${kafka_port}" + + if (enabled != null && enabled.equalsIgnoreCase("true")) { + + def catalogName = "test_kafka_catalog_eo" + def tableName = "kafka_streaming_exactly_once" + def topicName = "test_exactly_once_topic" + def jobName = "test_kafka_eo_job" + def dbName = context.config.getDbNameByFile(context.file) + + try { + // 1. Drop existing objects if any + sql "DROP JOB IF EXISTS ${jobName}" + sql "DROP TABLE IF EXISTS ${tableName}" + sql "DROP CATALOG IF EXISTS ${catalogName}" + + // 2. Create Trino Kafka Catalog + sql """ + CREATE CATALOG IF NOT EXISTS ${catalogName} PROPERTIES ( + "type" = "trino-connector", + "trino.connector.name" = "kafka", + "trino.kafka.nodes" = "${kafka_broker}", + "trino.kafka.table-names" = "${topicName}", + "trino.kafka.hide-internal-columns" = "false" + ) + """ + + // 3. Create target table with unique key for deduplication verification + sql """ + CREATE TABLE IF NOT EXISTS ${tableName} ( + unique_id VARCHAR(100), + data VARCHAR(100) + ) + UNIQUE KEY(unique_id) + DISTRIBUTED BY HASH(unique_id) BUCKETS 3 + PROPERTIES ("replication_num" = "1") + """ + + // 4. Create Kafka Streaming Job + sql """ + CREATE JOB ${jobName} + ON STREAMING + DO + INSERT INTO ${tableName} + SELECT * FROM kafka( + "catalog" = "${catalogName}", + "database" = "default", + "table" = "${topicName}", + "kafka_default_offsets" = "OFFSET_BEGINNING", + "max_batch_rows" = "100" + ) + """ + + // 5. Verify job was created + def result = sql "SHOW JOB ${jobName}" + logger.info("Exactly-once job created: ${result}") + assertTrue(result.size() > 0) + + // 6. The actual exactly-once verification requires: + // - Pre-populating Kafka with unique data + // - Running the job and verifying no duplicates + // This is documented in the test plan but requires + // a running Kafka cluster with test data + + } finally { + // Cleanup + try { + sql "STOP JOB ${jobName}" + } catch (Exception e) { + // Ignore + } + sql "DROP JOB IF EXISTS ${jobName}" + sql "DROP TABLE IF EXISTS ${tableName}" + sql "DROP CATALOG IF EXISTS ${catalogName}" + } + } else { + logger.info("Kafka test is disabled, skipping test_kafka_streaming_exactly_once") + } +} diff --git a/regression-test/suites/load_p0/kafka_streaming/test_kafka_streaming_multi_partition.groovy b/regression-test/suites/load_p0/kafka_streaming/test_kafka_streaming_multi_partition.groovy new file mode 100644 index 00000000000000..919b32668c0522 --- /dev/null +++ b/regression-test/suites/load_p0/kafka_streaming/test_kafka_streaming_multi_partition.groovy @@ -0,0 +1,99 @@ +// 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. + +suite("test_kafka_streaming_multi_partition", "p0,nonConcurrent") { + + // Test configuration + String enabled = context.config.otherConfigs.get("enableKafkaTest") + String kafka_port = context.config.otherConfigs.get("kafka_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + def kafka_broker = "${externalEnvIp}:${kafka_port}" + + if (enabled != null && enabled.equalsIgnoreCase("true")) { + + def catalogName = "test_kafka_catalog_mp" + def tableName = "kafka_streaming_multi_partition" + def topicName = "test_multi_partition_topic" // 3 partitions expected + def jobName = "test_kafka_mp_job" + def dbName = context.config.getDbNameByFile(context.file) + + try { + // 1. Drop existing objects if any + sql "DROP JOB IF EXISTS ${jobName}" + sql "DROP TABLE IF EXISTS ${tableName}" + sql "DROP CATALOG IF EXISTS ${catalogName}" + + // 2. Create Trino Kafka Catalog + sql """ + CREATE CATALOG IF NOT EXISTS ${catalogName} PROPERTIES ( + "type" = "trino-connector", + "trino.connector.name" = "kafka", + "trino.kafka.nodes" = "${kafka_broker}", + "trino.kafka.table-names" = "${topicName}", + "trino.kafka.hide-internal-columns" = "false" + ) + """ + + // 3. Create target table + sql """ + CREATE TABLE IF NOT EXISTS ${tableName} ( + id INT, + name VARCHAR(100) + ) + DISTRIBUTED BY HASH(id) BUCKETS 3 + PROPERTIES ("replication_num" = "1") + """ + + // 4. Create Kafka Streaming Job with multi-partition support + sql """ + CREATE JOB ${jobName} + ON STREAMING + DO + INSERT INTO ${tableName} + SELECT * FROM kafka( + "catalog" = "${catalogName}", + "database" = "default", + "table" = "${topicName}", + "kafka_default_offsets" = "OFFSET_BEGINNING", + "max_batch_rows" = "50" + ) + """ + + // 5. Verify job was created + def result = sql "SHOW JOB ${jobName}" + logger.info("Multi-partition job created: ${result}") + assertTrue(result.size() > 0) + + // 6. Verify job type + def showResult = sql "SHOW STREAMING JOB ${jobName}" + logger.info("Streaming job info: ${showResult}") + + } finally { + // Cleanup + try { + sql "STOP JOB ${jobName}" + } catch (Exception e) { + // Ignore + } + sql "DROP JOB IF EXISTS ${jobName}" + sql "DROP TABLE IF EXISTS ${tableName}" + sql "DROP CATALOG IF EXISTS ${catalogName}" + } + } else { + logger.info("Kafka test is disabled, skipping test_kafka_streaming_multi_partition") + } +} diff --git a/regression-test/suites/load_p0/kafka_streaming/test_kafka_streaming_resume.groovy b/regression-test/suites/load_p0/kafka_streaming/test_kafka_streaming_resume.groovy new file mode 100644 index 00000000000000..7e6ba5c57322de --- /dev/null +++ b/regression-test/suites/load_p0/kafka_streaming/test_kafka_streaming_resume.groovy @@ -0,0 +1,112 @@ +// 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. + +suite("test_kafka_streaming_resume", "p0,nonConcurrent") { + + // Test configuration + String enabled = context.config.otherConfigs.get("enableKafkaTest") + String kafka_port = context.config.otherConfigs.get("kafka_port") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + def kafka_broker = "${externalEnvIp}:${kafka_port}" + + if (enabled != null && enabled.equalsIgnoreCase("true")) { + + def catalogName = "test_kafka_catalog_resume" + def tableName = "kafka_streaming_resume" + def topicName = "test_resume_topic" + def jobName = "test_kafka_resume_job" + def dbName = context.config.getDbNameByFile(context.file) + + try { + // 1. Drop existing objects if any + sql "DROP JOB IF EXISTS ${jobName}" + sql "DROP TABLE IF EXISTS ${tableName}" + sql "DROP CATALOG IF EXISTS ${catalogName}" + + // 2. Create Trino Kafka Catalog + sql """ + CREATE CATALOG IF NOT EXISTS ${catalogName} PROPERTIES ( + "type" = "trino-connector", + "trino.connector.name" = "kafka", + "trino.kafka.nodes" = "${kafka_broker}", + "trino.kafka.table-names" = "${topicName}", + "trino.kafka.hide-internal-columns" = "false" + ) + """ + + // 3. Create target table + sql """ + CREATE TABLE IF NOT EXISTS ${tableName} ( + id INT, + name VARCHAR(100) + ) + DISTRIBUTED BY HASH(id) BUCKETS 3 + PROPERTIES ("replication_num" = "1") + """ + + // 4. Create Kafka Streaming Job + sql """ + CREATE JOB ${jobName} + ON STREAMING + DO + INSERT INTO ${tableName} + SELECT * FROM kafka( + "catalog" = "${catalogName}", + "database" = "default", + "table" = "${topicName}", + "kafka_default_offsets" = "OFFSET_BEGINNING", + "max_batch_rows" = "50" + ) + """ + + // 5. Verify job was created + def result = sql "SHOW JOB ${jobName}" + logger.info("Resume test job created: ${result}") + assertTrue(result.size() > 0) + + // 6. Pause the job + sql "PAUSE JOB ${jobName}" + Thread.sleep(2000) + + // 7. Verify job is paused + def pausedStatus = sql "SHOW JOB STATUS ${jobName}" + logger.info("Job status after pause: ${pausedStatus}") + // Status should be PAUSED + + // 8. Resume the job + sql "RESUME JOB ${jobName}" + Thread.sleep(2000) + + // 9. Verify job is running again + def resumedStatus = sql "SHOW JOB STATUS ${jobName}" + logger.info("Job status after resume: ${resumedStatus}") + + } finally { + // Cleanup + try { + sql "STOP JOB ${jobName}" + } catch (Exception e) { + // Ignore + } + sql "DROP JOB IF EXISTS ${jobName}" + sql "DROP TABLE IF EXISTS ${tableName}" + sql "DROP CATALOG IF EXISTS ${catalogName}" + } + } else { + logger.info("Kafka test is disabled, skipping test_kafka_streaming_resume") + } +}