You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ne...@apache.org on 2022/04/27 17:44:58 UTC

[pinot] branch master updated: Indicate what fields are getting ignored when adding configs (#8514)

This is an automated email from the ASF dual-hosted git repository.

nehapawar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new b38a59be7c Indicate what fields are getting ignored when adding configs (#8514)
b38a59be7c is described below

commit b38a59be7c128310a0fee2279472d29e9290c347
Author: Saurabh Dubey <sa...@gmail.com>
AuthorDate: Wed Apr 27 23:14:52 2022 +0530

    Indicate what fields are getting ignored when adding configs (#8514)
    
    This PR adds "unrecognizedProperties" to POST /tables/, PUT /tables/{tableName} and POST /tables/validate. This field will be a flattened map of json keys to values that will let the API users know when certain field(s) in the input config json are being ignored.
---
 ...essResponse.java => ConfigSuccessResponse.java} |  15 ++-
 .../api/resources/PinotTableRestletResource.java   |  45 ++++---
 .../controller/api/resources/SuccessResponse.java  |   2 +-
 .../api/PinotTableRestletResourceTest.java         | 132 ++++++++++++++-------
 .../java/org/apache/pinot/spi/utils/JsonUtils.java |  49 ++++++++
 .../org/apache/pinot/spi/utils/JsonUtilsTest.java  |  16 +++
 .../pinot/spi/utils/JsonUtilsTestSamplePojo.java   |  84 +++++++++++++
 7 files changed, 278 insertions(+), 65 deletions(-)

diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/SuccessResponse.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/ConfigSuccessResponse.java
similarity index 67%
copy from pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/SuccessResponse.java
copy to pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/ConfigSuccessResponse.java
index 3c5ba7a4f5..ab5eff7080 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/SuccessResponse.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/ConfigSuccessResponse.java
@@ -18,14 +18,17 @@
  */
 package org.apache.pinot.controller.api.resources;
 
-public final class SuccessResponse {
-  private final String _status;
+import java.util.Map;
 
-  public SuccessResponse(String status) {
-    _status = status;
+public final class ConfigSuccessResponse extends SuccessResponse {
+  private final Map<String, Object> _unrecognizedProperties;
+
+  public ConfigSuccessResponse(String status, Map<String, Object> unrecognizedProperties) {
+    super(status);
+    _unrecognizedProperties = unrecognizedProperties;
   }
 
-  public String getStatus() {
-    return _status;
+  public Map<String, Object> getUnrecognizedProperties() {
+    return _unrecognizedProperties;
   }
 }
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java
index 9b288be172..cc253b4183 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java
@@ -34,6 +34,7 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.Executor;
 import java.util.concurrent.ExecutorService;
 import javax.annotation.Nullable;
@@ -55,6 +56,7 @@ import javax.ws.rs.core.Response;
 import org.apache.commons.configuration.BaseConfiguration;
 import org.apache.commons.configuration.Configuration;
 import org.apache.commons.httpclient.HttpConnectionManager;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.helix.AccessOption;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
@@ -153,22 +155,25 @@ public class PinotTableRestletResource {
   @Produces(MediaType.APPLICATION_JSON)
   @Path("/tables")
   @ApiOperation(value = "Adds a table", notes = "Adds a table")
-  public SuccessResponse addTable(
+  public ConfigSuccessResponse addTable(
       String tableConfigStr,
       @ApiParam(value = "comma separated list of validation type(s) to skip. supported types: (ALL|TASK|UPSERT)")
       @QueryParam("validationTypesToSkip") @Nullable String typesToSkip, @Context HttpHeaders httpHeaders,
       @Context Request request) {
     // TODO introduce a table config ctor with json string.
+    Pair<TableConfig, Map<String, Object>> tableConfigAndUnrecognizedProperties;
     TableConfig tableConfig;
     String tableName;
     try {
-      tableConfig = JsonUtils.stringToObject(tableConfigStr, TableConfig.class);
+      tableConfigAndUnrecognizedProperties =
+          JsonUtils.stringToObjectAndUnrecognizedProperties(tableConfigStr, TableConfig.class);
+      tableConfig = tableConfigAndUnrecognizedProperties.getLeft();
 
       // validate permission
       tableName = tableConfig.getTableName();
       String endpointUrl = request.getRequestURL().toString();
-      _accessControlUtils
-          .validatePermission(tableName, AccessType.CREATE, httpHeaders, endpointUrl, _accessControlFactory.create());
+      _accessControlUtils.validatePermission(tableName, AccessType.CREATE, httpHeaders, endpointUrl,
+          _accessControlFactory.create());
 
       Schema schema = _pinotHelixResourceManager.getSchemaForTableConfig(tableConfig);
 
@@ -194,7 +199,8 @@ public class PinotTableRestletResource {
       // TODO: validate that table was created successfully
       // (in realtime case, metadata might not have been created but would be created successfully in the next run of
       // the validation manager)
-      return new SuccessResponse("Table " + tableName + " succesfully added");
+      return new ConfigSuccessResponse("Table " + tableName + " succesfully added",
+          tableConfigAndUnrecognizedProperties.getRight());
     } catch (Exception e) {
       _controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_TABLE_ADD_ERROR, 1L);
       if (e instanceof InvalidTableConfigException) {
@@ -437,14 +443,17 @@ public class PinotTableRestletResource {
   @Authenticate(AccessType.UPDATE)
   @Produces(MediaType.APPLICATION_JSON)
   @ApiOperation(value = "Updates table config for a table", notes = "Updates table config for a table")
-  public SuccessResponse updateTableConfig(
+  public ConfigSuccessResponse updateTableConfig(
       @ApiParam(value = "Name of the table to update", required = true) @PathParam("tableName") String tableName,
       @ApiParam(value = "comma separated list of validation type(s) to skip. supported types: (ALL|TASK|UPSERT)")
       @QueryParam("validationTypesToSkip") @Nullable String typesToSkip, String tableConfigString)
       throws Exception {
+    Pair<TableConfig, Map<String, Object>> tableConfigJsonPojoWithUnparsableProps;
     TableConfig tableConfig;
     try {
-      tableConfig = JsonUtils.stringToObject(tableConfigString, TableConfig.class);
+      tableConfigJsonPojoWithUnparsableProps =
+          JsonUtils.stringToObjectAndUnrecognizedProperties(tableConfigString, TableConfig.class);
+      tableConfig = tableConfigJsonPojoWithUnparsableProps.getLeft();
       Schema schema = _pinotHelixResourceManager.getSchemaForTableConfig(tableConfig);
       TableConfigUtils.validate(tableConfig, schema, typesToSkip, _controllerConf.isDisableIngestionGroovy());
     } catch (Exception e) {
@@ -482,8 +491,8 @@ public class PinotTableRestletResource {
       _controllerMetrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_TABLE_UPDATE_ERROR, 1L);
       throw e;
     }
-
-    return new SuccessResponse("Table config updated for " + tableName);
+    return new ConfigSuccessResponse("Table config updated for " + tableName,
+        tableConfigJsonPojoWithUnparsableProps.getRight());
   }
 
   @POST
@@ -492,18 +501,22 @@ public class PinotTableRestletResource {
   @ApiOperation(value = "Validate table config for a table",
       notes = "This API returns the table config that matches the one you get from 'GET /tables/{tableName}'."
           + " This allows us to validate table config before apply.")
-  public String checkTableConfig(
+  public ObjectNode checkTableConfig(
       String tableConfigStr,
       @ApiParam(value = "comma separated list of validation type(s) to skip. supported types: (ALL|TASK|UPSERT)")
       @QueryParam("validationTypesToSkip") @Nullable String typesToSkip) {
-    TableConfig tableConfig;
+    Pair<TableConfig, Map<String, Object>> tableConfig;
     try {
-      tableConfig = JsonUtils.stringToObject(tableConfigStr, TableConfig.class);
+      tableConfig = JsonUtils.stringToObjectAndUnrecognizedProperties(tableConfigStr, TableConfig.class);
     } catch (IOException e) {
       String msg = String.format("Invalid table config json string: %s", tableConfigStr);
       throw new ControllerApplicationException(LOGGER, msg, Response.Status.BAD_REQUEST, e);
     }
-    return validateConfig(tableConfig, _pinotHelixResourceManager.getSchemaForTableConfig(tableConfig), typesToSkip);
+    ObjectNode validationResponse =
+        validateConfig(tableConfig.getLeft(), _pinotHelixResourceManager.getSchemaForTableConfig(tableConfig.getLeft()),
+            typesToSkip);
+    validationResponse.set("unrecognizedProperties", JsonUtils.objectToJsonNode(tableConfig.getRight()));
+    return validationResponse;
   }
 
   @Deprecated
@@ -525,10 +538,10 @@ public class PinotTableRestletResource {
     if (schema == null) {
       schema = _pinotHelixResourceManager.getSchemaForTableConfig(tableConfig);
     }
-    return validateConfig(tableSchemaConfig.getTableConfig(), schema, typesToSkip);
+    return validateConfig(tableSchemaConfig.getTableConfig(), schema, typesToSkip).toString();
   }
 
-  private String validateConfig(TableConfig tableConfig, Schema schema, @Nullable String typesToSkip) {
+  private ObjectNode validateConfig(TableConfig tableConfig, Schema schema, @Nullable String typesToSkip) {
     try {
       if (schema == null) {
         throw new SchemaNotFoundException("Got empty schema");
@@ -540,7 +553,7 @@ public class PinotTableRestletResource {
       } else {
         tableConfigValidateStr.set(TableType.REALTIME.name(), tableConfig.toJsonNode());
       }
-      return tableConfigValidateStr.toString();
+      return tableConfigValidateStr;
     } catch (Exception e) {
       String msg = String.format("Invalid table config: %s. %s", tableConfig.getTableName(), e.getMessage());
       throw new ControllerApplicationException(LOGGER, msg, Response.Status.BAD_REQUEST, e);
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/SuccessResponse.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/SuccessResponse.java
index 3c5ba7a4f5..52e9437311 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/SuccessResponse.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/SuccessResponse.java
@@ -18,7 +18,7 @@
  */
 package org.apache.pinot.controller.api.resources;
 
-public final class SuccessResponse {
+public class SuccessResponse {
   private final String _status;
 
   public SuccessResponse(String status) {
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotTableRestletResourceTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotTableRestletResourceTest.java
index d4e75b4e3f..21fe16aeaf 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotTableRestletResourceTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/api/PinotTableRestletResourceTest.java
@@ -193,7 +193,8 @@ public class PinotTableRestletResourceTest {
             ImmutableMap.of(PinotTaskManager.SCHEDULE_KEY, "0 */10 * ? * * *")))).build();
     try {
       String response = ControllerTestUtils.sendPostRequest(_createTableUrl, tableConfig.toJsonString());
-      Assert.assertEquals(response, "{\"status\":\"Table test_table_cron_schedule_OFFLINE succesfully added\"}");
+      Assert.assertEquals(response,
+          "{\"unrecognizedProperties\":{},\"status\":\"Table test_table_cron_schedule_OFFLINE succesfully added\"}");
     } catch (IOException e) {
       // Expected 400 Bad Request
       Assert.fail("This is a valid table config with cron schedule");
@@ -204,9 +205,9 @@ public class PinotTableRestletResourceTest {
         new TableTaskConfig(ImmutableMap.of(MinionConstants.SegmentGenerationAndPushTask.TASK_TYPE,
             ImmutableMap.of(PinotTaskManager.SCHEDULE_KEY, "5 5 5 5 5 5 5")))).build();
     try {
-      ControllerTestUtils
-          .sendPutRequest(ControllerTestUtils.getControllerRequestURLBuilder().forUpdateTableConfig(rawTableName),
-              tableConfig.toJsonString());
+      ControllerTestUtils.sendPutRequest(
+          ControllerTestUtils.getControllerRequestURLBuilder().forUpdateTableConfig(rawTableName),
+          tableConfig.toJsonString());
       Assert.fail("Update of an OFFLINE table with an invalid cron expression does not fail");
     } catch (IOException e) {
       // Expected 400 Bad Request
@@ -313,9 +314,9 @@ public class PinotTableRestletResourceTest {
     tableConfig.getValidationConfig().setRetentionTimeUnit("HOURS");
     tableConfig.getValidationConfig().setRetentionTimeValue("10");
 
-    JsonNode jsonResponse = JsonUtils.stringToJsonNode(ControllerTestUtils
-        .sendPutRequest(ControllerTestUtils.getControllerRequestURLBuilder().forUpdateTableConfig(tableName),
-            tableConfig.toJsonString()));
+    JsonNode jsonResponse = JsonUtils.stringToJsonNode(ControllerTestUtils.sendPutRequest(
+        ControllerTestUtils.getControllerRequestURLBuilder().forUpdateTableConfig(tableName),
+        tableConfig.toJsonString()));
     Assert.assertTrue(jsonResponse.has("status"));
 
     TableConfig modifiedConfig = getTableConfig(tableName, "OFFLINE");
@@ -333,9 +334,9 @@ public class PinotTableRestletResourceTest {
 
     QuotaConfig quota = new QuotaConfig("10G", "100.0");
     tableConfig.setQuotaConfig(quota);
-    ControllerTestUtils
-        .sendPutRequest(ControllerTestUtils.getControllerRequestURLBuilder().forUpdateTableConfig(tableName),
-            tableConfig.toJsonString());
+    ControllerTestUtils.sendPutRequest(
+        ControllerTestUtils.getControllerRequestURLBuilder().forUpdateTableConfig(tableName),
+        tableConfig.toJsonString());
     modifiedConfig = getTableConfig(tableName, "REALTIME");
     Assert.assertNotNull(modifiedConfig.getQuotaConfig());
     Assert.assertEquals(modifiedConfig.getQuotaConfig().getStorage(), "10G");
@@ -345,9 +346,9 @@ public class PinotTableRestletResourceTest {
       // table does not exist
       ObjectNode tableConfigJson = (ObjectNode) tableConfig.toJsonNode();
       tableConfigJson.put(TableConfig.TABLE_NAME_KEY, "noSuchTable_REALTIME");
-      ControllerTestUtils
-          .sendPutRequest(ControllerTestUtils.getControllerRequestURLBuilder().forUpdateTableConfig("noSuchTable"),
-              tableConfigJson.toString());
+      ControllerTestUtils.sendPutRequest(
+          ControllerTestUtils.getControllerRequestURLBuilder().forUpdateTableConfig("noSuchTable"),
+          tableConfigJson.toString());
     } catch (Exception e) {
       Assert.assertTrue(e instanceof IOException);
     }
@@ -368,7 +369,8 @@ public class PinotTableRestletResourceTest {
 
     // Rebalance should return status NO_OP
     RebalanceResult rebalanceResult = JsonUtils.stringToObject(ControllerTestUtils.sendPostRequest(
-        ControllerTestUtils.getControllerRequestURLBuilder().forTableRebalance(OFFLINE_TABLE_NAME, "offline"), null),
+            ControllerTestUtils.getControllerRequestURLBuilder().forTableRebalance(OFFLINE_TABLE_NAME, "offline"),
+            null),
         RebalanceResult.class);
     Assert.assertEquals(rebalanceResult.getStatus(), RebalanceResult.Status.NO_OP);
   }
@@ -379,7 +381,8 @@ public class PinotTableRestletResourceTest {
     // Case 1: Create a REALTIME table and delete it directly w/o using query param.
     TableConfig realtimeTableConfig = _realtimeBuilder.setTableName("table0").build();
     String creationResponse = ControllerTestUtils.sendPostRequest(_createTableUrl, realtimeTableConfig.toJsonString());
-    Assert.assertEquals(creationResponse, "{\"status\":\"Table table0_REALTIME succesfully added\"}");
+    Assert.assertEquals(creationResponse,
+        "{\"unrecognizedProperties\":{},\"status\":\"Table table0_REALTIME succesfully added\"}");
 
     // Delete realtime table using REALTIME suffix.
     String deleteResponse = ControllerTestUtils.sendDeleteRequest(
@@ -389,7 +392,8 @@ public class PinotTableRestletResourceTest {
     // Case 2: Create an offline table and delete it directly w/o using query param.
     TableConfig offlineTableConfig = _offlineBuilder.setTableName("table0").build();
     creationResponse = ControllerTestUtils.sendPostRequest(_createTableUrl, offlineTableConfig.toJsonString());
-    Assert.assertEquals(creationResponse, "{\"status\":\"Table table0_OFFLINE succesfully added\"}");
+    Assert.assertEquals(creationResponse,
+        "{\"unrecognizedProperties\":{},\"status\":\"Table table0_OFFLINE succesfully added\"}");
 
     // Delete offline table using OFFLINE suffix.
     deleteResponse = ControllerTestUtils.sendDeleteRequest(
@@ -399,29 +403,34 @@ public class PinotTableRestletResourceTest {
     // Case 3: Create REALTIME and OFFLINE tables and delete both of them.
     TableConfig rtConfig1 = _realtimeBuilder.setTableName("table1").build();
     creationResponse = ControllerTestUtils.sendPostRequest(_createTableUrl, rtConfig1.toJsonString());
-    Assert.assertEquals(creationResponse, "{\"status\":\"Table table1_REALTIME succesfully added\"}");
+    Assert.assertEquals(creationResponse,
+        "{\"unrecognizedProperties\":{},\"status\":\"Table table1_REALTIME succesfully added\"}");
 
     TableConfig offlineConfig1 = _offlineBuilder.setTableName("table1").build();
     creationResponse = ControllerTestUtils.sendPostRequest(_createTableUrl, offlineConfig1.toJsonString());
-    Assert.assertEquals(creationResponse, "{\"status\":\"Table table1_OFFLINE succesfully added\"}");
+    Assert.assertEquals(creationResponse,
+        "{\"unrecognizedProperties\":{},\"status\":\"Table table1_OFFLINE succesfully added\"}");
 
-    deleteResponse = ControllerTestUtils
-        .sendDeleteRequest(StringUtil.join("/", ControllerTestUtils.getControllerBaseApiUrl(), "tables", "table1"));
+    deleteResponse = ControllerTestUtils.sendDeleteRequest(
+        StringUtil.join("/", ControllerTestUtils.getControllerBaseApiUrl(), "tables", "table1"));
     Assert.assertEquals(deleteResponse, "{\"status\":\"Tables: [table1_OFFLINE, table1_REALTIME] deleted\"}");
 
     // Case 4: Create REALTIME and OFFLINE tables and delete the realtime/offline table using query params.
     TableConfig rtConfig2 = _realtimeBuilder.setTableName("table2").build();
     creationResponse = ControllerTestUtils.sendPostRequest(_createTableUrl, rtConfig2.toJsonString());
-    Assert.assertEquals(creationResponse, "{\"status\":\"Table table2_REALTIME succesfully added\"}");
+    Assert.assertEquals(creationResponse,
+        "{\"unrecognizedProperties\":{},\"status\":\"Table table2_REALTIME succesfully added\"}");
 
     TableConfig offlineConfig2 = _offlineBuilder.setTableName("table2").build();
     creationResponse = ControllerTestUtils.sendPostRequest(_createTableUrl, offlineConfig2.toJsonString());
-    Assert.assertEquals(creationResponse, "{\"status\":\"Table table2_OFFLINE succesfully added\"}");
+    Assert.assertEquals(creationResponse,
+        "{\"unrecognizedProperties\":{},\"status\":\"Table table2_OFFLINE succesfully added\"}");
 
     // The conflict between param type and table name suffix causes no table being deleted.
     try {
-      ControllerTestUtils.sendDeleteRequest(StringUtil
-          .join("/", ControllerTestUtils.getControllerBaseApiUrl(), "tables", "table2_OFFLINE?type=realtime"));
+      ControllerTestUtils.sendDeleteRequest(
+          StringUtil.join("/", ControllerTestUtils.getControllerBaseApiUrl(), "tables",
+              "table2_OFFLINE?type=realtime"));
       Assert.fail("Deleting a realtime table with OFFLINE suffix.");
     } catch (Exception e) {
       Assert.assertTrue(e instanceof IOException);
@@ -447,11 +456,13 @@ public class PinotTableRestletResourceTest {
     // Case 6: Create REALTIME and OFFLINE tables and delete the realtime/offline table using query params and suffixes.
     TableConfig rtConfig3 = _realtimeBuilder.setTableName("table3").build();
     creationResponse = ControllerTestUtils.sendPostRequest(_createTableUrl, rtConfig3.toJsonString());
-    Assert.assertEquals(creationResponse, "{\"status\":\"Table table3_REALTIME succesfully added\"}");
+    Assert.assertEquals(creationResponse,
+        "{\"unrecognizedProperties\":{},\"status\":\"Table table3_REALTIME succesfully added\"}");
 
     TableConfig offlineConfig3 = _offlineBuilder.setTableName("table3").build();
     creationResponse = ControllerTestUtils.sendPostRequest(_createTableUrl, offlineConfig3.toJsonString());
-    Assert.assertEquals(creationResponse, "{\"status\":\"Table table3_OFFLINE succesfully added\"}");
+    Assert.assertEquals(creationResponse,
+        "{\"unrecognizedProperties\":{},\"status\":\"Table table3_OFFLINE succesfully added\"}");
 
     deleteResponse = ControllerTestUtils.sendDeleteRequest(
         StringUtil.join("/", ControllerTestUtils.getControllerBaseApiUrl(), "tables", "table3_REALTIME?type=realtime"));
@@ -469,27 +480,31 @@ public class PinotTableRestletResourceTest {
     // Create a valid REALTIME table
     TableConfig realtimeTableConfig = _realtimeBuilder.setTableName("testTable").build();
     String creationResponse = ControllerTestUtils.sendPostRequest(_createTableUrl, realtimeTableConfig.toJsonString());
-    Assert.assertEquals(creationResponse, "{\"status\":\"Table testTable_REALTIME succesfully added\"}");
+    Assert.assertEquals(creationResponse,
+        "{\"unrecognizedProperties\":{},\"status\":\"Table testTable_REALTIME succesfully added\"}");
 
     // Create a valid OFFLINE table
     TableConfig offlineTableConfig = _offlineBuilder.setTableName("testTable").build();
     creationResponse = ControllerTestUtils.sendPostRequest(_createTableUrl, offlineTableConfig.toJsonString());
-    Assert.assertEquals(creationResponse, "{\"status\":\"Table testTable_OFFLINE succesfully added\"}");
+    Assert.assertEquals(creationResponse,
+        "{\"unrecognizedProperties\":{},\"status\":\"Table testTable_OFFLINE succesfully added\"}");
 
     // Case 1: Check table state with specifying tableType as realtime should return 1 [enabled]
-    String realtimeStateResponse = ControllerTestUtils.sendGetRequest(StringUtil
-        .join("/", ControllerTestUtils.getControllerBaseApiUrl(), "tables", "testTable", "state?type=realtime"));
+    String realtimeStateResponse = ControllerTestUtils.sendGetRequest(
+        StringUtil.join("/", ControllerTestUtils.getControllerBaseApiUrl(), "tables", "testTable",
+            "state?type=realtime"));
     Assert.assertEquals(realtimeStateResponse, "{\"state\":\"enabled\"}");
 
     // Case 2: Check table state with specifying tableType as offline should return 1 [enabled]
-    String offlineStateResponse = ControllerTestUtils.sendGetRequest(StringUtil
-        .join("/", ControllerTestUtils.getControllerBaseApiUrl(), "tables", "testTable", "state?type=offline"));
+    String offlineStateResponse = ControllerTestUtils.sendGetRequest(
+        StringUtil.join("/", ControllerTestUtils.getControllerBaseApiUrl(), "tables", "testTable",
+            "state?type=offline"));
     Assert.assertEquals(offlineStateResponse, "{\"state\":\"enabled\"}");
 
     // Case 3: Request table state with invalid type should return bad request
     try {
-      ControllerTestUtils.sendGetRequest(StringUtil
-          .join("/", ControllerTestUtils.getControllerBaseApiUrl(), "tables", "testTable",
+      ControllerTestUtils.sendGetRequest(
+          StringUtil.join("/", ControllerTestUtils.getControllerBaseApiUrl(), "tables", "testTable",
               "state?type=non_valid_type"));
       Assert.fail("Requesting with invalid type should fail");
     } catch (Exception e) {
@@ -500,8 +515,9 @@ public class PinotTableRestletResourceTest {
     // Case 4: Request state for non-existent table should return not found
     boolean notFoundException = false;
     try {
-      ControllerTestUtils.sendGetRequest(StringUtil
-          .join("/", ControllerTestUtils.getControllerBaseApiUrl(), "tables", "table_not_exist", "state?type=offline"));
+      ControllerTestUtils.sendGetRequest(
+          StringUtil.join("/", ControllerTestUtils.getControllerBaseApiUrl(), "tables", "table_not_exist",
+              "state?type=offline"));
       Assert.fail("Requesting state for non-existent table should fail");
     } catch (Exception e) {
       // Expected 404 Not Found
@@ -523,9 +539,9 @@ public class PinotTableRestletResourceTest {
         _offlineBuilder.setTableName(tableName).setInvertedIndexColumns(Arrays.asList("dimA", "dimB")).build();
 
     try {
-      ControllerTestUtils
-          .sendPostRequest(StringUtil.join("/", ControllerTestUtils.getControllerBaseApiUrl(), "tables", "validate"),
-              offlineTableConfig.toJsonString());
+      ControllerTestUtils.sendPostRequest(
+          StringUtil.join("/", ControllerTestUtils.getControllerBaseApiUrl(), "tables", "validate"),
+          offlineTableConfig.toJsonString());
     } catch (IOException e) {
       Assert.fail("Valid table config with existing schema validation should succeed.");
     }
@@ -536,9 +552,9 @@ public class PinotTableRestletResourceTest {
             .build();
 
     try {
-      ControllerTestUtils
-          .sendPostRequest(StringUtil.join("/", ControllerTestUtils.getControllerBaseApiUrl(), "tables", "validate"),
-              offlineTableConfig.toJsonString());
+      ControllerTestUtils.sendPostRequest(
+          StringUtil.join("/", ControllerTestUtils.getControllerBaseApiUrl(), "tables", "validate"),
+          offlineTableConfig.toJsonString());
       Assert.fail("Validation of an invalid table config should fail.");
     } catch (IOException e) {
       // Expected 400 Bad Request
@@ -593,6 +609,38 @@ public class PinotTableRestletResourceTest {
     }
   }
 
+  @Test
+  public void testUnrecognizedProperties()
+      throws IOException {
+    // Create an OFFLINE table with a valid name but with unrecognizedProperties which should succeed
+    // Should have unrecognizedProperties set correctly
+    String tableName = "valid_table_name_extra_props";
+    TableConfig offlineTableConfig = _realtimeBuilder.setTableName("valid_table_name_extra_props").build();
+    JsonNode jsonNode = JsonUtils.objectToJsonNode(offlineTableConfig);
+    ((ObjectNode) jsonNode).put("illegalKey1", 1);
+    ObjectNode internalObj = JsonUtils.newObjectNode();
+    internalObj.put("illegalKey3", 2);
+    ((ObjectNode) jsonNode).put("illegalKey2", internalObj);
+
+    String creationResponse = ControllerTestUtils.sendPostRequest(_createTableUrl, jsonNode.toString());
+    Assert.assertEquals(creationResponse,
+        "{\"unrecognizedProperties\":{\"/illegalKey1\":1,\"/illegalKey2/illegalKey3\":2},\"status\":\"Table "
+            + "valid_table_name_extra_props_REALTIME succesfully added\"}");
+
+    // update table with unrecognizedProperties
+    String updationResponse = ControllerTestUtils.sendPutRequest(
+        ControllerTestUtils.getControllerRequestURLBuilder().forUpdateTableConfig(tableName), jsonNode.toString());
+    Assert.assertEquals(updationResponse,
+        "{\"unrecognizedProperties\":{\"/illegalKey1\":1,\"/illegalKey2/illegalKey3\":2},\"status\":\"Table "
+            + "config updated for valid_table_name_extra_props\"}");
+
+    // validate table with unrecognizedProperties
+    String validationResponse = ControllerTestUtils.sendPostRequest(
+        StringUtil.join("/", ControllerTestUtils.getControllerBaseApiUrl(), "tables", "validate"), jsonNode.toString());
+    Assert.assertTrue(validationResponse.contains(
+        "unrecognizedProperties\":{\"/illegalKey1\":1," + "\"/illegalKey2/illegalKey3\":2}}"));
+  }
+
   @AfterClass
   public void tearDown() {
     ControllerTestUtils.cleanup();
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/JsonUtils.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/JsonUtils.java
index 0089085464..4b62f8a188 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/JsonUtils.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/JsonUtils.java
@@ -30,19 +30,27 @@ import com.fasterxml.jackson.databind.node.ArrayNode;
 import com.fasterxml.jackson.databind.node.JsonNodeFactory;
 import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.MapDifference;
+import com.google.common.collect.Maps;
 import java.io.ByteArrayInputStream;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.util.AbstractMap;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Iterator;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
 import javax.annotation.Nullable;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.pinot.spi.config.table.ingestion.ComplexTypeConfig;
 import org.apache.pinot.spi.data.DateTimeFieldSpec;
 import org.apache.pinot.spi.data.DateTimeFormatSpec;
@@ -71,12 +79,53 @@ public class JsonUtils {
   public static final ObjectReader DEFAULT_READER = DEFAULT_MAPPER.reader();
   public static final ObjectWriter DEFAULT_WRITER = DEFAULT_MAPPER.writer();
   public static final ObjectWriter DEFAULT_PRETTY_WRITER = DEFAULT_MAPPER.writerWithDefaultPrettyPrinter();
+  private static final TypeReference<HashMap<String, Object>> GENERIC_JSON_TYPE =
+      new TypeReference<HashMap<String, Object>>() { };
 
   public static <T> T stringToObject(String jsonString, Class<T> valueType)
       throws IOException {
     return DEFAULT_READER.forType(valueType).readValue(jsonString);
   }
 
+  public static <T> Pair<T, Map<String, Object>> stringToObjectAndUnrecognizedProperties(String jsonString,
+      Class<T> valueType)
+      throws IOException {
+    T instance = DEFAULT_READER.forType(valueType).readValue(jsonString);
+
+    String instanceJson = DEFAULT_MAPPER.writeValueAsString(instance);
+    Map<String, Object> inputJsonMap = flatten(DEFAULT_MAPPER.readValue(jsonString, GENERIC_JSON_TYPE));
+    Map<String, Object> instanceJsonMap = flatten(DEFAULT_MAPPER.readValue(instanceJson, GENERIC_JSON_TYPE));
+
+    MapDifference<String, Object> difference = Maps.difference(inputJsonMap, instanceJsonMap);
+    return Pair.of(instance, difference.entriesOnlyOnLeft());
+  }
+
+  public static Map<String, Object> flatten(Map<String, Object> map) {
+    return map.entrySet().stream()
+        .flatMap(JsonUtils::flatten)
+        .collect(LinkedHashMap::new, (m, e) -> m.put("/" + e.getKey(), e.getValue()), LinkedHashMap::putAll);
+  }
+
+  private static Stream<Map.Entry<String, Object>> flatten(Map.Entry<String, Object> entry) {
+    if (entry == null) {
+      return Stream.empty();
+    }
+
+    if (entry.getValue() instanceof Map<?, ?>) {
+      return ((Map<?, ?>) entry.getValue()).entrySet().stream()
+          .flatMap(e -> flatten(new AbstractMap.SimpleEntry<>(entry.getKey() + "/" + e.getKey(), e.getValue())));
+    }
+
+    if (entry.getValue() instanceof List<?>) {
+      List<?> list = (List<?>) entry.getValue();
+      return IntStream.range(0, list.size())
+          .mapToObj(i -> new AbstractMap.SimpleEntry<String, Object>(entry.getKey() + "/" + i, list.get(i)))
+          .flatMap(JsonUtils::flatten);
+    }
+
+    return Stream.of(entry);
+  }
+
   public static <T> T stringToObject(String jsonString, TypeReference<T> valueTypeRef)
       throws IOException {
     return DEFAULT_READER.forType(valueTypeRef).readValue(jsonString);
diff --git a/pinot-spi/src/test/java/org/apache/pinot/spi/utils/JsonUtilsTest.java b/pinot-spi/src/test/java/org/apache/pinot/spi/utils/JsonUtilsTest.java
index db0325b983..b1df9ce6d8 100644
--- a/pinot-spi/src/test/java/org/apache/pinot/spi/utils/JsonUtilsTest.java
+++ b/pinot-spi/src/test/java/org/apache/pinot/spi/utils/JsonUtilsTest.java
@@ -27,6 +27,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.pinot.spi.config.table.ingestion.ComplexTypeConfig;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.Schema;
@@ -273,6 +274,21 @@ public class JsonUtilsTest {
     }
   }
 
+  @Test
+  public void testUnrecognizedJsonProperties()
+      throws Exception {
+    String inputJsonMissingProp = "{\"primitiveIntegerField\": 123, \"missingProp\": 567,"
+        + " \"missingObjectProp\": {\"somestuff\": \"data\", \"somemorestuff\":\"moredata\"},"
+        + "  \"classField\": {\"internalIntField\": 12, \"internalMissingField\": \"somedata\"}}";
+    Pair<JsonUtilsTestSamplePojo, Map<String, Object>> parsedResp =
+        JsonUtils.stringToObjectAndUnrecognizedProperties(inputJsonMissingProp, JsonUtilsTestSamplePojo.class);
+
+    Assert.assertTrue(parsedResp.getRight().containsKey("/missingProp"));
+    Assert.assertTrue(parsedResp.getRight().containsKey("/missingObjectProp/somestuff"));
+    Assert.assertTrue(parsedResp.getRight().containsKey("/missingObjectProp/somemorestuff"));
+    Assert.assertTrue(parsedResp.getRight().containsKey("/classField/internalMissingField"));
+  }
+
   @Test
   public void testInferSchema()
       throws Exception {
diff --git a/pinot-spi/src/test/java/org/apache/pinot/spi/utils/JsonUtilsTestSamplePojo.java b/pinot-spi/src/test/java/org/apache/pinot/spi/utils/JsonUtilsTestSamplePojo.java
new file mode 100644
index 0000000000..094ee7e679
--- /dev/null
+++ b/pinot-spi/src/test/java/org/apache/pinot/spi/utils/JsonUtilsTestSamplePojo.java
@@ -0,0 +1,84 @@
+/**
+ * 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.spi.utils;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class JsonUtilsTestSamplePojo {
+  int _primitiveIntegerField;
+  String _stringField;
+  Long _longField;
+  Clazz _classField;
+
+  public int getPrimitiveIntegerField() {
+    return _primitiveIntegerField;
+  }
+
+  public void setPrimitiveIntegerField(int primitiveIntegerField) {
+    _primitiveIntegerField = primitiveIntegerField;
+  }
+
+  public String getStringField() {
+    return _stringField;
+  }
+
+  public void setStringField(String stringField) {
+    _stringField = stringField;
+  }
+
+  public Long getLongField() {
+    return _longField;
+  }
+
+  public void setLongField(Long longField) {
+    _longField = longField;
+  }
+
+  public Clazz getClassField() {
+    return _classField;
+  }
+
+  public void setClassField(Clazz classField) {
+    _classField = classField;
+  }
+
+  @JsonIgnoreProperties(ignoreUnknown = true)
+  public class Clazz {
+    Integer _internalIntField;
+    String _internalStringField;
+
+    public int getInternalIntField() {
+      return _internalIntField;
+    }
+
+    public void setInternalIntField(int internalIntField) {
+      _internalIntField = internalIntField;
+    }
+
+    public String getInternalStringField() {
+      return _internalStringField;
+    }
+
+    public void setInternalStringField(String internalStringField) {
+      _internalStringField = internalStringField;
+    }
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org