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

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


The following commit(s) were added to refs/heads/master by this push:
     new 466057c61be Remove deprecated DruidException, EntryExistsException 
(#14448)
466057c61be is described below

commit 466057c61be3c3c019359ab2e0a67311f5dbb3a3
Author: Kashif Faraz <[email protected]>
AuthorDate: Fri Mar 15 21:29:11 2024 +0530

    Remove deprecated DruidException, EntryExistsException (#14448)
    
    Changes:
    - Remove deprecated `DruidException` (old one) and `EntryExistsException`
    - Use newly added comprehensive `DruidException` instead
    - Update error message in `SqlMetadataStorageActionHandler` when max packet 
limit is violated.
    - Factor out common code from several faults into `BaseFault`.
    - Slightly update javadoc in `DruidException` to render it correctly
    - Remove unused classes `SegmentToMove`, `SegmentToDrop`
    - Move `ServletResourceUtils` from module `druid-processing` to 
`druid-server`
    - Add utility method to build error Response from `DruidException`.
---
 .../MaterializedViewSupervisor.java                |  14 +-
 .../MaterializedViewSupervisorTest.java            | 244 +++++++++------------
 .../kinesis/supervisor/KinesisSupervisorTest.java  |   6 +-
 .../lookup/NamespaceLookupIntrospectHandler.java   |   2 +-
 .../indexing/overlord/HeapMemoryTaskStorage.java   |  12 +-
 .../indexing/overlord/MetadataTaskStorage.java     |   5 +-
 .../apache/druid/indexing/overlord/TaskQueue.java  |   8 +-
 .../druid/indexing/overlord/TaskStorage.java       |   5 +-
 .../indexing/overlord/http/OverlordResource.java   |  12 +-
 .../supervisor/SeekableStreamSupervisor.java       |   5 +-
 .../AppenderatorDriverRealtimeIndexTaskTest.java   |   6 +-
 .../indexing/common/task/IngestionTestBase.java    |   3 +-
 .../common/task/RealtimeIndexTaskTest.java         |   6 +-
 .../AbstractParallelIndexSupervisorTaskTest.java   |   8 +-
 .../overlord/TaskLockBoxConcurrencyTest.java       |   3 +-
 .../indexing/overlord/TaskLockConfigTest.java      |   7 +-
 .../druid/indexing/overlord/TaskLockboxTest.java   |  21 +-
 .../druid/indexing/overlord/TaskQueueTest.java     |  15 +-
 .../SeekableStreamIndexTaskTestBase.java           |   8 +-
 .../SeekableStreamSupervisorStateTest.java         |   3 +-
 .../druid/common/exception/DruidException.java     |  57 -----
 .../error/{NotFound.java => BaseFailure.java}      |  34 ++-
 .../org/apache/druid/error/DruidException.java     |  24 +-
 .../{NotFound.java => EntryAlreadyExists.java}     |  40 ++--
 .../java/org/apache/druid/error/Forbidden.java     |  29 +--
 .../apache/druid/error/InternalServerError.java    |  30 +--
 .../java/org/apache/druid/error/InvalidInput.java  |  33 +--
 .../main/java/org/apache/druid/error/NotFound.java |  31 +--
 .../druid/metadata/EntryExistsException.java       |  48 ----
 .../metadata/MetadataStorageActionHandler.java     |   3 +-
 .../druid/common/exception/DruidExceptionTest.java |  40 ----
 .../druid/metadata/EntryExistsExceptionTest.java   |  40 ----
 .../metadata/SQLMetadataStorageActionHandler.java  |  36 ++-
 .../query/lookup/LookupListeningResource.java      |   2 +-
 .../http/CoordinatorCompactionConfigsResource.java |   1 -
 .../http/CoordinatorDynamicConfigsResource.java    |   1 -
 .../server/http/LookupCoordinatorResource.java     |   1 -
 .../apache/druid/server/http/SegmentToDrop.java    |  51 -----
 .../apache/druid/server/http/SegmentToMove.java    |  59 -----
 .../druid/server/http}/ServletResourceUtils.java   |  14 +-
 .../listener/resource/AbstractListenerHandler.java |   2 +-
 .../server/listener/resource/ListenerResource.java |   2 +-
 .../SQLMetadataStorageActionHandlerTest.java       |  49 ++---
 .../server/http}/ServletResourceUtilsTest.java     |  24 +-
 44 files changed, 320 insertions(+), 724 deletions(-)

diff --git 
a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java
 
b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java
index 33386878015..7e0eaf60d83 100644
--- 
a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java
+++ 
b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java
@@ -27,6 +27,8 @@ import com.google.common.collect.Maps;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningScheduledExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.error.EntryAlreadyExists;
 import org.apache.druid.indexer.TaskStatus;
 import org.apache.druid.indexing.common.task.HadoopIndexTask;
 import org.apache.druid.indexing.overlord.DataSourceMetadata;
@@ -47,7 +49,6 @@ import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.concurrent.Execs;
 import org.apache.druid.java.util.common.guava.Comparators;
 import org.apache.druid.java.util.emitter.EmittingLogger;
-import org.apache.druid.metadata.EntryExistsException;
 import org.apache.druid.metadata.MetadataSupervisorManager;
 import org.apache.druid.metadata.SqlSegmentsMetadataManager;
 import org.apache.druid.timeline.DataSegment;
@@ -443,8 +444,15 @@ public class MaterializedViewSupervisor implements 
Supervisor
             runningTasks.put(entry.getKey(), task);
           }
         }
-        catch (EntryExistsException e) {
-          log.error("task %s already exsits", task);
+        catch (DruidException e) {
+          if (EntryAlreadyExists.ERROR_CODE.equals(e.getErrorCode())) {
+            log.error("Task[%s] already exists", task.getId());
+          } else {
+            throw e;
+          }
+        }
+        catch (RuntimeException e) {
+          throw e;
         }
         catch (Exception e) {
           throw new RuntimeException(e);
diff --git 
a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java
 
b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java
index 1e74180ae69..64070b11dc8 100644
--- 
a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java
+++ 
b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java
@@ -25,8 +25,10 @@ import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Sets;
+import junit.framework.AssertionFailedError;
 import org.apache.druid.data.input.impl.DimensionsSpec;
 import org.apache.druid.data.input.impl.StringDimensionSchema;
+import org.apache.druid.error.EntryAlreadyExists;
 import org.apache.druid.indexer.HadoopIOConfig;
 import org.apache.druid.indexer.HadoopIngestionSpec;
 import org.apache.druid.indexer.HadoopTuningConfig;
@@ -37,6 +39,7 @@ import org.apache.druid.indexing.overlord.TaskMaster;
 import org.apache.druid.indexing.overlord.TaskQueue;
 import org.apache.druid.indexing.overlord.TaskStorage;
 import 
org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig;
+import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.Pair;
 import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator;
@@ -58,11 +61,12 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.ExpectedException;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -71,10 +75,9 @@ import java.util.SortedMap;
 public class MaterializedViewSupervisorTest
 {
   @Rule
-  public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new 
TestDerbyConnector.DerbyConnectorRule();
+  public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule
+      = new TestDerbyConnector.DerbyConnectorRule();
 
-  @Rule
-  public final ExpectedException expectedException = ExpectedException.none();
   private TaskStorage taskStorage;
   private TaskMaster taskMaster;
   private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
@@ -102,7 +105,7 @@ public class MaterializedViewSupervisorTest
     metadataSupervisorManager = 
EasyMock.createMock(MetadataSupervisorManager.class);
     sqlSegmentsMetadataManager = 
EasyMock.createMock(SqlSegmentsMetadataManager.class);
     taskQueue = EasyMock.createMock(TaskQueue.class);
-    taskQueue.start();
+
     objectMapper.registerSubtypes(new 
NamedType(HashBasedNumberedShardSpec.class, "hashed"));
     spec = new MaterializedViewSupervisorSpec(
         "base",
@@ -113,7 +116,7 @@ public class MaterializedViewSupervisorTest
         null,
         null,
         null,
-        null,
+        Collections.singletonMap("maxTaskCount", 2),
         false,
         objectMapper,
         taskMaster,
@@ -133,125 +136,81 @@ public class MaterializedViewSupervisorTest
   @Test
   public void testCheckSegments() throws IOException
   {
-    Set<DataSegment> baseSegments = Sets.newHashSet(
-        new DataSegment(
-            "base",
-            Intervals.of("2015-01-01T00Z/2015-01-02T00Z"),
-            "2015-01-02",
-            ImmutableMap.of(),
-            ImmutableList.of("dim1", "dim2"),
-            ImmutableList.of("m1"),
-            new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null, null),
-            9,
-            1024
-        ),
-        new DataSegment(
-            "base",
-            Intervals.of("2015-01-02T00Z/2015-01-03T00Z"),
-            "2015-01-03",
-            ImmutableMap.of(),
-            ImmutableList.of("dim1", "dim2"),
-            ImmutableList.of("m1"),
-            new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null, null),
-            9,
-            1024
-        ),
-        new DataSegment(
-            "base",
-            Intervals.of("2015-01-03T00Z/2015-01-04T00Z"),
-            "2015-01-04",
-            ImmutableMap.of(),
-            ImmutableList.of("dim1", "dim2"),
-            ImmutableList.of("m1"),
-            new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null, null),
-            9,
-            1024
-        )
-    );
-    Set<DataSegment> derivativeSegments = Sets.newHashSet(
-        new DataSegment(
-            derivativeDatasourceName,
-            Intervals.of("2015-01-01T00Z/2015-01-02T00Z"),
-            "2015-01-02",
-            ImmutableMap.of(),
-            ImmutableList.of("dim1", "dim2"),
-            ImmutableList.of("m1"),
-            new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null, null),
-            9,
-            1024
-        ),
-        new DataSegment(
-            derivativeDatasourceName,
-            Intervals.of("2015-01-02T00Z/2015-01-03T00Z"),
-            "3015-01-01",
-            ImmutableMap.of(),
-            ImmutableList.of("dim1", "dim2"),
-            ImmutableList.of("m1"),
-            new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null, null),
-            9,
-            1024
-        )
+    List<DataSegment> baseSegments = createBaseSegments();
+    Set<DataSegment> derivativeSegments = 
Sets.newHashSet(createDerivativeSegments());
+
+    final Interval day1 = baseSegments.get(0).getInterval();
+    final Interval day2 = new Interval(day1.getStart().plusDays(1), 
day1.getEnd().plusDays(1));
+
+    indexerMetadataStorageCoordinator.commitSegments(new 
HashSet<>(baseSegments));
+    indexerMetadataStorageCoordinator.commitSegments(derivativeSegments);
+    
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
+    
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
+    
EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes();
+
+    Pair<SortedMap<Interval, String>, Map<Interval, List<DataSegment>>> 
toBuildInterval
+        = supervisor.checkSegments();
+
+    Map<Interval, List<DataSegment>> expectedSegments = ImmutableMap.of(
+        day1, Collections.singletonList(baseSegments.get(0)),
+        day2, Collections.singletonList(baseSegments.get(1))
     );
+    Assert.assertEquals(Collections.singleton(day1), 
toBuildInterval.lhs.keySet());
+    Assert.assertEquals(expectedSegments, toBuildInterval.rhs);
+  }
+
+  @Test
+  public void testSubmitTasksDoesNotFailIfTaskAlreadyExists() throws 
IOException
+  {
+    Set<DataSegment> baseSegments = Sets.newHashSet(createBaseSegments());
+    Set<DataSegment> derivativeSegments = 
Sets.newHashSet(createDerivativeSegments());
+
     indexerMetadataStorageCoordinator.commitSegments(baseSegments);
     indexerMetadataStorageCoordinator.commitSegments(derivativeSegments);
     
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
     
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
     
EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes();
-    Pair<SortedMap<Interval, String>, Map<Interval, List<DataSegment>>> 
toBuildInterval = supervisor.checkSegments();
-    Set<Interval> expectedToBuildInterval = 
Sets.newHashSet(Intervals.of("2015-01-01T00Z/2015-01-02T00Z"));
-    Map<Interval, List<DataSegment>> expectedSegments = new HashMap<>();
-    expectedSegments.put(
-        Intervals.of("2015-01-01T00Z/2015-01-02T00Z"),
-        Collections.singletonList(
-            new DataSegment(
-                "base",
-                Intervals.of("2015-01-01T00Z/2015-01-02T00Z"),
-                "2015-01-02",
-                ImmutableMap.of(),
-                ImmutableList.of("dim1", "dim2"),
-                ImmutableList.of("m1"),
-                new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null, null),
-                9,
-                1024
-            )
-        )
-    );
-    expectedSegments.put(
-        Intervals.of("2015-01-02T00Z/2015-01-03T00Z"),
-        Collections.singletonList(
-            new DataSegment(
-                "base",
-                Intervals.of("2015-01-02T00Z/2015-01-03T00Z"),
-                "2015-01-03",
-                ImmutableMap.of(),
-                ImmutableList.of("dim1", "dim2"),
-                ImmutableList.of("m1"),
-                new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null, null),
-                9,
-                1024
-            )
-        )
+
+    EasyMock.expect(taskQueue.add(EasyMock.anyObject()))
+            .andThrow(EntryAlreadyExists.exception("Task ID already exists"));
+
+    EasyMock.replay(taskMaster, taskStorage, taskQueue);
+
+    supervisor.checkSegmentsAndSubmitTasks();
+
+    EasyMock.verify(taskMaster, taskStorage, taskQueue);
+  }
+
+  @Test
+  public void testSubmitTasksFailsIfTaskCannotBeAdded() throws IOException
+  {
+    Set<DataSegment> baseSegments = Sets.newHashSet(createBaseSegments());
+    Set<DataSegment> derivativeSegments = 
Sets.newHashSet(createDerivativeSegments());
+
+    indexerMetadataStorageCoordinator.commitSegments(baseSegments);
+    indexerMetadataStorageCoordinator.commitSegments(derivativeSegments);
+    
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
+    
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
+    
EasyMock.expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes();
+
+    EasyMock.expect(taskQueue.add(EasyMock.anyObject()))
+            .andThrow(new ISE("Could not add task"));
+
+    EasyMock.replay(taskMaster, taskStorage, taskQueue);
+
+    ISE exception = Assert.assertThrows(
+        ISE.class,
+        () -> supervisor.checkSegmentsAndSubmitTasks()
     );
-    Assert.assertEquals(expectedToBuildInterval, toBuildInterval.lhs.keySet());
-    Assert.assertEquals(expectedSegments, toBuildInterval.rhs);
+    Assert.assertEquals("Could not add task", exception.getMessage());
+
+    EasyMock.verify(taskMaster, taskStorage, taskQueue);
   }
 
   @Test
   public void testCheckSegmentsAndSubmitTasks() throws IOException
   {
-    Set<DataSegment> baseSegments = Sets.newHashSet(
-        new DataSegment(
-            "base",
-            Intervals.of("2015-01-02T00Z/2015-01-03T00Z"),
-            "2015-01-03",
-            ImmutableMap.of(),
-            ImmutableList.of("dim1", "dim2"),
-            ImmutableList.of("m1"),
-            new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null, null),
-            9,
-            1024
-        )
-    );
+    Set<DataSegment> baseSegments = 
Collections.singleton(createBaseSegments().get(0));
     indexerMetadataStorageCoordinator.commitSegments(baseSegments);
     
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
     
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
@@ -315,28 +274,12 @@ public class MaterializedViewSupervisorTest
 
     Assert.assertEquals(expectedRunningTasks, runningTasks);
     Assert.assertEquals(expectedRunningVersion, runningVersion);
