xiangfu0 commented on code in PR #18054:
URL: https://github.com/apache/pinot/pull/18054#discussion_r3032171784


##########
pinot-connectors/pinot-flink-connector/src/main/java/org/apache/pinot/connector/flink/http/PinotConnectionUtils.java:
##########
@@ -82,7 +83,9 @@ public static TableConfig 
getTableConfig(ControllerRequestClient client, String
     if (batchConfigMaps == null) {
       
batchIngestionConfig.setBatchConfigMaps(Collections.singletonList(newBatchConfigMap));
     } else {
+      batchConfigMaps = new ArrayList<>(batchConfigMaps);
       batchConfigMaps.add(newBatchConfigMap);
+      batchIngestionConfig.setBatchConfigMaps(batchConfigMaps);

Review Comment:
   Done. Changed to merge overrides into the first existing batch config map 
and keep a singleton list. Also updated the test to assert size==1 with merged 
keys.



##########
pinot-connectors/pinot-flink-connector/src/test/java/org/apache/pinot/connector/flink/http/PinotConnectionUtilsTest.java:
##########
@@ -0,0 +1,104 @@
+/**
+ * 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.connector.flink.http;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.pinot.client.admin.PinotAdminClient;
+import org.apache.pinot.client.admin.PinotSchemaAdminClient;
+import org.apache.pinot.client.admin.PinotTableAdminClient;
+import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.config.table.TableType;
+import org.apache.pinot.spi.config.table.ingestion.BatchIngestionConfig;
+import org.apache.pinot.spi.config.table.ingestion.IngestionConfig;
+import org.apache.pinot.spi.data.FieldSpec;
+import org.apache.pinot.spi.data.Schema;
+import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties;
+import org.apache.pinot.spi.utils.builder.TableConfigBuilder;
+import org.mockito.Mockito;
+import org.testng.annotations.Test;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNotNull;
+
+
+public class PinotConnectionUtilsTest {
+  @Test
+  public void testGetSchemaUsesAdminClient()
+      throws Exception {
+    PinotAdminClient adminClient = Mockito.mock(PinotAdminClient.class);
+    PinotSchemaAdminClient schemaClient = 
Mockito.mock(PinotSchemaAdminClient.class);
+    Schema expectedSchema = new Schema.SchemaBuilder().setSchemaName("myTable")
+        .addSingleValueDimension("id", FieldSpec.DataType.INT).build();
+
+    Mockito.when(adminClient.getSchemaClient()).thenReturn(schemaClient);
+    
Mockito.when(schemaClient.getSchemaObject("myTable")).thenReturn(expectedSchema);
+
+    Schema schema = PinotConnectionUtils.getSchema(adminClient, "myTable");
+
+    assertEquals(schema.getSchemaName(), "myTable");
+    assertEquals(schema.getDimensionNames(), List.of("id"));
+  }
+
+  @Test
+  public void testGetTableConfigAddsControllerBatchConfig()
+      throws Exception {
+    PinotAdminClient adminClient = Mockito.mock(PinotAdminClient.class);
+    PinotTableAdminClient tableClient = 
Mockito.mock(PinotTableAdminClient.class);
+    TableConfig tableConfig = new 
TableConfigBuilder(TableType.OFFLINE).setTableName("myTable").build();
+
+    Mockito.when(adminClient.getTableClient()).thenReturn(tableClient);
+    Mockito.when(tableClient.getTableConfig("myTable", 
TableType.OFFLINE)).thenReturn(tableConfig);
+
+    TableConfig result =
+        PinotConnectionUtils.getTableConfig(adminClient, 
"http://localhost:9000";, "myTable", TableType.OFFLINE);
+
+    assertNotNull(result.getIngestionConfig());
+    BatchIngestionConfig batchIngestionConfig = 
result.getIngestionConfig().getBatchIngestionConfig();
+    assertNotNull(batchIngestionConfig);
+    Map<String, String> batchConfigMap = 
batchIngestionConfig.getBatchConfigMaps().get(0);
+    
assertEquals(batchConfigMap.get(BatchConfigProperties.PUSH_CONTROLLER_URI), 
"http://localhost:9000";);
+    assertEquals(batchConfigMap.get(BatchConfigProperties.OUTPUT_DIR_URI), 
"/tmp/pinotoutput");
+  }
+
+  @Test
+  public void testGetTableConfigAppendsToExistingBatchConfig()
+      throws Exception {
+    PinotAdminClient adminClient = Mockito.mock(PinotAdminClient.class);
+    PinotTableAdminClient tableClient = 
Mockito.mock(PinotTableAdminClient.class);
+    IngestionConfig ingestionConfig = new IngestionConfig();
+    ingestionConfig
+        .setBatchIngestionConfig(new 
BatchIngestionConfig(List.of(Map.of("existing", "value")), "APPEND", "DAILY"));
+    TableConfig tableConfig = new 
TableConfigBuilder(TableType.OFFLINE).setTableName("myTable")
+        .setIngestionConfig(ingestionConfig)
+        .build();
+
+    Mockito.when(adminClient.getTableClient()).thenReturn(tableClient);
+    Mockito.when(tableClient.getTableConfig("myTable", 
TableType.OFFLINE)).thenReturn(tableConfig);
+
+    TableConfig result =
+        PinotConnectionUtils.getTableConfig(adminClient, 
"http://localhost:9000";, "myTable", TableType.OFFLINE);
+
+    List<Map<String, String>> batchConfigMaps =
+        
result.getIngestionConfig().getBatchIngestionConfig().getBatchConfigMaps();
+    assertEquals(batchConfigMaps.size(), 2);
+    assertEquals(batchConfigMaps.get(0).get("existing"), "value");
+    
assertEquals(batchConfigMaps.get(1).get(BatchConfigProperties.PUSH_CONTROLLER_URI),
 "http://localhost:9000";);

Review Comment:
   Done. Test now asserts batchConfigMaps.size()==1 and verifies the existing 
key is preserved alongside the merged overrides.



##########
pinot-clients/pinot-java-client/src/main/java/org/apache/pinot/client/admin/PinotTableAdminClient.java:
##########
@@ -85,6 +89,30 @@ public String getTableConfig(String tableName)
     return response.toString();
   }
 
+  /**
+   * Gets the configuration for a specific table as a typed object.
+   *
+   * @param tableName Name of the table without type suffix
+   * @param tableType Table type to fetch
+   * @return Table configuration object
+   * @throws PinotAdminException If the request fails or the response cannot 
be parsed
+   */
+  public TableConfig getTableConfig(String tableName, TableType tableType)
+      throws PinotAdminException {
+    JsonNode response = _transport.executeGet(_controllerAddress, "/tables/" + 
tableName, null, _headers);
+    JsonNode tableConfigNode = response.get(tableType.toString());
+    if (tableConfigNode == null || tableConfigNode.isNull()) {

Review Comment:
   Done. Now passing `{"type": tableType.name()}` as query param. Also added a 
fallback to parse the entire response when the typed child node is absent, for 
backward compatibility with controllers that return the config directly.



##########
pinot-clients/pinot-java-client/src/main/java/org/apache/pinot/client/admin/PinotAdminClientExample.java:
##########
@@ -104,6 +107,12 @@ private static void 
exampleWithAuthentication(PinotAdminClient adminClient)
       String tableConfig = 
adminClient.getTableClient().getTableConfig("myTable");
       System.out.println("Table config: " + tableConfig);
 
+      TableConfig offlineTableConfig = 
adminClient.getTableClient().getTableConfig("myTable", TableType.OFFLINE);
+      System.out.println("Typed table config: " + 
offlineTableConfig.getTableName());
+
+      Schema schema = adminClient.getSchemaClient().getSchemaObject("myTable");
+      System.out.println("Typed schema: " + schema.getSchemaName());
+

Review Comment:
   Done. Replaced all System.out/err.println with SLF4J parameterized logging 
(LOGGER.info/error with {} placeholders).



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