You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ja...@apache.org on 2019/01/11 04:54:18 UTC

[incubator-pinot] branch master updated: Unify all JSON usage to fasterxml.jackson package (#3677)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new f6c4d3b  Unify all JSON usage to fasterxml.jackson package (#3677)
f6c4d3b is described below

commit f6c4d3bf7103a77a791d6e01c6b4221a355c35cf
Author: Xiaotian (Jackie) Jiang <17...@users.noreply.github.com>
AuthorDate: Thu Jan 10 20:54:13 2019 -0800

    Unify all JSON usage to fasterxml.jackson package (#3677)
    
    Remove the usage of json package from org.json, org.codehaus.jackson, com.google.code.gson, com.alibaba
    Add JsonUtils class to reuse ObjectMapper and provide util methods
---
 pinot-api/pom.xml                                  |   4 +-
 .../pinot/client/AggregationResultSet.java         |  23 +-
 .../com/linkedin/pinot/client/BrokerResponse.java  |  41 ++--
 .../linkedin/pinot/client/ExternalViewReader.java  |  60 +++--
 .../linkedin/pinot/client/GroupByResultSet.java    |  47 ++--
 .../client/JsonAsyncHttpPinotClientTransport.java  |  29 ++-
 .../com/linkedin/pinot/client/ResultSetGroup.java  |  16 +-
 .../linkedin/pinot/client/SelectionResultSet.java  |  45 ++--
 .../linkedin/pinot/client/ResultSetGroupTest.java  |   4 +-
 pinot-broker/pom.xml                               |  12 -
 .../broker/api/resources/PinotClientRequest.java   |  11 +-
 .../requesthandler/BaseBrokerRequestHandler.java   |  16 +-
 .../requesthandler/BrokerRequestHandler.java       |   8 +-
 .../routing/HelixExternalViewBasedRouting.java     |  23 +-
 .../pinot/broker/routing/TimeBoundaryService.java  |  12 +-
 .../queryquota/TableQueryQuotaManagerTest.java     |   4 +-
 .../pinot/broker/routing/RoutingTableTest.java     |   4 +-
 pinot-common/pom.xml                               |  20 +-
 .../pinot/common/config/ColumnPartitionConfig.java |   4 +-
 .../pinot/common/config/IndexingConfig.java        |   6 +-
 .../linkedin/pinot/common/config/QuotaConfig.java  |  24 +-
 .../common/config/ReplicaGroupStrategyConfig.java  |   2 +-
 .../pinot/common/config/RoutingConfig.java         |  21 +-
 .../common/config/SegmentPartitionConfig.java      |  24 +-
 .../SegmentsValidationAndRetentionConfig.java      |   4 +-
 .../pinot/common/config/StarTreeIndexConfig.java   |   2 +-
 .../common/config/StreamConsumptionConfig.java     |  21 +-
 .../linkedin/pinot/common/config/TableConfig.java  | 159 +++++++-------
 .../pinot/common/config/TableCustomConfig.java     |   2 +-
 .../pinot/common/config/TableTaskConfig.java       |  18 +-
 .../pinot/common/config/TagOverrideConfig.java     |   5 +-
 .../com/linkedin/pinot/common/config/Tenant.java   | 119 ++++------
 .../linkedin/pinot/common/config/TenantConfig.java |   3 +-
 .../pinot/common/data/DateTimeFieldSpec.java       |  14 +-
 .../pinot/common/data/DimensionFieldSpec.java      |   4 +-
 .../com/linkedin/pinot/common/data/FieldSpec.java  |  50 ++---
 .../pinot/common/data/MetricFieldSpec.java         |  14 +-
 .../com/linkedin/pinot/common/data/Schema.java     |  43 ++--
 .../pinot/common/data/StarTreeIndexSpec.java       |  17 +-
 .../linkedin/pinot/common/data/TimeFieldSpec.java  |  15 +-
 .../pinot/common/data/TimeGranularitySpec.java     |  23 +-
 .../metadata/segment/ColumnPartitionMetadata.java  |  24 +-
 .../metadata/segment/SegmentPartitionMetadata.java |  20 +-
 .../common/metadata/segment/SegmentZKMetadata.java |  10 +-
 .../SegmentZKMetadataCustomMapModifier.java        |  31 +--
 .../protocols/SegmentCompletionProtocol.java       | 148 +++++++------
 .../pinot/common/response/BrokerResponse.java      |  10 +-
 .../common/response/broker/AggregationResult.java  |  14 +-
 .../response/broker/BrokerResponseNative.java      |  44 ++--
 .../common/response/broker/GroupByResult.java      |   4 +-
 .../response/broker/QueryProcessingException.java  |   2 +-
 .../common/response/broker/SelectionResults.java   |   6 +-
 .../common/restlet/resources/RebalanceResult.java  |   2 +-
 .../common/restlet/resources/ResourceUtils.java    |   5 +-
 .../common/restlet/resources/SegmentSizeInfo.java  |   2 +-
 .../common/restlet/resources/TableSizeInfo.java    |   2 +-
 .../pinot/common/restlet/resources/TablesList.java |  11 +-
 .../common/utils/FileUploadDownloadClient.java     |   5 +-
 .../com/linkedin/pinot/common/utils/JsonUtils.java | 156 +++++++++++++
 .../com/linkedin/pinot/startree/hll/HllConfig.java |  15 +-
 .../pinot/common/config/IndexingConfigTest.java    | 134 ++++--------
 .../pinot/common/config/QuotaConfigTest.java       |  23 +-
 .../pinot/common/config/TableConfigTest.java       |  30 +--
 .../pinot/common/config/TagOverrideConfigTest.java |   6 +-
 .../linkedin/pinot/common/config/TenantTest.java   |  40 ++--
 .../linkedin/pinot/common/data/FieldSpecTest.java  |  32 +--
 .../common/utils/FileUploadDownloadClientTest.java |  18 +-
 .../pinot/request/BrokerResponseNativeTest.java    |  10 +-
 pinot-controller/pom.xml                           |   8 +-
 .../pinot/controller/api/pojos/Instance.java       |  11 -
 .../resources/PinotInstanceRestletResource.java    |  21 +-
 .../api/resources/PinotSchemaRestletResource.java  |   7 +-
 .../api/resources/PinotSegmentRestletResource.java |  81 +++----
 .../PinotSegmentUploadRestletResource.java         |  28 +--
 .../api/resources/PinotTableIndexingConfigs.java   |   7 +-
 .../api/resources/PinotTableInstances.java         | 121 ++++-------
 .../api/resources/PinotTableMetadataConfigs.java   |   3 +-
 .../api/resources/PinotTableRestletResource.java   |  52 ++---
 .../api/resources/PinotTableSegmentConfigs.java    |   7 +-
 .../api/resources/PinotTenantRestletResource.java  |  51 ++---
 .../api/resources/PinotVersionRestletResource.java |  10 +-
 .../controller/api/resources/PqlQueryResource.java |  10 +-
 .../api/resources/ServerTableSizeReader.java       |   6 +-
 .../resources/WebApplicationExceptionMapper.java   |  11 +-
 .../helix/ControllerRequestBuilderUtil.java        |  19 +-
 .../core/realtime/PinotRealtimeSegmentManager.java |   7 +-
 .../controller/util/AutoAddInvertedIndex.java      |  22 +-
 .../PinotInstanceRestletResourceTest.java          | 104 ++++-----
 .../resources/PinotSchemaRestletResourceTest.java  |  40 +---
 .../resources/PinotSegmentRestletResourceTest.java |   7 +-
 .../resources/PinotTableRestletResourceTest.java   |  14 +-
 .../resources/PinotTenantRestletResourceTest.java  |  25 ++-
 .../SegmentCompletionProtocolDeserTest.java        | 178 ++++++++-------
 .../api/resources/ServerTableSizeReaderTest.java   |   4 +-
 .../api/resources/TableSizeReaderTest.java         |   4 +-
 .../controller/api/resources/TableViewsTest.java   |   5 +-
 .../controller/helix/ControllerSentinelTestV2.java |   3 +-
 .../controller/helix/ControllerTenantTest.java     |  75 ++++---
 .../pinot/controller/helix/ControllerTest.java     |   7 +-
 .../segment/FlushThresholdUpdaterTest.java         |   6 +-
 .../rebalance/DefaultRebalanceStrategyTest.java    |   8 +-
 .../helix/core/retention/RetentionManagerTest.java |   4 +-
 pinot-core/pom.xml                                 |   4 -
 .../com/linkedin/pinot/core/data/GenericRow.java   |  16 +-
 .../realtime/HLRealtimeSegmentDataManager.java     |   2 +-
 .../realtime/LLRealtimeSegmentDataManager.java     |   4 +-
 .../pinot/core/data/readers/JSONRecordReader.java  |  14 +-
 .../generator/SegmentGeneratorConfig.java          |  12 +-
 .../impl/kafka/KafkaJSONMessageDecoder.java        |  79 +------
 .../pinot/core/segment/index/ColumnMetadata.java   |   5 +-
 .../core/segment/index/SegmentMetadataImpl.java    | 102 ++++-----
 .../segment/index/loader/IndexLoadingConfig.java   |   2 +-
 .../pinot/core/util/trace/TraceContext.java        |  27 ++-
 .../ServerSegmentCompletionProtocolHandler.java    |   4 +-
 .../realtime/LLRealtimeSegmentDataManagerTest.java |   8 +-
 .../core/data/readers/JSONRecordReaderTest.java    |  10 +-
 .../segment/index/SegmentMetadataImplTest.java     |  73 +++----
 .../pinot/core/util/trace/TraceContextTest.java    |   8 +-
 .../pinot/index/persist/AvroDataPublisherTest.java |  13 +-
 .../java/com/linkedin/pinot/util/TestUtils.java    |  36 ---
 .../pinot/hadoop/io/JsonPinotOutputFormat.java     | 144 +++++-------
 .../pinot/hadoop/job/ControllerRestApi.java        |  10 +-
 .../mapper/HadoopSegmentCreationMapReduceJob.java  |  35 +--
 ...umSegmentAssignmentStrategyIntegrationTest.java |   5 +-
 .../tests/BaseClusterIntegrationTestSet.java       |  61 +++---
 .../tests/ClusterIntegrationTestUtils.java         |  16 +-
 .../pinot/integration/tests/ClusterTest.java       |  26 ++-
 .../DeleteAPIHybridClusterIntegrationTest.java     |  71 +++---
 .../tests/HybridClusterIntegrationTest.java        |  41 ++--
 ...ridClusterIntegrationTestCommandLineRunner.java |  19 +-
 .../tests/LLCRealtimeClusterIntegrationTest.java   |  15 +-
 ...onaryAggregationPlanClusterIntegrationTest.java | 122 ++++++-----
 .../tests/OfflineClusterIntegrationTest.java       | 242 ++++++++++-----------
 .../tests/PinotURIUploadIntegrationTest.java       |   6 +-
 .../pinot/integration/tests/QueryGenerator.java    |  16 +-
 .../tests/StarTreeClusterIntegrationTest.java      |  11 +-
 .../linkedin/pinot/perf/BenchmarkQueryEngine.java  |  15 +-
 .../perf/BenchmarkRealtimeConsumptionSpeed.java    |  10 +-
 .../linkedin/pinot/perf/RealtimeStressTest.java    |  10 +-
 pinot-server/pom.xml                               |  14 +-
 .../api/resources/DefaultExceptionMapper.java      |  27 +--
 .../pinot/server/api/resources/TablesResource.java |   4 +-
 .../server/api/resources/TablesResourceTest.java   |  39 ++--
 .../com/linkedin/pinot/tools/PinotZKChanger.java   |  11 +-
 .../java/com/linkedin/pinot/tools/Quickstart.java  |  57 +++--
 .../linkedin/pinot/tools/StarTreeIndexViewer.java  |   9 +-
 .../admin/command/AbstractBaseAdminCommand.java    |  37 +---
 .../pinot/tools/admin/command/AddTableCommand.java |  19 +-
 .../command/BackfillDateTimeColumnCommand.java     |   7 +-
 .../tools/admin/command/CreateSegmentCommand.java  |   5 +-
 .../tools/admin/command/GenerateDataCommand.java   |  41 ++--
 .../tools/admin/command/MoveReplicaGroup.java      |  18 +-
 .../tools/admin/command/PostQueryCommand.java      |  44 +---
 .../tools/admin/command/QuickstartRunner.java      |   7 +-
 .../command/RealtimeProvisioningHelperCommand.java |   3 +-
 .../pinot/tools/backfill/BackfillSegmentUtils.java |   9 +-
 .../pinot/tools/data/generator/AvroWriter.java     |  18 +-
 .../pinot/tools/data/generator/DataGenerator.java  |   5 +-
 .../pinot/tools/perf/PerfBenchmarkDriver.java      |  15 +-
 .../com/linkedin/pinot/tools/perf/QueryRunner.java |  14 +-
 .../tools/query/comparison/QueryComparison.java    | 175 +++++++--------
 .../query/comparison/StarQueryComparison.java      |   7 +-
 .../pinot/tools/scan/query/QueryResponse.java      |  18 +-
 .../tools/scan/query/ScanBasedQueryProcessor.java  |  10 +-
 .../converter/ColumnarToStarTreeConverter.java     |   6 +-
 .../converter/PinotSegmentToJsonConverter.java     |  17 +-
 .../segment/converter/SegmentMergeCommand.java     |  30 ++-
 .../pinot/tools/streams/AirlineDataStream.java     |  31 +--
 .../pinot/tools/streams/MeetupRsvpStream.java      |  49 ++---
 pinot-transport/pom.xml                            |   4 -
 .../transport/perf/ScatterGatherPerfClient.java    |   7 +-
 pom.xml                                            | 193 ++++++----------
 172 files changed, 2170 insertions(+), 2817 deletions(-)

diff --git a/pinot-api/pom.xml b/pinot-api/pom.xml
index 9dc3304..45b5a41 100644
--- a/pinot-api/pom.xml
+++ b/pinot-api/pom.xml
@@ -57,8 +57,8 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.json</groupId>
-      <artifactId>json</artifactId>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
     </dependency>
     <dependency>
       <groupId>com.ning</groupId>
diff --git a/pinot-api/src/main/java/com/linkedin/pinot/client/AggregationResultSet.java b/pinot-api/src/main/java/com/linkedin/pinot/client/AggregationResultSet.java
index 18934f6..4afb5ff 100644
--- a/pinot-api/src/main/java/com/linkedin/pinot/client/AggregationResultSet.java
+++ b/pinot-api/src/main/java/com/linkedin/pinot/client/AggregationResultSet.java
@@ -18,8 +18,8 @@
  */
 package com.linkedin.pinot.client;
 
-import org.json.JSONException;
-import org.json.JSONObject;
+import com.fasterxml.jackson.databind.JsonNode;
+
 
 /**
  * A Pinot query result set for aggregation results without group by clauses, of which there is one
@@ -27,9 +27,9 @@ import org.json.JSONObject;
  * function in the query.
  */
 class AggregationResultSet extends AbstractResultSet {
-  private final JSONObject _jsonObject;
+  private final JsonNode _jsonObject;
 
-  public AggregationResultSet(JSONObject jsonObject) {
+  public AggregationResultSet(JsonNode jsonObject) {
     _jsonObject = jsonObject;
   }
 
@@ -45,29 +45,20 @@ class AggregationResultSet extends AbstractResultSet {
 
   @Override
   public String getColumnName(int columnIndex) {
-    try {
-      return _jsonObject.getString("function");
-    } catch (JSONException e) {
-      throw new PinotClientException(e);
-    }
+    return _jsonObject.get("function").asText();
   }
 
   @Override
   public String getString(int rowIndex, int columnIndex) {
     if (columnIndex != 0) {
-      throw new IllegalArgumentException(
-          "Column index must always be 0 for aggregation result sets");
+      throw new IllegalArgumentException("Column index must always be 0 for aggregation result sets");
     }
 
     if (rowIndex != 0) {
       throw new IllegalArgumentException("Row index must always be 0 for aggregation result sets");
     }
 
-    try {
-      return _jsonObject.get("value").toString();
-    } catch (Exception e) {
-      throw new PinotClientException(e);
-    }
+    return _jsonObject.get("value").asText();
   }
 
   @Override
diff --git a/pinot-api/src/main/java/com/linkedin/pinot/client/BrokerResponse.java b/pinot-api/src/main/java/com/linkedin/pinot/client/BrokerResponse.java
index cd81e7b..576da47 100644
--- a/pinot-api/src/main/java/com/linkedin/pinot/client/BrokerResponse.java
+++ b/pinot-api/src/main/java/com/linkedin/pinot/client/BrokerResponse.java
@@ -18,52 +18,39 @@
  */
 package com.linkedin.pinot.client;
 
-import java.util.List;
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
+import com.fasterxml.jackson.databind.JsonNode;
 
 
 /**
  * Reimplementation of BrokerResponse from pinot-common, so that pinot-api does not depend on pinot-common.
  */
 class BrokerResponse {
-  private JSONArray _aggregationResults;
-  private JSONObject _selectionResults;
-  private JSONArray _exceptions;
+  private JsonNode _aggregationResults;
+  private JsonNode _selectionResults;
+  private JsonNode _exceptions;
 
   private BrokerResponse() {
   }
 
-  private BrokerResponse(JSONObject brokerResponse) {
-    try {
-      if (brokerResponse.has("aggregationResults")) {
-        _aggregationResults = brokerResponse.getJSONArray("aggregationResults");
-      }
-      if (brokerResponse.has("exceptions")) {
-        _exceptions = brokerResponse.getJSONArray("exceptions");
-      }
-      if (brokerResponse.has("selectionResults")) {
-        _selectionResults = brokerResponse.getJSONObject("selectionResults");
-      }
-    } catch (JSONException e) {
-      throw new PinotClientException(e);
-    }
+  private BrokerResponse(JsonNode brokerResponse) {
+    _aggregationResults = brokerResponse.get("aggregationResults");
+    _exceptions = brokerResponse.get("exceptions");
+    _selectionResults = brokerResponse.get("selectionResults");
   }
 
   boolean hasExceptions() {
-    return _exceptions != null && _exceptions.length() != 0;
+    return _exceptions != null && _exceptions.size() != 0;
   }
 
-  JSONArray getExceptions() {
+  JsonNode getExceptions() {
     return _exceptions;
   }
 
-  JSONArray getAggregationResults() {
+  JsonNode getAggregationResults() {
     return _aggregationResults;
   }
 
-  JSONObject getSelectionResults() {
+  JsonNode getSelectionResults() {
     return _selectionResults;
   }
 
@@ -71,11 +58,11 @@ class BrokerResponse {
     if (_aggregationResults == null) {
       return 0;
     } else {
-      return _aggregationResults.length();
+      return _aggregationResults.size();
     }
   }
 
-  static BrokerResponse fromJson(JSONObject json) {
+  static BrokerResponse fromJson(JsonNode json) {
     return new BrokerResponse(json);
   }
 
diff --git a/pinot-api/src/main/java/com/linkedin/pinot/client/ExternalViewReader.java b/pinot-api/src/main/java/com/linkedin/pinot/client/ExternalViewReader.java
index 157d061..486a990 100644
--- a/pinot-api/src/main/java/com/linkedin/pinot/client/ExternalViewReader.java
+++ b/pinot-api/src/main/java/com/linkedin/pinot/client/ExternalViewReader.java
@@ -18,6 +18,9 @@
  */
 package com.linkedin.pinot.client;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
@@ -29,18 +32,18 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-
 import java.util.zip.GZIPInputStream;
 import org.I0Itec.zkclient.ZkClient;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+
 /**
  * Reads brokers external view from Zookeeper
  */
 public class ExternalViewReader {
   private static final Logger LOGGER = LoggerFactory.getLogger(ExternalViewReader.class);
+  private static final ObjectReader OBJECT_READER = new ObjectMapper().reader();
 
   private ZkClient zkClient;
 
@@ -53,23 +56,21 @@ public class ExternalViewReader {
   }
 
   public List<String> getLiveBrokers() {
-    List<String> brokerUrls = new ArrayList<String>();
+    List<String> brokerUrls = new ArrayList<>();
     try {
-
       byte[] brokerResourceNodeData = zkClient.readData(BROKER_EXTERNAL_VIEW_PATH, true);
       brokerResourceNodeData = unpackZnodeIfNecessary(brokerResourceNodeData);
-      JSONObject jsonObject = new JSONObject(new String(brokerResourceNodeData));
-      JSONObject brokerResourceNode = jsonObject.getJSONObject("mapFields");
+      JsonNode jsonObject = OBJECT_READER.readTree(new ByteArrayInputStream(brokerResourceNodeData));
+      JsonNode brokerResourceNode = jsonObject.get("mapFields");
 
-      Iterator<String> resourceNames = brokerResourceNode.keys();
-      while (resourceNames.hasNext()) {
-        String resourceName = resourceNames.next();
-        JSONObject resource = brokerResourceNode.getJSONObject(resourceName);
-
-        Iterator<String> brokerNames = resource.keys();
-        while (brokerNames.hasNext()) {
-          String brokerName = brokerNames.next();
-          if (brokerName.startsWith("Broker_") && "ONLINE".equals(resource.getString(brokerName))) {
+      Iterator<Entry<String, JsonNode>> resourceEntries = brokerResourceNode.fields();
+      while (resourceEntries.hasNext()) {
+        JsonNode resource = resourceEntries.next().getValue();
+        Iterator<Entry<String, JsonNode>> brokerEntries = resource.fields();
+        while (brokerEntries.hasNext()) {
+          Entry<String, JsonNode> brokerEntry = brokerEntries.next();
+          String brokerName = brokerEntry.getKey();
+          if (brokerName.startsWith("Broker_") && "ONLINE".equals(brokerEntry.getValue().asText())) {
             // Turn Broker_12.34.56.78_1234 into 12.34.56.78:1234
             String brokerHostPort = brokerName.replace("Broker_", "").replace("_", ":");
             brokerUrls.add(brokerHostPort);
@@ -83,29 +84,26 @@ public class ExternalViewReader {
     return brokerUrls;
   }
 
-  @SuppressWarnings("unchecked")
   public Map<String, List<String>> getTableToBrokersMap() {
     Map<String, Set<String>> brokerUrlsMap = new HashMap<>();
     try {
       byte[] brokerResourceNodeData = zkClient.readData("/EXTERNALVIEW/brokerResource", true);
       brokerResourceNodeData = unpackZnodeIfNecessary(brokerResourceNodeData);
-      JSONObject jsonObject = new JSONObject(new String(brokerResourceNodeData));
-      JSONObject brokerResourceNode = jsonObject.getJSONObject("mapFields");
+      JsonNode jsonObject = OBJECT_READER.readTree(new ByteArrayInputStream(brokerResourceNodeData));
+      JsonNode brokerResourceNode = jsonObject.get("mapFields");
 
-      Iterator<String> resourceNames = brokerResourceNode.keys();
-      while (resourceNames.hasNext()) {
-        String resourceName = resourceNames.next();
+      Iterator<Entry<String, JsonNode>> resourceEntries = brokerResourceNode.fields();
+      while (resourceEntries.hasNext()) {
+        Entry<String, JsonNode> resourceEntry = resourceEntries.next();
+        String resourceName = resourceEntry.getKey();
         String tableName = resourceName.replace(OFFLINE_SUFFIX, "").replace(REALTIME_SUFFIX, "");
-        Set<String> brokerUrls = brokerUrlsMap.get(tableName);
-        if (brokerUrls == null) {
-          brokerUrls = new HashSet<>();
-          brokerUrlsMap.put(tableName, brokerUrls);
-        }
-        JSONObject resource = brokerResourceNode.getJSONObject(resourceName);
-        Iterator<String> brokerNames = resource.keys();
-        while (brokerNames.hasNext()) {
-          String brokerName = brokerNames.next();
-          if (brokerName.startsWith("Broker_") && "ONLINE".equals(resource.getString(brokerName))) {
+        Set<String> brokerUrls = brokerUrlsMap.computeIfAbsent(tableName, k -> new HashSet<>());
+        JsonNode resource = resourceEntry.getValue();
+        Iterator<Entry<String, JsonNode>> brokerEntries = resource.fields();
+        while (brokerEntries.hasNext()) {
+          Entry<String, JsonNode> brokerEntry = brokerEntries.next();
+          String brokerName = brokerEntry.getKey();
+          if (brokerName.startsWith("Broker_") && "ONLINE".equals(brokerEntry.getValue().asText())) {
             // Turn Broker_12.34.56.78_1234 into 12.34.56.78:1234
             String brokerHostPort = brokerName.replace("Broker_", "").replace("_", ":");
             brokerUrls.add(brokerHostPort);
diff --git a/pinot-api/src/main/java/com/linkedin/pinot/client/GroupByResultSet.java b/pinot-api/src/main/java/com/linkedin/pinot/client/GroupByResultSet.java
index a510aad..e9d9423 100644
--- a/pinot-api/src/main/java/com/linkedin/pinot/client/GroupByResultSet.java
+++ b/pinot-api/src/main/java/com/linkedin/pinot/client/GroupByResultSet.java
@@ -18,27 +18,22 @@
  */
 package com.linkedin.pinot.client;
 
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
+import com.fasterxml.jackson.databind.JsonNode;
+
 
 /**
  * A Pinot query result set for group by results, of which there is one of per aggregation function
  * in the query.
  */
 class GroupByResultSet extends AbstractResultSet {
-  private final JSONArray _groupByResults;
-  private final JSONArray _groupByColumns;
+  private final JsonNode _groupByResults;
+  private final JsonNode _groupByColumns;
   private final String _functionName;
 
-  public GroupByResultSet(JSONObject jsonObject) {
-    try {
-      _groupByResults = jsonObject.getJSONArray("groupByResult");
-      _groupByColumns = jsonObject.getJSONArray("groupByColumns");
-      _functionName = jsonObject.getString("function");
-    } catch (JSONException e) {
-      throw new PinotClientException(e);
-    }
+  public GroupByResultSet(JsonNode jsonObject) {
+    _groupByResults = jsonObject.get("groupByResult");
+    _groupByColumns = jsonObject.get("groupByColumns");
+    _functionName = jsonObject.get("function").asText();
   }
 
   /**
@@ -47,7 +42,7 @@ class GroupByResultSet extends AbstractResultSet {
    */
   @Override
   public int getRowCount() {
-    return _groupByResults.length();
+    return _groupByResults.size();
   }
 
   @Override
@@ -63,38 +58,24 @@ class GroupByResultSet extends AbstractResultSet {
   @Override
   public String getString(int rowIndex, int columnIndex) {
     if (columnIndex != 0) {
-      throw new IllegalArgumentException(
-          "Column index must always be 0 for aggregation result sets");
-    }
-
-    try {
-      return _groupByResults.getJSONObject(rowIndex).get("value").toString();
-    } catch (Exception e) {
-      throw new PinotClientException(e);
+      throw new IllegalArgumentException("Column index must always be 0 for aggregation result sets");
     }
+    return _groupByResults.get(rowIndex).get("value").asText();
   }
 
   @Override
   public int getGroupKeyLength() {
-    return _groupByColumns.length();
+    return _groupByColumns.size();
   }
 
   @Override
   public String getGroupKeyString(int rowIndex, int groupKeyColumnIndex) {
-    try {
-      return _groupByResults.getJSONObject(rowIndex).getJSONArray("group").getString(groupKeyColumnIndex);
-    } catch (Exception e) {
-      throw new PinotClientException(e);
-    }
+    return _groupByResults.get(rowIndex).get("group").get(groupKeyColumnIndex).asText();
   }
 
   @Override
   public String getGroupKeyColumnName(int groupKeyColumnIndex) {
-    try {
-      return _groupByColumns.getString(groupKeyColumnIndex);
-    } catch (Exception e) {
-      throw new PinotClientException(e);
-    }
+    return _groupByColumns.get(groupKeyColumnIndex).asText();
   }
 
   @Override
diff --git a/pinot-api/src/main/java/com/linkedin/pinot/client/JsonAsyncHttpPinotClientTransport.java b/pinot-api/src/main/java/com/linkedin/pinot/client/JsonAsyncHttpPinotClientTransport.java
index f91c8cd..c20716e 100644
--- a/pinot-api/src/main/java/com/linkedin/pinot/client/JsonAsyncHttpPinotClientTransport.java
+++ b/pinot-api/src/main/java/com/linkedin/pinot/client/JsonAsyncHttpPinotClientTransport.java
@@ -18,13 +18,15 @@
  */
 package com.linkedin.pinot.client;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.node.JsonNodeFactory;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.ning.http.client.AsyncHttpClient;
 import com.ning.http.client.Response;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -34,6 +36,8 @@ import org.slf4j.LoggerFactory;
  */
 class JsonAsyncHttpPinotClientTransport implements PinotClientTransport {
   private static final Logger LOGGER = LoggerFactory.getLogger(JsonAsyncHttpPinotClientTransport.class);
+  private static final ObjectReader OBJECT_READER = new ObjectMapper().reader();
+
   AsyncHttpClient _httpClient = new AsyncHttpClient();
 
   @Override
@@ -48,7 +52,7 @@ class JsonAsyncHttpPinotClientTransport implements PinotClientTransport {
   @Override
   public Future<BrokerResponse> executeQueryAsync(String brokerAddress, final String query) {
     try {
-      final JSONObject json = new JSONObject();
+      ObjectNode json = JsonNodeFactory.instance.objectNode();
       json.put("pql", query);
 
       final String url = "http://" + brokerAddress + "/query";
@@ -88,19 +92,12 @@ class JsonAsyncHttpPinotClientTransport implements PinotClientTransport {
     }
 
     @Override
-    public BrokerResponse get()
-        throws InterruptedException, ExecutionException {
-      try {
-        return get(1000L, TimeUnit.DAYS);
-      } catch (TimeoutException e) {
-        LOGGER.error("Caught timeout during synchronous get", e);
-        throw new InterruptedException();
-      }
+    public BrokerResponse get() throws ExecutionException {
+      return get(1000L, TimeUnit.DAYS);
     }
 
     @Override
-    public BrokerResponse get(long timeout, TimeUnit unit)
-        throws InterruptedException, ExecutionException, TimeoutException {
+    public BrokerResponse get(long timeout, TimeUnit unit) throws ExecutionException {
       try {
         LOGGER.debug("Sending query {} to {}", _query, _url);
 
@@ -109,12 +106,12 @@ class JsonAsyncHttpPinotClientTransport implements PinotClientTransport {
         LOGGER.debug("Completed query, HTTP status is {}", httpResponse.getStatusCode());
 
         if (httpResponse.getStatusCode() != 200) {
-          throw new PinotClientException("Pinot returned HTTP status " + httpResponse.getStatusCode() +
-              ", expected 200");
+          throw new PinotClientException(
+              "Pinot returned HTTP status " + httpResponse.getStatusCode() + ", expected 200");
         }
 
         String responseBody = httpResponse.getResponseBody();
-        return BrokerResponse.fromJson(new JSONObject(responseBody));
+        return BrokerResponse.fromJson(OBJECT_READER.readTree(responseBody));
       } catch (Exception e) {
         throw new ExecutionException(e);
       }
diff --git a/pinot-api/src/main/java/com/linkedin/pinot/client/ResultSetGroup.java b/pinot-api/src/main/java/com/linkedin/pinot/client/ResultSetGroup.java
index 248ef19..73c9cb4 100644
--- a/pinot-api/src/main/java/com/linkedin/pinot/client/ResultSetGroup.java
+++ b/pinot-api/src/main/java/com/linkedin/pinot/client/ResultSetGroup.java
@@ -18,10 +18,9 @@
  */
 package com.linkedin.pinot.client;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import java.util.ArrayList;
 import java.util.List;
-import org.json.JSONException;
-import org.json.JSONObject;
 
 
 /**
@@ -31,7 +30,7 @@ public class ResultSetGroup {
   private final List<ResultSet> _resultSets;
 
   ResultSetGroup(BrokerResponse brokerResponse) {
-    _resultSets = new ArrayList<ResultSet>();
+    _resultSets = new ArrayList<>();
 
     if (brokerResponse.getSelectionResults() != null) {
       _resultSets.add(new SelectionResultSet(brokerResponse.getSelectionResults()));
@@ -39,12 +38,7 @@ public class ResultSetGroup {
 
     int aggregationResultCount = brokerResponse.getAggregationResultsSize();
     for (int i = 0; i < aggregationResultCount; i++) {
-      JSONObject aggregationResult;
-      try {
-        aggregationResult = brokerResponse.getAggregationResults().getJSONObject(i);
-      } catch (JSONException e) {
-        throw new PinotClientException(e);
-      }
+      JsonNode aggregationResult = brokerResponse.getAggregationResults().get(i);
       if (aggregationResult.has("value")) {
         _resultSets.add(new AggregationResultSet(aggregationResult));
       } else if (aggregationResult.has("groupByResult")) {
@@ -74,11 +68,11 @@ public class ResultSetGroup {
   public ResultSet getResultSet(int index) {
     return _resultSets.get(index);
   }
-  
+
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();
-    for(ResultSet resultSet:_resultSets){
+    for (ResultSet resultSet : _resultSets) {
       sb.append(resultSet);
       sb.append("\n");
     }
diff --git a/pinot-api/src/main/java/com/linkedin/pinot/client/SelectionResultSet.java b/pinot-api/src/main/java/com/linkedin/pinot/client/SelectionResultSet.java
index d26f41c..0b81f9a 100644
--- a/pinot-api/src/main/java/com/linkedin/pinot/client/SelectionResultSet.java
+++ b/pinot-api/src/main/java/com/linkedin/pinot/client/SelectionResultSet.java
@@ -18,51 +18,43 @@
  */
 package com.linkedin.pinot.client;
 
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
+import com.fasterxml.jackson.databind.JsonNode;
+
 
 /**
  * Selection result set, which contains the results of a selection query.
  */
 class SelectionResultSet extends AbstractResultSet {
-  private JSONArray _resultsArray;
-  private JSONArray _columnsArray;
+  private JsonNode _resultsArray;
+  private JsonNode _columnsArray;
 
-  public SelectionResultSet(JSONObject selectionResults) {
-    try {
-      _resultsArray = selectionResults.getJSONArray("results");
-      _columnsArray = selectionResults.getJSONArray("columns");
-    } catch (JSONException e) {
-      throw new PinotClientException(e);
-    }
+  public SelectionResultSet(JsonNode selectionResults) {
+    _resultsArray = selectionResults.get("results");
+    _columnsArray = selectionResults.get("columns");
   }
 
   @Override
   public int getRowCount() {
-    return _resultsArray.length();
+    return _resultsArray.size();
   }
 
   @Override
   public int getColumnCount() {
-    return _columnsArray.length();
+    return _columnsArray.size();
   }
 
   @Override
   public String getColumnName(int columnIndex) {
-    try {
-      return _columnsArray.getString(columnIndex);
-    } catch (JSONException e) {
-      throw new PinotClientException(e);
-    }
+    return _columnsArray.get(columnIndex).asText();
   }
 
   @Override
   public String getString(int rowIndex, int columnIndex) {
-    try {
-      return _resultsArray.getJSONArray(rowIndex).get(columnIndex).toString();
-    } catch (JSONException e) {
-      throw new PinotClientException(e);
+    JsonNode jsonValue = _resultsArray.get(rowIndex).get(columnIndex);
+    if (jsonValue.isTextual()) {
+      return jsonValue.textValue();
+    } else {
+      return jsonValue.toString();
     }
   }
 
@@ -87,11 +79,7 @@ class SelectionResultSet extends AbstractResultSet {
     TextTable table = new TextTable();
     String[] columnNames = new String[numColumns];
     for (int c = 0; c < numColumns; c++) {
-      try {
-        columnNames[c] = _columnsArray.getString(c);
-      } catch (JSONException e) {
-        columnNames[c] = "ERROR";
-      }
+      columnNames[c] = _columnsArray.get(c).asText();
     }
     table.addHeader(columnNames);
 
@@ -108,6 +96,5 @@ class SelectionResultSet extends AbstractResultSet {
       table.addRow(columnValues);
     }
     return table.toString();
-
   }
 }
diff --git a/pinot-api/src/test/java/com/linkedin/pinot/client/ResultSetGroupTest.java b/pinot-api/src/test/java/com/linkedin/pinot/client/ResultSetGroupTest.java
index 892cd7a..3d1cc3f 100644
--- a/pinot-api/src/test/java/com/linkedin/pinot/client/ResultSetGroupTest.java
+++ b/pinot-api/src/test/java/com/linkedin/pinot/client/ResultSetGroupTest.java
@@ -18,9 +18,9 @@
  */
 package com.linkedin.pinot.client;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
 import java.io.InputStream;
 import java.util.concurrent.Future;
-import org.json.JSONObject;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
@@ -133,7 +133,7 @@ public class ResultSetGroupTest {
           lastByte = stream.read();
         }
         String jsonText = builder.toString();
-        return BrokerResponse.fromJson(new JSONObject(jsonText));
+        return BrokerResponse.fromJson(new ObjectMapper().readTree(jsonText));
       } catch (Exception e) {
         Assert.fail("Unexpected exception", e);
         return null;
diff --git a/pinot-broker/pom.xml b/pinot-broker/pom.xml
index d975197..52905eb 100644
--- a/pinot-broker/pom.xml
+++ b/pinot-broker/pom.xml
@@ -113,18 +113,6 @@
       <artifactId>swagger-jersey2-jaxrs</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.codehaus.jackson</groupId>
-      <artifactId>jackson-mapper-asl</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.json</groupId>
-      <artifactId>json</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.alibaba</groupId>
-      <artifactId>fastjson</artifactId>
-    </dependency>
-    <dependency>
       <groupId>org.apache.helix</groupId>
       <artifactId>helix-core</artifactId>
       <exclusions>
diff --git a/pinot-broker/src/main/java/com/linkedin/pinot/broker/api/resources/PinotClientRequest.java b/pinot-broker/src/main/java/com/linkedin/pinot/broker/api/resources/PinotClientRequest.java
index 354d420..8573fbe 100644
--- a/pinot-broker/src/main/java/com/linkedin/pinot/broker/api/resources/PinotClientRequest.java
+++ b/pinot-broker/src/main/java/com/linkedin/pinot/broker/api/resources/PinotClientRequest.java
@@ -18,11 +18,14 @@
  */
 package com.linkedin.pinot.broker.api.resources;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.linkedin.pinot.broker.api.RequestStatistics;
 import com.linkedin.pinot.broker.requesthandler.BrokerRequestHandler;
 import com.linkedin.pinot.common.metrics.BrokerMeter;
 import com.linkedin.pinot.common.metrics.BrokerMetrics;
 import com.linkedin.pinot.common.response.BrokerResponse;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiParam;
@@ -37,7 +40,6 @@ import javax.ws.rs.QueryParam;
 import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -67,10 +69,9 @@ public class PinotClientRequest {
       // Query param "bql" is for backward compatibility
       @ApiParam(value = "Query", required = true) @QueryParam("bql") String query,
       @ApiParam(value = "Trace enabled") @QueryParam(TRACE) String traceEnabled,
-      @ApiParam(value = "Debug options") @QueryParam(DEBUG_OPTIONS) String debugOptions
-  ) {
+      @ApiParam(value = "Debug options") @QueryParam(DEBUG_OPTIONS) String debugOptions) {
     try {
-      JSONObject requestJson = new JSONObject();
+      ObjectNode requestJson = JsonUtils.newObjectNode();
       requestJson.put(PQL, query);
       if (traceEnabled != null) {
         requestJson.put(TRACE, traceEnabled);
@@ -97,7 +98,7 @@ public class PinotClientRequest {
   })
   public String processQueryPost(String query) {
     try {
-      JSONObject requestJson = new JSONObject(query);
+      JsonNode requestJson = JsonUtils.stringToJsonNode(query);
       BrokerResponse brokerResponse = requestHandler.handleRequest(requestJson, null, new RequestStatistics());
       return brokerResponse.toJsonString();
     } catch (Exception e) {
diff --git a/pinot-broker/src/main/java/com/linkedin/pinot/broker/requesthandler/BaseBrokerRequestHandler.java b/pinot-broker/src/main/java/com/linkedin/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
index 2b9304d..989cfbb 100644
--- a/pinot-broker/src/main/java/com/linkedin/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
+++ b/pinot-broker/src/main/java/com/linkedin/pinot/broker/requesthandler/BaseBrokerRequestHandler.java
@@ -18,6 +18,7 @@
  */
 package com.linkedin.pinot.broker.requesthandler;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.google.common.base.Splitter;
 import com.linkedin.pinot.broker.api.RequestStatistics;
 import com.linkedin.pinot.broker.api.RequesterIdentity;
@@ -51,7 +52,6 @@ import javax.annotation.Nullable;
 import javax.annotation.concurrent.ThreadSafe;
 import org.apache.commons.configuration.Configuration;
 import org.apache.commons.lang3.StringUtils;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -109,13 +109,7 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
   }
 
   @Override
-  public BrokerResponse handleRequest(JSONObject request, @Nullable RequesterIdentity requesterIdentity)
-      throws Exception {
-    return handleRequest(request, requesterIdentity, new RequestStatistics());
-  }
-
-  @Override
-  public BrokerResponse handleRequest(JSONObject request, @Nullable RequesterIdentity requesterIdentity,
+  public BrokerResponse handleRequest(JsonNode request, @Nullable RequesterIdentity requesterIdentity,
       RequestStatistics requestStatistics)
       throws Exception {
     long requestId = _requestIdGenerator.incrementAndGet();
@@ -123,7 +117,7 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
     requestStatistics.setRequestId(requestId);
     requestStatistics.setRequestArrivalTimeMillis(System.currentTimeMillis());
 
-    String query = request.getString(PQL);
+    String query = request.get(PQL).asText();
     LOGGER.debug("Query string for request {}: {}", requestId, query);
     requestStatistics.setPql(query);
 
@@ -211,7 +205,7 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
     }
 
     // Set extra settings into broker request
-    if (request.has(TRACE) && request.getBoolean(TRACE)) {
+    if (request.has(TRACE) && request.get(TRACE).asBoolean()) {
       LOGGER.debug("Enable trace for request {}: {}", requestId, query);
       brokerRequest.setEnableTrace(true);
     }
@@ -220,7 +214,7 @@ public abstract class BaseBrokerRequestHandler implements BrokerRequestHandler {
           .omitEmptyStrings()
           .trimResults()
           .withKeyValueSeparator('=')
-          .split(request.getString(DEBUG_OPTIONS));
+          .split(request.get(DEBUG_OPTIONS).asText());
       LOGGER.debug("Debug options are set to: {} for request {}: {}", debugOptions, requestId, query);
       brokerRequest.setDebugOptions(debugOptions);
     }
diff --git a/pinot-broker/src/main/java/com/linkedin/pinot/broker/requesthandler/BrokerRequestHandler.java b/pinot-broker/src/main/java/com/linkedin/pinot/broker/requesthandler/BrokerRequestHandler.java
index dd3337f..e01d6a1 100644
--- a/pinot-broker/src/main/java/com/linkedin/pinot/broker/requesthandler/BrokerRequestHandler.java
+++ b/pinot-broker/src/main/java/com/linkedin/pinot/broker/requesthandler/BrokerRequestHandler.java
@@ -18,12 +18,12 @@
  */
 package com.linkedin.pinot.broker.requesthandler;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.linkedin.pinot.broker.api.RequestStatistics;
 import com.linkedin.pinot.broker.api.RequesterIdentity;
 import com.linkedin.pinot.common.response.BrokerResponse;
 import javax.annotation.Nullable;
 import javax.annotation.concurrent.ThreadSafe;
-import org.json.JSONObject;
 
 
 @ThreadSafe
@@ -33,10 +33,6 @@ public interface BrokerRequestHandler {
 
   void shutDown();
 
-  @Deprecated
-  BrokerResponse handleRequest(JSONObject request, @Nullable RequesterIdentity requesterIdentity)
-      throws Exception;
-
-  BrokerResponse handleRequest(JSONObject request, @Nullable RequesterIdentity requesterIdentity,
+  BrokerResponse handleRequest(JsonNode request, @Nullable RequesterIdentity requesterIdentity,
       RequestStatistics requestStatistics) throws Exception;
 }
diff --git a/pinot-broker/src/main/java/com/linkedin/pinot/broker/routing/HelixExternalViewBasedRouting.java b/pinot-broker/src/main/java/com/linkedin/pinot/broker/routing/HelixExternalViewBasedRouting.java
index 7aa07d8..91ec52b 100644
--- a/pinot-broker/src/main/java/com/linkedin/pinot/broker/routing/HelixExternalViewBasedRouting.java
+++ b/pinot-broker/src/main/java/com/linkedin/pinot/broker/routing/HelixExternalViewBasedRouting.java
@@ -18,6 +18,8 @@
  */
 package com.linkedin.pinot.broker.routing;
 
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.google.common.collect.Sets;
 import com.linkedin.pinot.broker.routing.builder.RoutingTableBuilder;
 import com.linkedin.pinot.broker.routing.selector.SegmentSelector;
@@ -29,6 +31,7 @@ import com.linkedin.pinot.common.metrics.BrokerMetrics;
 import com.linkedin.pinot.common.metrics.BrokerTimer;
 import com.linkedin.pinot.common.utils.CommonConstants;
 import com.linkedin.pinot.common.utils.EqualityUtils;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.NetUtil;
 import com.linkedin.pinot.common.utils.helix.HelixHelper;
 import java.util.ArrayList;
@@ -50,8 +53,6 @@ import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
 import org.apache.zookeeper.data.Stat;
-import org.json.JSONArray;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -574,27 +575,27 @@ public class HelixExternalViewBasedRouting implements RoutingTable {
 
   @Override
   public String dumpSnapshot(String tableName) throws Exception {
-    JSONObject ret = new JSONObject();
-    JSONArray routingTableSnapshot = new JSONArray();
+    ObjectNode ret = JsonUtils.newObjectNode();
+    ArrayNode routingTableSnapshot = JsonUtils.newArrayNode();
 
     for (String currentTable : _routingTableBuilderMap.keySet()) {
       if (tableName == null || currentTable.startsWith(tableName)) {
-        JSONObject tableEntry = new JSONObject();
+        ObjectNode tableEntry = JsonUtils.newObjectNode();
         tableEntry.put("tableName", currentTable);
 
-        JSONArray entries = new JSONArray();
+        ArrayNode entries = JsonUtils.newArrayNode();
         RoutingTableBuilder routingTableBuilder = _routingTableBuilderMap.get(currentTable);
         List<Map<String, List<String>>> routingTables = routingTableBuilder.getRoutingTables();
         for (Map<String, List<String>> routingTable : routingTables) {
-          entries.put(new JSONObject(routingTable));
+          entries.add(JsonUtils.objectToJsonNode(routingTable));
         }
-        tableEntry.put("routingTableEntries", entries);
-        routingTableSnapshot.put(tableEntry);
+        tableEntry.set("routingTableEntries", entries);
+        routingTableSnapshot.add(tableEntry);
       }
     }
-    ret.put("routingTableSnapshot", routingTableSnapshot);
+    ret.set("routingTableSnapshot", routingTableSnapshot);
     ret.put("host", NetUtil.getHostnameOrAddress());
 
-    return ret.toString(2);
+    return JsonUtils.objectToPrettyString(ret);
   }
 }
diff --git a/pinot-broker/src/main/java/com/linkedin/pinot/broker/routing/TimeBoundaryService.java b/pinot-broker/src/main/java/com/linkedin/pinot/broker/routing/TimeBoundaryService.java
index d0aec08..aa9dfe5 100644
--- a/pinot-broker/src/main/java/com/linkedin/pinot/broker/routing/TimeBoundaryService.java
+++ b/pinot-broker/src/main/java/com/linkedin/pinot/broker/routing/TimeBoundaryService.java
@@ -18,7 +18,8 @@
  */
 package com.linkedin.pinot.broker.routing;
 
-import org.json.JSONObject;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.linkedin.pinot.common.utils.JsonUtils;
 
 
 public interface TimeBoundaryService {
@@ -37,7 +38,7 @@ public interface TimeBoundaryService {
    */
   void remove(String tableName);
 
-  public class TimeBoundaryInfo {
+  class TimeBoundaryInfo {
     private String _timeColumn;
     private String _timeValue;
 
@@ -57,11 +58,8 @@ public interface TimeBoundaryService {
       _timeValue = timeValue;
     }
 
-    public String toJsonString() throws Exception {
-      JSONObject obj = new JSONObject();
-      obj.put("timeColumnName", _timeColumn);
-      obj.put("timeColumnValue",_timeValue);
-      return obj.toString(2);
+    public String toJsonString() throws JsonProcessingException {
+      return JsonUtils.objectToPrettyString(this);
     }
   }
 }
diff --git a/pinot-broker/src/test/java/com/linkedin/pinot/broker/queryquota/TableQueryQuotaManagerTest.java b/pinot-broker/src/test/java/com/linkedin/pinot/broker/queryquota/TableQueryQuotaManagerTest.java
index c154a61..42102e9 100644
--- a/pinot-broker/src/test/java/com/linkedin/pinot/broker/queryquota/TableQueryQuotaManagerTest.java
+++ b/pinot-broker/src/test/java/com/linkedin/pinot/broker/queryquota/TableQueryQuotaManagerTest.java
@@ -24,7 +24,6 @@ import com.linkedin.pinot.common.config.TableNameBuilder;
 import com.linkedin.pinot.common.metadata.ZKMetadataProvider;
 import com.linkedin.pinot.common.utils.StringUtil;
 import com.linkedin.pinot.common.utils.ZkStarter;
-import java.io.IOException;
 import org.apache.commons.lang.StringUtils;
 import org.apache.helix.HelixManager;
 import org.apache.helix.InstanceType;
@@ -35,7 +34,6 @@ import org.apache.helix.manager.zk.ZkBaseDataAccessor;
 import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.json.JSONException;
 import org.testng.Assert;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.AfterTest;
@@ -358,7 +356,7 @@ public class TableQueryQuotaManagerTest {
     Assert.assertEquals(_tableQueryQuotaManager.getRateLimiterMapSize(), 1);
   }
 
-  private TableConfig generateDefaultTableConfig(String tableName) throws IOException, JSONException {
+  private TableConfig generateDefaultTableConfig(String tableName) {
     TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableName);
     TableConfig.Builder builder = new TableConfig.Builder(tableType);
     builder.setTableName(tableName);
diff --git a/pinot-broker/src/test/java/com/linkedin/pinot/broker/routing/RoutingTableTest.java b/pinot-broker/src/test/java/com/linkedin/pinot/broker/routing/RoutingTableTest.java
index cf2c232..a1410ad 100644
--- a/pinot-broker/src/test/java/com/linkedin/pinot/broker/routing/RoutingTableTest.java
+++ b/pinot-broker/src/test/java/com/linkedin/pinot/broker/routing/RoutingTableTest.java
@@ -31,7 +31,6 @@ import com.linkedin.pinot.common.utils.CommonConstants.Helix.TableType;
 import com.linkedin.pinot.common.utils.HLCSegmentName;
 import com.linkedin.pinot.common.utils.LLCSegmentName;
 import com.yammer.metrics.core.MetricsRegistry;
-import java.io.IOException;
 import java.lang.reflect.Field;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -42,7 +41,6 @@ import org.apache.commons.configuration.BaseConfiguration;
 import org.apache.commons.lang.mutable.MutableBoolean;
 import org.apache.helix.model.ExternalView;
 import org.apache.helix.model.InstanceConfig;
-import org.json.JSONException;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -288,7 +286,7 @@ public class RoutingTableTest {
     return configs;
   }
 
-  private TableConfig generateTableConfig(String tableName) throws IOException, JSONException {
+  private TableConfig generateTableConfig(String tableName) {
     TableType tableType = TableNameBuilder.getTableTypeFromTableName(tableName);
     Builder builder = new TableConfig.Builder(tableType);
     builder.setTableName(tableName);
diff --git a/pinot-common/pom.xml b/pinot-common/pom.xml
index 46b8ded..21d64e5 100644
--- a/pinot-common/pom.xml
+++ b/pinot-common/pom.xml
@@ -99,10 +99,6 @@
       <artifactId>antlr4-runtime</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.alibaba</groupId>
-      <artifactId>fastjson</artifactId>
-    </dependency>
-    <dependency>
       <groupId>org.testng</groupId>
       <artifactId>testng</artifactId>
       <scope>test</scope>
@@ -142,10 +138,6 @@
       <artifactId>joda-time</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.json</groupId>
-      <artifactId>json</artifactId>
-    </dependency>
-    <dependency>
       <groupId>commons-configuration</groupId>
       <artifactId>commons-configuration</artifactId>
       <exclusions>
@@ -227,16 +219,16 @@
       <artifactId>swagger-ui</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-annotations</artifactId>
-    </dependency>
-    <dependency>
       <groupId>com.google.code.findbugs</groupId>
       <artifactId>jsr305</artifactId>
     </dependency>
     <dependency>
-      <groupId>com.google.code.gson</groupId>
-      <artifactId>gson</artifactId>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/config/ColumnPartitionConfig.java b/pinot-common/src/main/java/com/linkedin/pinot/common/config/ColumnPartitionConfig.java
index ce39dfe..c62d4a6 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/config/ColumnPartitionConfig.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/config/ColumnPartitionConfig.java
@@ -18,14 +18,14 @@
  */
 package com.linkedin.pinot.common.config;
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Preconditions;
 import com.linkedin.pinot.common.utils.EqualityUtils;
 import java.util.ArrayList;
 import java.util.List;
 import javax.annotation.Nonnull;
 import org.apache.commons.lang.math.IntRange;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.annotate.JsonProperty;
 
 
 @SuppressWarnings("unused") // Suppress incorrect warnings as methods used for ser/de.
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/config/IndexingConfig.java b/pinot-common/src/main/java/com/linkedin/pinot/common/config/IndexingConfig.java
index 327e113..4c9b7be 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/config/IndexingConfig.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/config/IndexingConfig.java
@@ -18,6 +18,7 @@
  */
 package com.linkedin.pinot.common.config;
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.linkedin.pinot.common.data.StarTreeIndexSpec;
 import com.linkedin.pinot.common.utils.EqualityUtils;
 import java.lang.reflect.Field;
@@ -25,7 +26,6 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -172,7 +172,7 @@ public class IndexingConfig {
     return _noDictionaryColumns;
   }
 
-  public Map<String, String> getnoDictionaryConfig() {
+  public Map<String, String> getNoDictionaryConfig() {
     return _noDictionaryConfig;
   }
 
@@ -220,7 +220,7 @@ public class IndexingConfig {
     return _segmentPartitionConfig;
   }
 
-  public boolean getAggregateMetrics() {
+  public boolean isAggregateMetrics() {
     return _aggregateMetrics;
   }
 
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/config/QuotaConfig.java b/pinot-common/src/main/java/com/linkedin/pinot/common/config/QuotaConfig.java
index bfa2fb5..7dded32 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/config/QuotaConfig.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/config/QuotaConfig.java
@@ -18,13 +18,12 @@
  */
 package com.linkedin.pinot.common.config;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.linkedin.pinot.common.utils.DataSize;
 import com.linkedin.pinot.common.utils.EqualityUtils;
 import javax.annotation.Nullable;
 import org.apache.commons.configuration.ConfigurationRuntimeException;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.json.JSONException;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -35,8 +34,6 @@ import org.slf4j.LoggerFactory;
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class QuotaConfig {
   private static final Logger LOGGER = LoggerFactory.getLogger(QuotaConfig.class);
-  private static final String STORAGE_FIELD_NAME = "storage";
-  private static final String MAX_QUERIES_PER_SECOND_FIELD_NAME = "maxQueriesPerSecond";
 
   @ConfigKey("storage")
   @ConfigDoc(value = "Storage allocated for this table", exampleValue = "10 GiB")
@@ -70,21 +67,6 @@ public class QuotaConfig {
     return DataSize.toBytes(_storage);
   }
 
-  public JSONObject toJson() {
-    JSONObject quotaObject = new JSONObject();
-    try {
-      quotaObject.put(STORAGE_FIELD_NAME, _storage);
-      quotaObject.put(MAX_QUERIES_PER_SECOND_FIELD_NAME, _maxQueriesPerSecond);
-    } catch (JSONException e) {
-      LOGGER.error("Failed to convert to json", e);
-    }
-    return quotaObject;
-  }
-
-  public String toString() {
-    return toJson().toString();
-  }
-
   public void validate() {
     if (!isStorageValid()) {
       LOGGER.error("Failed to convert storage quota config: {} to bytes", _storage);
@@ -96,10 +78,12 @@ public class QuotaConfig {
     }
   }
 
+  @JsonIgnore
   public boolean isStorageValid() {
     return _storage == null || DataSize.toBytes(_storage) >= 0;
   }
 
+  @JsonIgnore
   public boolean isMaxQueriesPerSecondValid() {
     Double qps = null;
     if (_maxQueriesPerSecond != null) {
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/config/ReplicaGroupStrategyConfig.java b/pinot-common/src/main/java/com/linkedin/pinot/common/config/ReplicaGroupStrategyConfig.java
index 4de9d24..3f6b717 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/config/ReplicaGroupStrategyConfig.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/config/ReplicaGroupStrategyConfig.java
@@ -18,9 +18,9 @@
  */
 package com.linkedin.pinot.common.config;
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.linkedin.pinot.common.utils.EqualityUtils;
 import javax.annotation.Nullable;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
 /**
  * Class representing configurations related to segment assignment strategy.
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/config/RoutingConfig.java b/pinot-common/src/main/java/com/linkedin/pinot/common/config/RoutingConfig.java
index f948868..663fff5 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/config/RoutingConfig.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/config/RoutingConfig.java
@@ -18,28 +18,20 @@
  */
 package com.linkedin.pinot.common.config;
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.linkedin.pinot.common.utils.EqualityUtils;
-import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class RoutingConfig {
-  private static final Logger LOGGER = LoggerFactory.getLogger(RoutingConfig.class);
-
   public static final String ENABLE_DYNAMIC_COMPUTING_KEY = "enableDynamicComputing";
-  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
 
   @ConfigKey("routingTableBuilderName")
   private String _routingTableBuilderName;
 
-  private Map<String,String> _routingTableBuilderOptions = new HashMap<>();
+  private Map<String, String> _routingTableBuilderOptions = new HashMap<>();
 
   public String getRoutingTableBuilderName() {
     return _routingTableBuilderName;
@@ -57,15 +49,6 @@ public class RoutingConfig {
     _routingTableBuilderOptions = routingTableBuilderOptions;
   }
 
-  public String toString() {
-    try {
-      return OBJECT_MAPPER.writeValueAsString(this);
-    } catch (IOException e) {
-      //ignore
-    }
-    return "";
-  }
-
   @Override
   public boolean equals(Object o) {
     if (EqualityUtils.isSameReference(this, o)) {
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/config/SegmentPartitionConfig.java b/pinot-common/src/main/java/com/linkedin/pinot/common/config/SegmentPartitionConfig.java
index 967039d..f9788bc 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/config/SegmentPartitionConfig.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/config/SegmentPartitionConfig.java
@@ -18,22 +18,21 @@
  */
 package com.linkedin.pinot.common.config;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Preconditions;
 import com.linkedin.pinot.common.utils.EqualityUtils;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.io.IOException;
 import java.util.Map;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
-import org.codehaus.jackson.annotate.JsonIgnore;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.codehaus.jackson.map.ObjectMapper;
 
 
 @SuppressWarnings("unused") // Suppress incorrect warning, as methods are used for json ser/de.
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class SegmentPartitionConfig {
-  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
   public static final int INVALID_NUM_PARTITIONS = -1;
 
   @ConfigKey("columnPartitionMap")
@@ -96,9 +95,8 @@ public class SegmentPartitionConfig {
    * @return Instance of {@link SegmentPartitionConfig} built from the input string.
    * @throws IOException
    */
-  public static SegmentPartitionConfig fromJsonString(String jsonString)
-      throws IOException {
-    return OBJECT_MAPPER.readValue(jsonString, SegmentPartitionConfig.class);
+  public static SegmentPartitionConfig fromJsonString(String jsonString) throws IOException {
+    return JsonUtils.stringToObject(jsonString, SegmentPartitionConfig.class);
   }
 
   /**
@@ -107,9 +105,8 @@ public class SegmentPartitionConfig {
    * @return JSON string equivalent of the object.
    * @throws IOException
    */
-  public String toJsonString()
-      throws IOException {
-    return OBJECT_MAPPER.writeValueAsString(this);
+  public String toJsonString() throws IOException {
+    return JsonUtils.objectToString(this);
   }
 
   /**
@@ -144,9 +141,4 @@ public class SegmentPartitionConfig {
     int result = EqualityUtils.hashCodeOf(_columnPartitionMap);
     return result;
   }
-
-  @Override
-  public String toString() {
-    return "SegmentPartitionConfig{" + "_columnPartitionMap=" + _columnPartitionMap + '}';
-  }
 }
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/config/SegmentsValidationAndRetentionConfig.java b/pinot-common/src/main/java/com/linkedin/pinot/common/config/SegmentsValidationAndRetentionConfig.java
index a7f0d3d..ebb563d 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/config/SegmentsValidationAndRetentionConfig.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/config/SegmentsValidationAndRetentionConfig.java
@@ -18,11 +18,11 @@
  */
 package com.linkedin.pinot.common.config;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.linkedin.pinot.common.utils.EqualityUtils;
 import com.linkedin.pinot.startree.hll.HllConfig;
 import java.lang.reflect.Field;
-import org.codehaus.jackson.annotate.JsonIgnore;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/config/StarTreeIndexConfig.java b/pinot-common/src/main/java/com/linkedin/pinot/common/config/StarTreeIndexConfig.java
index 06ebabf..3ab06ae 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/config/StarTreeIndexConfig.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/config/StarTreeIndexConfig.java
@@ -18,8 +18,8 @@
  */
 package com.linkedin.pinot.common.config;
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import java.util.List;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
 
 @SuppressWarnings("unused")
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/config/StreamConsumptionConfig.java b/pinot-common/src/main/java/com/linkedin/pinot/common/config/StreamConsumptionConfig.java
index ff37c4d..180bc6b 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/config/StreamConsumptionConfig.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/config/StreamConsumptionConfig.java
@@ -18,20 +18,11 @@
  */
 package com.linkedin.pinot.common.config;
 
-import java.io.IOException;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 
 
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class StreamConsumptionConfig {
-
-  private static final Logger LOGGER = LoggerFactory.getLogger(StreamConsumptionConfig.class);
-
-  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
-
   private String _streamPartitionAssignmentStrategy;
 
   public String getStreamPartitionAssignmentStrategy() {
@@ -41,14 +32,4 @@ public class StreamConsumptionConfig {
   public void setStreamPartitionAssignmentStrategy(String streamPartitionAssignmentStrategy) {
     _streamPartitionAssignmentStrategy = streamPartitionAssignmentStrategy;
   }
-
-  @Override
-  public String toString() {
-    try {
-      return OBJECT_MAPPER.writeValueAsString(this);
-    } catch (IOException e) {
-      return e.toString();
-    }
-  }
-
 }
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/config/TableConfig.java b/pinot-common/src/main/java/com/linkedin/pinot/common/config/TableConfig.java
index 3d24d27..8ff2b1d 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/config/TableConfig.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/config/TableConfig.java
@@ -18,10 +18,14 @@
  */
 package com.linkedin.pinot.common.config;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.google.common.base.Preconditions;
 import com.linkedin.pinot.common.data.StarTreeIndexSpec;
 import com.linkedin.pinot.common.utils.CommonConstants.Helix.TableType;
 import com.linkedin.pinot.common.utils.EqualityUtils;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.startree.hll.HllConfig;
 import java.io.IOException;
 import java.util.Collections;
@@ -31,16 +35,12 @@ import java.util.Map;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 import org.apache.helix.ZNRecord;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.json.JSONException;
-import org.json.JSONObject;
 
 
+@SuppressWarnings("unused")
 @ConfigDoc(value = "Configuration for a table", mandatory = true)
 @ConfigKey("table")
 public class TableConfig {
-  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
-
   private static final String TABLE_NAME_KEY = "tableName";
   private static final String TABLE_TYPE_KEY = "tableType";
   private static final String VALIDATION_CONFIG_KEY = "segmentsConfig";
@@ -106,95 +106,106 @@ public class TableConfig {
   @Deprecated
   @Nonnull
   public static TableConfig init(@Nonnull String jsonConfigString)
-      throws IOException, JSONException {
-    return fromJSONConfig(new JSONObject(jsonConfigString));
+      throws IOException {
+    return fromJsonString(jsonConfigString);
+  }
+
+  public static TableConfig fromJsonString(String jsonString) throws IOException {
+    return fromJSONConfig(JsonUtils.stringToJsonNode(jsonString));
   }
 
   @Nonnull
-  public static TableConfig fromJSONConfig(@Nonnull JSONObject jsonConfig)
-      throws IOException, JSONException {
-    TableType tableType = TableType.valueOf(jsonConfig.getString(TABLE_TYPE_KEY).toUpperCase());
-    String tableName = TableNameBuilder.forType(tableType).tableNameWithType(jsonConfig.getString(TABLE_NAME_KEY));
+  public static TableConfig fromJSONConfig(@Nonnull JsonNode jsonConfig) throws IOException {
+    TableType tableType = TableType.valueOf(jsonConfig.get(TABLE_TYPE_KEY).asText().toUpperCase());
+    String tableName = TableNameBuilder.forType(tableType).tableNameWithType(jsonConfig.get(TABLE_NAME_KEY).asText());
+
     SegmentsValidationAndRetentionConfig validationConfig =
-        OBJECT_MAPPER.readValue(jsonConfig.getJSONObject(VALIDATION_CONFIG_KEY).toString(),
-            SegmentsValidationAndRetentionConfig.class);
-    TenantConfig tenantConfig =
-        OBJECT_MAPPER.readValue(jsonConfig.getJSONObject(TENANT_CONFIG_KEY).toString(), TenantConfig.class);
-    IndexingConfig indexingConfig =
-        OBJECT_MAPPER.readValue(jsonConfig.getJSONObject(INDEXING_CONFIG_KEY).toString(), IndexingConfig.class);
-    TableCustomConfig customConfig =
-        OBJECT_MAPPER.readValue(jsonConfig.getJSONObject(CUSTOM_CONFIG_KEY).toString(), TableCustomConfig.class);
+        extractChildConfig(jsonConfig, VALIDATION_CONFIG_KEY, SegmentsValidationAndRetentionConfig.class);
+    TenantConfig tenantConfig = extractChildConfig(jsonConfig, TENANT_CONFIG_KEY, TenantConfig.class);
+    IndexingConfig indexingConfig = extractChildConfig(jsonConfig, INDEXING_CONFIG_KEY, IndexingConfig.class);
+    TableCustomConfig customConfig = extractChildConfig(jsonConfig, CUSTOM_CONFIG_KEY, TableCustomConfig.class);
     QuotaConfig quotaConfig = null;
     if (jsonConfig.has(QUOTA_CONFIG_KEY)) {
-      quotaConfig = OBJECT_MAPPER.readValue(jsonConfig.getJSONObject(QUOTA_CONFIG_KEY).toString(), QuotaConfig.class);
+      quotaConfig = extractChildConfig(jsonConfig, QUOTA_CONFIG_KEY, QuotaConfig.class);
       quotaConfig.validate();
     }
     TableTaskConfig taskConfig = null;
     if (jsonConfig.has(TASK_CONFIG_KEY)) {
-      taskConfig = OBJECT_MAPPER.readValue(jsonConfig.getJSONObject(TASK_CONFIG_KEY).toString(), TableTaskConfig.class);
+      taskConfig = extractChildConfig(jsonConfig, TASK_CONFIG_KEY, TableTaskConfig.class);
     }
     RoutingConfig routingConfig = null;
     if (jsonConfig.has(ROUTING_CONFIG_KEY)) {
-      routingConfig =
-          OBJECT_MAPPER.readValue(jsonConfig.getJSONObject(ROUTING_CONFIG_KEY).toString(), RoutingConfig.class);
+      routingConfig = extractChildConfig(jsonConfig, ROUTING_CONFIG_KEY, RoutingConfig.class);
     }
 
     return new TableConfig(tableName, tableType, validationConfig, tenantConfig, indexingConfig, customConfig,
         quotaConfig, taskConfig, routingConfig);
   }
 
+  /**
+   * Extracts the child config from the table config.
+   * <p>
+   * NOTE: for historical reason, we support two kinds of nested config values: normal json and serialized json string
+   */
+  private static <T> T extractChildConfig(JsonNode jsonConfig, String childConfigKey, Class<T> childConfigClass)
+      throws IOException {
+    JsonNode childConfigNode = jsonConfig.get(childConfigKey);
+    if (childConfigNode.isObject()) {
+      return JsonUtils.jsonNodeToObject(childConfigNode, childConfigClass);
+    } else {
+      return JsonUtils.stringToObject(childConfigNode.asText(), childConfigClass);
+    }
+  }
+
   @Nonnull
-  public static JSONObject toJSONConfig(@Nonnull TableConfig tableConfig)
-      throws IOException, JSONException {
-    JSONObject jsonConfig = new JSONObject();
+  public static JsonNode toJSONConfig(@Nonnull TableConfig tableConfig) throws JsonProcessingException {
+    ObjectNode jsonConfig = JsonUtils.newObjectNode();
     jsonConfig.put(TABLE_NAME_KEY, tableConfig._tableName);
     jsonConfig.put(TABLE_TYPE_KEY, tableConfig._tableType.toString());
-    jsonConfig.put(VALIDATION_CONFIG_KEY,
-        new JSONObject(OBJECT_MAPPER.writeValueAsString(tableConfig._validationConfig)));
-    jsonConfig.put(TENANT_CONFIG_KEY, new JSONObject(OBJECT_MAPPER.writeValueAsString(tableConfig._tenantConfig)));
-    jsonConfig.put(INDEXING_CONFIG_KEY, new JSONObject(OBJECT_MAPPER.writeValueAsString(tableConfig._indexingConfig)));
-    jsonConfig.put(CUSTOM_CONFIG_KEY, new JSONObject(OBJECT_MAPPER.writeValueAsString(tableConfig._customConfig)));
+    jsonConfig.put(VALIDATION_CONFIG_KEY, JsonUtils.objectToString(tableConfig._validationConfig));
+    jsonConfig.put(TENANT_CONFIG_KEY, JsonUtils.objectToString(tableConfig._tenantConfig));
+    jsonConfig.put(INDEXING_CONFIG_KEY, JsonUtils.objectToString(tableConfig._indexingConfig));
+    jsonConfig.put(CUSTOM_CONFIG_KEY, JsonUtils.objectToString(tableConfig._customConfig));
     if (tableConfig._quotaConfig != null) {
-      jsonConfig.put(QUOTA_CONFIG_KEY, new JSONObject(OBJECT_MAPPER.writeValueAsString(tableConfig._quotaConfig)));
+      jsonConfig.put(QUOTA_CONFIG_KEY, JsonUtils.objectToString(tableConfig._quotaConfig));
     }
     if (tableConfig._taskConfig != null) {
-      jsonConfig.put(TASK_CONFIG_KEY, new JSONObject(OBJECT_MAPPER.writeValueAsString(tableConfig._taskConfig)));
+      jsonConfig.put(TASK_CONFIG_KEY, JsonUtils.objectToString(tableConfig._taskConfig));
     }
     if (tableConfig._routingConfig != null) {
-      jsonConfig.put(ROUTING_CONFIG_KEY, new JSONObject(OBJECT_MAPPER.writeValueAsString(tableConfig._routingConfig)));
+      jsonConfig.put(ROUTING_CONFIG_KEY, JsonUtils.objectToString(tableConfig._routingConfig));
     }
     return jsonConfig;
   }
 
   @Nonnull
-  public static TableConfig fromZnRecord(@Nonnull ZNRecord znRecord)
-      throws IOException, JSONException {
+  public static TableConfig fromZnRecord(@Nonnull ZNRecord znRecord) throws IOException {
     Map<String, String> simpleFields = znRecord.getSimpleFields();
     TableType tableType = TableType.valueOf(simpleFields.get(TABLE_TYPE_KEY).toUpperCase());
     String tableName = TableNameBuilder.forType(tableType).tableNameWithType(simpleFields.get(TABLE_NAME_KEY));
     SegmentsValidationAndRetentionConfig validationConfig =
-        OBJECT_MAPPER.readValue(simpleFields.get(VALIDATION_CONFIG_KEY), SegmentsValidationAndRetentionConfig.class);
-    TenantConfig tenantConfig = OBJECT_MAPPER.readValue(simpleFields.get(TENANT_CONFIG_KEY), TenantConfig.class);
+        JsonUtils.stringToObject(simpleFields.get(VALIDATION_CONFIG_KEY), SegmentsValidationAndRetentionConfig.class);
+    TenantConfig tenantConfig = JsonUtils.stringToObject(simpleFields.get(TENANT_CONFIG_KEY), TenantConfig.class);
     IndexingConfig indexingConfig =
-        OBJECT_MAPPER.readValue(simpleFields.get(INDEXING_CONFIG_KEY), IndexingConfig.class);
+        JsonUtils.stringToObject(simpleFields.get(INDEXING_CONFIG_KEY), IndexingConfig.class);
     TableCustomConfig customConfig =
-        OBJECT_MAPPER.readValue(simpleFields.get(CUSTOM_CONFIG_KEY), TableCustomConfig.class);
+        JsonUtils.stringToObject(simpleFields.get(CUSTOM_CONFIG_KEY), TableCustomConfig.class);
     QuotaConfig quotaConfig = null;
     String quotaConfigString = simpleFields.get(QUOTA_CONFIG_KEY);
     if (quotaConfigString != null) {
-      quotaConfig = OBJECT_MAPPER.readValue(quotaConfigString, QuotaConfig.class);
+      quotaConfig = JsonUtils.stringToObject(quotaConfigString, QuotaConfig.class);
       quotaConfig.validate();
     }
     TableTaskConfig taskConfig = null;
     String taskConfigString = simpleFields.get(TASK_CONFIG_KEY);
     if (taskConfigString != null) {
-      taskConfig = OBJECT_MAPPER.readValue(taskConfigString, TableTaskConfig.class);
+      taskConfig = JsonUtils.stringToObject(taskConfigString, TableTaskConfig.class);
     }
     String routingConfigString = simpleFields.get(ROUTING_CONFIG_KEY);
 
     RoutingConfig routingConfig = null;
     if (routingConfigString != null) {
-      routingConfig = OBJECT_MAPPER.readValue(routingConfigString, RoutingConfig.class);
+      routingConfig = JsonUtils.stringToObject(routingConfigString, RoutingConfig.class);
     }
 
     return new TableConfig(tableName, tableType, validationConfig, tenantConfig, indexingConfig, customConfig,
@@ -208,18 +219,18 @@ public class TableConfig {
     simpleFields.put(TABLE_NAME_KEY, tableConfig._tableName);
     simpleFields.put(TABLE_TYPE_KEY, tableConfig._tableType.toString());
     try {
-      simpleFields.put(VALIDATION_CONFIG_KEY, OBJECT_MAPPER.writeValueAsString(tableConfig._validationConfig));
-      simpleFields.put(TENANT_CONFIG_KEY, OBJECT_MAPPER.writeValueAsString(tableConfig._tenantConfig));
-      simpleFields.put(INDEXING_CONFIG_KEY, OBJECT_MAPPER.writeValueAsString(tableConfig._indexingConfig));
-      simpleFields.put(CUSTOM_CONFIG_KEY, OBJECT_MAPPER.writeValueAsString(tableConfig._customConfig));
+      simpleFields.put(VALIDATION_CONFIG_KEY, JsonUtils.objectToString(tableConfig._validationConfig));
+      simpleFields.put(TENANT_CONFIG_KEY, JsonUtils.objectToString(tableConfig._tenantConfig));
+      simpleFields.put(INDEXING_CONFIG_KEY, JsonUtils.objectToString(tableConfig._indexingConfig));
+      simpleFields.put(CUSTOM_CONFIG_KEY, JsonUtils.objectToString(tableConfig._customConfig));
       if (tableConfig._quotaConfig != null) {
-        simpleFields.put(QUOTA_CONFIG_KEY, OBJECT_MAPPER.writeValueAsString(tableConfig._quotaConfig));
+        simpleFields.put(QUOTA_CONFIG_KEY, JsonUtils.objectToString(tableConfig._quotaConfig));
       }
       if (tableConfig._taskConfig != null) {
-        simpleFields.put(TASK_CONFIG_KEY, OBJECT_MAPPER.writeValueAsString(tableConfig._taskConfig));
+        simpleFields.put(TASK_CONFIG_KEY, JsonUtils.objectToString(tableConfig._taskConfig));
       }
       if (tableConfig._routingConfig != null) {
-        simpleFields.put(ROUTING_CONFIG_KEY, OBJECT_MAPPER.writeValueAsString(tableConfig._routingConfig));
+        simpleFields.put(ROUTING_CONFIG_KEY, JsonUtils.objectToString(tableConfig._routingConfig));
       }
     } catch (IOException e) {
       throw new RuntimeException(e);
@@ -309,44 +320,39 @@ public class TableConfig {
   }
 
   @Nonnull
-  public String toJSONConfigString()
-      throws IOException, JSONException {
+  public String toJSONConfigString() throws IOException {
     return toJSONConfig(this).toString();
   }
 
   @Override
   public String toString() {
     try {
-      return toJSONConfig(this).toString(2);
-    } catch (Exception e) {
-      return e.toString();
+      return JsonUtils.objectToPrettyString(toJSONConfig(this));
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
     }
   }
 
   @Override
-  public boolean equals(Object o) {
-    if (EqualityUtils.isSameReference(this, o)) {
+  public boolean equals(Object obj) {
+    if (this == obj) {
       return true;
     }
-
-    if (EqualityUtils.isNullOrNotSameClass(this, o)) {
-      return false;
-    }
-
-    TableConfig that = (TableConfig) o;
-
-    return EqualityUtils.isEqual(_tableName, that._tableName) &&
-        EqualityUtils.isEqual(_tableType, that._tableType) &&
-        EqualityUtils.isEqual(_validationConfig, that._validationConfig) &&
-        EqualityUtils.isEqual(_tenantConfig, that._tenantConfig) &&
-        EqualityUtils.isEqual(_indexingConfig, that._indexingConfig) &&
-        EqualityUtils.isEqual(_customConfig, that._customConfig) &&
-        EqualityUtils.isEqual(_quotaConfig, that._quotaConfig) &&
-        EqualityUtils.isEqual(_taskConfig, that._taskConfig) &&
-        EqualityUtils.isEqual(_routingConfig, that._routingConfig);
+    if (obj instanceof TableConfig) {
+      TableConfig that = (TableConfig) obj;
+      return EqualityUtils.isEqual(_tableName, that._tableName)
+          && EqualityUtils.isEqual(_tableType, that._tableType)
+          && EqualityUtils.isEqual(_validationConfig, that._validationConfig)
+          && EqualityUtils.isEqual(_tenantConfig, that._tenantConfig)
+          && EqualityUtils.isEqual(_indexingConfig, that._indexingConfig)
+          && EqualityUtils.isEqual(_customConfig, that._customConfig)
+          && EqualityUtils.isEqual(_quotaConfig, that._quotaConfig)
+          && EqualityUtils.isEqual(_taskConfig, that._taskConfig)
+          && EqualityUtils.isEqual(_routingConfig, that._routingConfig);
+    }
+    return false;
   }
 
-
   @Override
   public int hashCode() {
     int result = EqualityUtils.hashCodeOf(_tableName);
@@ -558,8 +564,7 @@ public class TableConfig {
       return this;
     }
 
-    public TableConfig build()
-        throws IOException, JSONException {
+    public TableConfig build() {
       // Validation config
       SegmentsValidationAndRetentionConfig validationConfig = new SegmentsValidationAndRetentionConfig();
       validationConfig.setTimeColumnName(_timeColumnName);
@@ -600,7 +605,7 @@ public class TableConfig {
 
       if (_customConfig == null) {
         _customConfig = new TableCustomConfig();
-        _customConfig.setCustomConfigs(new HashMap<String, String>());
+        _customConfig.setCustomConfigs(new HashMap<>());
       }
 
       if (_routingConfig == null) {
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/config/TableCustomConfig.java b/pinot-common/src/main/java/com/linkedin/pinot/common/config/TableCustomConfig.java
index e888c52..5526015 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/config/TableCustomConfig.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/config/TableCustomConfig.java
@@ -18,10 +18,10 @@
  */
 package com.linkedin.pinot.common.config;
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.linkedin.pinot.common.utils.EqualityUtils;
 import java.lang.reflect.Field;
 import java.util.Map;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/config/TableTaskConfig.java b/pinot-common/src/main/java/com/linkedin/pinot/common/config/TableTaskConfig.java
index 7fc554e..45a559f 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/config/TableTaskConfig.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/config/TableTaskConfig.java
@@ -18,18 +18,15 @@
  */
 package com.linkedin.pinot.common.config;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.linkedin.pinot.common.utils.EqualityUtils;
 import java.util.Map;
-import org.codehaus.jackson.annotate.JsonIgnore;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.json.JSONException;
-import org.json.JSONObject;
 
 
 @SuppressWarnings("unused")
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class TableTaskConfig {
-  private static final String TASK_TYPE_CONFIGS_MAP_KEY = "taskTypeConfigsMap";
 
   @ConfigKey("taskConfig")
   @UseChildKeyHandler(TaskConfigMapChildKeyHandler.class)
@@ -54,17 +51,6 @@ public class TableTaskConfig {
   }
 
   @Override
-  public String toString() {
-    JSONObject jsonTaskConfigsMap = new JSONObject();
-    try {
-      jsonTaskConfigsMap.put(TASK_TYPE_CONFIGS_MAP_KEY, _taskTypeConfigsMap);
-      return jsonTaskConfigsMap.toString(2);
-    } catch (JSONException e) {
-      return e.toString();
-    }
-  }
-
-  @Override
   public boolean equals(Object o) {
     if (EqualityUtils.isSameReference(this, o)) {
       return true;
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/config/TagOverrideConfig.java b/pinot-common/src/main/java/com/linkedin/pinot/common/config/TagOverrideConfig.java
index b14480a..fa2532e 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/config/TagOverrideConfig.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/config/TagOverrideConfig.java
@@ -18,10 +18,8 @@
  */
 package com.linkedin.pinot.common.config;
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.linkedin.pinot.common.utils.EqualityUtils;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 
 /**
@@ -44,7 +42,6 @@ import org.slf4j.LoggerFactory;
  */
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class TagOverrideConfig {
-  private static final Logger LOGGER = LoggerFactory.getLogger(TagOverrideConfig.class);
 
   @ConfigKey("realtimeConsuming")
   @ConfigDoc("Tag override for realtime consuming segments")
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/config/Tenant.java b/pinot-common/src/main/java/com/linkedin/pinot/common/config/Tenant.java
index 02c9e92..fd30116 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/config/Tenant.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/config/Tenant.java
@@ -18,120 +18,91 @@
  */
 package com.linkedin.pinot.common.config;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.linkedin.pinot.common.utils.EqualityUtils;
-import org.codehaus.jackson.annotate.JsonIgnore;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.json.JSONException;
-import org.json.JSONObject;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Objects;
 import com.linkedin.pinot.common.utils.TenantRole;
 
+
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class Tenant {
+  private TenantRole _tenantRole;
+  private String _tenantName;
+  private int _numberOfInstances = 0;
+  private int _offlineInstances = 0;
+  private int _realtimeInstances = 0;
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(Tenant.class);
-
-  private String tenantRole;
-  private String tenantName;
-  private int numberOfInstances = 0;
-  private int offlineInstances = 0;
-  private int realtimeInstances = 0;
-
-  // private boolean colocated = false;
-
-  public void setTenantRole(String tenantRole) {
-    this.tenantRole = tenantRole;
+  public TenantRole getTenantRole() {
+    return _tenantRole;
   }
 
-  public void setTenantName(String tenantName) {
-    this.tenantName = tenantName;
+  public void setTenantRole(TenantRole tenantRole) {
+    _tenantRole = tenantRole;
   }
 
-  public void setNumberOfInstances(int numberOfInstances) {
-    this.numberOfInstances = numberOfInstances;
+  public String getTenantName() {
+    return _tenantName;
   }
 
-  public void setOfflineInstances(int offlineInstances) {
-    this.offlineInstances = offlineInstances;
+  public void setTenantName(String tenantName) {
+    _tenantName = tenantName;
   }
 
-  public void setRealtimeInstances(int realtimeInstances) {
-    this.realtimeInstances = realtimeInstances;
+  public int getNumberOfInstances() {
+    return _numberOfInstances;
   }
 
-  public TenantRole getTenantRole() {
-    return TenantRole.valueOf(tenantRole.toUpperCase());
+  public void setNumberOfInstances(int numberOfInstances) {
+    _numberOfInstances = numberOfInstances;
   }
 
-  public String getTenantName() {
-    return tenantName;
+  public int getOfflineInstances() {
+    return _offlineInstances;
   }
 
-  public int getNumberOfInstances() {
-    return numberOfInstances;
+  public void setOfflineInstances(int offlineInstances) {
+    _offlineInstances = offlineInstances;
   }
 
-  public int getOfflineInstances() {
-    return offlineInstances;
+  public int getRealtimeInstances() {
+    return _realtimeInstances;
   }
 
-  public int getRealtimeInstances() {
-    return realtimeInstances;
+  public void setRealtimeInstances(int realtimeInstances) {
+    _realtimeInstances = realtimeInstances;
   }
 
   @JsonIgnore
   public boolean isCoLocated() {
-    return (realtimeInstances + offlineInstances > numberOfInstances);
+    return _realtimeInstances + _offlineInstances > _numberOfInstances;
   }
 
   @Override
-  public boolean equals(Object o) {
-    if (EqualityUtils.isSameReference(this, o)) {
+  public boolean equals(Object obj) {
+    if (this == obj) {
       return true;
     }
-
-    if (EqualityUtils.isNullOrNotSameClass(this, o)) {
-      return false;
+    if (obj instanceof Tenant) {
+      Tenant that = (Tenant) obj;
+      return EqualityUtils.isEqual(_numberOfInstances, that._numberOfInstances)
+          && EqualityUtils.isEqual(_offlineInstances, that._offlineInstances)
+          && EqualityUtils.isEqual(_realtimeInstances, that._realtimeInstances)
+          && EqualityUtils.isEqual(_tenantRole, that._tenantRole)
+          && EqualityUtils.isEqual(_tenantName, that._tenantName);
     }
-
-    Tenant tenant = (Tenant) o;
-
-    return EqualityUtils.isEqual(numberOfInstances, tenant.numberOfInstances) &&
-        EqualityUtils.isEqual(offlineInstances, tenant.offlineInstances) &&
-        EqualityUtils.isEqual(realtimeInstances, tenant.realtimeInstances) &&
-        EqualityUtils.isEqual(tenantRole, tenant.tenantRole) &&
-        EqualityUtils.isEqual(tenantName, tenant.tenantName);
+    return false;
   }
 
   @Override
   public int hashCode() {
-    int result = EqualityUtils.hashCodeOf(tenantRole);
-    result = EqualityUtils.hashCodeOf(result, tenantName);
-    result = EqualityUtils.hashCodeOf(result, numberOfInstances);
-    result = EqualityUtils.hashCodeOf(result, offlineInstances);
-    result = EqualityUtils.hashCodeOf(result, realtimeInstances);
+    int result = EqualityUtils.hashCodeOf(_tenantRole);
+    result = EqualityUtils.hashCodeOf(result, _tenantName);
+    result = EqualityUtils.hashCodeOf(result, _numberOfInstances);
+    result = EqualityUtils.hashCodeOf(result, _offlineInstances);
+    result = EqualityUtils.hashCodeOf(result, _realtimeInstances);
     return result;
   }
 
-  @Override
-  public String toString() {
-    String ret = null;
-    try {
-      ret = new ObjectMapper().writeValueAsString(this);
-    } catch (Exception e) {
-      LOGGER.error("error toString for tenant ", e);
-    }
-    return ret;
-  }
-
-  public JSONObject toJSON() throws JSONException {
-    return new JSONObject(toString());
-  }
-
   public static class TenantBuilder {
     Tenant tenant;
 
@@ -141,7 +112,7 @@ public class Tenant {
     }
 
     public TenantBuilder setRole(TenantRole role) {
-      tenant.setTenantRole(role.toString());
+      tenant.setTenantRole(role);
       return this;
     }
 
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/config/TenantConfig.java b/pinot-common/src/main/java/com/linkedin/pinot/common/config/TenantConfig.java
index ba717f5..3918001 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/config/TenantConfig.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/config/TenantConfig.java
@@ -18,10 +18,9 @@
  */
 package com.linkedin.pinot.common.config;
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.linkedin.pinot.common.utils.EqualityUtils;
 import java.lang.reflect.Field;
-
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/data/DateTimeFieldSpec.java b/pinot-common/src/main/java/com/linkedin/pinot/common/data/DateTimeFieldSpec.java
index ace21c6..4772a20 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/data/DateTimeFieldSpec.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/data/DateTimeFieldSpec.java
@@ -18,13 +18,13 @@
  */
 package com.linkedin.pinot.common.data;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.google.common.base.Preconditions;
-import com.google.gson.JsonObject;
 import com.linkedin.pinot.common.config.ConfigKey;
 import com.linkedin.pinot.common.utils.EqualityUtils;
 import javax.annotation.Nonnull;
-import org.codehaus.jackson.annotate.JsonIgnore;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
 
 @SuppressWarnings("unused")
@@ -121,10 +121,10 @@ public final class DateTimeFieldSpec extends FieldSpec {
 
   @Nonnull
   @Override
-  public JsonObject toJsonObject() {
-    JsonObject jsonObject = super.toJsonObject();
-    jsonObject.addProperty("format", _format);
-    jsonObject.addProperty("granularity", _granularity);
+  public ObjectNode toJsonObject() {
+    ObjectNode jsonObject = super.toJsonObject();
+    jsonObject.put("format", _format);
+    jsonObject.put("granularity", _granularity);
     return jsonObject;
   }
 
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/data/DimensionFieldSpec.java b/pinot-common/src/main/java/com/linkedin/pinot/common/data/DimensionFieldSpec.java
index 970a896..db9937b 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/data/DimensionFieldSpec.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/data/DimensionFieldSpec.java
@@ -18,9 +18,9 @@
  */
 package com.linkedin.pinot.common.data;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import javax.annotation.Nonnull;
-import org.codehaus.jackson.annotate.JsonIgnore;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
 
 @JsonIgnoreProperties(ignoreUnknown = true)
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/data/FieldSpec.java b/pinot-common/src/main/java/com/linkedin/pinot/common/data/FieldSpec.java
index d61b1be..8fb1a15 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/data/FieldSpec.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/data/FieldSpec.java
@@ -18,13 +18,13 @@
  */
 package com.linkedin.pinot.common.data;
 
-import com.google.gson.JsonArray;
-import com.google.gson.JsonObject;
-import com.google.gson.JsonPrimitive;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.linkedin.pinot.common.Utils;
 import com.linkedin.pinot.common.config.ConfigKey;
 import com.linkedin.pinot.common.config.ConfigNodeLifecycleAware;
 import com.linkedin.pinot.common.utils.EqualityUtils;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import javax.annotation.Nullable;
 import org.apache.avro.Schema.Type;
 import org.apache.commons.codec.DecoderException;
@@ -267,63 +267,63 @@ public abstract class FieldSpec implements Comparable<FieldSpec>, ConfigNodeLife
   }
 
   /**
-   * Returns the {@link JsonObject} representing the field spec.
+   * Returns the {@link ObjectNode} representing the field spec.
    * <p>Only contains fields with non-default value.
-   * <p>NOTE: here we use {@link JsonObject} to preserve the insertion order.
+   * <p>NOTE: here we use {@link ObjectNode} to preserve the insertion order.
    */
-  public JsonObject toJsonObject() {
-    JsonObject jsonObject = new JsonObject();
-    jsonObject.addProperty("name", _name);
-    jsonObject.addProperty("dataType", _dataType.name());
+  public ObjectNode toJsonObject() {
+    ObjectNode jsonObject = JsonUtils.newObjectNode();
+    jsonObject.put("name", _name);
+    jsonObject.put("dataType", _dataType.name());
     if (!_isSingleValueField) {
-      jsonObject.addProperty("singleValueField", false);
+      jsonObject.put("singleValueField", false);
     }
     if (_maxLength != DEFAULT_MAX_LENGTH) {
-      jsonObject.addProperty("maxLength", _maxLength);
+      jsonObject.put("maxLength", _maxLength);
     }
     appendDefaultNullValue(jsonObject);
     return jsonObject;
   }
 
-  protected void appendDefaultNullValue(JsonObject jsonObject) {
+  protected void appendDefaultNullValue(ObjectNode jsonNode) {
     assert _defaultNullValue != null;
     if (!_defaultNullValue.equals(getDefaultNullValue(getFieldType(), _dataType, null))) {
       if (_defaultNullValue instanceof Number) {
-        jsonObject.add("defaultNullValue", new JsonPrimitive((Number) _defaultNullValue));
+        jsonNode.set("defaultNullValue", JsonUtils.objectToJsonNode(_defaultNullValue));
       } else {
-        jsonObject.addProperty("defaultNullValue", getStringValue(_defaultNullValue));
+        jsonNode.put("defaultNullValue", getStringValue(_defaultNullValue));
       }
     }
   }
 
-  public JsonObject toAvroSchemaJsonObject() {
-    JsonObject jsonSchema = new JsonObject();
-    jsonSchema.addProperty("name", _name);
+  public ObjectNode toAvroSchemaJsonObject() {
+    ObjectNode jsonSchema = JsonUtils.newObjectNode();
+    jsonSchema.put("name", _name);
     switch (_dataType) {
       case INT:
-        jsonSchema.add("type", convertStringsToJsonArray("null", "int"));
+        jsonSchema.set("type", convertStringsToJsonArray("null", "int"));
         return jsonSchema;
       case LONG:
-        jsonSchema.add("type", convertStringsToJsonArray("null", "long"));
+        jsonSchema.set("type", convertStringsToJsonArray("null", "long"));
         return jsonSchema;
       case FLOAT:
-        jsonSchema.add("type", convertStringsToJsonArray("null", "float"));
+        jsonSchema.set("type", convertStringsToJsonArray("null", "float"));
         return jsonSchema;
       case DOUBLE:
-        jsonSchema.add("type", convertStringsToJsonArray("null", "double"));
+        jsonSchema.set("type", convertStringsToJsonArray("null", "double"));
         return jsonSchema;
       case STRING:
-        jsonSchema.add("type", convertStringsToJsonArray("null", "string"));
+        jsonSchema.set("type", convertStringsToJsonArray("null", "string"));
         return jsonSchema;
       default:
         throw new UnsupportedOperationException();
     }
   }
 
-  private static JsonArray convertStringsToJsonArray(String... strings) {
-    JsonArray jsonArray = new JsonArray();
+  private static ArrayNode convertStringsToJsonArray(String... strings) {
+    ArrayNode jsonArray = JsonUtils.newArrayNode();
     for (String string : strings) {
-      jsonArray.add(new JsonPrimitive(string));
+      jsonArray.add(string);
     }
     return jsonArray;
   }
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/data/MetricFieldSpec.java b/pinot-common/src/main/java/com/linkedin/pinot/common/data/MetricFieldSpec.java
index 3e52a33..852ae01 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/data/MetricFieldSpec.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/data/MetricFieldSpec.java
@@ -18,13 +18,13 @@
  */
 package com.linkedin.pinot.common.data;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.google.common.base.Preconditions;
-import com.google.gson.JsonObject;
 import com.linkedin.pinot.common.config.ConfigKey;
 import com.linkedin.pinot.common.utils.EqualityUtils;
 import javax.annotation.Nonnull;
-import org.codehaus.jackson.annotate.JsonIgnore;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
 
 /**
@@ -141,13 +141,13 @@ public final class MetricFieldSpec extends FieldSpec {
 
   @Nonnull
   @Override
-  public JsonObject toJsonObject() {
-    JsonObject jsonObject = super.toJsonObject();
+  public ObjectNode toJsonObject() {
+    ObjectNode jsonObject = super.toJsonObject();
     if (_dataType == DataType.STRING && _fieldSize != UNDEFINED_METRIC_SIZE) {
-      jsonObject.addProperty("fieldSize", _fieldSize);
+      jsonObject.put("fieldSize", _fieldSize);
     }
     if (_derivedMetricType != null) {
-      jsonObject.addProperty("derivedMetricType", _derivedMetricType.name());
+      jsonObject.put("derivedMetricType", _derivedMetricType.name());
     }
     return jsonObject;
   }
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/data/Schema.java b/pinot-common/src/main/java/com/linkedin/pinot/common/data/Schema.java
index 9566eaa..ec38211 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/data/Schema.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/data/Schema.java
@@ -18,15 +18,18 @@
  */
 package com.linkedin.pinot.common.data;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.google.common.base.Preconditions;
-import com.google.gson.GsonBuilder;
-import com.google.gson.JsonArray;
-import com.google.gson.JsonObject;
 import com.linkedin.pinot.common.config.ConfigKey;
 import com.linkedin.pinot.common.config.UseChildKeyHandler;
 import com.linkedin.pinot.common.data.FieldSpec.DataType;
 import com.linkedin.pinot.common.data.FieldSpec.FieldType;
 import com.linkedin.pinot.common.utils.EqualityUtils;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
@@ -39,9 +42,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import javax.annotation.Nonnull;
-import org.codehaus.jackson.annotate.JsonIgnore;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.ObjectMapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -60,7 +60,6 @@ import org.slf4j.LoggerFactory;
 @JsonIgnoreProperties(ignoreUnknown = true)
 public final class Schema {
   private static final Logger LOGGER = LoggerFactory.getLogger(Schema.class);
-  private static final ObjectMapper MAPPER = new ObjectMapper();
 
   @ConfigKey("schemaName")
   private String _schemaName;
@@ -88,17 +87,17 @@ public final class Schema {
 
   @Nonnull
   public static Schema fromFile(@Nonnull File schemaFile) throws IOException {
-    return MAPPER.readValue(schemaFile, Schema.class);
+    return JsonUtils.fileToObject(schemaFile, Schema.class);
   }
 
   @Nonnull
   public static Schema fromString(@Nonnull String schemaString) throws IOException {
-    return MAPPER.readValue(schemaString, Schema.class);
+    return JsonUtils.stringToObject(schemaString, Schema.class);
   }
 
   @Nonnull
   public static Schema fromInputSteam(@Nonnull InputStream schemaInputStream) throws IOException {
-    return MAPPER.readValue(schemaInputStream, Schema.class);
+    return JsonUtils.inputStreamToObject(schemaInputStream, Schema.class);
   }
 
   /**
@@ -362,33 +361,37 @@ public final class Schema {
   @JsonIgnore
   @Nonnull
   public String getJSONSchema() {
-    JsonObject jsonSchema = new JsonObject();
-    jsonSchema.addProperty("schemaName", _schemaName);
+    ObjectNode jsonSchema = JsonUtils.newObjectNode();
+    jsonSchema.put("schemaName", _schemaName);
     if (!_dimensionFieldSpecs.isEmpty()) {
-      JsonArray jsonArray = new JsonArray();
+      ArrayNode jsonArray = JsonUtils.newArrayNode();
       for (DimensionFieldSpec dimensionFieldSpec : _dimensionFieldSpecs) {
         jsonArray.add(dimensionFieldSpec.toJsonObject());
       }
-      jsonSchema.add("dimensionFieldSpecs", jsonArray);
+      jsonSchema.set("dimensionFieldSpecs", jsonArray);
     }
     if (!_metricFieldSpecs.isEmpty()) {
-      JsonArray jsonArray = new JsonArray();
+      ArrayNode jsonArray = JsonUtils.newArrayNode();
       for (MetricFieldSpec metricFieldSpec : _metricFieldSpecs) {
         jsonArray.add(metricFieldSpec.toJsonObject());
       }
-      jsonSchema.add("metricFieldSpecs", jsonArray);
+      jsonSchema.set("metricFieldSpecs", jsonArray);
     }
     if (_timeFieldSpec != null) {
-      jsonSchema.add("timeFieldSpec", _timeFieldSpec.toJsonObject());
+      jsonSchema.set("timeFieldSpec", _timeFieldSpec.toJsonObject());
     }
     if (!_dateTimeFieldSpecs.isEmpty()) {
-      JsonArray jsonArray = new JsonArray();
+      ArrayNode jsonArray = JsonUtils.newArrayNode();
       for (DateTimeFieldSpec dateTimeFieldSpec : _dateTimeFieldSpecs) {
         jsonArray.add(dateTimeFieldSpec.toJsonObject());
       }
-      jsonSchema.add("dateTimeFieldSpecs", jsonArray);
+      jsonSchema.set("dateTimeFieldSpecs", jsonArray);
+    }
+    try {
+      return JsonUtils.objectToPrettyString(jsonSchema);
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
     }
-    return new GsonBuilder().setPrettyPrinting().create().toJson(jsonSchema);
   }
 
   /**
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/data/StarTreeIndexSpec.java b/pinot-common/src/main/java/com/linkedin/pinot/common/data/StarTreeIndexSpec.java
index 15b1b9e..92cfd3b 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/data/StarTreeIndexSpec.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/data/StarTreeIndexSpec.java
@@ -18,18 +18,19 @@
  */
 package com.linkedin.pinot.common.data;
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.core.JsonProcessingException;
 import com.google.common.collect.Sets;
+import com.linkedin.pinot.common.config.ConfigKey;
 import com.linkedin.pinot.common.segment.StarTreeMetadata;
 import com.linkedin.pinot.common.utils.EqualityUtils;
-import com.linkedin.pinot.common.config.ConfigKey;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.io.File;
 import java.io.IOException;
 import java.util.List;
 import java.util.Set;
 import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.apache.commons.lang3.builder.ToStringStyle;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.ObjectMapper;
 
 
 @SuppressWarnings("unused")
@@ -38,8 +39,6 @@ public class StarTreeIndexSpec {
   public static final int DEFAULT_MAX_LEAF_RECORDS = 100000; // TODO: determine a good number via experiment
   public static final int DEFAULT_SKIP_MATERIALIZATION_CARDINALITY_THRESHOLD = 10000;
 
-  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
-
   /** The upper bound on the number of leaf records to be scanned for any query */
   @ConfigKey("maxLeafRecords")
   private int _maxLeafRecords = DEFAULT_MAX_LEAF_RECORDS;
@@ -115,8 +114,8 @@ public class StarTreeIndexSpec {
     return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
   }
 
-  public String toJsonString() throws Exception {
-    return OBJECT_MAPPER.writerWithDefaultPrettyPrinter().writeValueAsString(this);
+  public String toJsonString() throws JsonProcessingException {
+    return JsonUtils.objectToString(this);
   }
 
   /**
@@ -127,11 +126,11 @@ public class StarTreeIndexSpec {
    * @throws IOException
    */
   public static StarTreeIndexSpec fromFile(File starTreeIndexSpecFile) throws IOException {
-    return OBJECT_MAPPER.readValue(starTreeIndexSpecFile, StarTreeIndexSpec.class);
+    return JsonUtils.fileToObject(starTreeIndexSpecFile, StarTreeIndexSpec.class);
   }
 
   public static StarTreeIndexSpec fromJsonString(String jsonString) throws IOException {
-    return OBJECT_MAPPER.readValue(jsonString, StarTreeIndexSpec.class);
+    return JsonUtils.stringToObject(jsonString, StarTreeIndexSpec.class);
   }
 
   public static StarTreeIndexSpec fromStarTreeMetadata(StarTreeMetadata starTreeMetadata) {
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/data/TimeFieldSpec.java b/pinot-common/src/main/java/com/linkedin/pinot/common/data/TimeFieldSpec.java
index adbe577..823247d 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/data/TimeFieldSpec.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/data/TimeFieldSpec.java
@@ -18,14 +18,15 @@
  */
 package com.linkedin.pinot.common.data;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.google.common.base.Preconditions;
-import com.google.gson.JsonObject;
 import com.linkedin.pinot.common.config.ConfigKey;
 import com.linkedin.pinot.common.utils.EqualityUtils;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.util.concurrent.TimeUnit;
 import javax.annotation.Nonnull;
-import org.codehaus.jackson.annotate.JsonIgnore;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
 
 @SuppressWarnings("unused")
@@ -205,11 +206,11 @@ public final class TimeFieldSpec extends FieldSpec {
 
   @Nonnull
   @Override
-  public JsonObject toJsonObject() {
-    JsonObject jsonObject = new JsonObject();
-    jsonObject.add("incomingGranularitySpec", _incomingGranularitySpec.toJsonObject());
+  public ObjectNode toJsonObject() {
+    ObjectNode jsonObject = JsonUtils.newObjectNode();
+    jsonObject.set("incomingGranularitySpec", _incomingGranularitySpec.toJsonObject());
     if (!getOutgoingGranularitySpec().equals(_incomingGranularitySpec)) {
-      jsonObject.add("outgoingGranularitySpec", _outgoingGranularitySpec.toJsonObject());
+      jsonObject.set("outgoingGranularitySpec", _outgoingGranularitySpec.toJsonObject());
     }
     appendDefaultNullValue(jsonObject);
     return jsonObject;
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/data/TimeGranularitySpec.java b/pinot-common/src/main/java/com/linkedin/pinot/common/data/TimeGranularitySpec.java
index 8ec74cb..32791de 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/data/TimeGranularitySpec.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/data/TimeGranularitySpec.java
@@ -18,14 +18,15 @@
  */
 package com.linkedin.pinot.common.data;
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.google.common.base.Preconditions;
-import com.google.gson.JsonObject;
 import com.linkedin.pinot.common.config.ConfigKey;
 import com.linkedin.pinot.common.data.FieldSpec.DataType;
 import com.linkedin.pinot.common.utils.EqualityUtils;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.util.concurrent.TimeUnit;
 import javax.annotation.Nonnull;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.joda.time.DateTime;
 import org.joda.time.format.DateTimeFormat;
 
@@ -249,20 +250,20 @@ public class TimeGranularitySpec {
   }
 
   /**
-   * Returns the {@link JsonObject} representing the time granularity spec.
+   * Returns the {@link ObjectNode} representing the time granularity spec.
    * <p>Only contains fields with non-default value.
-   * <p>NOTE: here we use {@link JsonObject} to preserve the insertion order.
+   * <p>NOTE: here we use {@link ObjectNode} to preserve the insertion order.
    */
-  public JsonObject toJsonObject() {
-    JsonObject jsonObject = new JsonObject();
-    jsonObject.addProperty("name", _name);
-    jsonObject.addProperty("dataType", _dataType.name());
-    jsonObject.addProperty("timeType", _timeType.name());
+  public ObjectNode toJsonObject() {
+    ObjectNode jsonObject = JsonUtils.newObjectNode();
+    jsonObject.put("name", _name);
+    jsonObject.put("dataType", _dataType.name());
+    jsonObject.put("timeType", _timeType.name());
     if (_timeUnitSize != DEFAULT_TIME_UNIT_SIZE) {
-      jsonObject.addProperty("timeUnitSize", _timeUnitSize);
+      jsonObject.put("timeUnitSize", _timeUnitSize);
     }
     if (!_timeFormat.equals(DEFAULT_TIME_FORMAT)) {
-      jsonObject.addProperty("timeFormat", _timeFormat);
+      jsonObject.put("timeFormat", _timeFormat);
     }
     return jsonObject;
   }
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/metadata/segment/ColumnPartitionMetadata.java b/pinot-common/src/main/java/com/linkedin/pinot/common/metadata/segment/ColumnPartitionMetadata.java
index b77e29a..12cfc14 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/metadata/segment/ColumnPartitionMetadata.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/metadata/segment/ColumnPartitionMetadata.java
@@ -18,22 +18,22 @@
  */
 package com.linkedin.pinot.common.metadata.segment;
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.JsonDeserializer;
+import com.fasterxml.jackson.databind.JsonSerializer;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
 import com.linkedin.pinot.common.config.ColumnPartitionConfig;
 import com.linkedin.pinot.common.utils.EqualityUtils;
 import java.io.IOException;
 import java.util.List;
 import javax.annotation.Nonnull;
 import org.apache.commons.lang.math.IntRange;
-import org.codehaus.jackson.JsonGenerator;
-import org.codehaus.jackson.JsonParser;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.codehaus.jackson.map.DeserializationContext;
-import org.codehaus.jackson.map.JsonDeserializer;
-import org.codehaus.jackson.map.JsonSerializer;
-import org.codehaus.jackson.map.SerializerProvider;
-import org.codehaus.jackson.map.annotate.JsonDeserialize;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 
 /**
@@ -43,7 +43,6 @@ import org.codehaus.jackson.map.annotate.JsonSerialize;
  *   <li> Number of partitions. </li>
  *   <li> List of partition ranges. </li>
  * </ul>
-
  */
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class ColumnPartitionMetadata extends ColumnPartitionConfig {
@@ -113,8 +112,7 @@ public class ColumnPartitionMetadata extends ColumnPartitionConfig {
   public static class PartitionRangesDeserializer extends JsonDeserializer<List<IntRange>> {
 
     @Override
-    public List<IntRange> deserialize(JsonParser jsonParser, DeserializationContext context)
-        throws IOException {
+    public List<IntRange> deserialize(JsonParser jsonParser, DeserializationContext context) throws IOException {
       return ColumnPartitionConfig.rangesFromString(jsonParser.getText());
     }
   }
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/metadata/segment/SegmentPartitionMetadata.java b/pinot-common/src/main/java/com/linkedin/pinot/common/metadata/segment/SegmentPartitionMetadata.java
index 86eb506..4fe59a6 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/metadata/segment/SegmentPartitionMetadata.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/metadata/segment/SegmentPartitionMetadata.java
@@ -34,17 +34,17 @@ package com.linkedin.pinot.common.metadata.segment;
  * limitations under the License.
  */
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Preconditions;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 import org.apache.commons.lang.math.IntRange;
-import org.codehaus.jackson.annotate.JsonIgnore;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.codehaus.jackson.map.ObjectMapper;
 
 
 /**
@@ -53,8 +53,6 @@ import org.codehaus.jackson.map.ObjectMapper;
 @SuppressWarnings("unused") // Suppress incorrect warning, as methods are used for json ser/de.
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class SegmentPartitionMetadata {
-  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
-
   public static final int INVALID_NUM_PARTITIONS = -1;
   private final Map<String, ColumnPartitionMetadata> _columnPartitionMap;
 
@@ -109,9 +107,8 @@ public class SegmentPartitionMetadata {
    * @return Instance of {@link SegmentPartitionMetadata} built from the input string.
    * @throws IOException
    */
-  public static SegmentPartitionMetadata fromJsonString(String jsonString)
-      throws IOException {
-    return OBJECT_MAPPER.readValue(jsonString, SegmentPartitionMetadata.class);
+  public static SegmentPartitionMetadata fromJsonString(String jsonString) throws IOException {
+    return JsonUtils.stringToObject(jsonString, SegmentPartitionMetadata.class);
   }
 
   /**
@@ -120,9 +117,8 @@ public class SegmentPartitionMetadata {
    * @return JSON string equivalent of the object.
    * @throws IOException
    */
-  public String toJsonString()
-      throws IOException {
-    return OBJECT_MAPPER.writeValueAsString(this);
+  public String toJsonString() throws IOException {
+    return JsonUtils.objectToString(this);
   }
 
   /**
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/metadata/segment/SegmentZKMetadata.java b/pinot-common/src/main/java/com/linkedin/pinot/common/metadata/segment/SegmentZKMetadata.java
index 08d0c03..8de8070 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/metadata/segment/SegmentZKMetadata.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/metadata/segment/SegmentZKMetadata.java
@@ -18,9 +18,11 @@
  */
 package com.linkedin.pinot.common.metadata.segment;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
 import com.linkedin.pinot.common.metadata.ZKMetadata;
 import com.linkedin.pinot.common.utils.CommonConstants;
 import com.linkedin.pinot.common.utils.CommonConstants.Segment.SegmentType;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
@@ -29,7 +31,6 @@ import javax.annotation.Nonnull;
 import org.apache.helix.ZNRecord;
 import org.joda.time.Duration;
 import org.joda.time.Interval;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -336,8 +337,11 @@ public abstract class SegmentZKMetadata implements ZKMetadata {
     if (_customMap == null) {
       configMap.put(CommonConstants.Segment.CUSTOM_MAP, null);
     } else {
-      JSONObject jsonObject = new JSONObject(_customMap);
-      configMap.put(CommonConstants.Segment.CUSTOM_MAP, jsonObject.toString());
+      try {
+        configMap.put(CommonConstants.Segment.CUSTOM_MAP, JsonUtils.objectToString(_customMap));
+      } catch (JsonProcessingException e) {
+        throw new RuntimeException(e);
+      }
     }
 
     return configMap;
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/metadata/segment/SegmentZKMetadataCustomMapModifier.java b/pinot-common/src/main/java/com/linkedin/pinot/common/metadata/segment/SegmentZKMetadataCustomMapModifier.java
index 84b8c0c..a8714e4 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/metadata/segment/SegmentZKMetadataCustomMapModifier.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/metadata/segment/SegmentZKMetadataCustomMapModifier.java
@@ -18,13 +18,15 @@
  */
 package com.linkedin.pinot.common.metadata.segment;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.linkedin.pinot.common.utils.JsonUtils;
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
-import org.json.JSONException;
-import org.json.JSONObject;
 
 
 /**
@@ -52,28 +54,27 @@ public class SegmentZKMetadataCustomMapModifier {
     }
   }
 
-  public SegmentZKMetadataCustomMapModifier(@Nonnull String jsonString) throws JSONException {
-    JSONObject jsonObject = new JSONObject(jsonString);
-    _modifyMode = ModifyMode.valueOf(jsonObject.getString(MAP_MODIFY_MODE_KEY));
-    JSONObject jsonMap = jsonObject.getJSONObject(MAP_KEY);
-    if (jsonMap == null || jsonMap.length() == 0) {
+  public SegmentZKMetadataCustomMapModifier(@Nonnull String jsonString) throws IOException {
+    JsonNode jsonNode = JsonUtils.stringToJsonNode(jsonString);
+    _modifyMode = ModifyMode.valueOf(jsonNode.get(MAP_MODIFY_MODE_KEY).asText());
+    JsonNode jsonMap = jsonNode.get(MAP_KEY);
+    if (jsonMap == null || jsonMap.size() == 0) {
       _map = null;
     } else {
       _map = new HashMap<>();
-      @SuppressWarnings("unchecked")
-      Iterator<String> keys = jsonMap.keys();
+      Iterator<String> keys = jsonMap.fieldNames();
       while (keys.hasNext()) {
         String key = keys.next();
-        _map.put(key, jsonMap.getString(key));
+        _map.put(key, jsonMap.get(key).asText());
       }
     }
   }
 
-  public String toJsonString() throws JSONException {
-    JSONObject jsonObject = new JSONObject();
-    jsonObject.put(MAP_MODIFY_MODE_KEY, _modifyMode);
-    jsonObject.put(MAP_KEY, _map);
-    return jsonObject.toString();
+  public String toJsonString() {
+    ObjectNode objectNode = JsonUtils.newObjectNode();
+    objectNode.put(MAP_MODIFY_MODE_KEY, _modifyMode.toString());
+    objectNode.set(MAP_KEY, JsonUtils.objectToJsonNode(_map));
+    return objectNode.toString();
   }
 
   public Map<String, String> modifyMap(@Nullable Map<String, String> existingMap) {
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/protocols/SegmentCompletionProtocol.java b/pinot-common/src/main/java/com/linkedin/pinot/common/protocols/SegmentCompletionProtocol.java
index c6acf52..350f7fc 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/protocols/SegmentCompletionProtocol.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/protocols/SegmentCompletionProtocol.java
@@ -18,8 +18,13 @@
  */
 package com.linkedin.pinot.common.protocols;
 
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.linkedin.pinot.common.utils.JsonUtils;
+import java.io.IOException;
 import java.util.concurrent.TimeUnit;
-import com.alibaba.fastjson.JSONObject;
+
 
 /*
  * This class encapsulates the segment completion protocol used by the server and the controller for
@@ -365,113 +370,116 @@ public class SegmentCompletionProtocol {
   }
 
   public static class Response {
-    final ControllerResponseStatus _status;
-    final long _offset;
-    final long _buildTimeSeconds;
-    final boolean _isSplitCommit;
-    final String _segmentLocation;
-    final String _controllerVipUrl;
-
-    public Response(String jsonRespStr) {
-      JSONObject jsonObject = JSONObject.parseObject(jsonRespStr);
-      long offset = -1;
-      if (jsonObject.containsKey(OFFSET_KEY)) {
-        offset = jsonObject.getLong(OFFSET_KEY);
-      }
-      _offset = offset;
-
-      String statusStr = jsonObject.getString(STATUS_KEY);
-      ControllerResponseStatus status;
-      try {
-        status = ControllerResponseStatus.valueOf(statusStr);
-      } catch (Exception e) {
-        status = ControllerResponseStatus.FAILED;
-      }
-      _status = status;
-
-      Long buildTimeObj = jsonObject.getLong(BUILD_TIME_KEY);
-      if (buildTimeObj == null) {
-        _buildTimeSeconds = -1;
-      } else {
-        _buildTimeSeconds = buildTimeObj;
-      }
-
-      boolean isSplitCommit = false;
-      if (jsonObject.containsKey(COMMIT_TYPE_KEY) && jsonObject.getBoolean(COMMIT_TYPE_KEY)) {
-        isSplitCommit = true;
-      }
-      _isSplitCommit = isSplitCommit;
-
-      String segmentLocation = null;
-      if (jsonObject.containsKey(SEGMENT_LOCATION_KEY)) {
-        segmentLocation = jsonObject.getString(SEGMENT_LOCATION_KEY);
-      }
-      _segmentLocation = segmentLocation;
-
-      String controllerVipUrl= null;
-      if (jsonObject.containsKey(CONTROLLER_VIP_URL_KEY)) {
-        controllerVipUrl = jsonObject.getString(CONTROLLER_VIP_URL_KEY);
-      }
-      _controllerVipUrl = controllerVipUrl;
+    private ControllerResponseStatus _status;
+    private long _offset = -1;
+    private long _buildTimeSeconds = -1;
+    private boolean _splitCommit;
+    private String _segmentLocation;
+    private String _controllerVipUrl;
+
+    public Response() {
     }
 
     public Response(Params params) {
       _status = params.getStatus();
       _offset = params.getOffset();
       _buildTimeSeconds = params.getBuildTimeSeconds();
-      _isSplitCommit = params.getIsSplitCommit();
+      _splitCommit = params.isSplitCommit();
       _segmentLocation = params.getSegmentLocation();
       _controllerVipUrl = params.getControllerVipUrl();
     }
 
+    @JsonProperty(STATUS_KEY)
     public ControllerResponseStatus getStatus() {
       return _status;
     }
 
+    @JsonProperty(STATUS_KEY)
+    public void setStatus(ControllerResponseStatus status) {
+      _status = status;
+    }
+
+    @JsonProperty(OFFSET_KEY)
     public long getOffset() {
       return _offset;
     }
 
+    @JsonProperty(OFFSET_KEY)
+    public void setOffset(long offset) {
+      _offset = offset;
+    }
+
+    @JsonProperty(BUILD_TIME_KEY)
     public long getBuildTimeSeconds() {
       return _buildTimeSeconds;
     }
 
-    public boolean getIsSplitCommit() {
-      return _isSplitCommit;
+    @JsonProperty(BUILD_TIME_KEY)
+    public void setBuildTimeSeconds(long buildTimeSeconds) {
+      _buildTimeSeconds = buildTimeSeconds;
+    }
+
+    @JsonProperty(COMMIT_TYPE_KEY)
+    public boolean isSplitCommit() {
+      return _splitCommit;
     }
 
+    @JsonProperty(COMMIT_TYPE_KEY)
+    public void setSplitCommit(boolean splitCommit) {
+      _splitCommit = splitCommit;
+    }
+
+    @JsonProperty(CONTROLLER_VIP_URL_KEY)
+    @JsonInclude(JsonInclude.Include.NON_NULL)
     public String getControllerVipUrl() {
       return _controllerVipUrl;
     }
 
+    @JsonProperty(CONTROLLER_VIP_URL_KEY)
+    public void setControllerVipUrl(String controllerVipUrl) {
+      _controllerVipUrl = controllerVipUrl;
+    }
+
+    @JsonProperty(SEGMENT_LOCATION_KEY)
+    @JsonInclude(JsonInclude.Include.NON_NULL)
     public String getSegmentLocation() {
       return _segmentLocation;
     }
 
+    @JsonProperty(SEGMENT_LOCATION_KEY)
+    public void setSegmentLocation(String segmentLocation) {
+      _segmentLocation = segmentLocation;
+    }
+
     public String toJsonString() {
-      StringBuilder builder = new StringBuilder();
-      builder.append("{\"" + STATUS_KEY + "\":" + "\"" + _status.name() + "\"," + "\""
-          + OFFSET_KEY + "\":" + _offset + ",\""
-          + COMMIT_TYPE_KEY + "\":" + _isSplitCommit
-          + (_segmentLocation != null ? ",\"" + SEGMENT_LOCATION_KEY + "\":\"" + _segmentLocation + "\"" : "")
-          + (_controllerVipUrl != null ? "," + "\"" + CONTROLLER_VIP_URL_KEY + "\":\"" + _controllerVipUrl + "\"" : ""));
-      builder.append("}");
-      return builder.toString();
+      try {
+        return JsonUtils.objectToString(this);
+      } catch (JsonProcessingException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    public static Response fromJsonString(String jsonString) {
+      try {
+        return JsonUtils.stringToObject(jsonString, Response.class);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
     }
 
     public static class Params {
       private ControllerResponseStatus _status;
       private long _offset;
-      private long _buildTimeSec;
-      private boolean _isSplitCommit;
+      private long _buildTimeSeconds;
+      private boolean _splitCommit;
       private String _segmentLocation;
       private String _controllerVipUrl;
 
       public Params() {
         _offset = -1L;
         _status = ControllerResponseStatus.FAILED;
-        _buildTimeSec = -1;
-        _isSplitCommit = false;
+        _buildTimeSeconds = -1;
+        _splitCommit = false;
         _segmentLocation = null;
         _controllerVipUrl = null;
       }
@@ -487,12 +495,12 @@ public class SegmentCompletionProtocol {
       }
 
       public Params withBuildTimeSeconds(long buildTimeSeconds) {
-        _buildTimeSec = buildTimeSeconds;
+        _buildTimeSeconds = buildTimeSeconds;
         return this;
       }
 
-      public Params withSplitCommit(boolean isSplitCommit) {
-        _isSplitCommit = isSplitCommit;
+      public Params withSplitCommit(boolean splitCommit) {
+        _splitCommit = splitCommit;
         return this;
       }
 
@@ -513,10 +521,10 @@ public class SegmentCompletionProtocol {
         return _offset;
       }
       public long getBuildTimeSeconds() {
-        return _buildTimeSec;
+        return _buildTimeSeconds;
       }
-      public boolean getIsSplitCommit() {
-        return _isSplitCommit;
+      public boolean isSplitCommit() {
+        return _splitCommit;
       }
       public String getSegmentLocation() {
         return _segmentLocation;
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/response/BrokerResponse.java b/pinot-common/src/main/java/com/linkedin/pinot/common/response/BrokerResponse.java
index 53319ac..34e473e 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/response/BrokerResponse.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/response/BrokerResponse.java
@@ -20,7 +20,6 @@ package com.linkedin.pinot.common.response;
 
 import com.linkedin.pinot.common.response.broker.QueryProcessingException;
 import java.util.List;
-import org.json.JSONObject;
 
 
 /**
@@ -53,16 +52,9 @@ public interface BrokerResponse {
   void setTimeUsedMs(long timeUsedMs);
 
   /**
-   * Convert the broker response to JSONObject.
-   */
-  JSONObject toJson()
-      throws Exception;
-
-  /**
    * Convert the broker response to JSON String.
    */
-  String toJsonString()
-      throws Exception;
+  String toJsonString() throws Exception;
 
   /**
    * Returns the number of servers queried.
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/AggregationResult.java b/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/AggregationResult.java
index f3edf92..2bcff48 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/AggregationResult.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/AggregationResult.java
@@ -18,11 +18,11 @@
  */
 package com.linkedin.pinot.common.response.broker;
 
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonPropertyOrder;
 import java.io.Serializable;
 import java.util.List;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.codehaus.jackson.annotate.JsonPropertyOrder;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
 
 
 /**
@@ -72,7 +72,7 @@ public class AggregationResult {
    * @return
    */
   @JsonProperty("function")
-  @JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+  @JsonInclude(JsonInclude.Include.NON_NULL)
   public String getFunction() {
     return _function;
   }
@@ -91,7 +91,7 @@ public class AggregationResult {
    * @return
    */
   @JsonProperty("value")
-  @JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+  @JsonInclude(JsonInclude.Include.NON_NULL)
   public Serializable getValue() {
     return _value;
   }
@@ -110,7 +110,7 @@ public class AggregationResult {
    * @return
    */
   @JsonProperty("groupByResult")
-  @JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+  @JsonInclude(JsonInclude.Include.NON_NULL)
   public List<GroupByResult> getGroupByResult() {
     return _groupByResults;
   }
@@ -129,7 +129,7 @@ public class AggregationResult {
    * @return
    */
   @JsonProperty("groupByColumns")
-  @JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+  @JsonInclude(JsonInclude.Include.NON_NULL)
   public List<String> getGroupByColumns() {
     return _groupByColumns;
   }
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/BrokerResponseNative.java b/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/BrokerResponseNative.java
index c358aeb..fadc945 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/BrokerResponseNative.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/BrokerResponseNative.java
@@ -18,22 +18,19 @@
  */
 package com.linkedin.pinot.common.response.broker;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonPropertyOrder;
 import com.linkedin.pinot.common.exception.QueryException;
 import com.linkedin.pinot.common.response.BrokerResponse;
 import com.linkedin.pinot.common.response.ProcessingException;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import org.codehaus.jackson.annotate.JsonIgnore;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.codehaus.jackson.annotate.JsonPropertyOrder;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.map.annotate.JsonSerialize;
-import org.codehaus.jackson.type.TypeReference;
-import org.json.JSONException;
-import org.json.JSONObject;
 
 
 /**
@@ -42,12 +39,11 @@ import org.json.JSONObject;
  *
  * Supports serialization via JSON.
  */
-@JsonPropertyOrder({ "selectionResults", "aggregationResults", "exceptions", "numServersQueried", "numServersResponded", "numSegmentsQueried",
-    "numSegmentsProcessed", "numSegmentsMatched", "numDocsScanned", "numEntriesScannedInFilter", "numEntriesScannedPostFilter", "numGroupsLimitReached",
-    "totalDocs", "timeUsedMs", "segmentStatistics", "traceInfo" })
+@JsonPropertyOrder({"selectionResults", "aggregationResults", "exceptions", "numServersQueried", "numServersResponded",
+    "numSegmentsQueried", "numSegmentsProcessed", "numSegmentsMatched", "numDocsScanned", "numEntriesScannedInFilter",
+    "numEntriesScannedPostFilter", "numGroupsLimitReached", "totalDocs", "timeUsedMs", "segmentStatistics",
+    "traceInfo"})
 public class BrokerResponseNative implements BrokerResponse {
-  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
-
   public static final BrokerResponseNative EMPTY_RESULT = BrokerResponseNative.empty();
   public static final BrokerResponseNative NO_TABLE_RESULT =
       new BrokerResponseNative(QueryException.BROKER_RESOURCE_MISSING_ERROR);
@@ -60,7 +56,7 @@ public class BrokerResponseNative implements BrokerResponse {
   private long _numSegmentsQueried = 0L;
   private long _numSegmentsProcessed = 0L;
   private long _numSegmentsMatched = 0L;
-  
+
   private long _totalDocs = 0L;
   private boolean _numGroupsLimitReached = false;
   private long _timeUsedMs = 0L;
@@ -93,7 +89,7 @@ public class BrokerResponseNative implements BrokerResponse {
   }
 
   @JsonProperty("selectionResults")
-  @JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+  @JsonInclude(JsonInclude.Include.NON_NULL)
   public SelectionResults getSelectionResults() {
     return _selectionResults;
   }
@@ -104,7 +100,7 @@ public class BrokerResponseNative implements BrokerResponse {
   }
 
   @JsonProperty("aggregationResults")
-  @JsonSerialize(include = JsonSerialize.Inclusion.NON_NULL)
+  @JsonInclude(JsonInclude.Include.NON_NULL)
   public List<AggregationResult> getAggregationResults() {
     return _aggregationResults;
   }
@@ -265,21 +261,11 @@ public class BrokerResponseNative implements BrokerResponse {
 
   @Override
   public String toJsonString() throws IOException {
-    return OBJECT_MAPPER.writeValueAsString(this);
-  }
-
-  @Override
-  public JSONObject toJson() throws IOException, JSONException {
-    return new JSONObject(toJsonString());
+    return JsonUtils.objectToString(this);
   }
 
   public static BrokerResponseNative fromJsonString(String jsonString) throws IOException {
-    return OBJECT_MAPPER.readValue(jsonString, new TypeReference<BrokerResponseNative>() {
-    });
-  }
-
-  public static BrokerResponseNative fromJsonObject(JSONObject jsonObject) throws IOException {
-    return fromJsonString(jsonObject.toString());
+    return JsonUtils.stringToObject(jsonString, BrokerResponseNative.class);
   }
 
   @JsonIgnore
@@ -299,6 +285,4 @@ public class BrokerResponseNative implements BrokerResponse {
   public int getExceptionsSize() {
     return _processingExceptions.size();
   }
-
-
 }
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/GroupByResult.java b/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/GroupByResult.java
index 69dbfe9..6a12eed 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/GroupByResult.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/GroupByResult.java
@@ -18,10 +18,10 @@
  */
 package com.linkedin.pinot.common.response.broker;
 
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonPropertyOrder;
 import java.io.Serializable;
 import java.util.List;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.codehaus.jackson.annotate.JsonPropertyOrder;
 
 
 @JsonPropertyOrder({"value", "group"})
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/QueryProcessingException.java b/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/QueryProcessingException.java
index c892ec5..3d35a62 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/QueryProcessingException.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/QueryProcessingException.java
@@ -18,7 +18,7 @@
  */
 package com.linkedin.pinot.common.response.broker;
 
-import org.codehaus.jackson.annotate.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonProperty;
 
 
 /**
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/SelectionResults.java b/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/SelectionResults.java
index 1915445..8c6baf5 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/SelectionResults.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/response/broker/SelectionResults.java
@@ -18,11 +18,11 @@
  */
 package com.linkedin.pinot.common.response.broker;
 
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonPropertyOrder;
 import java.io.Serializable;
 import java.util.List;
-import org.codehaus.jackson.annotate.JsonCreator;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.codehaus.jackson.annotate.JsonPropertyOrder;
 
 
 @JsonPropertyOrder({"columns", "results"})
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/RebalanceResult.java b/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/RebalanceResult.java
index 5378ac0..35894fe 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/RebalanceResult.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/RebalanceResult.java
@@ -19,9 +19,9 @@
 package com.linkedin.pinot.common.restlet.resources;
 
 import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
 import com.linkedin.pinot.common.partition.PartitionAssignment;
 import java.util.Map;
-import org.codehaus.jackson.annotate.JsonProperty;
 
 @JsonIgnoreProperties(ignoreUnknown = true)
 public class RebalanceResult {
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/ResourceUtils.java b/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/ResourceUtils.java
index 00debc9..bf21347 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/ResourceUtils.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/ResourceUtils.java
@@ -18,10 +18,10 @@
  */
 package com.linkedin.pinot.common.restlet.resources;
 
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.io.IOException;
 import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.Response;
-import org.codehaus.jackson.map.ObjectMapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -36,9 +36,8 @@ public class ResourceUtils {
   }
 
   public static String convertToJsonString(Object object) {
-    ObjectMapper mapper = new ObjectMapper();
     try {
-      return mapper.writerWithDefaultPrettyPrinter().writeValueAsString(object);
+      return JsonUtils.objectToPrettyString(object);
     } catch (IOException e) {
       LOGGER.error("Failed to convert json into string: ", e);
       throw new WebApplicationException("Failed to convert json into string.", Response.Status.INTERNAL_SERVER_ERROR);
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/SegmentSizeInfo.java b/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/SegmentSizeInfo.java
index 6b11dd6..90bda37 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/SegmentSizeInfo.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/SegmentSizeInfo.java
@@ -18,7 +18,7 @@
  */
 package com.linkedin.pinot.common.restlet.resources;
 
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 
 
 @JsonIgnoreProperties(ignoreUnknown = true)
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/TableSizeInfo.java b/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/TableSizeInfo.java
index 90fba72..1e8753b 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/TableSizeInfo.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/TableSizeInfo.java
@@ -18,9 +18,9 @@
  */
 package com.linkedin.pinot.common.restlet.resources;
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import java.util.ArrayList;
 import java.util.List;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 
 
 @JsonIgnoreProperties(ignoreUnknown = true)
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/TablesList.java b/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/TablesList.java
index 9b36494..bcf990b 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/TablesList.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/restlet/resources/TablesList.java
@@ -18,21 +18,18 @@
  */
 package com.linkedin.pinot.common.restlet.resources;
 
+import com.fasterxml.jackson.annotation.JsonProperty;
 import java.util.List;
-import org.codehaus.jackson.annotate.JsonProperty;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 
 public class TablesList {
-  private static final Logger LOGGER = LoggerFactory.getLogger(TablesList.class);
-  List<String> tables;
+  private List<String> _tables;
 
   public TablesList(@JsonProperty("tables") List<String> tables) {
-    this.tables = tables;
+    _tables = tables;
   }
 
   public List<String> getTables() {
-    return tables;
+    return _tables;
   }
 }
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/utils/FileUploadDownloadClient.java b/pinot-common/src/main/java/com/linkedin/pinot/common/utils/FileUploadDownloadClient.java
index b037835..d1e3351 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/utils/FileUploadDownloadClient.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/utils/FileUploadDownloadClient.java
@@ -55,7 +55,6 @@ import org.apache.http.entity.mime.content.InputStreamBody;
 import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.impl.client.HttpClients;
 import org.apache.http.util.EntityUtils;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -157,7 +156,7 @@ public class FileUploadDownloadClient implements Closeable {
   public static URI getOldUploadSegmentHttpsURI(String host, int port) throws URISyntaxException {
     return getURI(HTTPS, host, port, OLD_SEGMENT_PATH);
   }
-  
+
   public static URI getUploadSegmentHttpURI(String host, int port) throws URISyntaxException {
     return getURI(HTTP, host, port, SEGMENT_PATH);
   }
@@ -316,7 +315,7 @@ public class FileUploadDownloadClient implements Closeable {
     StatusLine statusLine = response.getStatusLine();
     String reason;
     try {
-      reason = new JSONObject(EntityUtils.toString(response.getEntity())).getString("error");
+      reason = JsonUtils.stringToJsonNode(EntityUtils.toString(response.getEntity())).get("error").asText();
     } catch (Exception e) {
       reason = "Failed to get reason";
     }
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/utils/JsonUtils.java b/pinot-common/src/main/java/com/linkedin/pinot/common/utils/JsonUtils.java
new file mode 100644
index 0000000..c23beae
--- /dev/null
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/utils/JsonUtils.java
@@ -0,0 +1,156 @@
+/**
+ * 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 com.linkedin.pinot.common.utils;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.ObjectWriter;
+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.linkedin.pinot.common.data.FieldSpec;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import javax.annotation.Nullable;
+
+
+public class JsonUtils {
+  private JsonUtils() {
+  }
+
+  public static final ObjectMapper DEFAULT_MAPPER = new ObjectMapper();
+  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();
+
+  public static <T> T stringToObject(String jsonString, Class<T> valueType) throws IOException {
+    return DEFAULT_MAPPER.readValue(jsonString, valueType);
+  }
+
+  public static JsonNode stringToJsonNode(String jsonString) throws IOException {
+    return DEFAULT_READER.readTree(jsonString);
+  }
+
+  public static <T> T fileToObject(File jsonFile, Class<T> valueType) throws IOException {
+    return DEFAULT_MAPPER.readValue(jsonFile, valueType);
+  }
+
+  public static JsonNode fileToJsonNode(File jsonFile) throws IOException {
+    try (InputStream inputStream = new FileInputStream(jsonFile)) {
+      return inputStreamToJsonNode(inputStream);
+    }
+  }
+
+  public static <T> T inputStreamToObject(InputStream jsonInputStream, Class<T> valueType) throws IOException {
+    return DEFAULT_MAPPER.readValue(jsonInputStream, valueType);
+  }
+
+  public static JsonNode inputStreamToJsonNode(InputStream jsonInputStream) throws IOException {
+    return DEFAULT_READER.readTree(jsonInputStream);
+  }
+
+  public static <T> T bytesToObject(byte[] jsonBytes, Class<T> valueType) throws IOException {
+    return DEFAULT_MAPPER.readValue(jsonBytes, valueType);
+  }
+
+  public static JsonNode bytesToJsonNode(byte[] jsonBytes) throws IOException {
+    return inputStreamToJsonNode(new ByteArrayInputStream(jsonBytes));
+  }
+
+  public static <T> T jsonNodeToObject(JsonNode jsonNode, Class<T> valueType) throws JsonProcessingException {
+    return DEFAULT_READER.treeToValue(jsonNode, valueType);
+  }
+
+  public static String objectToString(Object object) throws JsonProcessingException {
+    return DEFAULT_WRITER.writeValueAsString(object);
+  }
+
+  public static String objectToPrettyString(Object object) throws JsonProcessingException {
+    return DEFAULT_PRETTY_WRITER.writeValueAsString(object);
+  }
+
+  public static byte[] objectToBytes(Object object) throws JsonProcessingException {
+    return DEFAULT_WRITER.writeValueAsBytes(object);
+  }
+
+  public static JsonNode objectToJsonNode(Object object) {
+    return DEFAULT_MAPPER.valueToTree(object);
+  }
+
+  public static ObjectNode newObjectNode() {
+    return JsonNodeFactory.instance.objectNode();
+  }
+
+  public static ArrayNode newArrayNode() {
+    return JsonNodeFactory.instance.arrayNode();
+  }
+
+  public static Object extractValue(@Nullable JsonNode jsonValue, FieldSpec fieldSpec) {
+    if (fieldSpec.isSingleValueField()) {
+      if (jsonValue != null && !jsonValue.isNull()) {
+        return extractSingleValue(jsonValue, fieldSpec.getDataType());
+      } else {
+        return fieldSpec.getDefaultNullValue();
+      }
+    } else {
+      if (jsonValue != null && !jsonValue.isNull()) {
+        if (jsonValue.isArray()) {
+          int numValues = jsonValue.size();
+          if (numValues != 0) {
+            Object[] values = new Object[numValues];
+            for (int i = 0; i < numValues; i++) {
+              values[i] = extractSingleValue(jsonValue.get(i), fieldSpec.getDataType());
+            }
+            return values;
+          } else {
+            return new Object[]{fieldSpec.getDefaultNullValue()};
+          }
+        } else {
+          return new Object[]{extractSingleValue(jsonValue, fieldSpec.getDataType())};
+        }
+      } else {
+        return new Object[]{fieldSpec.getDefaultNullValue()};
+      }
+    }
+  }
+
+  private static Object extractSingleValue(JsonNode jsonValue, FieldSpec.DataType dataType) {
+    Preconditions.checkArgument(jsonValue.isValueNode());
+    switch (dataType) {
+      case INT:
+        return jsonValue.asInt();
+      case LONG:
+        return jsonValue.asLong();
+      case FLOAT:
+        return (float) jsonValue.asDouble();
+      case DOUBLE:
+        return jsonValue.asDouble();
+      case STRING:
+        return jsonValue.asText();
+      default:
+        throw new IllegalArgumentException();
+    }
+  }
+}
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/startree/hll/HllConfig.java b/pinot-common/src/main/java/com/linkedin/pinot/startree/hll/HllConfig.java
index 9d7215a..b00e89a 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/startree/hll/HllConfig.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/startree/hll/HllConfig.java
@@ -18,9 +18,11 @@
  */
 package com.linkedin.pinot.startree.hll;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.google.common.base.Preconditions;
 import com.linkedin.pinot.common.config.ConfigKey;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -28,13 +30,8 @@ import java.util.Map;
 import java.util.Set;
 import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.apache.commons.lang3.builder.ToStringStyle;
-import org.codehaus.jackson.annotate.JsonIgnore;
-import org.codehaus.jackson.map.ObjectMapper;
 
-import static com.linkedin.pinot.common.utils.EqualityUtils.hashCodeOf;
-import static com.linkedin.pinot.common.utils.EqualityUtils.isEqual;
-import static com.linkedin.pinot.common.utils.EqualityUtils.isNullOrNotSameClass;
-import static com.linkedin.pinot.common.utils.EqualityUtils.isSameReference;
+import static com.linkedin.pinot.common.utils.EqualityUtils.*;
 
 
 /**
@@ -58,8 +55,6 @@ public class HllConfig {
 
   private transient Map<String, String> derivedHllFieldToOriginMap;
 
-  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
-
   /**
    * HllConfig with default hll log2m. No Hll derived field is generated.
    */
@@ -149,11 +144,11 @@ public class HllConfig {
   }
 
   public String toJsonString() throws Exception {
-    return OBJECT_MAPPER.writerWithDefaultPrettyPrinter().writeValueAsString(this);
+    return JsonUtils.objectToPrettyString(this);
   }
 
   public static HllConfig fromJsonString(String jsonString) throws IOException {
-    return OBJECT_MAPPER.readValue(jsonString, HllConfig.class);
+    return JsonUtils.stringToObject(jsonString, HllConfig.class);
   }
 
   @Override
diff --git a/pinot-common/src/test/java/com/linkedin/pinot/common/config/IndexingConfigTest.java b/pinot-common/src/test/java/com/linkedin/pinot/common/config/IndexingConfigTest.java
index c73fc89..4c0c339 100644
--- a/pinot-common/src/test/java/com/linkedin/pinot/common/config/IndexingConfigTest.java
+++ b/pinot-common/src/test/java/com/linkedin/pinot/common/config/IndexingConfigTest.java
@@ -19,6 +19,7 @@
 package com.linkedin.pinot.common.config;
 
 import com.linkedin.pinot.common.data.StarTreeIndexSpec;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -27,79 +28,44 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
-import org.codehaus.jackson.JsonNode;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.json.JSONException;
-import org.json.JSONObject;
-import org.testng.Assert;
 import org.testng.annotations.Test;
 
+import static org.testng.Assert.*;
+
 
 public class IndexingConfigTest {
 
   @Test
-  public void testSerDe()
-      throws JSONException, IOException {
-    JSONObject json = new JSONObject();
-    json.put("invertedIndexColumns", Arrays.asList("a", "b", "c"));
-    json.put("sortedColumn", Arrays.asList("d", "e", "f"));
-
-    String[] expectedOnHeapDictionaryColumns = new String[] {"x", "y", "z"};
-    json.put("onHeapDictionaryColumns", Arrays.asList(expectedOnHeapDictionaryColumns));
-
-    json.put("loadMode", "MMAP");
-    json.put("keyThatIsUnknown", "randomValue");
-    json.put("aggregateMetrics", "true");
-
-    JSONObject noDictConfig = new JSONObject();
-    noDictConfig.put("a", "SNAPPY");
-    noDictConfig.put("b", "PASS_THROUGH");
-    json.put("noDictionaryConfig", noDictConfig);
-
-    String[] expectedBloomFilterColumns = new String[] {"a", "b"};
-    json.put("bloomFilterColumns", Arrays.asList(expectedBloomFilterColumns));
-
-    ObjectMapper mapper = new ObjectMapper();
-    JsonNode jsonNode = mapper.readTree(json.toString());
-    IndexingConfig indexingConfig = mapper.readValue(jsonNode, IndexingConfig.class);
-
-    Assert.assertEquals("MMAP", indexingConfig.getLoadMode());
-    List<String> invertedIndexColumns = indexingConfig.getInvertedIndexColumns();
-    Assert.assertEquals(3, invertedIndexColumns.size());
-    Assert.assertEquals("a", invertedIndexColumns.get(0));
-    Assert.assertEquals("b", invertedIndexColumns.get(1));
-    Assert.assertEquals("c", invertedIndexColumns.get(2));
-
-    List<String> sortedIndexColumns = indexingConfig.getSortedColumn();
-    Assert.assertEquals(3, sortedIndexColumns.size());
-    Assert.assertEquals("d", sortedIndexColumns.get(0));
-    Assert.assertEquals("e", sortedIndexColumns.get(1));
-    Assert.assertEquals("f", sortedIndexColumns.get(2));
-
-    // Test for noDictionaryConfig.
-    Map<String, String> noDictConfigMap = indexingConfig.getnoDictionaryConfig();
-    Assert.assertEquals(noDictConfigMap.size(), 2);
-    Assert.assertEquals(noDictConfig.get("a"), "SNAPPY");
-    Assert.assertEquals(noDictConfig.get("b"), "PASS_THROUGH");
-
-    List<String> actualOnHeapDictionaryColumns = indexingConfig.getOnHeapDictionaryColumns();
-    Assert.assertEquals(actualOnHeapDictionaryColumns.size(), expectedOnHeapDictionaryColumns.length);
-    for (int i = 0; i < expectedOnHeapDictionaryColumns.length; i++) {
-      Assert.assertEquals(actualOnHeapDictionaryColumns.get(i), expectedOnHeapDictionaryColumns[i]);
-    }
-
-    Assert.assertTrue(indexingConfig.getAggregateMetrics());
-
-    List<String> bloomFilterColumns = indexingConfig.getBloomFilterColumns();
-    Assert.assertEquals(bloomFilterColumns.size(), 2);
-    for (int i = 0; i < bloomFilterColumns.size(); i++) {
-      Assert.assertEquals(bloomFilterColumns.get(i), expectedBloomFilterColumns[i]);
-    }
+  public void testSerDe() throws IOException {
+    IndexingConfig indexingConfig = new IndexingConfig();
+    indexingConfig.setLoadMode("MMAP");
+    indexingConfig.setAggregateMetrics(true);
+    List<String> invertedIndexColumns = Arrays.asList("a", "b", "c");
+    indexingConfig.setInvertedIndexColumns(invertedIndexColumns);
+    List<String> sortedColumn = Arrays.asList("d", "e", "f");
+    indexingConfig.setSortedColumn(sortedColumn);
+    List<String> onHeapDictionaryColumns = Arrays.asList("x", "y", "z");
+    indexingConfig.setOnHeapDictionaryColumns(onHeapDictionaryColumns);
+    List<String> bloomFilterColumns = Arrays.asList("a", "b");
+    indexingConfig.setBloomFilterColumns(bloomFilterColumns);
+    Map<String, String> noDictionaryConfig = new HashMap<>();
+    noDictionaryConfig.put("a", "SNAPPY");
+    noDictionaryConfig.put("b", "PASS_THROUGH");
+    indexingConfig.setnoDictionaryConfig(noDictionaryConfig);
+
+    indexingConfig = JsonUtils.stringToObject(JsonUtils.objectToString(indexingConfig), IndexingConfig.class);
+
+    assertEquals(indexingConfig.getLoadMode(), "MMAP");
+    assertTrue(indexingConfig.isAggregateMetrics());
+    assertEquals(indexingConfig.getInvertedIndexColumns(), invertedIndexColumns);
+    assertEquals(indexingConfig.getSortedColumn(), sortedColumn);
+    assertEquals(indexingConfig.getOnHeapDictionaryColumns(), onHeapDictionaryColumns);
+    assertEquals(indexingConfig.getBloomFilterColumns(), bloomFilterColumns);
+    assertEquals(indexingConfig.getNoDictionaryConfig(), noDictionaryConfig);
   }
 
   @Test
-  public void testSegmentPartitionConfig()
-      throws IOException {
+  public void testSegmentPartitionConfig() throws IOException {
     int numColumns = 5;
     Map<String, ColumnPartitionConfig> expectedColumnPartitionMap = new HashMap<>(5);
     for (int i = 0; i < numColumns; i++) {
@@ -110,19 +76,16 @@ public class IndexingConfigTest {
     IndexingConfig expectedIndexingConfig = new IndexingConfig();
     expectedIndexingConfig.setSegmentPartitionConfig(expectedPartitionConfig);
 
-    ObjectMapper mapper = new ObjectMapper();
-    String indexingConfigString = mapper.writeValueAsString(expectedIndexingConfig);
-    IndexingConfig actualIndexingConfig = mapper.readValue(indexingConfigString, IndexingConfig.class);
+    IndexingConfig actualIndexingConfig =
+        JsonUtils.stringToObject(JsonUtils.objectToString(expectedIndexingConfig), IndexingConfig.class);
 
     SegmentPartitionConfig actualPartitionConfig = actualIndexingConfig.getSegmentPartitionConfig();
     Map<String, ColumnPartitionConfig> actualColumnPartitionMap = actualPartitionConfig.getColumnPartitionMap();
-    Assert.assertEquals(actualColumnPartitionMap.size(), expectedColumnPartitionMap.size());
+    assertEquals(actualColumnPartitionMap.size(), expectedColumnPartitionMap.size());
 
     for (String column : expectedColumnPartitionMap.keySet()) {
-      Assert.assertEquals(actualPartitionConfig.getFunctionName(column),
-          expectedPartitionConfig.getFunctionName(column));
-      Assert.assertEquals(actualPartitionConfig.getNumPartitions(column),
-          expectedPartitionConfig.getNumPartitions(column));
+      assertEquals(actualPartitionConfig.getFunctionName(column), expectedPartitionConfig.getFunctionName(column));
+      assertEquals(actualPartitionConfig.getNumPartitions(column), expectedPartitionConfig.getNumPartitions(column));
     }
   }
 
@@ -133,11 +96,9 @@ public class IndexingConfigTest {
    *   <li> Indexing config is first serialized into a string, and then read back from string. </li>
    *   <li> Test to ensure star tree index spec values are correct after serialization and de-serialization. </li>
    * </ul>
-   * @throws IOException
    */
   @Test
-  public void testStarTreeSpec()
-      throws IOException {
+  public void testStarTreeSpec() throws IOException {
     Random random = new Random(System.nanoTime());
     StarTreeIndexSpec expectedStarTreeSpec = new StarTreeIndexSpec();
 
@@ -150,29 +111,24 @@ public class IndexingConfigTest {
     int expectedSkipMaterializationThreshold = random.nextInt();
     expectedStarTreeSpec.setSkipMaterializationCardinalityThreshold(expectedSkipMaterializationThreshold);
 
-    Set<String> expectedSkipMaterializationDimensions = new HashSet<>(Arrays.asList(new String[]{"col4", "col5"}));
+    Set<String> expectedSkipMaterializationDimensions = new HashSet<>(Arrays.asList("col4", "col5"));
     expectedStarTreeSpec.setSkipMaterializationForDimensions(expectedSkipMaterializationDimensions);
 
-    Set<String> expectedSkipStarNodeCreationForDimension = new HashSet<>(Arrays.asList(new String[]{"col6", "col7"}));
+    Set<String> expectedSkipStarNodeCreationForDimension = new HashSet<>(Arrays.asList("col6", "col7"));
     expectedStarTreeSpec.setSkipStarNodeCreationForDimensions(expectedSkipStarNodeCreationForDimension);
 
     IndexingConfig expectedIndexingConfig = new IndexingConfig();
     expectedIndexingConfig.setStarTreeIndexSpec(expectedStarTreeSpec);
 
-    ObjectMapper objectMapper = new ObjectMapper();
-    String indexingConfigString = objectMapper.writeValueAsString(expectedIndexingConfig);
-
-    IndexingConfig actualIndexingConfig = objectMapper.readValue(indexingConfigString, IndexingConfig.class);
+    IndexingConfig actualIndexingConfig =
+        JsonUtils.stringToObject(JsonUtils.objectToString(expectedIndexingConfig), IndexingConfig.class);
     StarTreeIndexSpec actualStarTreeSpec = actualIndexingConfig.getStarTreeIndexSpec();
 
-    Assert.assertEquals(actualStarTreeSpec.getDimensionsSplitOrder(), expectedDimensionSplitOrder);
-    Assert.assertEquals(actualStarTreeSpec.getMaxLeafRecords(), expectedMaxLeafRecords);
+    assertEquals(actualStarTreeSpec.getDimensionsSplitOrder(), expectedDimensionSplitOrder);
+    assertEquals(actualStarTreeSpec.getMaxLeafRecords(), expectedMaxLeafRecords);
 
-    Assert.assertEquals(actualStarTreeSpec.getSkipMaterializationCardinalityThreshold(),
-        expectedSkipMaterializationThreshold);
-    Assert.assertEquals(actualStarTreeSpec.getSkipMaterializationForDimensions(),
-        expectedSkipMaterializationDimensions);
-    Assert.assertEquals(actualStarTreeSpec.getSkipStarNodeCreationForDimensions(),
-        expectedSkipStarNodeCreationForDimension);
+    assertEquals(actualStarTreeSpec.getSkipMaterializationCardinalityThreshold(), expectedSkipMaterializationThreshold);
+    assertEquals(actualStarTreeSpec.getSkipMaterializationForDimensions(), expectedSkipMaterializationDimensions);
+    assertEquals(actualStarTreeSpec.getSkipStarNodeCreationForDimensions(), expectedSkipStarNodeCreationForDimension);
   }
 }
diff --git a/pinot-common/src/test/java/com/linkedin/pinot/common/config/QuotaConfigTest.java b/pinot-common/src/test/java/com/linkedin/pinot/common/config/QuotaConfigTest.java
index e939288..76e6971 100644
--- a/pinot-common/src/test/java/com/linkedin/pinot/common/config/QuotaConfigTest.java
+++ b/pinot-common/src/test/java/com/linkedin/pinot/common/config/QuotaConfigTest.java
@@ -18,26 +18,27 @@
  */
 package com.linkedin.pinot.common.config;
 
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.io.IOException;
 import org.apache.commons.configuration.ConfigurationRuntimeException;
-import org.codehaus.jackson.map.ObjectMapper;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
+
 public class QuotaConfigTest {
 
   @Test
   public void testQuotaConfig() throws IOException {
     {
       String quotaConfigStr = "{\"storage\" : \"100g\"}";
-      QuotaConfig quotaConfig = new ObjectMapper().readValue(quotaConfigStr, QuotaConfig.class);
+      QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
 
       Assert.assertEquals(quotaConfig.getStorage(), "100g");
       Assert.assertEquals(quotaConfig.storageSizeBytes(), 100 * 1024 * 1024 * 1024L);
     }
     {
       String quotaConfigStr = "{}";
-      QuotaConfig quotaConfig = new ObjectMapper().readValue(quotaConfigStr, QuotaConfig.class);
+      QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
       Assert.assertNull(quotaConfig.getStorage());
       Assert.assertEquals(quotaConfig.storageSizeBytes(), -1);
     }
@@ -47,7 +48,7 @@ public class QuotaConfigTest {
   public void testBadQuotaConfig() throws IOException {
     {
       String quotaConfigStr = "{\"storage\" : \"124GB3GB\"}";
-      QuotaConfig quotaConfig = new ObjectMapper().readValue(quotaConfigStr, QuotaConfig.class);
+      QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
       Assert.assertNotNull(quotaConfig.getStorage());
       Assert.assertEquals(quotaConfig.storageSizeBytes(), -1);
     }
@@ -56,7 +57,7 @@ public class QuotaConfigTest {
   @Test(expectedExceptions = ConfigurationRuntimeException.class)
   public void testBadConfig() throws IOException {
     String quotaConfigStr = "{\"storage\":\"-1M\"}";
-    QuotaConfig quotaConfig = new ObjectMapper().readValue(quotaConfigStr, QuotaConfig.class);
+    QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
     quotaConfig.validate();
   }
 
@@ -64,7 +65,7 @@ public class QuotaConfigTest {
   public void testQpsQuota() throws IOException {
     {
       String quotaConfigStr = "{\"maxQueriesPerSecond\" : \"100.00\"}";
-      QuotaConfig quotaConfig = new ObjectMapper().readValue(quotaConfigStr, QuotaConfig.class);
+      QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
 
       Assert.assertNotNull(quotaConfig.getMaxQueriesPerSecond());
       Assert.assertEquals(quotaConfig.getMaxQueriesPerSecond(), "100.00");
@@ -72,7 +73,7 @@ public class QuotaConfigTest {
     }
     {
       String quotaConfigStr = "{\"maxQueriesPerSecond\" : \"0.5\"}";
-      QuotaConfig quotaConfig = new ObjectMapper().readValue(quotaConfigStr, QuotaConfig.class);
+      QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
 
       Assert.assertNotNull(quotaConfig.getMaxQueriesPerSecond());
       Assert.assertEquals(quotaConfig.getMaxQueriesPerSecond(), "0.5");
@@ -80,7 +81,7 @@ public class QuotaConfigTest {
     }
     {
       String quotaConfigStr = "{}";
-      QuotaConfig quotaConfig = new ObjectMapper().readValue(quotaConfigStr, QuotaConfig.class);
+      QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
       Assert.assertNull(quotaConfig.getMaxQueriesPerSecond());
       Assert.assertTrue(quotaConfig.isMaxQueriesPerSecondValid());
     }
@@ -89,7 +90,7 @@ public class QuotaConfigTest {
   @Test(expectedExceptions = ConfigurationRuntimeException.class)
   public void testInvalidQpsQuota() throws IOException {
     String quotaConfigStr = "{\"maxQueriesPerSecond\" : \"InvalidQpsQuota\"}";
-    QuotaConfig quotaConfig = new ObjectMapper().readValue(quotaConfigStr, QuotaConfig.class);
+    QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
     Assert.assertNotNull(quotaConfig.getMaxQueriesPerSecond());
     quotaConfig.validate();
   }
@@ -97,7 +98,7 @@ public class QuotaConfigTest {
   @Test(expectedExceptions = ConfigurationRuntimeException.class)
   public void testNegativeQpsQuota() throws IOException {
     String quotaConfigStr = "{\"maxQueriesPerSecond\" : \"-1.0\"}";
-    QuotaConfig quotaConfig = new ObjectMapper().readValue(quotaConfigStr, QuotaConfig.class);
+    QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
     Assert.assertNotNull(quotaConfig.getMaxQueriesPerSecond());
     quotaConfig.validate();
   }
@@ -105,7 +106,7 @@ public class QuotaConfigTest {
   @Test(expectedExceptions = ConfigurationRuntimeException.class)
   public void testBadQpsQuota() throws IOException {
     String quotaConfigStr = "{\"maxQueriesPerSecond\" : \"1.0Test\"}";
-    QuotaConfig quotaConfig = new ObjectMapper().readValue(quotaConfigStr, QuotaConfig.class);
+    QuotaConfig quotaConfig = JsonUtils.stringToObject(quotaConfigStr, QuotaConfig.class);
     Assert.assertNotNull(quotaConfig.getMaxQueriesPerSecond());
     quotaConfig.validate();
   }
diff --git a/pinot-common/src/test/java/com/linkedin/pinot/common/config/TableConfigTest.java b/pinot-common/src/test/java/com/linkedin/pinot/common/config/TableConfigTest.java
index f20c92b..8630ea8 100644
--- a/pinot-common/src/test/java/com/linkedin/pinot/common/config/TableConfigTest.java
+++ b/pinot-common/src/test/java/com/linkedin/pinot/common/config/TableConfigTest.java
@@ -25,7 +25,6 @@ import java.util.Collections;
 import java.util.HashSet;
 import java.util.Set;
 import org.apache.helix.ZNRecord;
-import org.json.JSONObject;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -45,8 +44,7 @@ public class TableConfigTest {
       Assert.assertNull(tableConfig.getQuotaConfig());
 
       // Serialize then de-serialize
-      JSONObject jsonConfig = TableConfig.toJSONConfig(tableConfig);
-      TableConfig tableConfigToCompare = TableConfig.fromJSONConfig(jsonConfig);
+      TableConfig tableConfigToCompare = TableConfig.fromJSONConfig(TableConfig.toJSONConfig(tableConfig));
       Assert.assertEquals(tableConfigToCompare.getTableName(), tableConfig.getTableName());
       Assert.assertNull(tableConfigToCompare.getQuotaConfig());
       Assert.assertNull(tableConfigToCompare.getValidationConfig().getReplicaGroupStrategyConfig());
@@ -75,12 +73,12 @@ public class TableConfigTest {
       // With qps quota
       quotaConfig.setMaxQueriesPerSecond("100.00");
       tableConfig = tableConfigBuilder.setQuotaConfig(quotaConfig).build();
+      Assert.assertNotNull(tableConfig.getQuotaConfig());
       Assert.assertNotNull(tableConfig.getQuotaConfig().getMaxQueriesPerSecond());
       Assert.assertEquals(tableConfig.getQuotaConfig().getMaxQueriesPerSecond(), "100.00");
 
       // Serialize then de-serialize
-      JSONObject jsonConfig = TableConfig.toJSONConfig(tableConfig);
-      TableConfig tableConfigToCompare = TableConfig.fromJSONConfig(jsonConfig);
+      TableConfig tableConfigToCompare = TableConfig.fromJSONConfig(TableConfig.toJSONConfig(tableConfig));
       Assert.assertEquals(tableConfigToCompare.getTableName(), tableConfig.getTableName());
       Assert.assertNotNull(tableConfigToCompare.getQuotaConfig());
       Assert.assertEquals(tableConfigToCompare.getQuotaConfig().getStorage(),
@@ -106,8 +104,7 @@ public class TableConfigTest {
       Assert.assertNull(tableConfig.getTenantConfig().getTagOverrideConfig());
 
       // Serialize then de-serialize
-      JSONObject jsonConfig = TableConfig.toJSONConfig(tableConfig);
-      TableConfig tableConfigToCompare = TableConfig.fromJSONConfig(jsonConfig);
+      TableConfig tableConfigToCompare = TableConfig.fromJSONConfig(TableConfig.toJSONConfig(tableConfig));
       Assert.assertEquals(tableConfigToCompare.getTableName(), tableConfig.getTableName());
       Assert.assertNotNull(tableConfigToCompare.getTenantConfig());
       Assert.assertEquals(tableConfigToCompare.getTenantConfig().getServer(),
@@ -140,8 +137,7 @@ public class TableConfigTest {
       Assert.assertNull(tableConfig.getTenantConfig().getTagOverrideConfig().getRealtimeCompleted());
 
       // Serialize then de-serialize
-      jsonConfig = TableConfig.toJSONConfig(tableConfig);
-      tableConfigToCompare = TableConfig.fromJSONConfig(jsonConfig);
+      tableConfigToCompare = TableConfig.fromJSONConfig(TableConfig.toJSONConfig(tableConfig));
       Assert.assertEquals(tableConfigToCompare.getTableName(), tableConfig.getTableName());
       Assert.assertNotNull(tableConfigToCompare.getTenantConfig());
       Assert.assertEquals(tableConfigToCompare.getTenantConfig().getServer(),
@@ -176,8 +172,7 @@ public class TableConfigTest {
       tableConfig.getValidationConfig().setReplicaGroupStrategyConfig(replicaGroupConfig);
 
       // Serialize then de-serialize
-      JSONObject jsonConfig = TableConfig.toJSONConfig(tableConfig);
-      TableConfig tableConfigToCompare = TableConfig.fromJSONConfig(jsonConfig);
+      TableConfig tableConfigToCompare = TableConfig.fromJSONConfig(TableConfig.toJSONConfig(tableConfig));
       checkTableConfigWithAssignmentConfig(tableConfig, tableConfigToCompare);
 
       ZNRecord znRecord = TableConfig.toZnRecord(tableConfig);
@@ -199,8 +194,7 @@ public class TableConfigTest {
               .getStreamPartitionAssignmentStrategy(), "BalancedStreamPartitionAssignment");
 
       // Serialize then de-serialize
-      JSONObject jsonConfig = TableConfig.toJSONConfig(tableConfig);
-      TableConfig tableConfigToCompare = TableConfig.fromJSONConfig(jsonConfig);
+      TableConfig tableConfigToCompare = TableConfig.fromJSONConfig(TableConfig.toJSONConfig(tableConfig));
       Assert.assertEquals(
           tableConfigToCompare.getIndexingConfig().getStreamConsumptionConfig()
               .getStreamPartitionAssignmentStrategy(), "BalancedStreamPartitionAssignment");
@@ -226,8 +220,7 @@ public class TableConfigTest {
       tableConfig.getIndexingConfig().setStarTreeIndexSpec(starTreeIndexSpec);
 
       // Serialize then de-serialize
-      JSONObject jsonConfig = TableConfig.toJSONConfig(tableConfig);
-      TableConfig tableConfigToCompare = TableConfig.fromJSONConfig(jsonConfig);
+      TableConfig tableConfigToCompare = TableConfig.fromJSONConfig(TableConfig.toJSONConfig(tableConfig));
       checkTableConfigWithStarTreeConfig(tableConfig, tableConfigToCompare);
 
       ZNRecord znRecord = TableConfig.toZnRecord(tableConfig);
@@ -255,8 +248,7 @@ public class TableConfigTest {
       tableConfig.getValidationConfig().setHllConfig(hllConfig);
 
       // Serialize then de-serialize
-      JSONObject jsonConfig = TableConfig.toJSONConfig(tableConfig);
-      TableConfig tableConfigToCompare = TableConfig.fromJSONConfig(jsonConfig);
+      TableConfig tableConfigToCompare = TableConfig.fromJSONConfig(TableConfig.toJSONConfig(tableConfig));
       checkTableConfigWithHllConfig(tableConfig, tableConfigToCompare);
 
       ZNRecord znRecord = TableConfig.toZnRecord(tableConfig);
@@ -275,7 +267,7 @@ public class TableConfigTest {
     // Check that the configurations are correct.
     ReplicaGroupStrategyConfig strategyConfig =
         tableConfigToCompare.getValidationConfig().getReplicaGroupStrategyConfig();
-    Assert.assertEquals(strategyConfig.getMirrorAssignmentAcrossReplicaGroups(), true);
+    Assert.assertTrue(strategyConfig.getMirrorAssignmentAcrossReplicaGroups());
     Assert.assertEquals(strategyConfig.getNumInstancesPerPartition(), 5);
     Assert.assertEquals(strategyConfig.getPartitionColumn(), "memberId");
   }
@@ -318,7 +310,7 @@ public class TableConfigTest {
     columns.add("column");
     columns.add("column2");
 
-    Assert.assertTrue(hllConfig.getColumnsToDeriveHllFields().equals(columns));
+    Assert.assertEquals(hllConfig.getColumnsToDeriveHllFields(), columns);
     Assert.assertEquals(hllConfig.getHllLog2m(), 9);
     Assert.assertEquals(hllConfig.getHllDeriveColumnSuffix(), "suffix");
   }
diff --git a/pinot-common/src/test/java/com/linkedin/pinot/common/config/TagOverrideConfigTest.java b/pinot-common/src/test/java/com/linkedin/pinot/common/config/TagOverrideConfigTest.java
index 701edfc..423d2d1 100644
--- a/pinot-common/src/test/java/com/linkedin/pinot/common/config/TagOverrideConfigTest.java
+++ b/pinot-common/src/test/java/com/linkedin/pinot/common/config/TagOverrideConfigTest.java
@@ -19,10 +19,8 @@
 package com.linkedin.pinot.common.config;
 
 import com.linkedin.pinot.common.utils.CommonConstants;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import org.json.JSONException;
 import org.testng.Assert;
 import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
@@ -31,7 +29,7 @@ import org.testng.annotations.Test;
 public class TagOverrideConfigTest {
 
   @DataProvider(name = "realtimeTagConfigTestDataProvider")
-  public Object[][] realtimeTagConfigTestDataProvider() throws IOException, JSONException {
+  public Object[][] realtimeTagConfigTestDataProvider() {
     TableConfig.Builder tableConfigBuilder = new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE);
     tableConfigBuilder.setTableName("testRealtimeTable")
         .setTimeColumnName("timeColumn")
@@ -85,7 +83,7 @@ public class TagOverrideConfigTest {
   }
 
   @DataProvider(name = "offlineTagConfigTestDataProvider")
-  public Object[][] offlineTagConfigTestDataProvider() throws IOException, JSONException {
+  public Object[][] offlineTagConfigTestDataProvider() {
     TableConfig.Builder tableConfigBuilder = new TableConfig.Builder(CommonConstants.Helix.TableType.OFFLINE);
     tableConfigBuilder.setTableName("testOfflineTable")
         .setTimeColumnName("timeColumn")
diff --git a/pinot-common/src/test/java/com/linkedin/pinot/common/config/TenantTest.java b/pinot-common/src/test/java/com/linkedin/pinot/common/config/TenantTest.java
index 0ef29d3..f98efdb 100644
--- a/pinot-common/src/test/java/com/linkedin/pinot/common/config/TenantTest.java
+++ b/pinot-common/src/test/java/com/linkedin/pinot/common/config/TenantTest.java
@@ -18,35 +18,31 @@
  */
 package com.linkedin.pinot.common.config;
 
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.TenantRole;
 import java.io.IOException;
-import org.codehaus.jackson.JsonNode;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.json.JSONException;
-import org.json.JSONObject;
-import org.testng.Assert;
 import org.testng.annotations.Test;
 
+import static org.testng.Assert.*;
+
+
 public class TenantTest {
 
   @Test
-  public void testDeserializeFromJson()
-      throws JSONException, IOException {
-    JSONObject json = new JSONObject();
-    json.put("tenantRole", "SERVER");
-    json.put("tenantName", "newTenant");
-    json.put("numberOfInstances", 10);
-    json.put("offlineInstances", 5);
-    json.put("realtimeInstances", 5);
-    json.put("keyIDontKnow", "blahblahblah");
+  public void testDeserializeFromJson() throws IOException {
+    Tenant tenant = new Tenant();
+    tenant.setTenantRole(TenantRole.SERVER);
+    tenant.setTenantName("newTenant");
+    tenant.setNumberOfInstances(10);
+    tenant.setOfflineInstances(5);
+    tenant.setRealtimeInstances(5);
 
-    ObjectMapper mapper = new ObjectMapper();
-    JsonNode jsonNode = mapper.readTree(json.toString());
-    Tenant tenant = mapper.readValue(jsonNode, Tenant.class);
-    Assert.assertEquals(5, tenant.getOfflineInstances());
-    Assert.assertEquals(10, tenant.getNumberOfInstances());
-    Assert.assertEquals("newTenant", tenant.getTenantName());
-    Assert.assertEquals(TenantRole.SERVER, tenant.getTenantRole());
-  }
+    tenant = JsonUtils.stringToObject(JsonUtils.objectToString(tenant), Tenant.class);
 
+    assertEquals(tenant.getTenantRole(), TenantRole.SERVER);
+    assertEquals(tenant.getTenantName(), "newTenant");
+    assertEquals(tenant.getNumberOfInstances(), 10);
+    assertEquals(tenant.getOfflineInstances(), 5);
+    assertEquals(tenant.getRealtimeInstances(), 5);
+  }
 }
diff --git a/pinot-common/src/test/java/com/linkedin/pinot/common/data/FieldSpecTest.java b/pinot-common/src/test/java/com/linkedin/pinot/common/data/FieldSpecTest.java
index 70ca7a5..87efe6a 100644
--- a/pinot-common/src/test/java/com/linkedin/pinot/common/data/FieldSpecTest.java
+++ b/pinot-common/src/test/java/com/linkedin/pinot/common/data/FieldSpecTest.java
@@ -18,13 +18,13 @@
  */
 package com.linkedin.pinot.common.data;
 
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 import org.apache.avro.Schema;
-import org.codehaus.jackson.map.ObjectMapper;
 import org.testng.Assert;
 import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
@@ -39,7 +39,6 @@ public class FieldSpecTest {
   private static final long RANDOM_SEED = System.currentTimeMillis();
   private static final Random RANDOM = new Random(RANDOM_SEED);
   private static final String ERROR_MESSAGE = "Random seed is: " + RANDOM_SEED;
-  private static final ObjectMapper MAPPER = new ObjectMapper();
 
   /**
    * Test all {@link FieldSpec.DataType}.
@@ -134,7 +133,7 @@ public class FieldSpecTest {
 
     // Test serialize deserialize.
     MetricFieldSpec derivedMetricField2 =
-        MAPPER.readValue(derivedMetricField.toJsonObject().toString(), MetricFieldSpec.class);
+        JsonUtils.stringToObject(derivedMetricField.toJsonObject().toString(), MetricFieldSpec.class);
     Assert.assertEquals(derivedMetricField2, derivedMetricField);
   }
 
@@ -278,7 +277,8 @@ public class FieldSpecTest {
   public void testOrderOfFields() throws Exception {
     // Metric field with default null value.
     String[] metricFields = {"\"name\":\"metric\"", "\"dataType\":\"INT\"", "\"defaultNullValue\":-1"};
-    MetricFieldSpec metricFieldSpec1 = MAPPER.readValue(getRandomOrderJsonString(metricFields), MetricFieldSpec.class);
+    MetricFieldSpec metricFieldSpec1 =
+        JsonUtils.stringToObject(getRandomOrderJsonString(metricFields), MetricFieldSpec.class);
     MetricFieldSpec metricFieldSpec2 = new MetricFieldSpec("metric", INT, -1);
     Assert.assertEquals(metricFieldSpec1, metricFieldSpec2, ERROR_MESSAGE);
     Assert.assertEquals(metricFieldSpec1.getDefaultNullValue(), -1, ERROR_MESSAGE);
@@ -286,7 +286,7 @@ public class FieldSpecTest {
     // Single-value boolean type dimension field with default null value.
     String[] dimensionFields = {"\"name\":\"dimension\"", "\"dataType\":\"BOOLEAN\"", "\"defaultNullValue\":false"};
     DimensionFieldSpec dimensionFieldSpec1 =
-        MAPPER.readValue(getRandomOrderJsonString(dimensionFields), DimensionFieldSpec.class);
+        JsonUtils.stringToObject(getRandomOrderJsonString(dimensionFields), DimensionFieldSpec.class);
     DimensionFieldSpec dimensionFieldSpec2 = new DimensionFieldSpec("dimension", BOOLEAN, true, false);
     Assert.assertEquals(dimensionFieldSpec1, dimensionFieldSpec2, ERROR_MESSAGE);
     Assert.assertEquals(dimensionFieldSpec1.getDefaultNullValue(), "false", ERROR_MESSAGE);
@@ -294,7 +294,7 @@ public class FieldSpecTest {
     // Multi-value dimension field with default null value.
     dimensionFields =
         new String[]{"\"name\":\"dimension\"", "\"dataType\":\"STRING\"", "\"singleValueField\":false", "\"defaultNullValue\":\"default\""};
-    dimensionFieldSpec1 = MAPPER.readValue(getRandomOrderJsonString(dimensionFields), DimensionFieldSpec.class);
+    dimensionFieldSpec1 = JsonUtils.stringToObject(getRandomOrderJsonString(dimensionFields), DimensionFieldSpec.class);
     dimensionFieldSpec2 = new DimensionFieldSpec("dimension", STRING, false, "default");
     Assert.assertEquals(dimensionFieldSpec1, dimensionFieldSpec2, ERROR_MESSAGE);
     Assert.assertEquals(dimensionFieldSpec1.getDefaultNullValue(), "default", ERROR_MESSAGE);
@@ -302,7 +302,7 @@ public class FieldSpecTest {
     // Time field with default null value.
     String[] timeFields =
         {"\"incomingGranularitySpec\":{\"timeType\":\"MILLISECONDS\",\"dataType\":\"LONG\",\"name\":\"incomingTime\"}", "\"outgoingGranularitySpec\":{\"timeType\":\"SECONDS\",\"dataType\":\"INT\",\"name\":\"outgoingTime\"}", "\"defaultNullValue\":-1"};
-    TimeFieldSpec timeFieldSpec1 = MAPPER.readValue(getRandomOrderJsonString(timeFields), TimeFieldSpec.class);
+    TimeFieldSpec timeFieldSpec1 = JsonUtils.stringToObject(getRandomOrderJsonString(timeFields), TimeFieldSpec.class);
     TimeFieldSpec timeFieldSpec2 =
         new TimeFieldSpec("incomingTime", LONG, TimeUnit.MILLISECONDS, "outgoingTime", INT, TimeUnit.SECONDS, -1);
     Assert.assertEquals(timeFieldSpec1, timeFieldSpec2, ERROR_MESSAGE);
@@ -312,7 +312,7 @@ public class FieldSpecTest {
     String[] dateTimeFields =
         {"\"name\":\"Date\"", "\"dataType\":\"LONG\"", "\"format\":\"1:MILLISECONDS:EPOCH\"", "\"granularity\":\"5:MINUTES\"", "\"dateTimeType\":\"PRIMARY\""};
     DateTimeFieldSpec dateTimeFieldSpec1 =
-        MAPPER.readValue(getRandomOrderJsonString(dateTimeFields), DateTimeFieldSpec.class);
+        JsonUtils.stringToObject(getRandomOrderJsonString(dateTimeFields), DateTimeFieldSpec.class);
     DateTimeFieldSpec dateTimeFieldSpec2 = new DateTimeFieldSpec("Date", LONG, "1:MILLISECONDS:EPOCH", "5:MINUTES");
     Assert.assertEquals(dateTimeFieldSpec1, dateTimeFieldSpec2, ERROR_MESSAGE);
   }
@@ -327,29 +327,29 @@ public class FieldSpecTest {
 
     // Single-value boolean type dimension field with default null value.
     String[] dimensionFields = {"\"name\":\"dimension\"", "\"dataType\":\"BOOLEAN\"", "\"defaultNullValue\":false"};
-    first = MAPPER.readValue(getRandomOrderJsonString(dimensionFields), DimensionFieldSpec.class);
-    second = MAPPER.readValue(first.toJsonObject().toString(), DimensionFieldSpec.class);
+    first = JsonUtils.stringToObject(getRandomOrderJsonString(dimensionFields), DimensionFieldSpec.class);
+    second = JsonUtils.stringToObject(first.toJsonObject().toString(), DimensionFieldSpec.class);
     Assert.assertEquals(first, second, ERROR_MESSAGE);
 
     // Multi-value dimension field with default null value.
     dimensionFields =
         new String[]{"\"name\":\"dimension\"", "\"dataType\":\"STRING\"", "\"singleValueField\":false", "\"defaultNullValue\":\"default\""};
-    first = MAPPER.readValue(getRandomOrderJsonString(dimensionFields), DimensionFieldSpec.class);
-    second = MAPPER.readValue(first.toJsonObject().toString(), DimensionFieldSpec.class);
+    first = JsonUtils.stringToObject(getRandomOrderJsonString(dimensionFields), DimensionFieldSpec.class);
+    second = JsonUtils.stringToObject(first.toJsonObject().toString(), DimensionFieldSpec.class);
     Assert.assertEquals(first, second, ERROR_MESSAGE);
 
     // Time field with default value.
     String[] timeFields =
         {"\"incomingGranularitySpec\":{\"timeUnitSize\":1, \"timeType\":\"MILLISECONDS\",\"dataType\":\"LONG\",\"name\":\"incomingTime\"}", "\"outgoingGranularitySpec\":{\"timeType\":\"SECONDS\",\"dataType\":\"INT\",\"name\":\"outgoingTime\"}", "\"defaultNullValue\":-1"};
-    first = MAPPER.readValue(getRandomOrderJsonString(timeFields), TimeFieldSpec.class);
-    second = MAPPER.readValue(first.toJsonObject().toString(), TimeFieldSpec.class);
+    first = JsonUtils.stringToObject(getRandomOrderJsonString(timeFields), TimeFieldSpec.class);
+    second = JsonUtils.stringToObject(first.toJsonObject().toString(), TimeFieldSpec.class);
     Assert.assertEquals(first, second, ERROR_MESSAGE);
 
     // DateTime field
     String[] dateTimeFields =
         {"\"name\":\"Date\"", "\"dataType\":\"LONG\"", "\"format\":\"1:MILLISECONDS:EPOCH\"", "\"granularity\":\"5:MINUTES\"", "\"dateTimeType\":\"PRIMARY\""};
-    first = MAPPER.readValue(getRandomOrderJsonString(dateTimeFields), DateTimeFieldSpec.class);
-    second = MAPPER.readValue(first.toJsonObject().toString(), DateTimeFieldSpec.class);
+    first = JsonUtils.stringToObject(getRandomOrderJsonString(dateTimeFields), DateTimeFieldSpec.class);
+    second = JsonUtils.stringToObject(first.toJsonObject().toString(), DateTimeFieldSpec.class);
     Assert.assertEquals(first, second, ERROR_MESSAGE);
   }
 
diff --git a/pinot-common/src/test/java/com/linkedin/pinot/common/utils/FileUploadDownloadClientTest.java b/pinot-common/src/test/java/com/linkedin/pinot/common/utils/FileUploadDownloadClientTest.java
index 722cdd3..53e882a 100644
--- a/pinot-common/src/test/java/com/linkedin/pinot/common/utils/FileUploadDownloadClientTest.java
+++ b/pinot-common/src/test/java/com/linkedin/pinot/common/utils/FileUploadDownloadClientTest.java
@@ -18,6 +18,7 @@
  */
 package com.linkedin.pinot.common.utils;
 
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.linkedin.pinot.common.utils.FileUploadDownloadClient.FileUploadType;
 import com.sun.net.httpserver.Headers;
 import com.sun.net.httpserver.HttpExchange;
@@ -35,8 +36,6 @@ import org.apache.http.Header;
 import org.apache.http.HttpStatus;
 import org.apache.http.NameValuePair;
 import org.apache.http.message.BasicHeader;
-import org.json.JSONException;
-import org.json.JSONObject;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
@@ -68,20 +67,15 @@ public class FileUploadDownloadClientTest {
       FileUploadType uploadType = FileUploadType.valueOf(uploadTypeStr);
 
       String downloadUri = null;
-      String crypter = null;
 
       if (uploadType == FileUploadType.JSON) {
         InputStream bodyStream = httpExchange.getRequestBody();
-        try {
-          JSONObject jsonObject = new JSONObject(IOUtils.toString(bodyStream, "UTF-8"));
-          downloadUri = (String) jsonObject.get(CommonConstants.Segment.Offline.DOWNLOAD_URL);
-        } catch (JSONException e) {
-          throw new RuntimeException(e);
-        }
-        Assert.assertEquals(downloadUri, TEST_URI);
+        downloadUri = JsonUtils.stringToJsonNode(IOUtils.toString(bodyStream, "UTF-8"))
+            .get(CommonConstants.Segment.Offline.DOWNLOAD_URL)
+            .asText();
       } else if (uploadType == FileUploadType.URI) {
         downloadUri = requestHeaders.getFirst(FileUploadDownloadClient.CustomHeaders.DOWNLOAD_URI);
-        crypter = requestHeaders.getFirst(FileUploadDownloadClient.CustomHeaders.CRYPTER);
+        String crypter = requestHeaders.getFirst(FileUploadDownloadClient.CustomHeaders.CRYPTER);
         Assert.assertEquals(crypter, TEST_CRYPTER);
       } else {
         Assert.fail();
@@ -117,7 +111,7 @@ public class FileUploadDownloadClientTest {
 
   @Test
   public void testSendFileWithJson() throws Exception {
-    JSONObject segmentJson = new JSONObject();
+    ObjectNode segmentJson = JsonUtils.newObjectNode();
     segmentJson.put(CommonConstants.Segment.Offline.DOWNLOAD_URL, TEST_URI);
     String jsonString = segmentJson.toString();
     try (FileUploadDownloadClient fileUploadDownloadClient = new FileUploadDownloadClient()) {
diff --git a/pinot-common/src/test/java/com/linkedin/pinot/request/BrokerResponseNativeTest.java b/pinot-common/src/test/java/com/linkedin/pinot/request/BrokerResponseNativeTest.java
index 96a7ea8..4635384 100644
--- a/pinot-common/src/test/java/com/linkedin/pinot/request/BrokerResponseNativeTest.java
+++ b/pinot-common/src/test/java/com/linkedin/pinot/request/BrokerResponseNativeTest.java
@@ -22,7 +22,6 @@ import com.linkedin.pinot.common.exception.QueryException;
 import com.linkedin.pinot.common.response.broker.BrokerResponseNative;
 import com.linkedin.pinot.common.response.broker.QueryProcessingException;
 import java.io.IOException;
-import org.json.JSONException;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -30,8 +29,7 @@ import org.testng.annotations.Test;
 public class BrokerResponseNativeTest {
 
   @Test
-  public void testEmptyResponse()
-      throws JSONException, IOException {
+  public void testEmptyResponse() throws IOException {
     BrokerResponseNative expected = BrokerResponseNative.EMPTY_RESULT;
     String brokerString = expected.toJsonString();
     BrokerResponseNative actual = BrokerResponseNative.fromJsonString(brokerString);
@@ -42,8 +40,7 @@ public class BrokerResponseNativeTest {
   }
 
   @Test
-  public void testNullResponse()
-      throws JSONException, IOException {
+  public void testNullResponse() throws IOException {
     BrokerResponseNative expected = BrokerResponseNative.NO_TABLE_RESULT;
     String brokerString = expected.toJsonString();
     BrokerResponseNative actual = BrokerResponseNative.fromJsonString(brokerString);
@@ -54,8 +51,7 @@ public class BrokerResponseNativeTest {
   }
 
   @Test
-  public void testMultipleExceptionsResponse()
-      throws JSONException, IOException {
+  public void testMultipleExceptionsResponse() throws IOException {
     BrokerResponseNative expected = BrokerResponseNative.NO_TABLE_RESULT;
     String errorMsgStr = "Some random string!";
     QueryProcessingException processingException = new QueryProcessingException(400, errorMsgStr);
diff --git a/pinot-controller/pom.xml b/pinot-controller/pom.xml
index 989e40a..ddb45db 100644
--- a/pinot-controller/pom.xml
+++ b/pinot-controller/pom.xml
@@ -98,10 +98,6 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.codehaus.jackson</groupId>
-      <artifactId>jackson-mapper-asl</artifactId>
-    </dependency>
-    <dependency>
       <groupId>log4j</groupId>
       <artifactId>log4j</artifactId>
     </dependency>
@@ -131,11 +127,11 @@
     </dependency>
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
+      <artifactId>jackson-annotations</artifactId>
     </dependency>
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-annotations</artifactId>
+      <artifactId>jackson-databind</artifactId>
     </dependency>
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
diff --git a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/pojos/Instance.java b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/pojos/Instance.java
index a0887cc..5f0a174 100644
--- a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/pojos/Instance.java
+++ b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/pojos/Instance.java
@@ -22,8 +22,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.linkedin.pinot.common.utils.CommonConstants;
 import org.apache.helix.model.InstanceConfig;
-import org.json.JSONException;
-import org.json.JSONObject;
 
 
 /**
@@ -111,15 +109,6 @@ public class Instance {
     return bld.toString();
   }
 
-  public JSONObject toJSON() throws JSONException {
-    final JSONObject ret = new JSONObject();
-    ret.put("host", _host);
-    ret.put("port", _port);
-    ret.put("type", _type);
-    ret.put("tag", getTagOrDefaultTag());
-    return ret;
-  }
-
   public InstanceConfig toInstanceConfig() {
     final InstanceConfig iConfig = new InstanceConfig(toInstanceId());
     iConfig.setHostName(_host);
diff --git a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotInstanceRestletResource.java b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotInstanceRestletResource.java
index ab6b38b..88fb213 100644
--- a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotInstanceRestletResource.java
+++ b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotInstanceRestletResource.java
@@ -19,6 +19,8 @@
 package com.linkedin.pinot.controller.api.resources;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.controller.api.pojos.Instance;
 import com.linkedin.pinot.controller.helix.core.PinotHelixResourceManager;
 import com.linkedin.pinot.controller.helix.core.PinotResourceManagerResponse;
@@ -39,9 +41,6 @@ import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import org.apache.helix.model.InstanceConfig;
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -100,16 +99,12 @@ public class PinotInstanceRestletResource {
           Response.Status.NOT_FOUND);
     }
     InstanceConfig instanceConfig = pinotHelixResourceManager.getHelixInstanceConfig(instanceName);
-    JSONObject response = new JSONObject();
-    try {
-      response.put("instanceName", instanceConfig.getInstanceName());
-      response.put("hostName", instanceConfig.getHostName());
-      response.put("enabled", instanceConfig.getInstanceEnabled());
-      response.put("port", instanceConfig.getPort());
-      response.put("tags", new JSONArray(instanceConfig.getTags()));
-    } catch (JSONException e) {
-      throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.INTERNAL_SERVER_ERROR, e);
-    }
+    ObjectNode response = JsonUtils.newObjectNode();
+    response.put("instanceName", instanceConfig.getInstanceName());
+    response.put("hostName", instanceConfig.getHostName());
+    response.put("enabled", instanceConfig.getInstanceEnabled());
+    response.put("port", instanceConfig.getPort());
+    response.set("tags", JsonUtils.objectToJsonNode(instanceConfig.getTags()));
     return response.toString();
   }
 
diff --git a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotSchemaRestletResource.java b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotSchemaRestletResource.java
index 51574ab..9a4b83d 100644
--- a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotSchemaRestletResource.java
+++ b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotSchemaRestletResource.java
@@ -18,10 +18,12 @@
  */
 package com.linkedin.pinot.controller.api.resources;
 
+import com.fasterxml.jackson.databind.node.ArrayNode;
 import com.linkedin.pinot.common.config.TableConfig;
 import com.linkedin.pinot.common.data.Schema;
 import com.linkedin.pinot.common.metrics.ControllerMeter;
 import com.linkedin.pinot.common.metrics.ControllerMetrics;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.controller.api.events.MetadataEventNotifierFactory;
 import com.linkedin.pinot.controller.api.events.SchemaEventType;
 import com.linkedin.pinot.controller.helix.core.PinotHelixResourceManager;
@@ -47,7 +49,6 @@ import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import org.glassfish.jersey.media.multipart.FormDataBodyPart;
 import org.glassfish.jersey.media.multipart.FormDataMultiPart;
-import org.json.JSONArray;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -72,11 +73,11 @@ public class PinotSchemaRestletResource {
   @ApiOperation(value = "List all schema names", notes = "Lists all schema names")
   public String listSchemaNames() {
     List<String> schemaNames = _pinotHelixResourceManager.getSchemaNames();
-    JSONArray ret = new JSONArray();
+    ArrayNode ret = JsonUtils.newArrayNode();
 
     if (schemaNames != null) {
       for (String schema : schemaNames) {
-        ret.put(schema);
+        ret.add(schema);
       }
     }
     return ret.toString();
diff --git a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotSegmentRestletResource.java b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotSegmentRestletResource.java
index 936db77..904eeb3 100644
--- a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotSegmentRestletResource.java
+++ b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotSegmentRestletResource.java
@@ -19,12 +19,14 @@
 package com.linkedin.pinot.controller.api.resources;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.linkedin.pinot.common.config.TableNameBuilder;
 import com.linkedin.pinot.common.metadata.ZKMetadataProvider;
 import com.linkedin.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
 import com.linkedin.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
 import com.linkedin.pinot.common.utils.CommonConstants;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.controller.helix.core.PinotHelixResourceManager;
 import com.linkedin.pinot.controller.helix.core.PinotResourceManagerResponse;
 import io.swagger.annotations.Api;
@@ -52,9 +54,6 @@ import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -127,7 +126,7 @@ public class PinotSegmentRestletResource {
     if (stateStr == null) {
       return getInstanceToSegmentsMap(tableName, tableType);
     }
-    return toggleStateInternal(tableName, state, tableType, null, _pinotHelixResourceManager).toString();
+    return toggleStateInternal(tableName, state, tableType, null, _pinotHelixResourceManager);
   }
 
   @GET
@@ -138,8 +137,7 @@ public class PinotSegmentRestletResource {
       @ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName,
       @ApiParam(value = "Name of segment", required = true) @PathParam("segmentName") @Encoded String segmentName,
       @ApiParam(value = "enable|disable|drop", required = false) @QueryParam("state") String stateStr,
-      @ApiParam(value = "realtime|offline", required = false) @QueryParam("type") String tableTypeStr)
-      throws JSONException {
+      @ApiParam(value = "realtime|offline", required = false) @QueryParam("type") String tableTypeStr) {
     segmentName = checkGetEncodedParam(segmentName);
     // segmentName will never be null,otherwise we would reach the method toggleStateOrListMetadataForAllSegments()
     CommonConstants.Helix.TableType tableType = Constants.validateTableType(tableTypeStr);
@@ -160,7 +158,7 @@ public class PinotSegmentRestletResource {
           tableType = CommonConstants.Helix.TableType.OFFLINE;
         }
       }
-      return toggleStateInternal(tableName, stateType, tableType, segmentName, _pinotHelixResourceManager).toString();
+      return toggleStateInternal(tableName, stateType, tableType, segmentName, _pinotHelixResourceManager);
     }
   }
 
@@ -194,26 +192,25 @@ public class PinotSegmentRestletResource {
   public String listMetadataForOneSegment(
       @ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName,
       @ApiParam(value = "Name of segment", required = true) @PathParam("segmentName") @Encoded String segmentName,
-      @ApiParam(value = "realtime|offline", required = false) @QueryParam("type") String tableTypeStr)
-      throws JSONException {
+      @ApiParam(value = "realtime|offline", required = false) @QueryParam("type") String tableTypeStr) {
     segmentName = checkGetEncodedParam(segmentName);
     CommonConstants.Helix.TableType tableType = Constants.validateTableType(tableTypeStr);
     return listSegmentMetadataInternal(tableName, segmentName, tableType);
   }
 
   private String listSegmentMetadataInternal(@Nonnull String tableName, @Nonnull String segmentName,
-      @Nullable CommonConstants.Helix.TableType tableType) throws JSONException {
+      @Nullable CommonConstants.Helix.TableType tableType) {
     List<String> tableNamesWithType = getExistingTableNamesWithType(tableName, tableType);
 
-    JSONArray result = new JSONArray();
+    ArrayNode result = JsonUtils.newArrayNode();
     for (String tableNameWithType : tableNamesWithType) {
-      JSONArray segmentMetaData = getSegmentMetaData(tableNameWithType, segmentName);
+      ArrayNode segmentMetaData = getSegmentMetaData(tableNameWithType, segmentName);
       if (segmentMetaData != null) {
-        result.put(segmentMetaData);
+        result.add(segmentMetaData);
       }
     }
 
-    if (result.length() == 0) {
+    if (result.size() == 0) {
       String errorMessage = "Failed to find segment: " + segmentName + " in table: " + tableName;
       if (tableType != null) {
         errorMessage += " of type: " + tableType;
@@ -358,9 +355,9 @@ public class PinotSegmentRestletResource {
   }
 
   // TODO: refactor this method using getExistingTableNamesWithType()
-  public static JSONArray toggleStateInternal(@Nonnull String tableName, StateType state,
+  public static String toggleStateInternal(@Nonnull String tableName, StateType state,
       CommonConstants.Helix.TableType tableType, String segmentName, PinotHelixResourceManager helixResourceManager) {
-    JSONArray ret = new JSONArray();
+    ArrayNode ret = JsonUtils.newArrayNode();
     List<String> segmentsToToggle = new ArrayList<>();
     String offlineTableName = TableNameBuilder.OFFLINE.tableNameWithType(tableName);
     String realtimeTableName = TableNameBuilder.REALTIME.tableNameWithType(tableName);
@@ -380,8 +377,8 @@ public class PinotSegmentRestletResource {
       List<PinotResourceManagerResponse> responses = new ArrayList<>();
       responses.add(responseRealtime);
       responses.add(responseOffline);
-      ret.put(responses);
-      return ret;
+      ret.add(JsonUtils.objectToJsonNode(responses));
+      return ret.toString();
     } else if (CommonConstants.Helix.TableType.REALTIME == tableType) {
       if (helixResourceManager.hasRealtimeTable(tableName)) {
         tableNameWithType = realtimeTableName;
@@ -407,8 +404,8 @@ public class PinotSegmentRestletResource {
     }
     PinotResourceManagerResponse resourceManagerResponse =
         toggleSegmentsForTable(segmentsToToggle, tableNameWithType, segmentName, state, helixResourceManager);
-    ret.put(resourceManagerResponse);
-    return ret;
+    ret.add(JsonUtils.objectToJsonNode(resourceManagerResponse));
+    return ret.toString();
   }
 
   /**
@@ -419,8 +416,6 @@ public class PinotSegmentRestletResource {
    * @param tableName: Internal name (created by TableNameBuilder) for the table
    * @param segmentName: Segment to set the state for.
    * @param state: Value of state to set.
-   * @return
-   * @throws JSONException
    */
   // TODO: move this method into PinotHelixResourceManager
   private static PinotResourceManagerResponse toggleSegmentsForTable(@Nonnull List<String> segmentsToToggle,
@@ -469,21 +464,13 @@ public class PinotSegmentRestletResource {
       @Nullable CommonConstants.Helix.TableType tableType) {
     List<String> tableNamesWithType = getExistingTableNamesWithType(tableName, tableType);
 
-    JSONArray result = new JSONArray();
-    ObjectMapper objectMapper = new ObjectMapper();
+    ArrayNode result = JsonUtils.newArrayNode();
     for (String tableNameWithType : tableNamesWithType) {
-      JSONObject resultForTable = new JSONObject();
-      try {
-        resultForTable.put(FileUploadPathProvider.TABLE_NAME, tableNameWithType);
-        // TODO: maybe better to put as json instead of json string
-        resultForTable.put("segments",
-            objectMapper.writeValueAsString(_pinotHelixResourceManager.getServerToSegmentsMap(tableNameWithType)));
-      } catch (JSONException | JsonProcessingException e) {
-        throw new ControllerApplicationException(LOGGER,
-            "Caught JSON exception while getting instance to segments map for table: " + tableNameWithType,
-            Response.Status.INTERNAL_SERVER_ERROR, e);
-      }
-      result.put(resultForTable);
+      ObjectNode resultForTable = JsonUtils.newObjectNode();
+      resultForTable.put(FileUploadPathProvider.TABLE_NAME, tableNameWithType);
+      resultForTable.set("segments",
+          JsonUtils.objectToJsonNode(_pinotHelixResourceManager.getServerToSegmentsMap(tableNameWithType)));
+      result.add(resultForTable);
     }
 
     return result.toString();
@@ -495,37 +482,34 @@ public class PinotSegmentRestletResource {
    * @param tableNameWithType Table name with type suffix
    * @param segmentName Segment name
    * @return Singleton JSON array of the segment metadata
-   * @throws JSONException
    */
-  private JSONArray getSegmentMetaData(String tableNameWithType, String segmentName) throws JSONException {
+  private ArrayNode getSegmentMetaData(String tableNameWithType, String segmentName) {
     ZkHelixPropertyStore<ZNRecord> propertyStore = _pinotHelixResourceManager.getPropertyStore();
     if (!ZKMetadataProvider.isSegmentExisted(propertyStore, tableNameWithType, segmentName)) {
       return null;
     }
 
-    JSONArray ret = new JSONArray();
-    JSONObject jsonObj = new JSONObject();
+    ArrayNode ret = JsonUtils.newArrayNode();
+    ObjectNode jsonObj = JsonUtils.newObjectNode();
     jsonObj.put(TABLE_NAME, tableNameWithType);
 
     if (TableNameBuilder.OFFLINE.tableHasTypeSuffix(tableNameWithType)) {
       // OFFLINE table
       OfflineSegmentZKMetadata offlineSegmentZKMetadata =
           ZKMetadataProvider.getOfflineSegmentZKMetadata(propertyStore, tableNameWithType, segmentName);
-      jsonObj.put(STATE, offlineSegmentZKMetadata.toMap());
+      jsonObj.set(STATE, JsonUtils.objectToJsonNode(offlineSegmentZKMetadata.toMap()));
     } else {
       // REALTIME table
       RealtimeSegmentZKMetadata realtimeSegmentZKMetadata =
           ZKMetadataProvider.getRealtimeSegmentZKMetadata(propertyStore, tableNameWithType, segmentName);
-      jsonObj.put(STATE, realtimeSegmentZKMetadata.toMap());
+      jsonObj.set(STATE, JsonUtils.objectToJsonNode(realtimeSegmentZKMetadata.toMap()));
     }
 
-    ret.put(jsonObj);
+    ret.add(jsonObj);
     return ret;
   }
 
   private String getAllCrcMetadataForTable(String tableName) {
-    PinotResourceManagerResponse response;
-
     // TODO
     // In the restlet.resource version, we see this code block below
     // seems to be wrong comparing the table name to have the table type, but we copy it here anyway.
@@ -541,10 +525,9 @@ public class PinotSegmentRestletResource {
     }
 
     Map<String, String> segmentCrcForTable = _pinotHelixResourceManager.getSegmentsCrcForTable(offlineTableName);
-    ObjectMapper mapper = new ObjectMapper();
-    String result = null;
+    String result;
     try {
-      result = mapper.writerWithDefaultPrettyPrinter().writeValueAsString(segmentCrcForTable);
+      result = JsonUtils.objectToPrettyString(segmentCrcForTable);
     } catch (JsonProcessingException e) {
       throw new ControllerApplicationException(LOGGER,
           String.format("Failed to write segment crc values for table: %s", tableName),
diff --git a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotSegmentUploadRestletResource.java b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotSegmentUploadRestletResource.java
index 50eba13..28e04f9 100644
--- a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotSegmentUploadRestletResource.java
+++ b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotSegmentUploadRestletResource.java
@@ -18,6 +18,8 @@
  */
 package com.linkedin.pinot.controller.api.resources;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
 import com.linkedin.pinot.common.config.TableNameBuilder;
 import com.linkedin.pinot.common.metrics.ControllerMeter;
 import com.linkedin.pinot.common.metrics.ControllerMetrics;
@@ -25,6 +27,7 @@ import com.linkedin.pinot.common.segment.SegmentMetadata;
 import com.linkedin.pinot.common.segment.fetcher.SegmentFetcherFactory;
 import com.linkedin.pinot.common.utils.CommonConstants;
 import com.linkedin.pinot.common.utils.FileUploadDownloadClient;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.StringUtil;
 import com.linkedin.pinot.common.utils.helix.HelixHelper;
 import com.linkedin.pinot.controller.ControllerConf;
@@ -81,8 +84,6 @@ import org.glassfish.grizzly.http.server.Request;
 import org.glassfish.jersey.media.multipart.FormDataBodyPart;
 import org.glassfish.jersey.media.multipart.FormDataMultiPart;
 import org.glassfish.jersey.server.ManagedAsync;
-import org.json.JSONArray;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -118,7 +119,7 @@ public class PinotSegmentUploadRestletResource {
   @Deprecated
   public String listAllSegmentNames() throws Exception {
     FileUploadPathProvider provider = new FileUploadPathProvider(_controllerConf);
-    final JSONArray ret = new JSONArray();
+    ArrayNode ret = JsonUtils.newArrayNode();
     for (final File file : provider.getBaseDataDir().listFiles()) {
       final String fileName = file.getName();
       if (fileName.equalsIgnoreCase("fileUploadTemp") || fileName.equalsIgnoreCase("schemasTemp")) {
@@ -126,7 +127,7 @@ public class PinotSegmentUploadRestletResource {
       }
 
       final String url = _controllerConf.generateVipUrl() + "/segments/" + fileName;
-      ret.put(url);
+      ret.add(url);
     }
     return ret.toString();
   }
@@ -138,14 +139,13 @@ public class PinotSegmentUploadRestletResource {
   public String listAllSegmentNames(
       @ApiParam(value = "Name of the table", required = true) @PathParam("tableName") String tableName,
       @ApiParam(value = "realtime|offline") @QueryParam("type") String tableTypeStr) {
-    JSONArray ret = new JSONArray();
-
+    ArrayNode ret = JsonUtils.newArrayNode();
     CommonConstants.Helix.TableType tableType = Constants.validateTableType(tableTypeStr);
     if (tableTypeStr == null) {
-      ret.put(formatSegments(tableName, CommonConstants.Helix.TableType.OFFLINE));
-      ret.put(formatSegments(tableName, CommonConstants.Helix.TableType.REALTIME));
+      ret.add(formatSegments(tableName, CommonConstants.Helix.TableType.OFFLINE));
+      ret.add(formatSegments(tableName, CommonConstants.Helix.TableType.REALTIME));
     } else {
-      ret.put(formatSegments(tableName, tableType));
+      ret.add(formatSegments(tableName, tableType));
     }
     return ret.toString();
   }
@@ -490,12 +490,12 @@ public class PinotSegmentUploadRestletResource {
     }
   }
 
-  private JSONObject formatSegments(String tableName, CommonConstants.Helix.TableType tableType) {
-    return new JSONObject().put(tableType.toString(), getSegments(tableName, tableType.toString()));
+  private JsonNode formatSegments(String tableName, CommonConstants.Helix.TableType tableType) {
+    return JsonUtils.newObjectNode().set(tableType.toString(), getSegments(tableName, tableType.toString()));
   }
 
-  private JSONArray getSegments(String tableName, String tableType) {
-    JSONArray segments = new JSONArray();
+  private ArrayNode getSegments(String tableName, String tableType) {
+    ArrayNode segments = JsonUtils.newArrayNode();
 
     String realtimeTableName = TableNameBuilder.REALTIME.tableNameWithType(tableName);
     String offlineTableName = TableNameBuilder.OFFLINE.tableNameWithType(tableName);
@@ -517,7 +517,7 @@ public class PinotSegmentUploadRestletResource {
         continue;
       }
       if (!map.containsValue(PinotHelixSegmentOnlineOfflineStateModelGenerator.OFFLINE_STATE)) {
-        segments.put(segmentName);
+        segments.add(segmentName);
       }
     }
 
diff --git a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableIndexingConfigs.java b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableIndexingConfigs.java
index 4fbfddd..a2579dd 100644
--- a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableIndexingConfigs.java
+++ b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableIndexingConfigs.java
@@ -33,8 +33,6 @@ import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-import org.json.JSONException;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -60,13 +58,12 @@ public class PinotTableIndexingConfigs {
       @ApiParam(value = "Table name (without type)", required = true) @PathParam("tableName") String tableName,
       String body
   ) {
-    TableConfig tableConfig = null;
     try {
-      tableConfig = TableConfig.fromJSONConfig(new JSONObject(body));
+      TableConfig tableConfig = TableConfig.fromJsonString(body);
       pinotHelixResourceManager.updateIndexingConfigFor(tableConfig.getTableName(), tableConfig.getTableType(),
           tableConfig.getIndexingConfig());
       return new SuccessResponse("Updated indexing config for table " + tableName);
-    } catch (JSONException | IOException e) {
+    } catch (IOException e) {
       String errStr = "Error converting request to table config for table: " + tableName;
       throw new ControllerApplicationException(LOGGER, errStr, Response.Status.BAD_REQUEST, e);
     } catch (Exception e) {
diff --git a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableInstances.java b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableInstances.java
index 707578e..4fb1a10 100644
--- a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableInstances.java
+++ b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableInstances.java
@@ -18,15 +18,16 @@
  */
 package com.linkedin.pinot.controller.api.resources;
 
-import com.alibaba.fastjson.JSONArray;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.linkedin.pinot.common.utils.CommonConstants.Helix.TableType;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.controller.helix.core.PinotHelixResourceManager;
 import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiParam;
 import io.swagger.annotations.ApiResponse;
 import io.swagger.annotations.ApiResponses;
-import java.util.List;
 import javax.inject.Inject;
 import javax.ws.rs.DefaultValue;
 import javax.ws.rs.GET;
@@ -35,17 +36,11 @@ import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
 import javax.ws.rs.QueryParam;
 import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
-import org.json.JSONException;
-import org.json.JSONObject;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 
 @Api(tags = Constants.TABLE_TAG)
 @Path("/")
 public class PinotTableInstances {
-  private static final Logger LOGGER = LoggerFactory.getLogger(PinotTableInstances.class);
 
   @Inject
   PinotHelixResourceManager pinotHelixResourceManager;
@@ -59,80 +54,60 @@ public class PinotTableInstances {
       @ApiResponse(code = 500, message = "Internal server error")})
   public String getTableInstances(
       @ApiParam(value = "Table name without type", required = true) @PathParam("tableName") String tableName,
-      @ApiParam(value = "Instance type", required = false, example = "broker", allowableValues = "BROKER, SERVER")
-          @DefaultValue("") @QueryParam("type") String type
-  ) {
-    try {
-      JSONObject ret = new JSONObject();
-      ret.put("tableName", tableName);
-      JSONArray brokers = new JSONArray();
-      JSONArray servers = new JSONArray();
+      @ApiParam(value = "Instance type", example = "broker", allowableValues = "BROKER, SERVER") @DefaultValue("") @QueryParam("type") String type) {
+    ObjectNode ret = JsonUtils.newObjectNode();
+    ret.put("tableName", tableName);
+    ArrayNode brokers = JsonUtils.newArrayNode();
+    ArrayNode servers = JsonUtils.newArrayNode();
 
-      if (type == null || type.isEmpty() || type.toLowerCase().equals("broker")) {
-        if (pinotHelixResourceManager.hasOfflineTable(tableName)) {
-          JSONObject e = new JSONObject();
-          e.put("tableType", "offline");
-          JSONArray a = new JSONArray();
-          for (String ins : pinotHelixResourceManager.getBrokerInstancesForTable(tableName, TableType.OFFLINE)) {
-            a.add(ins);
-          }
-          e.put("instances", a);
-          brokers.add(e);
+    if (type == null || type.isEmpty() || type.toLowerCase().equals("broker")) {
+      if (pinotHelixResourceManager.hasOfflineTable(tableName)) {
+        ObjectNode e = JsonUtils.newObjectNode();
+        e.put("tableType", "offline");
+        ArrayNode a = JsonUtils.newArrayNode();
+        for (String ins : pinotHelixResourceManager.getBrokerInstancesForTable(tableName, TableType.OFFLINE)) {
+          a.add(ins);
         }
-        if (pinotHelixResourceManager.hasRealtimeTable(tableName)) {
-          JSONObject e = new JSONObject();
-          e.put("tableType", "realtime");
-          JSONArray a = new JSONArray();
-          for (String ins : pinotHelixResourceManager.getBrokerInstancesForTable(tableName, TableType.REALTIME)) {
-            a.add(ins);
-          }
-          e.put("instances", a);
-          brokers.add(e);
+        e.set("instances", a);
+        brokers.add(e);
+      }
+      if (pinotHelixResourceManager.hasRealtimeTable(tableName)) {
+        ObjectNode e = JsonUtils.newObjectNode();
+        e.put("tableType", "realtime");
+        ArrayNode a = JsonUtils.newArrayNode();
+        for (String ins : pinotHelixResourceManager.getBrokerInstancesForTable(tableName, TableType.REALTIME)) {
+          a.add(ins);
         }
+        e.set("instances", a);
+        brokers.add(e);
       }
+    }
 
-      if (type == null || type.isEmpty() || type.toLowerCase().equals("server")) {
-        if (pinotHelixResourceManager.hasOfflineTable(tableName)) {
-          JSONObject e = new JSONObject();
-          e.put("tableType", "offline");
-          JSONArray a = new JSONArray();
-          for (String ins : pinotHelixResourceManager.getServerInstancesForTable(tableName, TableType.OFFLINE)) {
-            a.add(ins);
-          }
-          e.put("instances", a);
-          servers.add(e);
+    if (type == null || type.isEmpty() || type.toLowerCase().equals("server")) {
+      if (pinotHelixResourceManager.hasOfflineTable(tableName)) {
+        ObjectNode e = JsonUtils.newObjectNode();
+        e.put("tableType", "offline");
+        ArrayNode a = JsonUtils.newArrayNode();
+        for (String ins : pinotHelixResourceManager.getServerInstancesForTable(tableName, TableType.OFFLINE)) {
+          a.add(ins);
         }
+        e.set("instances", a);
+        servers.add(e);
+      }
 
-        if (pinotHelixResourceManager.hasRealtimeTable(tableName)) {
-          JSONObject e = new JSONObject();
-          e.put("tableType", "realtime");
-          JSONArray a = new JSONArray();
-          for (String ins : pinotHelixResourceManager.getServerInstancesForTable(tableName, TableType.REALTIME)) {
-            a.add(ins);
-          }
-          e.put("instances", a);
-          servers.add(e);
+      if (pinotHelixResourceManager.hasRealtimeTable(tableName)) {
+        ObjectNode e = JsonUtils.newObjectNode();
+        e.put("tableType", "realtime");
+        ArrayNode a = JsonUtils.newArrayNode();
+        for (String ins : pinotHelixResourceManager.getServerInstancesForTable(tableName, TableType.REALTIME)) {
+          a.add(ins);
         }
+        e.set("instances", a);
+        servers.add(e);
       }
-      ret.put("brokers", brokers);
-      ret.put("server", servers);   // Keeping compatibility with previous API, so "server" and "brokers"
-      return ret.toString();
-    } catch (JSONException e) {
-      String errStr = "Error listing all table instances for table: " + tableName;
-      throw new ControllerApplicationException(LOGGER, errStr, Response.Status.INTERNAL_SERVER_ERROR, e);
-    }
-  }
-    private JSONObject getInstances(List<String> instanceList, TableType tableType)
-      throws JSONException {
-    JSONObject e = new JSONObject();
-    // not sure how using enum toString will impact clients
-    String typeStr = tableType==TableType.REALTIME ? "realtime" : "offline";
-    e.put("tableType", typeStr);
-    JSONArray a = new JSONArray();
-    for (String ins : instanceList) {
-      a.add(ins);
     }
-    e.put("instances", a);
-    return e;
+    ret.set("brokers", brokers);
+    ret.set("server", servers);   // Keeping compatibility with previous API, so "server" and "brokers"
+    return ret.toString();
   }
 }
diff --git a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableMetadataConfigs.java b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableMetadataConfigs.java
index 2391116..fdff7b6 100644
--- a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableMetadataConfigs.java
+++ b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableMetadataConfigs.java
@@ -31,7 +31,6 @@ import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -54,7 +53,7 @@ public class PinotTableMetadataConfigs {
   public SuccessResponse updateTableMetadata(@PathParam("tableName")String tableName, String requestBody
   ) {
     try {
-      TableConfig tableConfig = TableConfig.fromJSONConfig(new JSONObject(requestBody));
+      TableConfig tableConfig = TableConfig.fromJsonString(requestBody);
       pinotHelixResourceManager.updateMetadataConfigFor(tableConfig.getTableName(), tableConfig.getTableType(),
         tableConfig.getCustomConfig());
       return new SuccessResponse("Successfully updated " + tableName + " configuration");
diff --git a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableRestletResource.java b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableRestletResource.java
index b1c18bc..d319510 100644
--- a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableRestletResource.java
+++ b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableRestletResource.java
@@ -18,6 +18,8 @@
  */
 package com.linkedin.pinot.controller.api.resources;
 
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.google.common.base.Preconditions;
 import com.linkedin.pinot.common.config.SegmentsValidationAndRetentionConfig;
 import com.linkedin.pinot.common.config.TableConfig;
@@ -30,6 +32,7 @@ import com.linkedin.pinot.common.restlet.resources.RebalanceResult;
 import com.linkedin.pinot.common.restlet.resources.ResourceUtils;
 import com.linkedin.pinot.common.utils.CommonConstants;
 import com.linkedin.pinot.common.utils.CommonConstants.Helix.TableType;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.controller.ControllerConf;
 import com.linkedin.pinot.controller.helix.core.PinotHelixResourceManager;
 import com.linkedin.pinot.controller.helix.core.PinotResourceManagerResponse;
@@ -38,7 +41,6 @@ import com.linkedin.pinot.core.realtime.stream.StreamConfig;
 import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiParam;
-import java.io.IOException;
 import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
@@ -60,9 +62,6 @@ import javax.ws.rs.core.Response;
 import org.apache.commons.configuration.Configuration;
 import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.commons.lang3.EnumUtils;
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
 import org.slf4j.LoggerFactory;
 
 
@@ -106,15 +105,14 @@ public class PinotTableRestletResource {
   @Produces(MediaType.APPLICATION_JSON)
   @Path("/tables")
   @ApiOperation(value = "Adds a table", notes = "Adds a table")
-  public SuccessResponse addTable(String tableConfigStr) throws Exception {
+  public SuccessResponse addTable(String tableConfigStr) {
     // TODO introduce a table config ctor with json string.
     TableConfig tableConfig;
     String tableName;
     try {
-      JSONObject tableConfigJson = new JSONObject(tableConfigStr);
-      tableConfig = TableConfig.fromJSONConfig(tableConfigJson);
+      tableConfig = TableConfig.fromJsonString(tableConfigStr);
       tableName = tableConfig.getTableName();
-    } catch (IOException | JSONException | IllegalArgumentException e) {
+    } catch (Exception e) {
       throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.BAD_REQUEST, e);
     }
     try {
@@ -160,10 +158,7 @@ public class PinotTableRestletResource {
       }
 
       Collections.sort(tableNames);
-      JSONArray tableArray = new JSONArray(tableNames);
-      JSONObject resultObject = new JSONObject();
-      resultObject.put("tables", tableArray);
-      return resultObject.toString();
+      return JsonUtils.newObjectNode().set("tables", JsonUtils.objectToJsonNode(tableNames)).toString();
     } catch (Exception e) {
       throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.INTERNAL_SERVER_ERROR, e);
     }
@@ -171,20 +166,20 @@ public class PinotTableRestletResource {
 
   private String listTableConfigs(@Nonnull String tableName, @Nullable String tableTypeStr) {
     try {
-      JSONObject ret = new JSONObject();
+      ObjectNode ret = JsonUtils.newObjectNode();
 
       if ((tableTypeStr == null || CommonConstants.Helix.TableType.OFFLINE.name().equalsIgnoreCase(tableTypeStr))
           && _pinotHelixResourceManager.hasOfflineTable(tableName)) {
         TableConfig tableConfig = _pinotHelixResourceManager.getOfflineTableConfig(tableName);
         Preconditions.checkNotNull(tableConfig);
-        ret.put(CommonConstants.Helix.TableType.OFFLINE.name(), TableConfig.toJSONConfig(tableConfig));
+        ret.set(CommonConstants.Helix.TableType.OFFLINE.name(), TableConfig.toJSONConfig(tableConfig));
       }
 
       if ((tableTypeStr == null || CommonConstants.Helix.TableType.REALTIME.name().equalsIgnoreCase(tableTypeStr))
           && _pinotHelixResourceManager.hasRealtimeTable(tableName)) {
         TableConfig tableConfig = _pinotHelixResourceManager.getRealtimeTableConfig(tableName);
         Preconditions.checkNotNull(tableConfig);
-        ret.put(CommonConstants.Helix.TableType.REALTIME.name(), TableConfig.toJSONConfig(tableConfig));
+        ret.set(CommonConstants.Helix.TableType.REALTIME.name(), TableConfig.toJSONConfig(tableConfig));
       }
       return ret.toString();
     } catch (Exception e) {
@@ -207,29 +202,29 @@ public class PinotTableRestletResource {
         return listTableConfigs(tableName, tableTypeStr);
       }
       Constants.validateState(stateStr);
-      JSONArray ret = new JSONArray();
+      ArrayNode ret = JsonUtils.newArrayNode();
       boolean tableExists = false;
 
       if ((tableTypeStr == null || CommonConstants.Helix.TableType.OFFLINE.name().equalsIgnoreCase(tableTypeStr))
           && _pinotHelixResourceManager.hasOfflineTable(tableName)) {
         String offlineTableName = TableNameBuilder.OFFLINE.tableNameWithType(tableName);
-        JSONObject offline = new JSONObject();
+        ObjectNode offline = JsonUtils.newObjectNode();
         tableExists = true;
 
         offline.put(FileUploadPathProvider.TABLE_NAME, offlineTableName);
-        offline.put(FileUploadPathProvider.STATE, toggleTableState(offlineTableName, stateStr));
-        ret.put(offline);
+        offline.set(FileUploadPathProvider.STATE, JsonUtils.objectToJsonNode(toggleTableState(offlineTableName, stateStr)));
+        ret.add(offline);
       }
 
       if ((tableTypeStr == null || CommonConstants.Helix.TableType.REALTIME.name().equalsIgnoreCase(tableTypeStr))
           && _pinotHelixResourceManager.hasRealtimeTable(tableName)) {
         String realTimeTableName = TableNameBuilder.REALTIME.tableNameWithType(tableName);
-        JSONObject realTime = new JSONObject();
+        ObjectNode realTime = JsonUtils.newObjectNode();
         tableExists = true;
 
         realTime.put(FileUploadPathProvider.TABLE_NAME, realTimeTableName);
-        realTime.put(FileUploadPathProvider.STATE, toggleTableState(realTimeTableName, stateStr));
-        ret.put(realTime);
+        realTime.set(FileUploadPathProvider.STATE, JsonUtils.objectToJsonNode(toggleTableState(realTimeTableName, stateStr)));
+        ret.add(realTime);
       }
       if (tableExists) {
         return ret.toString();
@@ -274,8 +269,7 @@ public class PinotTableRestletResource {
       String tableConfigStr) throws Exception {
     TableConfig tableConfig;
     try {
-      JSONObject tableConfigJson = new JSONObject(tableConfigStr);
-      tableConfig = TableConfig.fromJSONConfig(tableConfigJson);
+      tableConfig = TableConfig.fromJsonString(tableConfigStr);
     } catch (Exception e) {
       throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.BAD_REQUEST);
     }
@@ -323,15 +317,15 @@ 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(String tableConfigStr) throws Exception {
+  public String checkTableConfig(String tableConfigStr) {
     try {
-      JSONObject tableConfigValidateStr = new JSONObject();
-      TableConfig tableConfig = TableConfig.fromJSONConfig(new JSONObject(tableConfigStr));
+      ObjectNode tableConfigValidateStr = JsonUtils.newObjectNode();
+      TableConfig tableConfig = TableConfig.fromJsonString(tableConfigStr);
       if (tableConfig.getTableType() == CommonConstants.Helix.TableType.OFFLINE) {
-        tableConfigValidateStr.put(CommonConstants.Helix.TableType.OFFLINE.name(),
+        tableConfigValidateStr.set(CommonConstants.Helix.TableType.OFFLINE.name(),
             TableConfig.toJSONConfig(tableConfig));
       } else {
-        tableConfigValidateStr.put(CommonConstants.Helix.TableType.REALTIME.name(),
+        tableConfigValidateStr.set(CommonConstants.Helix.TableType.REALTIME.name(),
             TableConfig.toJSONConfig(tableConfig));
       }
       return tableConfigValidateStr.toString();
diff --git a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableSegmentConfigs.java b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableSegmentConfigs.java
index fb28b99..3c9a68e 100644
--- a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableSegmentConfigs.java
+++ b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTableSegmentConfigs.java
@@ -27,6 +27,7 @@ import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiParam;
 import io.swagger.annotations.ApiResponse;
 import io.swagger.annotations.ApiResponses;
+import java.io.IOException;
 import javax.inject.Inject;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
@@ -34,8 +35,6 @@ import javax.ws.rs.PathParam;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-import org.json.JSONException;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -64,11 +63,11 @@ public class PinotTableSegmentConfigs {
       String requestBody
   ) {
     try {
-      TableConfig tableConfig = TableConfig.fromJSONConfig(new JSONObject(requestBody));
+      TableConfig tableConfig = TableConfig.fromJsonString(requestBody);
       pinotHelixResourceManager.updateSegmentsValidationAndRetentionConfigFor(tableConfig.getTableName(),
           tableConfig.getTableType(), tableConfig.getValidationConfig());
       return new SuccessResponse("Update segmentsConfig for table: " + tableName);
-    } catch (JSONException e) {
+    } catch (IOException e) {
       metrics.addMeteredGlobalValue(ControllerMeter.CONTROLLER_TABLE_SCHEMA_UPDATE_ERROR, 1L);
       throw new ControllerApplicationException(LOGGER,
           String.format("Invalid json while updating segments config for table: %s", tableName),
diff --git a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTenantRestletResource.java b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTenantRestletResource.java
index b0e4d3b..34e19ca 100644
--- a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTenantRestletResource.java
+++ b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotTenantRestletResource.java
@@ -20,11 +20,12 @@ package com.linkedin.pinot.controller.api.resources;
 
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.linkedin.pinot.common.config.TableConfig;
 import com.linkedin.pinot.common.config.Tenant;
 import com.linkedin.pinot.common.metrics.ControllerMeter;
 import com.linkedin.pinot.common.metrics.ControllerMetrics;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.TenantRole;
 import com.linkedin.pinot.controller.helix.core.PinotHelixResourceManager;
 import com.linkedin.pinot.controller.helix.core.PinotResourceManagerResponse;
@@ -49,8 +50,6 @@ import javax.ws.rs.Produces;
 import javax.ws.rs.QueryParam;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-import org.json.JSONException;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -220,10 +219,9 @@ public class PinotTenantRestletResource {
     return getTablesServedFromTenant(tenantName);
   }
 
-  private String getTablesServedFromTenant(String tenantName)
-      throws JSONException {
-    Set<String> tables = new HashSet<String>();
-    JSONObject resourceGetRet = new JSONObject();
+  private String getTablesServedFromTenant(String tenantName) {
+    Set<String> tables = new HashSet<>();
+    ObjectNode resourceGetRet = JsonUtils.newObjectNode();
 
     for (String table : pinotHelixResourceManager.getAllTables()) {
       TableConfig tableConfig = pinotHelixResourceManager.getTableConfig(table);
@@ -233,14 +231,14 @@ public class PinotTenantRestletResource {
       }
     }
 
-    resourceGetRet.put(TABLES, tables);
+    resourceGetRet.set(TABLES, JsonUtils.objectToJsonNode(tables));
     return resourceGetRet.toString();
   }
 
-  private String toggleTenantState(String tenantName, String stateStr, @Nullable String tenantType) throws JSONException{
-    Set<String> serverInstances = new HashSet<String>();
-    Set<String> brokerInstances = new HashSet<String>();
-    JSONObject instanceResult = new JSONObject();
+  private String toggleTenantState(String tenantName, String stateStr, @Nullable String tenantType) {
+    Set<String> serverInstances = new HashSet<>();
+    Set<String> brokerInstances = new HashSet<>();
+    ObjectNode instanceResult = JsonUtils.newObjectNode();
 
     if ((tenantType == null) || tenantType.equalsIgnoreCase("server")) {
       serverInstances = pinotHelixResourceManager.getAllInstancesForServerTenant(tenantName);
@@ -267,27 +265,30 @@ public class PinotTenantRestletResource {
     boolean enable = StateType.ENABLE.name().equalsIgnoreCase(stateStr) ? true : false;
     for (String instance : allInstances) {
       if (enable) {
-        instanceResult.put(instance, pinotHelixResourceManager.enableInstance(instance));
+        instanceResult.put(instance, JsonUtils.objectToJsonNode(pinotHelixResourceManager.enableInstance(instance)));
       } else {
-        instanceResult.put(instance, pinotHelixResourceManager.disableInstance(instance));
+        instanceResult.put(instance, JsonUtils.objectToJsonNode(pinotHelixResourceManager.disableInstance(instance)));
       }
     }
 
     return null;
   }
 
-  private String listInstancesForTenant(String tenantName, String tenantType)
-      throws JSONException {
-    JSONObject resourceGetRet = new JSONObject();
+  private String listInstancesForTenant(String tenantName, String tenantType) {
+    ObjectNode resourceGetRet = JsonUtils.newObjectNode();
     if (tenantType == null) {
-      resourceGetRet.put("ServerInstances", pinotHelixResourceManager.getAllInstancesForServerTenant(tenantName));
-      resourceGetRet.put("BrokerInstances", pinotHelixResourceManager.getAllInstancesForBrokerTenant(tenantName));
+      resourceGetRet.set("ServerInstances",
+          JsonUtils.objectToJsonNode(pinotHelixResourceManager.getAllInstancesForServerTenant(tenantName)));
+      resourceGetRet.set("BrokerInstances",
+          JsonUtils.objectToJsonNode(pinotHelixResourceManager.getAllInstancesForBrokerTenant(tenantName)));
     } else {
       if (tenantType.equalsIgnoreCase("server")) {
-        resourceGetRet.put("ServerInstances", pinotHelixResourceManager.getAllInstancesForServerTenant(tenantName));
+        resourceGetRet.set("ServerInstances",
+            JsonUtils.objectToJsonNode(pinotHelixResourceManager.getAllInstancesForServerTenant(tenantName)));
       }
       if (tenantType.equalsIgnoreCase("broker")) {
-        resourceGetRet.put("BrokerInstances", pinotHelixResourceManager.getAllInstancesForBrokerTenant(tenantName));
+        resourceGetRet.set("BrokerInstances",
+            JsonUtils.objectToJsonNode(pinotHelixResourceManager.getAllInstancesForBrokerTenant(tenantName)));
       }
     }
     resourceGetRet.put(TENANT_NAME, tenantName);
@@ -374,7 +375,7 @@ public class PinotTenantRestletResource {
       pinotHelixResourceManager.deleteOfflineServerTenantFor(tenantName);
       pinotHelixResourceManager.deleteRealtimeServerTenantFor(tenantName);
       try {
-        return new ObjectMapper().writeValueAsString(new SuccessResponse("Deleted tenant " + tenantName));
+        return JsonUtils.objectToString(new SuccessResponse("Deleted tenant " + tenantName));
       } catch (JsonProcessingException e) {
          LOGGER.error("Error serializing response to json");
         return "{\"message\" : \"Deleted tenant\" " + tenantName + "}";
@@ -382,15 +383,15 @@ public class PinotTenantRestletResource {
     }
 
     boolean enable = StateType.ENABLE.name().equalsIgnoreCase(state) ? true : false;
-    JSONObject instanceResult = new JSONObject();
+    ObjectNode instanceResult = JsonUtils.newObjectNode();
     String instance = null;
     try {
       for (String i : allInstances) {
         instance = i;
         if (enable) {
-          instanceResult.put(instance, pinotHelixResourceManager.enableInstance(instance));
+          instanceResult.set(instance, JsonUtils.objectToJsonNode(pinotHelixResourceManager.enableInstance(instance)));
         } else {
-          instanceResult.put(instance, pinotHelixResourceManager.disableInstance(instance));
+          instanceResult.set(instance, JsonUtils.objectToJsonNode(pinotHelixResourceManager.disableInstance(instance)));
         }
       }
     } catch (Exception e) {
diff --git a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotVersionRestletResource.java b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotVersionRestletResource.java
index 4875036..9e24ab3 100644
--- a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotVersionRestletResource.java
+++ b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PinotVersionRestletResource.java
@@ -18,7 +18,9 @@
  */
 package com.linkedin.pinot.controller.api.resources;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
 import com.linkedin.pinot.common.Utils;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiResponse;
@@ -27,7 +29,6 @@ import javax.ws.rs.GET;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
-import org.json.JSONObject;
 
 
 /**
@@ -42,7 +43,10 @@ public class PinotVersionRestletResource {
   @ApiOperation(value = "Get version number of Pinot components")
   @ApiResponses(value = {@ApiResponse(code = 200, message = "Success")})
   public String getVersionNumber() {
-    JSONObject jsonObject = new JSONObject(Utils.getComponentVersions());
-    return jsonObject.toString();
+    try {
+      return JsonUtils.objectToString(Utils.getComponentVersions());
+    } catch (JsonProcessingException e) {
+      throw new RuntimeException(e);
+    }
   }
 }
diff --git a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PqlQueryResource.java b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PqlQueryResource.java
index d9d9bb5..dba3b85 100644
--- a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PqlQueryResource.java
+++ b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/PqlQueryResource.java
@@ -18,10 +18,13 @@
  */
 package com.linkedin.pinot.controller.api.resources;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.linkedin.pinot.common.Utils;
 import com.linkedin.pinot.common.config.TableNameBuilder;
 import com.linkedin.pinot.common.exception.QueryException;
 import com.linkedin.pinot.common.request.BrokerRequest;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.controller.api.access.AccessControl;
 import com.linkedin.pinot.controller.api.access.AccessControlFactory;
 import com.linkedin.pinot.controller.helix.core.PinotHelixResourceManager;
@@ -47,7 +50,6 @@ import javax.ws.rs.QueryParam;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.HttpHeaders;
 import org.apache.helix.model.InstanceConfig;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -68,8 +70,8 @@ public class PqlQueryResource {
   @Path("pql")
   public String post(String requestJsonStr, @Context HttpHeaders httpHeaders) {
     try {
-      JSONObject requestJson = new JSONObject(requestJsonStr);
-      String pqlQuery = requestJson.getString("pql");
+      JsonNode requestJson = JsonUtils.stringToJsonNode(requestJsonStr);
+      String pqlQuery = requestJson.get("pql").asText();
       String traceEnabled = "false";
       if (requestJson.has("trace")) {
         traceEnabled = requestJson.get("trace").toString();
@@ -216,7 +218,7 @@ public class PqlQueryResource {
   public String sendPQLRaw(String url, String pqlRequest, String traceEnabled) {
     try {
       final long startTime = System.currentTimeMillis();
-      final JSONObject bqlJson = new JSONObject().put("pql", pqlRequest);
+      ObjectNode bqlJson = JsonUtils.newObjectNode().put("pql", pqlRequest);
       if (traceEnabled != null && !traceEnabled.isEmpty()) {
         bqlJson.put("trace", traceEnabled);
       }
diff --git a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/ServerTableSizeReader.java b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/ServerTableSizeReader.java
index 545a53d..57963a6 100644
--- a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/ServerTableSizeReader.java
+++ b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/ServerTableSizeReader.java
@@ -22,6 +22,7 @@ import com.google.common.collect.BiMap;
 import com.linkedin.pinot.common.http.MultiGetRequest;
 import com.linkedin.pinot.common.restlet.resources.SegmentSizeInfo;
 import com.linkedin.pinot.common.restlet.resources.TableSizeInfo;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -31,7 +32,6 @@ import java.util.concurrent.Executor;
 import org.apache.commons.httpclient.HttpConnectionManager;
 import org.apache.commons.httpclient.URI;
 import org.apache.commons.httpclient.methods.GetMethod;
-import org.codehaus.jackson.map.ObjectMapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -42,7 +42,6 @@ import org.slf4j.LoggerFactory;
  */
 public class ServerTableSizeReader {
   private static final Logger LOGGER = LoggerFactory.getLogger(ServerTableSizeReader.class);
-  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
 
   private final Executor _executor;
   private final HttpConnectionManager _connectionManager;
@@ -80,7 +79,8 @@ public class ServerTableSizeReader {
           LOGGER.error("Server: {} returned error: {}", instance, getMethod.getStatusCode());
           continue;
         }
-        TableSizeInfo tableSizeInfo = OBJECT_MAPPER.readValue(getMethod.getResponseBodyAsStream(), TableSizeInfo.class);
+        TableSizeInfo tableSizeInfo =
+            JsonUtils.inputStreamToObject(getMethod.getResponseBodyAsStream(), TableSizeInfo.class);
         serverToSegmentSizeInfoListMap.put(instance, tableSizeInfo.segments);
       } catch (Exception e) {
         // Ignore individual exceptions because the exception has been logged in MultiGetRequest
diff --git a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/WebApplicationExceptionMapper.java b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/WebApplicationExceptionMapper.java
index d4d1cb6..765c289 100644
--- a/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/WebApplicationExceptionMapper.java
+++ b/pinot-controller/src/main/java/com/linkedin/pinot/controller/api/resources/WebApplicationExceptionMapper.java
@@ -21,7 +21,7 @@ package com.linkedin.pinot.controller.api.resources;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
@@ -33,7 +33,6 @@ import org.slf4j.LoggerFactory;
 @Provider
 public class WebApplicationExceptionMapper implements ExceptionMapper<Throwable> {
   private static final Logger LOGGER = LoggerFactory.getLogger(WebApplicationExceptionMapper.class);
-  private static final ObjectMapper JSON_MAPPER = new ObjectMapper();
 
   @Override
   public Response toResponse(Throwable t) {
@@ -46,14 +45,10 @@ public class WebApplicationExceptionMapper implements ExceptionMapper<Throwable>
 
     ErrorInfo einfo = new ErrorInfo(status, t.getMessage());
     try {
-      return Response.status(status).entity(JSON_MAPPER.writeValueAsString(einfo))
-          .type(MediaType.APPLICATION_JSON)
-          .build();
+      return Response.status(status).entity(JsonUtils.objectToString(einfo)).type(MediaType.APPLICATION_JSON).build();
     } catch (JsonProcessingException e) {
       String err = String.format("{\"status\":%d, \"error\":%s}", einfo.code, einfo.error);
-      return Response.status(status).entity(err)
-          .type(MediaType.APPLICATION_JSON)
-          .build();
+      return Response.status(status).entity(err).type(MediaType.APPLICATION_JSON).build();
     }
   }
 
diff --git a/pinot-controller/src/main/java/com/linkedin/pinot/controller/helix/ControllerRequestBuilderUtil.java b/pinot-controller/src/main/java/com/linkedin/pinot/controller/helix/ControllerRequestBuilderUtil.java
index 146ac02..5a01f7c 100644
--- a/pinot-controller/src/main/java/com/linkedin/pinot/controller/helix/ControllerRequestBuilderUtil.java
+++ b/pinot-controller/src/main/java/com/linkedin/pinot/controller/helix/ControllerRequestBuilderUtil.java
@@ -18,11 +18,13 @@
  */
 package com.linkedin.pinot.controller.helix;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
 import com.linkedin.pinot.common.config.TableNameBuilder;
 import com.linkedin.pinot.common.config.TagNameUtils;
 import com.linkedin.pinot.common.config.Tenant;
 import com.linkedin.pinot.common.config.Tenant.TenantBuilder;
 import com.linkedin.pinot.common.utils.CommonConstants;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.TenantRole;
 import java.util.HashMap;
 import java.util.Map;
@@ -33,11 +35,8 @@ import org.apache.helix.model.HelixConfigScope;
 import org.apache.helix.model.builder.HelixConfigScopeBuilder;
 import org.apache.helix.participant.StateMachineEngine;
 import org.apache.helix.participant.statemachine.StateModelFactory;
-import org.json.JSONException;
-import org.json.JSONObject;
 
-import static com.linkedin.pinot.common.utils.CommonConstants.Helix.UNTAGGED_BROKER_INSTANCE;
-import static com.linkedin.pinot.common.utils.CommonConstants.Helix.UNTAGGED_SERVER_INSTANCE;
+import static com.linkedin.pinot.common.utils.CommonConstants.Helix.*;
 
 
 public class ControllerRequestBuilderUtil {
@@ -130,23 +129,23 @@ public class ControllerRequestBuilderUtil {
     helixZkManager.getClusterManagmentTool().setConfig(scope, props);
   }
 
-  public static JSONObject buildBrokerTenantCreateRequestJSON(String tenantName, int numberOfInstances)
-      throws JSONException {
+  public static String buildBrokerTenantCreateRequestJSON(String tenantName, int numberOfInstances)
+      throws JsonProcessingException {
     Tenant tenant = new TenantBuilder(tenantName).setRole(TenantRole.BROKER)
         .setTotalInstances(numberOfInstances)
         .setOfflineInstances(0)
         .setRealtimeInstances(0)
         .build();
-    return tenant.toJSON();
+    return JsonUtils.objectToString(tenant);
   }
 
-  public static JSONObject buildServerTenantCreateRequestJSON(String tenantName, int numberOfInstances,
-      int offlineInstances, int realtimeInstances) throws JSONException {
+  public static String buildServerTenantCreateRequestJSON(String tenantName, int numberOfInstances,
+      int offlineInstances, int realtimeInstances) throws JsonProcessingException {
     Tenant tenant = new TenantBuilder(tenantName).setRole(TenantRole.SERVER)
         .setTotalInstances(numberOfInstances)
         .setOfflineInstances(offlineInstances)
         .setRealtimeInstances(realtimeInstances)
         .build();
-    return tenant.toJSON();
+    return JsonUtils.objectToString(tenant);
   }
 }
diff --git a/pinot-controller/src/main/java/com/linkedin/pinot/controller/helix/core/realtime/PinotRealtimeSegmentManager.java b/pinot-controller/src/main/java/com/linkedin/pinot/controller/helix/core/realtime/PinotRealtimeSegmentManager.java
index 51bf0e2..5d8ae5b 100644
--- a/pinot-controller/src/main/java/com/linkedin/pinot/controller/helix/core/realtime/PinotRealtimeSegmentManager.java
+++ b/pinot-controller/src/main/java/com/linkedin/pinot/controller/helix/core/realtime/PinotRealtimeSegmentManager.java
@@ -35,13 +35,12 @@ import com.linkedin.pinot.common.utils.HLCSegmentName;
 import com.linkedin.pinot.common.utils.SegmentName;
 import com.linkedin.pinot.common.utils.helix.HelixHelper;
 import com.linkedin.pinot.common.utils.retry.RetryPolicies;
-import com.linkedin.pinot.controller.LeadershipChangeSubscriber;
 import com.linkedin.pinot.controller.ControllerLeadershipManager;
+import com.linkedin.pinot.controller.LeadershipChangeSubscriber;
 import com.linkedin.pinot.controller.helix.core.PinotHelixResourceManager;
 import com.linkedin.pinot.controller.helix.core.PinotTableIdealStateBuilder;
 import com.linkedin.pinot.core.query.utils.Pair;
 import com.linkedin.pinot.core.realtime.stream.StreamConfig;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -60,7 +59,6 @@ import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.store.HelixPropertyListener;
 import org.apache.zookeeper.data.Stat;
-import org.json.JSONException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -115,8 +113,7 @@ public class PinotRealtimeSegmentManager implements HelixPropertyListener, IZkCh
     _pinotHelixResourceManager.getPropertyStore().stop();
   }
 
-  private synchronized void assignRealtimeSegmentsToServerInstancesIfNecessary()
-      throws JSONException, IOException {
+  private synchronized void assignRealtimeSegmentsToServerInstancesIfNecessary() {
     // Fetch current ideal state snapshot
     Map<String, IdealState> idealStateMap = new HashMap<>();
 
diff --git a/pinot-controller/src/main/java/com/linkedin/pinot/controller/util/AutoAddInvertedIndex.java b/pinot-controller/src/main/java/com/linkedin/pinot/controller/util/AutoAddInvertedIndex.java
index a4ded85..7901c07 100644
--- a/pinot-controller/src/main/java/com/linkedin/pinot/controller/util/AutoAddInvertedIndex.java
+++ b/pinot-controller/src/main/java/com/linkedin/pinot/controller/util/AutoAddInvertedIndex.java
@@ -18,6 +18,7 @@
  */
 package com.linkedin.pinot.controller.util;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.google.common.base.Preconditions;
 import com.linkedin.pinot.common.config.IndexingConfig;
 import com.linkedin.pinot.common.config.TableConfig;
@@ -26,6 +27,7 @@ import com.linkedin.pinot.common.data.FieldSpec;
 import com.linkedin.pinot.common.data.Schema;
 import com.linkedin.pinot.common.data.TimeFieldSpec;
 import com.linkedin.pinot.common.metadata.ZKMetadataProvider;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.controller.helix.ControllerRequestURLBuilder;
 import java.io.BufferedReader;
 import java.io.BufferedWriter;
@@ -45,7 +47,6 @@ import org.apache.helix.ZNRecord;
 import org.apache.helix.manager.zk.ZKHelixAdmin;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -240,8 +241,8 @@ public class AutoAddInvertedIndex {
       }
 
       // Only add inverted index to table larger than a threshold
-      JSONObject queryResponse = sendQuery("SELECT COUNT(*) FROM " + tableNameWithType);
-      long numTotalDocs = queryResponse.getLong("totalDocs");
+      JsonNode queryResponse = sendQuery("SELECT COUNT(*) FROM " + tableNameWithType);
+      long numTotalDocs = queryResponse.get("totalDocs").asLong();
       LOGGER.info("Table: {}, number of total documents: {}", tableNameWithType, numTotalDocs);
       if (numTotalDocs <= _tableSizeThreshold) {
         LOGGER.info("Table: {}, skip adding inverted index because the table is too small", tableNameWithType);
@@ -250,8 +251,7 @@ public class AutoAddInvertedIndex {
 
       // Get each dimension's cardinality on one timestamp's data
       queryResponse = sendQuery("SELECT Max(" + timeColumnName + ") FROM " + tableNameWithType);
-      long maxTimeStamp =
-          new Double(queryResponse.getJSONArray("aggregationResults").getJSONObject(0).getDouble("value")).longValue();
+      long maxTimeStamp = queryResponse.get("aggregationResults").get(0).get("value").asLong();
 
       LOGGER.info("Table: {}, max time column {}: {}", tableNameWithType, timeColumnName, maxTimeStamp);
 
@@ -262,9 +262,9 @@ public class AutoAddInvertedIndex {
             "SELECT DISTINCTCOUNT(" + dimensionName + ") FROM " + tableNameWithType + " WHERE " + timeColumnName + " = "
                 + maxTimeStamp;
         queryResponse = sendQuery(query);
-        JSONObject result = queryResponse.getJSONArray("aggregationResults").getJSONObject(0);
-        resultPairs.add(
-            new ResultPair(result.getString("function").substring("distinctCount_".length()), result.getLong("value")));
+        JsonNode result = queryResponse.get("aggregationResults").get(0);
+        resultPairs.add(new ResultPair(result.get("function").asText().substring("distinctCount_".length()),
+            result.get("value").asLong()));
       }
 
       // Sort the dimensions based on their cardinalities
@@ -315,16 +315,16 @@ public class AutoAddInvertedIndex {
     }
   }
 
-  private JSONObject sendQuery(String query) throws Exception {
+  private JsonNode sendQuery(String query) throws Exception {
     URLConnection urlConnection = new URL("http://" + _brokerAddress + "/query").openConnection();
     urlConnection.setDoOutput(true);
 
     BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(urlConnection.getOutputStream(), "UTF-8"));
-    writer.write(new JSONObject().put("pql", query).toString());
+    writer.write(JsonUtils.newObjectNode().put("pql", query).toString());
     writer.flush();
 
     BufferedReader reader = new BufferedReader(new InputStreamReader(urlConnection.getInputStream(), "UTF-8"));
-    return new JSONObject(reader.readLine());
+    return JsonUtils.stringToJsonNode(reader.readLine());
   }
 
   private boolean updateIndexConfig(String tableName, TableConfig tableConfig) throws Exception {
diff --git a/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotInstanceRestletResourceTest.java b/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotInstanceRestletResourceTest.java
index c9ab9e6..b48c4d7 100644
--- a/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotInstanceRestletResourceTest.java
+++ b/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotInstanceRestletResourceTest.java
@@ -18,11 +18,13 @@
  */
 package com.linkedin.pinot.controller.api.resources;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.linkedin.pinot.common.utils.CommonConstants;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.controller.helix.ControllerTest;
 import com.linkedin.pinot.util.TestUtils;
 import java.io.IOException;
-import org.json.JSONObject;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
@@ -42,23 +44,25 @@ public class PinotInstanceRestletResourceTest extends ControllerTest {
   @Test
   public void testInstanceListingAndCreation() throws Exception {
     // Check that there are no instances
-    JSONObject instanceList = new JSONObject(sendGetRequest(_controllerRequestURLBuilder.forInstanceList()));
-    assertEquals(instanceList.getJSONArray("instances").length(), 0,
-        "Expected empty instance list at beginning of test");
+    JsonNode instanceList = JsonUtils.stringToJsonNode(sendGetRequest(_controllerRequestURLBuilder.forInstanceList()));
+    assertEquals(instanceList.get("instances").size(), 0, "Expected empty instance list at beginning of test");
 
     // Create untagged broker and server instances
-    JSONObject brokerInstance = new JSONObject("{\"host\":\"1.2.3.4\", \"type\":\"broker\", \"port\":\"1234\"}");
+    ObjectNode brokerInstance =
+        (ObjectNode) JsonUtils.stringToJsonNode("{\"host\":\"1.2.3.4\", \"type\":\"broker\", \"port\":\"1234\"}");
     sendPostRequest(_controllerRequestURLBuilder.forInstanceCreate(), brokerInstance.toString());
 
-    JSONObject serverInstance = new JSONObject("{\"host\":\"1.2.3.4\", \"type\":\"server\", \"port\":\"2345\"}");
+    ObjectNode serverInstance =
+        (ObjectNode) JsonUtils.stringToJsonNode("{\"host\":\"1.2.3.4\", \"type\":\"server\", \"port\":\"2345\"}");
     sendPostRequest(_controllerRequestURLBuilder.forInstanceCreate(), serverInstance.toString());
 
     // Check that there are two instances
     TestUtils.waitForCondition(aVoid -> {
       try {
-        // Check that there are four instances
-        JSONObject instanceList1 = new JSONObject(sendGetRequest(_controllerRequestURLBuilder.forInstanceList()));
-        return instanceList1.getJSONArray("instances").length() == 2;
+        // Check that there are two instances
+        return JsonUtils.stringToJsonNode(sendGetRequest(_controllerRequestURLBuilder.forInstanceList()))
+            .get("instances")
+            .size() == 2;
       } catch (Exception e) {
         throw new RuntimeException(e);
       }
@@ -77,8 +81,9 @@ public class PinotInstanceRestletResourceTest extends ControllerTest {
     TestUtils.waitForCondition(aVoid -> {
       try {
         // Check that there are four instances
-        JSONObject instanceList1 = new JSONObject(sendGetRequest(_controllerRequestURLBuilder.forInstanceList()));
-        return instanceList1.getJSONArray("instances").length() == 4;
+        return JsonUtils.stringToJsonNode(sendGetRequest(_controllerRequestURLBuilder.forInstanceList()))
+            .get("instances")
+            .size() == 4;
       } catch (Exception e) {
         throw new RuntimeException(e);
       }
@@ -100,46 +105,47 @@ public class PinotInstanceRestletResourceTest extends ControllerTest {
     }
 
     // Check that there are four instances
-    instanceList = new JSONObject(sendGetRequest(_controllerRequestURLBuilder.forInstanceList()));
-    assertEquals(instanceList.getJSONArray("instances").length(), 4,
-        "Expected four instances after creation of duplicate instances");
+    JsonUtils.stringToJsonNode(sendGetRequest(_controllerRequestURLBuilder.forInstanceList()));
+    assertEquals(JsonUtils.stringToJsonNode(sendGetRequest(_controllerRequestURLBuilder.forInstanceList()))
+        .get("instances")
+        .size(), 4, "Expected fore instances after creation of duplicate instances");
 
     // Check that the instances are properly created
-    JSONObject instance =
-        new JSONObject(sendGetRequest(_controllerRequestURLBuilder.forInstanceInformation("Broker_1.2.3.4_1234")));
-    assertEquals(instance.get("instanceName"), "Broker_1.2.3.4_1234");
-    assertEquals(instance.get("hostName"), "1.2.3.4");
-    assertEquals(instance.get("port"), "1234");
-    assertEquals(instance.get("enabled"), true);
-    assertEquals(instance.getJSONArray("tags").length(), 1);
-    assertEquals(instance.getJSONArray("tags").get(0), CommonConstants.Helix.UNTAGGED_BROKER_INSTANCE);
-
-    instance =
-        new JSONObject(sendGetRequest(_controllerRequestURLBuilder.forInstanceInformation("Server_1.2.3.4_2345")));
-    assertEquals(instance.get("instanceName"), "Server_1.2.3.4_2345");
-    assertEquals(instance.get("hostName"), "1.2.3.4");
-    assertEquals(instance.get("port"), "2345");
-    assertEquals(instance.get("enabled"), true);
-    assertEquals(instance.getJSONArray("tags").length(), 1);
-    assertEquals(instance.getJSONArray("tags").get(0), CommonConstants.Helix.UNTAGGED_SERVER_INSTANCE);
-
-    instance =
-        new JSONObject(sendGetRequest(_controllerRequestURLBuilder.forInstanceInformation("Broker_2.3.4.5_1234")));
-    assertEquals(instance.get("instanceName"), "Broker_2.3.4.5_1234");
-    assertEquals(instance.get("hostName"), "2.3.4.5");
-    assertEquals(instance.get("port"), "1234");
-    assertEquals(instance.get("enabled"), true);
-    assertEquals(instance.getJSONArray("tags").length(), 1);
-    assertEquals(instance.getJSONArray("tags").get(0), "someTag");
-
-    instance =
-        new JSONObject(sendGetRequest(_controllerRequestURLBuilder.forInstanceInformation("Server_2.3.4.5_2345")));
-    assertEquals(instance.get("instanceName"), "Server_2.3.4.5_2345");
-    assertEquals(instance.get("hostName"), "2.3.4.5");
-    assertEquals(instance.get("port"), "2345");
-    assertEquals(instance.get("enabled"), true);
-    assertEquals(instance.getJSONArray("tags").length(), 1);
-    assertEquals(instance.getJSONArray("tags").get(0), "someTag");
+    JsonNode instance = JsonUtils.stringToJsonNode(
+        sendGetRequest(_controllerRequestURLBuilder.forInstanceInformation("Broker_1.2.3.4_1234")));
+    assertEquals(instance.get("instanceName").asText(), "Broker_1.2.3.4_1234");
+    assertEquals(instance.get("hostName").asText(), "1.2.3.4");
+    assertEquals(instance.get("port").asText(), "1234");
+    assertTrue(instance.get("enabled").asBoolean());
+    assertEquals(instance.get("tags").size(), 1);
+    assertEquals(instance.get("tags").get(0).asText(), CommonConstants.Helix.UNTAGGED_BROKER_INSTANCE);
+
+    instance = JsonUtils.stringToJsonNode(
+        sendGetRequest(_controllerRequestURLBuilder.forInstanceInformation("Server_1.2.3.4_2345")));
+    assertEquals(instance.get("instanceName").asText(), "Server_1.2.3.4_2345");
+    assertEquals(instance.get("hostName").asText(), "1.2.3.4");
+    assertEquals(instance.get("port").asText(), "2345");
+    assertTrue(instance.get("enabled").asBoolean());
+    assertEquals(instance.get("tags").size(), 1);
+    assertEquals(instance.get("tags").get(0).asText(), CommonConstants.Helix.UNTAGGED_SERVER_INSTANCE);
+
+    instance = JsonUtils.stringToJsonNode(
+        sendGetRequest(_controllerRequestURLBuilder.forInstanceInformation("Broker_2.3.4.5_1234")));
+    assertEquals(instance.get("instanceName").asText(), "Broker_2.3.4.5_1234");
+    assertEquals(instance.get("hostName").asText(), "2.3.4.5");
+    assertEquals(instance.get("port").asText(), "1234");
+    assertTrue(instance.get("enabled").asBoolean());
+    assertEquals(instance.get("tags").size(), 1);
+    assertEquals(instance.get("tags").get(0).asText(), "someTag");
+
+    instance = JsonUtils.stringToJsonNode(
+        sendGetRequest(_controllerRequestURLBuilder.forInstanceInformation("Server_2.3.4.5_2345")));
+    assertEquals(instance.get("instanceName").asText(), "Server_2.3.4.5_2345");
+    assertEquals(instance.get("hostName").asText(), "2.3.4.5");
+    assertEquals(instance.get("port").asText(), "2345");
+    assertTrue(instance.get("enabled").asBoolean());
+    assertEquals(instance.get("tags").size(), 1);
+    assertEquals(instance.get("tags").get(0).asText(), "someTag");
 
     // Check that an error is given for an instance that does not exist
     try {
diff --git a/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotSchemaRestletResourceTest.java b/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotSchemaRestletResourceTest.java
index 05a3f53..4294fbc 100644
--- a/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotSchemaRestletResourceTest.java
+++ b/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotSchemaRestletResourceTest.java
@@ -18,27 +18,20 @@
  */
 package com.linkedin.pinot.controller.api.resources;
 
+import com.linkedin.pinot.common.data.DimensionFieldSpec;
+import com.linkedin.pinot.common.data.FieldSpec;
+import com.linkedin.pinot.common.data.Schema;
+import com.linkedin.pinot.controller.helix.ControllerTest;
 import java.io.IOException;
 import org.apache.commons.httpclient.methods.PostMethod;
 import org.apache.commons.httpclient.methods.PutMethod;
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.linkedin.pinot.common.data.DimensionFieldSpec;
-import com.linkedin.pinot.common.data.FieldSpec;
-import com.linkedin.pinot.common.data.MetricFieldSpec;
-import com.linkedin.pinot.common.data.Schema;
-import com.linkedin.pinot.controller.helix.ControllerTest;
 
 
 public class PinotSchemaRestletResourceTest extends ControllerTest {
-  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
 
   @BeforeClass
   public void setUp() {
@@ -46,29 +39,8 @@ public class PinotSchemaRestletResourceTest extends ControllerTest {
     startController();
   }
 
-  public JSONObject createDefaultSchema() throws JSONException, JsonProcessingException {
-    JSONObject schemaJson = new JSONObject();
-    schemaJson.put("schemaName", "testSchema");
-    JSONArray dimFieldSpec = new JSONArray();
-    schemaJson.put("dimensionFieldSpecs", dimFieldSpec);
-    JSONArray metricFieldSpec = new JSONArray();
-    schemaJson.put("metricFieldSpecs", metricFieldSpec);
-
-    DimensionFieldSpec df = new DimensionFieldSpec("dimA", FieldSpec.DataType.STRING, true, "");
-    dimFieldSpec.put(new JSONObject(OBJECT_MAPPER.writeValueAsString(df)));
-    df = new DimensionFieldSpec("dimB", FieldSpec.DataType.LONG, true, 0);
-    dimFieldSpec.put(new JSONObject(OBJECT_MAPPER.writeValueAsString(df)));
-
-    MetricFieldSpec mf = new MetricFieldSpec("metricA", FieldSpec.DataType.INT, 0);
-    metricFieldSpec.put(new JSONObject(OBJECT_MAPPER.writeValueAsString(mf)));
-
-    mf = new MetricFieldSpec("metricB", FieldSpec.DataType.DOUBLE, -1);
-    metricFieldSpec.put(new JSONObject(OBJECT_MAPPER.writeValueAsString(mf)));
-    return schemaJson;
-  }
-
   @Test
-  public void testBadContentType() throws JSONException, JsonProcessingException {
+  public void testBadContentType() {
     Schema schema = createDummySchema("testSchema");
     try {
       sendPostRequest(_controllerRequestURLBuilder.forSchemaCreate(), schema.getJSONSchema());
@@ -82,7 +54,7 @@ public class PinotSchemaRestletResourceTest extends ControllerTest {
   }
 
   @Test
-  public void testCreateUpdateSchema() throws JSONException, IOException {
+  public void testCreateUpdateSchema() throws IOException {
     String schemaName = "testSchema";
     Schema schema = createDummySchema(schemaName);
     String url = _controllerRequestURLBuilder.forSchemaCreate();
diff --git a/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotSegmentRestletResourceTest.java b/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotSegmentRestletResourceTest.java
index a102d78..c1ba1e1 100644
--- a/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotSegmentRestletResourceTest.java
+++ b/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotSegmentRestletResourceTest.java
@@ -18,12 +18,11 @@
  */
 package com.linkedin.pinot.controller.api.resources;
 
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
 import com.linkedin.pinot.common.config.TableConfig;
 import com.linkedin.pinot.common.config.TableNameBuilder;
 import com.linkedin.pinot.common.segment.SegmentMetadata;
 import com.linkedin.pinot.common.utils.CommonConstants;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.ZkStarter;
 import com.linkedin.pinot.controller.helix.ControllerRequestBuilderUtil;
 import com.linkedin.pinot.controller.helix.ControllerTest;
@@ -39,7 +38,6 @@ import org.testng.annotations.Test;
 public class PinotSegmentRestletResourceTest extends ControllerTest {
   private final static String ZK_SERVER = ZkStarter.DEFAULT_ZK_STR;
   private final static String TABLE_NAME = "testTable";
-  private final static ObjectMapper OBJECT_MAPPER = new ObjectMapper();
 
   @BeforeClass
   public void setUp() throws Exception {
@@ -113,8 +111,7 @@ public class PinotSegmentRestletResourceTest extends ControllerTest {
 
   private void checkCrcRequest(Map<String, SegmentMetadata> metadataTable, int expectedSize) throws Exception {
     String crcMapStr = sendGetRequest(_controllerRequestURLBuilder.forListAllCrcInformationForTable(TABLE_NAME));
-    Map<String, String> crcMap = OBJECT_MAPPER.readValue(crcMapStr, new TypeReference<Map<String, Object>>() {
-    });
+    Map<String, String> crcMap = JsonUtils.stringToObject(crcMapStr, Map.class);
     for (String segmentName : crcMap.keySet()) {
       SegmentMetadata metadata = metadataTable.get(segmentName);
       Assert.assertTrue(metadata != null);
diff --git a/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotTableRestletResourceTest.java b/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotTableRestletResourceTest.java
index f2b0f23..223f432 100644
--- a/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotTableRestletResourceTest.java
+++ b/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotTableRestletResourceTest.java
@@ -18,9 +18,11 @@
  */
 package com.linkedin.pinot.controller.api.resources;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.linkedin.pinot.common.config.QuotaConfig;
 import com.linkedin.pinot.common.config.TableConfig;
 import com.linkedin.pinot.common.utils.CommonConstants.Helix.TableType;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.ZkStarter;
 import com.linkedin.pinot.controller.ControllerConf;
 import com.linkedin.pinot.controller.helix.ControllerRequestBuilderUtil;
@@ -32,8 +34,6 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
-import org.json.JSONException;
-import org.json.JSONObject;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
@@ -211,7 +211,7 @@ public class PinotTableRestletResourceTest extends ControllerTest {
 
   private TableConfig getTableConfig(String tableName, String tableType) throws Exception {
     String tableConfigString = sendGetRequest(_controllerRequestURLBuilder.forTableGet(tableName));
-    return TableConfig.fromJSONConfig(new JSONObject(tableConfigString).getJSONObject(tableType));
+    return TableConfig.fromJSONConfig(JsonUtils.stringToJsonNode(tableConfigString).get(tableType));
   }
 
   @Test
@@ -228,11 +228,9 @@ public class PinotTableRestletResourceTest extends ControllerTest {
     tableConfig.getValidationConfig().setRetentionTimeUnit("HOURS");
     tableConfig.getValidationConfig().setRetentionTimeValue("10");
 
-    JSONObject jsonResponse = new JSONObject(
+    JsonNode jsonResponse = JsonUtils.stringToJsonNode(
         sendPutRequest(_controllerRequestURLBuilder.forUpdateTableConfig(tableName), tableConfig.toJSONConfigString()));
     Assert.assertTrue(jsonResponse.has("status"));
-    // TODO Verify success code, not success response string (Jersey API change)
-//    Assert.assertEquals(jsonResponse.getString("status"), "Success");
 
     TableConfig modifiedConfig = getTableConfig(tableName, "OFFLINE");
     Assert.assertEquals(modifiedConfig.getValidationConfig().getRetentionTimeUnit(), "HOURS");
@@ -277,14 +275,14 @@ public class PinotTableRestletResourceTest extends ControllerTest {
   }
 
   @Test(expectedExceptions = FileNotFoundException.class)
-  public void rebalanceNonExistentOfflineTable() throws IOException, JSONException {
+  public void rebalanceNonExistentOfflineTable() throws IOException {
     String tableName = "nonExistentTable";
     // should result in file not found exception
     sendPostRequest(_controllerRequestURLBuilder.forTableRebalance(tableName, "offline"), null);
   }
 
   @Test(expectedExceptions = FileNotFoundException.class)
-  public void rebalanceNonExistentRealtimeTable() throws IOException, JSONException {
+  public void rebalanceNonExistentRealtimeTable() throws IOException {
     String tableName = "nonExistentTable";
     // should result in file not found exception
     sendPostRequest(_controllerRequestURLBuilder.forTableRebalance(tableName, "realtime"), null);
diff --git a/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotTenantRestletResourceTest.java b/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotTenantRestletResourceTest.java
index ec928ce..f6bd45b 100644
--- a/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotTenantRestletResourceTest.java
+++ b/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/PinotTenantRestletResourceTest.java
@@ -18,12 +18,14 @@
  */
 package com.linkedin.pinot.controller.api.resources;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.linkedin.pinot.common.config.TableConfig;
 import com.linkedin.pinot.common.utils.CommonConstants;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.ZkStarter;
 import com.linkedin.pinot.controller.helix.ControllerRequestBuilderUtil;
 import com.linkedin.pinot.controller.helix.ControllerTest;
-import org.json.JSONObject;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
@@ -48,13 +50,11 @@ public class PinotTenantRestletResourceTest extends ControllerTest {
 
   @Test
   public void testTableListForTenant() throws Exception {
-    JSONObject tableList = null;
-
     // Create untagged broker and server instances
-    JSONObject brokerInstance = new JSONObject("{\"host\":\"1.2.3.4\", \"type\":\"broker\", \"port\":\"1234\"}");
+    ObjectNode brokerInstance = (ObjectNode) JsonUtils.stringToJsonNode("{\"host\":\"1.2.3.4\", \"type\":\"broker\", \"port\":\"1234\"}");
     sendPostRequest(_controllerRequestURLBuilder.forInstanceCreate(), brokerInstance.toString());
 
-    JSONObject serverInstance = new JSONObject("{\"host\":\"1.2.3.4\", \"type\":\"server\", \"port\":\"2345\"}");
+    ObjectNode serverInstance = (ObjectNode) JsonUtils.stringToJsonNode("{\"host\":\"1.2.3.4\", \"type\":\"server\", \"port\":\"2345\"}");
     sendPostRequest(_controllerRequestURLBuilder.forInstanceCreate(), serverInstance.toString());
 
     // Create tagged broker and server instances
@@ -67,12 +67,13 @@ public class PinotTenantRestletResourceTest extends ControllerTest {
     sendPostRequest(_controllerRequestURLBuilder.forInstanceCreate(), serverInstance.toString());
 
     // Check that no tables on tenant works
-    tableList = new JSONObject(sendGetRequest(_controllerRequestURLBuilder.forTablesFromTenant("server_REALTIME")));
-    assertEquals(tableList.getJSONArray("tables").length(), 0, "Expected no tables");
+    JsonNode tableList =
+        JsonUtils.stringToJsonNode(sendGetRequest(_controllerRequestURLBuilder.forTablesFromTenant("server_REALTIME")));
+    assertEquals(tableList.get("tables").size(), 0, "Expected no tables");
 
     // Try to make sure both kinds of tags work
-    tableList = new JSONObject(sendGetRequest(_controllerRequestURLBuilder.forTablesFromTenant("server")));
-    assertEquals(tableList.getJSONArray("tables").length(), 0, "Expected no tables");
+    tableList = JsonUtils.stringToJsonNode(sendGetRequest(_controllerRequestURLBuilder.forTablesFromTenant("server")));
+    assertEquals(tableList.get("tables").size(), 0, "Expected no tables");
 
     // Add a table to the server
     String createTableUrl = _controllerRequestURLBuilder.forTableCreate();
@@ -95,9 +96,9 @@ public class PinotTenantRestletResourceTest extends ControllerTest {
     sendPostRequest(createTableUrl, offlineTableJSONConfigString);
 
     // Try to make sure both kinds of tags work
-    tableList = new JSONObject(sendGetRequest(_controllerRequestURLBuilder.forTablesFromTenant("DefaultTenant")));
-    assertEquals(tableList.getJSONArray("tables").length(), 1, "Expected 1 table");
-    assertEquals(tableList.getJSONArray("tables").get(0), "mytable_OFFLINE");
+    tableList = JsonUtils.stringToJsonNode(sendGetRequest(_controllerRequestURLBuilder.forTablesFromTenant("DefaultTenant")));
+    assertEquals(tableList.get("tables").size(), 1, "Expected 1 table");
+    assertEquals(tableList.get("tables").get(0).asText(), "mytable_OFFLINE");
   }
 
   @AfterClass
diff --git a/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/SegmentCompletionProtocolDeserTest.java b/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/SegmentCompletionProtocolDeserTest.java
index b1b543a..bd7ec0a 100644
--- a/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/SegmentCompletionProtocolDeserTest.java
+++ b/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/SegmentCompletionProtocolDeserTest.java
@@ -18,12 +18,13 @@
  */
 package com.linkedin.pinot.controller.api.resources;
 
-import com.alibaba.fastjson.JSON;
+import com.fasterxml.jackson.databind.JsonNode;
 import com.linkedin.pinot.common.protocols.SegmentCompletionProtocol;
-import org.json.JSONException;
-import org.testng.Assert;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import org.testng.annotations.Test;
 
+import static org.testng.Assert.*;
+
 
 public class SegmentCompletionProtocolDeserTest {
   private final int OFFSET = 1;
@@ -34,134 +35,131 @@ public class SegmentCompletionProtocolDeserTest {
   @Test
   public void testCompleteResponseParams() {
     // Test with all params
-    SegmentCompletionProtocol.Response.Params params = new SegmentCompletionProtocol.Response.Params()
-        .withBuildTimeSeconds(BUILD_TIME_MILLIS)
-        .withOffset(OFFSET)
-        .withSegmentLocation(SEGMENT_LOCATION)
-        .withSplitCommit(true)
-        .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT);
+    SegmentCompletionProtocol.Response.Params params =
+        new SegmentCompletionProtocol.Response.Params().withBuildTimeSeconds(BUILD_TIME_MILLIS)
+            .withOffset(OFFSET)
+            .withSegmentLocation(SEGMENT_LOCATION)
+            .withSplitCommit(true)
+            .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT);
 
     SegmentCompletionProtocol.Response response = new SegmentCompletionProtocol.Response(params);
-    Assert.assertEquals(response.getBuildTimeSeconds(), BUILD_TIME_MILLIS);
-    Assert.assertEquals(response.getOffset(), OFFSET);
-    Assert.assertEquals(response.getSegmentLocation(), SEGMENT_LOCATION);
-    Assert.assertEquals(response.getIsSplitCommit(), true);
-    Assert.assertEquals(response.getStatus(), SegmentCompletionProtocol.ControllerResponseStatus.COMMIT);
+    assertEquals(response.getBuildTimeSeconds(), BUILD_TIME_MILLIS);
+    assertEquals(response.getOffset(), OFFSET);
+    assertEquals(response.getSegmentLocation(), SEGMENT_LOCATION);
+    assertTrue(response.isSplitCommit());
+    assertEquals(response.getStatus(), SegmentCompletionProtocol.ControllerResponseStatus.COMMIT);
   }
 
   @Test
   public void testIncompleteResponseParams() {
     // Test with reduced params
-    SegmentCompletionProtocol.Response.Params params = new SegmentCompletionProtocol.Response.Params()
-        .withBuildTimeSeconds(BUILD_TIME_MILLIS)
-        .withOffset(OFFSET)
-        .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT);
+    SegmentCompletionProtocol.Response.Params params =
+        new SegmentCompletionProtocol.Response.Params().withBuildTimeSeconds(BUILD_TIME_MILLIS)
+            .withOffset(OFFSET)
+            .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT);
 
     SegmentCompletionProtocol.Response response = new SegmentCompletionProtocol.Response(params);
-    Assert.assertEquals(response.getBuildTimeSeconds(), BUILD_TIME_MILLIS);
-    Assert.assertEquals(response.getOffset(), OFFSET);
-    Assert.assertEquals(response.getSegmentLocation(), null);
-    Assert.assertEquals(response.getIsSplitCommit(), false);
-    Assert.assertEquals(response.getStatus(), SegmentCompletionProtocol.ControllerResponseStatus.COMMIT);
+    assertEquals(response.getBuildTimeSeconds(), BUILD_TIME_MILLIS);
+    assertEquals(response.getOffset(), OFFSET);
+    assertNull(response.getSegmentLocation());
+    assertFalse(response.isSplitCommit());
+    assertEquals(response.getStatus(), SegmentCompletionProtocol.ControllerResponseStatus.COMMIT);
   }
 
   @Test
-  public void testJsonResponseWithAllParams() throws JSONException {
+  public void testJsonResponseWithAllParams() {
     // Test with all params
-    SegmentCompletionProtocol.Response.Params params = new SegmentCompletionProtocol.Response.Params()
-        .withBuildTimeSeconds(BUILD_TIME_MILLIS)
-        .withOffset(OFFSET)
-        .withSegmentLocation(SEGMENT_LOCATION)
-        .withSplitCommit(true)
-        .withControllerVipUrl(CONTROLLER_VIP_URL).withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT);
+    SegmentCompletionProtocol.Response.Params params =
+        new SegmentCompletionProtocol.Response.Params().withBuildTimeSeconds(BUILD_TIME_MILLIS)
+            .withOffset(OFFSET)
+            .withSegmentLocation(SEGMENT_LOCATION)
+            .withSplitCommit(true)
+            .withControllerVipUrl(CONTROLLER_VIP_URL)
+            .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT);
 
     SegmentCompletionProtocol.Response response = new SegmentCompletionProtocol.Response(params);
+    JsonNode jsonNode = JsonUtils.objectToJsonNode(response);
 
-    com.alibaba.fastjson.JSONObject jsonObject = JSON.parseObject(response.toJsonString());
-
-    Assert.assertEquals(jsonObject.get("offset"), OFFSET);
-    Assert.assertEquals(jsonObject.get("segmentLocation"), SEGMENT_LOCATION);
-    Assert.assertEquals(jsonObject.get("isSplitCommitType"), true);
-    Assert.assertEquals(jsonObject.get("status"), SegmentCompletionProtocol.ControllerResponseStatus.COMMIT.toString());
-    Assert.assertEquals(jsonObject.get("controllerVipUrl"), CONTROLLER_VIP_URL);
+    assertEquals(jsonNode.get("offset").asInt(), OFFSET);
+    assertEquals(jsonNode.get("segmentLocation").asText(), SEGMENT_LOCATION);
+    assertTrue(jsonNode.get("isSplitCommitType").asBoolean());
+    assertEquals(jsonNode.get("status").asText(), SegmentCompletionProtocol.ControllerResponseStatus.COMMIT.toString());
+    assertEquals(jsonNode.get("controllerVipUrl").asText(), CONTROLLER_VIP_URL);
   }
 
   @Test
-  public void testJsonNullSegmentLocationAndVip() throws JSONException {
-    SegmentCompletionProtocol.Response.Params params = new SegmentCompletionProtocol.Response.Params()
-        .withBuildTimeSeconds(BUILD_TIME_MILLIS)
-        .withOffset(OFFSET)
-        .withSplitCommit(false)
-        .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT);
+  public void testJsonNullSegmentLocationAndVip() {
+    SegmentCompletionProtocol.Response.Params params =
+        new SegmentCompletionProtocol.Response.Params().withBuildTimeSeconds(BUILD_TIME_MILLIS)
+            .withOffset(OFFSET)
+            .withSplitCommit(false)
+            .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT);
 
     SegmentCompletionProtocol.Response response = new SegmentCompletionProtocol.Response(params);
+    JsonNode jsonNode = JsonUtils.objectToJsonNode(response);
 
-    com.alibaba.fastjson.JSONObject jsonObject = JSON.parseObject(response.toJsonString());
-
-    Assert.assertEquals(jsonObject.get("offset"), OFFSET);
-    Assert.assertEquals(jsonObject.get("segmentLocation"), null);
-    Assert.assertEquals(jsonObject.get("isSplitCommitType"), false);
-    Assert.assertEquals(jsonObject.get("status"), SegmentCompletionProtocol.ControllerResponseStatus.COMMIT.toString());
-    Assert.assertEquals(jsonObject.get("controllerVipUrl"), null);
+    assertEquals(jsonNode.get("offset").asInt(), OFFSET);
+    assertNull(jsonNode.get("segmentLocation"));
+    assertFalse(jsonNode.get("isSplitCommitType").asBoolean());
+    assertEquals(jsonNode.get("status").asText(), SegmentCompletionProtocol.ControllerResponseStatus.COMMIT.toString());
+    assertNull(jsonNode.get("controllerVipUrl"));
   }
 
   @Test
-  public void testJsonResponseWithoutSplitCommit() throws JSONException {
-    SegmentCompletionProtocol.Response.Params params = new SegmentCompletionProtocol.Response.Params()
-        .withBuildTimeSeconds(BUILD_TIME_MILLIS)
-        .withOffset(OFFSET)
-        .withSplitCommit(false)
-        .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT);
+  public void testJsonResponseWithoutSplitCommit() {
+    SegmentCompletionProtocol.Response.Params params =
+        new SegmentCompletionProtocol.Response.Params().withBuildTimeSeconds(BUILD_TIME_MILLIS)
+            .withOffset(OFFSET)
+            .withSplitCommit(false)
+            .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT);
 
     SegmentCompletionProtocol.Response response = new SegmentCompletionProtocol.Response(params);
+    JsonNode jsonNode = JsonUtils.objectToJsonNode(response);
 
-    com.alibaba.fastjson.JSONObject jsonObject = JSON.parseObject(response.toJsonString());
-
-    Assert.assertEquals(jsonObject.get("offset"), OFFSET);
-    Assert.assertEquals(jsonObject.get("isSplitCommitType"), false);
-    Assert.assertEquals(jsonObject.get("status"), SegmentCompletionProtocol.ControllerResponseStatus.COMMIT.toString());
-    Assert.assertEquals(jsonObject.get("controllerVipUrl"), null);
+    assertEquals(jsonNode.get("offset").asInt(), OFFSET);
+    assertNull(jsonNode.get("segmentLocation"));
+    assertFalse(jsonNode.get("isSplitCommitType").asBoolean());
+    assertEquals(jsonNode.get("status").asText(), SegmentCompletionProtocol.ControllerResponseStatus.COMMIT.toString());
+    assertNull(jsonNode.get("controllerVipUrl"));
   }
 
   @Test
-  public void testJsonResponseWithSegmentLocationNullVip() throws JSONException {
+  public void testJsonResponseWithSegmentLocationNullVip() {
     // Should never happen because if split commit, should have both location and VIP, but testing deserialization regardless
-    SegmentCompletionProtocol.Response.Params params = new SegmentCompletionProtocol.Response.Params()
-        .withBuildTimeSeconds(BUILD_TIME_MILLIS)
-        .withOffset(OFFSET)
-        .withSegmentLocation(SEGMENT_LOCATION)
-        .withSplitCommit(false)
-        .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT);
+    SegmentCompletionProtocol.Response.Params params =
+        new SegmentCompletionProtocol.Response.Params().withBuildTimeSeconds(BUILD_TIME_MILLIS)
+            .withOffset(OFFSET)
+            .withSegmentLocation(SEGMENT_LOCATION)
+            .withSplitCommit(false)
+            .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT);
 
     SegmentCompletionProtocol.Response response = new SegmentCompletionProtocol.Response(params);
+    JsonNode jsonNode = JsonUtils.objectToJsonNode(response);
 
-    com.alibaba.fastjson.JSONObject jsonObject = JSON.parseObject(response.toJsonString());
-
-    Assert.assertEquals(jsonObject.get("offset"), OFFSET);
-    Assert.assertEquals(jsonObject.get("isSplitCommitType"), false);
-    Assert.assertEquals(jsonObject.get("segmentLocation"), SEGMENT_LOCATION);
-    Assert.assertEquals(jsonObject.get("status"), SegmentCompletionProtocol.ControllerResponseStatus.COMMIT.toString());
-    Assert.assertEquals(jsonObject.get("controllerVipUrl"), null);
+    assertEquals(jsonNode.get("offset").asInt(), OFFSET);
+    assertEquals(jsonNode.get("segmentLocation").asText(), SEGMENT_LOCATION);
+    assertFalse(jsonNode.get("isSplitCommitType").asBoolean());
+    assertEquals(jsonNode.get("status").asText(), SegmentCompletionProtocol.ControllerResponseStatus.COMMIT.toString());
+    assertNull(jsonNode.get("controllerVipUrl"));
   }
 
   @Test
-  public void testJsonResponseWithVipAndNullSegmentLocation() throws JSONException {
+  public void testJsonResponseWithVipAndNullSegmentLocation() {
     // Should never happen because if split commit, should have both location and VIP, but testing deserialization regardless
-    SegmentCompletionProtocol.Response.Params params = new SegmentCompletionProtocol.Response.Params()
-        .withBuildTimeSeconds(BUILD_TIME_MILLIS)
-        .withOffset(OFFSET)
-        .withControllerVipUrl(CONTROLLER_VIP_URL)
-        .withSplitCommit(false)
-        .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT);
+    SegmentCompletionProtocol.Response.Params params =
+        new SegmentCompletionProtocol.Response.Params().withBuildTimeSeconds(BUILD_TIME_MILLIS)
+            .withOffset(OFFSET)
+            .withControllerVipUrl(CONTROLLER_VIP_URL)
+            .withSplitCommit(false)
+            .withStatus(SegmentCompletionProtocol.ControllerResponseStatus.COMMIT);
 
     SegmentCompletionProtocol.Response response = new SegmentCompletionProtocol.Response(params);
+    JsonNode jsonNode = JsonUtils.objectToJsonNode(response);
 
-    com.alibaba.fastjson.JSONObject jsonObject = JSON.parseObject(response.toJsonString());
-
-    Assert.assertEquals(jsonObject.get("offset"), OFFSET);
-    Assert.assertEquals(jsonObject.get("isSplitCommitType"), false);
-    Assert.assertEquals(jsonObject.get("segmentLocation"), null);
-    Assert.assertEquals(jsonObject.get("status"), SegmentCompletionProtocol.ControllerResponseStatus.COMMIT.toString());
-    Assert.assertEquals(jsonObject.get("controllerVipUrl"), CONTROLLER_VIP_URL);
+    assertEquals(jsonNode.get("offset").asInt(), OFFSET);
+    assertNull(jsonNode.get("segmentLocation"));
+    assertFalse(jsonNode.get("isSplitCommitType").asBoolean());
+    assertEquals(jsonNode.get("status").asText(), SegmentCompletionProtocol.ControllerResponseStatus.COMMIT.toString());
+    assertEquals(jsonNode.get("controllerVipUrl").asText(), CONTROLLER_VIP_URL);
   }
 }
diff --git a/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/ServerTableSizeReaderTest.java b/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/ServerTableSizeReaderTest.java
index 687557e..9c29a6b 100644
--- a/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/ServerTableSizeReaderTest.java
+++ b/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/ServerTableSizeReaderTest.java
@@ -18,11 +18,11 @@
  */
 package com.linkedin.pinot.controller.api.resources;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.BiMap;
 import com.google.common.collect.HashBiMap;
 import com.linkedin.pinot.common.restlet.resources.SegmentSizeInfo;
 import com.linkedin.pinot.common.restlet.resources.TableSizeInfo;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.sun.net.httpserver.HttpExchange;
 import com.sun.net.httpserver.HttpHandler;
 import com.sun.net.httpserver.HttpServer;
@@ -123,7 +123,7 @@ public class ServerTableSizeReaderTest {
             LOGGER.info("Handler interrupted during sleep");
           }
         }
-        String json = new ObjectMapper().writeValueAsString(tableSize);
+        String json = JsonUtils.objectToString(tableSize);
         httpExchange.sendResponseHeaders(status, json.length());
         OutputStream responseBody = httpExchange.getResponseBody();
         responseBody.write(json.getBytes());
diff --git a/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/TableSizeReaderTest.java b/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/TableSizeReaderTest.java
index 9d6dcf4..eb32dca 100644
--- a/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/TableSizeReaderTest.java
+++ b/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/TableSizeReaderTest.java
@@ -18,7 +18,6 @@
  */
 package com.linkedin.pinot.controller.api.resources;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.BiMap;
 import com.google.common.collect.HashBiMap;
 import com.linkedin.pinot.common.config.TableNameBuilder;
@@ -27,6 +26,7 @@ import com.linkedin.pinot.common.metrics.ControllerGauge;
 import com.linkedin.pinot.common.metrics.ControllerMetrics;
 import com.linkedin.pinot.common.restlet.resources.SegmentSizeInfo;
 import com.linkedin.pinot.common.restlet.resources.TableSizeInfo;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.controller.helix.core.PinotHelixResourceManager;
 import com.linkedin.pinot.controller.util.TableSizeReader;
 import com.sun.net.httpserver.HttpExchange;
@@ -156,7 +156,7 @@ public class TableSizeReaderTest {
           tableInfo.diskSizeInBytes += segmentSize.diskSizeInBytes;
         }
 
-        String json = new ObjectMapper().writeValueAsString(tableInfo);
+        String json = JsonUtils.objectToString(tableInfo);
         httpExchange.sendResponseHeaders(status, json.length());
         OutputStream responseBody = httpExchange.getResponseBody();
         responseBody.write(json.getBytes());
diff --git a/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/TableViewsTest.java b/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/TableViewsTest.java
index 0a1e433..7f14ef0 100644
--- a/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/TableViewsTest.java
+++ b/pinot-controller/src/test/java/com/linkedin/pinot/controller/api/resources/TableViewsTest.java
@@ -18,10 +18,10 @@
  */
 package com.linkedin.pinot.controller.api.resources;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 import com.linkedin.pinot.common.config.TableConfig;
 import com.linkedin.pinot.common.config.TableNameBuilder;
 import com.linkedin.pinot.common.utils.CommonConstants;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.ZkStarter;
 import com.linkedin.pinot.controller.helix.ControllerRequestBuilderUtil;
 import com.linkedin.pinot.controller.helix.ControllerTest;
@@ -42,7 +42,6 @@ import org.testng.annotations.Test;
 
 
 public class TableViewsTest extends ControllerTest {
-  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
   private static final String OFFLINE_TABLE_NAME = "offlineTable";
   private static final String OFFLINE_SEGMENT_NAME = "offlineSegment";
   private static final String HYBRID_TABLE_NAME = "hybridTable";
@@ -183,7 +182,7 @@ public class TableViewsTest extends ControllerTest {
   }
 
   private TableViews.TableView getTableView(String tableName, String view, String tableType) throws Exception {
-    return OBJECT_MAPPER.readValue(
+    return JsonUtils.stringToObject(
         sendGetRequest(_controllerRequestURLBuilder.forTableView(tableName, view, tableType)),
         TableViews.TableView.class);
   }
diff --git a/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/ControllerSentinelTestV2.java b/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/ControllerSentinelTestV2.java
index ff84fea..ea4a868 100644
--- a/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/ControllerSentinelTestV2.java
+++ b/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/ControllerSentinelTestV2.java
@@ -24,7 +24,6 @@ import com.linkedin.pinot.common.utils.CommonConstants;
 import com.linkedin.pinot.common.utils.ZkStarter;
 import com.linkedin.pinot.controller.utils.SegmentMetadataMockUtils;
 import java.io.IOException;
-import org.json.JSONException;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
@@ -51,7 +50,7 @@ public class ControllerSentinelTestV2 extends ControllerTest {
   }
 
   @Test
-  public void testOfflineTableLifeCycle() throws IOException, JSONException {
+  public void testOfflineTableLifeCycle() throws IOException {
     // Create offline table creation request
     String tableName = "testTable";
     String tableJSONConfigString =
diff --git a/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/ControllerTenantTest.java b/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/ControllerTenantTest.java
index d56d2bf..264eb6b 100644
--- a/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/ControllerTenantTest.java
+++ b/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/ControllerTenantTest.java
@@ -18,13 +18,13 @@
  */
 package com.linkedin.pinot.controller.helix;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.linkedin.pinot.common.config.TagNameUtils;
 import com.linkedin.pinot.common.metadata.ZKMetadataProvider;
 import com.linkedin.pinot.common.utils.CommonConstants;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.ZkStarter;
 import java.io.IOException;
-import org.json.JSONException;
-import org.json.JSONObject;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
@@ -56,41 +56,43 @@ public class ControllerTenantTest extends ControllerTest {
   }
 
   @Test
-  public void testBrokerTenant() throws IOException, JSONException {
+  public void testBrokerTenant() throws IOException {
     // Create broker tenants
     for (int i = 1; i <= NUM_BROKER_TAGS; i++) {
       String brokerTenant = BROKER_TAG_PREFIX + i;
-      JSONObject payload =
+      String payload =
           ControllerRequestBuilderUtil.buildBrokerTenantCreateRequestJSON(brokerTenant, NUM_BROKERS_PER_TAG);
-      sendPostRequest(_controllerRequestURLBuilder.forTenantCreate(), payload.toString());
-      Assert.assertEquals(_helixAdmin.getInstancesInClusterWithTag(_helixClusterName,
-          TagNameUtils.getBrokerTagForTenant(brokerTenant)).size(), NUM_BROKERS_PER_TAG);
+      sendPostRequest(_controllerRequestURLBuilder.forTenantCreate(), payload);
+      Assert.assertEquals(
+          _helixAdmin.getInstancesInClusterWithTag(_helixClusterName, TagNameUtils.getBrokerTagForTenant(brokerTenant))
+              .size(), NUM_BROKERS_PER_TAG);
       Assert.assertEquals(
           _helixAdmin.getInstancesInClusterWithTag(_helixClusterName, CommonConstants.Helix.UNTAGGED_BROKER_INSTANCE)
               .size(), NUM_INSTANCES - i * NUM_BROKERS_PER_TAG);
     }
 
     // Get broker tenants
-    JSONObject response = new JSONObject(sendGetRequest(_controllerRequestURLBuilder.forTenantGet()));
-    Assert.assertEquals(response.getJSONArray("BROKER_TENANTS").length(), NUM_BROKER_TAGS);
+    JsonNode response = JsonUtils.stringToJsonNode(sendGetRequest(_controllerRequestURLBuilder.forTenantGet()));
+    Assert.assertEquals(response.get("BROKER_TENANTS").size(), NUM_BROKER_TAGS);
     for (int i = 1; i <= NUM_BROKER_TAGS; i++) {
       String brokerTag = BROKER_TAG_PREFIX + i;
-      response = new JSONObject(sendGetRequest(_controllerRequestURLBuilder.forBrokerTenantGet(brokerTag)));
-      Assert.assertEquals(response.getJSONArray("BrokerInstances").length(), NUM_BROKERS_PER_TAG);
-      Assert.assertEquals(response.getString("tenantName"), brokerTag);
-      response = new JSONObject(sendGetRequest(_controllerRequestURLBuilder.forTenantGet(brokerTag)));
-      Assert.assertEquals(response.getJSONArray("BrokerInstances").length(), NUM_BROKERS_PER_TAG);
-      Assert.assertEquals(response.getJSONArray("ServerInstances").length(), 0);
-      Assert.assertEquals(response.getString("tenantName"), brokerTag);
+      response = JsonUtils.stringToJsonNode(sendGetRequest(_controllerRequestURLBuilder.forBrokerTenantGet(brokerTag)));
+      Assert.assertEquals(response.get("BrokerInstances").size(), NUM_BROKERS_PER_TAG);
+      Assert.assertEquals(response.get("tenantName").asText(), brokerTag);
+      response = JsonUtils.stringToJsonNode(sendGetRequest(_controllerRequestURLBuilder.forTenantGet(brokerTag)));
+      Assert.assertEquals(response.get("BrokerInstances").size(), NUM_BROKERS_PER_TAG);
+      Assert.assertEquals(response.get("ServerInstances").size(), 0);
+      Assert.assertEquals(response.get("tenantName").asText(), brokerTag);
     }
 
     // Update broker tenants
     for (int i = 0; i <= NUM_INSTANCES - (NUM_BROKER_TAGS - 1) * NUM_BROKERS_PER_TAG; i++) {
       String brokerTenant = BROKER_TAG_PREFIX + 1;
-      JSONObject payload = ControllerRequestBuilderUtil.buildBrokerTenantCreateRequestJSON(brokerTenant, i);
-      sendPutRequest(_controllerRequestURLBuilder.forTenantCreate(), payload.toString());
-      Assert.assertEquals(_helixAdmin.getInstancesInClusterWithTag(_helixClusterName,
-          TagNameUtils.getBrokerTagForTenant(brokerTenant)).size(), i);
+      String payload = ControllerRequestBuilderUtil.buildBrokerTenantCreateRequestJSON(brokerTenant, i);
+      sendPutRequest(_controllerRequestURLBuilder.forTenantCreate(), payload);
+      Assert.assertEquals(
+          _helixAdmin.getInstancesInClusterWithTag(_helixClusterName, TagNameUtils.getBrokerTagForTenant(brokerTenant))
+              .size(), i);
       Assert.assertEquals(
           _helixAdmin.getInstancesInClusterWithTag(_helixClusterName, CommonConstants.Helix.UNTAGGED_BROKER_INSTANCE)
               .size(), NUM_INSTANCES - (NUM_BROKER_TAGS - 1) * NUM_BROKERS_PER_TAG - i);
@@ -109,16 +111,17 @@ public class ControllerTenantTest extends ControllerTest {
   }
 
   @Test
-  public void testServerTenant() throws IOException, JSONException {
+  public void testServerTenant() throws IOException {
     // Create server tenants
     for (int i = 1; i <= NUM_SERVER_TAGS; i++) {
       String serverTenant = SERVER_TAG_PREFIX + i;
-      JSONObject payload =
+      String payload =
           ControllerRequestBuilderUtil.buildServerTenantCreateRequestJSON(serverTenant, NUM_SERVERS_PER_TAG,
               NUM_OFFLINE_SERVERS_PER_TAG, NUM_REALTIME_SERVERS_PER_TAG);
-      sendPostRequest(_controllerRequestURLBuilder.forTenantCreate(), payload.toString());
-      Assert.assertEquals(_helixAdmin.getInstancesInClusterWithTag(_helixClusterName,
-          TagNameUtils.getOfflineTagForTenant(serverTenant)).size(), NUM_OFFLINE_SERVERS_PER_TAG);
+      sendPostRequest(_controllerRequestURLBuilder.forTenantCreate(), payload);
+      Assert.assertEquals(
+          _helixAdmin.getInstancesInClusterWithTag(_helixClusterName, TagNameUtils.getOfflineTagForTenant(serverTenant))
+              .size(), NUM_OFFLINE_SERVERS_PER_TAG);
       Assert.assertEquals(_helixAdmin.getInstancesInClusterWithTag(_helixClusterName,
           TagNameUtils.getRealtimeTagForTenant(serverTenant)).size(), NUM_REALTIME_SERVERS_PER_TAG);
       Assert.assertEquals(
@@ -127,27 +130,27 @@ public class ControllerTenantTest extends ControllerTest {
     }
 
     // Get server tenants
-    JSONObject response = new JSONObject(sendGetRequest(_controllerRequestURLBuilder.forTenantGet()));
-    Assert.assertEquals(response.getJSONArray("SERVER_TENANTS").length(), NUM_SERVER_TAGS);
+    JsonNode response = JsonUtils.stringToJsonNode(sendGetRequest(_controllerRequestURLBuilder.forTenantGet()));
+    Assert.assertEquals(response.get("SERVER_TENANTS").size(), NUM_SERVER_TAGS);
     for (int i = 1; i <= NUM_SERVER_TAGS; i++) {
       String serverTag = SERVER_TAG_PREFIX + i;
-      response = new JSONObject(sendGetRequest(_controllerRequestURLBuilder.forServerTenantGet(serverTag)));
-      Assert.assertEquals(response.getJSONArray("ServerInstances").length(), NUM_SERVERS_PER_TAG);
-      Assert.assertEquals(response.getString("tenantName"), serverTag);
-      response = new JSONObject(sendGetRequest(_controllerRequestURLBuilder.forTenantGet(serverTag)));
-      Assert.assertEquals(response.getJSONArray("BrokerInstances").length(), 0);
-      Assert.assertEquals(response.getJSONArray("ServerInstances").length(), NUM_SERVERS_PER_TAG);
-      Assert.assertEquals(response.getString("tenantName"), serverTag);
+      response = JsonUtils.stringToJsonNode(sendGetRequest(_controllerRequestURLBuilder.forServerTenantGet(serverTag)));
+      Assert.assertEquals(response.get("ServerInstances").size(), NUM_SERVERS_PER_TAG);
+      Assert.assertEquals(response.get("tenantName").asText(), serverTag);
+      response = JsonUtils.stringToJsonNode(sendGetRequest(_controllerRequestURLBuilder.forTenantGet(serverTag)));
+      Assert.assertEquals(response.get("BrokerInstances").size(), 0);
+      Assert.assertEquals(response.get("ServerInstances").size(), NUM_SERVERS_PER_TAG);
+      Assert.assertEquals(response.get("tenantName").asText(), serverTag);
     }
 
     // Update server tenants
     // Note: server tenants cannot scale down
     for (int i = 0; i <= (NUM_INSTANCES - NUM_SERVER_TAGS * NUM_SERVERS_PER_TAG) / 2; i++) {
       String serverTenant = SERVER_TAG_PREFIX + 1;
-      JSONObject payload =
+      String payload =
           ControllerRequestBuilderUtil.buildServerTenantCreateRequestJSON(serverTenant, NUM_SERVERS_PER_TAG + i * 2,
               NUM_OFFLINE_SERVERS_PER_TAG + i, NUM_REALTIME_SERVERS_PER_TAG + i);
-      sendPutRequest(_controllerRequestURLBuilder.forTenantCreate(), payload.toString());
+      sendPutRequest(_controllerRequestURLBuilder.forTenantCreate(), payload);
       Assert.assertEquals(_helixAdmin.getInstancesInClusterWithTag(_helixClusterName,
           TagNameUtils.getOfflineTagForTenant(serverTenant)).size(),
           NUM_OFFLINE_SERVERS_PER_TAG + i);
diff --git a/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/ControllerTest.java b/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/ControllerTest.java
index fd84a3d..7c05d92 100644
--- a/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/ControllerTest.java
+++ b/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/ControllerTest.java
@@ -18,7 +18,6 @@
  */
 package com.linkedin.pinot.controller.helix;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 import com.linkedin.pinot.common.data.DimensionFieldSpec;
 import com.linkedin.pinot.common.data.FieldSpec;
 import com.linkedin.pinot.common.data.MetricFieldSpec;
@@ -49,7 +48,6 @@ import org.apache.helix.HelixManager;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.manager.zk.ZkClient;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
-import org.json.JSONException;
 import org.testng.Assert;
 
 
@@ -58,7 +56,6 @@ import org.testng.Assert;
  */
 public abstract class ControllerTest {
   public static final String LOCAL_HOST = "localhost";
-  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
 
   private static final int DEFAULT_CONTROLLER_PORT = 8998;
   private static final String DEFAULT_DATA_DIR =
@@ -148,7 +145,7 @@ public abstract class ControllerTest {
     _zkClient.close();
   }
 
-  protected Schema createDummySchema(String tableName) throws JSONException {
+  protected Schema createDummySchema(String tableName) {
     Schema schema = new Schema();
     schema.setSchemaName(tableName);
     schema.addField(new DimensionFieldSpec("dimA", FieldSpec.DataType.STRING, true, ""));
@@ -160,7 +157,7 @@ public abstract class ControllerTest {
     return schema;
   }
 
-  protected void addDummySchema(String tableName) throws JSONException, IOException {
+  protected void addDummySchema(String tableName) throws IOException {
     addSchema(createDummySchema(tableName).getJSONSchema());
   }
 
diff --git a/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdaterTest.java b/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdaterTest.java
index 204637a..f730550 100644
--- a/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdaterTest.java
+++ b/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/core/realtime/segment/FlushThresholdUpdaterTest.java
@@ -27,13 +27,11 @@ import com.linkedin.pinot.core.realtime.impl.kafka.KafkaAvroMessageDecoder;
 import com.linkedin.pinot.core.realtime.impl.kafka.KafkaConsumerFactory;
 import com.linkedin.pinot.core.realtime.stream.StreamConfig;
 import com.linkedin.pinot.core.realtime.stream.StreamConfigProperties;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
-import org.json.JSONException;
 import org.testng.Assert;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
@@ -272,11 +270,9 @@ public class FlushThresholdUpdaterTest {
 
   /**
    * Tests that the flush threshold manager returns the right updater given various scenarios of flush threshold setting in the table config
-   * @throws IOException
-   * @throws JSONException
    */
   @Test
-  public void testFlushThresholdUpdater() throws IOException, JSONException {
+  public void testFlushThresholdUpdater() {
     FlushThresholdUpdateManager manager = new FlushThresholdUpdateManager();
     TableConfig.Builder tableConfigBuilder = new TableConfig.Builder(CommonConstants.Helix.TableType.REALTIME);
     tableConfigBuilder.setTableName("tableName_REALTIME");
diff --git a/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/core/rebalance/DefaultRebalanceStrategyTest.java b/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/core/rebalance/DefaultRebalanceStrategyTest.java
index 85a6394..0ba5f4f 100644
--- a/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/core/rebalance/DefaultRebalanceStrategyTest.java
+++ b/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/core/rebalance/DefaultRebalanceStrategyTest.java
@@ -33,7 +33,6 @@ import com.linkedin.pinot.controller.helix.core.PinotHelixSegmentOnlineOfflineSt
 import com.linkedin.pinot.core.realtime.impl.kafka.KafkaAvroMessageDecoder;
 import com.linkedin.pinot.core.realtime.impl.kafka.KafkaConsumerFactory;
 import com.linkedin.pinot.core.realtime.stream.StreamConfigProperties;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -45,7 +44,6 @@ import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.helix.HelixManager;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.builder.CustomModeISBuilder;
-import org.json.JSONException;
 import org.testng.Assert;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
@@ -281,8 +279,7 @@ public class DefaultRebalanceStrategyTest {
   }
 
   @Test
-  public void testGetRebalancedIdealStateOffline() throws IOException, JSONException {
-
+  public void testGetRebalancedIdealStateOffline() {
     String offlineTableName = "letsRebalanceThisTable_OFFLINE";
     TableConfig tableConfig;
 
@@ -380,8 +377,7 @@ public class DefaultRebalanceStrategyTest {
   }
 
   @Test
-  public void testGetRebalancedIdealStateRealtime() throws IOException, JSONException {
-
+  public void testGetRebalancedIdealStateRealtime() {
     String realtimeTableName = "letsRebalanceThisTable_REALTIME";
     TableConfig tableConfig;
 
diff --git a/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/core/retention/RetentionManagerTest.java b/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/core/retention/RetentionManagerTest.java
index d2fe9b3..a43e137 100644
--- a/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/core/retention/RetentionManagerTest.java
+++ b/pinot-controller/src/test/java/com/linkedin/pinot/controller/helix/core/retention/RetentionManagerTest.java
@@ -31,7 +31,6 @@ import com.linkedin.pinot.controller.helix.core.PinotHelixResourceManager;
 import com.linkedin.pinot.controller.helix.core.PinotTableIdealStateBuilder;
 import com.linkedin.pinot.controller.helix.core.SegmentDeletionManager;
 import com.linkedin.pinot.controller.helix.core.util.ZKMetadataUtils;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -40,7 +39,6 @@ import org.apache.helix.HelixAdmin;
 import org.apache.helix.model.IdealState;
 import org.joda.time.Duration;
 import org.joda.time.Interval;
-import org.json.JSONException;
 import org.mockito.ArgumentMatchers;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -151,7 +149,7 @@ public class RetentionManagerTest {
         .build();
   }
 
-  private TableConfig createRealtimeTableConfig1(int replicaCount) throws IOException, JSONException {
+  private TableConfig createRealtimeTableConfig1(int replicaCount) {
     return new TableConfig.Builder(CommonConstants.Helix.TableType.REALTIME).setTableName(TEST_TABLE_NAME)
         .setLLC(true)
         .setRetentionTimeUnit("DAYS")
diff --git a/pinot-core/pom.xml b/pinot-core/pom.xml
index 81580f8..b0bc493 100644
--- a/pinot-core/pom.xml
+++ b/pinot-core/pom.xml
@@ -119,10 +119,6 @@
       <artifactId>commons-lang</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.json</groupId>
-      <artifactId>json</artifactId>
-    </dependency>
-    <dependency>
       <groupId>commons-io</groupId>
       <artifactId>commons-io</artifactId>
     </dependency>
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/data/GenericRow.java b/pinot-core/src/main/java/com/linkedin/pinot/core/data/GenericRow.java
index 91c790a..f6857f2 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/data/GenericRow.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/data/GenericRow.java
@@ -19,15 +19,12 @@
 package com.linkedin.pinot.core.data;
 
 import com.linkedin.pinot.common.data.RowEvent;
-import com.linkedin.pinot.common.utils.StringUtil;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.io.IOException;
-import java.io.StringWriter;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.type.TypeReference;
 
 
 /**
@@ -35,8 +32,7 @@ import org.codehaus.jackson.type.TypeReference;
  * {@link GenericRow#createOrReuseRow(GenericRow)}
  */
 public class GenericRow implements RowEvent {
-  private Map<String, Object> _fieldMap = new HashMap<String, Object>();
-  private static ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+  private Map<String, Object> _fieldMap = new HashMap<>();
 
   @Override
   public void init(Map<String, Object> field) {
@@ -101,19 +97,15 @@ public class GenericRow implements RowEvent {
     }
   }
 
-  static TypeReference typeReference = new TypeReference<Map<String, Object>>() {};
-
   public static GenericRow fromBytes(byte[] buffer) throws IOException {
-    Map<String, Object> fieldMap = (Map<String, Object>) OBJECT_MAPPER.readValue(buffer, typeReference);
+    Map<String, Object> fieldMap = JsonUtils.bytesToObject(buffer, Map.class);
     GenericRow genericRow = new GenericRow();
     genericRow.init(fieldMap);
     return genericRow;
   }
 
   public byte[] toBytes() throws IOException {
-    StringWriter writer = new StringWriter();
-    OBJECT_MAPPER.writeValue(writer, _fieldMap);
-    return StringUtil.encodeUtf8(writer.toString());
+    return JsonUtils.objectToBytes(_fieldMap);
   }
 
   /**
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/data/manager/realtime/HLRealtimeSegmentDataManager.java b/pinot-core/src/main/java/com/linkedin/pinot/core/data/manager/realtime/HLRealtimeSegmentDataManager.java
index 265c36e..0bd58eb 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/data/manager/realtime/HLRealtimeSegmentDataManager.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/data/manager/realtime/HLRealtimeSegmentDataManager.java
@@ -169,7 +169,7 @@ public class HLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
     tableStreamName = tableName + "_" + _streamConfig.getTopicName();
 
     IndexingConfig indexingConfig = tableConfig.getIndexingConfig();
-    if (indexingConfig != null && indexingConfig.getAggregateMetrics()) {
+    if (indexingConfig != null && indexingConfig.isAggregateMetrics()) {
       LOGGER.warn("Updating of metrics only supported for LLC consumer, ignoring.");
     }
 
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java b/pinot-core/src/main/java/com/linkedin/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java
index bb25b27..e083500 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManager.java
@@ -744,7 +744,7 @@ public class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
       throw new RuntimeException("Segment file does not exist:" + segTarFileName);
     }
     SegmentCompletionProtocol.Response returnedResponse;
-    if (response.getIsSplitCommit() && _indexLoadingConfig.isEnableSplitCommit()) {
+    if (response.isSplitCommit() && _indexLoadingConfig.isEnableSplitCommit()) {
       // Send segmentStart, segmentUpload, & segmentCommitEnd to the controller
       // if that succeeds, swap in-memory segment with the one built.
       returnedResponse = doSplitCommit(response);
@@ -1062,7 +1062,7 @@ public class LLRealtimeSegmentDataManager extends RealtimeSegmentDataManager {
             .setOffHeap(_isOffHeap)
             .setMemoryManager(_memoryManager)
             .setStatsHistory(realtimeTableDataManager.getStatsHistory())
-            .setAggregateMetrics(indexingConfig.getAggregateMetrics());
+            .setAggregateMetrics(indexingConfig.isAggregateMetrics());
 
     // Create message decoder
     _messageDecoder = StreamDecoderProvider.create(_partitionLevelStreamConfig, _schema);
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/data/readers/JSONRecordReader.java b/pinot-core/src/main/java/com/linkedin/pinot/core/data/readers/JSONRecordReader.java
index 50e4a18..bc156e5 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/data/readers/JSONRecordReader.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/data/readers/JSONRecordReader.java
@@ -18,14 +18,12 @@
  */
 package com.linkedin.pinot.core.data.readers;
 
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonParser;
 import com.linkedin.pinot.common.data.FieldSpec;
 import com.linkedin.pinot.common.data.Schema;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.core.data.GenericRow;
-
-import org.codehaus.jackson.JsonFactory;
-import org.codehaus.jackson.JsonParser;
-import org.codehaus.jackson.map.ObjectMapper;
-
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -37,8 +35,6 @@ import java.util.Map;
  * Record reader for JSON file.
  */
 public class JSONRecordReader implements RecordReader {
-  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
-
   private final JsonFactory _factory = new JsonFactory();
   private final File _dataFile;
   private final Schema _schema;
@@ -54,9 +50,9 @@ public class JSONRecordReader implements RecordReader {
   }
 
   private void init() throws IOException {
-    _parser = _factory.createJsonParser(RecordReaderUtils.getFileReader(_dataFile));
+    _parser = _factory.createParser(RecordReaderUtils.getFileReader(_dataFile));
     try {
-      _iterator = OBJECT_MAPPER.readValues(_parser, Map.class);
+      _iterator = JsonUtils.DEFAULT_MAPPER.readValues(_parser, Map.class);
     } catch (Exception e) {
       _parser.close();
       throw e;
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/generator/SegmentGeneratorConfig.java b/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/generator/SegmentGeneratorConfig.java
index 10ece6b..f9875d7 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/generator/SegmentGeneratorConfig.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/generator/SegmentGeneratorConfig.java
@@ -18,6 +18,8 @@
  */
 package com.linkedin.pinot.core.indexsegment.generator;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.google.common.base.Preconditions;
 import com.linkedin.pinot.common.config.IndexingConfig;
 import com.linkedin.pinot.common.config.SegmentPartitionConfig;
@@ -29,6 +31,7 @@ import com.linkedin.pinot.common.data.FieldSpec.FieldType;
 import com.linkedin.pinot.common.data.Schema;
 import com.linkedin.pinot.common.data.StarTreeIndexSpec;
 import com.linkedin.pinot.common.data.TimeFieldSpec;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.core.data.readers.CSVRecordReaderConfig;
 import com.linkedin.pinot.core.data.readers.FileFormat;
 import com.linkedin.pinot.core.data.readers.RecordReaderConfig;
@@ -53,9 +56,6 @@ import java.util.stream.Collectors;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 import org.apache.commons.lang.StringUtils;
-import org.codehaus.jackson.annotate.JsonIgnore;
-import org.codehaus.jackson.annotate.JsonIgnoreProperties;
-import org.codehaus.jackson.map.ObjectMapper;
 import org.joda.time.format.DateTimeFormat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -172,7 +172,7 @@ public class SegmentGeneratorConfig {
 
     IndexingConfig indexingConfig = tableConfig.getIndexingConfig();
     List<String> noDictionaryColumns = indexingConfig.getNoDictionaryColumns();
-    Map<String, String> noDictionaryColumnMap = indexingConfig.getnoDictionaryConfig();
+    Map<String, String> noDictionaryColumnMap = indexingConfig.getNoDictionaryConfig();
 
     if (noDictionaryColumns != null) {
       this.setRawIndexCreationColumns(noDictionaryColumns);
@@ -584,8 +584,6 @@ public class SegmentGeneratorConfig {
    */
   @Deprecated
   public void loadConfigFiles() throws IOException {
-    ObjectMapper objectMapper = new ObjectMapper();
-
     Schema schema;
     if (_schemaFile != null) {
       schema = Schema.fromFile(new File(_schemaFile));
@@ -605,7 +603,7 @@ public class SegmentGeneratorConfig {
     }
 
     if (_readerConfigFile != null) {
-      setReaderConfig(objectMapper.readValue(new File(_readerConfigFile), CSVRecordReaderConfig.class));
+      setReaderConfig(JsonUtils.fileToObject(new File(_readerConfigFile), CSVRecordReaderConfig.class));
     }
   }
 
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/realtime/impl/kafka/KafkaJSONMessageDecoder.java b/pinot-core/src/main/java/com/linkedin/pinot/core/realtime/impl/kafka/KafkaJSONMessageDecoder.java
index 5852b78..b509b4e 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/realtime/impl/kafka/KafkaJSONMessageDecoder.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/realtime/impl/kafka/KafkaJSONMessageDecoder.java
@@ -18,16 +18,15 @@
  */
 package com.linkedin.pinot.core.realtime.impl.kafka;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.linkedin.pinot.common.data.FieldSpec;
 import com.linkedin.pinot.common.data.Schema;
-import com.linkedin.pinot.common.data.TimeFieldSpec;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.core.data.GenericRow;
 import com.linkedin.pinot.core.realtime.stream.StreamMessageDecoder;
+import java.io.ByteArrayInputStream;
 import java.util.Arrays;
 import java.util.Map;
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,20 +44,11 @@ public class KafkaJSONMessageDecoder implements StreamMessageDecoder<byte[]> {
   @Override
   public GenericRow decode(byte[] payload, GenericRow destination) {
     try {
-      String text = new String(payload, "UTF-8");
-      JSONObject message = new JSONObject(text);
-
-      for (FieldSpec dimensionSpec : schema.getDimensionFieldSpecs()) {
-        readFieldValue(destination, message, dimensionSpec);
-      }
-
-      for (FieldSpec metricSpec : schema.getMetricFieldSpecs()) {
-        readFieldValue(destination, message, metricSpec);
+      JsonNode message = JsonUtils.bytesToJsonNode(payload);
+      for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+        String column = fieldSpec.getName();
+        destination.putField(column, JsonUtils.extractValue(message.get(column), fieldSpec));
       }
-
-      TimeFieldSpec timeSpec = schema.getTimeFieldSpec();
-      readFieldValue(destination, message, timeSpec);
-
       return destination;
     } catch (Exception e) {
       LOGGER.error("Caught exception while decoding row, discarding row.", e);
@@ -66,63 +56,8 @@ public class KafkaJSONMessageDecoder implements StreamMessageDecoder<byte[]> {
     }
   }
 
-  private void readFieldValue(GenericRow destination, JSONObject message, FieldSpec dimensionSpec)
-      throws JSONException {
-    String columnName = dimensionSpec.getName();
-    if (message.has(columnName) && !message.isNull(columnName)) {
-      Object entry;
-      if (dimensionSpec.isSingleValueField()) {
-        entry = stringToDataType(dimensionSpec, message.get(columnName).toString());
-      } else {
-        JSONArray jsonArray = message.getJSONArray(columnName);
-        Object[] array = new Object[jsonArray.length()];
-        for (int i = 0; i < array.length; i++) {
-          array[i] = stringToDataType(dimensionSpec, jsonArray.getString(i));
-        }
-        if (array.length == 0) {
-          entry = new Object[]{dimensionSpec.getDefaultNullValue()};
-        } else {
-          entry = array;
-        }
-      }
-      destination.putField(columnName, entry);
-    } else {
-      Object entry = dimensionSpec.getDefaultNullValue();
-      destination.putField(columnName, entry);
-    }
-  }
-
   @Override
   public GenericRow decode(byte[] payload, int offset, int length, GenericRow destination) {
     return decode(Arrays.copyOfRange(payload, offset, offset + length), destination);
   }
-
-  private Object stringToDataType(FieldSpec spec, String inString) {
-    if (inString == null) {
-      return spec.getDefaultNullValue();
-    }
-
-    try {
-      switch (spec.getDataType()) {
-        case INT:
-          return Integer.parseInt(inString);
-        case LONG:
-          return Long.parseLong(inString);
-        case FLOAT:
-          return Float.parseFloat(inString);
-        case DOUBLE:
-          return Double.parseDouble(inString);
-        case BOOLEAN:
-        case STRING:
-          return inString;
-        default:
-          return null;
-      }
-    } catch (NumberFormatException e) {
-      Object nullValue = spec.getDefaultNullValue();
-      LOGGER.warn("Failed to parse {} as a value of type {} for column {}, defaulting to {}", inString,
-          spec.getDataType(), spec.getName(), nullValue, e);
-      return nullValue;
-    }
-  }
 }
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/ColumnMetadata.java b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/ColumnMetadata.java
index 5f6592c..5536fcf 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/ColumnMetadata.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/ColumnMetadata.java
@@ -18,6 +18,7 @@
  */
 package com.linkedin.pinot.core.segment.index;
 
+import com.fasterxml.jackson.annotation.JsonProperty;
 import com.linkedin.pinot.common.config.ColumnPartitionConfig;
 import com.linkedin.pinot.common.data.DateTimeFieldSpec;
 import com.linkedin.pinot.common.data.DimensionFieldSpec;
@@ -37,13 +38,11 @@ import java.util.concurrent.TimeUnit;
 import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.commons.lang.math.IntRange;
-import org.codehaus.jackson.annotate.JsonProperty;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static com.linkedin.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Column.*;
-import static com.linkedin.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Segment.SEGMENT_PADDING_CHARACTER;
-import static com.linkedin.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Segment.TIME_UNIT;
+import static com.linkedin.pinot.core.segment.creator.impl.V1Constants.MetadataKeys.Segment.*;
 
 
 public class ColumnMetadata {
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/SegmentMetadataImpl.java b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/SegmentMetadataImpl.java
index 8407ebd..34901c0 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/SegmentMetadataImpl.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/SegmentMetadataImpl.java
@@ -18,12 +18,17 @@
  */
 package com.linkedin.pinot.core.segment.index;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.google.common.base.Preconditions;
 import com.linkedin.pinot.common.data.MetricFieldSpec;
 import com.linkedin.pinot.common.data.Schema;
 import com.linkedin.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
 import com.linkedin.pinot.common.segment.SegmentMetadata;
 import com.linkedin.pinot.common.segment.StarTreeMetadata;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.time.TimeUtils;
 import com.linkedin.pinot.core.indexsegment.generator.SegmentVersion;
 import com.linkedin.pinot.core.segment.creator.impl.V1Constants;
@@ -52,12 +57,8 @@ import javax.annotation.Nullable;
 import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.commons.lang.StringEscapeUtils;
-import org.codehaus.jackson.map.ObjectMapper;
 import org.joda.time.Duration;
 import org.joda.time.Interval;
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -596,62 +597,53 @@ public class SegmentMetadataImpl implements SegmentMetadata {
    *                     the parameter value is null
    * @return json representation of segment metadata
    */
-  public JSONObject toJson(@Nullable Set<String> columnFilter) throws JSONException {
-    JSONObject rootMeta = new JSONObject();
-    try {
-      rootMeta.put("segmentName", _segmentName);
-      rootMeta.put("schemaName", _schema != null ? _schema.getSchemaName() : JSONObject.NULL);
-      rootMeta.put("crc", _crc);
-      rootMeta.put("creationTimeMillis", _creationTime);
-      TimeZone timeZone = TimeZone.getTimeZone("UTC");
-      DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss:SSS' UTC'");
-      dateFormat.setTimeZone(timeZone);
-      String creationTimeStr = _creationTime != Long.MIN_VALUE ? dateFormat.format(new Date(_creationTime)) : "";
-      rootMeta.put("creationTimeReadable", creationTimeStr);
-      rootMeta.put("timeGranularitySec", _timeGranularity != null ? _timeGranularity.getStandardSeconds() : null);
-      if (_timeInterval == null) {
-        rootMeta.put("startTimeMillis", (String) null);
-        rootMeta.put("startTimeReadable", "null");
-        rootMeta.put("endTimeMillis", (String) null);
-        rootMeta.put("endTimeReadable", "null");
-      } else {
-        rootMeta.put("startTimeMillis", _timeInterval.getStartMillis());
-        rootMeta.put("startTimeReadable", _timeInterval.getStart().toString());
-        rootMeta.put("endTimeMillis", _timeInterval.getEndMillis());
-        rootMeta.put("endTimeReadable", _timeInterval.getEnd().toString());
-      }
-
-      rootMeta.put("pushTimeMillis", _pushTime);
-      String pushTimeStr = _pushTime != Long.MIN_VALUE ? dateFormat.format(new Date(_pushTime)) : "";
-      rootMeta.put("pushTimeReadable", pushTimeStr);
+  public JsonNode toJson(@Nullable Set<String> columnFilter) throws JsonProcessingException {
+    ObjectNode segmentMetadata = JsonUtils.newObjectNode();
+    segmentMetadata.put("segmentName", _segmentName);
+    segmentMetadata.put("schemaName", _schema != null ? _schema.getSchemaName() : null);
+    segmentMetadata.put("crc", _crc);
+    segmentMetadata.put("creationTimeMillis", _creationTime);
+    TimeZone timeZone = TimeZone.getTimeZone("UTC");
+    DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss:SSS' UTC'");
+    dateFormat.setTimeZone(timeZone);
+    String creationTimeStr = _creationTime != Long.MIN_VALUE ? dateFormat.format(new Date(_creationTime)) : null;
+    segmentMetadata.put("creationTimeReadable", creationTimeStr);
+    segmentMetadata.put("timeGranularitySec", _timeGranularity != null ? _timeGranularity.getStandardSeconds() : null);
+    if (_timeInterval == null) {
+      segmentMetadata.set("startTimeMillis", null);
+      segmentMetadata.set("startTimeReadable", null);
+      segmentMetadata.set("endTimeMillis", null);
+      segmentMetadata.set("endTimeReadable", null);
+    } else {
+      segmentMetadata.put("startTimeMillis", _timeInterval.getStartMillis());
+      segmentMetadata.put("startTimeReadable", _timeInterval.getStart().toString());
+      segmentMetadata.put("endTimeMillis", _timeInterval.getEndMillis());
+      segmentMetadata.put("endTimeReadable", _timeInterval.getEnd().toString());
+    }
 
-      rootMeta.put("refreshTimeMillis", _refreshTime);
-      String refreshTimeStr = _refreshTime != Long.MIN_VALUE ? dateFormat.format(new Date(_refreshTime)) : "";
-      rootMeta.put("refreshTimeReadable", refreshTimeStr);
+    segmentMetadata.put("pushTimeMillis", _pushTime);
+    String pushTimeStr = _pushTime != Long.MIN_VALUE ? dateFormat.format(new Date(_pushTime)) : null;
+    segmentMetadata.put("pushTimeReadable", pushTimeStr);
 
-      rootMeta.put("segmentVersion", _segmentVersion.toString());
-      rootMeta.put("hasStarTree", hasStarTree());
-      rootMeta.put("creatorName", _creatorName == null ? JSONObject.NULL : _creatorName);
-      rootMeta.put("paddingCharacter", String.valueOf(_paddingCharacter));
-      rootMeta.put("hllLog2m", _hllLog2m);
+    segmentMetadata.put("refreshTimeMillis", _refreshTime);
+    String refreshTimeStr = _refreshTime != Long.MIN_VALUE ? dateFormat.format(new Date(_refreshTime)) : null;
+    segmentMetadata.put("refreshTimeReadable", refreshTimeStr);
 
-      JSONArray columnsJson = new JSONArray();
-      ObjectMapper mapper = new ObjectMapper();
+    segmentMetadata.put("segmentVersion", _segmentVersion.toString());
+    segmentMetadata.put("hasStarTree", hasStarTree());
+    segmentMetadata.put("creatorName", _creatorName);
+    segmentMetadata.put("paddingCharacter", String.valueOf(_paddingCharacter));
+    segmentMetadata.put("hllLog2m", _hllLog2m);
 
-      for (String column : _allColumns) {
-        if (columnFilter != null && !columnFilter.contains(column)) {
-          continue;
-        }
-        ColumnMetadata columnMetadata = _columnMetadataMap.get(column);
-        JSONObject columnJson = new JSONObject(mapper.writeValueAsString(columnMetadata));
-        columnsJson.put(columnJson);
+    ArrayNode columnsMetadata = JsonUtils.newArrayNode();
+    for (String column : _allColumns) {
+      if (columnFilter != null && !columnFilter.contains(column)) {
+        continue;
       }
-
-      rootMeta.put("columns", columnsJson);
-      return rootMeta;
-    } catch (Exception e) {
-      LOGGER.error("Failed to convert field to json for segment: {}", _segmentName, e);
-      throw new RuntimeException("Failed to convert segment metadata to json", e);
+      columnsMetadata.add(JsonUtils.objectToJsonNode(_columnMetadataMap.get(column)));
     }
+    segmentMetadata.set("columns", columnsMetadata);
+
+    return segmentMetadata;
   }
 }
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/loader/IndexLoadingConfig.java b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/loader/IndexLoadingConfig.java
index 7ce6ef4..5063eb8 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/loader/IndexLoadingConfig.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/loader/IndexLoadingConfig.java
@@ -91,7 +91,7 @@ public class IndexLoadingConfig {
       _noDictionaryColumns.addAll(noDictionaryColumns);
     }
 
-    Map<String, String> noDictionaryConfig = indexingConfig.getnoDictionaryConfig();
+    Map<String, String> noDictionaryConfig = indexingConfig.getNoDictionaryConfig();
     if (noDictionaryConfig != null) {
       _noDictionaryConfig.putAll(noDictionaryConfig);
     }
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/util/trace/TraceContext.java b/pinot-core/src/main/java/com/linkedin/pinot/core/util/trace/TraceContext.java
index aa465e3..e490ba6 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/util/trace/TraceContext.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/util/trace/TraceContext.java
@@ -18,17 +18,18 @@
  */
 package com.linkedin.pinot.core.util.trace;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.google.common.annotations.VisibleForTesting;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import javax.annotation.Nullable;
-import org.json.JSONArray;
-import org.json.JSONObject;
 
 
 /**
@@ -60,8 +61,10 @@ public final class TraceContext {
         _value = value;
       }
 
-      JSONObject toJson() {
-        return new JSONObject(Collections.singletonMap(_key, _value));
+      JsonNode toJson() {
+        ObjectNode jsonLog = JsonUtils.newObjectNode();
+        jsonLog.set(_key, JsonUtils.objectToJsonNode(_value));
+        return jsonLog;
       }
     }
 
@@ -85,12 +88,14 @@ public final class TraceContext {
       return _traceId + "_" + _numChildren++;
     }
 
-    JSONObject toJson() {
-      JSONArray jsonLogs = new JSONArray();
+    JsonNode toJson() {
+      ArrayNode jsonLogs = JsonUtils.newArrayNode();
       for (LogEntry log : _logs) {
-        jsonLogs.put(log.toJson());
+        jsonLogs.add(log.toJson());
       }
-      return new JSONObject(Collections.singletonMap(_traceId, jsonLogs));
+      ObjectNode jsonTrace = JsonUtils.newObjectNode();
+      jsonTrace.set(_traceId, jsonLogs);
+      return jsonTrace;
     }
   }
 
@@ -184,9 +189,9 @@ public final class TraceContext {
    * Get the trace information added so far.
    */
   public static String getTraceInfo() {
-    JSONArray jsonTraces = new JSONArray();
+    ArrayNode jsonTraces = JsonUtils.newArrayNode();
     for (Trace trace : REQUEST_TO_TRACES_MAP.get(TRACE_ENTRY_THREAD_LOCAL.get()._requestId)) {
-      jsonTraces.put(trace.toJson());
+      jsonTraces.add(trace.toJson());
     }
     return jsonTraces.toString();
   }
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java b/pinot-core/src/main/java/com/linkedin/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java
index 2ad97b4..6abb9fe 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/server/realtime/ServerSegmentCompletionProtocolHandler.java
@@ -165,7 +165,7 @@ public class ServerSegmentCompletionProtocolHandler {
       String responseStr =
           _fileUploadDownloadClient.sendSegmentCompletionProtocolRequest(new URI(url), OTHER_REQUESTS_TIMEOUT)
               .getResponse();
-      response = new SegmentCompletionProtocol.Response(responseStr);
+      response = SegmentCompletionProtocol.Response.fromJsonString(responseStr);
       LOGGER.info("Controller response {} for {}", response.toJsonString(), url);
       if (response.getStatus().equals(SegmentCompletionProtocol.ControllerResponseStatus.NOT_LEADER)) {
         ControllerLeaderLocator.getInstance().invalidateCachedControllerLeader();
@@ -189,7 +189,7 @@ public class ServerSegmentCompletionProtocolHandler {
       String responseStr =
           _fileUploadDownloadClient.uploadSegment(new URI(url), segmentName, segmentTarFile, null, null,
               SEGMENT_UPLOAD_REQUEST_TIMEOUT_MS).getResponse();
-      response = new SegmentCompletionProtocol.Response(responseStr);
+      response = SegmentCompletionProtocol.Response.fromJsonString(responseStr);
       LOGGER.info("Controller response {} for {}", response.toJsonString(), url);
       if (response.getStatus().equals(SegmentCompletionProtocol.ControllerResponseStatus.NOT_LEADER)) {
         ControllerLeaderLocator.getInstance().invalidateCachedControllerLeader();
diff --git a/pinot-core/src/test/java/com/linkedin/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManagerTest.java b/pinot-core/src/test/java/com/linkedin/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManagerTest.java
index a9901a2..a8aae44 100644
--- a/pinot-core/src/test/java/com/linkedin/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManagerTest.java
+++ b/pinot-core/src/test/java/com/linkedin/pinot/core/data/manager/realtime/LLRealtimeSegmentDataManagerTest.java
@@ -25,6 +25,7 @@ import com.linkedin.pinot.common.metadata.segment.LLCRealtimeSegmentZKMetadata;
 import com.linkedin.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
 import com.linkedin.pinot.common.metrics.ServerMetrics;
 import com.linkedin.pinot.common.protocols.SegmentCompletionProtocol;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.LLCSegmentName;
 import com.linkedin.pinot.core.data.GenericRow;
 import com.linkedin.pinot.core.data.manager.config.InstanceDataManagerConfig;
@@ -42,10 +43,9 @@ import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.util.LinkedList;
 import java.util.Map;
-import junit.framework.Assert;
 import org.apache.commons.io.FileUtils;
 import org.apache.kafka.common.protocol.Errors;
-import org.json.JSONObject;
+import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
@@ -115,7 +115,7 @@ public class LLRealtimeSegmentDataManagerTest {
 
   private TableConfig createTableConfig()
       throws Exception {
-    return TableConfig.fromJSONConfig(new JSONObject(_tableConfigJson));
+    return TableConfig.fromJsonString(_tableConfigJson);
   }
 
   private RealtimeTableDataManager createTableDataManager() {
@@ -684,7 +684,7 @@ public class LLRealtimeSegmentDataManagerTest {
 
     public boolean invokeCommit(String segTarFileName) {
       SegmentCompletionProtocol.Response response = mock(SegmentCompletionProtocol.Response.class);
-      when(response.getIsSplitCommit()).thenReturn(false);
+      when(response.isSplitCommit()).thenReturn(false);
       return super.commitSegment(response);
     }
 
diff --git a/pinot-core/src/test/java/com/linkedin/pinot/core/data/readers/JSONRecordReaderTest.java b/pinot-core/src/test/java/com/linkedin/pinot/core/data/readers/JSONRecordReaderTest.java
index 179ecf4..addd51a 100644
--- a/pinot-core/src/test/java/com/linkedin/pinot/core/data/readers/JSONRecordReaderTest.java
+++ b/pinot-core/src/test/java/com/linkedin/pinot/core/data/readers/JSONRecordReaderTest.java
@@ -18,11 +18,11 @@
  */
 package com.linkedin.pinot.core.data.readers;
 
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.io.File;
 import java.io.FileWriter;
 import org.apache.commons.io.FileUtils;
-import org.json.JSONArray;
-import org.json.JSONObject;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
@@ -38,12 +38,12 @@ public class JSONRecordReaderTest extends RecordReaderTest {
 
     try (FileWriter fileWriter = new FileWriter(DATA_FILE)) {
       for (Object[] record : RECORDS) {
-        JSONObject jsonRecord = new JSONObject();
+        ObjectNode jsonRecord = JsonUtils.newObjectNode();
         if (record[0] != null) {
-          jsonRecord.put(COLUMNS[0], record[0]);
+          jsonRecord.set(COLUMNS[0], JsonUtils.objectToJsonNode(record[0]));
         }
         if (record[1] != null) {
-          jsonRecord.put(COLUMNS[1], new JSONArray(record[1]));
+          jsonRecord.set(COLUMNS[1], JsonUtils.objectToJsonNode(record[1]));
         }
         fileWriter.write(jsonRecord.toString());
       }
diff --git a/pinot-core/src/test/java/com/linkedin/pinot/core/segment/index/SegmentMetadataImplTest.java b/pinot-core/src/test/java/com/linkedin/pinot/core/segment/index/SegmentMetadataImplTest.java
index 10ba828..853021e 100644
--- a/pinot-core/src/test/java/com/linkedin/pinot/core/segment/index/SegmentMetadataImplTest.java
+++ b/pinot-core/src/test/java/com/linkedin/pinot/core/segment/index/SegmentMetadataImplTest.java
@@ -18,6 +18,7 @@
  */
 package com.linkedin.pinot.core.segment.index;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.linkedin.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
 import com.linkedin.pinot.core.segment.creator.SegmentIndexCreationDriver;
 import com.linkedin.pinot.core.segment.creator.impl.SegmentCreationDriverFactory;
@@ -27,34 +28,30 @@ import java.io.File;
 import java.io.IOException;
 import java.nio.file.Files;
 import java.util.concurrent.TimeUnit;
-import junit.framework.Assert;
-import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.io.FileUtils;
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
 import org.testng.annotations.AfterMethod;
 import org.testng.annotations.BeforeMethod;
 import org.testng.annotations.Test;
 
+import static org.testng.Assert.*;
+
 
 public class SegmentMetadataImplTest {
- private static final String AVRO_DATA = "data/test_data-mv.avro";
+  private static final String AVRO_DATA = "data/test_data-mv.avro";
   private File INDEX_DIR;
   private File segmentDirectory;
 
   @BeforeMethod
-  public void setUp()
-      throws Exception {
+  public void setUp() throws Exception {
     INDEX_DIR = Files.createTempDirectory(SegmentMetadataImplTest.class.getName() + "_segmentDir").toFile();
 
     final String filePath =
         TestUtils.getFileFromResourceUrl(SegmentMetadataImplTest.class.getClassLoader().getResource(AVRO_DATA));
 
     // intentionally changed this to TimeUnit.Hours to make it non-default for testing
-    final SegmentGeneratorConfig config = SegmentTestUtils
-        .getSegmentGenSpecWithSchemAndProjectedColumns(new File(filePath), INDEX_DIR, "daysSinceEpoch", TimeUnit.HOURS,
-            "testTable");
+    final SegmentGeneratorConfig config =
+        SegmentTestUtils.getSegmentGenSpecWithSchemAndProjectedColumns(new File(filePath), INDEX_DIR, "daysSinceEpoch",
+            TimeUnit.HOURS, "testTable");
     config.setSegmentNamePostfix("1");
     config.setTimeColumnName("daysSinceEpoch");
     final SegmentIndexCreationDriver driver = SegmentCreationDriverFactory.get(null);
@@ -69,33 +66,35 @@ public class SegmentMetadataImplTest {
   }
 
   @Test
-  public void testToJson()
-      throws IOException, ConfigurationException, JSONException {
+  public void testToJson() throws IOException {
     SegmentMetadataImpl metadata = new SegmentMetadataImpl(segmentDirectory);
-    Assert.assertNotNull(metadata);
-    JSONObject jsonMeta = metadata.toJson(null);
-    Assert.assertEquals(jsonMeta.get("segmentName"), metadata.getName());
-    Assert.assertEquals(jsonMeta.getLong("crc"), Long.valueOf(metadata.getCrc()).longValue());
-    Assert.assertEquals(jsonMeta.getString("paddingCharacter"), String.valueOf(metadata.getPaddingCharacter()));
-    Assert.assertEquals(jsonMeta.get("creatorName"), metadata.getCreatorName());
-    Assert.assertEquals(jsonMeta.get("creationTimeMillis"), metadata.getIndexCreationTime());
-    Assert.assertEquals(jsonMeta.get("startTimeMillis"), metadata.getTimeInterval().getStartMillis());
-    Assert.assertEquals(jsonMeta.get("endTimeMillis"), metadata.getTimeInterval().getEndMillis());
-    Assert.assertEquals(jsonMeta.get("pushTimeMillis"), metadata.getPushTime());
-    Assert.assertEquals(jsonMeta.get("refreshTimeMillis"), metadata.getPushTime());
-    JSONArray jsonColumnList = jsonMeta.getJSONArray("columns");
-    Assert.assertEquals(jsonColumnList.length(), metadata.getAllColumns().size());
-    for (int i = 0; i < jsonColumnList.length(); i++) {
-      JSONObject  jsonColumn = jsonColumnList.getJSONObject(i);
-      ColumnMetadata colMeta = metadata.getColumnMetadataFor(jsonColumn.getString("columnName"));
-      Assert.assertEquals(jsonColumn.get("cardinality"), colMeta.getCardinality());
-      Assert.assertEquals(jsonColumn.get("totalRawDocs"), colMeta.getTotalRawDocs());
-      Assert.assertEquals(jsonColumn.get("bitsPerElement"), colMeta.getBitsPerElement());
-      Assert.assertEquals(jsonColumn.getBoolean("sorted"), colMeta.isSorted());
-      Assert.assertEquals(jsonColumn.get("totalAggDocs"), colMeta.getTotalAggDocs());
-      Assert.assertEquals(jsonColumn.get("containsNulls"), colMeta.hasNulls());
-      Assert.assertEquals(jsonColumn.getBoolean("hasDictionary"), colMeta.hasDictionary());
-    }
+    assertNotNull(metadata);
 
+    JsonNode jsonMeta = metadata.toJson(null);
+    assertEquals(jsonMeta.get("segmentName").asText(), metadata.getName());
+    assertEquals(jsonMeta.get("crc").asLong(), Long.valueOf(metadata.getCrc()).longValue());
+    assertEquals(jsonMeta.get("paddingCharacter").asText(), String.valueOf(metadata.getPaddingCharacter()));
+    assertTrue(jsonMeta.get("creatorName").isNull());
+    assertEquals(jsonMeta.get("creationTimeMillis").asLong(), metadata.getIndexCreationTime());
+    assertEquals(jsonMeta.get("startTimeMillis").asLong(), metadata.getTimeInterval().getStartMillis());
+    assertEquals(jsonMeta.get("endTimeMillis").asLong(), metadata.getTimeInterval().getEndMillis());
+    assertEquals(jsonMeta.get("pushTimeMillis").asLong(), metadata.getPushTime());
+    assertEquals(jsonMeta.get("refreshTimeMillis").asLong(), metadata.getPushTime());
+
+    JsonNode jsonColumnsMeta = jsonMeta.get("columns");
+    int numColumns = jsonColumnsMeta.size();
+    assertEquals(numColumns, metadata.getAllColumns().size());
+    for (int i = 0; i < numColumns; i++) {
+      JsonNode jsonColumnMeta = jsonColumnsMeta.get(i);
+      ColumnMetadata columnMeta = metadata.getColumnMetadataFor(jsonColumnMeta.get("columnName").asText());
+      assertNotNull(columnMeta);
+      assertEquals(jsonColumnMeta.get("cardinality").asInt(), columnMeta.getCardinality());
+      assertEquals(jsonColumnMeta.get("totalRawDocs").asInt(), columnMeta.getTotalRawDocs());
+      assertEquals(jsonColumnMeta.get("bitsPerElement").asInt(), columnMeta.getBitsPerElement());
+      assertEquals(jsonColumnMeta.get("sorted").asBoolean(), columnMeta.isSorted());
+      assertEquals(jsonColumnMeta.get("totalAggDocs").asInt(), columnMeta.getTotalAggDocs());
+      assertEquals(jsonColumnMeta.get("containsNulls").asBoolean(), columnMeta.hasNulls());
+      assertEquals(jsonColumnMeta.get("hasDictionary").asBoolean(), columnMeta.hasDictionary());
+    }
   }
 }
diff --git a/pinot-core/src/test/java/com/linkedin/pinot/core/util/trace/TraceContextTest.java b/pinot-core/src/test/java/com/linkedin/pinot/core/util/trace/TraceContextTest.java
index 14ad642..5f9343a 100644
--- a/pinot-core/src/test/java/com/linkedin/pinot/core/util/trace/TraceContextTest.java
+++ b/pinot-core/src/test/java/com/linkedin/pinot/core/util/trace/TraceContextTest.java
@@ -18,7 +18,8 @@
  */
 package com.linkedin.pinot.core.util.trace;
 
-import java.util.Collections;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Queue;
@@ -27,7 +28,6 @@ import java.util.Set;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
-import org.json.JSONObject;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -117,6 +117,8 @@ public class TraceContextTest {
   }
 
   private static String getTraceString(String key, Object value) {
-    return new JSONObject(Collections.singletonMap(key, value)).toString();
+    ObjectNode jsonTrace = JsonUtils.newObjectNode();
+    jsonTrace.set(key, JsonUtils.objectToJsonNode(value));
+    return jsonTrace.toString();
   }
 }
\ No newline at end of file
diff --git a/pinot-core/src/test/java/com/linkedin/pinot/index/persist/AvroDataPublisherTest.java b/pinot-core/src/test/java/com/linkedin/pinot/index/persist/AvroDataPublisherTest.java
index c1c577e..594fc73 100644
--- a/pinot-core/src/test/java/com/linkedin/pinot/index/persist/AvroDataPublisherTest.java
+++ b/pinot-core/src/test/java/com/linkedin/pinot/index/persist/AvroDataPublisherTest.java
@@ -18,8 +18,10 @@
  */
 package com.linkedin.pinot.index.persist;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.linkedin.pinot.common.data.FieldSpec.DataType;
 import com.linkedin.pinot.common.data.Schema;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.core.data.GenericRow;
 import com.linkedin.pinot.core.data.readers.AvroRecordReader;
 import com.linkedin.pinot.core.data.readers.FileFormat;
@@ -32,7 +34,6 @@ import java.io.File;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.commons.io.FileUtils;
-import org.json.JSONObject;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
@@ -63,13 +64,12 @@ public class AvroDataPublisherTest {
 
     int cnt = 0;
     for (String line : FileUtils.readLines(new File(jsonPath))) {
-
-      JSONObject obj = new JSONObject(line);
+      JsonNode jsonNode = JsonUtils.stringToJsonNode(line);
       if (avroDataPublisher.hasNext()) {
         GenericRow recordRow = avroDataPublisher.next();
 
         for (String column : recordRow.getFieldNames()) {
-          String valueFromJson = obj.get(column).toString();
+          String valueFromJson = jsonNode.get(column).asText();
           String valueFromAvro = recordRow.getValue(column).toString();
           if (cnt > 1) {
             Assert.assertEquals(valueFromJson, valueFromAvro);
@@ -104,14 +104,13 @@ public class AvroDataPublisherTest {
 
     int cnt = 0;
     for (final String line : FileUtils.readLines(new File(jsonPath))) {
-
-      final JSONObject obj = new JSONObject(line);
+      JsonNode jsonNode = JsonUtils.stringToJsonNode(line);
       if (avroDataPublisher.hasNext()) {
         final GenericRow recordRow = avroDataPublisher.next();
         // System.out.println(recordRow);
         Assert.assertEquals(recordRow.getFieldNames().length, 2);
         for (final String column : recordRow.getFieldNames()) {
-          final String valueFromJson = obj.get(column).toString();
+          final String valueFromJson = jsonNode.get(column).asText();
           final String valueFromAvro = recordRow.getValue(column).toString();
           if (cnt > 1) {
             Assert.assertEquals(valueFromAvro, valueFromJson);
diff --git a/pinot-core/src/test/java/com/linkedin/pinot/util/TestUtils.java b/pinot-core/src/test/java/com/linkedin/pinot/util/TestUtils.java
index 00934f8..5db3aa9 100644
--- a/pinot-core/src/test/java/com/linkedin/pinot/util/TestUtils.java
+++ b/pinot-core/src/test/java/com/linkedin/pinot/util/TestUtils.java
@@ -29,8 +29,6 @@ import java.util.Map;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 import org.apache.commons.io.FileUtils;
-import org.json.JSONArray;
-import org.json.JSONException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.testng.Assert;
@@ -120,40 +118,6 @@ public class TestUtils {
     LOGGER.info("group overlap rate: " + (cnt+0.0)/mapEstimate.keySet().size());
   }
 
-  public static void assertJSONArrayApproximation(JSONArray jsonArrayEstimate, JSONArray jsonArrayActual, double precision) {
-    LOGGER.info("====== assertJSONArrayApproximation ======");
-    try {
-      HashMap<String, Double> mapEstimate = genMapFromJSONArray(jsonArrayEstimate);
-      HashMap<String, Double> mapActual = genMapFromJSONArray(jsonArrayActual);
-
-      // estimation should not affect number of groups formed
-      Assert.assertEquals(mapEstimate.keySet().size(), mapActual.keySet().size());
-      LOGGER.info("estimate: " + mapEstimate.keySet());
-      LOGGER.info("actual: " + mapActual.keySet());
-      int cnt = 0;
-      for (String key: mapEstimate.keySet()) {
-        // Not strictly enforced, since in quantile, top 100 groups from accurate maybe not be top 100 from estimate
-        // Assert.assertEquals(mapActual.keySet().contains(key), true);
-        if (mapActual.keySet().contains(key)) {
-          assertApproximation(mapEstimate.get(key), mapActual.get(key), precision);
-          cnt += 1;
-        }
-      }
-      LOGGER.info("group overlap rate: " + (cnt+0.0)/mapEstimate.keySet().size());
-    } catch (JSONException e) {
-      e.printStackTrace();
-    }
-  }
-
-  private static HashMap<String, Double> genMapFromJSONArray(JSONArray array) throws JSONException {
-    HashMap<String, Double> map = new HashMap<String, Double>();
-    for (int i = 0; i < array.length(); ++i) {
-      map.put(array.getJSONObject(i).getJSONArray("group").getString(0),
-          array.getJSONObject(i).getDouble("value"));
-    }
-    return map;
-  }
-
   /**
    * Ensure the given directories exist and are empty.
    *
diff --git a/pinot-hadoop/src/main/java/com/linkedin/pinot/hadoop/io/JsonPinotOutputFormat.java b/pinot-hadoop/src/main/java/com/linkedin/pinot/hadoop/io/JsonPinotOutputFormat.java
index a156c02..5e0ad55 100644
--- a/pinot-hadoop/src/main/java/com/linkedin/pinot/hadoop/io/JsonPinotOutputFormat.java
+++ b/pinot-hadoop/src/main/java/com/linkedin/pinot/hadoop/io/JsonPinotOutputFormat.java
@@ -18,113 +18,81 @@
  */
 package com.linkedin.pinot.hadoop.io;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.linkedin.pinot.common.data.FieldSpec;
 import com.linkedin.pinot.common.data.Schema;
-
+import com.linkedin.pinot.common.utils.JsonUtils;
+import java.io.IOException;
+import java.io.Serializable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.JobContext;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
 
-import java.io.IOException;
-import java.io.Serializable;
 
 /**
  * OutputFormat implementation for Json source
  */
 public class JsonPinotOutputFormat<K, V extends Serializable> extends PinotOutputFormat<K, V> {
+  private static final String JSON_READER_CLASS = "json.reader.class";
 
-    private static final String JSON_READER_CLASS = "json.reader.class";
+  @Override
+  public void configure(Configuration conf) {
+    conf.set(PinotOutputFormat.PINOT_RECORD_SERIALIZATION_CLASS, JsonPinotRecordSerialization.class.getName());
+  }
 
-    @Override
-    public void configure(Configuration conf) {
-        conf.set(PinotOutputFormat.PINOT_RECORD_SERIALIZATION_CLASS, JsonPinotRecordSerialization.class.getName());
-    }
-
-    public static void setJsonReaderClass(JobContext context, Class<?> clazz) {
-        context.getConfiguration().set(JSON_READER_CLASS, clazz.getName());
-    }
+  public static void setJsonReaderClass(JobContext context, Class<?> clazz) {
+    context.getConfiguration().set(JSON_READER_CLASS, clazz.getName());
+  }
 
-    public static String getJsonReaderClass(Configuration conf) {
-        if (conf.get(JSON_READER_CLASS) == null) {
-            throw new RuntimeException("Json reader class not set");
-        }
-        return conf.get(JSON_READER_CLASS);
+  public static String getJsonReaderClass(Configuration conf) {
+    if (conf.get(JSON_READER_CLASS) == null) {
+      throw new RuntimeException("Json reader class not set");
     }
+    return conf.get(JSON_READER_CLASS);
+  }
 
-    public static class JsonPinotRecordSerialization<T> implements PinotRecordSerialization<T> {
-
-        private PinotRecord _record;
-        private Schema _schema;
-        private ObjectMapper _mapper;
-        private Configuration _conf;
-
-        @Override
-        public void init(Configuration conf, Schema schema) {
-            _schema = schema;
-            _mapper = new ObjectMapper();
-            _conf = conf;
-        }
+  public static class JsonPinotRecordSerialization<T> implements PinotRecordSerialization<T> {
+    private Schema _schema;
+    private Configuration _conf;
+    private PinotRecord _record;
 
-        @Override
-        public PinotRecord serialize(T t) throws IOException {
-            _record = PinotRecord.createOrReuseRecord(_record, _schema);
-            try {
-                JSONObject obj = new JSONObject(_mapper.writeValueAsString(t));
-                for (FieldSpec fieldSpec : _record.getSchema().getAllFieldSpecs()) {
-                    Object fieldValue = getFieldValue(obj, fieldSpec);
-                    _record.putField(fieldSpec.getName(), fieldValue);
-                }
-            } catch (JSONException e) {
-                throw new RuntimeException("Serialization exception", e);
-            }
-            return _record;
-        }
-
-        @Override
-        public T deserialize(PinotRecord record) throws IOException {
-            JSONObject obj = new JSONObject();
-            for (FieldSpec fieldSpec : _record.getSchema().getAllFieldSpecs()) {
-                Object value = record.getValue(fieldSpec.getName());
-                addJsonField(obj, fieldSpec, value);
-            }
-            return _mapper.readValue(obj.toString().getBytes("UTF-8"), getJsonReaderClass(_conf));
-        }
-
-        private void addJsonField(JSONObject obj, FieldSpec fieldSpec, Object value) {
-            try {
-                if (value instanceof Object[]) {
-                    obj.put(fieldSpec.getName(), new JSONArray(value));
-                } else {
-                    obj.put(fieldSpec.getName(), value);
-                }
-            } catch (JSONException e) {
-                throw new RuntimeException("Error initialize json reader class", e);
-            }
-        }
-
-        @Override
-        public void close() {
+    @Override
+    public void init(Configuration conf, Schema schema) {
+      _schema = schema;
+      _conf = conf;
+      _record = new PinotRecord(_schema);
+    }
 
-        }
+    @Override
+    public PinotRecord serialize(T t) {
+      _record.clear();
+      JsonNode jsonRecord = JsonUtils.objectToJsonNode(t);
+      for (FieldSpec fieldSpec : _schema.getAllFieldSpecs()) {
+        String column = fieldSpec.getName();
+        _record.putField(column, JsonUtils.extractValue(jsonRecord.get(column), fieldSpec));
+      }
+      return _record;
+    }
 
-        private Class<T> getJsonReaderClass(Configuration conf) {
-            try {
-                return (Class<T>) Class.forName(JsonPinotOutputFormat.getJsonReaderClass(conf));
-            } catch (ClassNotFoundException e) {
-                throw new RuntimeException("Error initialize json reader class", e);
-            }
-        }
+    @Override
+    public T deserialize(PinotRecord record) throws IOException {
+      ObjectNode jsonRecord = JsonUtils.newObjectNode();
+      for (String column : _schema.getColumnNames()) {
+        jsonRecord.set(column, JsonUtils.objectToJsonNode(record.getValue(column)));
+      }
+      return JsonUtils.jsonNodeToObject(jsonRecord, getJsonReaderClass(_conf));
+    }
 
-        private Object getFieldValue(JSONObject obj, FieldSpec fieldSpec) {
-            Object fieldValue = obj.opt(fieldSpec.getName());
-            if (fieldValue != null) {
-                return fieldValue;
-            }
-            return fieldSpec.getDefaultNullValue();
-        }
+    @Override
+    public void close() {
     }
 
+    private Class<T> getJsonReaderClass(Configuration conf) {
+      try {
+        return (Class<T>) Class.forName(JsonPinotOutputFormat.getJsonReaderClass(conf));
+      } catch (ClassNotFoundException e) {
+        throw new RuntimeException("Error initialize json reader class", e);
+      }
+    }
+  }
 }
diff --git a/pinot-hadoop/src/main/java/com/linkedin/pinot/hadoop/job/ControllerRestApi.java b/pinot-hadoop/src/main/java/com/linkedin/pinot/hadoop/job/ControllerRestApi.java
index c77d63c..fa3e6f7 100644
--- a/pinot-hadoop/src/main/java/com/linkedin/pinot/hadoop/job/ControllerRestApi.java
+++ b/pinot-hadoop/src/main/java/com/linkedin/pinot/hadoop/job/ControllerRestApi.java
@@ -18,15 +18,16 @@
  */
 package com.linkedin.pinot.hadoop.job;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.linkedin.pinot.common.config.TableConfig;
 import com.linkedin.pinot.common.utils.FileUploadDownloadClient;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.SimpleHttpResponse;
 import com.linkedin.pinot.hadoop.utils.PushLocation;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.List;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -60,10 +61,9 @@ public class ControllerRestApi {
     for (URI uri : tableConfigURIs) {
       try {
         SimpleHttpResponse response = fileUploadDownloadClient.getTableConfig(uri);
-        JSONObject queryResponse = new JSONObject(response.getResponse());
-        JSONObject offlineTableConfig = queryResponse.getJSONObject(OFFLINE);
-        LOGGER.info("Got table config {}", offlineTableConfig);
-        if (!queryResponse.isNull(OFFLINE)) {
+        JsonNode offlineTableConfig = JsonUtils.stringToJsonNode(response.getResponse()).get(OFFLINE);
+        if (offlineTableConfig != null) {
+          LOGGER.info("Got table config {}", offlineTableConfig);
           return TableConfig.fromJSONConfig(offlineTableConfig);
         }
       } catch (Exception e) {
diff --git a/pinot-hadoop/src/main/java/com/linkedin/pinot/hadoop/job/mapper/HadoopSegmentCreationMapReduceJob.java b/pinot-hadoop/src/main/java/com/linkedin/pinot/hadoop/job/mapper/HadoopSegmentCreationMapReduceJob.java
index 8c13db8..24ba4d0 100644
--- a/pinot-hadoop/src/main/java/com/linkedin/pinot/hadoop/job/mapper/HadoopSegmentCreationMapReduceJob.java
+++ b/pinot-hadoop/src/main/java/com/linkedin/pinot/hadoop/job/mapper/HadoopSegmentCreationMapReduceJob.java
@@ -21,6 +21,7 @@ package com.linkedin.pinot.hadoop.job.mapper;
 import com.linkedin.pinot.common.config.TableConfig;
 import com.linkedin.pinot.common.data.Schema;
 import com.linkedin.pinot.common.utils.DataSize;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.TarGzCompressionUtils;
 import com.linkedin.pinot.core.data.readers.CSVRecordReaderConfig;
 import com.linkedin.pinot.core.data.readers.FileFormat;
@@ -31,6 +32,7 @@ import com.linkedin.pinot.core.segment.creator.impl.SegmentIndexCreationDriverIm
 import com.linkedin.pinot.hadoop.job.JobConfigConstants;
 import java.io.File;
 import java.io.IOException;
+import java.io.InputStream;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -39,10 +41,9 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.json.JSONException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.codehaus.jackson.map.ObjectMapper;
+
 
 public class HadoopSegmentCreationMapReduceJob {
 
@@ -110,16 +111,16 @@ public class HadoopSegmentCreationMapReduceJob {
                 + "\n\ttableName: " + _properties.get(JobConfigConstants.SEGMENT_TABLE_NAME));
       }
 
-        String tableConfigString = _properties.get(JobConfigConstants.TABLE_CONFIG);
-        if (tableConfigString != null) {
-          try {
-            _tableConfig = TableConfig.init(tableConfigString);
-          } catch (JSONException e) {
-            // Though we get table config directly from the controller of hosts and port of push location are set,
-            // it is possible for the user to pass in a table config as a parameter
-            LOGGER.error("Exception when parsing table config: {}", tableConfigString);
-          }
+      String tableConfigString = _properties.get(JobConfigConstants.TABLE_CONFIG);
+      if (tableConfigString != null) {
+        try {
+          _tableConfig = TableConfig.fromJsonString(tableConfigString);
+        } catch (IOException e) {
+          // Though we get table config directly from the controller of hosts and port of push location are set,
+          // it is possible for the user to pass in a table config as a parameter
+          LOGGER.error("Exception when parsing table config: {}", tableConfigString);
         }
+      }
     }
 
     protected String getTableName() {
@@ -212,7 +213,8 @@ public class HadoopSegmentCreationMapReduceJob {
       LOGGER.info("Finished the job successfully");
     }
 
-    protected void setSegmentNameGenerator(SegmentGeneratorConfig segmentGeneratorConfig, Integer seqId, Path hdfsAvroPath, File dataPath) {
+    protected void setSegmentNameGenerator(SegmentGeneratorConfig segmentGeneratorConfig, Integer seqId,
+        Path hdfsAvroPath, File dataPath) {
     }
 
     protected String createSegment(String dataFilePath, Schema schema, Integer seqId, Path hdfsInputFilePath,
@@ -284,13 +286,14 @@ public class HadoopSegmentCreationMapReduceJob {
       RecordReaderConfig readerConfig = null;
       switch (fileFormat) {
         case CSV:
-          if(_readerConfigFile == null) {
+          if (_readerConfigFile == null) {
             readerConfig = new CSVRecordReaderConfig();
-          }
-          else {
+          } else {
             LOGGER.info("Reading CSV Record Reader Config from: {}", _readerConfigFile);
             Path readerConfigPath = new Path(_readerConfigFile);
-            readerConfig = new ObjectMapper().readValue(_fileSystem.open(readerConfigPath), CSVRecordReaderConfig.class);
+            try (InputStream inputStream = _fileSystem.open(readerConfigPath)) {
+              readerConfig = JsonUtils.inputStreamToObject(inputStream, CSVRecordReaderConfig.class);
+            }
             LOGGER.info("CSV Record Reader Config: {}", readerConfig.toString());
           }
           break;
diff --git a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/BalanceNumSegmentAssignmentStrategyIntegrationTest.java b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/BalanceNumSegmentAssignmentStrategyIntegrationTest.java
index f9fcb0b..e2fb9f1 100644
--- a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/BalanceNumSegmentAssignmentStrategyIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/BalanceNumSegmentAssignmentStrategyIntegrationTest.java
@@ -18,13 +18,14 @@
  */
 package com.linkedin.pinot.integration.tests;
 
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.core.indexsegment.generator.SegmentVersion;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import org.apache.helix.model.IdealState;
-import org.json.JSONObject;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.DataProvider;
 import org.testng.annotations.Test;
@@ -47,7 +48,7 @@ public class BalanceNumSegmentAssignmentStrategyIntegrationTest extends UploadRe
 
     // Create eight dummy server instances
     for(int i = 0; i < 8; ++i) {
-      JSONObject serverInstance = new JSONObject();
+      ObjectNode serverInstance = JsonUtils.newObjectNode();
       serverInstance.put("host", hostName);
       serverInstance.put("port", Integer.toString(basePort + i));
       serverInstance.put("tag", serverTenant);
diff --git a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/BaseClusterIntegrationTestSet.java b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/BaseClusterIntegrationTestSet.java
index 54e9f7d..971ea7d 100644
--- a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/BaseClusterIntegrationTestSet.java
+++ b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/BaseClusterIntegrationTestSet.java
@@ -18,13 +18,15 @@
  */
 package com.linkedin.pinot.integration.tests;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.google.common.base.Function;
-import com.linkedin.pinot.common.config.CombinedConfig;
-import com.linkedin.pinot.common.config.Serializer;
 import com.linkedin.pinot.client.ResultSet;
 import com.linkedin.pinot.client.ResultSetGroup;
+import com.linkedin.pinot.common.config.CombinedConfig;
+import com.linkedin.pinot.common.config.Serializer;
 import com.linkedin.pinot.common.config.TableNameBuilder;
 import com.linkedin.pinot.common.utils.CommonConstants;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.util.TestUtils;
 import java.io.BufferedReader;
 import java.io.File;
@@ -39,10 +41,7 @@ import java.util.Random;
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 import org.apache.helix.model.InstanceConfig;
-import org.json.JSONArray;
-import org.json.JSONObject;
 import org.testng.Assert;
-import org.testng.annotations.Test;
 
 
 /**
@@ -142,7 +141,7 @@ public abstract class BaseClusterIntegrationTestSet extends BaseClusterIntegrati
   }
   /**
    * Test to ensure that broker response contains expected stats
-   * 
+   *
    * @throws Exception
    */
   public void testBrokerResponseMetadata() throws Exception {
@@ -155,7 +154,7 @@ public abstract class BaseClusterIntegrationTestSet extends BaseClusterIntegrati
         "numSegmentsMatched", "numDocsScanned", "totalDocs", "timeUsedMs", "numEntriesScannedInFilter", "numEntriesScannedPostFilter" };
 
     for (String query : pqlQueries) {
-      JSONObject response = postQuery(query);
+      JsonNode response = postQuery(query);
       for (String statName : statNames) {
         Assert.assertTrue(response.has(statName));
       }
@@ -201,13 +200,13 @@ public abstract class BaseClusterIntegrationTestSet extends BaseClusterIntegrati
           return;
         }
 
-        JSONObject query = new JSONObject(queryString);
-        String pqlQuery = query.getString("pql");
-        JSONArray hsqls = query.getJSONArray("hsqls");
+        JsonNode query = JsonUtils.stringToJsonNode(queryString);
+        String pqlQuery = query.get("pql").asText();
+        JsonNode hsqls = query.get("hsqls");
         List<String> sqlQueries = new ArrayList<>();
-        int length = hsqls.length();
+        int length = hsqls.size();
         for (int i = 0; i < length; i++) {
-          sqlQueries.add(hsqls.getString(i));
+          sqlQueries.add(hsqls.get(i).asText());
         }
         testQuery(pqlQuery, sqlQueries);
       }
@@ -255,8 +254,8 @@ public abstract class BaseClusterIntegrationTestSet extends BaseClusterIntegrati
   }
 
   private void testQueryException(String query) throws Exception {
-    JSONObject jsonObject = postQuery(query);
-    Assert.assertTrue(jsonObject.getJSONArray("exceptions").length() > 0);
+    JsonNode jsonObject = postQuery(query);
+    Assert.assertTrue(jsonObject.get("exceptions").size() > 0);
   }
 
   /**
@@ -327,17 +326,17 @@ public abstract class BaseClusterIntegrationTestSet extends BaseClusterIntegrati
       @Override
       public Boolean apply(@Nullable Void aVoid) {
         try {
-          JSONArray routingTableSnapshot =
-              getDebugInfo("debug/routingTable/" + getTableName()).getJSONArray("routingTableSnapshot");
-          int numTables = routingTableSnapshot.length();
+          JsonNode routingTableSnapshot =
+              getDebugInfo("debug/routingTable/" + getTableName()).get("routingTableSnapshot");
+          int numTables = routingTableSnapshot.size();
           for (int i = 0; i < numTables; i++) {
-            JSONObject tableRouting = routingTableSnapshot.getJSONObject(i);
-            String tableNameWithType = tableRouting.getString("tableName");
+            JsonNode tableRouting = routingTableSnapshot.get(i);
+            String tableNameWithType = tableRouting.get("tableName").asText();
             if (TableNameBuilder.extractRawTableName(tableNameWithType).equals(getTableName())) {
-              JSONArray routingTableEntries = tableRouting.getJSONArray("routingTableEntries");
-              int numRoutingTableEntries = routingTableEntries.length();
+              JsonNode routingTableEntries = tableRouting.get("routingTableEntries");
+              int numRoutingTableEntries = routingTableEntries.size();
               for (int j = 0; j < numRoutingTableEntries; j++) {
-                JSONObject routingTableEntry = routingTableEntries.getJSONObject(j);
+                JsonNode routingTableEntry = routingTableEntries.get(j);
                 if (routingTableEntry.has(instanceName)) {
                   return shouldExist;
                 }
@@ -364,18 +363,18 @@ public abstract class BaseClusterIntegrationTestSet extends BaseClusterIntegrati
       @Override
       public Boolean apply(@Nullable Void aVoid) {
         try {
-          JSONArray routingTableSnapshot =
-              getDebugInfo("debug/routingTable/" + getTableName()).getJSONArray("routingTableSnapshot");
-          int numTables = routingTableSnapshot.length();
+          JsonNode routingTableSnapshot =
+              getDebugInfo("debug/routingTable/" + getTableName()).get("routingTableSnapshot");
+          int numTables = routingTableSnapshot.size();
           for (int i = 0; i < numTables; i++) {
-            JSONObject tableRouting = routingTableSnapshot.getJSONObject(i);
-            String tableNameWithType = tableRouting.getString("tableName");
+            JsonNode tableRouting = routingTableSnapshot.get(i);
+            String tableNameWithType = tableRouting.get("tableName").asText();
             if (TableNameBuilder.extractRawTableName(tableNameWithType).equals(getTableName())) {
-              JSONArray routingTableEntries = tableRouting.getJSONArray("routingTableEntries");
-              int numRoutingTableEntries = routingTableEntries.length();
+              JsonNode routingTableEntries = tableRouting.get("routingTableEntries");
+              int numRoutingTableEntries = routingTableEntries.size();
               for (int j = 0; j < numRoutingTableEntries; j++) {
-                JSONObject routingTableEntry = routingTableEntries.getJSONObject(j);
-                if (routingTableEntry.length() == 0) {
+                JsonNode routingTableEntry = routingTableEntries.get(j);
+                if (routingTableEntry.size() == 0) {
                   if (!shouldBeEmpty) {
                     return false;
                   }
diff --git a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/ClusterIntegrationTestUtils.java b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/ClusterIntegrationTestUtils.java
index 223e6c8..c5f2042 100644
--- a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/ClusterIntegrationTestUtils.java
+++ b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/ClusterIntegrationTestUtils.java
@@ -18,9 +18,11 @@
  */
 package com.linkedin.pinot.integration.tests;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.google.common.math.DoubleMath;
 import com.google.common.primitives.Longs;
 import com.linkedin.pinot.client.ResultSetGroup;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.StringUtil;
 import com.linkedin.pinot.common.utils.TarGzCompressionUtils;
 import com.linkedin.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
@@ -60,8 +62,6 @@ import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.io.BinaryEncoder;
 import org.apache.avro.io.EncoderFactory;
 import org.apache.avro.util.Utf8;
-import org.json.JSONArray;
-import org.json.JSONObject;
 import org.testng.Assert;
 
 
@@ -504,7 +504,7 @@ public class ClusterIntegrationTestUtils {
       @Nonnull com.linkedin.pinot.client.Connection pinotConnection, @Nullable List<String> sqlQueries,
       @Nullable Connection h2Connection) throws Exception {
     // Use broker response for metadata check, connection response for value check
-    JSONObject pinotResponse = ClusterTest.postQuery(pqlQuery, brokerUrl);
+    JsonNode pinotResponse = ClusterTest.postQuery(pqlQuery, brokerUrl);
     ResultSetGroup pinotResultSetGroup = pinotConnection.execute(pqlQuery);
 
     // Skip comparison if SQL queries are not specified
@@ -515,7 +515,7 @@ public class ClusterIntegrationTestUtils {
     Assert.assertNotNull(h2Connection);
     Statement h2statement = h2Connection.createStatement(ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY);
 
-    int pinotNumRecordsSelected = pinotResponse.getInt("numDocsScanned");
+    long pinotNumRecordsSelected = pinotResponse.get("numDocsScanned").asLong();
 
     // Aggregation results
     if (pinotResponse.has("aggregationResults")) {
@@ -530,7 +530,7 @@ public class ClusterIntegrationTestUtils {
       }
 
       // Get aggregation type
-      JSONObject pinotFirstAggregationResult = pinotResponse.getJSONArray("aggregationResults").getJSONObject(0);
+      JsonNode pinotFirstAggregationResult = pinotResponse.get("aggregationResults").get(0);
 
       // Aggregation-only results
       if (pinotFirstAggregationResult.has("value")) {
@@ -767,11 +767,11 @@ public class ClusterIntegrationTestUtils {
             // TODO: Find a better way to identify multi-value column
             if (columnResult.charAt(0) == '[') {
               // Multi-value column
-              JSONArray columnValues = new JSONArray(columnResult);
+              JsonNode columnValues = JsonUtils.stringToJsonNode(columnResult);
               List<String> multiValue = new ArrayList<>();
-              int length = columnValues.length();
+              int length = columnValues.size();
               for (int elementIndex = 0; elementIndex < length; elementIndex++) {
-                multiValue.add(columnValues.getString(elementIndex));
+                multiValue.add(columnValues.get(elementIndex).asText());
               }
               for (int elementIndex = length; elementIndex < MAX_NUM_ELEMENTS_IN_MULTI_VALUE_TO_COMPARE;
                   elementIndex++) {
diff --git a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/ClusterTest.java b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/ClusterTest.java
index 88a0026..1ca8e1a 100644
--- a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/ClusterTest.java
+++ b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/ClusterTest.java
@@ -18,6 +18,8 @@
  */
 package com.linkedin.pinot.integration.tests;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.linkedin.pinot.broker.broker.BrokerServerBuilder;
 import com.linkedin.pinot.broker.broker.BrokerTestUtils;
 import com.linkedin.pinot.broker.broker.helix.HelixBrokerStarter;
@@ -30,6 +32,7 @@ import com.linkedin.pinot.common.utils.CommonConstants.Helix;
 import com.linkedin.pinot.common.utils.CommonConstants.Minion;
 import com.linkedin.pinot.common.utils.CommonConstants.Server;
 import com.linkedin.pinot.common.utils.FileUploadDownloadClient;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.ZkStarter;
 import com.linkedin.pinot.controller.helix.ControllerRequestBuilderUtil;
 import com.linkedin.pinot.controller.helix.ControllerTest;
@@ -69,7 +72,6 @@ import org.apache.commons.configuration.Configuration;
 import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.commons.io.FileUtils;
 import org.apache.http.HttpStatus;
-import org.json.JSONObject;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.testng.Assert;
@@ -463,34 +465,34 @@ public abstract class ClusterTest extends ControllerTest {
   }
 
   protected void createBrokerTenant(String tenantName, int brokerCount) throws Exception {
-    JSONObject request = ControllerRequestBuilderUtil.buildBrokerTenantCreateRequestJSON(tenantName, brokerCount);
-    sendPostRequest(_controllerRequestURLBuilder.forBrokerTenantCreate(), request.toString());
+    String request = ControllerRequestBuilderUtil.buildBrokerTenantCreateRequestJSON(tenantName, brokerCount);
+    sendPostRequest(_controllerRequestURLBuilder.forBrokerTenantCreate(), request);
   }
 
   protected void createServerTenant(String tenantName, int offlineServerCount, int realtimeServerCount)
       throws Exception {
-    JSONObject request = ControllerRequestBuilderUtil.buildServerTenantCreateRequestJSON(tenantName,
+    String request = ControllerRequestBuilderUtil.buildServerTenantCreateRequestJSON(tenantName,
         offlineServerCount + realtimeServerCount, offlineServerCount, realtimeServerCount);
-    sendPostRequest(_controllerRequestURLBuilder.forServerTenantCreate(), request.toString());
+    sendPostRequest(_controllerRequestURLBuilder.forServerTenantCreate(), request);
   }
 
-  protected JSONObject getDebugInfo(final String uri) throws Exception {
-    return new JSONObject(sendGetRequest(_brokerBaseApiUrl + "/" + uri));
+  protected JsonNode getDebugInfo(final String uri) throws Exception {
+    return JsonUtils.stringToJsonNode(sendGetRequest(_brokerBaseApiUrl + "/" + uri));
   }
 
-  protected JSONObject postQuery(String query) throws Exception {
+  protected JsonNode postQuery(String query) throws Exception {
     return postQuery(query, _brokerBaseApiUrl);
   }
 
-  public static JSONObject postQuery(String query, String brokerBaseApiUrl) throws Exception {
+  public static JsonNode postQuery(String query, String brokerBaseApiUrl) throws Exception {
     return postQuery(query, brokerBaseApiUrl, false);
   }
 
-  public static JSONObject postQuery(String query, String brokerBaseApiUrl, boolean enableTrace) throws Exception {
-    JSONObject payload = new JSONObject();
+  public static JsonNode postQuery(String query, String brokerBaseApiUrl, boolean enableTrace) throws Exception {
+    ObjectNode payload = JsonUtils.newObjectNode();
     payload.put("pql", query);
     payload.put("trace", enableTrace);
 
-    return new JSONObject(sendPostRequest(brokerBaseApiUrl + "/query", payload.toString()));
+    return JsonUtils.stringToJsonNode(sendPostRequest(brokerBaseApiUrl + "/query", payload.toString()));
   }
 }
diff --git a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/DeleteAPIHybridClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/DeleteAPIHybridClusterIntegrationTest.java
index 1523ef8..ffae4fa 100644
--- a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/DeleteAPIHybridClusterIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/DeleteAPIHybridClusterIntegrationTest.java
@@ -18,14 +18,13 @@
  */
 package com.linkedin.pinot.integration.tests;
 
-import com.alibaba.fastjson.JSON;
-import com.alibaba.fastjson.JSONArray;
-import com.alibaba.fastjson.JSONObject;
+import com.fasterxml.jackson.databind.JsonNode;
 import com.linkedin.pinot.common.metadata.ZKMetadataProvider;
 import com.linkedin.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
 import com.linkedin.pinot.common.metadata.segment.RealtimeSegmentZKMetadata;
 import com.linkedin.pinot.common.utils.CommonConstants;
-import java.util.Collections;
+import com.linkedin.pinot.common.utils.JsonUtils;
+import java.util.Iterator;
 import junit.framework.Assert;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
@@ -50,13 +49,12 @@ public class DeleteAPIHybridClusterIntegrationTest extends HybridClusterIntegrat
   }
 
   private long numRowsReturned(CommonConstants.Helix.TableType tableType) throws Exception {
-    org.json.JSONObject response = postQuery("select count(*) from '" + TABLE_NAME + "_" + tableType + "'");
-    if (response.get("numDocsScanned").equals(new Integer(0))) {
+    JsonNode response = postQuery("select count(*) from '" + TABLE_NAME + "_" + tableType + "'");
+    if (response.get("numDocsScanned").asLong() == 0) {
       return 0;
     } else {
       // Throws a null pointer exception when there are no rows because it can't find "aggregationResults"
-      String pinotValue =
-          ((org.json.JSONArray) response.get("aggregationResults")).getJSONObject(0).get("value").toString();
+      String pinotValue = response.get("aggregationResults").get(0).get("value").asText();
       return Long.parseLong(pinotValue);
     }
   }
@@ -106,10 +104,10 @@ public class DeleteAPIHybridClusterIntegrationTest extends HybridClusterIntegrat
 
     String segmentList = sendGetRequest(_controllerRequestURLBuilder.
         forSegmentListAPIWithTableType(TABLE_NAME, CommonConstants.Helix.TableType.REALTIME.toString()));
-    JSONArray realtimeSegmentsList =
+    JsonNode realtimeSegmentsList =
         getSegmentsFromJsonSegmentAPI(segmentList, CommonConstants.Helix.TableType.REALTIME.toString());
 
-    String removedSegment = realtimeSegmentsList.get(0).toString();
+    String removedSegment = realtimeSegmentsList.get(0).asText();
     long removedSegmentRows = getNumRowsFromRealtimeMetadata(removedSegment);
     Assert.assertNotSame(removedSegmentRows, 0L);
 
@@ -121,9 +119,9 @@ public class DeleteAPIHybridClusterIntegrationTest extends HybridClusterIntegrat
     String postDeleteSegmentList = sendGetRequest(
         _controllerRequestURLBuilder.forSegmentListAPIWithTableType(TABLE_NAME,
             CommonConstants.Helix.TableType.REALTIME.toString()));
-    JSONArray realtimeSegmentsListReturn =
+    JsonNode realtimeSegmentsListReturn =
         getSegmentsFromJsonSegmentAPI(postDeleteSegmentList, CommonConstants.Helix.TableType.REALTIME.toString());
-    realtimeSegmentsList.remove(removedSegment);
+    removeValue(realtimeSegmentsList, removedSegment);
     Assert.assertEquals(realtimeSegmentsListReturn, realtimeSegmentsList);
   }
 
@@ -133,10 +131,10 @@ public class DeleteAPIHybridClusterIntegrationTest extends HybridClusterIntegrat
 
     String segmentList = sendGetRequest(_controllerRequestURLBuilder.forSegmentListAPIWithTableType(TABLE_NAME,
         CommonConstants.Helix.TableType.REALTIME.toString()));
-    JSONArray realtimeSegmentsList =
+    JsonNode realtimeSegmentsList =
         getSegmentsFromJsonSegmentAPI(segmentList, CommonConstants.Helix.TableType.REALTIME.toString());
 
-    String removedSegment = realtimeSegmentsList.get(0).toString();
+    String removedSegment = realtimeSegmentsList.get(0).asText();
     long removedSegmentRows = getNumRowsFromRealtimeMetadata(removedSegment);
     Assert.assertNotSame(removedSegmentRows, 0L);
 
@@ -148,9 +146,9 @@ public class DeleteAPIHybridClusterIntegrationTest extends HybridClusterIntegrat
     String postDeleteSegmentList = sendGetRequest(
         _controllerRequestURLBuilder.forSegmentListAPIWithTableType(TABLE_NAME,
             CommonConstants.Helix.TableType.REALTIME.toString()));
-    JSONArray realtimeSegmentsListReturn =
+    JsonNode realtimeSegmentsListReturn =
         getSegmentsFromJsonSegmentAPI(postDeleteSegmentList, CommonConstants.Helix.TableType.REALTIME.toString());
-    realtimeSegmentsList.remove(removedSegment);
+    removeValue(realtimeSegmentsList, removedSegment);
     Assert.assertEquals(realtimeSegmentsListReturn, realtimeSegmentsList);
   }
 
@@ -166,11 +164,11 @@ public class DeleteAPIHybridClusterIntegrationTest extends HybridClusterIntegrat
   public void deleteFromDeleteAPI() throws Exception {
     String segmentList = sendGetRequest(_controllerRequestURLBuilder.
         forSegmentListAPIWithTableType(TABLE_NAME, CommonConstants.Helix.TableType.OFFLINE.toString()));
-    JSONArray offlineSegmentsList =
+    JsonNode offlineSegmentsList =
         getSegmentsFromJsonSegmentAPI(segmentList, CommonConstants.Helix.TableType.OFFLINE.toString());
     Assert.assertNotNull(offlineSegmentsList);
 
-    String removedSegment = offlineSegmentsList.get(0).toString();
+    String removedSegment = offlineSegmentsList.get(0).asText();
     long removedSegmentRows = getNumRowsFromOfflineMetadata(removedSegment);
     Assert.assertNotSame(removedSegmentRows, 0L);
 
@@ -181,9 +179,9 @@ public class DeleteAPIHybridClusterIntegrationTest extends HybridClusterIntegrat
 
     String postDeleteSegmentList = sendGetRequest(_controllerRequestURLBuilder.
         forSegmentListAPIWithTableType(TABLE_NAME, CommonConstants.Helix.TableType.OFFLINE.toString()));
-    JSONArray offlineSegmentsListReturn =
+    JsonNode offlineSegmentsListReturn =
         getSegmentsFromJsonSegmentAPI(postDeleteSegmentList, CommonConstants.Helix.TableType.OFFLINE.toString());
-    offlineSegmentsList.remove(removedSegment);
+    removeValue(offlineSegmentsList, removedSegment);
     Assert.assertEquals(offlineSegmentsListReturn, offlineSegmentsList);
 
     // Testing Delete All API here
@@ -195,9 +193,8 @@ public class DeleteAPIHybridClusterIntegrationTest extends HybridClusterIntegrat
     String postDeleteSegmentListAll = sendGetRequest(_controllerRequestURLBuilder.
         forSegmentListAPIWithTableType(TABLE_NAME, CommonConstants.Helix.TableType.OFFLINE.toString()));
 
-    Assert.assertEquals(
-        getSegmentsFromJsonSegmentAPI(postDeleteSegmentListAll, CommonConstants.Helix.TableType.OFFLINE.toString()),
-        Collections.emptyList());
+    Assert.assertEquals(getSegmentsFromJsonSegmentAPI(postDeleteSegmentListAll,
+        CommonConstants.Helix.TableType.OFFLINE.toString()).size(), 0);
 
     waitForSegmentsToBeInDeleteDirectory();
     repushOfflineSegments();
@@ -214,10 +211,10 @@ public class DeleteAPIHybridClusterIntegrationTest extends HybridClusterIntegrat
     String segmentList = sendGetRequest(_controllerRequestURLBuilder.
         forSegmentListAPIWithTableType(TABLE_NAME, CommonConstants.Helix.TableType.OFFLINE.toString()));
 
-    JSONArray offlineSegmentsList =
+    JsonNode offlineSegmentsList =
         getSegmentsFromJsonSegmentAPI(segmentList, CommonConstants.Helix.TableType.OFFLINE.toString());
 
-    String removedSegment = offlineSegmentsList.get(0).toString();
+    String removedSegment = offlineSegmentsList.get(0).asText();
 
     long removedSegmentRows = getNumRowsFromOfflineMetadata(removedSegment);
     Assert.assertNotSame(removedSegmentRows, 0L);
@@ -229,9 +226,9 @@ public class DeleteAPIHybridClusterIntegrationTest extends HybridClusterIntegrat
 
     String postDeleteSegmentList = sendGetRequest(_controllerRequestURLBuilder.
         forSegmentListAPIWithTableType(TABLE_NAME, CommonConstants.Helix.TableType.OFFLINE.toString()));
-    JSONArray offlineSegmentsListReturn =
+    JsonNode offlineSegmentsListReturn =
         getSegmentsFromJsonSegmentAPI(postDeleteSegmentList, CommonConstants.Helix.TableType.OFFLINE.toString());
-    offlineSegmentsList.remove(removedSegment);
+    removeValue(offlineSegmentsList, removedSegment);
     Assert.assertEquals(offlineSegmentsListReturn, offlineSegmentsList);
 
     // Testing Delete All API here
@@ -243,9 +240,8 @@ public class DeleteAPIHybridClusterIntegrationTest extends HybridClusterIntegrat
     String postDeleteSegmentListAll = sendGetRequest(_controllerRequestURLBuilder.
         forSegmentListAPIWithTableType(TABLE_NAME, CommonConstants.Helix.TableType.OFFLINE.toString()));
 
-    Assert.assertEquals(
-        getSegmentsFromJsonSegmentAPI(postDeleteSegmentListAll, CommonConstants.Helix.TableType.OFFLINE.toString()),
-        Collections.emptyList());
+    Assert.assertEquals(getSegmentsFromJsonSegmentAPI(postDeleteSegmentListAll,
+        CommonConstants.Helix.TableType.OFFLINE.toString()).size(), 0);
 
     waitForSegmentsToBeInDeleteDirectory();
     repushOfflineSegments();
@@ -263,13 +259,22 @@ public class DeleteAPIHybridClusterIntegrationTest extends HybridClusterIntegrat
     return segmentZKMetadata.getTotalRawDocs();
   }
 
-  private com.alibaba.fastjson.JSONArray getSegmentsFromJsonSegmentAPI(String json, String type) throws Exception {
-    JSONObject tableTypeAndSegments = (JSONObject) JSON.parseArray(json).get(0);
-    return (JSONArray) tableTypeAndSegments.get(type);
+  private JsonNode getSegmentsFromJsonSegmentAPI(String json, String type) throws Exception {
+    return JsonUtils.stringToJsonNode(json).get(0).get(type);
   }
 
   private void repushOfflineSegments() throws Exception {
     uploadSegments(_tarDir);
     waitForNumRows(nOfflineRows, CommonConstants.Helix.TableType.OFFLINE);
   }
+
+  private static void removeValue(JsonNode jsonArray, String value) {
+    Iterator<JsonNode> elements = jsonArray.elements();
+    while (elements.hasNext()) {
+      if (elements.next().asText().equals(value)) {
+        elements.remove();
+        return;
+      }
+    }
+  }
 }
diff --git a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/HybridClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/HybridClusterIntegrationTest.java
index 2c7326d..5a91db5 100644
--- a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/HybridClusterIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/HybridClusterIntegrationTest.java
@@ -18,10 +18,12 @@
  */
 package com.linkedin.pinot.integration.tests;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.google.common.base.Function;
 import com.linkedin.pinot.common.config.TableNameBuilder;
 import com.linkedin.pinot.common.data.Schema;
 import com.linkedin.pinot.common.utils.CommonConstants;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.KafkaStarterUtils;
 import com.linkedin.pinot.controller.ControllerConf;
 import com.linkedin.pinot.util.TestUtils;
@@ -34,8 +36,6 @@ import java.util.concurrent.TimeUnit;
 import javax.annotation.Nullable;
 import kafka.server.KafkaServerStartable;
 import org.apache.commons.io.FileUtils;
-import org.json.JSONArray;
-import org.json.JSONObject;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
@@ -180,40 +180,40 @@ public class HybridClusterIntegrationTest extends BaseClusterIntegrationTestSet
     {
       String jsonOutputStr = sendGetRequest(_controllerRequestURLBuilder.
           forSegmentListAPIWithTableType(getTableName(), CommonConstants.Helix.TableType.OFFLINE.toString()));
-      JSONArray array = new JSONArray(jsonOutputStr);
+      JsonNode array = JsonUtils.stringToJsonNode(jsonOutputStr);
       // There should be one element in the array
-      JSONObject element = (JSONObject) array.get(0);
-      JSONArray segments = (JSONArray) element.get("OFFLINE");
-      Assert.assertEquals(segments.length(), 8);
+      JsonNode element = array.get(0);
+      JsonNode segments = element.get("OFFLINE");
+      Assert.assertEquals(segments.size(), 8);
     }
     {
       String jsonOutputStr = sendGetRequest(_controllerRequestURLBuilder.
           forSegmentListAPIWithTableType(getTableName(), CommonConstants.Helix.TableType.REALTIME.toString()));
-      JSONArray array = new JSONArray(jsonOutputStr);
+      JsonNode array = JsonUtils.stringToJsonNode(jsonOutputStr);
       // There should be one element in the array
-      JSONObject element = (JSONObject) array.get(0);
-      JSONArray segments = (JSONArray) element.get("REALTIME");
-      Assert.assertEquals(segments.length(), 3);
+      JsonNode element = array.get(0);
+      JsonNode segments = element.get("REALTIME");
+      Assert.assertEquals(segments.size(), 3);
     }
     {
       String jsonOutputStr = sendGetRequest(_controllerRequestURLBuilder. forSegmentListAPI(getTableName()));
-      JSONArray array = new JSONArray(jsonOutputStr);
+      JsonNode array = JsonUtils.stringToJsonNode(jsonOutputStr);
       // there should be 2 elements in the array now.
       int realtimeIndex = 0;
       int offlineIndex = 1;
-      JSONObject element = (JSONObject) array.get(realtimeIndex);
+      JsonNode element = array.get(realtimeIndex);
       if (!element.has("REALTIME")) {
         realtimeIndex = 1;
         offlineIndex = 0;
       }
-      JSONObject offlineElement = (JSONObject)array.get(offlineIndex);
-      JSONObject realtimeElement = (JSONObject)array.get(realtimeIndex);
+      JsonNode offlineElement = array.get(offlineIndex);
+      JsonNode realtimeElement = array.get(realtimeIndex);
 
-      JSONArray realtimeSegments = (JSONArray) realtimeElement.get("REALTIME");
-      Assert.assertEquals(realtimeSegments.length(), 3);
+      JsonNode realtimeSegments = realtimeElement.get("REALTIME");
+      Assert.assertEquals(realtimeSegments.size(), 3);
 
-      JSONArray offlineSegments = (JSONArray) offlineElement.get("OFFLINE");
-      Assert.assertEquals(offlineSegments.length(), 8);
+      JsonNode offlineSegments = offlineElement.get("OFFLINE");
+      Assert.assertEquals(offlineSegments.size(), 8);
     }
   }
 
@@ -277,9 +277,8 @@ public class HybridClusterIntegrationTest extends BaseClusterIntegrationTestSet
       @Override
       public Boolean apply(@Nullable Void aVoid) {
         try {
-          JSONArray routingTableSnapshot =
-              getDebugInfo("debug/routingTable/" + tableName).getJSONArray("routingTableSnapshot");
-          return routingTableSnapshot.length() == 0;
+          JsonNode routingTableSnapshot = getDebugInfo("debug/routingTable/" + tableName).get("routingTableSnapshot");
+          return routingTableSnapshot.size() == 0;
         } catch (Exception e) {
           return null;
         }
diff --git a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/HybridClusterIntegrationTestCommandLineRunner.java b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/HybridClusterIntegrationTestCommandLineRunner.java
index 6e8addb..63c1a7c 100644
--- a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/HybridClusterIntegrationTestCommandLineRunner.java
+++ b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/HybridClusterIntegrationTestCommandLineRunner.java
@@ -18,8 +18,10 @@
  */
 package com.linkedin.pinot.integration.tests;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.google.common.base.Preconditions;
 import com.linkedin.pinot.common.data.Schema;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.KafkaStarterUtils;
 import com.linkedin.pinot.controller.ControllerConf;
 import com.linkedin.pinot.tools.query.comparison.QueryComparison;
@@ -40,7 +42,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import javax.annotation.Nonnull;
 import kafka.server.KafkaServerStartable;
 import org.apache.commons.io.FileUtils;
-import org.json.JSONObject;
 import org.testng.Assert;
 import org.testng.ITestResult;
 import org.testng.TestListenerAdapter;
@@ -240,9 +241,7 @@ public class HybridClusterIntegrationTestCommandLineRunner {
 
     @Override
     protected long getCurrentCountStarResult() throws Exception {
-      return postQuery("SELECT COUNT(*) FROM " + getTableName()).getJSONArray("aggregationResults")
-          .getJSONObject(0)
-          .getLong("value");
+      return postQuery("SELECT COUNT(*) FROM " + getTableName()).get("aggregationResults").get(0).get("value").asLong();
     }
 
     @BeforeClass
@@ -317,7 +316,7 @@ public class HybridClusterIntegrationTestCommandLineRunner {
           try (BufferedReader queryFileReader = new BufferedReader(new FileReader(_queryFile))) {
             // Set the expected COUNT(*) result and wait for all documents loaded
             responseFileReader.mark(4096);
-            _countStarResult = new JSONObject(responseFileReader.readLine()).getLong("totalDocs");
+            _countStarResult = JsonUtils.stringToJsonNode(responseFileReader.readLine()).get("totalDocs").asLong();
             responseFileReader.reset();
             waitForAllDocsLoaded(600_000L);
 
@@ -327,20 +326,20 @@ public class HybridClusterIntegrationTestCommandLineRunner {
                     new ThreadPoolExecutor.CallerRunsPolicy());
             String query;
             while ((query = queryFileReader.readLine()) != null) {
-              final String currentQuery = query;
-              final JSONObject expectedResponse = new JSONObject(responseFileReader.readLine());
+              String currentQuery = query;
+              JsonNode expectedResponse = JsonUtils.stringToJsonNode(responseFileReader.readLine());
               executorService.execute(new Runnable() {
                 @Override
                 public void run() {
                   try {
-                    JSONObject actualResponse = postQuery(currentQuery, "http://localhost:" + BROKER_BASE_PORT);
+                    JsonNode actualResponse = postQuery(currentQuery, "http://localhost:" + BROKER_BASE_PORT);
                     if (QueryComparison.compareWithEmpty(actualResponse, expectedResponse)
                         == QueryComparison.ComparisonStatus.FAILED) {
                       numFailedQueries.getAndIncrement();
                       System.out.println(
                           "Query comparison failed for query: " + currentQuery
-                              + "\nActual: " + actualResponse.toString(2)
-                              + "\nExpected: " + expectedResponse.toString(2));
+                              + "\nActual: " + actualResponse.toString()
+                              + "\nExpected: " + expectedResponse.toString());
                     }
                   } catch (Exception e) {
                     numFailedQueries.getAndIncrement();
diff --git a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java
index 358d2cb..d9c487a 100644
--- a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/LLCRealtimeClusterIntegrationTest.java
@@ -18,10 +18,10 @@
  */
 package com.linkedin.pinot.integration.tests;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.google.common.base.Function;
 import com.linkedin.pinot.common.config.TableNameBuilder;
 import com.linkedin.pinot.common.utils.CommonConstants;
-import com.linkedin.pinot.core.indexsegment.generator.SegmentVersion;
 import com.linkedin.pinot.util.TestUtils;
 import java.io.File;
 import java.util.Arrays;
@@ -31,7 +31,6 @@ import org.apache.avro.reflect.Nullable;
 import org.apache.commons.configuration.Configuration;
 import org.apache.commons.io.FileUtils;
 import org.apache.helix.ZNRecord;
-import org.json.JSONObject;
 import org.testng.Assert;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
@@ -113,10 +112,10 @@ public class LLCRealtimeClusterIntegrationTest extends RealtimeClusterIntegratio
 
     final long numTotalDocs = getCountStarResult();
 
-    JSONObject queryResponse = postQuery(TEST_UPDATED_INVERTED_INDEX_QUERY);
-    Assert.assertEquals(queryResponse.getLong("totalDocs"), numTotalDocs);
+    JsonNode queryResponse = postQuery(TEST_UPDATED_INVERTED_INDEX_QUERY);
+    Assert.assertEquals(queryResponse.get("totalDocs").asLong(), numTotalDocs);
     // TODO: investigate why assert for a specific value fails intermittently
-    Assert.assertNotSame(queryResponse.getLong("numEntriesScannedInFilter"), 0);
+    Assert.assertNotSame(queryResponse.get("numEntriesScannedInFilter").asLong(), 0);
 
     updateRealtimeTableConfig(getTableName(), UPDATED_INVERTED_INDEX_COLUMNS, null);
 
@@ -126,10 +125,10 @@ public class LLCRealtimeClusterIntegrationTest extends RealtimeClusterIntegratio
       @Override
       public Boolean apply(@javax.annotation.Nullable Void aVoid) {
         try {
-          JSONObject queryResponse = postQuery(TEST_UPDATED_INVERTED_INDEX_QUERY);
+          JsonNode queryResponse = postQuery(TEST_UPDATED_INVERTED_INDEX_QUERY);
           // Total docs should not change during reload
-          Assert.assertEquals(queryResponse.getLong("totalDocs"), numTotalDocs);
-          return queryResponse.getLong("numEntriesScannedInFilter") == 0;
+          Assert.assertEquals(queryResponse.get("totalDocs").asLong(), numTotalDocs);
+          return queryResponse.get("numEntriesScannedInFilter").asLong() == 0;
         } catch (Exception e) {
           throw new RuntimeException(e);
         }
diff --git a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/MetadataAndDictionaryAggregationPlanClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/MetadataAndDictionaryAggregationPlanClusterIntegrationTest.java
index 04cf178..0c4bdaa 100644
--- a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/MetadataAndDictionaryAggregationPlanClusterIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/MetadataAndDictionaryAggregationPlanClusterIntegrationTest.java
@@ -18,6 +18,7 @@
  */
 package com.linkedin.pinot.integration.tests;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.google.common.collect.Lists;
 import com.linkedin.pinot.common.data.Schema;
 import com.linkedin.pinot.util.TestUtils;
@@ -29,12 +30,12 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import javax.annotation.Nonnull;
 import org.apache.commons.io.FileUtils;
-import org.json.JSONObject;
-import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
+import static org.testng.Assert.*;
+
 
 /**
  * /**
@@ -82,7 +83,6 @@ public class MetadataAndDictionaryAggregationPlanClusterIntegrationTest extends
     return _currentTable;
   }
 
-
   @BeforeClass
   public void setUp() throws Exception {
     TestUtils.ensureDirectoriesExistAndEmpty(_tempDir);
@@ -155,31 +155,31 @@ public class MetadataAndDictionaryAggregationPlanClusterIntegrationTest extends
     testQuery(pqlQuery, Collections.singletonList(sqlQuery));
     testQuery(pqlStarTreeQuery, Collections.singletonList(sqlQuery));
     pqlQuery = "SELECT MIN(ArrTime) FROM " + DEFAULT_TABLE_NAME;
-    pqlStarTreeQuery =  "SELECT MIN(ArrTime) FROM " + STAR_TREE_TABLE_NAME;
+    pqlStarTreeQuery = "SELECT MIN(ArrTime) FROM " + STAR_TREE_TABLE_NAME;
     sqlQuery = "SELECT MIN(ArrTime) FROM " + DEFAULT_TABLE_NAME;
     testQuery(pqlQuery, Collections.singletonList(sqlQuery));
     testQuery(pqlStarTreeQuery, Collections.singletonList(sqlQuery));
     pqlQuery = "SELECT MINMAXRANGE(ArrTime) FROM " + DEFAULT_TABLE_NAME;
-    pqlStarTreeQuery =  "SELECT MINMAXRANGE(ArrTime) FROM "+ STAR_TREE_TABLE_NAME;
+    pqlStarTreeQuery = "SELECT MINMAXRANGE(ArrTime) FROM " + STAR_TREE_TABLE_NAME;
     sqlQuery = "SELECT MAX(ArrTime)-MIN(ArrTime) FROM " + DEFAULT_TABLE_NAME;
     testQuery(pqlQuery, Collections.singletonList(sqlQuery));
     testQuery(pqlStarTreeQuery, Collections.singletonList(sqlQuery));
     pqlQuery = "SELECT MIN(ArrTime), MAX(ArrTime), MINMAXRANGE(ArrTime) FROM " + DEFAULT_TABLE_NAME;
-    pqlStarTreeQuery =  "SELECT MIN(ArrTime), MAX(ArrTime), MINMAXRANGE(ArrTime) FROM "+ STAR_TREE_TABLE_NAME;
+    pqlStarTreeQuery = "SELECT MIN(ArrTime), MAX(ArrTime), MINMAXRANGE(ArrTime) FROM " + STAR_TREE_TABLE_NAME;
     sqlQuery1 = "SELECT MIN(ArrTime) FROM " + DEFAULT_TABLE_NAME;
     sqlQuery2 = "SELECT MAX(ArrTime) FROM " + DEFAULT_TABLE_NAME;
     sqlQuery3 = "SELECT MAX(ArrTime)-MIN(ArrTime) FROM " + DEFAULT_TABLE_NAME;
     testQuery(pqlQuery, Lists.newArrayList(sqlQuery1, sqlQuery2, sqlQuery3));
     testQuery(pqlStarTreeQuery, Lists.newArrayList(sqlQuery1, sqlQuery2, sqlQuery3));
     pqlQuery = "SELECT MIN(ArrTime), COUNT(*) FROM " + DEFAULT_TABLE_NAME;
-    pqlStarTreeQuery =  "SELECT MIN(ArrTime), COUNT(*) FROM "+ STAR_TREE_TABLE_NAME;
+    pqlStarTreeQuery = "SELECT MIN(ArrTime), COUNT(*) FROM " + STAR_TREE_TABLE_NAME;
     sqlQuery1 = "SELECT MIN(ArrTime) FROM " + DEFAULT_TABLE_NAME;
     sqlQuery2 = "SELECT COUNT(*) FROM " + DEFAULT_TABLE_NAME;
     testQuery(pqlQuery, Lists.newArrayList(sqlQuery1, sqlQuery2));
     testQuery(pqlStarTreeQuery, Lists.newArrayList(sqlQuery1, sqlQuery2));
     // float
     pqlQuery = "SELECT MAX(DepDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
-    pqlStarTreeQuery =  "SELECT MAX(DepDelayMinutes) FROM "+ STAR_TREE_TABLE_NAME;
+    pqlStarTreeQuery = "SELECT MAX(DepDelayMinutes) FROM " + STAR_TREE_TABLE_NAME;
     sqlQuery = "SELECT MAX(DepDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
     testQuery(pqlQuery, Collections.singletonList(sqlQuery));
     testQuery(pqlStarTreeQuery, Collections.singletonList(sqlQuery));
@@ -189,19 +189,21 @@ public class MetadataAndDictionaryAggregationPlanClusterIntegrationTest extends
     testQuery(pqlQuery, Collections.singletonList(sqlQuery));
     testQuery(pqlStarTreeQuery, Collections.singletonList(sqlQuery));
     pqlQuery = "SELECT MINMAXRANGE(DepDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
-    pqlStarTreeQuery =  "SELECT MINMAXRANGE(DepDelayMinutes) FROM "+ STAR_TREE_TABLE_NAME;
+    pqlStarTreeQuery = "SELECT MINMAXRANGE(DepDelayMinutes) FROM " + STAR_TREE_TABLE_NAME;
     sqlQuery = "SELECT MAX(DepDelayMinutes)-MIN(DepDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
     testQuery(pqlQuery, Collections.singletonList(sqlQuery));
     testQuery(pqlStarTreeQuery, Collections.singletonList(sqlQuery));
-    pqlQuery = "SELECT MIN(DepDelayMinutes), MAX(DepDelayMinutes), MINMAXRANGE(DepDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
-    pqlStarTreeQuery =  "SELECT MIN(DepDelayMinutes), MAX(DepDelayMinutes), MINMAXRANGE(DepDelayMinutes) FROM "+ STAR_TREE_TABLE_NAME;
+    pqlQuery =
+        "SELECT MIN(DepDelayMinutes), MAX(DepDelayMinutes), MINMAXRANGE(DepDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
+    pqlStarTreeQuery =
+        "SELECT MIN(DepDelayMinutes), MAX(DepDelayMinutes), MINMAXRANGE(DepDelayMinutes) FROM " + STAR_TREE_TABLE_NAME;
     sqlQuery1 = "SELECT MIN(DepDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
     sqlQuery2 = "SELECT MAX(DepDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
     sqlQuery3 = "SELECT MAX(DepDelayMinutes)-MIN(DepDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
     testQuery(pqlQuery, Lists.newArrayList(sqlQuery1, sqlQuery2, sqlQuery3));
     testQuery(pqlStarTreeQuery, Lists.newArrayList(sqlQuery1, sqlQuery2, sqlQuery3));
     pqlQuery = "SELECT MIN(DepDelayMinutes), COUNT(*) FROM " + DEFAULT_TABLE_NAME;
-    pqlStarTreeQuery =  "SELECT MIN(DepDelayMinutes), COUNT(*) FROM "+ STAR_TREE_TABLE_NAME;
+    pqlStarTreeQuery = "SELECT MIN(DepDelayMinutes), COUNT(*) FROM " + STAR_TREE_TABLE_NAME;
     sqlQuery1 = "SELECT MIN(DepDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
     sqlQuery2 = "SELECT COUNT(*) FROM " + DEFAULT_TABLE_NAME;
     testQuery(pqlQuery, Lists.newArrayList(sqlQuery1, sqlQuery2));
@@ -209,22 +211,24 @@ public class MetadataAndDictionaryAggregationPlanClusterIntegrationTest extends
 
     // double
     pqlQuery = "SELECT MAX(ArrDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
-    pqlStarTreeQuery =  "SELECT MAX(ArrDelayMinutes) FROM "+ STAR_TREE_TABLE_NAME;
+    pqlStarTreeQuery = "SELECT MAX(ArrDelayMinutes) FROM " + STAR_TREE_TABLE_NAME;
     sqlQuery = "SELECT MAX(ArrDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
     testQuery(pqlQuery, Collections.singletonList(sqlQuery));
     testQuery(pqlStarTreeQuery, Collections.singletonList(sqlQuery));
     pqlQuery = "SELECT MIN(ArrDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
-    pqlStarTreeQuery =  "SELECT MIN(ArrDelayMinutes) FROM "+ STAR_TREE_TABLE_NAME;
+    pqlStarTreeQuery = "SELECT MIN(ArrDelayMinutes) FROM " + STAR_TREE_TABLE_NAME;
     sqlQuery = "SELECT MIN(ArrDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
     testQuery(pqlQuery, Collections.singletonList(sqlQuery));
     testQuery(pqlStarTreeQuery, Collections.singletonList(sqlQuery));
     pqlQuery = "SELECT MINMAXRANGE(ArrDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
-    pqlStarTreeQuery =  "SELECT MINMAXRANGE(ArrDelayMinutes) FROM "+ STAR_TREE_TABLE_NAME;
+    pqlStarTreeQuery = "SELECT MINMAXRANGE(ArrDelayMinutes) FROM " + STAR_TREE_TABLE_NAME;
     sqlQuery = "SELECT MAX(ArrDelayMinutes)-MIN(ArrDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
     testQuery(pqlQuery, Collections.singletonList(sqlQuery));
     testQuery(pqlStarTreeQuery, Collections.singletonList(sqlQuery));
-    pqlQuery = "SELECT MIN(ArrDelayMinutes), MAX(ArrDelayMinutes), MINMAXRANGE(ArrDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
-    pqlStarTreeQuery = "SELECT MIN(ArrDelayMinutes), MAX(ArrDelayMinutes), MINMAXRANGE(ArrDelayMinutes) FROM " + STAR_TREE_TABLE_NAME;
+    pqlQuery =
+        "SELECT MIN(ArrDelayMinutes), MAX(ArrDelayMinutes), MINMAXRANGE(ArrDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
+    pqlStarTreeQuery =
+        "SELECT MIN(ArrDelayMinutes), MAX(ArrDelayMinutes), MINMAXRANGE(ArrDelayMinutes) FROM " + STAR_TREE_TABLE_NAME;
     sqlQuery1 = "SELECT MIN(ArrDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
     sqlQuery2 = "SELECT MAX(ArrDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
     sqlQuery3 = "SELECT MAX(ArrDelayMinutes)-MIN(ArrDelayMinutes) FROM " + DEFAULT_TABLE_NAME;
@@ -239,7 +243,7 @@ public class MetadataAndDictionaryAggregationPlanClusterIntegrationTest extends
 
     // long
     pqlQuery = "SELECT MAX(AirlineID) FROM " + DEFAULT_TABLE_NAME;
-    pqlStarTreeQuery =  "SELECT MAX(AirlineID) FROM "+ STAR_TREE_TABLE_NAME;
+    pqlStarTreeQuery = "SELECT MAX(AirlineID) FROM " + STAR_TREE_TABLE_NAME;
     sqlQuery = "SELECT MAX(AirlineID) FROM " + DEFAULT_TABLE_NAME;
     testQuery(pqlQuery, Collections.singletonList(sqlQuery));
     testQuery(pqlStarTreeQuery, Collections.singletonList(sqlQuery));
@@ -287,8 +291,10 @@ public class MetadataAndDictionaryAggregationPlanClusterIntegrationTest extends
     sqlQuery = "SELECT MAX(ActualElapsedTime)-MIN(ActualElapsedTime) FROM " + DEFAULT_TABLE_NAME;
     testQuery(pqlQuery, Collections.singletonList(sqlQuery));
     testQuery(pqlStarTreeQuery, Collections.singletonList(sqlQuery));
-    pqlQuery = "SELECT MIN(ActualElapsedTime), MAX(ActualElapsedTime), MINMAXRANGE(ActualElapsedTime) FROM " + DEFAULT_TABLE_NAME;
-    pqlStarTreeQuery = "SELECT MIN(ActualElapsedTime), MAX(ActualElapsedTime), MINMAXRANGE(ActualElapsedTime) FROM " + STAR_TREE_TABLE_NAME;
+    pqlQuery = "SELECT MIN(ActualElapsedTime), MAX(ActualElapsedTime), MINMAXRANGE(ActualElapsedTime) FROM "
+        + DEFAULT_TABLE_NAME;
+    pqlStarTreeQuery = "SELECT MIN(ActualElapsedTime), MAX(ActualElapsedTime), MINMAXRANGE(ActualElapsedTime) FROM "
+        + STAR_TREE_TABLE_NAME;
     sqlQuery1 = "SELECT MIN(ActualElapsedTime) FROM " + DEFAULT_TABLE_NAME;
     sqlQuery2 = "SELECT MAX(ActualElapsedTime) FROM " + DEFAULT_TABLE_NAME;
     sqlQuery3 = "SELECT MAX(ActualElapsedTime)-MIN(ActualElapsedTime) FROM " + DEFAULT_TABLE_NAME;
@@ -365,51 +371,50 @@ public class MetadataAndDictionaryAggregationPlanClusterIntegrationTest extends
     // TODO: add test cases for string column when we add support for min and max on string datatype columns
 
     // Check execution stats
-    JSONObject response;
+    JsonNode response;
 
     // Dictionary column: answered by DictionaryBasedAggregationOperator
     pqlQuery = "SELECT MAX(ArrTime) FROM " + DEFAULT_TABLE_NAME;
     response = postQuery(pqlQuery);
-    Assert.assertEquals(response.getLong("numEntriesScannedPostFilter"), 0);
-    Assert.assertEquals(response.getLong("numEntriesScannedInFilter"), 0);
-    Assert.assertEquals(response.getLong("totalDocs"), response.getLong("numDocsScanned"));
+    assertEquals(response.get("numEntriesScannedPostFilter").asLong(), 0);
+    assertEquals(response.get("numEntriesScannedInFilter").asLong(), 0);
+    assertEquals(response.get("totalDocs").asLong(), response.get("numDocsScanned").asLong());
 
     // Non dictionary column: not answered by DictionaryBasedAggregationOperator
     pqlQuery = "SELECT MAX(DepDelay) FROM " + DEFAULT_TABLE_NAME;
     response = postQuery(pqlQuery);
-    Assert.assertEquals(response.getLong("numEntriesScannedPostFilter"), response.getLong("numDocsScanned"));
-    Assert.assertEquals(response.getLong("numEntriesScannedInFilter"), 0);
-    Assert.assertEquals(response.getLong("totalDocs"), response.getLong("numDocsScanned"));
+    assertEquals(response.get("numEntriesScannedPostFilter").asLong(), response.get("numDocsScanned").asLong());
+    assertEquals(response.get("numEntriesScannedInFilter").asLong(), 0);
+    assertEquals(response.get("totalDocs").asLong(), response.get("numDocsScanned").asLong());
 
     // multiple dictionary based aggregation functions, dictionary columns: answered by DictionaryBasedAggregationOperator
     pqlQuery = "SELECT MAX(ArrTime),MIN(ArrTime) FROM " + DEFAULT_TABLE_NAME;
     response = postQuery(pqlQuery);
-    Assert.assertEquals(response.getLong("numEntriesScannedPostFilter"), 0);
-    Assert.assertEquals(response.getLong("numEntriesScannedInFilter"), 0);
-    Assert.assertEquals(response.getLong("totalDocs"), response.getLong("numDocsScanned"));
+    assertEquals(response.get("numEntriesScannedPostFilter").asLong(), 0);
+    assertEquals(response.get("numEntriesScannedInFilter").asLong(), 0);
+    assertEquals(response.get("totalDocs").asLong(), response.get("numDocsScanned").asLong());
 
     // multiple aggregation functions, mix of dictionary based and non dictionary based: not answered by DictionaryBasedAggregationOperator
     pqlQuery = "SELECT MAX(ArrTime),COUNT(ArrTime) FROM " + DEFAULT_TABLE_NAME;
     response = postQuery(pqlQuery);
-    Assert.assertEquals(response.getLong("numEntriesScannedPostFilter"), response.getLong("numDocsScanned"));
-    Assert.assertEquals(response.getLong("numEntriesScannedInFilter"), 0);
-    Assert.assertEquals(response.getLong("totalDocs"), response.getLong("numDocsScanned"));
+    assertEquals(response.get("numEntriesScannedPostFilter").asLong(), response.get("numDocsScanned").asLong());
+    assertEquals(response.get("numEntriesScannedInFilter").asLong(), 0);
+    assertEquals(response.get("totalDocs").asLong(), response.get("numDocsScanned").asLong());
 
     // group by in query : not answered by DictionaryBasedAggregationOperator
     pqlQuery = "SELECT MAX(ArrTime) FROM " + DEFAULT_TABLE_NAME + "  group by DaysSinceEpoch";
     response = postQuery(pqlQuery);
-    Assert.assertEquals(response.getLong("numEntriesScannedPostFilter") > 0, true);
-    Assert.assertEquals(response.getLong("numEntriesScannedInFilter"), 0);
-    Assert.assertEquals(response.getLong("totalDocs"), response.getLong("numDocsScanned"));
+    assertTrue(response.get("numEntriesScannedPostFilter").asLong() > 0);
+    assertEquals(response.get("numEntriesScannedInFilter").asLong(), 0);
+    assertEquals(response.get("totalDocs").asLong(), response.get("numDocsScanned").asLong());
 
     // filter in query: not answered by DictionaryBasedAggregationOperator
     pqlQuery = "SELECT MAX(ArrTime) FROM " + DEFAULT_TABLE_NAME + " where DaysSinceEpoch > 16100";
     response = postQuery(pqlQuery);
-    Assert.assertEquals(response.getLong("numEntriesScannedPostFilter") > 0, true);
-    Assert.assertEquals(response.getLong("numEntriesScannedInFilter") > 0, true);
+    assertTrue(response.get("numEntriesScannedPostFilter").asLong() > 0);
+    assertTrue(response.get("numEntriesScannedInFilter").asLong() > 0);
   }
 
-
   @Test
   public void testMetadataBasedQueries() throws Exception {
 
@@ -432,51 +437,48 @@ public class MetadataAndDictionaryAggregationPlanClusterIntegrationTest extends
     testQuery(pqlStarTreeQuery, Collections.singletonList(sqlQuery));
 
     // Check execution stats
-    JSONObject response;
+    JsonNode response;
 
     pqlQuery = "SELECT COUNT(*) FROM " + DEFAULT_TABLE_NAME;
     response = postQuery(pqlQuery);
-    Assert.assertEquals(response.getLong("numEntriesScannedPostFilter"), 0);
-    Assert.assertEquals(response.getLong("numEntriesScannedInFilter"), 0);
-    Assert.assertEquals(response.getLong("totalDocs"), response.getLong("numDocsScanned"));
+    assertEquals(response.get("numEntriesScannedPostFilter").asLong(), 0);
+    assertEquals(response.get("numEntriesScannedInFilter").asLong(), 0);
+    assertEquals(response.get("totalDocs").asLong(), response.get("numDocsScanned").asLong());
 
     pqlStarTreeQuery = "SELECT COUNT(*) FROM " + STAR_TREE_TABLE_NAME;
     response = postQuery(pqlStarTreeQuery);
-    Assert.assertEquals(response.getLong("numEntriesScannedPostFilter"), 0);
-    Assert.assertEquals(response.getLong("numEntriesScannedInFilter"), 0);
-    Assert.assertEquals(response.getLong("totalDocs"), response.getLong("numDocsScanned"));
-
+    assertEquals(response.get("numEntriesScannedPostFilter").asLong(), 0);
+    assertEquals(response.get("numEntriesScannedInFilter").asLong(), 0);
+    assertEquals(response.get("totalDocs").asLong(), response.get("numDocsScanned").asLong());
 
     // group by present in query: not answered by MetadataBasedAggregationOperator
     pqlQuery = "SELECT COUNT(*) FROM " + DEFAULT_TABLE_NAME + " GROUP BY DaysSinceEpoch";
     response = postQuery(pqlQuery);
-    Assert.assertEquals(response.getLong("numEntriesScannedPostFilter") > 0, true);
-    Assert.assertEquals(response.getLong("numEntriesScannedInFilter"), 0);
-    Assert.assertEquals(response.getLong("totalDocs"), response.getLong("numDocsScanned"));
+    assertTrue(response.get("numEntriesScannedPostFilter").asLong() > 0);
+    assertEquals(response.get("numEntriesScannedInFilter").asLong(), 0);
+    assertEquals(response.get("totalDocs").asLong(), response.get("numDocsScanned").asLong());
 
     // filter present in query: not answered by MetadataBasedAggregationOperator
     pqlQuery = "SELECT COUNT(*) FROM " + DEFAULT_TABLE_NAME + " WHERE DaysSinceEpoch > 16100";
     response = postQuery(pqlQuery);
-    Assert.assertEquals(response.getLong("numEntriesScannedPostFilter"), 0);
-    Assert.assertEquals(response.getLong("numEntriesScannedInFilter") > 0, true);
+    assertEquals(response.get("numEntriesScannedPostFilter").asLong(), 0);
+    assertTrue(response.get("numEntriesScannedInFilter").asLong() > 0);
 
     // mixed aggregation functions in query: not answered by MetadataBasedAggregationOperator
     pqlQuery = "SELECT COUNT(*),MAX(ArrTime) FROM " + DEFAULT_TABLE_NAME;
     response = postQuery(pqlQuery);
-    Assert.assertEquals(response.getLong("numEntriesScannedPostFilter") > 0, true);
-    Assert.assertEquals(response.getLong("numEntriesScannedInFilter"), 0);
-    Assert.assertEquals(response.getLong("totalDocs"), response.getLong("numDocsScanned"));
+    assertTrue(response.get("numEntriesScannedPostFilter").asLong() > 0);
+    assertEquals(response.get("numEntriesScannedInFilter").asLong(), 0);
+    assertEquals(response.get("totalDocs").asLong(), response.get("numDocsScanned").asLong());
 
     // mixed aggregation functions in star tree query: not answered by MetadataBasedAggregationOperator
     pqlStarTreeQuery = "SELECT COUNT(*),MAX(DaysSinceEpoch) FROM " + STAR_TREE_TABLE_NAME;
     response = postQuery(pqlStarTreeQuery);
-    Assert.assertEquals(response.getLong("numEntriesScannedPostFilter") > 0, true);
-    Assert.assertEquals(response.getLong("numEntriesScannedInFilter"), 0);
-    Assert.assertEquals(response.getLong("totalDocs"), response.getLong("numDocsScanned"));
+    assertTrue(response.get("numEntriesScannedPostFilter").asLong() > 0);
+    assertEquals(response.get("numEntriesScannedInFilter").asLong(), 0);
+    assertEquals(response.get("totalDocs").asLong(), response.get("numDocsScanned").asLong());
   }
 
-
-
   @AfterClass
   public void tearDown() throws Exception {
     dropOfflineTable(DEFAULT_TABLE_NAME);
diff --git a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/OfflineClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/OfflineClusterIntegrationTest.java
index 7cbe5bb..150eae9 100644
--- a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/OfflineClusterIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/OfflineClusterIntegrationTest.java
@@ -18,9 +18,11 @@
  */
 package com.linkedin.pinot.integration.tests;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import com.linkedin.pinot.common.utils.CommonConstants;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.ServiceStatus;
 import com.linkedin.pinot.core.indexsegment.generator.SegmentVersion;
 import com.linkedin.pinot.util.TestUtils;
@@ -36,13 +38,12 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import javax.annotation.Nullable;
 import org.apache.commons.io.FileUtils;
-import org.json.JSONArray;
-import org.json.JSONObject;
-import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
+import static org.testng.Assert.*;
+
 
 /**
  * Integration test that converts Avro data for 12 segments and runs queries against it.
@@ -141,9 +142,9 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
 
   @Test
   public void testInstancesStarted() {
-    Assert.assertEquals(_serviceStatusCallbacks.size(), getNumBrokers() + getNumServers());
+    assertEquals(_serviceStatusCallbacks.size(), getNumBrokers() + getNumServers());
     for (ServiceStatus.ServiceStatusCallback serviceStatusCallback : _serviceStatusCallbacks) {
-      Assert.assertEquals(serviceStatusCallback.getServiceStatus(), ServiceStatus.Status.GOOD);
+      assertEquals(serviceStatusCallback.getServiceStatus(), ServiceStatus.Status.GOOD);
     }
   }
 
@@ -151,8 +152,8 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
   public void testInvertedIndexTriggering() throws Exception {
     final long numTotalDocs = getCountStarResult();
 
-    JSONObject queryResponse = postQuery(TEST_UPDATED_INVERTED_INDEX_QUERY);
-    Assert.assertEquals(queryResponse.getLong("numEntriesScannedInFilter"), numTotalDocs);
+    JsonNode queryResponse = postQuery(TEST_UPDATED_INVERTED_INDEX_QUERY);
+    assertEquals(queryResponse.get("numEntriesScannedInFilter").asLong(), numTotalDocs);
 
     // Update table config and trigger reload
     updateOfflineTable(getTableName(), null, null, null, null, getLoadMode(), SegmentVersion.v1,
@@ -166,10 +167,10 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
       @Override
       public Boolean apply(@Nullable Void aVoid) {
         try {
-          JSONObject queryResponse = postQuery(TEST_UPDATED_INVERTED_INDEX_QUERY);
+          JsonNode queryResponse = postQuery(TEST_UPDATED_INVERTED_INDEX_QUERY);
           // Total docs should not change during reload
-          Assert.assertEquals(queryResponse.getLong("totalDocs"), numTotalDocs);
-          return queryResponse.getLong("numEntriesScannedInFilter") == 0L;
+          assertEquals(queryResponse.get("totalDocs").asLong(), numTotalDocs);
+          return queryResponse.get("numEntriesScannedInFilter").asLong() == 0L;
         } catch (Exception e) {
           throw new RuntimeException(e);
         }
@@ -180,12 +181,12 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
   @Test
   public void testBloomFilterTriggering() throws Exception {
     final long numTotalDocs = getCountStarResult();
-    JSONObject queryResponse = postQuery(TEST_UPDATED_BLOOM_FILTER_QUERY);
-    Assert.assertEquals(queryResponse.getLong("numSegmentsProcessed"), NUM_SEGMENTS);
+    JsonNode queryResponse = postQuery(TEST_UPDATED_BLOOM_FILTER_QUERY);
+    assertEquals(queryResponse.get("numSegmentsProcessed").asLong(), NUM_SEGMENTS);
 
     // Update table config and trigger reload
-    updateOfflineTable(getTableName(), null, null, null, null, getLoadMode(), SegmentVersion.v1,
-        null, UPDATED_BLOOM_FLITER_COLUMNS, getTaskConfig());
+    updateOfflineTable(getTableName(), null, null, null, null, getLoadMode(), SegmentVersion.v1, null,
+        UPDATED_BLOOM_FLITER_COLUMNS, getTaskConfig());
 
     updateTableConfiguration();
 
@@ -195,10 +196,10 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
       @Override
       public Boolean apply(@Nullable Void aVoid) {
         try {
-          JSONObject queryResponse = postQuery(TEST_UPDATED_BLOOM_FILTER_QUERY);
+          JsonNode queryResponse = postQuery(TEST_UPDATED_BLOOM_FILTER_QUERY);
           // Total docs should not change during reload
-          Assert.assertEquals(queryResponse.getLong("totalDocs"), numTotalDocs);
-          return queryResponse.getLong("numSegmentsProcessed") == 0L;
+          assertEquals(queryResponse.get("totalDocs").asLong(), numTotalDocs);
+          return queryResponse.get("numSegmentsProcessed").asLong() == 0L;
         } catch (Exception e) {
           throw new RuntimeException(e);
         }
@@ -227,16 +228,16 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
     long numTotalDocs = getCountStarResult();
 
     reloadDefaultColumns(true);
-    JSONObject queryResponse = postQuery(SELECT_STAR_QUERY);
-    Assert.assertEquals(queryResponse.getLong("totalDocs"), numTotalDocs);
-    Assert.assertEquals(queryResponse.getJSONObject("selectionResults").getJSONArray("columns").length(), 89);
+    JsonNode queryResponse = postQuery(SELECT_STAR_QUERY);
+    assertEquals(queryResponse.get("totalDocs").asLong(), numTotalDocs);
+    assertEquals(queryResponse.get("selectionResults").get("columns").size(), 89);
 
     testNewAddedColumns();
 
     reloadDefaultColumns(false);
     queryResponse = postQuery(SELECT_STAR_QUERY);
-    Assert.assertEquals(queryResponse.getLong("totalDocs"), numTotalDocs);
-    Assert.assertEquals(queryResponse.getJSONObject("selectionResults").getJSONArray("columns").length(), 79);
+    assertEquals(queryResponse.get("totalDocs").asLong(), numTotalDocs);
+    assertEquals(queryResponse.get("selectionResults").get("columns").size(), 79);
   }
 
   private void reloadDefaultColumns(final boolean withExtraColumns) throws Exception {
@@ -264,10 +265,10 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
       @Override
       public Boolean apply(@Nullable Void aVoid) {
         try {
-          JSONObject queryResponse = postQuery(TEST_DEFAULT_COLUMNS_QUERY);
+          JsonNode queryResponse = postQuery(TEST_DEFAULT_COLUMNS_QUERY);
           // Total docs should not change during reload
-          Assert.assertEquals(queryResponse.getLong("totalDocs"), numTotalDocs);
-          long count = queryResponse.getJSONArray("aggregationResults").getJSONObject(0).getLong("value");
+          assertEquals(queryResponse.get("totalDocs").asLong(), numTotalDocs);
+          long count = queryResponse.get("aggregationResults").get(0).get("value").asLong();
           if (withExtraColumns) {
             return count == numTotalDocs;
           } else {
@@ -282,7 +283,7 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
 
   private void sendSchema(String resourceName) throws Exception {
     URL resource = OfflineClusterIntegrationTest.class.getClassLoader().getResource(resourceName);
-    Assert.assertNotNull(resource);
+    assertNotNull(resource);
     File schemaFile = new File(resource.getFile());
     addSchema(schemaFile, getTableName());
   }
@@ -341,56 +342,53 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
     testQuery(pqlQuery, Collections.singletonList(sqlQuery));
 
     // Test other query forms with new added columns
-    JSONObject response;
-    JSONObject groupByResult;
+    JsonNode response;
+    JsonNode groupByResult;
     pqlQuery = "SELECT SUM(NewAddedFloatMetric) FROM mytable GROUP BY NewAddedSVStringDimension";
     response = postQuery(pqlQuery);
-    groupByResult =
-        response.getJSONArray("aggregationResults").getJSONObject(0).getJSONArray("groupByResult").getJSONObject(0);
-    Assert.assertEquals(groupByResult.getDouble("value"), 0.0);
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(0), "null");
+    groupByResult = response.get("aggregationResults").get(0).get("groupByResult").get(0);
+    assertEquals(groupByResult.get("value").asDouble(), 0.0);
+    assertEquals(groupByResult.get("group").get(0).asText(), "null");
     pqlQuery = "SELECT SUM(NewAddedDoubleMetric) FROM mytable GROUP BY NewAddedIntDimension";
     response = postQuery(pqlQuery);
-    groupByResult =
-        response.getJSONArray("aggregationResults").getJSONObject(0).getJSONArray("groupByResult").getJSONObject(0);
-    Assert.assertEquals(groupByResult.getDouble("value"), 0.0);
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(0), String.valueOf(Integer.MIN_VALUE));
+    groupByResult = response.get("aggregationResults").get(0).get("groupByResult").get(0);
+    assertEquals(groupByResult.get("value").asDouble(), 0.0);
+    assertEquals(groupByResult.get("group").get(0).asInt(), Integer.MIN_VALUE);
     pqlQuery = "SELECT SUM(NewAddedIntMetric) FROM mytable GROUP BY NewAddedLongDimension";
     response = postQuery(pqlQuery);
-    groupByResult =
-        response.getJSONArray("aggregationResults").getJSONObject(0).getJSONArray("groupByResult").getJSONObject(0);
-    Assert.assertEquals(groupByResult.getDouble("value"), numTotalDocsInDouble);
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(0), String.valueOf(Long.MIN_VALUE));
+    groupByResult = response.get("aggregationResults").get(0).get("groupByResult").get(0);
+    assertEquals(groupByResult.get("value").asDouble(), numTotalDocsInDouble);
+    assertEquals(groupByResult.get("group").get(0).asLong(), Long.MIN_VALUE);
     pqlQuery =
         "SELECT SUM(NewAddedIntMetric), SUM(NewAddedLongMetric), SUM(NewAddedFloatMetric), SUM(NewAddedDoubleMetric) "
             + "FROM mytable GROUP BY NewAddedIntDimension, NewAddedLongDimension, NewAddedFloatDimension, "
             + "NewAddedDoubleDimension, NewAddedSVStringDimension, NewAddedMVStringDimension";
     response = postQuery(pqlQuery);
-    JSONArray groupByResultArray = response.getJSONArray("aggregationResults");
-    groupByResult = groupByResultArray.getJSONObject(0).getJSONArray("groupByResult").getJSONObject(0);
-    Assert.assertEquals(groupByResult.getDouble("value"), numTotalDocsInDouble);
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(0), String.valueOf(Integer.MIN_VALUE));
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(1), String.valueOf(Long.MIN_VALUE));
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(2), String.valueOf(Float.NEGATIVE_INFINITY));
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(3), String.valueOf(Double.NEGATIVE_INFINITY));
-    groupByResult = groupByResultArray.getJSONObject(1).getJSONArray("groupByResult").getJSONObject(0);
-    Assert.assertEquals(groupByResult.getDouble("value"), numTotalDocsInDouble);
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(0), String.valueOf(Integer.MIN_VALUE));
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(1), String.valueOf(Long.MIN_VALUE));
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(2), String.valueOf(Float.NEGATIVE_INFINITY));
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(3), String.valueOf(Double.NEGATIVE_INFINITY));
-    groupByResult = groupByResultArray.getJSONObject(2).getJSONArray("groupByResult").getJSONObject(0);
-    Assert.assertEquals(groupByResult.getDouble("value"), 0.0);
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(0), String.valueOf(Integer.MIN_VALUE));
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(1), String.valueOf(Long.MIN_VALUE));
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(2), String.valueOf(Float.NEGATIVE_INFINITY));
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(3), String.valueOf(Double.NEGATIVE_INFINITY));
-    groupByResult = groupByResultArray.getJSONObject(3).getJSONArray("groupByResult").getJSONObject(0);
-    Assert.assertEquals(groupByResult.getDouble("value"), 0.0);
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(0), String.valueOf(Integer.MIN_VALUE));
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(1), String.valueOf(Long.MIN_VALUE));
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(2), String.valueOf(Float.NEGATIVE_INFINITY));
-    Assert.assertEquals(groupByResult.getJSONArray("group").getString(3), String.valueOf(Double.NEGATIVE_INFINITY));
+    JsonNode groupByResultArray = response.get("aggregationResults");
+    groupByResult = groupByResultArray.get(0).get("groupByResult").get(0);
+    assertEquals(groupByResult.get("value").asDouble(), numTotalDocsInDouble);
+    assertEquals(groupByResult.get("group").get(0).asInt(), Integer.MIN_VALUE);
+    assertEquals(groupByResult.get("group").get(1).asLong(), Long.MIN_VALUE);
+    assertEquals((float) groupByResult.get("group").get(2).asDouble(), Float.NEGATIVE_INFINITY);
+    assertEquals(groupByResult.get("group").get(3).asDouble(), Double.NEGATIVE_INFINITY);
+    groupByResult = groupByResultArray.get(1).get("groupByResult").get(0);
+    assertEquals(groupByResult.get("value").asDouble(), numTotalDocsInDouble);
+    assertEquals(groupByResult.get("group").get(0).asInt(), Integer.MIN_VALUE);
+    assertEquals(groupByResult.get("group").get(1).asLong(), Long.MIN_VALUE);
+    assertEquals((float) groupByResult.get("group").get(2).asDouble(), Float.NEGATIVE_INFINITY);
+    assertEquals(groupByResult.get("group").get(3).asDouble(), Double.NEGATIVE_INFINITY);
+    groupByResult = groupByResultArray.get(2).get("groupByResult").get(0);
+    assertEquals(groupByResult.get("value").asDouble(), 0.0);
+    assertEquals(groupByResult.get("group").get(0).asInt(), Integer.MIN_VALUE);
+    assertEquals(groupByResult.get("group").get(1).asLong(), Long.MIN_VALUE);
+    assertEquals((float) groupByResult.get("group").get(2).asDouble(), Float.NEGATIVE_INFINITY);
+    assertEquals(groupByResult.get("group").get(3).asDouble(), Double.NEGATIVE_INFINITY);
+    groupByResult = groupByResultArray.get(3).get("groupByResult").get(0);
+    assertEquals(groupByResult.get("value").asDouble(), 0.0);
+    assertEquals(groupByResult.get("group").get(0).asInt(), Integer.MIN_VALUE);
+    assertEquals(groupByResult.get("group").get(1).asLong(), Long.MIN_VALUE);
+    assertEquals((float) groupByResult.get("group").get(2).asDouble(), Float.NEGATIVE_INFINITY);
+    assertEquals(groupByResult.get("group").get(3).asDouble(), Double.NEGATIVE_INFINITY);
   }
 
   @Test
@@ -402,77 +400,74 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
   @Test
   public void testUDF() throws Exception {
     String pqlQuery = "SELECT COUNT(*) FROM mytable GROUP BY timeConvert(DaysSinceEpoch,'DAYS','SECONDS')";
-    JSONObject response = postQuery(pqlQuery);
-    JSONObject groupByResult = response.getJSONArray("aggregationResults").getJSONObject(0);
-    JSONObject groupByEntry = groupByResult.getJSONArray("groupByResult").getJSONObject(0);
-    Assert.assertEquals(groupByEntry.getDouble("value"), 605.0);
-    Assert.assertEquals(groupByEntry.getJSONArray("group").getString(0), Integer.toString(16138 * 24 * 3600));
-    Assert.assertEquals(groupByResult.getJSONArray("groupByColumns").getString(0),
-        "timeconvert(DaysSinceEpoch,'DAYS','SECONDS')");
+    JsonNode response = postQuery(pqlQuery);
+    JsonNode groupByResult = response.get("aggregationResults").get(0);
+    JsonNode groupByEntry = groupByResult.get("groupByResult").get(0);
+    assertEquals(groupByEntry.get("value").asDouble(), 605.0);
+    assertEquals(groupByEntry.get("group").get(0).asInt(), 16138 * 24 * 3600);
+    assertEquals(groupByResult.get("groupByColumns").get(0).asText(), "timeconvert(DaysSinceEpoch,'DAYS','SECONDS')");
 
     pqlQuery =
         "SELECT COUNT(*) FROM mytable GROUP BY dateTimeConvert(DaysSinceEpoch,'1:DAYS:EPOCH','1:HOURS:EPOCH','1:HOURS')";
     response = postQuery(pqlQuery);
-    groupByResult = response.getJSONArray("aggregationResults").getJSONObject(0);
-    groupByEntry = groupByResult.getJSONArray("groupByResult").getJSONObject(0);
-    Assert.assertEquals(groupByEntry.getDouble("value"), 605.0);
-    Assert.assertEquals(groupByEntry.getJSONArray("group").getString(0), Integer.toString(16138 * 24));
-    Assert.assertEquals(groupByResult.getJSONArray("groupByColumns").getString(0),
+    groupByResult = response.get("aggregationResults").get(0);
+    groupByEntry = groupByResult.get("groupByResult").get(0);
+    assertEquals(groupByEntry.get("value").asDouble(), 605.0);
+    assertEquals(groupByEntry.get("group").get(0).asInt(), 16138 * 24);
+    assertEquals(groupByResult.get("groupByColumns").get(0).asText(),
         "datetimeconvert(DaysSinceEpoch,'1:DAYS:EPOCH','1:HOURS:EPOCH','1:HOURS')");
 
     pqlQuery = "SELECT COUNT(*) FROM mytable GROUP BY add(DaysSinceEpoch,DaysSinceEpoch,15)";
     response = postQuery(pqlQuery);
-    groupByResult = response.getJSONArray("aggregationResults").getJSONObject(0);
-    groupByEntry = groupByResult.getJSONArray("groupByResult").getJSONObject(0);
-    Assert.assertEquals(groupByEntry.getDouble("value"), 605.0);
-    Assert.assertEquals(groupByEntry.getJSONArray("group").getString(0),
-        Double.toString((double) (16138 + 16138 + 15)));
-    Assert.assertEquals(groupByResult.getJSONArray("groupByColumns").getString(0),
-        "add(DaysSinceEpoch,DaysSinceEpoch,'15')");
+    groupByResult = response.get("aggregationResults").get(0);
+    groupByEntry = groupByResult.get("groupByResult").get(0);
+    assertEquals(groupByEntry.get("value").asDouble(), 605.0);
+    assertEquals(groupByEntry.get("group").get(0).asDouble(), 16138.0 + 16138 + 15);
+    assertEquals(groupByResult.get("groupByColumns").get(0).asText(), "add(DaysSinceEpoch,DaysSinceEpoch,'15')");
 
     pqlQuery = "SELECT COUNT(*) FROM mytable GROUP BY sub(DaysSinceEpoch,25)";
     response = postQuery(pqlQuery);
-    groupByResult = response.getJSONArray("aggregationResults").getJSONObject(0);
-    groupByEntry = groupByResult.getJSONArray("groupByResult").getJSONObject(0);
-    Assert.assertEquals(groupByEntry.getDouble("value"), 605.0);
-    Assert.assertEquals(groupByEntry.getJSONArray("group").getString(0), Double.toString((double) 16138 - 25));
-    Assert.assertEquals(groupByResult.getJSONArray("groupByColumns").getString(0), "sub(DaysSinceEpoch,'25')");
+    groupByResult = response.get("aggregationResults").get(0);
+    groupByEntry = groupByResult.get("groupByResult").get(0);
+    assertEquals(groupByEntry.get("value").asDouble(), 605.0);
+    assertEquals(groupByEntry.get("group").get(0).asDouble(), 16138.0 - 25);
+    assertEquals(groupByResult.get("groupByColumns").get(0).asText(), "sub(DaysSinceEpoch,'25')");
 
     pqlQuery = "SELECT COUNT(*) FROM mytable GROUP BY mult(DaysSinceEpoch,24,3600)";
     response = postQuery(pqlQuery);
-    groupByResult = response.getJSONArray("aggregationResults").getJSONObject(0);
-    groupByEntry = groupByResult.getJSONArray("groupByResult").getJSONObject(0);
-    Assert.assertEquals(groupByEntry.getDouble("value"), 605.0);
-    Assert.assertEquals(groupByEntry.getJSONArray("group").getString(0), Double.toString((double) 16138 * 24 * 3600));
-    Assert.assertEquals(groupByResult.getJSONArray("groupByColumns").getString(0), "mult(DaysSinceEpoch,'24','3600')");
+    groupByResult = response.get("aggregationResults").get(0);
+    groupByEntry = groupByResult.get("groupByResult").get(0);
+    assertEquals(groupByEntry.get("value").asDouble(), 605.0);
+    assertEquals(groupByEntry.get("group").get(0).asDouble(), 16138.0 * 24 * 3600);
+    assertEquals(groupByResult.get("groupByColumns").get(0).asText(), "mult(DaysSinceEpoch,'24','3600')");
 
     pqlQuery = "SELECT COUNT(*) FROM mytable GROUP BY div(DaysSinceEpoch,2)";
     response = postQuery(pqlQuery);
-    groupByResult = response.getJSONArray("aggregationResults").getJSONObject(0);
-    groupByEntry = groupByResult.getJSONArray("groupByResult").getJSONObject(0);
-    Assert.assertEquals(groupByEntry.getDouble("value"), 605.0);
-    Assert.assertEquals(groupByEntry.getJSONArray("group").getString(0), Double.toString((double) 16138 / 2));
-    Assert.assertEquals(groupByResult.getJSONArray("groupByColumns").getString(0), "div(DaysSinceEpoch,'2')");
+    groupByResult = response.get("aggregationResults").get(0);
+    groupByEntry = groupByResult.get("groupByResult").get(0);
+    assertEquals(groupByEntry.get("value").asDouble(), 605.0);
+    assertEquals(groupByEntry.get("group").get(0).asDouble(), 16138.0 / 2);
+    assertEquals(groupByResult.get("groupByColumns").get(0).asText(), "div(DaysSinceEpoch,'2')");
 
     pqlQuery = "SELECT COUNT(*) FROM mytable GROUP BY valueIn(DivAirports,'DFW','ORD')";
     response = postQuery(pqlQuery);
-    groupByResult = response.getJSONArray("aggregationResults").getJSONObject(0);
-    groupByEntry = groupByResult.getJSONArray("groupByResult").getJSONObject(0);
-    Assert.assertEquals(groupByEntry.getDouble("value"), 336.0);
-    Assert.assertEquals(groupByEntry.getJSONArray("group").getString(0), "ORD");
-    Assert.assertEquals(groupByResult.getJSONArray("groupByColumns").getString(0), "valuein(DivAirports,'DFW','ORD')");
+    groupByResult = response.get("aggregationResults").get(0);
+    groupByEntry = groupByResult.get("groupByResult").get(0);
+    assertEquals(groupByEntry.get("value").asDouble(), 336.0);
+    assertEquals(groupByEntry.get("group").get(0).asText(), "ORD");
+    assertEquals(groupByResult.get("groupByColumns").get(0).asText(), "valuein(DivAirports,'DFW','ORD')");
 
     pqlQuery = "SELECT MAX(timeConvert(DaysSinceEpoch,'DAYS','SECONDS')) FROM mytable";
     response = postQuery(pqlQuery);
-    JSONObject aggregationResult = response.getJSONArray("aggregationResults").getJSONObject(0);
-    Assert.assertEquals(aggregationResult.get("function"), "max_timeconvert(DaysSinceEpoch,'DAYS','SECONDS')");
-    Assert.assertEquals(aggregationResult.getDouble("value"), 16435.0 * 24 * 3600, 1e-5);
+    JsonNode aggregationResult = response.get("aggregationResults").get(0);
+    assertEquals(aggregationResult.get("function").asText(), "max_timeconvert(DaysSinceEpoch,'DAYS','SECONDS')");
+    assertEquals(aggregationResult.get("value").asDouble(), 16435.0 * 24 * 3600);
 
     pqlQuery = "SELECT MIN(div(DaysSinceEpoch,2)) FROM mytable";
     response = postQuery(pqlQuery);
-    aggregationResult = response.getJSONArray("aggregationResults").getJSONObject(0);
-    Assert.assertEquals(aggregationResult.get("function"), "min_div(DaysSinceEpoch,'2')");
-    Assert.assertEquals(aggregationResult.getDouble("value"), 16071.0 / 2, 1e-5);
+    aggregationResult = response.get("aggregationResults").get(0);
+    assertEquals(aggregationResult.get("function").asText(), "min_div(DaysSinceEpoch,'2')");
+    assertEquals(aggregationResult.get("value").asDouble(), 16071.0 / 2);
   }
 
   @AfterClass
@@ -488,16 +483,16 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
 
   private void testInstanceDecommission() throws Exception {
     // Fetch all instances
-    JSONObject response = new JSONObject(sendGetRequest(_controllerRequestURLBuilder.forInstanceList()));
-    JSONArray instanceList = response.getJSONArray("instances");
-    int numInstances = instanceList.length();
-    Assert.assertEquals(numInstances, getNumBrokers() + getNumServers());
+    JsonNode response = JsonUtils.stringToJsonNode(sendGetRequest(_controllerRequestURLBuilder.forInstanceList()));
+    JsonNode instanceList = response.get("instances");
+    int numInstances = instanceList.size();
+    assertEquals(numInstances, getNumBrokers() + getNumServers());
 
     // Try to delete a server that does not exist
     String deleteInstanceRequest = _controllerRequestURLBuilder.forInstanceDelete("potato");
     try {
       sendDeleteRequest(deleteInstanceRequest);
-      Assert.fail("Delete should have returned a failure status (404)");
+      fail("Delete should have returned a failure status (404)");
     } catch (IOException e) {
       // Expected exception on 404 status code
     }
@@ -506,7 +501,7 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
     String serverName = null;
     String brokerName = null;
     for (int i = 0; i < numInstances; i++) {
-      String instanceName = instanceList.getString(i);
+      String instanceName = instanceList.get(i).asText();
       if (instanceName.startsWith(CommonConstants.Helix.PREFIX_OF_SERVER_INSTANCE)) {
         serverName = instanceName;
       } else if (instanceName.startsWith(CommonConstants.Helix.PREFIX_OF_BROKER_INSTANCE)) {
@@ -518,7 +513,7 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
     deleteInstanceRequest = _controllerRequestURLBuilder.forInstanceDelete(serverName);
     try {
       sendDeleteRequest(deleteInstanceRequest);
-      Assert.fail("Delete should have returned a failure status (409)");
+      fail("Delete should have returned a failure status (409)");
     } catch (IOException e) {
       // Expected exception on 409 status code
     }
@@ -529,7 +524,7 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
     // Try to delete a server whose information is still on the ideal state
     try {
       sendDeleteRequest(deleteInstanceRequest);
-      Assert.fail("Delete should have returned a failure status (409)");
+      fail("Delete should have returned a failure status (409)");
     } catch (IOException e) {
       // Expected exception on 409 status code
     }
@@ -538,15 +533,14 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
     dropOfflineTable(getTableName());
 
     // Now, delete server should work
-    response = new JSONObject(sendDeleteRequest(deleteInstanceRequest));
-    // TODO Cannot compare messages. We need to compare response code.
-//    Assert.assertEquals(response.getString("status"), "success");
+    response = JsonUtils.stringToJsonNode(sendDeleteRequest(deleteInstanceRequest));
+    assertTrue(response.has("status"));
 
     // Try to delete a broker whose information is still live
     try {
       deleteInstanceRequest = _controllerRequestURLBuilder.forInstanceDelete(brokerName);
       sendDeleteRequest(deleteInstanceRequest);
-      Assert.fail("Delete should have returned a failure status (409)");
+      fail("Delete should have returned a failure status (409)");
     } catch (IOException e) {
       // Expected exception on 409 status code
     }
@@ -558,11 +552,11 @@ public class OfflineClusterIntegrationTest extends BaseClusterIntegrationTestSet
 
     // Check if '/INSTANCES/<serverName>' has been erased correctly
     String instancePath = "/" + _clusterName + "/INSTANCES/" + serverName;
-    Assert.assertFalse(_propertyStore.exists(instancePath, 0));
+    assertFalse(_propertyStore.exists(instancePath, 0));
 
     // Check if '/CONFIGS/PARTICIPANT/<serverName>' has been erased correctly
     String configPath = "/" + _clusterName + "/CONFIGS/PARTICIPANT/" + serverName;
-    Assert.assertFalse(_propertyStore.exists(configPath, 0));
+    assertFalse(_propertyStore.exists(configPath, 0));
   }
 
   @Override
diff --git a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/PinotURIUploadIntegrationTest.java b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/PinotURIUploadIntegrationTest.java
index 3d5c632..d673557 100644
--- a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/PinotURIUploadIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/PinotURIUploadIntegrationTest.java
@@ -18,8 +18,10 @@
  */
 package com.linkedin.pinot.integration.tests;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.linkedin.pinot.common.utils.FileUploadDownloadClient;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.common.utils.TarGzCompressionUtils;
 import com.linkedin.pinot.core.indexsegment.generator.SegmentVersion;
 import com.linkedin.pinot.util.TestUtils;
@@ -44,8 +46,6 @@ import org.apache.avro.generic.GenericDatumWriter;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.commons.httpclient.HttpStatus;
 import org.apache.commons.io.FileUtils;
-import org.apache.htrace.fasterxml.jackson.databind.JsonNode;
-import org.apache.htrace.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.http.Header;
 import org.apache.http.HttpHost;
 import org.apache.http.HttpResponse;
@@ -244,7 +244,7 @@ public class PinotURIUploadIntegrationTest extends BaseClusterIntegrationTest {
         throw new IllegalStateException(res.getStatusLine().toString());
       }
       InputStream content = res.getEntity().getContent();
-      JsonNode segmentsData = new ObjectMapper().readTree(content);
+      JsonNode segmentsData = JsonUtils.inputStreamToJsonNode(content);
 
       if (segmentsData != null) {
         JsonNode offlineSegments = segmentsData.get(0).get("OFFLINE");
diff --git a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/QueryGenerator.java b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/QueryGenerator.java
index 5c10f23..d9bf84b 100644
--- a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/QueryGenerator.java
+++ b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/QueryGenerator.java
@@ -18,6 +18,8 @@
  */
 package com.linkedin.pinot.integration.tests;
 
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileWriter;
@@ -37,8 +39,6 @@ import org.apache.avro.generic.GenericData;
 import org.apache.avro.generic.GenericDatumReader;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.commons.lang.StringUtils;
-import org.json.JSONArray;
-import org.json.JSONObject;
 
 
 /**
@@ -220,9 +220,9 @@ public class QueryGenerator {
     try (BufferedWriter writer = new BufferedWriter(new FileWriter(outputFile))) {
       for (int i = 0; i < 10000; i++) {
         Query query = queryGenerator.generateQuery();
-        JSONObject queryJson = new JSONObject();
+        ObjectNode queryJson = JsonUtils.newObjectNode();
         queryJson.put("pql", query.generatePql());
-        queryJson.put("hsqls", new JSONArray(query.generateH2Sql()));
+        queryJson.set("hsqls", JsonUtils.objectToJsonNode(query.generateH2Sql()));
         writer.write(queryJson.toString());
         writer.newLine();
       }
@@ -358,7 +358,6 @@ public class QueryGenerator {
     }
   }
 
-
   /**
    * Query interface with capability of generating PQL and H2 SQL query.
    */
@@ -830,13 +829,12 @@ public class QueryGenerator {
       }
       //Generate a HAVING predicate
       ArrayList<String> arrayOfAggregationColumnsAndFunctions = new ArrayList<>(aggregationColumnsAndFunctions);
-      HavingQueryFragment havingPredicate= generateHavingPredicate(arrayOfAggregationColumnsAndFunctions);
+      HavingQueryFragment havingPredicate = generateHavingPredicate(arrayOfAggregationColumnsAndFunctions);
       // Generate a result limit of at most MAX_RESULT_LIMIT.
       TopQueryFragment top = new TopQueryFragment(RANDOM.nextInt(MAX_RESULT_LIMIT + 1));
       return new AggregationQuery(arrayOfAggregationColumnsAndFunctions, predicate, groupColumns, top, havingPredicate);
     }
 
-
     /**
      * Helper method to generate a having predicate query fragment.
      *
@@ -849,10 +847,10 @@ public class QueryGenerator {
       List<String> havingClauseBooleanOperators = new ArrayList<String>();
       createHavingClause(arrayOfAggregationColumnsAndFunctions, havingClauseAggregationFunctions,
           havingClauseOperatorsAndValues, havingClauseBooleanOperators);
-      return new HavingQueryFragment(havingClauseAggregationFunctions,havingClauseOperatorsAndValues,havingClauseBooleanOperators);
+      return new HavingQueryFragment(havingClauseAggregationFunctions, havingClauseOperatorsAndValues,
+          havingClauseBooleanOperators);
     }
 
-
     private void createHavingClause(ArrayList<String> arrayOfAggregationColumnsAndFunctions,
         List<String> havingClauseAggregationFunctions, List<String> havingClauseOperatorsAndValues,
         List<String> havingClauseBooleanOperators) {
diff --git a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/StarTreeClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/StarTreeClusterIntegrationTest.java
index 5e62392..053ac03 100644
--- a/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/StarTreeClusterIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/com/linkedin/pinot/integration/tests/StarTreeClusterIntegrationTest.java
@@ -18,6 +18,7 @@
  */
 package com.linkedin.pinot.integration.tests;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.linkedin.pinot.common.data.Schema;
 import com.linkedin.pinot.tools.query.comparison.QueryComparison;
 import com.linkedin.pinot.tools.query.comparison.SegmentInfoProvider;
@@ -33,7 +34,6 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import javax.annotation.Nonnull;
 import org.apache.commons.io.FileUtils;
-import org.json.JSONObject;
 import org.testng.Assert;
 import org.testng.annotations.AfterClass;
 import org.testng.annotations.BeforeClass;
@@ -182,14 +182,13 @@ public class StarTreeClusterIntegrationTest extends BaseClusterIntegrationTest {
 
   private void testStarQuery(String starQuery) throws Exception {
     String referenceQuery = starQuery.replace(STAR_TREE_TABLE_NAME, DEFAULT_TABLE_NAME) + " TOP 10000";
-    JSONObject starResponse = postQuery(starQuery);
-    JSONObject referenceResponse = postQuery(referenceQuery);
+    JsonNode starResponse = postQuery(starQuery);
+    JsonNode referenceResponse = postQuery(referenceQuery);
 
     // Skip comparison if not all results returned in reference response
     if (referenceResponse.has("aggregationResults")) {
-      JSONObject aggregationResults = referenceResponse.getJSONArray("aggregationResults").getJSONObject(0);
-      if (aggregationResults.has("groupByResult")
-          && aggregationResults.getJSONArray("groupByResult").length() == 10000) {
+      JsonNode aggregationResults = referenceResponse.get("aggregationResults").get(0);
+      if (aggregationResults.has("groupByResult") && aggregationResults.get("groupByResult").size() == 10000) {
         return;
       }
     }
diff --git a/pinot-perf/src/main/java/com/linkedin/pinot/perf/BenchmarkQueryEngine.java b/pinot-perf/src/main/java/com/linkedin/pinot/perf/BenchmarkQueryEngine.java
index f6d53f4..38768b9 100644
--- a/pinot-perf/src/main/java/com/linkedin/pinot/perf/BenchmarkQueryEngine.java
+++ b/pinot-perf/src/main/java/com/linkedin/pinot/perf/BenchmarkQueryEngine.java
@@ -32,7 +32,6 @@ import java.util.concurrent.TimeUnit;
 import org.I0Itec.zkclient.ZkClient;
 import org.apache.helix.ZNRecord;
 import org.apache.helix.manager.zk.ZNRecordSerializer;
-import org.json.JSONObject;
 import org.openjdk.jmh.annotations.Benchmark;
 import org.openjdk.jmh.annotations.BenchmarkMode;
 import org.openjdk.jmh.annotations.Fork;
@@ -53,9 +52,7 @@ import org.openjdk.jmh.runner.options.TimeValue;
 @Fork(value = 1, jvmArgs = {"-server", "-Xmx8G", "-XX:MaxDirectMemorySize=16G"})
 public class BenchmarkQueryEngine {
   /** List of query patterns used in the benchmark */
-  private static final String[] QUERY_PATTERNS = new String[] {
-      "SELECT count(*) from myTable"
-  };
+  private static final String[] QUERY_PATTERNS = new String[]{"SELECT count(*) from myTable"};
 
   /** List of optimization flags to test,
    * see {@link OptimizationFlags#getOptimizationFlags(BrokerRequest)} for the syntax
@@ -144,22 +141,18 @@ public class BenchmarkQueryEngine {
 
     ranOnce = false;
 
-    System.out.println(
-        _perfBenchmarkDriver.postQuery(QUERY_PATTERNS[queryPattern], optimizationFlags).toString(2)
-    );
+    System.out.println(_perfBenchmarkDriver.postQuery(QUERY_PATTERNS[queryPattern], optimizationFlags).toString());
   }
 
   @Benchmark
   @BenchmarkMode({Mode.SampleTime})
   @OutputTimeUnit(TimeUnit.MILLISECONDS)
   public int sendQueryToPinot() throws Exception {
-    JSONObject returnValue = _perfBenchmarkDriver.postQuery(QUERY_PATTERNS[queryPattern], optimizationFlags);
-    return returnValue.getInt("totalDocs");
+    return _perfBenchmarkDriver.postQuery(QUERY_PATTERNS[queryPattern], optimizationFlags).get("totalDocs").asInt();
   }
 
   public static void main(String[] args) throws Exception {
-    ChainedOptionsBuilder opt = new OptionsBuilder()
-        .include(BenchmarkQueryEngine.class.getSimpleName())
+    ChainedOptionsBuilder opt = new OptionsBuilder().include(BenchmarkQueryEngine.class.getSimpleName())
         .warmupTime(TimeValue.seconds(30))
         .warmupIterations(4)
         .measurementTime(TimeValue.seconds(30))
diff --git a/pinot-perf/src/main/java/com/linkedin/pinot/perf/BenchmarkRealtimeConsumptionSpeed.java b/pinot-perf/src/main/java/com/linkedin/pinot/perf/BenchmarkRealtimeConsumptionSpeed.java
index 0f62a58..47da7f7 100644
--- a/pinot-perf/src/main/java/com/linkedin/pinot/perf/BenchmarkRealtimeConsumptionSpeed.java
+++ b/pinot-perf/src/main/java/com/linkedin/pinot/perf/BenchmarkRealtimeConsumptionSpeed.java
@@ -18,6 +18,7 @@
  */
 package com.linkedin.pinot.perf;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.google.common.util.concurrent.Uninterruptibles;
 import com.linkedin.pinot.common.utils.KafkaStarterUtils;
 import com.linkedin.pinot.common.utils.TarGzCompressionUtils;
@@ -30,8 +31,6 @@ import java.util.List;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 import kafka.server.KafkaServerStartable;
-import org.json.JSONArray;
-import org.json.JSONObject;
 
 
 /**
@@ -110,11 +109,8 @@ public class BenchmarkRealtimeConsumptionSpeed extends RealtimeClusterIntegratio
 
       // Run the query
       try {
-        JSONObject response = postQuery("select count(*) from mytable");
-        JSONArray aggregationResultsArray = response.getJSONArray("aggregationResults");
-        JSONObject firstAggregationResult = aggregationResultsArray.getJSONObject(0);
-        String pinotValue = firstAggregationResult.get("value").toString();
-        pinotRecordCount = Integer.parseInt(pinotValue);
+        JsonNode response = postQuery("select count(*) from mytable");
+        pinotRecordCount = response.get("aggregationResults").get(0).get("value").asInt();
       } catch (Exception e) {
         // Ignore
         continue;
diff --git a/pinot-perf/src/main/java/com/linkedin/pinot/perf/RealtimeStressTest.java b/pinot-perf/src/main/java/com/linkedin/pinot/perf/RealtimeStressTest.java
index a7f87c4..1b8c36f 100644
--- a/pinot-perf/src/main/java/com/linkedin/pinot/perf/RealtimeStressTest.java
+++ b/pinot-perf/src/main/java/com/linkedin/pinot/perf/RealtimeStressTest.java
@@ -18,6 +18,7 @@
  */
 package com.linkedin.pinot.perf;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.google.common.util.concurrent.Uninterruptibles;
 import com.linkedin.pinot.common.utils.KafkaStarterUtils;
 import com.linkedin.pinot.common.utils.TarGzCompressionUtils;
@@ -31,8 +32,6 @@ import java.util.List;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 import kafka.server.KafkaServerStartable;
-import org.json.JSONArray;
-import org.json.JSONObject;
 
 
 /**
@@ -105,11 +104,8 @@ public class RealtimeStressTest extends RealtimeClusterIntegrationTest {
 
       // Run the query
       try {
-        JSONObject response = postQuery("select count(*) from mytable");
-        JSONArray aggregationResultsArray = response.getJSONArray("aggregationResults");
-        JSONObject firstAggregationResult = aggregationResultsArray.getJSONObject(0);
-        String pinotValue = firstAggregationResult.get("value").toString();
-        pinotRecordCount = Long.parseLong(pinotValue);
+        JsonNode response = postQuery("select count(*) from mytable");
+        pinotRecordCount = response.get("aggregationResults").get(0).get("value").asLong();
       } catch (Exception e) {
         // Ignore
         continue;
diff --git a/pinot-server/pom.xml b/pinot-server/pom.xml
index 7cb4998..ddde34d 100644
--- a/pinot-server/pom.xml
+++ b/pinot-server/pom.xml
@@ -112,14 +112,6 @@
       <artifactId>commons-io</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.json</groupId>
-      <artifactId>json</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.alibaba</groupId>
-      <artifactId>fastjson</artifactId>
-    </dependency>
-    <dependency>
       <groupId>org.apache.helix</groupId>
       <artifactId>helix-core</artifactId>
       <exclusions>
@@ -191,15 +183,15 @@
     </dependency>
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-databind</artifactId>
+      <artifactId>jackson-annotations</artifactId>
     </dependency>
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-annotations</artifactId>
+      <artifactId>jackson-core</artifactId>
     </dependency>
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
-      <artifactId>jackson-core</artifactId>
+      <artifactId>jackson-databind</artifactId>
     </dependency>
     <dependency>
       <groupId>io.swagger</groupId>
diff --git a/pinot-server/src/main/java/com/linkedin/pinot/server/api/resources/DefaultExceptionMapper.java b/pinot-server/src/main/java/com/linkedin/pinot/server/api/resources/DefaultExceptionMapper.java
index 3bf7a85..2e6c23c 100644
--- a/pinot-server/src/main/java/com/linkedin/pinot/server/api/resources/DefaultExceptionMapper.java
+++ b/pinot-server/src/main/java/com/linkedin/pinot/server/api/resources/DefaultExceptionMapper.java
@@ -19,39 +19,32 @@
 package com.linkedin.pinot.server.api.resources;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.ext.ExceptionMapper;
 import javax.ws.rs.ext.Provider;
 
+
 @Provider
-public class DefaultExceptionMapper implements ExceptionMapper<WebApplicationException>
-{
-  private static final transient ObjectMapper MAPPER = new ObjectMapper();
+public class DefaultExceptionMapper implements ExceptionMapper<WebApplicationException> {
 
   @Override
-  public Response toResponse(final WebApplicationException exception)
-  {
-    Response.ResponseBuilder builder = Response.status(exception.getResponse().getStatus())
-                                      .entity(toJson(exception))
-                                      .type(MediaType.APPLICATION_JSON);
+  public Response toResponse(final WebApplicationException exception) {
+    Response.ResponseBuilder builder =
+        Response.status(exception.getResponse().getStatus()).entity(toJson(exception)).type(MediaType.APPLICATION_JSON);
     return builder.build();
   }
 
-  private String toJson(final WebApplicationException exception)
-  {
+  private String toJson(final WebApplicationException exception) {
     ErrorInfo errorInfo = new ErrorInfo(exception);
 
     // difference between try and catch block is that
     // ErrorInfo can contain more information that just message
-    try
-    {
-      return MAPPER.writeValueAsString(errorInfo);
-    }
-    catch (JsonProcessingException e)
-    {
+    try {
+      return JsonUtils.objectToString(errorInfo);
+    } catch (JsonProcessingException e) {
       return "{\"message\":\"Error converting error message: " + e.getMessage() + " to string\"}";
     }
   }
diff --git a/pinot-server/src/main/java/com/linkedin/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/com/linkedin/pinot/server/api/resources/TablesResource.java
index af95b8c..e567275 100644
--- a/pinot-server/src/main/java/com/linkedin/pinot/server/api/resources/TablesResource.java
+++ b/pinot-server/src/main/java/com/linkedin/pinot/server/api/resources/TablesResource.java
@@ -18,6 +18,7 @@
  */
 package com.linkedin.pinot.server.api.resources;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
 import com.linkedin.pinot.common.restlet.resources.ResourceUtils;
 import com.linkedin.pinot.common.restlet.resources.TableSegments;
 import com.linkedin.pinot.common.restlet.resources.TablesList;
@@ -47,7 +48,6 @@ import javax.ws.rs.QueryParam;
 import javax.ws.rs.WebApplicationException;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
-import org.json.JSONException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -150,7 +150,7 @@ public class TablesResource {
       }
       try {
         return segmentMetadata.toJson(columnSet).toString();
-      } catch (JSONException e) {
+      } catch (JsonProcessingException e) {
         LOGGER.error("Failed to convert table {} segment {} to json", tableName, segmentMetadata);
         throw new WebApplicationException("Failed to convert segment metadata to json",
             Response.Status.INTERNAL_SERVER_ERROR);
diff --git a/pinot-server/src/test/java/com/linkedin/pinot/server/api/resources/TablesResourceTest.java b/pinot-server/src/test/java/com/linkedin/pinot/server/api/resources/TablesResourceTest.java
index d9eaac5..1c13b01 100644
--- a/pinot-server/src/test/java/com/linkedin/pinot/server/api/resources/TablesResourceTest.java
+++ b/pinot-server/src/test/java/com/linkedin/pinot/server/api/resources/TablesResourceTest.java
@@ -18,21 +18,20 @@
  */
 package com.linkedin.pinot.server.api.resources;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.linkedin.pinot.common.restlet.resources.TableSegments;
 import com.linkedin.pinot.common.restlet.resources.TablesList;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.core.indexsegment.IndexSegment;
 import com.linkedin.pinot.core.indexsegment.immutable.ImmutableSegment;
 import com.linkedin.pinot.core.segment.index.SegmentMetadataImpl;
 import java.util.List;
 import javax.ws.rs.core.Response;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.json.JSONObject;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
 
 public class TablesResourceTest extends BaseResourceTest {
-  private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
 
   @Test
   public void getTables() throws Exception {
@@ -40,7 +39,7 @@ public class TablesResourceTest extends BaseResourceTest {
 
     Response response = _webTarget.path(tablesPath).request().get(Response.class);
     String responseBody = response.readEntity(String.class);
-    TablesList tablesList = OBJECT_MAPPER.readValue(responseBody, TablesList.class);
+    TablesList tablesList = JsonUtils.stringToObject(responseBody, TablesList.class);
 
     Assert.assertNotNull(tablesList);
     List<String> tables = tablesList.getTables();
@@ -52,7 +51,7 @@ public class TablesResourceTest extends BaseResourceTest {
     addTable(secondTable);
     response = _webTarget.path(tablesPath).request().get(Response.class);
     responseBody = response.readEntity(String.class);
-    tablesList = OBJECT_MAPPER.readValue(responseBody, TablesList.class);
+    tablesList = JsonUtils.stringToObject(responseBody, TablesList.class);
 
     Assert.assertNotNull(tablesList);
     tables = tablesList.getTables();
@@ -94,32 +93,32 @@ public class TablesResourceTest extends BaseResourceTest {
     IndexSegment defaultSegment = _indexSegments.get(0);
     String segmentMetadataPath = "/tables/" + TABLE_NAME + "/segments/" + defaultSegment.getSegmentName() + "/metadata";
 
-    JSONObject jsonResponse = new JSONObject(_webTarget.path(segmentMetadataPath).request().get(String.class));
+    JsonNode jsonResponse = JsonUtils.stringToJsonNode(_webTarget.path(segmentMetadataPath).request().get(String.class));
     SegmentMetadataImpl segmentMetadata = (SegmentMetadataImpl) defaultSegment.getSegmentMetadata();
-    Assert.assertEquals(jsonResponse.getString("segmentName"), segmentMetadata.getName());
-    Assert.assertEquals(jsonResponse.get("crc").toString(), segmentMetadata.getCrc());
-    Assert.assertEquals(jsonResponse.getLong("creationTimeMillis"), segmentMetadata.getIndexCreationTime());
-    Assert.assertEquals(jsonResponse.getString("paddingCharacter"),
+    Assert.assertEquals(jsonResponse.get("segmentName").asText(), segmentMetadata.getName());
+    Assert.assertEquals(jsonResponse.get("crc").asText(), segmentMetadata.getCrc());
+    Assert.assertEquals(jsonResponse.get("creationTimeMillis").asLong(), segmentMetadata.getIndexCreationTime());
+    Assert.assertEquals(jsonResponse.get("paddingCharacter").asText(),
         String.valueOf(segmentMetadata.getPaddingCharacter()));
-    Assert.assertEquals(jsonResponse.getLong("refreshTimeMillis"), segmentMetadata.getRefreshTime());
-    Assert.assertEquals(jsonResponse.getLong("pushTimeMillis"), segmentMetadata.getPushTime());
+    Assert.assertEquals(jsonResponse.get("refreshTimeMillis").asLong(), segmentMetadata.getRefreshTime());
+    Assert.assertEquals(jsonResponse.get("pushTimeMillis").asLong(), segmentMetadata.getPushTime());
     Assert.assertTrue(jsonResponse.has("pushTimeReadable"));
     Assert.assertTrue(jsonResponse.has("refreshTimeReadable"));
     Assert.assertTrue(jsonResponse.has("startTimeReadable"));
     Assert.assertTrue(jsonResponse.has("endTimeReadable"));
     Assert.assertTrue(jsonResponse.has("creationTimeReadable"));
-    Assert.assertEquals(jsonResponse.getJSONArray("columns").length(), 0);
+    Assert.assertEquals(jsonResponse.get("columns").size(), 0);
 
-    jsonResponse = new JSONObject(_webTarget.path(segmentMetadataPath)
+    jsonResponse = JsonUtils.stringToJsonNode(_webTarget.path(segmentMetadataPath)
         .queryParam("columns", "column1")
         .queryParam("columns", "column2")
         .request()
         .get(String.class));
-    Assert.assertEquals(jsonResponse.getJSONArray("columns").length(), 2);
+    Assert.assertEquals(jsonResponse.get("columns").size(), 2);
 
-    jsonResponse =
-        new JSONObject(_webTarget.path(segmentMetadataPath).queryParam("columns", "*").request().get(String.class));
-    Assert.assertEquals(jsonResponse.getJSONArray("columns").length(), segmentMetadata.getAllColumns().size());
+    jsonResponse = JsonUtils.stringToJsonNode(
+        (_webTarget.path(segmentMetadataPath).queryParam("columns", "*").request().get(String.class)));
+    Assert.assertEquals(jsonResponse.get("columns").size(), segmentMetadata.getAllColumns().size());
 
     Response response = _webTarget.path("/tables/UNKNOWN_TABLE/segments/" + defaultSegment.getSegmentName())
         .request()
@@ -139,13 +138,13 @@ public class TablesResourceTest extends BaseResourceTest {
 
     // Trigger crc api to fetch crc information
     String response = _webTarget.path(segmentsCrcPath).request().get(String.class);
-    JSONObject segmentsCrc = new JSONObject(response);
+    JsonNode segmentsCrc = JsonUtils.stringToJsonNode(response);
 
     // Check that crc info is correct
     for (ImmutableSegment immutableSegment : immutableSegments) {
       String segmentName = immutableSegment.getSegmentName();
       String crc = immutableSegment.getSegmentMetadata().getCrc();
-      Assert.assertEquals(segmentsCrc.getString(segmentName), crc);
+      Assert.assertEquals(segmentsCrc.get(segmentName).asText(), crc);
     }
   }
 }
diff --git a/pinot-tools/src/main/java/com/linkedin/pinot/tools/PinotZKChanger.java b/pinot-tools/src/main/java/com/linkedin/pinot/tools/PinotZKChanger.java
index 5a28988..102c1bd 100644
--- a/pinot-tools/src/main/java/com/linkedin/pinot/tools/PinotZKChanger.java
+++ b/pinot-tools/src/main/java/com/linkedin/pinot/tools/PinotZKChanger.java
@@ -18,7 +18,7 @@
  */
 package com.linkedin.pinot.tools;
 
-import java.io.StringWriter;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -37,7 +37,6 @@ import org.apache.helix.model.IdealState;
 import org.apache.helix.store.zk.ZkHelixPropertyStore;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import com.fasterxml.jackson.databind.ObjectMapper;
 
 
 /**
@@ -50,7 +49,6 @@ public class PinotZKChanger {
   protected HelixManager helixManager;
   protected String clusterName;
   protected ZkHelixPropertyStore<ZNRecord> propertyStore;
-  protected ObjectMapper objectMapper;
 
   public PinotZKChanger(String zkAddress, String clusterName) {
     this.clusterName = clusterName;
@@ -66,7 +64,6 @@ public class PinotZKChanger {
     ZNRecordSerializer serializer = new ZNRecordSerializer();
     String path = PropertyPathConfig.getPath(PropertyType.PROPERTYSTORE, clusterName);
     propertyStore = new ZkHelixPropertyStore<>(zkAddress, serializer, path);
-    objectMapper = new ObjectMapper();
   }
 
   public ZKHelixAdmin getHelixAdmin() {
@@ -121,15 +118,13 @@ public class PinotZKChanger {
   }
 
   protected void printSegmentAssignment(Map<String, Map<String, String>> mapping) throws Exception {
-    StringWriter sw = new StringWriter();
-    objectMapper.writerWithDefaultPrettyPrinter().writeValue(sw, mapping);
-    LOGGER.info(sw.toString());
+    LOGGER.info(JsonUtils.objectToPrettyString(mapping));
     Map<String, List<String>> serverToSegmentMapping = new TreeMap<>();
     for (String segment : mapping.keySet()) {
       Map<String, String> serverToStateMap = mapping.get(segment);
       for (String server : serverToStateMap.keySet()) {
         if (!serverToSegmentMapping.containsKey(server)) {
-          serverToSegmentMapping.put(server, new ArrayList<String>());
+          serverToSegmentMapping.put(server, new ArrayList<>());
         }
         serverToSegmentMapping.get(server).add(segment);
       }
diff --git a/pinot-tools/src/main/java/com/linkedin/pinot/tools/Quickstart.java b/pinot-tools/src/main/java/com/linkedin/pinot/tools/Quickstart.java
index 5c01374..4d13e99 100644
--- a/pinot-tools/src/main/java/com/linkedin/pinot/tools/Quickstart.java
+++ b/pinot-tools/src/main/java/com/linkedin/pinot/tools/Quickstart.java
@@ -18,6 +18,7 @@
  */
 package com.linkedin.pinot.tools;
 
+import com.fasterxml.jackson.databind.JsonNode;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.linkedin.pinot.core.data.readers.FileFormat;
@@ -29,9 +30,6 @@ import org.apache.commons.io.FileUtils;
 import org.apache.log4j.Category;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
-import org.json.JSONArray;
-import org.json.JSONException;
-import org.json.JSONObject;
 
 
 public class Quickstart {
@@ -58,24 +56,23 @@ public class Quickstart {
     System.out.println(color._code + message + Color.RESET._code);
   }
 
-  public static String prettyPrintResponse(JSONObject response)
-      throws JSONException {
+  public static String prettyPrintResponse(JsonNode response) {
     StringBuilder responseBuilder = new StringBuilder();
 
     // Selection query
     if (response.has("selectionResults")) {
-      JSONArray columns = response.getJSONObject("selectionResults").getJSONArray("columns");
-      int numColumns = columns.length();
+      JsonNode columns = response.get("selectionResults").get("columns");
+      int numColumns = columns.size();
       for (int i = 0; i < numColumns; i++) {
-        responseBuilder.append(columns.getString(i)).append(TAB);
+        responseBuilder.append(columns.get(i).asText()).append(TAB);
       }
       responseBuilder.append(NEW_LINE);
-      JSONArray rows = response.getJSONObject("selectionResults").getJSONArray("results");
-      int numRows = rows.length();
+      JsonNode rows = response.get("selectionResults").get("results");
+      int numRows = rows.size();
       for (int i = 0; i < numRows; i++) {
-        JSONArray row = rows.getJSONArray(i);
+        JsonNode row = rows.get(i);
         for (int j = 0; j < numColumns; j++) {
-          responseBuilder.append(row.getString(j)).append(TAB);
+          responseBuilder.append(row.get(j).asText()).append(TAB);
         }
         responseBuilder.append(NEW_LINE);
       }
@@ -83,40 +80,40 @@ public class Quickstart {
     }
 
     // Aggregation only query
-    if (!response.getJSONArray("aggregationResults").getJSONObject(0).has("groupByResult")) {
-      JSONArray aggregationResults = response.getJSONArray("aggregationResults");
-      int numAggregations = aggregationResults.length();
+    if (!response.get("aggregationResults").get(0).has("groupByResult")) {
+      JsonNode aggregationResults = response.get("aggregationResults");
+      int numAggregations = aggregationResults.size();
       for (int i = 0; i < numAggregations; i++) {
-        responseBuilder.append(aggregationResults.getJSONObject(i).getString("function")).append(TAB);
+        responseBuilder.append(aggregationResults.get(i).get("function").asText()).append(TAB);
       }
       responseBuilder.append(NEW_LINE);
       for (int i = 0; i < numAggregations; i++) {
-        responseBuilder.append(aggregationResults.getJSONObject(i).getString("value")).append(TAB);
+        responseBuilder.append(aggregationResults.get(i).get("value").asText()).append(TAB);
       }
       responseBuilder.append(NEW_LINE);
       return responseBuilder.toString();
     }
 
     // Aggregation group-by query
-    JSONArray groupByResults = response.getJSONArray("aggregationResults");
-    int numGroupBys = groupByResults.length();
+    JsonNode groupByResults = response.get("aggregationResults");
+    int numGroupBys = groupByResults.size();
     for (int i = 0; i < numGroupBys; i++) {
-      JSONObject groupByResult = groupByResults.getJSONObject(i);
-      responseBuilder.append(groupByResult.getString("function")).append(TAB);
-      JSONArray columns = groupByResult.getJSONArray("groupByColumns");
-      int numColumns = columns.length();
+      JsonNode groupByResult = groupByResults.get(i);
+      responseBuilder.append(groupByResult.get("function").asText()).append(TAB);
+      JsonNode columns = groupByResult.get("groupByColumns");
+      int numColumns = columns.size();
       for (int j = 0; j < numColumns; j++) {
-        responseBuilder.append(columns.getString(j)).append(TAB);
+        responseBuilder.append(columns.get(j).asText()).append(TAB);
       }
       responseBuilder.append(NEW_LINE);
-      JSONArray rows = groupByResult.getJSONArray("groupByResult");
-      int numRows = rows.length();
+      JsonNode rows = groupByResult.get("groupByResult");
+      int numRows = rows.size();
       for (int j = 0; j < numRows; j++) {
-        JSONObject row = rows.getJSONObject(j);
-        responseBuilder.append(row.getString("value")).append(TAB);
-        JSONArray columnValues = row.getJSONArray("group");
+        JsonNode row = rows.get(j);
+        responseBuilder.append(row.get("value").asText()).append(TAB);
+        JsonNode columnValues = row.get("group");
         for (int k = 0; k < numColumns; k++) {
-          responseBuilder.append(columnValues.getString(k)).append(TAB);
+          responseBuilder.append(columnValues.get(k).asText()).append(TAB);
         }
         responseBuilder.append(NEW_LINE);
       }
diff --git a/pinot-tools/src/main/java/com/linkedin/pinot/tools/StarTreeIndexViewer.java b/pinot-tools/src/main/java/com/linkedin/pinot/tools/StarTreeIndexViewer.java
index 847b074..f68ce46 100644
--- a/pinot-tools/src/main/java/com/linkedin/pinot/tools/StarTreeIndexViewer.java
+++ b/pinot-tools/src/main/java/com/linkedin/pinot/tools/StarTreeIndexViewer.java
@@ -18,8 +18,10 @@
  */
 package com.linkedin.pinot.tools;
 
+import com.fasterxml.jackson.annotation.JsonInclude;
 import com.google.common.collect.MinMaxPriorityQueue;
 import com.linkedin.pinot.common.segment.ReadMode;
+import com.linkedin.pinot.common.utils.JsonUtils;
 import com.linkedin.pinot.core.common.Block;
 import com.linkedin.pinot.core.common.BlockSingleValIterator;
 import com.linkedin.pinot.core.common.BlockValSet;
@@ -48,8 +50,6 @@ import javax.ws.rs.Produces;
 import javax.ws.rs.core.MediaType;
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.commons.lang3.tuple.Pair;
-import org.codehaus.jackson.map.ObjectMapper;
... 1928 lines suppressed ...


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