-
   }
 
-  /**
-   * Verifies that creating HadoopIndexTask compleates without raising 
exception.
-   */
   @Test
-  public void testCreateTask()
+  public void testCreateTaskSucceeds()
   {
-    List<DataSegment> baseSegments = Collections.singletonList(
-        new DataSegment(
-            "base",
-            Intervals.of("2015-01-02T00Z/2015-01-03T00Z"),
-            "2015-01-03",
-            ImmutableMap.of(),
-            ImmutableList.of("dim1", "dim2"),
-            ImmutableList.of("m1"),
-            new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null, null),
-            9,
-            1024
-        )
-    );
+    List<DataSegment> baseSegments = createBaseSegments().subList(0, 1);
 
     HadoopIndexTask task = spec.createTask(
         Intervals.of("2015-01-02T00Z/2015-01-03T00Z"),
@@ -348,7 +291,7 @@ public class MaterializedViewSupervisorTest
   }
 
   @Test
-  public void testSuspendedDoesntRun()
+  public void testSuspendedDoesNotRun()
   {
     MaterializedViewSupervisorSpec suspended = new 
MaterializedViewSupervisorSpec(
         "base",
@@ -378,10 +321,7 @@ public class MaterializedViewSupervisorTest
     // which will be true if truly suspended, since this is the first 
operation of the 'run' method otherwise
     IndexerSQLMetadataStorageCoordinator mock = 
EasyMock.createMock(IndexerSQLMetadataStorageCoordinator.class);
     
EasyMock.expect(mock.retrieveDataSourceMetadata(suspended.getDataSourceName()))
-            .andAnswer(() -> {
-              Assert.fail();
-              return null;
-            })
+            .andThrow(new AssertionFailedError())
             .anyTimes();
 
     EasyMock.replay(mock);
@@ -420,4 +360,36 @@ public class MaterializedViewSupervisorTest
         () -> supervisor.resetOffsets(null)
     );
   }
+
+  private List<DataSegment> createBaseSegments()
+  {
+    return Arrays.asList(
+        createSegment("base", "2015-01-01T00Z/2015-01-02T00Z", "2015-01-02"),
+        createSegment("base", "2015-01-02T00Z/2015-01-03T00Z", "2015-01-03"),
+        createSegment("base", "2015-01-03T00Z/2015-01-04T00Z", "2015-01-04")
+    );
+  }
+
+  private List<DataSegment> createDerivativeSegments()
+  {
+    return Arrays.asList(
+        createSegment(derivativeDatasourceName, 
"2015-01-01T00Z/2015-01-02T00Z", "2015-01-02"),
+        createSegment(derivativeDatasourceName, 
"2015-01-02T00Z/2015-01-03T00Z", "3015-01-01")
+    );
+  }
+
+  private DataSegment createSegment(String datasource, String interval, String 
version)
+  {
+    return new DataSegment(
+        datasource,
+        Intervals.of(interval),
+        version,
+        Collections.emptyMap(),
+        Arrays.asList("dim1", "dim2"),
+        Collections.singletonList("m2"),
+        new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null, null),
+        9,
+        1024
+    );
+  }
 }
diff --git 
a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java
 
b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java
index a2f3e98deca..a4ab6a30a80 100644
--- 
a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java
+++ 
b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java
@@ -79,7 +79,6 @@ import 
org.apache.druid.java.util.emitter.service.AlertBuilder;
 import org.apache.druid.java.util.emitter.service.AlertEvent;
 import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
 import org.apache.druid.java.util.metrics.StubServiceEmitter;
-import org.apache.druid.metadata.EntryExistsException;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.aggregation.CountAggregatorFactory;
 import org.apache.druid.segment.TestHelper;
@@ -3744,7 +3743,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
 
   @Test
   public void testDoNotKillCompatibleTasks()
-      throws InterruptedException, EntryExistsException
+      throws InterruptedException
   {
     // This supervisor always returns true for isTaskCurrent -> it should not 
kill its tasks
     int numReplicas = 2;
@@ -3840,8 +3839,7 @@ public class KinesisSupervisorTest extends EasyMockSupport
   }
 
   @Test
-  public void testKillIncompatibleTasks()
-      throws InterruptedException, EntryExistsException
+  public void testKillIncompatibleTasks() throws InterruptedException
   {
     // This supervisor always returns false for isTaskCurrent -> it should 
kill its tasks
     int numReplicas = 2;
diff --git 
a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/query/lookup/NamespaceLookupIntrospectHandler.java
 
b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/query/lookup/NamespaceLookupIntrospectHandler.java
index 889467324da..d95361a1bdf 100644
--- 
a/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/query/lookup/NamespaceLookupIntrospectHandler.java
+++ 
b/extensions-core/lookups-cached-global/src/main/java/org/apache/druid/query/lookup/NamespaceLookupIntrospectHandler.java
@@ -20,8 +20,8 @@
 package org.apache.druid.query.lookup;
 
 import com.google.common.collect.ImmutableMap;
-import org.apache.druid.common.utils.ServletResourceUtils;
 import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.server.http.ServletResourceUtils;
 import org.apache.druid.server.lookup.namespace.cache.CacheScheduler;
 
 import javax.ws.rs.GET;
diff --git 
a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java
 
b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java
index d01b4b4a3f8..e84976db86c 100644
--- 
a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java
+++ 
b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/HeapMemoryTaskStorage.java
@@ -28,6 +28,7 @@ import com.google.common.collect.Multimap;
 import com.google.common.collect.Ordering;
 import com.google.errorprone.annotations.concurrent.GuardedBy;
 import com.google.inject.Inject;
+import org.apache.druid.error.EntryAlreadyExists;
 import org.apache.druid.indexer.TaskInfo;
 import org.apache.druid.indexer.TaskStatus;
 import org.apache.druid.indexer.TaskStatusPlus;
@@ -37,7 +38,6 @@ import 
org.apache.druid.indexing.common.config.TaskStorageConfig;
 import org.apache.druid.indexing.common.task.Task;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.metadata.EntryExistsException;
 import org.apache.druid.metadata.TaskLookup;
 import org.apache.druid.metadata.TaskLookup.CompleteTaskLookup;
 import org.apache.druid.metadata.TaskLookup.TaskLookupType;
@@ -76,7 +76,7 @@ public class HeapMemoryTaskStorage implements TaskStorage
   }
 
   @Override
