Copilot commented on code in PR #60116:
URL: https://github.com/apache/doris/pull/60116#discussion_r2979959428


##########
fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/config/GlobalExceptionHandler.java:
##########
@@ -35,4 +39,17 @@ public Object exceptionHandler(HttpServletRequest request, 
Exception e) {
         log.error("Unexpected exception", e);
         return RestResponse.internalError(e.getMessage());
     }
+
+    @ExceptionHandler(StreamException.class)
+    public Object streamExceptionHandler(StreamException e) {
+        // Directly throwing an exception allows curl to detect anomalies in 
the streaming response.
+        log.error("Exception in streaming response, re-throwing to client", e);
+        throw e;
+    }
+
+    @ExceptionHandler(CommonException.class)
+    public Object commonExceptionHandler(CommonException e) {
+        log.error("Unexpected common exception", e);
+        return 
ResponseEntity.status(HttpStatus.INTERNAL_SERVER_ERROR).body(e.getMessage());

Review Comment:
   `CommonException` is handled differently from the existing `Exception` 
handler: it returns a plain `ResponseEntity<String>` instead of the standard 
`RestResponse` error envelope used by other endpoints. This makes error 
responses inconsistent for clients and complicates parsing. Consider returning 
`RestResponse.internalError(...)` (optionally with an HTTP 500 status) for 
`CommonException` as well, or otherwise document/standardize the response 
format differences for streaming vs non-streaming endpoints.
   ```suggestion
           return RestResponse.internalError(e.getMessage());
   ```



##########
fs_brokers/cdc_client/src/main/java/org/apache/doris/cdcclient/service/PipelineCoordinator.java:
##########
@@ -90,6 +102,156 @@ public PipelineCoordinator() {
                         new ThreadPoolExecutor.AbortPolicy());
     }
 
