You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by dz...@apache.org on 2023/05/15 09:03:59 UTC

[drill] branch master updated: DRILL-8430: Create JacksonUtils (#2800)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 111e0e8c1b DRILL-8430: Create JacksonUtils (#2800)
111e0e8c1b is described below

commit 111e0e8c1bc470036ba52a84a7022f6d3c1d71ca
Author: PJ Fanning <pj...@users.noreply.github.com>
AuthorDate: Mon May 15 10:03:51 2023 +0100

    DRILL-8430: Create JacksonUtils (#2800)
---
 .../apache/drill/common/scanner/BuildTimeScan.java |  3 +-
 .../org/apache/drill/common/util/JacksonUtils.java | 64 ++++++++++++++++++++++
 .../test/java/org/apache/drill/test/DrillTest.java |  4 +-
 .../apache/drill/exec/store/log/TestLogReader.java |  3 +-
 .../store/pcap/TestPcapWithPersistentStore.java    |  8 ++-
 .../store/drill/plugin/DrillStoragePlugin.java     |  3 +-
 .../drill/exec/store/druid/DruidRecordReader.java  |  3 +-
 .../drill/exec/store/druid/DruidStoragePlugin.java |  3 +-
 .../exec/store/druid/common/DruidFilterBase.java   |  3 +-
 .../exec/store/druid/druid/ScanQueryBuilder.java   |  3 +-
 .../exec/store/druid/rest/DruidAdminClient.java    |  3 +-
 .../exec/store/druid/rest/DruidQueryClient.java    |  3 +-
 .../store/druid/DruidStoragePluginConfigTest.java  |  5 +-
 .../drill/exec/store/druid/DruidTestSuite.java     |  3 +-
 .../drill/exec/store/druid/TestDataGenerator.java  |  3 +-
 .../elasticsearch/ElasticsearchStorageConfig.java  |  4 +-
 .../TestGoogleSheetsLimitPushdown.java             |  3 +-
 .../googlesheets/TestGoogleSheetsQueries.java      |  3 +-
 .../store/googlesheets/TestGoogleSheetsWriter.java |  3 +-
 .../drill/exec/store/hbase/HBaseStoragePlugin.java |  5 +-
 .../drill/exec/store/hive/HiveStoragePlugin.java   |  3 +-
 .../drill/exec/store/kafka/KafkaStoragePlugin.java |  3 +-
 .../drill/exec/store/kudu/KuduStoragePlugin.java   |  3 +-
 .../drill/exec/store/mongo/MongoStoragePlugin.java |  3 +-
 .../exec/store/openTSDB/OpenTSDBStoragePlugin.java |  3 +-
 .../org/apache/drill/exec/client/DrillClient.java  |  3 +-
 .../expr/fn/registry/RemoteFunctionRegistry.java   |  3 +-
 .../impl/statistics/AvgWidthMergedStatistic.java   | 10 ++--
 .../impl/statistics/NDVMergedStatistic.java        |  8 ++-
 .../drill/exec/planner/common/DrillStatsTable.java |  6 +-
 .../exec/planner/common/DrillValuesRelBase.java    |  3 +-
 .../sql/handlers/DescribeSchemaHandler.java        |  6 +-
 .../record/metadata/schema/PathSchemaProvider.java |  4 +-
 .../drill/exec/rpc/control/ControlTunnel.java      |  8 ++-
 .../exec/rpc/user/InboundImpersonationManager.java |  3 +-
 .../exec/serialization/JacksonSerializer.java      |  7 +++
 .../exec/server/options/PersistedOptionValue.java  |  4 +-
 .../exec/server/rest/profile/ProfileWrapper.java   |  3 +-
 .../exec/store/DrillbitPluginRegistryContext.java  |  3 +-
 .../easy/json/parser/JsonStructureParser.java      |  3 +-
 .../store/easy/json/reader/BaseJsonProcessor.java  |  3 +-
 .../drill/exec/store/http/oauth/OAuthUtils.java    |  5 +-
 .../drill/exec/store/mock/MockStorageEngine.java   |  5 +-
 .../exec/store/parquet/metadata/Metadata.java      | 15 +++--
 .../exec/store/sys/PersistentStoreConfig.java      |  4 ++
 .../drill/exec/testing/ExecutionControls.java      |  3 +-
 .../org/apache/drill/exec/work/QueryWorkUnit.java  | 11 ++--
 .../apache/drill/exec/work/foreman/Foreman.java    |  3 +-
 .../apache/drill/exec/TestPathSerialization.java   | 12 ++--
 .../server/options/PersistedOptionValueTest.java   |  7 +--
 .../drill/exec/server/rest/TestRestJson.java       |  3 +-
 .../org/apache/drill/exec/sql/TestInfoSchema.java  |  3 +-
 .../store/parquet/TestParquetReaderConfig.java     |  3 +-
 .../drill/exec/store/sys/PStoreTestUtil.java       |  4 +-
 .../exec/store/sys/TestLocalPersistentStore.java   |  3 +-
 .../drill/exec/store/sys/TestPStoreProviders.java  |  3 +-
 .../exec/vector/complex/writer/TestRepeated.java   |  5 --
 .../apache/drill/test/PhysicalOpUnitTestBase.java  |  3 +-
 .../drill/exec/record/metadata/TupleMetadata.java  |  6 +-
 .../drill/exec/util/JsonStringArrayList.java       | 10 ++--
 .../apache/drill/exec/util/JsonStringHashMap.java  |  3 +-
 .../java/org/apache/drill/common/JSONOptions.java  |  6 ++
 .../common/config/LogicalPlanPersistence.java      |  3 +-
 .../drill/common/util/AbstractDynamicBean.java     |  4 +-
 .../metastore/statistics/ColumnStatistics.java     | 13 +++--
 .../metastore/statistics/StatisticsHolder.java     |  6 +-
 .../drill/metastore/rdbms/util/ConverterUtil.java  |  3 +-
 67 files changed, 244 insertions(+), 126 deletions(-)

diff --git a/common/src/main/java/org/apache/drill/common/scanner/BuildTimeScan.java b/common/src/main/java/org/apache/drill/common/scanner/BuildTimeScan.java
index 794983e434..4c65eb3528 100644
--- a/common/src/main/java/org/apache/drill/common/scanner/BuildTimeScan.java
+++ b/common/src/main/java/org/apache/drill/common/scanner/BuildTimeScan.java
@@ -32,6 +32,7 @@ import org.apache.drill.common.config.ConfigConstants;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.scanner.persistence.ScanResult;
+import org.apache.drill.common.util.JacksonUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -47,7 +48,7 @@ public class BuildTimeScan {
   private static final Logger logger = LoggerFactory.getLogger(BuildTimeScan.class);
   private static final String REGISTRY_FILE = "META-INF/drill-module-scan/registry.json";
 
-  private static final ObjectMapper mapper = new ObjectMapper().enable(INDENT_OUTPUT);
+  private static final ObjectMapper mapper = JacksonUtils.createObjectMapper().enable(INDENT_OUTPUT);
   private static final ObjectReader reader = mapper.readerFor(ScanResult.class);
   private static final ObjectWriter writer = mapper.writerFor(ScanResult.class);
 
diff --git a/common/src/main/java/org/apache/drill/common/util/JacksonUtils.java b/common/src/main/java/org/apache/drill/common/util/JacksonUtils.java
new file mode 100644
index 0000000000..e0cb0dee80
--- /dev/null
+++ b/common/src/main/java/org/apache/drill/common/util/JacksonUtils.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.common.util;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+
+/**
+ * Utility class which contain methods for interacting with Jackson.
+ */
+public final class JacksonUtils {
+
+  private JacksonUtils() {}
+
+  /**
+   * Creates a new instance of the Jackson {@link ObjectMapper}.
+   * @return an {@link ObjectMapper} instance
+   */
+  public static ObjectMapper createObjectMapper() {
+    return createJsonMapperBuilder().build();
+  }
+
+  /**
+   * Creates a new instance of the Jackson {@link ObjectMapper}.
+   * @param factory a {@link JsonFactory} instance
+   * @return an {@link ObjectMapper} instance
+   */
+  public static ObjectMapper createObjectMapper(final JsonFactory factory) {
+    return createJsonMapperBuilder(factory).build();
+  }
+
+  /**
+   * Creates a new instance of the Jackson {@link JsonMapper.Builder}.
+   * @return an {@link JsonMapper.Builder} instance
+   */
+  public static JsonMapper.Builder createJsonMapperBuilder() {
+    return JsonMapper.builder();
+  }
+
+  /**
+   * Creates a new instance of the Jackson {@link JsonMapper.Builder}.
+   * @param factory a {@link JsonFactory} instance
+   * @return an {@link JsonMapper.Builder} instance
+   */
+  public static JsonMapper.Builder createJsonMapperBuilder(final JsonFactory factory) {
+    return JsonMapper.builder(factory);
+  }
+}
diff --git a/common/src/test/java/org/apache/drill/test/DrillTest.java b/common/src/test/java/org/apache/drill/test/DrillTest.java
index f009d2a5f0..19efad6010 100644
--- a/common/src/test/java/org/apache/drill/test/DrillTest.java
+++ b/common/src/test/java/org/apache/drill/test/DrillTest.java
@@ -23,6 +23,7 @@ import java.lang.management.MemoryMXBean;
 import java.util.List;
 
 import org.apache.drill.common.util.DrillStringUtils;
+import org.apache.drill.common.util.JacksonUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -37,11 +38,10 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 
 public class DrillTest extends BaseTest {
 
-  protected static final ObjectMapper objectMapper;
+  private static final ObjectMapper objectMapper = JacksonUtils.createObjectMapper();
 
   static {
     System.setProperty("line.separator", "\n");
-    objectMapper = new ObjectMapper();
   }
 
   static final SystemManager manager = new SystemManager();
diff --git a/contrib/format-log/src/test/java/org/apache/drill/exec/store/log/TestLogReader.java b/contrib/format-log/src/test/java/org/apache/drill/exec/store/log/TestLogReader.java
index 02f5d8209c..7d55f59cd9 100644
--- a/contrib/format-log/src/test/java/org/apache/drill/exec/store/log/TestLogReader.java
+++ b/contrib/format-log/src/test/java/org/apache/drill/exec/store/log/TestLogReader.java
@@ -29,6 +29,7 @@ import org.apache.drill.categories.RowSetTest;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.record.metadata.SchemaBuilder;
 import org.apache.drill.exec.record.metadata.TupleMetadata;
@@ -735,7 +736,7 @@ public class TestLogReader extends ClusterTest {
    */
   @Test
   public void testPluginSerialization() throws IOException {
-    ObjectMapper mapper = new ObjectMapper();
+    ObjectMapper mapper = JacksonUtils.createObjectMapper();
     assertTrue(mapper.canSerialize(LogFormatPlugin.class));
 
     LogFormatConfig sampleConfig = dateOnlyConfig();
diff --git a/contrib/format-pcapng/src/test/java/org/apache/drill/exec/persistent/store/pcap/TestPcapWithPersistentStore.java b/contrib/format-pcapng/src/test/java/org/apache/drill/exec/persistent/store/pcap/TestPcapWithPersistentStore.java
index 9fee14a31e..b44d297cd7 100644
--- a/contrib/format-pcapng/src/test/java/org/apache/drill/exec/persistent/store/pcap/TestPcapWithPersistentStore.java
+++ b/contrib/format-pcapng/src/test/java/org/apache/drill/exec/persistent/store/pcap/TestPcapWithPersistentStore.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.persistent.store.pcap;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
 import org.apache.curator.framework.CuratorFramework;
@@ -43,6 +44,7 @@ import java.util.Map;
 import static org.junit.Assert.assertTrue;
 
 public class TestPcapWithPersistentStore extends TestWithZookeeper {
+
     /**
      * DRILL-7828
      * Note: If this test breaks you are probably breaking backward and forward compatibility. Verify with the community
@@ -54,10 +56,10 @@ public class TestPcapWithPersistentStore extends TestWithZookeeper {
 
         try (CuratorFramework curator = createCurator()) {
             curator.start();
-            ObjectMapper objectMapper = new ObjectMapper();
-            objectMapper.registerSubtypes(PcapFormatConfig.class, PcapngFormatConfig.class);
+            ObjectMapper testMapper = JacksonUtils.createObjectMapper();
+            testMapper.registerSubtypes(PcapFormatConfig.class, PcapngFormatConfig.class);
             PersistentStoreConfig<FileSystemConfig> storeConfig =
-                    PersistentStoreConfig.newJacksonBuilder(objectMapper, FileSystemConfig.class).name("type").build();
+                    PersistentStoreConfig.newJacksonBuilder(testMapper, FileSystemConfig.class).name("type").build();
 
 
             try (ZookeeperClient zkClient = new ZookeeperClient(curator,
diff --git a/contrib/storage-drill/src/main/java/org/apache/drill/exec/store/drill/plugin/DrillStoragePlugin.java b/contrib/storage-drill/src/main/java/org/apache/drill/exec/store/drill/plugin/DrillStoragePlugin.java
index b4c77ef5cc..ff5e6bdddd 100644
--- a/contrib/storage-drill/src/main/java/org/apache/drill/exec/store/drill/plugin/DrillStoragePlugin.java
+++ b/contrib/storage-drill/src/main/java/org/apache/drill/exec/store/drill/plugin/DrillStoragePlugin.java
@@ -18,7 +18,6 @@
 package org.apache.drill.exec.store.drill.plugin;
 
 import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.schema.SchemaPlus;
@@ -96,7 +95,7 @@ public class DrillStoragePlugin extends AbstractStoragePlugin {
 
   @Override
   public AbstractGroupScan getPhysicalScan(String userName, JSONOptions selection) throws IOException {
-    DrillScanSpec scanSpec = selection.getListWith(new ObjectMapper(), new TypeReference<DrillScanSpec>() {
+    DrillScanSpec scanSpec = selection.getListWith(new TypeReference<DrillScanSpec>() {
     });
     return new DrillGroupScan(userName, drillConfig, scanSpec);
   }
diff --git a/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/DruidRecordReader.java b/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/DruidRecordReader.java
index 6ae735b754..69ed0b9ead 100755
--- a/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/DruidRecordReader.java
+++ b/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/DruidRecordReader.java
@@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ObjectNode;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.ops.OperatorContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
@@ -51,7 +52,7 @@ import java.util.concurrent.TimeUnit;
 public class DruidRecordReader extends AbstractRecordReader {
 
   private static final Logger logger = LoggerFactory.getLogger(DruidRecordReader.class);
-  private static final ObjectMapper objectMapper = new ObjectMapper();
+  private static final ObjectMapper objectMapper = JacksonUtils.createObjectMapper();
   private final DruidStoragePlugin plugin;
   private final DruidSubScan.DruidSubScanSpec scanSpec;
   private final List<String> columns;
diff --git a/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/DruidStoragePlugin.java b/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/DruidStoragePlugin.java
index 45ee0a2b32..b7479fb632 100755
--- a/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/DruidStoragePlugin.java
+++ b/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/DruidStoragePlugin.java
@@ -18,7 +18,6 @@
 package org.apache.drill.exec.store.druid;
 
 import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
@@ -59,7 +58,7 @@ public class DruidStoragePlugin extends AbstractStoragePlugin {
 
   @Override
   public DruidGroupScan getPhysicalScan(String userName, JSONOptions selection) throws IOException {
-    DruidScanSpec scanSpec = selection.getListWith(new ObjectMapper(), new TypeReference<DruidScanSpec>() {});
+    DruidScanSpec scanSpec = selection.getListWith(new TypeReference<DruidScanSpec>() {});
     return new DruidGroupScan(userName, this, scanSpec, null, -1);
   }
 
diff --git a/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/common/DruidFilterBase.java b/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/common/DruidFilterBase.java
index 481357bc6e..b5216358f8 100755
--- a/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/common/DruidFilterBase.java
+++ b/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/common/DruidFilterBase.java
@@ -19,9 +19,10 @@ package org.apache.drill.exec.store.druid.common;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.common.util.JacksonUtils;
 
 public class DruidFilterBase implements DruidFilter {
-  private static final ObjectMapper objectMapper = new ObjectMapper();
+  private static final ObjectMapper objectMapper = JacksonUtils.createObjectMapper();
 
   public String toJson() {
     try {
diff --git a/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/druid/ScanQueryBuilder.java b/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/druid/ScanQueryBuilder.java
index df00ca2420..e7e85025b6 100644
--- a/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/druid/ScanQueryBuilder.java
+++ b/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/druid/ScanQueryBuilder.java
@@ -22,6 +22,7 @@ import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.store.druid.common.DruidFilter;
 
 import java.math.BigInteger;
@@ -33,7 +34,7 @@ import java.util.stream.Stream;
 import static org.apache.drill.exec.store.druid.common.DruidConstants.INTERVAL_DIMENSION_NAME;
 
 public class ScanQueryBuilder {
-  private static final ObjectMapper objectMapper = new ObjectMapper();
+  private static final ObjectMapper objectMapper = JacksonUtils.createObjectMapper();
 
   public ScanQueryBuilder() {}
 
diff --git a/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/rest/DruidAdminClient.java b/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/rest/DruidAdminClient.java
index 09c99fc834..4859f3064a 100755
--- a/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/rest/DruidAdminClient.java
+++ b/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/rest/DruidAdminClient.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.store.druid.rest;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.store.druid.druid.SimpleDatasourceInfo;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -34,7 +35,7 @@ public class DruidAdminClient {
   private static final Logger logger = LoggerFactory.getLogger(DruidAdminClient.class);
 
   private static final String DATASOURCES_BASE_URI = "/druid/coordinator/v1/datasources?simple";
-  private static final ObjectMapper mapper = new ObjectMapper();
+  private static final ObjectMapper mapper = JacksonUtils.createObjectMapper();
 
   private final String coordinatorAddress;
   private final RestClient restClient;
diff --git a/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/rest/DruidQueryClient.java b/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/rest/DruidQueryClient.java
index fe82650199..e6ba7718ab 100755
--- a/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/rest/DruidQueryClient.java
+++ b/contrib/storage-druid/src/main/java/org/apache/drill/exec/store/druid/rest/DruidQueryClient.java
@@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.node.ArrayNode;
 import com.fasterxml.jackson.databind.node.ObjectNode;
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.store.druid.druid.DruidScanResponse;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -35,7 +36,7 @@ public class DruidQueryClient {
   private static final Logger logger = LoggerFactory.getLogger(DruidQueryClient.class);
 
   private static final String QUERY_BASE_URI = "/druid/v2";
-  private static final ObjectMapper mapper = new ObjectMapper();
+  private static final ObjectMapper mapper = JacksonUtils.createObjectMapper();
 
   private final RestClient restClient;
   private final String queryUrl;
diff --git a/contrib/storage-druid/src/test/java/org/apache/drill/exec/store/druid/DruidStoragePluginConfigTest.java b/contrib/storage-druid/src/test/java/org/apache/drill/exec/store/druid/DruidStoragePluginConfigTest.java
index 354b23c5e5..3e3f5495d0 100644
--- a/contrib/storage-druid/src/test/java/org/apache/drill/exec/store/druid/DruidStoragePluginConfigTest.java
+++ b/contrib/storage-druid/src/test/java/org/apache/drill/exec/store/druid/DruidStoragePluginConfigTest.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.store.druid;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.shaded.guava.com.google.common.io.Resources;
 import org.junit.Test;
 
@@ -34,7 +35,7 @@ public class DruidStoragePluginConfigTest {
   @Test
   public void testDruidStoragePluginConfigSuccessfullyParsed()
       throws URISyntaxException, IOException {
-    ObjectMapper mapper = new ObjectMapper();
+    ObjectMapper mapper = JacksonUtils.createObjectMapper();
     JsonNode storagePluginJson = mapper.readTree(new File(
         Resources.getResource("bootstrap-storage-plugins.json").toURI()));
     DruidStoragePluginConfig druidStoragePluginConfig =
@@ -54,7 +55,7 @@ public class DruidStoragePluginConfigTest {
         + "      \"brokerAddress\" : \"http://localhost:8082\",\n"
         + "      \"coordinatorAddress\": \"http://localhost:8081\",\n"
         + "      \"enabled\" : false\n" + "    }\n" + "  }\n" + "}\n";
-    ObjectMapper mapper = new ObjectMapper();
+    ObjectMapper mapper = JacksonUtils.createObjectMapper();
     JsonNode storagePluginJson = mapper.readTree(druidConfigStr);
     DruidStoragePluginConfig druidStoragePluginConfig =
         mapper.treeToValue(storagePluginJson.get("storage").get("druid"), DruidStoragePluginConfig.class);
diff --git a/contrib/storage-druid/src/test/java/org/apache/drill/exec/store/druid/DruidTestSuite.java b/contrib/storage-druid/src/test/java/org/apache/drill/exec/store/druid/DruidTestSuite.java
index 92d3c3486a..d35f0c1ee5 100644
--- a/contrib/storage-druid/src/test/java/org/apache/drill/exec/store/druid/DruidTestSuite.java
+++ b/contrib/storage-druid/src/test/java/org/apache/drill/exec/store/druid/DruidTestSuite.java
@@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.drill.categories.DruidStorageTest;
 import org.apache.drill.categories.SlowTest;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.store.druid.rest.DruidQueryClientTest;
 import org.apache.drill.shaded.guava.com.google.common.io.Resources;
 import org.junit.BeforeClass;
@@ -46,7 +47,7 @@ import java.io.File;
 public class DruidTestSuite {
   private static final Logger logger = LoggerFactory.getLogger(DruidTestSuite.class);
 
-  private static final ObjectMapper mapper = new ObjectMapper();
+  private static final ObjectMapper mapper = JacksonUtils.createObjectMapper();
 
   private static DruidStoragePluginConfig druidStoragePluginConfig = null;
 
diff --git a/contrib/storage-druid/src/test/java/org/apache/drill/exec/store/druid/TestDataGenerator.java b/contrib/storage-druid/src/test/java/org/apache/drill/exec/store/druid/TestDataGenerator.java
index 3b96db7cdb..2d009f7787 100644
--- a/contrib/storage-druid/src/test/java/org/apache/drill/exec/store/druid/TestDataGenerator.java
+++ b/contrib/storage-druid/src/test/java/org/apache/drill/exec/store/druid/TestDataGenerator.java
@@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.shaded.guava.com.google.common.io.Resources;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -43,7 +44,7 @@ public class TestDataGenerator {
 
   private static final OkHttpClient httpClient = new OkHttpClient();
 
-  private static final ObjectMapper mapper = new ObjectMapper();
+  private static final ObjectMapper mapper = JacksonUtils.createObjectMapper();
 
   private static final String RESPONSE_SUCCESS = "SUCCESS";
 
diff --git a/contrib/storage-elasticsearch/src/main/java/org/apache/drill/exec/store/elasticsearch/ElasticsearchStorageConfig.java b/contrib/storage-elasticsearch/src/main/java/org/apache/drill/exec/store/elasticsearch/ElasticsearchStorageConfig.java
index 8244505c9e..23be77f543 100644
--- a/contrib/storage-elasticsearch/src/main/java/org/apache/drill/exec/store/elasticsearch/ElasticsearchStorageConfig.java
+++ b/contrib/storage-elasticsearch/src/main/java/org/apache/drill/exec/store/elasticsearch/ElasticsearchStorageConfig.java
@@ -24,11 +24,11 @@ import com.fasterxml.jackson.annotation.JsonInclude.Include;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonTypeName;
 import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectWriter;
 import org.apache.drill.common.PlanStringBuilder;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.common.logical.security.PlainCredentialsProvider;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.store.security.CredentialProviderUtils;
 import org.apache.drill.common.logical.security.CredentialsProvider;
 import org.apache.drill.exec.store.security.UsernamePasswordCredentials;
@@ -45,7 +45,7 @@ import java.util.Optional;
 public class ElasticsearchStorageConfig extends StoragePluginConfig {
   public static final String NAME = "elastic";
 
-  private static final ObjectWriter OBJECT_WRITER = new ObjectMapper().writerFor(List.class);
+  private static final ObjectWriter OBJECT_WRITER = JacksonUtils.createObjectMapper().writerFor(List.class);
 
   private static final String HOSTS = "hosts";
 
diff --git a/contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsLimitPushdown.java b/contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsLimitPushdown.java
index b6442856da..0b6f95b037 100644
--- a/contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsLimitPushdown.java
+++ b/contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsLimitPushdown.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.store.googlesheets;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.drill.common.util.DrillFileUtils;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.oauth.PersistentTokenTable;
 import org.apache.drill.exec.store.StoragePluginRegistry;
@@ -56,7 +57,7 @@ public class TestGoogleSheetsLimitPushdown extends ClusterTest {
 
     String oauthJson = Files.asCharSource(DrillFileUtils.getResourceAsFile("/tokens/oauth_tokens.json"), Charsets.UTF_8).read();
 
-    ObjectMapper mapper = new ObjectMapper();
+    ObjectMapper mapper = JacksonUtils.createObjectMapper();
     Map<String,String> tokenMap = mapper.readValue(oauthJson, Map.class);
 
     String clientID = tokenMap.get("client_id");
diff --git a/contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsQueries.java b/contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsQueries.java
index 2410311910..df72b01efd 100644
--- a/contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsQueries.java
+++ b/contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsQueries.java
@@ -21,6 +21,7 @@ package org.apache.drill.exec.store.googlesheets;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.util.DrillFileUtils;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.oauth.PersistentTokenTable;
 import org.apache.drill.exec.physical.rowSet.RowSet;
@@ -73,7 +74,7 @@ public class TestGoogleSheetsQueries extends ClusterTest {
 
     String oauthJson = Files.asCharSource(DrillFileUtils.getResourceAsFile("/tokens/oauth_tokens.json"), Charsets.UTF_8).read();
 
-    ObjectMapper mapper = new ObjectMapper();
+    ObjectMapper mapper = JacksonUtils.createObjectMapper();
     Map<String,String> tokenMap = mapper.readValue(oauthJson, Map.class);
 
     clientID = tokenMap.get("client_id");
diff --git a/contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsWriter.java b/contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsWriter.java
index 2b06de1e85..d819cce66d 100644
--- a/contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsWriter.java
+++ b/contrib/storage-googlesheets/src/test/java/org/apache/drill/exec/store/googlesheets/TestGoogleSheetsWriter.java
@@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.drill.categories.RowSetTest;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.util.DrillFileUtils;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.oauth.PersistentTokenTable;
 import org.apache.drill.exec.physical.rowSet.RowSet;
 import org.apache.drill.exec.record.metadata.SchemaBuilder;
@@ -75,7 +76,7 @@ public class TestGoogleSheetsWriter extends ClusterTest {
 
     String oauthJson = Files.asCharSource(DrillFileUtils.getResourceAsFile("/tokens/oauth_tokens.json"), Charsets.UTF_8).read();
 
-    ObjectMapper mapper = new ObjectMapper();
+    ObjectMapper mapper = JacksonUtils.createObjectMapper();
     Map<String,String> tokenMap = mapper.readValue(oauthJson, Map.class);
 
     String clientID = tokenMap.get("client_id");
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseStoragePlugin.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseStoragePlugin.java
index 64797a3546..4d1e05ed73 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseStoragePlugin.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseStoragePlugin.java
@@ -29,12 +29,11 @@ import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.store.SchemaConfig;
 import org.apache.drill.exec.store.StoragePluginOptimizerRule;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Connection;
 
 import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
 
 public class HBaseStoragePlugin extends AbstractStoragePlugin {
   private static final HBaseConnectionManager hbaseConnectionManager = HBaseConnectionManager.INSTANCE;
@@ -61,7 +60,7 @@ public class HBaseStoragePlugin extends AbstractStoragePlugin {
 
   @Override
   public HBaseGroupScan getPhysicalScan(String userName, JSONOptions selection) throws IOException {
-    HBaseScanSpec scanSpec = selection.getListWith(new ObjectMapper(), new TypeReference<HBaseScanSpec>() {});
+    HBaseScanSpec scanSpec = selection.getListWith(new TypeReference<HBaseScanSpec>() {});
     return new HBaseGroupScan(userName, this, scanSpec, null);
   }
 
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
index 1ce138c0e3..863f96aa0c 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
@@ -28,7 +28,6 @@ import java.util.function.Function;
 import java.util.stream.Collectors;
 
 import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
 
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.commons.lang3.StringEscapeUtils;
@@ -99,7 +98,7 @@ public class HiveStoragePlugin extends AbstractStoragePlugin {
 
   @Override
   public HiveScan getPhysicalScan(String userName, JSONOptions selection, List<SchemaPath> columns, SessionOptionManager options) throws IOException {
-    HiveReadEntry hiveReadEntry = selection.getListWith(new ObjectMapper(), new TypeReference<HiveReadEntry>(){});
+    HiveReadEntry hiveReadEntry = selection.getListWith(new TypeReference<HiveReadEntry>(){});
     try {
       Map<String, String> confProperties = new HashMap<>();
       if (options != null) {
diff --git a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaStoragePlugin.java b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaStoragePlugin.java
index 7fb53ce3a3..d376bb2fb9 100644
--- a/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaStoragePlugin.java
+++ b/contrib/storage-kafka/src/main/java/org/apache/drill/exec/store/kafka/KafkaStoragePlugin.java
@@ -34,7 +34,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
 
 public class KafkaStoragePlugin extends AbstractStoragePlugin {
@@ -82,7 +81,7 @@ public class KafkaStoragePlugin extends AbstractStoragePlugin {
 
   @Override
   public AbstractGroupScan getPhysicalScan(String userName, JSONOptions selection) throws IOException {
-    KafkaScanSpec kafkaScanSpec = selection.getListWith(new ObjectMapper(),
+    KafkaScanSpec kafkaScanSpec = selection.getListWith(
         new TypeReference<KafkaScanSpec>() {
         });
     return new KafkaGroupScan(this, kafkaScanSpec, null, -1);
diff --git a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduStoragePlugin.java b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduStoragePlugin.java
index fea8197cc4..4b366532ac 100644
--- a/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduStoragePlugin.java
+++ b/contrib/storage-kudu/src/main/java/org/apache/drill/exec/store/kudu/KuduStoragePlugin.java
@@ -27,7 +27,6 @@ import org.apache.drill.exec.store.SchemaConfig;
 import org.apache.kudu.client.KuduClient;
 
 import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
 
 public class KuduStoragePlugin extends AbstractStoragePlugin {
 
@@ -60,7 +59,7 @@ public class KuduStoragePlugin extends AbstractStoragePlugin {
 
   @Override
   public KuduGroupScan getPhysicalScan(String userName, JSONOptions selection) throws IOException {
-    KuduScanSpec scanSpec = selection.getListWith(new ObjectMapper(), new TypeReference<KuduScanSpec>() {});
+    KuduScanSpec scanSpec = selection.getListWith(new TypeReference<KuduScanSpec>() {});
     return new KuduGroupScan(this, scanSpec, null);
   }
 
diff --git a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoStoragePlugin.java b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoStoragePlugin.java
index cb0fb276bf..7b78e24b54 100644
--- a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoStoragePlugin.java
+++ b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoStoragePlugin.java
@@ -18,7 +18,6 @@
 package org.apache.drill.exec.store.mongo;
 
 import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
 import com.mongodb.ConnectionString;
 import com.mongodb.MongoClientSettings;
 import com.mongodb.client.MongoClient;
@@ -155,7 +154,7 @@ public class MongoStoragePlugin extends AbstractStoragePlugin {
 
   @Override
   public AbstractGroupScan getPhysicalScan(String userName, JSONOptions selection) throws IOException {
-    MongoScanSpec mongoScanSpec = selection.getListWith(new ObjectMapper(), new TypeReference<MongoScanSpec>() {
+    MongoScanSpec mongoScanSpec = selection.getListWith(new TypeReference<MongoScanSpec>() {
     });
     return new MongoGroupScan(userName, this, mongoScanSpec, null, false);
   }
diff --git a/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/OpenTSDBStoragePlugin.java b/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/OpenTSDBStoragePlugin.java
index c83a4e5a37..e9ade66962 100644
--- a/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/OpenTSDBStoragePlugin.java
+++ b/contrib/storage-opentsdb/src/main/java/org/apache/drill/exec/store/openTSDB/OpenTSDBStoragePlugin.java
@@ -18,7 +18,6 @@
 package org.apache.drill.exec.store.openTSDB;
 
 import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.exec.server.DrillbitContext;
@@ -56,7 +55,7 @@ public class OpenTSDBStoragePlugin extends AbstractStoragePlugin {
 
   @Override
   public OpenTSDBGroupScan getPhysicalScan(String userName, JSONOptions selection) throws IOException {
-    OpenTSDBScanSpec scanSpec = selection.getListWith(new ObjectMapper(), new TypeReference<OpenTSDBScanSpec>() {
+    OpenTSDBScanSpec scanSpec = selection.getListWith(new TypeReference<OpenTSDBScanSpec>() {
     });
     return new OpenTSDBGroupScan(this, scanSpec, null);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
index 372bbe666f..62d04e3963 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
@@ -42,6 +42,7 @@ import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.config.DrillProperties;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.coord.zk.ZKClusterCoordinator;
@@ -107,7 +108,7 @@ import io.netty.channel.EventLoopGroup;
 public class DrillClient implements Closeable, ConnectionThrottle {
   private static Logger logger = LoggerFactory.getLogger(DrillClient.class);
   public static final String DEFAULT_CLIENT_NAME = "Apache Drill Java client";
-  private static final ObjectMapper objectMapper = new ObjectMapper();
+  private static final ObjectMapper objectMapper = JacksonUtils.createObjectMapper();
 
   private final DrillConfig config;
   private UserClient client;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/RemoteFunctionRegistry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/RemoteFunctionRegistry.java
index 6c84c065d9..076227b5e6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/RemoteFunctionRegistry.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/registry/RemoteFunctionRegistry.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.expr.fn.registry;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
 import org.apache.drill.common.AutoCloseables;
@@ -87,7 +88,7 @@ public class RemoteFunctionRegistry implements AutoCloseable {
 
   private static final String REGISTRY_PATH = "registry";
   private static final Logger logger = LoggerFactory.getLogger(RemoteFunctionRegistry.class);
-  private static final ObjectMapper mapper = new ObjectMapper().enable(INDENT_OUTPUT);
+  private static final ObjectMapper mapper = JacksonUtils.createObjectMapper().enable(INDENT_OUTPUT);
 
   private final TransientStoreListener unregistrationListener;
   private int retryAttempts;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java
index 0b07213553..cd46b0a5ef 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/AvgWidthMergedStatistic.java
@@ -24,6 +24,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.record.MajorTypeSerDe;
 import org.apache.drill.exec.vector.NullableFloat8Vector;
 import org.apache.drill.exec.vector.ValueVector;
@@ -128,10 +129,11 @@ public class AvgWidthMergedStatistic extends AbstractMergedStatistic {
   private long getRowCount(String colName) {
     byte[] typeAsBytes = types.getStat(colName);
     int type  = -1;
-    ObjectMapper mapper = new ObjectMapper();
-    SimpleModule deModule = new SimpleModule("StatisticsSerDeModeule") //
-            .addDeserializer(TypeProtos.MajorType.class, new MajorTypeSerDe.De());
-    mapper.registerModule(deModule);
+    SimpleModule deModule = new SimpleModule("StatisticsSerDeModule") //
+        .addDeserializer(TypeProtos.MajorType.class, new MajorTypeSerDe.De());
+    ObjectMapper mapper = JacksonUtils.createJsonMapperBuilder()
+        .addModule(deModule)
+        .build();
     try {
       type = mapper.readValue(typeAsBytes, TypeProtos.MajorType.class).getMinorType().getNumber();
     } catch (IOException ex) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NDVMergedStatistic.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NDVMergedStatistic.java
index ad77b210d1..f8701c099c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NDVMergedStatistic.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/statistics/NDVMergedStatistic.java
@@ -30,6 +30,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.record.MajorTypeSerDe;
 import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.vector.NullableBigIntVector;
@@ -176,10 +177,11 @@ public class NDVMergedStatistic extends AbstractMergedStatistic {
   private long getRowCount(String colName) {
     byte[] typeAsBytes = types.getStat(colName);
     int type  = -1;
-    ObjectMapper mapper = new ObjectMapper();
     SimpleModule deModule = new SimpleModule("StatisticsSerDeModule") //
-            .addDeserializer(TypeProtos.MajorType.class, new MajorTypeSerDe.De());
-    mapper.registerModule(deModule);
+        .addDeserializer(TypeProtos.MajorType.class, new MajorTypeSerDe.De());
+    ObjectMapper mapper = JacksonUtils.createJsonMapperBuilder()
+        .addModule(deModule)
+        .build();
     try {
       type = mapper.readValue(typeAsBytes, TypeProtos.MajorType.class).getMinorType().getNumber();
     } catch (IOException ex) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java
index 70f3e16d06..bfb58cb514 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStatsTable.java
@@ -41,6 +41,7 @@ import java.util.Set;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.planner.logical.DrillTable;
@@ -465,12 +466,13 @@ public class DrillStatsTable {
    * {@link TableStatistics} from/to JSON
    */
   public static ObjectMapper getMapper() {
-    ObjectMapper mapper = new ObjectMapper();
     SimpleModule deModule = new SimpleModule("StatisticsSerDeModule")
         .addSerializer(TypeProtos.MajorType.class, new MajorTypeSerDe.Se())
         .addDeserializer(TypeProtos.MajorType.class, new MajorTypeSerDe.De())
         .addDeserializer(SchemaPath.class, new SchemaPath.De());
-    mapper.registerModule(deModule);
+    ObjectMapper mapper = JacksonUtils.createJsonMapperBuilder()
+        .addModule(deModule)
+        .build();
     mapper.registerSubtypes(new NamedType(NumericEquiDepthHistogram.class, "numeric-equi-depth"));
     return mapper;
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillValuesRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillValuesRelBase.java
index 5042c432fb..233414c20b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillValuesRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillValuesRelBase.java
@@ -35,6 +35,7 @@ import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.util.NlsString;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.util.GuavaUtils;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.vector.complex.fn.BasicJsonOutput;
 import org.apache.drill.exec.vector.complex.fn.JsonOutput;
 import org.joda.time.DateTime;
@@ -50,7 +51,7 @@ import com.fasterxml.jackson.databind.util.TokenBuffer;
  */
 public abstract class DrillValuesRelBase extends Values implements DrillRelNode {
 
-  private static final ObjectMapper MAPPER = new ObjectMapper();
+  private static final ObjectMapper MAPPER = JacksonUtils.createObjectMapper();
 
   protected final String content;
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeSchemaHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeSchemaHandler.java
index cee39ebece..515d130c91 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeSchemaHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeSchemaHandler.java
@@ -17,10 +17,12 @@
  */
 package org.apache.drill.exec.planner.sql.handlers;
 
+import com.fasterxml.jackson.core.JsonFactory;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.core.SerializableString;
 import com.fasterxml.jackson.core.io.CharacterEscapes;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.store.SchemaFactory;
 import org.apache.drill.shaded.guava.com.google.common.base.Joiner;
 import org.slf4j.Logger;
@@ -50,8 +52,8 @@ public class DescribeSchemaHandler extends DefaultSqlHandler {
   private static final Logger logger = LoggerFactory.getLogger(DescribeSchemaHandler.class);
 
   @SuppressWarnings("serial")
-  private static final ObjectMapper mapper = new ObjectMapper(
-      new ObjectMapper().getFactory().setCharacterEscapes(new CharacterEscapes() {
+  private static final ObjectMapper mapper = JacksonUtils.createObjectMapper(
+      new JsonFactory().setCharacterEscapes(new CharacterEscapes() {
     @Override
     public int[] getEscapeCodesForAscii() {
       // add standard set of escaping characters
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/schema/PathSchemaProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/schema/PathSchemaProvider.java
index fe2d9208f4..d704fd828b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/schema/PathSchemaProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/metadata/schema/PathSchemaProvider.java
@@ -23,6 +23,7 @@ import com.fasterxml.jackson.core.util.DefaultPrettyPrinter;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectReader;
 import com.fasterxml.jackson.databind.ObjectWriter;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.util.ImpersonationUtil;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -52,7 +53,8 @@ public class PathSchemaProvider implements SchemaProvider {
   public static final ObjectWriter WRITER;
 
   static {
-    ObjectMapper mapper = new ObjectMapper().enable(INDENT_OUTPUT).configure(JsonParser.Feature.ALLOW_COMMENTS, true);
+    ObjectMapper mapper = JacksonUtils.createObjectMapper().enable(INDENT_OUTPUT)
+        .configure(JsonParser.Feature.ALLOW_COMMENTS, true);
 
     READER = mapper.readerFor(SchemaContainer.class);
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlTunnel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlTunnel.java
index 492d4de625..f242f772ad 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlTunnel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/control/ControlTunnel.java
@@ -31,6 +31,7 @@ import com.google.protobuf.MessageLite;
 import com.google.protobuf.Parser;
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.DrillBuf;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.proto.BitControl.CustomMessage;
 import org.apache.drill.exec.proto.BitControl.FinishedReceiver;
 import org.apache.drill.exec.proto.BitControl.FragmentStatus;
@@ -465,17 +466,18 @@ public class ControlTunnel {
     private final ObjectReader reader;
 
     public JacksonSerDe(Class<MSG> clazz) {
-      ObjectMapper mapper = new ObjectMapper();
+      ObjectMapper mapper = JacksonUtils.createObjectMapper();
       writer = mapper.writerFor(clazz);
       reader = mapper.readerFor(clazz);
     }
 
     public JacksonSerDe(Class<MSG> clazz, JsonSerializer<MSG> serializer, JsonDeserializer<MSG> deserializer) {
-      ObjectMapper mapper = new ObjectMapper();
       SimpleModule module = new SimpleModule();
-      mapper.registerModule(module);
       module.addSerializer(clazz, serializer);
       module.addDeserializer(clazz, deserializer);
+      ObjectMapper mapper = JacksonUtils.createJsonMapperBuilder()
+          .addModule(module)
+          .build();
       writer = mapper.writerFor(clazz);
       reader = mapper.readerFor(clazz);
     }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/InboundImpersonationManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/InboundImpersonationManager.java
index 40ea0a3718..69acf7823c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/InboundImpersonationManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/InboundImpersonationManager.java
@@ -21,6 +21,7 @@ import com.fasterxml.jackson.core.JsonGenerator;
 import com.fasterxml.jackson.core.JsonParser;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
 import org.apache.drill.shaded.guava.com.google.common.collect.Sets;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
@@ -58,7 +59,7 @@ public class InboundImpersonationManager {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(InboundImpersonationManager.class);
 
   private static final String STAR = "*";
-  private static final ObjectMapper impersonationPolicyMapper = new ObjectMapper();
+  private static final ObjectMapper impersonationPolicyMapper = JacksonUtils.createObjectMapper();
 
   private List<ImpersonationPolicy> impersonationPolicies;
   private String policiesString; // used to test if policies changed
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/serialization/JacksonSerializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/serialization/JacksonSerializer.java
index d870561386..99ccbe6288 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/serialization/JacksonSerializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/serialization/JacksonSerializer.java
@@ -22,12 +22,19 @@ import java.io.IOException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectReader;
 import com.fasterxml.jackson.databind.ObjectWriter;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.shaded.guava.com.google.common.base.Objects;
 
 public class JacksonSerializer<T> implements InstanceSerializer<T> {
+  private static final ObjectMapper DEFAULT_MAPPER = JacksonUtils.createObjectMapper();
   private final ObjectReader reader;
   private final ObjectWriter writer;
 
+  public JacksonSerializer(final Class<T> klazz) {
+    this.reader = DEFAULT_MAPPER.readerFor(klazz);
+    this.writer = DEFAULT_MAPPER.writer();
+  }
+
   public JacksonSerializer(final ObjectMapper mapper, final Class<T> klazz) {
     this.reader = mapper.readerFor(klazz);
     this.writer = mapper.writer();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/PersistedOptionValue.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/PersistedOptionValue.java
index d670c40e06..6a35f19967 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/PersistedOptionValue.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/PersistedOptionValue.java
@@ -25,9 +25,9 @@ import com.fasterxml.jackson.core.ObjectCodec;
 import com.fasterxml.jackson.databind.DeserializationContext;
 import com.fasterxml.jackson.databind.JavaType;
 import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
 import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -316,7 +316,7 @@ public class PersistedOptionValue {
       }
 
       if (value == null) {
-        logger.error("Invalid json stored {}.", new ObjectMapper().writeValueAsString(node));
+        logger.error("Invalid json stored {}.", JacksonUtils.createObjectMapper().writeValueAsString(node));
       }
 
       return new PersistedOptionValue(value);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java
index b0838da039..8576d2f3f0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java
@@ -30,6 +30,7 @@ import java.util.stream.Collectors;
 
 import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.proto.UserBitShared.MajorFragmentProfile;
 import org.apache.drill.exec.proto.UserBitShared.MinorFragmentProfile;
@@ -54,7 +55,7 @@ public class ProfileWrapper {
   private static final String ESTIMATED_LABEL = " (Estimated)";
   private static final String NOT_AVAILABLE_LABEL = "Not Available";
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProfileWrapper.class);
-  private static final ObjectMapper mapper = new ObjectMapper().enable(INDENT_OUTPUT);
+  private static final ObjectMapper mapper = JacksonUtils.createObjectMapper().enable(INDENT_OUTPUT);
 
   private final QueryProfile profile;
   private final String id;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/DrillbitPluginRegistryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/DrillbitPluginRegistryContext.java
index fc75f82c79..4565b55fa7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/DrillbitPluginRegistryContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/DrillbitPluginRegistryContext.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.store;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.config.LogicalPlanPersistence;
 import org.apache.drill.common.scanner.persistence.ScanResult;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.server.DrillbitContext;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
@@ -44,7 +45,7 @@ public class DrillbitPluginRegistryContext implements PluginRegistryContext {
     // to handle HOCON format in the override file
     LogicalPlanPersistence persistence = new LogicalPlanPersistence(drillbitContext.getConfig(),
         drillbitContext.getClasspathScan(),
-        new ObjectMapper(new HoconFactory()));
+        JacksonUtils.createObjectMapper(new HoconFactory()));
     hoconMapper = persistence.getMapper();
   }
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonStructureParser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonStructureParser.java
index 4d33403070..f9bc515fa0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonStructureParser.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/parser/JsonStructureParser.java
@@ -25,6 +25,7 @@ import java.util.Map;
 import java.util.function.Function;
 
 import org.apache.commons.collections4.IterableUtils;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.store.easy.json.parser.MessageParser.MessageContextException;
 import org.apache.drill.exec.store.easy.json.parser.RootParser.EmbeddedArrayParser;
 import org.apache.drill.exec.store.easy.json.parser.RootParser.EmbeddedObjectParser;
@@ -160,7 +161,7 @@ public class JsonStructureParser {
   private JsonStructureParser(JsonStructureParserBuilder builder) {
     this.options = Preconditions.checkNotNull(builder.options);
     this.errorFactory = Preconditions.checkNotNull(builder.errorFactory);
-    ObjectMapper mapper = new ObjectMapper()
+    ObjectMapper mapper = JacksonUtils.createObjectMapper()
         .configure(JsonParser.Feature.ALLOW_COMMENTS, true)
         .configure(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES, true)
         .configure(JsonReadFeature.ALLOW_NON_NUMERIC_NUMBERS.mappedFeature(),
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/BaseJsonProcessor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/BaseJsonProcessor.java
index 4b61863ec3..b842d77b72 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/BaseJsonProcessor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/reader/BaseJsonProcessor.java
@@ -22,6 +22,7 @@ import io.netty.buffer.DrillBuf;
 import java.io.IOException;
 import java.io.InputStream;
 
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.store.easy.json.JsonProcessor;
 
 import com.fasterxml.jackson.core.JsonLocation;
@@ -69,7 +70,7 @@ public abstract class BaseJsonProcessor implements JsonProcessor {
    * @return Default json mapper
    */
   public static ObjectMapper getDefaultMapper() {
-    return new ObjectMapper().configure(
+    return JacksonUtils.createObjectMapper().configure(
         JsonParser.Feature.ALLOW_COMMENTS, true).configure(
         JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES, true);
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/http/oauth/OAuthUtils.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/http/oauth/OAuthUtils.java
index b7c09f628f..5f4bd1e501 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/http/oauth/OAuthUtils.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/http/oauth/OAuthUtils.java
@@ -26,6 +26,7 @@ import okhttp3.RequestBody;
 import okhttp3.Response;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.logical.security.CredentialsProvider;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.store.security.oauth.OAuthTokenCredentials;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -36,6 +37,7 @@ import java.util.Map;
 
 public class OAuthUtils {
   private static final Logger logger = LoggerFactory.getLogger(OAuthUtils.class);
+  private static final ObjectMapper MAPPER = JacksonUtils.createObjectMapper();
 
   /**
    * Crafts a POST request to obtain an access token.
@@ -144,8 +146,7 @@ public class OAuthUtils {
       }
 
       logger.debug("Response: {}", responseBody);
-      ObjectMapper mapper = new ObjectMapper();
-      Map<String, Object> parsedJson = mapper.readValue(responseBody, Map.class);
+      Map<String, Object> parsedJson = MAPPER.readValue(responseBody, Map.class);
 
       if (parsedJson.containsKey("access_token")) {
         accessToken = (String) parsedJson.get("access_token");
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
index 983e150be6..53240d8001 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
@@ -32,6 +32,7 @@ import org.apache.calcite.schema.Table;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
 import org.apache.drill.exec.planner.logical.DynamicDrillTable;
 import org.apache.drill.exec.server.DrillbitContext;
@@ -65,7 +66,7 @@ public class MockStorageEngine extends AbstractStoragePlugin {
       throws IOException {
 
     MockTableSelection tableSelection = selection.getWith(
-      new ObjectMapper(),
+      JacksonUtils.createObjectMapper(),
       MockTableSelection.class
     );
     List<MockScanEntry> readEntries = tableSelection.getEntries();
@@ -152,7 +153,7 @@ public class MockStorageEngine extends AbstractStoragePlugin {
       MockTableDef mockTableDefn;
       try {
         String json = Resources.toString(url, Charsets.UTF_8);
-        final ObjectMapper mapper = new ObjectMapper();
+        final ObjectMapper mapper = JacksonUtils.createObjectMapper();
         mapper.configure(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES, true);
         mockTableDefn = mapper.readValue(json, MockTableDef.class);
       } catch (JsonParseException e) {
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata.java
index 4e294e5b36..a9202bc1de 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/metadata/Metadata.java
@@ -29,6 +29,7 @@ import org.apache.commons.lang3.tuple.Pair;
 import org.apache.drill.common.collections.Collectors;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.util.DrillVersionInfo;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.serialization.PathSerDe;
 import org.apache.drill.exec.store.TimedCallable;
 import org.apache.drill.exec.store.dfs.MetadataContext;
@@ -531,13 +532,14 @@ public class Metadata {
     JsonFactory jsonFactory = new JsonFactory();
     jsonFactory.configure(Feature.AUTO_CLOSE_TARGET, false);
     jsonFactory.configure(JsonParser.Feature.AUTO_CLOSE_SOURCE, false);
-    ObjectMapper mapper = new ObjectMapper(jsonFactory);
     SimpleModule module = new SimpleModule();
     module.addSerializer(Path.class, new PathSerDe.Se());
     if (parquetMetadata instanceof Metadata_V4.FileMetadata) {
       module.addSerializer(ColumnMetadata_v4.class, new ColumnMetadata_v4.Serializer());
     }
-    mapper.registerModule(module);
+    ObjectMapper mapper = JacksonUtils.createJsonMapperBuilder(jsonFactory)
+        .addModule(module)
+        .build();
     OutputStream os = fs.create(p);
     mapper.writerWithDefaultPrettyPrinter().writeValue(os, parquetMetadata);
     os.flush();
@@ -556,7 +558,7 @@ public class Metadata {
     Stopwatch timer = logger.isDebugEnabled() ? Stopwatch.createStarted() : null;
     Path metadataParentDir = Path.getPathWithoutSchemeAndAuthority(path.getParent());
     String metadataParentDirPath = metadataParentDir.toUri().getPath();
-    ObjectMapper mapper = new ObjectMapper();
+    ObjectMapper mapper = JacksonUtils.createObjectMapper();
 
     final SimpleModule serialModule = new SimpleModule();
     serialModule.addDeserializer(SchemaPath.class, new SchemaPath.De());
@@ -729,14 +731,15 @@ public class Metadata {
           }
         }
         // Read the existing metadataSummary cache file to get the metadataSummary
-        ObjectMapper mapper = new ObjectMapper();
         final SimpleModule serialModule = new SimpleModule();
         serialModule.addDeserializer(SchemaPath.class, new SchemaPath.De());
         serialModule.addKeyDeserializer(ColumnTypeMetadata_v4.Key.class, new ColumnTypeMetadata_v4.Key.DeSerializer());
         AfterburnerModule module = new AfterburnerModule();
         module.setUseOptimizedBeanDeserializer(true);
-        mapper.registerModule(serialModule);
-        mapper.registerModule(module);
+        ObjectMapper mapper = JacksonUtils.createJsonMapperBuilder()
+            .addModule(serialModule)
+            .addModule(module)
+            .build();
         mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
         InputStream is = fs.open(summaryFile);
         return mapper.readValue(is, Metadata_V4.MetadataSummary.class);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/PersistentStoreConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/PersistentStoreConfig.java
index 668306ebfb..da48c6e0b6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/PersistentStoreConfig.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/PersistentStoreConfig.java
@@ -83,6 +83,10 @@ public class PersistentStoreConfig<V> {
     return new StoreConfigBuilder<>(new ProtoSerializer<>(readSchema, writeSchema));
   }
 
+  public static <V> StoreConfigBuilder<V> newJacksonBuilder(Class<V> clazz) {
+    return new StoreConfigBuilder<>(new JacksonSerializer<>(clazz));
+  }
+
   public static <V> StoreConfigBuilder<V> newJacksonBuilder(ObjectMapper mapper, Class<V> clazz) {
     return new StoreConfigBuilder<>(new JacksonSerializer<>(mapper, clazz));
   }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java
index 3543253a55..728c26c0b1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/testing/ExecutionControls.java
@@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes.Type;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.shaded.guava.com.google.common.annotations.VisibleForTesting;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.UserException;
@@ -47,7 +48,7 @@ public final class ExecutionControls {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExecutionControls.class);
 
   // used to map JSON specified injections to POJOs
-  public static final ObjectMapper controlsOptionMapper = new ObjectMapper();
+  public static final ObjectMapper controlsOptionMapper = JacksonUtils.createObjectMapper();
 
   static {
     controlsOptionMapper.addMixInAnnotations(Injection.class, InjectionMixIn.class);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/QueryWorkUnit.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/QueryWorkUnit.java
index 2e3a0d0ee9..54fee8c795 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/QueryWorkUnit.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/QueryWorkUnit.java
@@ -20,7 +20,9 @@ package org.apache.drill.exec.work;
 import java.util.ArrayList;
 import java.util.List;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
 import org.apache.drill.exec.planner.fragment.Wrapper;
@@ -29,12 +31,12 @@ import org.apache.drill.exec.proto.CoordinationProtos;
 import org.apache.drill.exec.proto.ExecProtos;
 import org.apache.drill.exec.server.options.OptionList;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
-
-import com.fasterxml.jackson.core.JsonProcessingException;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 
 public class QueryWorkUnit {
 
+  private static final ObjectMapper MAPPER = JacksonUtils.createObjectMapper();
+
   /**
    * Definition of a minor fragment that contains the (unserialized) fragment operator
    * tree and the (partially built) fragment. Allows the resource manager to apply
@@ -149,10 +151,9 @@ public class QueryWorkUnit {
 
       String jsonString = "<<malformed JSON>>";
       stringBuilder.append("  fragment_json: ");
-      final ObjectMapper objectMapper = new ObjectMapper();
       try {
-        final Object json = objectMapper.readValue(planFragment.getFragmentJson(), Object.class);
-        jsonString = objectMapper.writerWithDefaultPrettyPrinter().writeValueAsString(json);
+        final Object json = MAPPER.readValue(planFragment.getFragmentJson(), Object.class);
+        jsonString = MAPPER.writerWithDefaultPrettyPrinter().writeValueAsString(json);
       } catch (final Exception e) {
         // we've already set jsonString to a fallback value
       }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
index 202163533d..3cdc9616f0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.work.foreman;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.work.filter.RuntimeFilterRouter;
 import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
@@ -100,7 +101,7 @@ public class Foreman implements Runnable {
 
   public enum ProfileOption { SYNC, ASYNC, NONE }
 
-  private static final ObjectMapper MAPPER = new ObjectMapper();
+  private static final ObjectMapper MAPPER = JacksonUtils.createObjectMapper();
 
   private final QueryId queryId;
   private final String queryIdString;
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/TestPathSerialization.java b/exec/java-exec/src/test/java/org/apache/drill/exec/TestPathSerialization.java
index fd697f5df7..532a567ba5 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/TestPathSerialization.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/TestPathSerialization.java
@@ -18,7 +18,9 @@
 package org.apache.drill.exec;
 
 
+import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.module.SimpleModule;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.serialization.PathSerDe;
 import org.apache.drill.exec.store.schedule.CompleteFileWork;
 import org.apache.drill.test.DrillTest;
@@ -39,9 +41,10 @@ public class TestPathSerialization extends DrillTest {
 
     SimpleModule module = new SimpleModule();
     module.addSerializer(Path.class, new PathSerDe.Se());
-    objectMapper.registerModule(module);
+    ObjectMapper testMapper = JacksonUtils.createObjectMapper();
+    testMapper.registerModule(module);
 
-    CompleteFileWork.FileWorkImpl bean = objectMapper.readValue(jsonString, CompleteFileWork.FileWorkImpl.class);
+    CompleteFileWork.FileWorkImpl bean = testMapper.readValue(jsonString, CompleteFileWork.FileWorkImpl.class);
 
     assertThat(bean.getStart() == 1,  equalTo( true ));
     assertThat(bean.getLength() == 2, equalTo( true ));
@@ -53,10 +56,11 @@ public class TestPathSerialization extends DrillTest {
     CompleteFileWork.FileWorkImpl fileWork = new CompleteFileWork.FileWorkImpl(5, 6, new Path("/tmp"));
     SimpleModule module = new SimpleModule();
     module.addSerializer(Path.class, new PathSerDe.Se());
-    objectMapper.registerModule(module);
+    ObjectMapper testMapper = JacksonUtils.createObjectMapper();
+    testMapper.registerModule(module);
 
     CompleteFileWork.FileWorkImpl bean =
-        objectMapper.readValue(objectMapper.writeValueAsString(fileWork), CompleteFileWork.FileWorkImpl.class);
+        testMapper.readValue(testMapper.writeValueAsString(fileWork), CompleteFileWork.FileWorkImpl.class);
 
     assertThat(bean.getStart() == 5,  equalTo( true ));
     assertThat(bean.getLength() == 6, equalTo( true ));
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/options/PersistedOptionValueTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/options/PersistedOptionValueTest.java
index 04b7bb94c9..1f82f142ce 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/options/PersistedOptionValueTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/options/PersistedOptionValueTest.java
@@ -19,7 +19,6 @@ package org.apache.drill.exec.server.options;
 
 import com.fasterxml.jackson.annotation.JsonInclude;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.drill.common.util.DrillFileUtils;
 import org.apache.drill.exec.serialization.JacksonSerializer;
 import org.apache.drill.test.BaseTest;
@@ -46,7 +45,7 @@ public class PersistedOptionValueTest extends BaseTest {
 
   private void testHelper(String booleanOptionFile, String doubleOptionFile,
                           String longOptionFile, String stringOptionFile) throws IOException {
-    JacksonSerializer serializer = new JacksonSerializer<>(new ObjectMapper(), PersistedOptionValue.class);
+    JacksonSerializer serializer = new JacksonSerializer<>(PersistedOptionValue.class);
     String booleanOptionJson = DrillFileUtils.getResourceAsString(booleanOptionFile);
     String doubleOptionJson = DrillFileUtils.getResourceAsString(doubleOptionFile);
     String longOptionJson = DrillFileUtils.getResourceAsString(longOptionFile);
@@ -97,8 +96,8 @@ public class PersistedOptionValueTest extends BaseTest {
   public void testForwardCompatibility() throws IOException {
     final String name = "myOption";
 
-    JacksonSerializer realSerializer = new JacksonSerializer<>(new ObjectMapper(), PersistedOptionValue.class);
-    JacksonSerializer mockSerializer = new JacksonSerializer<>(new ObjectMapper(), MockPersistedOptionValue.class);
+    JacksonSerializer realSerializer = new JacksonSerializer<>(PersistedOptionValue.class);
+    JacksonSerializer mockSerializer = new JacksonSerializer<>(MockPersistedOptionValue.class);
 
     final String stringContent = "val1";
     PersistedOptionValue stringValue =
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/TestRestJson.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/TestRestJson.java
index 5449a3c42e..b4325112e8 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/TestRestJson.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/TestRestJson.java
@@ -28,6 +28,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.store.easy.text.TextFormatConfig;
@@ -81,7 +82,7 @@ public class TestRestJson extends ClusterTest {
       .readTimeout(TIMEOUT, TimeUnit.SECONDS)
       .build();
 
-  private final ObjectMapper mapper = new ObjectMapper();
+  private final ObjectMapper mapper = JacksonUtils.createObjectMapper();
   private final FileVerifier verifier = new FileVerifier("/rest");
 
   @BeforeClass
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java
index 469462e601..7619ea013a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java
@@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.drill.categories.SqlTest;
 import org.apache.drill.common.exceptions.UserRemoteException;
 import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.physical.rowSet.DirectRowSet;
 import org.apache.drill.exec.physical.rowSet.RowSet;
 import org.apache.drill.exec.physical.rowSet.RowSetBuilder;
@@ -63,7 +64,7 @@ import static org.junit.Assert.assertTrue;
 @Category(SqlTest.class)
 public class TestInfoSchema extends ClusterTest {
   private static final String TEST_SUB_DIR = "testSubDir";
-  private static final ObjectMapper mapper = new ObjectMapper().enable(INDENT_OUTPUT);
+  private static final ObjectMapper mapper = JacksonUtils.createObjectMapper().enable(INDENT_OUTPUT);
 
   @BeforeClass
   public static void setupFiles() throws Exception {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetReaderConfig.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetReaderConfig.java
index f44a56a6a8..18c4b259e9 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetReaderConfig.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetReaderConfig.java
@@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.drill.categories.ParquetTest;
 import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.server.options.SessionOptionManager;
 import org.apache.drill.exec.server.options.SystemOptionManager;
@@ -40,7 +41,7 @@ public class TestParquetReaderConfig extends BaseTest {
 
   @Test
   public void testDefaultsDeserialization() throws Exception {
-    ObjectMapper mapper = new ObjectMapper();
+    ObjectMapper mapper = JacksonUtils.createObjectMapper();
     ParquetReaderConfig readerConfig = ParquetReaderConfig.builder().build(); // all defaults
     String value = mapper.writeValueAsString(readerConfig);
     assertEquals(ParquetReaderConfig.getDefaultInstance(), readerConfig); // compare with default instance
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/PStoreTestUtil.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/PStoreTestUtil.java
index d8145519bc..b8b9acdb73 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/PStoreTestUtil.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/PStoreTestUtil.java
@@ -26,13 +26,11 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-
 public class PStoreTestUtil {
 
   public static void test(PersistentStoreProvider provider) throws Exception {
     PersistentStore<String> store = provider.getOrCreateStore(
-      PersistentStoreConfig.newJacksonBuilder(new ObjectMapper(), String.class)
+      PersistentStoreConfig.newJacksonBuilder(String.class)
         .name("sys.test")
         .build()
     );
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestLocalPersistentStore.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestLocalPersistentStore.java
index b26cccacf1..42dbf922a3 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestLocalPersistentStore.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestLocalPersistentStore.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.store.sys;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
@@ -51,7 +50,7 @@ public class TestLocalPersistentStore extends BaseTest {
   public TemporaryFolder root = new TemporaryFolder();
 
   private static final PersistentStoreConfig<String> DEFAULT_STORE_CONFIG = PersistentStoreConfig
-    .newJacksonBuilder(new ObjectMapper(), String.class)
+    .newJacksonBuilder(String.class)
     .name("local-test-store")
     .build();
 
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestPStoreProviders.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestPStoreProviders.java
index a90d94798c..89819e0d47 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestPStoreProviders.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/sys/TestPStoreProviders.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.store.sys;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.drill.categories.FlakyTest;
 import org.apache.drill.categories.SlowTest;
@@ -123,7 +122,7 @@ public class TestPStoreProviders extends TestWithZookeeper {
       curator.start();
 
       PersistentStoreConfig<PersistedOptionValue> storeConfig =
-        PersistentStoreConfig.newJacksonBuilder(new ObjectMapper(), PersistedOptionValue.class).name("sys.test").build();
+        PersistentStoreConfig.newJacksonBuilder(PersistedOptionValue.class).name("sys.test").build();
 
       try (ZookeeperClient zkClient = new ZookeeperClient(curator,
         PathUtils.join("/", storeConfig.getName()), CreateMode.PERSISTENT)) {
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestRepeated.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestRepeated.java
index 5a29f2a1e5..bf880c4e33 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestRepeated.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestRepeated.java
@@ -36,9 +36,6 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.ObjectWriter;
-
 public class TestRepeated extends BaseTest {
   // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestRepeated.class);
 
@@ -238,8 +235,6 @@ public class TestRepeated extends BaseTest {
       map.end();
     }
 
-    final ObjectWriter ow = new ObjectMapper().writer().withDefaultPrettyPrinter();
-
     final ByteArrayOutputStream stream = new ByteArrayOutputStream();
     final JsonWriter jsonWriter = new JsonWriter(stream, true, true);
     final FieldReader reader = mapVector.getChild("col", MapVector.class).getReader();
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java b/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
index 8d20789117..e2ed409861 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/PhysicalOpUnitTestBase.java
@@ -20,6 +20,7 @@ package org.apache.drill.test;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.coord.ClusterCoordinator;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.AccountingDataTunnel;
@@ -374,7 +375,7 @@ public class PhysicalOpUnitTestBase extends ExecTest {
    * @return The {@link org.apache.drill.exec.store.easy.json.JSONRecordReader} corresponding to each given jsonBatch.
    */
   public static Iterator<RecordReader> getJsonReadersFromBatchString(List<String> jsonBatches, FragmentContext fragContext, List<SchemaPath> columnsToRead) {
-    ObjectMapper mapper = new ObjectMapper();
+    ObjectMapper mapper = JacksonUtils.createObjectMapper();
     List<RecordReader> readers = new ArrayList<>();
     for (String batchJason : jsonBatches) {
       JsonNode records;
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleMetadata.java b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleMetadata.java
index 6a25a56d27..674ece656b 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleMetadata.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/record/metadata/TupleMetadata.java
@@ -23,9 +23,9 @@ import java.util.List;
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
 import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectReader;
 import com.fasterxml.jackson.databind.ObjectWriter;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.exec.record.MaterializedField;
 
 /**
@@ -56,8 +56,8 @@ import org.apache.drill.exec.record.MaterializedField;
 })
 public interface TupleMetadata extends Propertied, Iterable<ColumnMetadata> {
 
-  ObjectWriter WRITER = new ObjectMapper().writerFor(TupleMetadata.class);
-  ObjectReader READER = new ObjectMapper().readerFor(TupleMetadata.class);
+  ObjectWriter WRITER = JacksonUtils.createObjectMapper().writerFor(TupleMetadata.class);
+  ObjectReader READER = JacksonUtils.createObjectMapper().readerFor(TupleMetadata.class);
 
   String IS_STRICT_SCHEMA_PROP = DRILL_PROP_PREFIX + "strict";
 
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringArrayList.java b/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringArrayList.java
index 19ff04b12e..d0e149fae0 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringArrayList.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringArrayList.java
@@ -21,15 +21,13 @@ import java.util.ArrayList;
 import java.util.List;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.common.util.JacksonUtils;
 
 public class JsonStringArrayList<E> extends ArrayList<E> {
 
-  private static ObjectMapper mapper;
-
-  static {
-    mapper = new ObjectMapper();
-    mapper.registerModule(SerializationModule.getModule());
-  }
+  private static final ObjectMapper mapper = JacksonUtils.createJsonMapperBuilder()
+      .addModule(SerializationModule.getModule())
+      .build();
 
   @Override
   public boolean equals(Object obj) {
diff --git a/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringHashMap.java b/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringHashMap.java
index 337858fa6c..3fcecb4e70 100644
--- a/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringHashMap.java
+++ b/exec/vector/src/main/java/org/apache/drill/exec/util/JsonStringHashMap.java
@@ -24,6 +24,7 @@ import java.util.Map;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.datatype.joda.JodaModule;
+import org.apache.drill.common.util.JacksonUtils;
 
 /*
  * Simple class that extends the regular java.util.HashMap but overrides the
@@ -31,7 +32,7 @@ import com.fasterxml.jackson.datatype.joda.JodaModule;
  */
 public class JsonStringHashMap<K, V> extends LinkedHashMap<K, V> {
 
-  private static final ObjectMapper mapper = new ObjectMapper()
+  private static final ObjectMapper mapper = JacksonUtils.createObjectMapper()
       .registerModule(SerializationModule.getModule())
       .registerModule(new JodaModule());
 
diff --git a/logical/src/main/java/org/apache/drill/common/JSONOptions.java b/logical/src/main/java/org/apache/drill/common/JSONOptions.java
index 5bfdb567d9..9bdafaa15d 100644
--- a/logical/src/main/java/org/apache/drill/common/JSONOptions.java
+++ b/logical/src/main/java/org/apache/drill/common/JSONOptions.java
@@ -25,6 +25,7 @@ import org.apache.drill.common.JSONOptions.De;
 import org.apache.drill.common.JSONOptions.Se;
 import org.apache.drill.common.config.LogicalPlanPersistence;
 import org.apache.drill.common.exceptions.LogicalPlanParsingException;
+import org.apache.drill.common.util.JacksonUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,6 +52,7 @@ import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 public class JSONOptions {
 
   private final static Logger logger = LoggerFactory.getLogger(JSONOptions.class);
+  private final static ObjectMapper DEFAULT_MAPPER = JacksonUtils.createObjectMapper();
 
   private JsonNode root;
   private JsonLocation location;
@@ -130,6 +132,10 @@ public class JSONOptions {
     return mapper.treeAsTokens(root).readValueAs(t);
   }
 
+  public <T> T getListWith(TypeReference<T> t) throws IOException {
+    return getListWith(DEFAULT_MAPPER, t);
+  }
+
   public JsonNode path(String name) {
     return root.path(name);
   }
diff --git a/logical/src/main/java/org/apache/drill/common/config/LogicalPlanPersistence.java b/logical/src/main/java/org/apache/drill/common/config/LogicalPlanPersistence.java
index 719e838e63..903b8f014f 100644
--- a/logical/src/main/java/org/apache/drill/common/config/LogicalPlanPersistence.java
+++ b/logical/src/main/java/org/apache/drill/common/config/LogicalPlanPersistence.java
@@ -28,6 +28,7 @@ import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.common.logical.data.LogicalOperator;
 import org.apache.drill.common.scanner.persistence.ScanResult;
 import org.apache.drill.common.logical.security.CredentialsProvider;
+import org.apache.drill.common.util.JacksonUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,7 +46,7 @@ public class LogicalPlanPersistence {
   private final ObjectMapper mapper;
 
   public LogicalPlanPersistence(DrillConfig conf, ScanResult scanResult) {
-    this(conf, scanResult, new ObjectMapper());
+    this(conf, scanResult, JacksonUtils.createObjectMapper());
   }
 
   public LogicalPlanPersistence(DrillConfig conf, ScanResult scanResult, ObjectMapper mapper) {
diff --git a/logical/src/main/java/org/apache/drill/common/util/AbstractDynamicBean.java b/logical/src/main/java/org/apache/drill/common/util/AbstractDynamicBean.java
index 5765a5e218..0814d18371 100644
--- a/logical/src/main/java/org/apache/drill/common/util/AbstractDynamicBean.java
+++ b/logical/src/main/java/org/apache/drill/common/util/AbstractDynamicBean.java
@@ -69,8 +69,8 @@ public abstract class AbstractDynamicBean {
 
 
   private static synchronized ObjectMapper getMapper(){
-    if(MAPPER == null){
-      ObjectMapper mapper = new ObjectMapper();
+    if (MAPPER == null) {
+      ObjectMapper mapper = JacksonUtils.createObjectMapper();
       mapper.enable(SerializationFeature.INDENT_OUTPUT);
       mapper.configure(Feature.ALLOW_UNQUOTED_FIELD_NAMES, true);
       mapper.configure(Feature.ALLOW_COMMENTS, true);
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ColumnStatistics.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ColumnStatistics.java
index 5fbb85a0b3..b909280a5e 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ColumnStatistics.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/ColumnStatistics.java
@@ -28,6 +28,7 @@ import com.fasterxml.jackson.databind.ObjectReader;
 import com.fasterxml.jackson.databind.ObjectWriter;
 import com.fasterxml.jackson.datatype.joda.JodaModule;
 import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.metastore.util.TableMetadataUtils;
 
 import java.io.IOException;
@@ -66,13 +67,13 @@ import java.util.stream.Collectors;
 @JsonPropertyOrder({"statistics", "comparator"})
 public class ColumnStatistics<T> {
 
-  private static final ObjectWriter OBJECT_WRITER = new ObjectMapper()
-      .registerModule(new JodaModule())
-      .writerFor(ColumnStatistics.class);
+  private static final ObjectMapper MAPPER = JacksonUtils.createJsonMapperBuilder()
+      .addModule(new JodaModule())
+      .build();
 
-  private static final ObjectReader OBJECT_READER = new ObjectMapper()
-      .registerModule(new JodaModule())
-      .readerFor(ColumnStatistics.class);
+  private static final ObjectWriter OBJECT_WRITER = MAPPER.writerFor(ColumnStatistics.class);
+
+  private static final ObjectReader OBJECT_READER = MAPPER.readerFor(ColumnStatistics.class);
 
   private final Map<String, StatisticsHolder<?>> statistics;
   private final Comparator<T> valueComparator;
diff --git a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/StatisticsHolder.java b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/StatisticsHolder.java
index ffbe3accce..a7964a7c23 100644
--- a/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/StatisticsHolder.java
+++ b/metastore/metastore-api/src/main/java/org/apache/drill/metastore/statistics/StatisticsHolder.java
@@ -25,6 +25,7 @@ import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectReader;
 import com.fasterxml.jackson.databind.ObjectWriter;
+import org.apache.drill.common.util.JacksonUtils;
 
 import java.io.IOException;
 import java.util.Objects;
@@ -38,8 +39,9 @@ import java.util.StringJoiner;
 @JsonInclude(JsonInclude.Include.NON_DEFAULT)
 public class StatisticsHolder<T> {
 
-  private static final ObjectWriter OBJECT_WRITER = new ObjectMapper().writerFor(StatisticsHolder.class);
-  private static final ObjectReader OBJECT_READER = new ObjectMapper().readerFor(StatisticsHolder.class);
+  private static final ObjectMapper OBJECT_MAPPER = JacksonUtils.createObjectMapper();
+  private static final ObjectWriter OBJECT_WRITER = OBJECT_MAPPER.writerFor(StatisticsHolder.class);
+  private static final ObjectReader OBJECT_READER = OBJECT_MAPPER.readerFor(StatisticsHolder.class);
 
   private final T statisticsValue;
   private final BaseStatisticsKind<?> statisticsKind;
diff --git a/metastore/rdbms-metastore/src/main/java/org/apache/drill/metastore/rdbms/util/ConverterUtil.java b/metastore/rdbms-metastore/src/main/java/org/apache/drill/metastore/rdbms/util/ConverterUtil.java
index e1f2ae7f1e..e8bd423122 100644
--- a/metastore/rdbms-metastore/src/main/java/org/apache/drill/metastore/rdbms/util/ConverterUtil.java
+++ b/metastore/rdbms-metastore/src/main/java/org/apache/drill/metastore/rdbms/util/ConverterUtil.java
@@ -20,6 +20,7 @@ package org.apache.drill.metastore.rdbms.util;
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.drill.common.util.JacksonUtils;
 import org.apache.drill.metastore.rdbms.exception.RdbmsMetastoreException;
 
 import java.io.IOException;
@@ -31,7 +32,7 @@ import java.util.Map;
  */
 public class ConverterUtil {
 
-  private static final ObjectMapper MAPPER = new ObjectMapper();
+  private static final ObjectMapper MAPPER = JacksonUtils.createObjectMapper();
 
   private static final TypeReference<List<String>> LIST_STRING_TYPE_REF = new TypeReference<List<String>>() {
   };