-  public void insert(Task task, TaskStatus status) throws EntryExistsException
+  public void insert(Task task, TaskStatus status)
   {
     Preconditions.checkNotNull(task, "task");
     Preconditions.checkNotNull(status, "status");
@@ -88,12 +88,12 @@ public class HeapMemoryTaskStorage implements TaskStorage
     );
 
     TaskStuff newTaskStuff = new TaskStuff(task, status, DateTimes.nowUtc(), 
task.getDataSource());
-    TaskStuff alreadyExisted = tasks.putIfAbsent(task.getId(), newTaskStuff);
-    if (alreadyExisted != null) {
-      throw new EntryExistsException("Task", task.getId());
+    TaskStuff existingTaskStuff = tasks.putIfAbsent(task.getId(), 
newTaskStuff);
+    if (existingTaskStuff != null) {
+      throw EntryAlreadyExists.exception("Task[%s] already exists", 
task.getId());
     }
 
-    log.info("Inserted task %s with status: %s", task.getId(), status);
+    log.info("Inserted task[%s] with status[%s]", task.getId(), status);
   }
 
   @Override
diff --git 
a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java
 
b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java
index 251a2c8ce31..15730d48bb1 100644
--- 
a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java
+++ 
b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/MetadataTaskStorage.java
@@ -25,7 +25,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import com.google.inject.Inject;
-import org.apache.druid.common.exception.DruidException;
+import org.apache.druid.error.DruidException;
 import org.apache.druid.indexer.TaskInfo;
 import org.apache.druid.indexer.TaskStatus;
 import org.apache.druid.indexer.TaskStatusPlus;
@@ -38,7 +38,6 @@ import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
 import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
 import org.apache.druid.java.util.emitter.EmittingLogger;
-import org.apache.druid.metadata.EntryExistsException;
 import org.apache.druid.metadata.MetadataStorageActionHandler;
 import org.apache.druid.metadata.MetadataStorageActionHandlerFactory;
 import org.apache.druid.metadata.MetadataStorageActionHandlerTypes;
@@ -126,7 +125,7 @@ public class MetadataTaskStorage implements TaskStorage
   }
 
   @Override