+    /** return data for http_file_reader */
+    public StreamingResponseBody fetchRecordStream(FetchRecordRequest 
fetchReq) throws Exception {
+        SourceReader sourceReader;
+        SplitReadResult readResult;
+        try {
+            if (fetchReq.getTaskId() == null && fetchReq.getMeta() == null) {
+                LOG.info(
+                        "Generate initial meta for fetch record request, 
jobId={}, taskId={}",
+                        fetchReq.getJobId(),
+                        fetchReq.getTaskId());
+                // means the request did not originate from the job, only tvf
+                Map<String, Object> meta = generateMeta(fetchReq.getConfig());
+                fetchReq.setMeta(meta);
+            }
+
+            sourceReader = Env.getCurrentEnv().getReader(fetchReq);
+            readResult = sourceReader.prepareAndSubmitSplit(fetchReq);
+        } catch (Exception ex) {
+            throw new CommonException(ex);
+        }
+
+        return outputStream -> {
+            try {
+                buildStreamRecords(sourceReader, fetchReq, readResult, 
outputStream);
+            } catch (Exception ex) {
+                LOG.error(
+                        "Failed fetch record, jobId={}, taskId={}",
+                        fetchReq.getJobId(),
+                        fetchReq.getTaskId(),
+                        ex);
+                throw new StreamException(ex);
+            }
+        };
+    }
+
+    private void buildStreamRecords(
+            SourceReader sourceReader,
+            FetchRecordRequest fetchRecord,
+            SplitReadResult readResult,
+            OutputStream rawOutputStream)
+            throws Exception {
+        SourceSplit split = readResult.getSplit();
+        Map<String, String> lastMeta = null;
+        int rowCount = 0;
+        BufferedOutputStream bos = new BufferedOutputStream(rawOutputStream);
+        try {
+            // Poll records using the existing mechanism
+            boolean shouldStop = false;
+            long startTime = System.currentTimeMillis();
+            while (!shouldStop) {
+                Iterator<SourceRecord> recordIterator = 
sourceReader.pollRecords();
+                if (!recordIterator.hasNext()) {
+                    Thread.sleep(100);
+                    long elapsedTime = System.currentTimeMillis() - startTime;
+                    if (elapsedTime > Constants.POLL_SPLIT_RECORDS_TIMEOUTS) {
+                        break;
+                    }
+                    continue;
+                }
+                while (recordIterator.hasNext()) {
+                    SourceRecord element = recordIterator.next();
+                    if (isHeartbeatEvent(element)) {
+                        shouldStop = true;
+                        break;
+                    }
+                    DeserializeResult result =
+                            sourceReader.deserialize(fetchRecord.getConfig(), 
element);
+                    if (!CollectionUtils.isEmpty(result.getRecords())) {
+                        for (String record : result.getRecords()) {
+                            bos.write(record.getBytes(StandardCharsets.UTF_8));
+                            bos.write(LINE_DELIMITER);
+                        }
+                        rowCount += result.getRecords().size();
+                    }
+                }
+            }
+            // force flush buffer
+            bos.flush();
+        } finally {
+            // Commit offset and cleanup
+            sourceReader.commitSourceOffset(fetchRecord.getJobId(), 
readResult.getSplit());
+            sourceReader.finishSplitRecords();
+        }
+
+        LOG.info(
+                "Fetch records completed, jobId={}, taskId={}, splitId={}, 
rowCount={}",
+                fetchRecord.getJobId(),
+                fetchRecord.getTaskId(),
+                split.splitId(),
+                rowCount);
+
+        if (readResult.getSplitState() != null) {
+            if (sourceReader.isSnapshotSplit(split)) {
+                lastMeta = 
sourceReader.extractSnapshotStateOffset(readResult.getSplitState());
+                lastMeta.put(SPLIT_ID, split.splitId());
+            } else if (sourceReader.isBinlogSplit(split)) {
+                lastMeta = 
sourceReader.extractBinlogStateOffset(readResult.getSplitState());
+                lastMeta.put(SPLIT_ID, BinlogSplit.BINLOG_SPLIT_ID);
+            } else {
+                throw new RuntimeException(
+                        "unknown split type: " + 
split.getClass().getSimpleName());
+            }
+        } else {
+            throw new RuntimeException("split state is null");
+        }
+
+        if (StringUtils.isNotEmpty(fetchRecord.getTaskId())) {
+            taskOffsetCache.put(fetchRecord.getTaskId(), lastMeta);
+        }
+
+        if (!isLong(fetchRecord.getJobId())) {
+            // TVF requires closing the window after each execution,
+            // while PG requires dropping the slot.
+            sourceReader.close(fetchRecord);

Review Comment:
   For TVF requests (non-numeric `jobId`), `sourceReader.close(fetchRecord)` is 
called but the `Env` job context is never removed. Since TVF generates a new 
random jobId per execution, this will leak entries in 
`Env.jobContexts/jobLocks` over time. After closing the reader for TVF, also 
call `Env.getCurrentEnv().close(fetchRecord.getJobId())` (and consider also 
cleaning any related stream-load state if applicable).
   ```suggestion
               sourceReader.close(fetchRecord);
               Env.getCurrentEnv().close(fetchRecord.getJobId());
   ```



##########
be/src/io/fs/http_file_reader.cpp:
##########
@@ -96,32 +111,64 @@ Status HttpFileReader::open(const FileReaderOptions& opts) 
{
         return Status::OK();
     }
 
-    // Step 1: HEAD request to get file metadata
-    RETURN_IF_ERROR(prepare_client(/*set_fail_on_error=*/true));
-    _client->set_method(HttpMethod::HEAD);
-    RETURN_IF_ERROR(_client->execute());
+    // start CDC client
+    auto enable_cdc_iter = _extend_kv.find("enable_cdc_client");
+    if (enable_cdc_iter != _extend_kv.end() && enable_cdc_iter->second == 
"true") {
+        LOG(INFO) << "CDC client is enabled, starting CDC client for " << _url;
+        ExecEnv* env = ExecEnv::GetInstance();
+        if (env == nullptr || env->cdc_client_mgr() == nullptr) {
+            return Status::InternalError("ExecEnv or CdcClientMgr is not 
initialized");
+        }
 
-    uint64_t content_length = 0;
-    RETURN_IF_ERROR(_client->get_content_length(&content_length));
+        PRequestCdcClientResult result;
+        Status start_st = env->cdc_client_mgr()->start_cdc_client(&result);
+        if (!start_st.ok()) {
+            LOG(ERROR) << "Failed to start CDC client, status=" << 
start_st.to_string();
+            return start_st;
+        }
 
-    _file_size = content_length;
-    _size_known = true;
+        // Replace CDC_CLIENT_PORT placeholder with actual CDC client port
+        const std::string placeholder = "CDC_CLIENT_PORT";
+        size_t pos = _url.find(placeholder);
+        if (pos != std::string::npos) {
+            _url.replace(pos, placeholder.size(), 
std::to_string(doris::config::cdc_client_port));
+        }
+        LOG(INFO) << "CDC client started successfully for " << _url;
+    }

Review Comment:
   `open()` returns immediately when `_initialized` is true (e.g. when FE 
provided a `file_size`). With CDC-stream TVF, FE provides `file_size`, so this 
early return will skip the new CDC-client startup and `CDC_CLIENT_PORT` 
placeholder replacement logic, leaving `_url` invalid and the CDC client 
potentially not started. Move the CDC-client initialization / placeholder 
replacement before the `_initialized` short-circuit (or refine what 
`_initialized` means) so this setup always runs when `enable_cdc_client=true`.



##########
fe/fe-core/src/main/java/org/apache/doris/tablefunction/CdcStreamTableValuedFunction.java:
##########
@@ -0,0 +1,130 @@
+// 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.BrokerDesc;
+import org.apache.doris.analysis.StorageBackend.StorageType;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.datasource.jdbc.client.JdbcClient;
+import org.apache.doris.job.cdc.DataSourceConfigKeys;
+import org.apache.doris.job.cdc.request.FetchRecordRequest;
+import org.apache.doris.job.common.DataSourceType;
+import org.apache.doris.job.util.StreamingJobUtils;
+import org.apache.doris.thrift.TBrokerFileStatus;
+import org.apache.doris.thrift.TFileType;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+public class CdcStreamTableValuedFunction extends 
ExternalFileTableValuedFunction {
+    private static final ObjectMapper objectMapper = new ObjectMapper();
+    private static final String URI = 
"http://127.0.0.1:CDC_CLIENT_PORT/api/fetchRecordStream";;
+    private final Map<String, String> originProps;
+
+    public CdcStreamTableValuedFunction(Map<String, String> properties) throws 
AnalysisException {
+        this.originProps = properties;
+        validate(properties);
+        processProps(properties);
+    }
+
+    private void processProps(Map<String, String> properties) throws 
AnalysisException {
+        Map<String, String> copyProps = new HashMap<>(properties);
+        copyProps.put("format", "json");
+        super.parseCommonProperties(copyProps);
+        this.processedParams.put("enable_cdc_client", "true");
+        this.processedParams.put("uri", URI);
+        this.processedParams.put("http.enable.range.request", "false");
+        this.processedParams.put("http.enable.chunk.response", "true");
+        this.processedParams.put("http.method", "POST");
+
+        String payload = generateParams(properties);
+        this.processedParams.put("http.payload", payload);
+        this.backendConnectProperties.putAll(processedParams);
+        generateFileStatus();
+    }
+
+    private String generateParams(Map<String, String> properties) throws 
AnalysisException {
+        FetchRecordRequest recordRequest = new FetchRecordRequest();
+        recordRequest.setJobId(UUID.randomUUID().toString().replace("-", ""));
+        recordRequest.setDataSource(properties.get(DataSourceConfigKeys.TYPE));
+        recordRequest.setConfig(properties);
+        try {
+            return objectMapper.writeValueAsString(recordRequest);
+        } catch (IOException e) {
+            LOG.info("Failed to serialize fetch record request," + 
e.getMessage());
+            throw new AnalysisException(e.getMessage());
+        }
+    }
+
+    private void validate(Map<String, String> properties) {
+        
Preconditions.checkArgument(properties.containsKey(DataSourceConfigKeys.JDBC_URL),
 "jdbc_url is required");
+        
Preconditions.checkArgument(properties.containsKey(DataSourceConfigKeys.TYPE), 
"type is required");
+        
Preconditions.checkArgument(properties.containsKey(DataSourceConfigKeys.TABLE), 
"table is required");
+        
Preconditions.checkArgument(properties.containsKey(DataSourceConfigKeys.OFFSET),
 "offset is required");
+    }
+
+    private void generateFileStatus() {
+        this.fileStatuses.clear();
+        this.fileStatuses.add(new TBrokerFileStatus(URI, false, 
Integer.MAX_VALUE, false));

Review Comment:
   `generateFileStatus()` hard-codes the file size to `Integer.MAX_VALUE`. On 
the BE side `HttpFileReader` treats a provided `file_size` as 
already-initialized metadata and returns early from `open()`, which means the 
CDC-client startup + `CDC_CLIENT_PORT` placeholder replacement logic will be 
skipped and the request URL may never become valid. Consider using an “unknown 
size” sentinel (or avoiding setting `file_size` at all) so BE `open()` still 
runs its CDC-client initialization, or adjust BE initialization logic to run 
regardless of `file_size`.
   ```suggestion
           // Use unknown size sentinel (-1) so BE HttpFileReader still runs 
its initialization logic.
           this.fileStatuses.add(new TBrokerFileStatus(URI, false, -1, false));
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/tablefunction/CdcStreamTableValuedFunction.java:
##########
@@ -0,0 +1,130 @@
+// 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.BrokerDesc;
+import org.apache.doris.analysis.StorageBackend.StorageType;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.datasource.jdbc.client.JdbcClient;
+import org.apache.doris.job.cdc.DataSourceConfigKeys;
+import org.apache.doris.job.cdc.request.FetchRecordRequest;
+import org.apache.doris.job.common.DataSourceType;
+import org.apache.doris.job.util.StreamingJobUtils;
+import org.apache.doris.thrift.TBrokerFileStatus;
+import org.apache.doris.thrift.TFileType;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+public class CdcStreamTableValuedFunction extends 
ExternalFileTableValuedFunction {
+    private static final ObjectMapper objectMapper = new ObjectMapper();
+    private static final String URI = 
"http://127.0.0.1:CDC_CLIENT_PORT/api/fetchRecordStream";;
+    private final Map<String, String> originProps;
+
+    public CdcStreamTableValuedFunction(Map<String, String> properties) throws 
AnalysisException {
+        this.originProps = properties;
+        validate(properties);
+        processProps(properties);
+    }
+
+    private void processProps(Map<String, String> properties) throws 
AnalysisException {
+        Map<String, String> copyProps = new HashMap<>(properties);
+        copyProps.put("format", "json");
+        super.parseCommonProperties(copyProps);
+        this.processedParams.put("enable_cdc_client", "true");
+        this.processedParams.put("uri", URI);
+        this.processedParams.put("http.enable.range.request", "false");
+        this.processedParams.put("http.enable.chunk.response", "true");
+        this.processedParams.put("http.method", "POST");
+
+        String payload = generateParams(properties);
+        this.processedParams.put("http.payload", payload);
+        this.backendConnectProperties.putAll(processedParams);
+        generateFileStatus();
+    }
+
+    private String generateParams(Map<String, String> properties) throws 
AnalysisException {
+        FetchRecordRequest recordRequest = new FetchRecordRequest();
+        recordRequest.setJobId(UUID.randomUUID().toString().replace("-", ""));
+        recordRequest.setDataSource(properties.get(DataSourceConfigKeys.TYPE));
+        recordRequest.setConfig(properties);
+        try {
+            return objectMapper.writeValueAsString(recordRequest);
+        } catch (IOException e) {
+            LOG.info("Failed to serialize fetch record request," + 
e.getMessage());
+            throw new AnalysisException(e.getMessage());
+        }
+    }
+
+    private void validate(Map<String, String> properties) {
+        
Preconditions.checkArgument(properties.containsKey(DataSourceConfigKeys.JDBC_URL),
 "jdbc_url is required");
+        
Preconditions.checkArgument(properties.containsKey(DataSourceConfigKeys.TYPE), 
"type is required");
+        
Preconditions.checkArgument(properties.containsKey(DataSourceConfigKeys.TABLE), 
"table is required");
+        
Preconditions.checkArgument(properties.containsKey(DataSourceConfigKeys.OFFSET),
 "offset is required");
+    }
+
+    private void generateFileStatus() {
+        this.fileStatuses.clear();
+        this.fileStatuses.add(new TBrokerFileStatus(URI, false, 
Integer.MAX_VALUE, false));
+    }
+
+    @Override
+    public List<Column> getTableColumns() throws AnalysisException {
+        DataSourceType dataSourceType =
+                
DataSourceType.valueOf(processedParams.get(DataSourceConfigKeys.TYPE).toUpperCase());
+        JdbcClient jdbcClient = 
StreamingJobUtils.getJdbcClient(dataSourceType, processedParams);
+        String database = StreamingJobUtils.getRemoteDbName(dataSourceType, 
processedParams);
+        String table = processedParams.get(DataSourceConfigKeys.TABLE);
+        boolean tableExist = jdbcClient.isTableExist(database, table);
+        Preconditions.checkArgument(tableExist, "Table does not exist: " + 
table);
+        return jdbcClient.getColumnsFromJdbc(database, table);
+    }
+
+    @Override
+    public TFileType getTFileType() {
+        return TFileType.FILE_HTTP;
+    }
+
+    @Override
+    public String getFilePath() {
+        return URI;
+    }
+
+    @Override
+    public BrokerDesc getBrokerDesc() {
+        return new BrokerDesc("CdcStreamTvfBroker", StorageType.HTTP, 
originProps);

Review Comment:
   `getBrokerDesc()` is built from `originProps`, which does not include the 
computed HTTP properties (e.g. `uri`, `http.method`, `http.payload`, 
`enable_cdc_client`). This will prevent the BE HTTP reader from receiving the 
necessary request configuration and can also cause 
`StorageProperties.createPrimary(...)` to fail because `fs.http.support=true` 
is not set. Build the `BrokerDesc` from the processed/derived properties (and 
set `fs.http.support` like `HttpTableValuedFunction` does) so HTTP storage is 
detected correctly and the BE sees the TVF-specific HTTP options.
   ```suggestion
           Map<String, String> brokerProps = new HashMap<>(processedParams);
           brokerProps.put("fs.http.support", "true");
           return new BrokerDesc("CdcStreamTvfBroker", StorageType.HTTP, 
brokerProps);
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

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


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to