Copilot commented on code in PR #17167:
URL: https://github.com/apache/pinot/pull/17167#discussion_r3039202101


##########
pinot-clients/pinot-java-client/src/main/java/org/apache/pinot/client/admin/PinotFileIngestClient.java:
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.pinot.client.admin;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hc.client5.http.classic.methods.HttpPost;
+import org.apache.hc.client5.http.entity.mime.FileBody;
+import org.apache.hc.client5.http.entity.mime.MultipartEntityBuilder;
+import org.apache.hc.client5.http.impl.classic.CloseableHttpClient;
+import org.apache.hc.client5.http.impl.classic.CloseableHttpResponse;
+import org.apache.hc.client5.http.impl.classic.HttpClients;
+import org.apache.hc.core5.http.HttpEntity;
+import org.apache.hc.core5.http.io.entity.StringEntity;
+
+
+/**
+ * Explicit client for Pinot controller endpoints that accept file or raw-body 
uploads.
+ */
+public class PinotFileIngestClient extends AbstractPinotAdminClient {
+  public PinotFileIngestClient(PinotAdminTransport transport, String 
controllerAddress, Map<String, String> headers) {
+    super(transport, controllerAddress, headers);
+  }
+
+  /**
+   * Builds the ingestion URL for ingestFromFile.
+   */
+  public String buildIngestFromFileUrl(String tableNameWithType, Map<String, 
String> batchConfigMap) {
+    String batchConfigMapStr =
+        batchConfigMap.entrySet().stream().map(e -> "\"" + e.getKey() + 
"\":\"" + e.getValue() + "\"")
+            .collect(Collectors.joining(",", "{", "}"));
+    String baseUrl = _transport.getScheme() + "://" + _controllerAddress;
+    return baseUrl + "/ingestFromFile?tableNameWithType=" + tableNameWithType 
+ "&batchConfigMapStr="
+        + URLEncoder.encode(batchConfigMapStr, StandardCharsets.UTF_8);
+  }
+
+  /**
+   * Builds the ingestion URL for ingestFromURI.
+   */
+  public String buildIngestFromUriUrl(String tableNameWithType, Map<String, 
String> batchConfigMap, String sourceUri) {
+    String batchConfigMapStr =
+        batchConfigMap.entrySet().stream().map(e -> "\"" + e.getKey() + 
"\":\"" + e.getValue() + "\"")
+            .collect(Collectors.joining(",", "{", "}"));
+    String baseUrl = _transport.getScheme() + "://" + _controllerAddress;
+    return baseUrl + "/ingestFromURI?tableNameWithType=" + tableNameWithType + 
"&batchConfigMapStr="
+        + URLEncoder.encode(batchConfigMapStr, StandardCharsets.UTF_8) + 
"&sourceURIStr="
+        + URLEncoder.encode(sourceUri, StandardCharsets.UTF_8);
+  }
+
+  /**
+   * Posts a multipart file upload to the ingestFromFile endpoint.
+   */
+  public int ingestFromFile(String tableNameWithType, Map<String, String> 
batchConfigMap, File inputFile)
+      throws PinotAdminException {
+    return postFile(buildIngestFromFileUrl(tableNameWithType, batchConfigMap), 
inputFile);
+  }
+
+  /**
+   * Posts a multipart file upload to the ingestFromURI endpoint.
+   */
+  public int ingestFromUri(String tableNameWithType, Map<String, String> 
batchConfigMap, String sourceUri,
+      File inputFile)
+      throws PinotAdminException {
+    return postFile(buildIngestFromUriUrl(tableNameWithType, batchConfigMap, 
sourceUri), inputFile);
+  }
+
+  /**
+   * Posts a multipart file to an explicit URL.
+   */
+  public int postFile(String url, File inputFile)
+      throws PinotAdminException {
+    HttpEntity reqEntity =
+        MultipartEntityBuilder.create().addPart("file", new 
FileBody(inputFile.getAbsoluteFile())).build();
+    return execute(url, reqEntity);
+  }
+
+  /**
+   * Posts a plain string body to an explicit URL.
+   */
+  public int postString(String url, String body)
+      throws PinotAdminException {
+    return execute(url, new StringEntity(body));
+  }
+
+  private int execute(String url, HttpEntity entity)
+      throws PinotAdminException {
+    try (CloseableHttpClient httpClient = HttpClients.createDefault()) {
+      HttpPost httpPost = new HttpPost(url);
+      for (Map.Entry<String, String> header : _headers.entrySet()) {
+        httpPost.setHeader(header.getKey(), header.getValue());
+      }
+      httpPost.setEntity(entity);
+      try (CloseableHttpResponse response = httpClient.execute(httpPost)) {
+        return response.getCode();
+      }

Review Comment:
   `execute()` creates a new `CloseableHttpClient` per request and does not 
apply the `SSLContext` configured on `PinotAdminTransport` (so 
HTTPS/self-signed test setups can fail even when the admin client was 
constructed with a custom SSL context). Consider reusing a single client 
instance and configuring it with `_transport`’s SSL context + reasonable 
timeouts.



##########
pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/FlinkQuickStart.java:
##########
@@ -79,12 +76,11 @@ public static void main(String[] args)
     execEnv.setParallelism(2);
     DataStream<Row> srcDs = 
execEnv.fromCollection(data).returns(TEST_TYPE_INFO).keyBy(r -> r.getField(0));
 
-    HttpClient httpClient = HttpClient.getInstance();
-    ControllerRequestClient client = new ControllerRequestClient(
-        ControllerRequestURLBuilder.baseUrl(DEFAULT_CONTROLLER_URL), 
httpClient);
-    Schema schema = PinotConnectionUtils.getSchema(client, "starbucksStores");
-    TableConfig tableConfig = PinotConnectionUtils.getTableConfig(client, 
"starbucksStores", "OFFLINE");
-    srcDs.addSink(new PinotSinkFunction<>(new 
FlinkRowGenericRowConverter(TEST_TYPE_INFO), tableConfig, schema));
-    execEnv.execute();
+    try (PinotAdminClient client = new PinotAdminClient("localhost:9000")) {
+      Schema schema = 
client.getSchemaClient().getSchemaObject("starbucksStores");
+      TableConfig tableConfig = 
client.getTableClient().getTableConfigObject("starbucksStores", "OFFLINE");
+      srcDs.addSink(new PinotSinkFunction<>(new 
FlinkRowGenericRowConverter(TEST_TYPE_INFO), tableConfig, schema));
+      execEnv.execute();

Review Comment:
   `FlinkQuickStart` now hard-codes the controller address as 
`"localhost:9000"`, while `DEFAULT_CONTROLLER_URL` is still defined as 
`"http://localhost:9000"`. This makes it harder to switch schemes/hosts and is 
inconsistent with the README example; consider using `DEFAULT_CONTROLLER_URL` 
(parsing to host:port + scheme properties) or at least reusing a single 
constant for the address.



##########
pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/LaunchBackfillIngestionJobCommand.java:
##########
@@ -102,7 +102,7 @@ public boolean execute()
       Properties transportProperties = getTransportProperties(spec);
       PinotAdminClient adminClient = new PinotAdminClient(controllerAddress, 
transportProperties, authHeader);
 
-      _pinotSegmentApiClient = adminClient.getSegmentApiClient();
+      _pinotSegmentAdminClient = adminClient.getSegmentClient();
       // 1. Fetch existing segments that need to be backfilled (to be replaced)

Review Comment:
   `PinotAdminClient` is created but never closed. Since it owns an underlying 
async HTTP transport, this can leak threads/connections across command runs. 
Wrap the client creation in try-with-resources (or ensure it’s closed in a 
finally block) and keep the segment client usage within that scope.



##########
pinot-clients/pinot-java-client/src/main/java/org/apache/pinot/client/admin/PinotFileIngestClient.java:
##########
@@ -0,0 +1,120 @@
+/**
+ * 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.pinot.client.admin;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.hc.client5.http.classic.methods.HttpPost;
+import org.apache.hc.client5.http.entity.mime.FileBody;
+import org.apache.hc.client5.http.entity.mime.MultipartEntityBuilder;
+import org.apache.hc.client5.http.impl.classic.CloseableHttpClient;
+import org.apache.hc.client5.http.impl.classic.CloseableHttpResponse;
+import org.apache.hc.client5.http.impl.classic.HttpClients;
+import org.apache.hc.core5.http.HttpEntity;
+import org.apache.hc.core5.http.io.entity.StringEntity;
+
+
+/**
+ * Explicit client for Pinot controller endpoints that accept file or raw-body 
uploads.
+ */
+public class PinotFileIngestClient extends AbstractPinotAdminClient {
+  public PinotFileIngestClient(PinotAdminTransport transport, String 
controllerAddress, Map<String, String> headers) {
+    super(transport, controllerAddress, headers);
+  }
+
+  /**
+   * Builds the ingestion URL for ingestFromFile.
+   */
+  public String buildIngestFromFileUrl(String tableNameWithType, Map<String, 
String> batchConfigMap) {
+    String batchConfigMapStr =
+        batchConfigMap.entrySet().stream().map(e -> "\"" + e.getKey() + 
"\":\"" + e.getValue() + "\"")
+            .collect(Collectors.joining(",", "{", "}"));
+    String baseUrl = _transport.getScheme() + "://" + _controllerAddress;
+    return baseUrl + "/ingestFromFile?tableNameWithType=" + tableNameWithType 
+ "&batchConfigMapStr="
+        + URLEncoder.encode(batchConfigMapStr, StandardCharsets.UTF_8);
+  }

Review Comment:
   `batchConfigMapStr` is manually assembled with string concatenation, which 
won’t correctly escape keys/values containing quotes, backslashes, or other 
JSON-sensitive characters. Build this JSON via the shared 
ObjectMapper/JsonUtils (e.g., serialize `batchConfigMap`) and URL-encode 
`tableNameWithType` as well (it’s currently appended unencoded).



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