-  public void insert(final Task task, final TaskStatus status) throws 
EntryExistsException
+  public void insert(final Task task, final TaskStatus status)
   {
     Preconditions.checkNotNull(task, "task");
     Preconditions.checkNotNull(status, "status");
diff --git 
a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java
 
b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java
index 830ae9b732b..6dbf6e70798 100644
--- 
a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java
+++ 
b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java
@@ -32,6 +32,7 @@ import com.google.errorprone.annotations.concurrent.GuardedBy;
 import org.apache.druid.annotations.SuppressFBWarnings;
 import org.apache.druid.common.utils.IdUtils;
 import org.apache.druid.error.DruidException;
+import org.apache.druid.error.EntryAlreadyExists;
 import org.apache.druid.indexer.RunnerTaskState;
 import org.apache.druid.indexer.TaskLocation;
 import org.apache.druid.indexer.TaskStatus;
@@ -55,7 +56,6 @@ import 
org.apache.druid.java.util.common.lifecycle.LifecycleStop;
 import org.apache.druid.java.util.emitter.EmittingLogger;
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
 import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
-import org.apache.druid.metadata.EntryExistsException;
 import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
 import org.apache.druid.utils.CollectionUtils;
 
@@ -495,16 +495,14 @@ public class TaskQueue
    * @param task task to add
    *
    * @return true
-   *
-   * @throws EntryExistsException if the task already exists
    */
-  public boolean add(final Task task) throws EntryExistsException
+  public boolean add(final Task task)
   {
     // Before adding the task, validate the ID, so it can be safely used in 
file paths, znodes, etc.
     IdUtils.validateId("Task ID", task.getId());
 
     if (taskStorage.getTask(task.getId()).isPresent()) {
-      throw new EntryExistsException("Task", task.getId());
+      throw EntryAlreadyExists.exception("Task[%s] already exists", 
task.getId());
     }
 
     // Set forceTimeChunkLock before adding task spec to taskStorage, so that 
we can see always consistent task spec.
diff --git 
a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java
 
b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java
index e9b0af24057..235c763e1f7 100644
--- 
a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java
+++ 
b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskStorage.java
@@ -26,7 +26,6 @@ import org.apache.druid.indexer.TaskStatusPlus;
 import org.apache.druid.indexing.common.TaskLock;
 import org.apache.druid.indexing.common.actions.TaskAction;
 import org.apache.druid.indexing.common.task.Task;
-import org.apache.druid.metadata.EntryExistsException;
 import org.apache.druid.metadata.TaskLookup;
 import org.apache.druid.metadata.TaskLookup.TaskLookupType;
 
@@ -42,10 +41,8 @@ public interface TaskStorage
    *
    * @param task   task to add
    * @param status task status
-   *
-   * @throws EntryExistsException if the task ID already exists
    */
-  void insert(Task task, TaskStatus status) throws EntryExistsException;
+  void insert(Task task, TaskStatus status);
 
   /**
    * Persists task status in the storage facility. This method should throw an 
exception if the task status lifecycle
diff --git 
a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java
 
b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java
index 8416b2e0968..3e6ebc4816f 100644
--- 
a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java
+++ 
b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java
@@ -35,7 +35,6 @@ import org.apache.druid.client.indexing.ClientTaskQuery;
 import org.apache.druid.common.config.ConfigManager.SetResult;
 import org.apache.druid.common.config.JacksonConfigManager;
 import org.apache.druid.error.DruidException;
-import org.apache.druid.error.ErrorResponse;
 import org.apache.druid.indexer.RunnerTaskState;
 import org.apache.druid.indexer.TaskIdentifier;
 import org.apache.druid.indexer.TaskInfo;
@@ -71,6 +70,7 @@ import org.apache.druid.metadata.TaskLookup.ActiveTaskLookup;
 import org.apache.druid.metadata.TaskLookup.CompleteTaskLookup;
 import org.apache.druid.metadata.TaskLookup.TaskLookupType;
 import org.apache.druid.server.http.HttpMediaType;
+import org.apache.druid.server.http.ServletResourceUtils;
 import org.apache.druid.server.http.security.ConfigResourceFilter;
 import org.apache.druid.server.http.security.DatasourceResourceFilter;
 import org.apache.druid.server.http.security.StateResourceFilter;
@@ -241,15 +241,7 @@ public class OverlordResource
             return Response.ok(ImmutableMap.of("task", task.getId())).build();
           }
           catch (DruidException e) {
-            return Response
-                .status(e.getStatusCode())
-                .entity(new ErrorResponse(e))
-                .build();
-          }
-          catch (org.apache.druid.common.exception.DruidException e) {
-            return Response.status(e.getResponseCode())
-                           .entity(ImmutableMap.of("error", e.getMessage()))
-                           .build();
+            return ServletResourceUtils.buildErrorResponseFrom(e);
           }
         }
     );
diff --git 
a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
 
b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
index 5ea7a4168cb..507001a7af6 100644
--- 
a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
+++ 
b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
@@ -88,7 +88,6 @@ import org.apache.druid.java.util.common.concurrent.Execs;
 import org.apache.druid.java.util.emitter.EmittingLogger;
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
 import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
-import org.apache.druid.metadata.EntryExistsException;
 import org.apache.druid.metadata.MetadataSupervisorManager;
 import org.apache.druid.query.DruidMetrics;
 import org.apache.druid.query.ordering.StringComparators;
@@ -3934,9 +3933,9 @@ public abstract class 
SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
         try {
           taskQueue.get().add(indexTask);
         }
-        catch (EntryExistsException e) {
+        catch (DruidException e) {
           stateManager.recordThrowableEvent(e);
-          log.error("Tried to add task [%s] but it already exists", 
indexTask.getId());
+          log.noStackTrace().error(e, "Tried to add task [%s] but encountered 
error", indexTask.getId());
         }
       } else {
         log.error("Failed to get task queue because I'm not the leader!");
diff --git 
a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
 
b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
index b864fdb44a5..ef769d73006 100644
--- 
a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
+++ 
b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java
@@ -48,6 +48,7 @@ import org.apache.druid.data.input.impl.TimestampSpec;
 import org.apache.druid.discovery.DataNodeService;
 import org.apache.druid.discovery.DruidNodeAnnouncer;
 import org.apache.druid.discovery.LookupNodeService;
+import org.apache.druid.error.DruidException;
 import org.apache.druid.indexer.IngestionState;
 import org.apache.druid.indexer.TaskState;
 import org.apache.druid.indexer.TaskStatus;
@@ -92,7 +93,6 @@ import org.apache.druid.java.util.emitter.core.NoopEmitter;
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
 import org.apache.druid.java.util.metrics.MonitorScheduler;
 import org.apache.druid.math.expr.ExprMacroTable;
-import org.apache.druid.metadata.EntryExistsException;
 import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator;
 import org.apache.druid.metadata.TestDerbyConnector;
 import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate;
@@ -1302,8 +1302,8 @@ public class AppenderatorDriverRealtimeIndexTaskTest 
extends InitializedNullHand
     try {
       taskStorage.insert(task, TaskStatus.running(task.getId()));
     }
-    catch (EntryExistsException e) {
-      // suppress
+    catch (DruidException e) {
+      log.noStackTrace().info(e, "Suppressing exception while inserting task 
[%s]", task.getId());
     }
     taskLockbox.syncFromStorage();
     final TaskToolbox toolbox = taskToolboxFactory.build(task);
diff --git 
a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java
 
b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java
index bec9bd135e7..b82dafccc6b 100644
--- 
a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java
+++ 
b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java
@@ -62,7 +62,6 @@ import org.apache.druid.java.util.common.Pair;
 import org.apache.druid.java.util.common.RE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.emitter.EmittingLogger;
-import org.apache.druid.metadata.EntryExistsException;
 import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator;
 import org.apache.druid.metadata.SQLMetadataConnector;
 import org.apache.druid.metadata.SegmentsMetadataManager;
@@ -158,7 +157,7 @@ public abstract class IngestionTestBase extends 
InitializedNullHandlingTest
     return new TestLocalTaskActionClient(task);
   }
 
-  public void prepareTaskForLocking(Task task) throws EntryExistsException
+  public void prepareTaskForLocking(Task task)
   {
     lockbox.add(task);
     taskStorage.insert(task, TaskStatus.running(task.getId()));
diff --git 
a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java
 
b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java
index 2e3e7d5d8ce..43253a10bcc 100644
--- 
a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java
+++ 
b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java
@@ -41,6 +41,7 @@ import org.apache.druid.data.input.impl.TimestampSpec;
 import org.apache.druid.discovery.DataNodeService;
 import org.apache.druid.discovery.DruidNodeAnnouncer;
 import org.apache.druid.discovery.LookupNodeService;
+import org.apache.druid.error.DruidException;
 import org.apache.druid.indexer.TaskState;
 import org.apache.druid.indexer.TaskStatus;
 import org.apache.druid.indexing.common.SegmentCacheManagerFactory;
@@ -80,7 +81,6 @@ import org.apache.druid.java.util.emitter.core.NoopEmitter;
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
 import org.apache.druid.java.util.metrics.MonitorScheduler;
 import org.apache.druid.math.expr.ExprMacroTable;
-import org.apache.druid.metadata.EntryExistsException;
 import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate;
 import org.apache.druid.query.DirectQueryProcessingPool;
 import org.apache.druid.query.Druids;
@@ -911,8 +911,8 @@ public class RealtimeIndexTaskTest extends 
InitializedNullHandlingTest
     try {
       taskStorage.insert(task, TaskStatus.running(task.getId()));
     }
-    catch (EntryExistsException e) {
-      // suppress
+    catch (DruidException e) {
+      log.noStackTrace().info(e, "Suppressing exception while inserting task 
[%s]", task.getId());
     }
     taskLockbox.syncFromStorage();
     final TaskActionToolbox taskActionToolbox = new TaskActionToolbox(
diff --git 
a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java
 
b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java
index f438b7bba84..6b662e473b0 100644
--- 
a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java
+++ 
b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java
@@ -83,7 +83,6 @@ import org.apache.druid.java.util.common.concurrent.Execs;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.java.util.metrics.StubServiceEmitter;
 import org.apache.druid.math.expr.ExprMacroTable;
-import org.apache.druid.metadata.EntryExistsException;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
 import org.apache.druid.query.expression.LookupEnabledTestExprMacroTable;
@@ -451,12 +450,7 @@ public class AbstractParallelIndexSupervisorTaskTest 
extends IngestionTestBase
       if (tasks.put(task.getId(), taskContainer) != null) {
         throw new ISE("Duplicate task ID[%s]", task.getId());
       }
-      try {
-        prepareTaskForLocking(task);
-      }
-      catch (EntryExistsException e) {
-        throw new RuntimeException(e);
-      }
+      prepareTaskForLocking(task);
       task.addToContextIfAbsent(
           
SinglePhaseParallelIndexTaskRunner.CTX_USE_LINEAGE_BASED_SEGMENT_ALLOCATION_KEY,
           
SinglePhaseParallelIndexTaskRunner.DEFAULT_USE_LINEAGE_BASED_SEGMENT_ALLOCATION
diff --git 
a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java
 
b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java
index 8e809169b30..19af66254ba 100644
--- 
a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java
+++ 
b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockBoxConcurrencyTest.java
@@ -31,7 +31,6 @@ import org.apache.druid.jackson.DefaultObjectMapper;
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.concurrent.Execs;
 import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory;
-import org.apache.druid.metadata.EntryExistsException;
 import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator;
 import org.apache.druid.metadata.TestDerbyConnector;
 import org.joda.time.Interval;
@@ -100,7 +99,7 @@ public class TaskLockBoxConcurrencyTest
 
   @Test(timeout = 60_000L)
   public void testDoInCriticalSectionWithDifferentTasks()
-      throws ExecutionException, InterruptedException, EntryExistsException
+      throws ExecutionException, InterruptedException
   {
     final Interval interval = Intervals.of("2017-01-01/2017-01-02");
     final Task lowPriorityTask = NoopTask.ofPriority(10);
diff --git 
a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockConfigTest.java
 
b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockConfigTest.java
index 364b492148d..7d2fbcd7923 100644
--- 
a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockConfigTest.java
+++ 
b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockConfigTest.java
@@ -31,7 +31,6 @@ import 
org.apache.druid.indexing.overlord.config.TaskLockConfig;
 import org.apache.druid.indexing.overlord.config.TaskQueueConfig;
 import org.apache.druid.indexing.test.TestIndexerMetadataStorageCoordinator;
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
-import org.apache.druid.metadata.EntryExistsException;
 import org.apache.druid.server.metrics.NoopServiceEmitter;
 import org.easymock.EasyMock;
 import org.junit.Assert;
@@ -51,7 +50,7 @@ public class TaskLockConfigTest
   }
 
   @Test
-  public void testDefault() throws EntryExistsException
+  public void testDefault()
   {
     final TaskQueue taskQueue = createTaskQueue(null);
     taskQueue.start();
@@ -65,7 +64,7 @@ public class TaskLockConfigTest
   }
 
   @Test
-  public void testNotForceTimeChunkLock() throws EntryExistsException
+  public void testNotForceTimeChunkLock()
   {
     final TaskQueue taskQueue = createTaskQueue(false);
     taskQueue.start();
@@ -79,7 +78,7 @@ public class TaskLockConfigTest
   }
 
   @Test
-  public void testOverwriteDefault() throws EntryExistsException
+  public void testOverwriteDefault()
   {
     final TaskQueue taskQueue = createTaskQueue(null);
     taskQueue.start();
diff --git 
a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java
 
b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java
index c4ee78ea6a8..307879fed8b 100644
--- 
a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java
+++ 
b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java
@@ -52,7 +52,6 @@ import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.emitter.EmittingLogger;
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
 import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory;
-import org.apache.druid.metadata.EntryExistsException;
 import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator;
 import org.apache.druid.metadata.LockFilterPolicy;
 import org.apache.druid.metadata.MetadataStorageTablesConfig;
@@ -228,7 +227,7 @@ public class TaskLockboxTest
   }
 
   @Test
-  public void testTryMixedLocks() throws EntryExistsException
+  public void testTryMixedLocks()
   {
     final Task lowPriorityTask = NoopTask.ofPriority(0);
     final Task lowPriorityTask2 = NoopTask.ofPriority(0);
@@ -320,7 +319,7 @@ public class TaskLockboxTest
   }
 
   @Test
-  public void testSyncFromStorage() throws EntryExistsException
+  public void testSyncFromStorage()
   {
     final TaskLockbox originalBox = new TaskLockbox(taskStorage, 
metadataStorageCoordinator);
     for (int i = 0; i < 5; i++) {
@@ -358,7 +357,7 @@ public class TaskLockboxTest
   }
 
   @Test
-  public void testSyncFromStorageWithMissingTaskLockPriority() throws 
EntryExistsException
+  public void testSyncFromStorageWithMissingTaskLockPriority()
   {
     final Task task = NoopTask.create();
     taskStorage.insert(task, TaskStatus.running(task.getId()));
@@ -382,7 +381,7 @@ public class TaskLockboxTest
   }
 
   @Test
-  public void testSyncFromStorageWithMissingTaskPriority() throws 
EntryExistsException
+  public void testSyncFromStorageWithMissingTaskPriority()
   {
     final Task task = NoopTask.create();
     taskStorage.insert(task, TaskStatus.running(task.getId()));
@@ -413,7 +412,7 @@ public class TaskLockboxTest
   }
 
   @Test
-  public void testSyncFromStorageWithInvalidPriority() throws 
EntryExistsException
+  public void testSyncFromStorageWithInvalidPriority()
   {
     final Task task = NoopTask.create();
     taskStorage.insert(task, TaskStatus.running(task.getId()));
@@ -481,7 +480,7 @@ public class TaskLockboxTest
   }
 
   @Test
-  public void testRevokedLockSyncFromStorage() throws EntryExistsException
+  public void testRevokedLockSyncFromStorage()
   {
     final TaskLockbox originalBox = new TaskLockbox(taskStorage, 
metadataStorageCoordinator);
 
@@ -628,7 +627,7 @@ public class TaskLockboxTest
   }
 
   @Test(timeout = 60_000L)
-  public void testAcquireLockAfterRevoked() throws EntryExistsException, 
InterruptedException
+  public void testAcquireLockAfterRevoked() throws InterruptedException
   {
     final Interval interval = Intervals.of("2017-01-01/2017-01-02");
     final Task lowPriorityTask = NoopTask.ofPriority(0);
@@ -653,7 +652,7 @@ public class TaskLockboxTest
   }
 
   @Test
-  public void testUnlock() throws EntryExistsException
+  public void testUnlock()
   {
     final List<Task> lowPriorityTasks = new ArrayList<>();
     final List<Task> highPriorityTasks = new ArrayList<>();
@@ -718,7 +717,7 @@ public class TaskLockboxTest
   }
 
   @Test
-  public void testFindLockPosseAfterRevokeWithDifferentLockIntervals() throws 
EntryExistsException
+  public void testFindLockPosseAfterRevokeWithDifferentLockIntervals()
   {
     final Task lowPriorityTask = NoopTask.ofPriority(0);
     final Task highPriorityTask = NoopTask.ofPriority(10);
@@ -828,7 +827,7 @@ public class TaskLockboxTest
   }
 
   @Test
-  public void 
testSegmentAndTimeChunkLockForSameIntervalWithDifferentPriority() throws 
EntryExistsException
+  public void testSegmentAndTimeChunkLockForSameIntervalWithDifferentPriority()
   {
     final Task task1 = NoopTask.ofPriority(10);
     lockbox.add(task1);
diff --git 
a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java
 
b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java
index 8ca1ff49f05..8ee341c5db1 100644
--- 
a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java
+++ 
b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java
@@ -64,7 +64,6 @@ import 
org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.granularity.Granularity;
 import org.apache.druid.java.util.http.client.HttpClient;
 import org.apache.druid.java.util.metrics.StubServiceEmitter;
-import org.apache.druid.metadata.EntryExistsException;
 import org.apache.druid.segment.TestHelper;
 import org.apache.druid.server.coordinator.stats.CoordinatorRunStats;
 import org.apache.druid.server.initialization.IndexerZkConfig;
@@ -209,7 +208,7 @@ public class TaskQueueTest extends IngestionTestBase
   }
 
   @Test
-  public void testSetUseLineageBasedSegmentAllocationByDefault() throws 
EntryExistsException
+  public void testSetUseLineageBasedSegmentAllocationByDefault()
   {
     final TaskActionClientFactory actionClientFactory = 
createActionClientFactory();
     final TaskQueue taskQueue = new TaskQueue(
@@ -234,7 +233,7 @@ public class TaskQueueTest extends IngestionTestBase
   }
 
   @Test
-  public void 
testDefaultTaskContextOverrideDefaultLineageBasedSegmentAllocation() throws 
EntryExistsException
+  public void 
testDefaultTaskContextOverrideDefaultLineageBasedSegmentAllocation()
   {
     final TaskActionClientFactory actionClientFactory = 
createActionClientFactory();
     final TaskQueue taskQueue = new TaskQueue(
@@ -269,7 +268,7 @@ public class TaskQueueTest extends IngestionTestBase
   }
 
   @Test
-  public void 
testUserProvidedTaskContextOverrideDefaultLineageBasedSegmentAllocation() 
throws EntryExistsException
+  public void 
testUserProvidedTaskContextOverrideDefaultLineageBasedSegmentAllocation()
   {
     final TaskActionClientFactory actionClientFactory = 
createActionClientFactory();
     final TaskQueue taskQueue = new TaskQueue(
@@ -301,7 +300,7 @@ public class TaskQueueTest extends IngestionTestBase
   }
 
   @Test
-  public void testLockConfigTakePrecedenceThanDefaultTaskContext() throws 
EntryExistsException
+  public void testLockConfigTakePrecedenceThanDefaultTaskContext()
   {
     final TaskActionClientFactory actionClientFactory = 
createActionClientFactory();
     final TaskQueue taskQueue = new TaskQueue(
@@ -334,7 +333,7 @@ public class TaskQueueTest extends IngestionTestBase
   }
 
   @Test
-  public void testUserProvidedContextOverrideLockConfig() throws 
EntryExistsException
+  public void testUserProvidedContextOverrideLockConfig()
   {
     final TaskActionClientFactory actionClientFactory = 
createActionClientFactory();
     final TaskQueue taskQueue = new TaskQueue(
@@ -364,7 +363,7 @@ public class TaskQueueTest extends IngestionTestBase
   }
 
   @Test
-  public void testTaskStatusWhenExceptionIsThrownInIsReady() throws 
EntryExistsException
+  public void testTaskStatusWhenExceptionIsThrownInIsReady()
   {
     final TaskActionClientFactory actionClientFactory = 
createActionClientFactory();
     final TaskQueue taskQueue = new TaskQueue(
@@ -400,7 +399,7 @@ public class TaskQueueTest extends IngestionTestBase
   }
 
   @Test
-  public void testKilledTasksEmitRuntimeMetricWithHttpRemote() throws 
EntryExistsException, InterruptedException
+  public void testKilledTasksEmitRuntimeMetricWithHttpRemote() throws 
InterruptedException
   {
     final TaskActionClientFactory actionClientFactory = 
createActionClientFactory();
     final HttpRemoteTaskRunner taskRunner = 
createHttpRemoteTaskRunner(ImmutableList.of("t1"));
diff --git 
a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java
 
b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java
index de854d1be94..2db5da143ed 100644
--- 
a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java
+++ 
b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java
@@ -50,6 +50,7 @@ import org.apache.druid.data.input.impl.TimestampSpec;
 import org.apache.druid.discovery.DataNodeService;
 import org.apache.druid.discovery.DruidNodeAnnouncer;
 import org.apache.druid.discovery.LookupNodeService;
+import org.apache.druid.error.DruidException;
 import org.apache.druid.indexer.TaskStatus;
 import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
 import org.apache.druid.indexing.common.LockGranularity;
@@ -88,7 +89,6 @@ import org.apache.druid.java.util.common.parsers.JSONPathSpec;
 import org.apache.druid.java.util.emitter.service.ServiceEmitter;
 import org.apache.druid.java.util.metrics.MonitorScheduler;
 import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory;
-import org.apache.druid.metadata.EntryExistsException;
 import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator;
 import org.apache.druid.metadata.TestDerbyConnector;
 import org.apache.druid.query.DirectQueryProcessingPool;
@@ -154,6 +154,8 @@ import java.util.stream.Collectors;
 
 public abstract class SeekableStreamIndexTaskTestBase extends EasyMockSupport
 {
+  private static final Logger log = new 
Logger(SeekableStreamIndexTaskTestBase.class);
+
   @Rule
   public final TemporaryFolder tempFolder = new TemporaryFolder();
 
@@ -475,8 +477,8 @@ public abstract class SeekableStreamIndexTaskTestBase 
extends EasyMockSupport
     try {
       taskStorage.insert(task, TaskStatus.running(task.getId()));
     }
-    catch (EntryExistsException e) {
-      // suppress
+    catch (DruidException e) {
+      log.noStackTrace().info(e, "Suppressing exception while inserting task 
[%s]", task.getId());
     }
     taskLockbox.syncFromStorage();
     final TaskToolbox toolbox = toolboxFactory.build(task);
diff --git 
a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java
 
b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java
index ea083423eaa..2602f8e5441 100644
--- 
a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java
+++ 
b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java
@@ -79,7 +79,6 @@ import 
org.apache.druid.java.util.common.granularity.Granularities;
 import org.apache.druid.java.util.common.parsers.JSONPathSpec;
 import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
 import org.apache.druid.java.util.metrics.StubServiceEmitter;
-import org.apache.druid.metadata.EntryExistsException;
 import org.apache.druid.query.DruidMetrics;
 import org.apache.druid.query.aggregation.AggregatorFactory;
 import org.apache.druid.query.aggregation.CountAggregatorFactory;
@@ -2074,7 +2073,7 @@ public class SeekableStreamSupervisorStateTest extends 
EasyMockSupport
     EasyMock.verify(executorService, spec);
   }
 
-  private void expectEmitterSupervisor(boolean suspended) throws 
EntryExistsException
+  private void expectEmitterSupervisor(boolean suspended)
   {
     spec = createMock(SeekableStreamSupervisorSpec.class);
     
EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes();
diff --git 
a/processing/src/main/java/org/apache/druid/common/exception/DruidException.java
 
b/processing/src/main/java/org/apache/druid/common/exception/DruidException.java
deleted file mode 100644
index 42a679b6bd2..00000000000
--- 
a/processing/src/main/java/org/apache/druid/common/exception/DruidException.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.druid.common.exception;
-
-/**
- * A generic exception thrown by Druid.
- *
- * This class is deprecated and should not be used.  {@link 
org.apache.druid.error.DruidException} should be used for
- * any error that is intended to be delivered to the end user.
- */
-@Deprecated
-public class DruidException extends RuntimeException
-{
-  public static final int HTTP_CODE_SERVER_ERROR = 500;
-  public static final int HTTP_CODE_BAD_REQUEST = 400;
-
-  private final int responseCode;
-  private final boolean isTransient;
-
-  public DruidException(String message, int responseCode, Throwable cause, 
boolean isTransient)
-  {
-    super(message, cause);
-    this.responseCode = responseCode;
-    this.isTransient = isTransient;
-  }
-
-  public int getResponseCode()
-  {
-    return responseCode;
-  }
-
-  /**
-   * Returns true if this is a transient exception and might go away if the
-   * operation is retried.
-   */
-  public boolean isTransient()
-  {
-    return isTransient;
-  }
-}
diff --git a/processing/src/main/java/org/apache/druid/error/NotFound.java 
b/processing/src/main/java/org/apache/druid/error/BaseFailure.java
similarity index 66%
copy from processing/src/main/java/org/apache/druid/error/NotFound.java
copy to processing/src/main/java/org/apache/druid/error/BaseFailure.java
index 03d2a107702..a18186ef93f 100644
--- a/processing/src/main/java/org/apache/druid/error/NotFound.java
+++ b/processing/src/main/java/org/apache/druid/error/BaseFailure.java
@@ -19,42 +19,38 @@
 
 package org.apache.druid.error;
 
-public class NotFound extends DruidException.Failure
+/**
+ * {@link DruidException.Failure} for a specific errorCode, category and 
persona.
+ */
+public abstract class BaseFailure extends DruidException.Failure
 {
-
-  public static DruidException exception(String msg, Object... args)
-  {
-    return exception(null, msg, args);
-  }
-
-  public static DruidException exception(Throwable t, String msg, Object... 
args)
-  {
-    return DruidException.fromFailure(new NotFound(t, msg, args));
-  }
-
   private final Throwable t;
   private final String msg;
   private final Object[] args;
+  private final DruidException.Persona persona;
+  private final DruidException.Category category;
 
-  public NotFound(
+  protected BaseFailure(
+      String errorCode,
+      DruidException.Persona persona,
+      DruidException.Category category,
       Throwable t,
       String msg,
       Object... args
   )
   {
-    super("notFound");
+    super(errorCode);
+    this.persona = persona;
+    this.category = category;
     this.t = t;
     this.msg = msg;
     this.args = args;
   }
 
-
   @Override
-  public DruidException makeException(DruidException.DruidExceptionBuilder bob)
+  protected DruidException makeException(DruidException.DruidExceptionBuilder 
bob)
   {
-    bob = bob.forPersona(DruidException.Persona.USER)
-             .ofCategory(DruidException.Category.NOT_FOUND);
-
+    bob = bob.forPersona(persona).ofCategory(category);
     if (t == null) {
       return bob.build(msg, args);
     } else {
diff --git 
a/processing/src/main/java/org/apache/druid/error/DruidException.java 
b/processing/src/main/java/org/apache/druid/error/DruidException.java
index 61f2b657b45..c4ec6bcb115 100644
--- a/processing/src/main/java/org/apache/druid/error/DruidException.java
+++ b/processing/src/main/java/org/apache/druid/error/DruidException.java
@@ -36,18 +36,18 @@ import java.util.Map;
  * be using the DruidException.
  * <p>
  * Said another way, when a developer builds a DruidException in the code, 
they should be confident that the exception
- * will make its way back to the user.  DruidException is always the answer to 
"how do I generate an error message and
+ * will make its way back to the user.  DruidException is always the answer to 
"How do I generate an error message and
  * deliver it to the user"?
  * <p>
  * At the time that DruidException was introduced, this type of "show this to 
the user please" exception was largely
  * handled by created {@link org.apache.druid.java.util.common.RE}, {@link 
org.apache.druid.java.util.common.IAE}, or
  * {@link org.apache.druid.java.util.common.ISE} objects.  It is intended that 
DruidException replaces all usage of
  * these exceptions where the intention is to deliver a message to the user, 
which we believe to be the vast majority
- * of usages.  In cases where those exceptions are with the intention of being 
caught and acted upon, they should
+ * of usages.  In cases where those exceptions are thrown with the intention 
of being caught and acted upon,
  * no change should occur.
- *
- * <h>Notes about exception messages</h>
  * <p>
+ * <h3>Notes about exception messages:</h3>
+ *
  * Firstly, exception messages should always be written with the notions from 
the style conventions covered in
  * {@code dev/style-conventions.md}.  Whenever possible, we should also try to 
provide an action to take to resolve
  * the issue.
@@ -78,8 +78,8 @@ import java.util.Map;
  * which is something that we would expect an operator to be in charge of.  
So, we would pick the OPERATOR persona
  * message, which also allows us to include more specific information about 
what server was not found and provide a
  * more meaningful action to take (check the health of your brokers).
- *
- * <h>Description of fields of DruidException</h>
+ * <p>
+ * <h3>Description of fields of DruidException</h3>
  * Every error consists of:
  * <ul>
  * <li>A target persona</li>
@@ -90,7 +90,7 @@ import java.util.Map;
  * </ul>
  * <p>
  * <p>
- * The target persona indicates who the message is written for.  This is 
important for 2 reasons
+ * The target persona indicates who the message is written for.  This is 
important for 2 reasons.
  * <ol>
  *   <li>It identifies why the developer is creating the exception and who 
they believe can take action on it.
  *   This context allows for code reviewers and other developers to evaluate 
the message with the persona in mind</li>
@@ -114,8 +114,8 @@ import java.util.Map;
  * The context is a place to add extra information about the error that is not 
necessarily interpolated into the
  * error message.  It's a way to carry extra information that might be useful 
to a developer, but not necessarily to
  * the target persona.
- *
- * <h>Notes for developers working with DruidException</h>
+ * <p>
+ * <h3>Notes for developers working with DruidException:</h3>
  * <p>
  * A DruidException can be built from one of 2 static methods: {@link 
#forPersona} or {@link #fromFailure(Failure)}.
  * The only way to set a specific error code is to build a DruidException from 
a Failure, when built in-line using
@@ -131,7 +131,7 @@ import java.util.Map;
 public class DruidException extends RuntimeException
 {
   /**
-   * Starts building a "general" DruidException targeting the specific persona.
+   * Starts building a "general" DruidException targeting the specified 
persona.
    *
    * @param persona the target persona of the exception message
    * @return a builder that can be used to complete the creation of the 
DruidException
@@ -386,8 +386,8 @@ public class DruidException extends RuntimeException
 
   public static class PartialDruidExceptionBuilder
   {
-    private String errorCode;
-    private Persona targetPersona;
+    private final String errorCode;
+    private final Persona targetPersona;
 
     private PartialDruidExceptionBuilder(String errorCode, Persona 
targetPersona)
     {
diff --git a/processing/src/main/java/org/apache/druid/error/NotFound.java 
b/processing/src/main/java/org/apache/druid/error/EntryAlreadyExists.java
similarity index 63%
copy from processing/src/main/java/org/apache/druid/error/NotFound.java
copy to processing/src/main/java/org/apache/druid/error/EntryAlreadyExists.java
index 03d2a107702..2e6966ce315 100644
--- a/processing/src/main/java/org/apache/druid/error/NotFound.java
+++ b/processing/src/main/java/org/apache/druid/error/EntryAlreadyExists.java
@@ -19,8 +19,14 @@
 
 package org.apache.druid.error;
 
-public class NotFound extends DruidException.Failure
+/**
+ * A failure type used to make {@link DruidException}s of category
+ * {@link DruidException.Category#INVALID_INPUT} for persona {@link 
DruidException.Persona#USER},
+ * denoting that a certain entry already exists.
+ */
+public class EntryAlreadyExists extends BaseFailure
 {
+  public static final String ERROR_CODE = "entryAlreadyExists";
 
   public static DruidException exception(String msg, Object... args)
   {
@@ -29,36 +35,20 @@ public class NotFound extends DruidException.Failure
 
   public static DruidException exception(Throwable t, String msg, Object... 
args)
   {
-    return DruidException.fromFailure(new NotFound(t, msg, args));
+    return DruidException.fromFailure(new EntryAlreadyExists(t, msg, args));
   }
 
-  private final Throwable t;
-  private final String msg;
-  private final Object[] args;
-
-  public NotFound(
+  public EntryAlreadyExists(
       Throwable t,
       String msg,
       Object... args
   )
   {
-    super("notFound");
-    this.t = t;
-    this.msg = msg;
-    this.args = args;
-  }
-
-
-  @Override
-  public DruidException makeException(DruidException.DruidExceptionBuilder bob)
-  {
-    bob = bob.forPersona(DruidException.Persona.USER)
-             .ofCategory(DruidException.Category.NOT_FOUND);
-
-    if (t == null) {
-      return bob.build(msg, args);
-    } else {
-      return bob.build(t, msg, args);
-    }
+    super(
+        ERROR_CODE,
+        DruidException.Persona.USER,
+        DruidException.Category.INVALID_INPUT,
+        t, msg, args
+    );
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/error/Forbidden.java 
b/processing/src/main/java/org/apache/druid/error/Forbidden.java
index 13470d241c4..b06431617af 100644
--- a/processing/src/main/java/org/apache/druid/error/Forbidden.java
+++ b/processing/src/main/java/org/apache/druid/error/Forbidden.java
@@ -19,9 +19,8 @@
 
 package org.apache.druid.error;
 
-public class Forbidden extends DruidException.Failure
+public class Forbidden extends BaseFailure
 {
-
   public static DruidException exception()
   {
     return exception("Unauthorized");
@@ -37,32 +36,18 @@ public class Forbidden extends DruidException.Failure
     return DruidException.fromFailure(new Forbidden(t, msg, args));
   }
 
-  private final Throwable t;
-  private final String msg;
-  private final Object[] args;
-
   private Forbidden(
       Throwable t,
       String msg,
       Object... args
   )
   {
-    super("forbidden");
-    this.t = t;
-    this.msg = msg;
-    this.args = args;
+    super(
+        "forbidden",
+        DruidException.Persona.USER,
+        DruidException.Category.FORBIDDEN,
+        t, msg, args
+    );
   }
 
-  @Override
-  public DruidException makeException(DruidException.DruidExceptionBuilder bob)
-  {
-    bob = bob.forPersona(DruidException.Persona.USER)
-             .ofCategory(DruidException.Category.FORBIDDEN);
-
-    if (t == null) {
-      return bob.build(msg, args);
-    } else {
-      return bob.build(t, msg, args);
-    }
-  }
 }
diff --git 
a/processing/src/main/java/org/apache/druid/error/InternalServerError.java 
b/processing/src/main/java/org/apache/druid/error/InternalServerError.java
index b730acb0e3d..4ba221920d5 100644
--- a/processing/src/main/java/org/apache/druid/error/InternalServerError.java
+++ b/processing/src/main/java/org/apache/druid/error/InternalServerError.java
@@ -19,21 +19,18 @@
 
 package org.apache.druid.error;
 
-public class InternalServerError extends DruidException.Failure
+public class InternalServerError extends BaseFailure
 {
   public static DruidException exception(String errorCode, String msg, 
Object... args)
   {
     return exception(null, errorCode, msg, args);
   }
+
   public static DruidException exception(Throwable t, String errorCode, String 
msg, Object... args)
   {
     return DruidException.fromFailure(new InternalServerError(t, errorCode, 
msg, args));
   }
 
-  private final Throwable t;
-  private final String msg;
-  private final Object[] args;
-
   private InternalServerError(
       Throwable t,
       String errorCode,
@@ -41,22 +38,11 @@ public class InternalServerError extends 
DruidException.Failure
       Object... args
   )
   {
-    super(errorCode);
-    this.t = t;
-    this.msg = msg;
-    this.args = args;
-  }
-
-  @Override
-  public DruidException makeException(DruidException.DruidExceptionBuilder bob)
-  {
-    bob = bob.forPersona(DruidException.Persona.OPERATOR)
-        .ofCategory(DruidException.Category.RUNTIME_FAILURE);
-
-    if (t == null) {
-      return bob.build(msg, args);
-    } else {
-      return bob.build(t, msg, args);
-    }
+    super(
+        errorCode,
+        DruidException.Persona.OPERATOR,
+        DruidException.Category.RUNTIME_FAILURE,
+        t, msg, args
+    );
   }
 }
diff --git a/processing/src/main/java/org/apache/druid/error/InvalidInput.java 
b/processing/src/main/java/org/apache/druid/error/InvalidInput.java
index ce50d4db376..136dfca5077 100644
--- a/processing/src/main/java/org/apache/druid/error/InvalidInput.java
+++ b/processing/src/main/java/org/apache/druid/error/InvalidInput.java
@@ -19,7 +19,11 @@
 
 package org.apache.druid.error;
 
-public class InvalidInput extends DruidException.Failure
+/**
+ * A failure type used to make {@link DruidException}s of category
+ * {@link DruidException.Category#INVALID_INPUT} for persona {@link 
DruidException.Persona#USER}.
+ */
+public class InvalidInput extends BaseFailure
 {
   public static DruidException exception(String msg, Object... args)
   {
@@ -31,33 +35,18 @@ public class InvalidInput extends DruidException.Failure
     return DruidException.fromFailure(new InvalidInput(t, msg, args));
   }
 
-  private final Throwable t;
-  private final String msg;
-  private final Object[] args;
-
   public InvalidInput(
       Throwable t,
       String msg,
       Object... args
   )
   {
-    super("invalidInput");
-    this.t = t;
-    this.msg = msg;
-    this.args = args;
+    super(
+        "invalidInput",
+        DruidException.Persona.USER,
+        DruidException.Category.INVALID_INPUT,
+        t, msg, args
+    );
   }
 
-
-  @Override
-  public DruidException makeException(DruidException.DruidExceptionBuilder bob)
-  {
-    bob = bob.forPersona(DruidException.Persona.USER)
-             .ofCategory(DruidException.Category.INVALID_INPUT);
-
-    if (t == null) {
-      return bob.build(msg, args);
-    } else {
-      return bob.build(t, msg, args);
-    }
-  }
 }
diff --git a/processing/src/main/java/org/apache/druid/error/NotFound.java 
b/processing/src/main/java/org/apache/druid/error/NotFound.java
index 03d2a107702..cc2a731177b 100644
--- a/processing/src/main/java/org/apache/druid/error/NotFound.java
+++ b/processing/src/main/java/org/apache/druid/error/NotFound.java
@@ -19,9 +19,8 @@
 
 package org.apache.druid.error;
 
-public class NotFound extends DruidException.Failure
+public class NotFound extends BaseFailure
 {
-
   public static DruidException exception(String msg, Object... args)
   {
     return exception(null, msg, args);
@@ -32,33 +31,17 @@ public class NotFound extends DruidException.Failure
     return DruidException.fromFailure(new NotFound(t, msg, args));
   }
 
-  private final Throwable t;
-  private final String msg;
-  private final Object[] args;
-
   public NotFound(
       Throwable t,
       String msg,
       Object... args
   )
   {
-    super("notFound");
-    this.t = t;
-    this.msg = msg;
-    this.args = args;
-  }
-
-
-  @Override
-  public DruidException makeException(DruidException.DruidExceptionBuilder bob)
-  {
-    bob = bob.forPersona(DruidException.Persona.USER)
-             .ofCategory(DruidException.Category.NOT_FOUND);
-
-    if (t == null) {
-      return bob.build(msg, args);
-    } else {
-      return bob.build(t, msg, args);
-    }
+    super(
+        "notFound",
+        DruidException.Persona.USER,
+        DruidException.Category.NOT_FOUND,
+        t, msg, args
+    );
   }
 }
diff --git 
a/processing/src/main/java/org/apache/druid/metadata/EntryExistsException.java 
b/processing/src/main/java/org/apache/druid/metadata/EntryExistsException.java
deleted file mode 100644
index 445ad7ad7b3..00000000000
--- 
a/processing/src/main/java/org/apache/druid/metadata/EntryExistsException.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.druid.metadata;
-
-import org.apache.druid.common.exception.DruidException;
-import org.apache.druid.java.util.common.StringUtils;
-
-/**
- * A non-transient Druid metadata exception thrown when trying to insert a
- * duplicate entry in the metadata.
- *
- * @deprecated Usages of this exception will be replaced by the new
- * {@link org.apache.druid.error.DruidException} in a future release.
- */
-@Deprecated
-public class EntryExistsException extends DruidException
-{
-
-  private static final int HTTP_BAD_REQUEST = 400;
-
-  public EntryExistsException(String entryType, String entryId)
-  {
-    this(entryType, entryId, null);
-  }
-
-  public EntryExistsException(String entryType, String entryId, Throwable t)
-  {
-    super(StringUtils.format("%s [%s] already exists.", entryType, entryId), 
HTTP_BAD_REQUEST, t, false);
-  }
-
-}
diff --git 
a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java
 
b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java
index ca6ede6d32f..45bfde49a8a 100644
--- 
a/processing/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java
+++ 
b/processing/src/main/java/org/apache/druid/metadata/MetadataStorageActionHandler.java
@@ -44,7 +44,6 @@ public interface MetadataStorageActionHandler<EntryType, 
StatusType, LogType, Lo
    * @param status status object associated wit this object, can be null
    * @param type entry type
    * @param groupId entry group id
-   * @throws EntryExistsException
    */
   void insert(
       @NotNull String id,
@@ -55,7 +54,7 @@ public interface MetadataStorageActionHandler<EntryType, 
StatusType, LogType, Lo
       @Nullable StatusType status,
       @NotNull String type,
       @NotNull String groupId
-  ) throws EntryExistsException;
+  );
 
   /**
    * Sets or updates the status for any active entry with the given id.
diff --git 
a/processing/src/test/java/org/apache/druid/common/exception/DruidExceptionTest.java
 
b/processing/src/test/java/org/apache/druid/common/exception/DruidExceptionTest.java
deleted file mode 100644
index 9f0d53930e5..00000000000
--- 
a/processing/src/test/java/org/apache/druid/common/exception/DruidExceptionTest.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.druid.common.exception;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-public class DruidExceptionTest
-{
-  @Test
-  public void testExceptionMessageAndResponseCode()
-  {
-    DruidException exception = Assert.assertThrows(
-        DruidException.class,
-        () -> {
-          throw new DruidException("an error has occurred", 401, null, true);
-        }
-    );
-    Assert.assertEquals("an error has occurred", exception.getMessage());
-    Assert.assertEquals(401, exception.getResponseCode());
-    Assert.assertTrue(exception.isTransient());
-  }
-}
diff --git 
a/processing/src/test/java/org/apache/druid/metadata/EntryExistsExceptionTest.java
 
b/processing/src/test/java/org/apache/druid/metadata/EntryExistsExceptionTest.java
deleted file mode 100644
index 68b8a39b0d4..00000000000
--- 
a/processing/src/test/java/org/apache/druid/metadata/EntryExistsExceptionTest.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.druid.metadata;
-
-import org.apache.druid.common.exception.DruidException;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class EntryExistsExceptionTest
-{
-  @Test
-  public void testExceptionMessageAndResponseCode()
-  {
-    EntryExistsException exception = Assert.assertThrows(
-        EntryExistsException.class,
-        () -> {
-          throw new EntryExistsException("task", "100");
-        }
-    );
-    Assert.assertEquals("task [100] already exists.", exception.getMessage());
-    Assert.assertEquals(DruidException.HTTP_CODE_BAD_REQUEST, 
exception.getResponseCode());
-  }
-}
diff --git 
a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java
 
b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java
index 9fbc79f273c..8b003340bed 100644
--- 
a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java
+++ 
b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java
@@ -28,7 +28,8 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import com.google.common.base.Throwables;
 import com.google.common.collect.Maps;
-import org.apache.druid.common.exception.DruidException;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.error.InvalidInput;
 import org.apache.druid.indexer.TaskIdentifier;
 import org.apache.druid.indexer.TaskInfo;
 import org.apache.druid.java.util.common.DateTimes;
@@ -68,6 +69,7 @@ public abstract class 
SQLMetadataStorageActionHandler<EntryType, StatusType, Log
     implements MetadataStorageActionHandler<EntryType, StatusType, LogType, 
LockType>
 {
   private static final EmittingLogger log = new 
EmittingLogger(SQLMetadataStorageActionHandler.class);
+  private static final String CONTEXT_KEY_IS_TRANSIENT = "isTransient";
 
   private final SQLMetadataConnector connector;
   private final ObjectMapper jsonMapper;
@@ -163,7 +165,7 @@ public abstract class 
SQLMetadataStorageActionHandler<EntryType, StatusType, Log
       final StatusType status,
       final String type,
       final String groupId
-  ) throws EntryExistsException
+  )
   {
     try {
       getConnector().retryWithHandle(
@@ -236,7 +238,7 @@ public abstract class 
SQLMetadataStorageActionHandler<EntryType, StatusType, Log
     if (t instanceof CallbackFailedException) {
       return isTransientDruidException(t.getCause());
     } else if (t instanceof DruidException) {
-      return ((DruidException) t).isTransient();
+      return Boolean.parseBoolean(((DruidException) 
t).getContextValue(CONTEXT_KEY_IS_TRANSIENT));
     } else {
       return getConnector().isTransientException(t);
     }
@@ -433,26 +435,20 @@ public abstract class 
SQLMetadataStorageActionHandler<EntryType, StatusType, Log
   private DruidException wrapInDruidException(String taskId, Throwable t)
   {
     if (isStatementException(t) && getEntry(taskId).isPresent()) {
-      return new EntryExistsException("Task", taskId);
+      return InvalidInput.exception("Task [%s] already exists", taskId);
     } else if (connector.isRootCausePacketTooBigException(t)) {
-      return new DruidException(
-          StringUtils.format(
-              "Payload for task [%s] exceeds the packet limit."
-              + " Update the max_allowed_packet on your metadata store"
-              + " server or in the connection properties.",
-              taskId
-          ),
-          DruidException.HTTP_CODE_BAD_REQUEST,
-          t,
-          false
+      return InvalidInput.exception(
+          "Payload for task [%s] exceeds the max allowed packet limit."
+          + " If you encountered this error while running native batch 
ingestion,"
+          + " set a 'splitHintSpec' to reduce the payload of each task."
+          + " If not running native batch ingestion, report this error to your 
operator.",
+          taskId
       );
     } else {
-      return new DruidException(
-          StringUtils.format("Encountered metadata exception for task [%s]", 
taskId),
-          DruidException.HTTP_CODE_SERVER_ERROR,
-          t,
-          connector.isTransientException(t)
-      );
+      return DruidException.forPersona(DruidException.Persona.OPERATOR)
+                           .ofCategory(DruidException.Category.RUNTIME_FAILURE)
+                           .build(t, "Encountered metadata exception for task 
[%s]", taskId)
+                           .withContext(CONTEXT_KEY_IS_TRANSIENT, 
connector.isTransientException(t));
     }
   }
 
diff --git 
a/server/src/main/java/org/apache/druid/query/lookup/LookupListeningResource.java
 
b/server/src/main/java/org/apache/druid/query/lookup/LookupListeningResource.java
index 53072b96b10..51117835b3b 100644
--- 
a/server/src/main/java/org/apache/druid/query/lookup/LookupListeningResource.java
+++ 
b/server/src/main/java/org/apache/druid/query/lookup/LookupListeningResource.java
@@ -24,10 +24,10 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.collect.ImmutableMap;
 import com.google.inject.Inject;
 import com.sun.jersey.spi.container.ResourceFilters;
-import org.apache.druid.common.utils.ServletResourceUtils;
 import org.apache.druid.guice.annotations.Json;
 import org.apache.druid.guice.annotations.Smile;
 import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.server.http.ServletResourceUtils;
 import org.apache.druid.server.http.security.ConfigResourceFilter;
 import org.apache.druid.server.listener.resource.AbstractListenerHandler;
 import org.apache.druid.server.listener.resource.ListenerResource;
diff --git 
a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java
 
b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java
index 3939b866977..57833506f09 100644
--- 
a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java
+++ 
b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java
@@ -27,7 +27,6 @@ import org.apache.druid.audit.AuditEntry;
 import org.apache.druid.audit.AuditInfo;
 import org.apache.druid.audit.AuditManager;
 import org.apache.druid.common.config.ConfigManager.SetResult;
-import org.apache.druid.common.utils.ServletResourceUtils;
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.logger.Logger;
 import org.apache.druid.server.coordinator.CoordinatorCompactionConfig;
diff --git 
a/server/src/main/java/org/apache/druid/server/http/CoordinatorDynamicConfigsResource.java
 
b/server/src/main/java/org/apache/druid/server/http/CoordinatorDynamicConfigsResource.java
index a1ec2ea30e1..1dcbdb7e5f2 100644
--- 
a/server/src/main/java/org/apache/druid/server/http/CoordinatorDynamicConfigsResource.java
+++ 
b/server/src/main/java/org/apache/druid/server/http/CoordinatorDynamicConfigsResource.java
@@ -22,7 +22,6 @@ package org.apache.druid.server.http;
 import com.sun.jersey.spi.container.ResourceFilters;
 import org.apache.druid.audit.AuditManager;
 import org.apache.druid.common.config.ConfigManager.SetResult;
-import org.apache.druid.common.utils.ServletResourceUtils;
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.server.coordinator.CoordinatorConfigManager;
 import org.apache.druid.server.coordinator.CoordinatorDynamicConfig;
diff --git 
a/server/src/main/java/org/apache/druid/server/http/LookupCoordinatorResource.java
 
b/server/src/main/java/org/apache/druid/server/http/LookupCoordinatorResource.java
index 9e435e72f95..e954fb52136 100644
--- 
a/server/src/main/java/org/apache/druid/server/http/LookupCoordinatorResource.java
+++ 
b/server/src/main/java/org/apache/druid/server/http/LookupCoordinatorResource.java
@@ -29,7 +29,6 @@ import com.google.common.base.Strings;
 import com.google.common.net.HostAndPort;
 import com.google.inject.Inject;
 import com.sun.jersey.spi.container.ResourceFilters;
-import org.apache.druid.common.utils.ServletResourceUtils;
 import org.apache.druid.guice.annotations.Json;
 import org.apache.druid.guice.annotations.Smile;
 import org.apache.druid.java.util.common.IAE;
diff --git 
a/server/src/main/java/org/apache/druid/server/http/SegmentToDrop.java 
b/server/src/main/java/org/apache/druid/server/http/SegmentToDrop.java
deleted file mode 100644
index 67289caaa5b..00000000000
--- a/server/src/main/java/org/apache/druid/server/http/SegmentToDrop.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.druid.server.http;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-/**
- */
-public class SegmentToDrop
-{
-  private final String fromServer;
-  private final String segmentName;
-
-  @JsonCreator
-  public SegmentToDrop(
-      @JsonProperty("from") String fromServer,
-      @JsonProperty("segmentName") String segmentName
-  )
-  {
-    this.fromServer = fromServer;
-    this.segmentName = segmentName;
-  }
-
-  public String getFromServer()
-  {
-    return fromServer;
-  }
-
-  public String getSegmentName()
-  {
-    return segmentName;
-  }
-}
diff --git 
a/server/src/main/java/org/apache/druid/server/http/SegmentToMove.java 
b/server/src/main/java/org/apache/druid/server/http/SegmentToMove.java
deleted file mode 100644
index f0361e2e238..00000000000
--- a/server/src/main/java/org/apache/druid/server/http/SegmentToMove.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * 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.druid.server.http;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-/**
- */
-public class SegmentToMove
-{
-  private final String fromServer;
-  private final String toServer;
-  private final String segmentName;
-
-  @JsonCreator
-  public SegmentToMove(
-      @JsonProperty("from") String fromServer,
-      @JsonProperty("to") String toServer,
-      @JsonProperty("segmentName") String segmentName
-  )
-  {
-    this.fromServer = fromServer;
-    this.toServer = toServer;
-    this.segmentName = segmentName;
-  }
-
-  public String getFromServer()
-  {
-    return fromServer;
-  }
-
-  public String getToServer()
-  {
-    return toServer;
-  }
-
-  public String getSegmentName()
-  {
-    return segmentName;
-  }
-}
diff --git 
a/processing/src/main/java/org/apache/druid/common/utils/ServletResourceUtils.java
 b/server/src/main/java/org/apache/druid/server/http/ServletResourceUtils.java
similarity index 81%
rename from 
processing/src/main/java/org/apache/druid/common/utils/ServletResourceUtils.java
rename to 
server/src/main/java/org/apache/druid/server/http/ServletResourceUtils.java
index d28f612481d..58f78258789 100644
--- 
a/processing/src/main/java/org/apache/druid/common/utils/ServletResourceUtils.java
+++ 
b/server/src/main/java/org/apache/druid/server/http/ServletResourceUtils.java
@@ -17,12 +17,16 @@
  * under the License.
  */
 
-package org.apache.druid.common.utils;
+package org.apache.druid.server.http;
 
 import com.google.common.collect.ImmutableMap;
+import org.apache.druid.error.DruidException;
+import org.apache.druid.error.ErrorResponse;
 import org.apache.druid.java.util.common.StringUtils;
 
 import javax.annotation.Nullable;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
 import java.util.Map;
 
 public class ServletResourceUtils
@@ -49,4 +53,12 @@ public class ServletResourceUtils
   {
     return ImmutableMap.of("error", StringUtils.nonStrictFormat(msgFormat, 
args));
   }
+
+  public static Response buildErrorResponseFrom(DruidException e)
+  {
+    return Response.status(e.getStatusCode())
+                   .type(MediaType.APPLICATION_JSON_TYPE)
+                   .entity(new ErrorResponse(e))
+                   .build();
+  }
 }
diff --git 
a/server/src/main/java/org/apache/druid/server/listener/resource/AbstractListenerHandler.java
 
b/server/src/main/java/org/apache/druid/server/listener/resource/AbstractListenerHandler.java
index d6b608ad0a1..5553b7320c2 100644
--- 
a/server/src/main/java/org/apache/druid/server/listener/resource/AbstractListenerHandler.java
+++ 
b/server/src/main/java/org/apache/druid/server/listener/resource/AbstractListenerHandler.java
@@ -26,9 +26,9 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Function;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
-import org.apache.druid.common.utils.ServletResourceUtils;
 import org.apache.druid.java.util.common.jackson.JacksonUtils;
 import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.server.http.ServletResourceUtils;
 
 import javax.annotation.Nullable;
 import javax.ws.rs.core.Response;
diff --git 
a/server/src/main/java/org/apache/druid/server/listener/resource/ListenerResource.java
 
b/server/src/main/java/org/apache/druid/server/listener/resource/ListenerResource.java
index 569f20da775..f13f2a4360b 100644
--- 
a/server/src/main/java/org/apache/druid/server/listener/resource/ListenerResource.java
+++ 
b/server/src/main/java/org/apache/druid/server/listener/resource/ListenerResource.java
@@ -23,10 +23,10 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
-import org.apache.druid.common.utils.ServletResourceUtils;
 import org.apache.druid.guice.annotations.Json;
 import org.apache.druid.guice.annotations.Smile;
 import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.server.http.ServletResourceUtils;
 
 import javax.servlet.http.HttpServletRequest;
 import javax.ws.rs.Consumes;
diff --git 
a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java
 
b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java
index cee24d314e3..6cddcd5d646 100644
--- 
a/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java
+++ 
b/server/src/test/java/org/apache/druid/metadata/SQLMetadataStorageActionHandlerTest.java
@@ -25,6 +25,7 @@ import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
+import org.apache.druid.error.DruidException;
 import org.apache.druid.indexer.TaskIdentifier;
 import org.apache.druid.indexer.TaskInfo;
 import org.apache.druid.indexer.TaskLocation;
@@ -59,6 +60,7 @@ public class SQLMetadataStorageActionHandlerTest
   public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new 
TestDerbyConnector.DerbyConnectorRule();
 
   private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper();
+
   private static final Random RANDOM = new Random(1);
 
   private SQLMetadataStorageActionHandler<Map<String, Object>, Map<String, 
Object>, Map<String, String>, Map<String, Object>> handler;
@@ -131,17 +133,10 @@ public class SQLMetadataStorageActionHandlerTest
 
     handler.insert(entryId, DateTimes.of("2014-01-02T00:00:00.123"), 
"testDataSource", entry, true, null, "type", "group");
 
-    Assert.assertEquals(
-        Optional.of(entry),
-        handler.getEntry(entryId)
-    );
-
+    Assert.assertEquals(Optional.of(entry), handler.getEntry(entryId));
     Assert.assertEquals(Optional.absent(), 
handler.getEntry("non_exist_entry"));
-
     Assert.assertEquals(Optional.absent(), handler.getStatus(entryId));
-
     Assert.assertEquals(Optional.absent(), 
handler.getStatus("non_exist_entry"));
-
     Assert.assertTrue(handler.setStatus(entryId, true, status1));
 
     Assert.assertEquals(
@@ -175,21 +170,12 @@ public class SQLMetadataStorageActionHandlerTest
     // inactive statuses cannot be updated, this should fail
     Assert.assertFalse(handler.setStatus(entryId, false, status2));
 
-    Assert.assertEquals(
-        Optional.of(status1),
-        handler.getStatus(entryId)
-    );
-
-    Assert.assertEquals(
-        Optional.of(entry),
-        handler.getEntry(entryId)
-    );
-
+    Assert.assertEquals(Optional.of(status1), handler.getStatus(entryId));
+    Assert.assertEquals(Optional.of(entry), handler.getEntry(entryId));
     Assert.assertEquals(
         ImmutableList.of(),
         handler.getTaskInfos(CompleteTaskLookup.withTasksCreatedPriorTo(null, 
DateTimes.of("2014-01-03")), null)
     );
-
     Assert.assertEquals(
         ImmutableList.of(status1),
         handler.getTaskInfos(CompleteTaskLookup.withTasksCreatedPriorTo(null, 
DateTimes.of("2014-01-01")), null)
@@ -200,7 +186,7 @@ public class SQLMetadataStorageActionHandlerTest
   }
 
   @Test
-  public void testGetRecentStatuses() throws EntryExistsException
+  public void testGetRecentStatuses()
   {
     for (int i = 1; i < 11; i++) {
       final String entryId = "abcd_" + i;
@@ -211,10 +197,7 @@ public class SQLMetadataStorageActionHandlerTest
     }
 
     final List<TaskInfo<Map<String, Object>, Map<String, Object>>> statuses = 
handler.getTaskInfos(
-        CompleteTaskLookup.withTasksCreatedPriorTo(
-            7,
-            DateTimes.of("2014-01-01")
-        ),
+        CompleteTaskLookup.withTasksCreatedPriorTo(7, 
DateTimes.of("2014-01-01")),
         null
     );
     Assert.assertEquals(7, statuses.size());
@@ -225,7 +208,7 @@ public class SQLMetadataStorageActionHandlerTest
   }
 
   @Test
-  public void testGetRecentStatuses2() throws EntryExistsException
+  public void testGetRecentStatuses2()
   {
     for (int i = 1; i < 6; i++) {
       final String entryId = "abcd_" + i;
@@ -236,10 +219,7 @@ public class SQLMetadataStorageActionHandlerTest
     }
 
     final List<TaskInfo<Map<String, Object>, Map<String, Object>>> statuses = 
handler.getTaskInfos(
-        CompleteTaskLookup.withTasksCreatedPriorTo(
-            10,
-            DateTimes.of("2014-01-01")
-        ),
+        CompleteTaskLookup.withTasksCreatedPriorTo(10, 
DateTimes.of("2014-01-01")),
         null
     );
     Assert.assertEquals(5, statuses.size());
@@ -257,10 +237,13 @@ public class SQLMetadataStorageActionHandlerTest
     Map<String, Object> status = ImmutableMap.of("count", 42);
 
     handler.insert(entryId, DateTimes.of("2014-01-01"), "test", entry, true, 
status, "type", "group");
-    Assert.assertThrows(
-        EntryExistsException.class,
+
+    DruidException exception = Assert.assertThrows(
+        DruidException.class,
         () -> handler.insert(entryId, DateTimes.of("2014-01-01"), "test", 
entry, true, status, "type", "group")
     );
+    Assert.assertEquals("invalidInput", exception.getErrorCode());
+    Assert.assertEquals("Task [abcd] already exists", exception.getMessage());
   }
 
   @Test
@@ -341,7 +324,7 @@ public class SQLMetadataStorageActionHandlerTest
   }
 
   @Test
-  public void testReplaceLock() throws EntryExistsException
+  public void testReplaceLock()
   {
     final String entryId = "ABC123";
     Map<String, Object> entry = ImmutableMap.of("a", 1);
@@ -371,7 +354,7 @@ public class SQLMetadataStorageActionHandlerTest
   }
 
   @Test
-  public void testGetLockId() throws EntryExistsException
+  public void testGetLockId()
   {
     final String entryId = "ABC123";
     Map<String, Object> entry = ImmutableMap.of("a", 1);
diff --git 
a/processing/src/test/java/org/apache/druid/common/utils/ServletResourceUtilsTest.java
 
b/server/src/test/java/org/apache/druid/server/http/ServletResourceUtilsTest.java
similarity index 61%
rename from 
processing/src/test/java/org/apache/druid/common/utils/ServletResourceUtilsTest.java
rename to 
server/src/test/java/org/apache/druid/server/http/ServletResourceUtilsTest.java
index 2469f599a58..0af66bf0df5 100644
--- 
a/processing/src/test/java/org/apache/druid/common/utils/ServletResourceUtilsTest.java
+++ 
b/server/src/test/java/org/apache/druid/server/http/ServletResourceUtilsTest.java
@@ -17,11 +17,18 @@
  * under the License.
  */
 
-package org.apache.druid.common.utils;
+package org.apache.druid.server.http;
 
+import org.apache.druid.error.DruidException;
+import org.apache.druid.error.DruidExceptionMatcher;
+import org.apache.druid.error.ErrorResponse;
+import org.apache.druid.error.InvalidInput;
+import org.hamcrest.MatcherAssert;
 import org.junit.Assert;
 import org.junit.Test;
 
+import javax.ws.rs.core.Response;
+
 public class ServletResourceUtilsTest
 {
 
@@ -40,4 +47,19 @@ public class ServletResourceUtilsTest
       }
     }).get("error"));
   }
+
+  @Test
+  public void testBuildErrorReponseFrom()
+  {
+    DruidException exception = InvalidInput.exception("Invalid value of [%s]", 
"inputKey");
+    Response response = ServletResourceUtils.buildErrorResponseFrom(exception);
+    Assert.assertEquals(exception.getStatusCode(), response.getStatus());
+
+    Object entity = response.getEntity();
+    Assert.assertTrue(entity instanceof ErrorResponse);
+    MatcherAssert.assertThat(
+        ((ErrorResponse) entity).getUnderlyingException(),
+        DruidExceptionMatcher.invalidInput().expectMessageIs("Invalid value of 
[inputKey]")
+    );
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to