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


##########
pinot-connectors/pinot-flink-connector/README.md:
##########
@@ -36,9 +36,9 @@ ControllerRequestClient client = new ControllerRequestClient(
 ControllerRequestURLBuilder.baseUrl(DEFAULT_CONTROLLER_URL), httpClient);
 
 // fetch Pinot schema
-Schema schema = PinotConnectionUtils.getSchema(client, "starbucksStores");
+Schema schema = client.getSchemaClient().getSchemaObject("starbucksStores");
 // fetch Pinot table config
-TableConfig tableConfig = PinotConnectionUtils.getTableConfig(client, 
"starbucksStores", "OFFLINE");
+TableConfig tableConfig = 
client.getTableClient().getTableConfigObject("starbucksStores", "OFFLINE");

Review Comment:
   The README code snippet still instantiates a ControllerRequestClient, but 
then calls admin-client style APIs (getSchemaClient()/getTableClient()), which 
ControllerRequestClient doesn't have. Update the snippet to use 
PinotAdminClient (and ideally try-with-resources) so the example compiles and 
matches the new recommended client.



##########
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();
+      }
+    } catch (IOException e) {
+      throw new PinotAdminException("Failed to execute file ingest request to 
" + url, e);
+    }

Review Comment:
   PinotFileIngestClient uses a standalone Apache HttpClient 
(HttpClients.createDefault()) and ignores any SSLContext configured on 
PinotAdminTransport/PinotAdminClient. This will break ingest/upload calls 
against HTTPS controllers that require custom trust material or 
skip-cert-validation. Consider building the HttpClient with the transport's 
SSLContext (when present) so file ingest behavior matches the rest of the admin 
client.



##########
pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/AddTenantCommand.java:
##########
@@ -156,15 +128,15 @@ public boolean execute()
     }
 
     Tenant tenant = new Tenant(_role, _name, _instanceCount, 
_offlineInstanceCount, _realtimeInstanceCount);
-    String res = AbstractBaseAdminCommand
-        .sendRequest("POST", 
ControllerRequestURLBuilder.baseUrl(_controllerAddress).forTenantCreate(),
-            tenant.toJsonString(), AuthProviderUtils.makeAuthHeaders(
-                AuthProviderUtils.makeAuthProvider(_authProvider, 
_authTokenUrl, _authToken, _user,
-                _password)));
-
-    LOGGER.info(res);
-    System.out.print(res);
-    return true;
+    try (PinotAdminClient adminClient = getPinotAdminClient()) {
+      String res = 
adminClient.getTenantClient().createTenant(tenant.toJsonString());
+      LOGGER.info(res);
+      System.out.print(res);
+      return true;

Review Comment:
   This command prints the controller response to stdout via 
System.out.print(). Per project guidelines, prefer SLF4J logging (or returning 
the response to the caller) and avoid writing to stdout from commands/library 
code.



##########
pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/PeerDownloadLLCRealtimeClusterIntegrationTest.java:
##########
@@ -82,9 +82,9 @@ public class PeerDownloadLLCRealtimeClusterIntegrationTest 
extends BaseRealtimeC
   @Override
   public void setUp()
       throws Exception {
-    System.out.println(String.format(
-        "Using random seed: %s, isDirectAlloc: %s, isConsumerDirConfigured: 
%s, enableLeadControllerResource: %s",
-        RANDOM_SEED, _isDirectAlloc, _isConsumerDirConfigured, 
_enableLeadControllerResource));
+    System.out.printf(
+        "Using random seed: %s, isDirectAlloc: %s, isConsumerDirConfigured: 
%s, enableLeadControllerResource: %s%n",
+        RANDOM_SEED, _isDirectAlloc, _isConsumerDirConfigured, 
_enableLeadControllerResource);
 

Review Comment:
   This test uses System.out.printf() for logging. Prefer using the existing 
SLF4J logger (LOGGER) so output is consistently captured/filtered by the test 
logging configuration.



##########
pinot-clients/pinot-java-client/src/main/java/org/apache/pinot/client/admin/PinotSchemaAdminClient.java:
##########
@@ -150,7 +192,7 @@ public String createSchema(String schemaConfig, boolean 
force)
    */
   public String createSchema(String schemaConfig)
       throws PinotAdminException {
-    return createSchema(schemaConfig, false);
+    return createSchema(schemaConfig, true, false);
   }

Review Comment:
   PinotSchemaRestletResource defaults schema creation's `override` query param 
to true, and the sync createSchema() methods now always send `override`. 
However createSchemaAsync() still only sends `force`, which makes async 
behavior diverge from sync behavior. Align the async method with the sync API 
(add `override`, or provide an async overload that accepts it) to avoid 
surprising differences.



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