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

domgarguilo pushed a commit to branch 3.1
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/3.1 by this push:
     new 867d0eb6a6 Backport removal of problem reports to 3.1 (#5118)
867d0eb6a6 is described below

commit 867d0eb6a60ef4ad9f2697ff0fcf88dc75321f6a
Author: Dom G. <domgargu...@apache.org>
AuthorDate: Fri Dec 6 14:53:27 2024 -0500

    Backport removal of problem reports to 3.1 (#5118)
    
    * Backport removal of problem reports to 3.1
    
    ---------
    
    Co-authored-by: Keith Turner <ktur...@apache.org>
---
 .../java/org/apache/accumulo/core/Constants.java   |   2 -
 .../apache/accumulo/core/logging/TabletLogger.java |  13 +
 .../core/metadata/schema/MetadataSchema.java       |  17 --
 .../accumulo/server/compaction/FileCompactor.java  |  12 +-
 .../org/apache/accumulo/server/fs/FileManager.java |  15 +-
 .../accumulo/server/init/ZooKeeperInitializer.java |   2 -
 .../accumulo/server/problems/ProblemReport.java    | 252 ----------------
 .../server/problems/ProblemReportingIterator.java  |  24 +-
 .../accumulo/server/problems/ProblemReports.java   | 323 ---------------------
 .../accumulo/server/problems/ProblemType.java      |  23 --
 .../server/problems/ProblemReportTest.java         | 205 -------------
 .../problems/ProblemReportingIteratorTest.java     |   2 +-
 .../accumulo/manager/tableOps/delete/CleanUp.java  |   8 -
 .../accumulo/manager/upgrade/Upgrader11to12.java   | 147 ++++++++++
 .../manager/upgrade/Upgrader11to12Test.java        |   2 +
 .../java/org/apache/accumulo/monitor/Monitor.java  |  21 --
 .../monitor/rest/problems/ProblemDetail.java       |  42 ---
 .../rest/problems/ProblemDetailInformation.java    |  69 -----
 .../monitor/rest/problems/ProblemSummary.java      |  42 ---
 .../rest/problems/ProblemSummaryInformation.java   |  62 ----
 .../monitor/rest/problems/ProblemsResource.java    | 185 ------------
 .../monitor/rest/status/StatusInformation.java     |   6 +-
 .../monitor/rest/status/StatusResource.java        |   3 +-
 .../accumulo/monitor/resources/js/functions.js     |  40 +--
 .../apache/accumulo/monitor/resources/js/navbar.js |  34 ---
 .../accumulo/monitor/resources/js/problems.js      | 167 -----------
 .../apache/accumulo/monitor/templates/navbar.ftl   |   8 -
 .../apache/accumulo/monitor/templates/problems.ftl |  71 -----
 .../apache/accumulo/tserver/AssignmentHandler.java |  17 +-
 .../accumulo/tserver/tablet/MinorCompactor.java    |  15 -
 .../org/apache/accumulo/tserver/tablet/Tablet.java |   5 -
 31 files changed, 187 insertions(+), 1647 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/Constants.java 
b/core/src/main/java/org/apache/accumulo/core/Constants.java
index ca9e0b62ff..e3480686d7 100644
--- a/core/src/main/java/org/apache/accumulo/core/Constants.java
+++ b/core/src/main/java/org/apache/accumulo/core/Constants.java
@@ -75,8 +75,6 @@ public class Constants {
   public static final String ZDEAD = "/dead";
   public static final String ZDEADTSERVERS = ZDEAD + "/tservers";
 
-  public static final String ZPROBLEMS = "/problems";
-
   public static final String BULK_ARBITRATOR_TYPE = "bulkTx";
 
   public static final String ZFATE = "/fate";
diff --git 
a/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java 
b/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java
index 2209e41c27..0b6849216a 100644
--- a/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java
+++ b/core/src/main/java/org/apache/accumulo/core/logging/TabletLogger.java
@@ -27,6 +27,7 @@ import java.util.UUID;
 
 import org.apache.accumulo.core.client.admin.CompactionConfig;
 import org.apache.accumulo.core.client.admin.compaction.CompactableFile;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
 import org.apache.accumulo.core.metadata.CompactableFileImpl;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
@@ -110,11 +111,23 @@ public class TabletLogger {
     }
   }
 
+  public static void tabletLoadFailed(KeyExtent extent, Exception e) {
+    locLog.warn("Failed to load tablet {}", extent, e);
+  }
+
   private static String getSize(Collection<CompactableFile> files) {
     long sum = 
files.stream().mapToLong(CompactableFile::getEstimatedSize).sum();
     return FileUtils.byteCountToDisplaySize(sum);
   }
 
+  public static void fileReadFailed(String path, TableId tableId, Exception e) 
{
+    fileLog.error("For table {} failed to read {} ", tableId, path, e);
+  }
+
+  public static void fileReadFailed(String path, KeyExtent tablet, Exception 
e) {
+    fileLog.error("For tablet {} failed to read {} ", tablet, path, e);
+  }
+
   /**
    * Lazily converts TableFile to file names. The lazy part is really 
important because when it is
    * not called with log.isDebugEnabled().
diff --git 
a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
 
b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
index 4d5a3dea3f..b7269eb494 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
@@ -415,23 +415,6 @@ public class MetadataSchema {
 
   }
 
-  /**
-   * Holds error message processing flags
-   */
-  public static class ProblemSection {
-    private static final Section section =
-        new Section(RESERVED_PREFIX + "err_", true, RESERVED_PREFIX + "err`", 
false);
-
-    public static Range getRange() {
-      return section.getRange();
-    }
-
-    public static String getRowPrefix() {
-      return section.getRowPrefix();
-    }
-
-  }
-
   public static class ExternalCompactionSection {
     private static final Section section =
         new Section(RESERVED_PREFIX + "ecomp", true, RESERVED_PREFIX + 
"ecomq", false);
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java
 
b/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java
index 335f583fd6..65dd1a8ef2 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java
@@ -60,6 +60,7 @@ import 
org.apache.accumulo.core.iteratorsImpl.system.DeletingIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.InterruptibleIterator;
 import 
org.apache.accumulo.core.iteratorsImpl.system.IterationInterruptedException;
 import org.apache.accumulo.core.iteratorsImpl.system.MultiIterator;
+import org.apache.accumulo.core.logging.TabletLogger;
 import org.apache.accumulo.core.metadata.AccumuloTable;
 import org.apache.accumulo.core.metadata.ReferencedTabletFile;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
@@ -75,10 +76,7 @@ import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.iterators.SystemIteratorEnvironment;
 import org.apache.accumulo.server.mem.LowMemoryDetector.DetectionScope;
-import org.apache.accumulo.server.problems.ProblemReport;
 import org.apache.accumulo.server.problems.ProblemReportingIterator;
-import org.apache.accumulo.server.problems.ProblemReports;
-import org.apache.accumulo.server.problems.ProblemType;
 import org.apache.hadoop.fs.FileSystem;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -438,7 +436,7 @@ public class FileCompactor implements 
Callable<CompactionStats> {
 
         readers.add(reader);
 
-        InterruptibleIterator iter = new ProblemReportingIterator(context, 
extent.tableId(),
+        InterruptibleIterator iter = new 
ProblemReportingIterator(extent.tableId(),
             dataFile.getNormalizedPathStr(), false, reader);
         iter.setInterruptFlag(interruptFlag);
 
@@ -447,11 +445,7 @@ public class FileCompactor implements 
Callable<CompactionStats> {
         iters.add(iter);
 
       } catch (Exception e) {
-
-        ProblemReports.getInstance(context).report(new 
ProblemReport(extent.tableId(),
-            ProblemType.FILE_READ, dataFile.getNormalizedPathStr(), e));
-
-        log.warn("Some problem opening data file {} {}", dataFile, 
e.getMessage(), e);
+        TabletLogger.fileReadFailed(dataFile.toString(), extent, e);
         // failed to open some data file... close the ones that were opened
         for (FileSKVIterator reader : readers) {
           try {
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/fs/FileManager.java 
b/server/base/src/main/java/org/apache/accumulo/server/fs/FileManager.java
index 59ac2484af..7a3c900c92 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/FileManager.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/FileManager.java
@@ -46,15 +46,13 @@ import 
org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.InterruptibleIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.SourceSwitchingIterator;
 import 
org.apache.accumulo.core.iteratorsImpl.system.SourceSwitchingIterator.DataSource;
+import org.apache.accumulo.core.logging.TabletLogger;
 import org.apache.accumulo.core.metadata.StoredTabletFile;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
 import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
 import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.problems.ProblemReport;
 import org.apache.accumulo.server.problems.ProblemReportingIterator;
-import org.apache.accumulo.server.problems.ProblemReports;
-import org.apache.accumulo.server.problems.ProblemType;
 import org.apache.hadoop.fs.FileSystem;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -308,16 +306,12 @@ public class FileManager {
             .withFileLenCache(fileLenCache).build();
         readersReserved.put(reader, file);
       } catch (Exception e) {
-
-        ProblemReports.getInstance(context)
-            .report(new ProblemReport(tablet.tableId(), ProblemType.FILE_READ, 
file.toString(), e));
-
+        TabletLogger.fileReadFailed(file.toString(), tablet, e);
         if (continueOnFailure) {
           // release the permit for the file that failed to open
           if (!tablet.isMeta()) {
             filePermits.release(1);
           }
-          log.warn("Failed to open file {} {} continuing...", file, 
e.getMessage(), e);
         } else {
           // close whatever files were opened
           closeReaders(readersReserved.keySet());
@@ -326,7 +320,6 @@ public class FileManager {
             filePermits.release(files.size());
           }
 
-          log.error("Failed to open file {} {}", file, e.getMessage());
           throw new IOException("Failed to open " + file, e);
         }
       }
@@ -519,8 +512,8 @@ public class FileManager {
           }
         }
 
-        iter = new ProblemReportingIterator(context, tablet.tableId(), 
file.toString(),
-            continueOnFailure, detachable ? getSsi(file, source) : source);
+        iter = new ProblemReportingIterator(tablet.tableId(), file.toString(), 
continueOnFailure,
+            detachable ? getSsi(file, source) : source);
 
         if (someIteratorsWillWrap) {
           // constructing FileRef is expensive so avoid if not needed
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java
 
b/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java
index 8a78f6c254..6be6e4feed 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/init/ZooKeeperInitializer.java
@@ -134,8 +134,6 @@ public class ZooKeeperInitializer {
 
     zoo.putPersistentData(zkInstanceRoot + Constants.ZTSERVERS, 
EMPTY_BYTE_ARRAY,
         ZooUtil.NodeExistsPolicy.FAIL);
-    zoo.putPersistentData(zkInstanceRoot + Constants.ZPROBLEMS, 
EMPTY_BYTE_ARRAY,
-        ZooUtil.NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + RootTable.ZROOT_TABLET,
         getInitialRootTabletJson(rootTabletDirName, rootTabletFileUri),
         ZooUtil.NodeExistsPolicy.FAIL);
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java
 
b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java
deleted file mode 100644
index cd221e436d..0000000000
--- 
a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java
+++ /dev/null
@@ -1,252 +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
- *
- *   https://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.accumulo.server.problems;
-
-import static java.util.Objects.requireNonNull;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy;
-import org.apache.accumulo.core.metadata.AccumuloTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ProblemSection;
-import org.apache.accumulo.core.util.Encoding;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.zookeeper.KeeperException;
-
-public class ProblemReport {
-  private final TableId tableId;
-  private final ProblemType problemType;
-  private final String resource;
-  private String exception;
-  private String server;
-  private long creationTime;
-
-  public ProblemReport(TableId tableId, ProblemType problemType, String 
resource, String server,
-      Throwable e, long creationTime) {
-    requireNonNull(tableId, "tableId is null");
-    requireNonNull(problemType, "problemType is null");
-    requireNonNull(resource, "resource is null");
-    this.tableId = tableId;
-
-    this.problemType = problemType;
-    this.resource = resource;
-
-    if (e != null) {
-      this.exception = e.getMessage();
-    }
-
-    if (server == null) {
-      try {
-        server = InetAddress.getLocalHost().getHostAddress();
-      } catch (UnknownHostException e1) {
-
-      }
-    }
-
-    this.server = server;
-    this.creationTime = creationTime;
-  }
-
-  public ProblemReport(TableId tableId, ProblemType problemType, String 
resource, String server,
-      Throwable e) {
-    this(tableId, problemType, resource, server, e, 
System.currentTimeMillis());
-  }
-
-  public ProblemReport(TableId tableId, ProblemType problemType, String 
resource, Throwable e) {
-    this(tableId, problemType, resource, null, e);
-  }
-
-  private ProblemReport(TableId table, ProblemType problemType, String 
resource, byte[] enc)
-      throws IOException {
-    requireNonNull(table, "table is null");
-    requireNonNull(problemType, "problemType is null");
-    requireNonNull(resource, "resource is null");
-    this.tableId = table;
-    this.problemType = problemType;
-    this.resource = resource;
-
-    decode(enc);
-  }
-
-  private byte[] encode() throws IOException {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-
-    dos.writeLong(creationTime);
-
-    dos.writeBoolean(server != null);
-    if (server != null) {
-      dos.writeUTF(server);
-    }
-
-    dos.writeBoolean(exception != null);
-    if (exception != null) {
-      dos.writeUTF(exception);
-    }
-
-    dos.close();
-    baos.close();
-
-    return baos.toByteArray();
-  }
-
-  private void decode(byte[] enc) throws IOException {
-    ByteArrayInputStream bais = new ByteArrayInputStream(enc);
-    DataInputStream dis = new DataInputStream(bais);
-
-    creationTime = dis.readLong();
-
-    if (dis.readBoolean()) {
-      server = dis.readUTF();
-    } else {
-      server = null;
-    }
-
-    if (dis.readBoolean()) {
-      exception = dis.readUTF();
-    } else {
-      exception = null;
-    }
-  }
-
-  void removeFromMetadataTable(ServerContext context) throws Exception {
-    Mutation m = new Mutation(ProblemSection.getRowPrefix() + tableId);
-    m.putDelete(problemType.name(), resource);
-    try (var writer = 
context.createBatchWriter(AccumuloTable.METADATA.tableName())) {
-      writer.addMutation(m);
-    }
-  }
-
-  void saveToMetadataTable(ServerContext context) throws Exception {
-    Mutation m = new Mutation(ProblemSection.getRowPrefix() + tableId);
-    m.put(problemType.name(), resource, new Value(encode()));
-    try (var writer = 
context.createBatchWriter(AccumuloTable.METADATA.tableName())) {
-      writer.addMutation(m);
-    }
-  }
-
-  void removeFromZooKeeper(ServerContext context) throws Exception {
-    removeFromZooKeeper(context.getZooReaderWriter(), context);
-  }
-
-  void removeFromZooKeeper(ZooReaderWriter zoorw, ServerContext context)
-      throws IOException, KeeperException, InterruptedException {
-    String zpath = getZPath(context.getZooKeeperRoot());
-    zoorw.recursiveDelete(zpath, NodeMissingPolicy.SKIP);
-  }
-
-  void saveToZooKeeper(ServerContext context)
-      throws IOException, KeeperException, InterruptedException {
-    
context.getZooReaderWriter().putPersistentData(getZPath(context.getZooKeeperRoot()),
 encode(),
-        NodeExistsPolicy.OVERWRITE);
-  }
-
-  private String getZPath(String zkRoot) throws IOException {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-    dos.writeUTF(getTableId().canonical());
-    dos.writeUTF(getProblemType().name());
-    dos.writeUTF(getResource());
-    dos.close();
-    baos.close();
-
-    return zkRoot + Constants.ZPROBLEMS + "/" + 
Encoding.encodeAsBase64FileName(baos.toByteArray());
-  }
-
-  static ProblemReport decodeZooKeeperEntry(ServerContext context, String node)
-      throws IOException, KeeperException, InterruptedException {
-    byte[] bytes = Encoding.decodeBase64FileName(node);
-
-    ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
-    DataInputStream dis = new DataInputStream(bais);
-
-    TableId tableId = TableId.of(dis.readUTF());
-    String problemType = dis.readUTF();
-    String resource = dis.readUTF();
-
-    String zpath = context.getZooKeeperRoot() + Constants.ZPROBLEMS + "/" + 
node;
-    byte[] enc = context.getZooReaderWriter().getData(zpath);
-
-    return new ProblemReport(tableId, ProblemType.valueOf(problemType), 
resource, enc);
-
-  }
-
-  public static ProblemReport decodeMetadataEntry(Entry<Key,Value> entry) 
throws IOException {
-    TableId tableId = TableId
-        
.of(entry.getKey().getRow().toString().substring(ProblemSection.getRowPrefix().length()));
-    String problemType = entry.getKey().getColumnFamily().toString();
-    String resource = entry.getKey().getColumnQualifier().toString();
-
-    return new ProblemReport(tableId, ProblemType.valueOf(problemType), 
resource,
-        entry.getValue().get());
-  }
-
-  public TableId getTableId() {
-    return tableId;
-  }
-
-  public ProblemType getProblemType() {
-    return problemType;
-  }
-
-  public String getResource() {
-    return resource;
-  }
-
-  public String getException() {
-    return exception;
-  }
-
-  public String getServer() {
-    return server;
-  }
-
-  public long getTime() {
-    return creationTime;
-  }
-
-  @Override
-  public int hashCode() {
-    return tableId.hashCode() + problemType.hashCode() + resource.hashCode();
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof ProblemReport) {
-      ProblemReport opr = (ProblemReport) o;
-      return tableId.equals(opr.tableId) && problemType.equals(opr.problemType)
-          && resource.equals(opr.resource);
-    }
-    return false;
-  }
-}
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReportingIterator.java
 
b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReportingIterator.java
index 4344c6ba97..5cfd27063b 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReportingIterator.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReportingIterator.java
@@ -31,7 +31,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.InterruptibleIterator;
-import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.core.logging.TabletLogger;
 
 public class ProblemReportingIterator implements InterruptibleIterator {
   private final SortedKeyValueIterator<Key,Value> source;
@@ -39,11 +39,10 @@ public class ProblemReportingIterator implements 
InterruptibleIterator {
   private final boolean continueOnError;
   private final String resource;
   private final TableId tableId;
-  private final ServerContext context;
 
-  public ProblemReportingIterator(ServerContext context, TableId tableId, 
String resource,
-      boolean continueOnError, SortedKeyValueIterator<Key,Value> source) {
-    this.context = context;
+  public ProblemReportingIterator(TableId tableId, String resource, boolean 
continueOnError,
+      SortedKeyValueIterator<Key,Value> source) {
+
     this.tableId = tableId;
     this.resource = resource;
     this.continueOnError = continueOnError;
@@ -52,8 +51,7 @@ public class ProblemReportingIterator implements 
InterruptibleIterator {
 
   @Override
   public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
-    return new ProblemReportingIterator(context, tableId, resource, 
continueOnError,
-        source.deepCopy(env));
+    return new ProblemReportingIterator(tableId, resource, continueOnError, 
source.deepCopy(env));
   }
 
   @Override
@@ -86,10 +84,10 @@ public class ProblemReportingIterator implements 
InterruptibleIterator {
       source.next();
     } catch (IOException ioe) {
       sawError = true;
-      ProblemReports.getInstance(context)
-          .report(new ProblemReport(tableId, ProblemType.FILE_READ, resource, 
ioe));
+      TabletLogger.fileReadFailed(resource, tableId, ioe);
       if (!continueOnError) {
-        throw ioe;
+        // include the name of the resource being read from in the exception 
error message
+        throw new IOException("Error reading from " + resource + " for table " 
+ tableId, ioe);
       }
     }
   }
@@ -105,10 +103,10 @@ public class ProblemReportingIterator implements 
InterruptibleIterator {
       source.seek(range, columnFamilies, inclusive);
     } catch (IOException ioe) {
       sawError = true;
-      ProblemReports.getInstance(context)
-          .report(new ProblemReport(tableId, ProblemType.FILE_READ, resource, 
ioe));
+      TabletLogger.fileReadFailed(resource, tableId, ioe);
       if (!continueOnError) {
-        throw ioe;
+        // include the name of the resource being read from in the exception 
error message
+        throw new IOException("Error reading from " + resource + " for table " 
+ tableId, ioe);
       }
     }
   }
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
 
b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
deleted file mode 100644
index 4672c279a8..0000000000
--- 
a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
+++ /dev/null
@@ -1,323 +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
- *
- *   https://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.accumulo.server.problems;
-
-import static java.util.concurrent.TimeUnit.SECONDS;
-
-import java.io.IOException;
-import java.io.UncheckedIOException;
-import java.util.Collections;
-import java.util.EnumMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.NoSuchElementException;
-import java.util.TreeMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.conf.SiteConfiguration;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Range;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.core.iterators.SortedKeyIterator;
-import org.apache.accumulo.core.metadata.AccumuloTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ProblemSection;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.threads.ThreadPools;
-import org.apache.accumulo.server.ServerContext;
-import org.apache.commons.collections4.map.LRUMap;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class ProblemReports implements Iterable<ProblemReport> {
-
-  private static final Logger log = 
LoggerFactory.getLogger(ProblemReports.class);
-
-  private final LRUMap<ProblemReport,Long> problemReports = new LRUMap<>(1000);
-
-  /**
-   * use a thread pool so that reporting a problem never blocks
-   *
-   * make the thread pool use a bounded queue to avoid the case where problem 
reports are not being
-   * processed because the whole system is in a really bad state (like HDFS is 
down) and everything
-   * is reporting lots of problems, but problem reports can not be processed
-   */
-  private final ExecutorService reportExecutor = 
ThreadPools.getServerThreadPools()
-      .getPoolBuilder("problem.reporter").numCoreThreads(0).numMaxThreads(1)
-      .withTimeOut(60L, SECONDS).withQueue(new 
LinkedBlockingQueue<>(500)).build();
-
-  private final ServerContext context;
-
-  public ProblemReports(ServerContext context) {
-    this.context = context;
-  }
-
-  public void report(final ProblemReport pr) {
-
-    synchronized (problemReports) {
-      if (problemReports.containsKey(pr)) {
-        return;
-      }
-
-      problemReports.put(pr, System.currentTimeMillis());
-    }
-
-    Runnable r = () -> {
-
-      log.debug("Filing problem report {} {} {}", pr.getTableId(), 
pr.getProblemType(),
-          pr.getResource());
-
-      try {
-        if (isMeta(pr.getTableId())) {
-          // file report in zookeeper
-          pr.saveToZooKeeper(context);
-        } else {
-          // file report in metadata table
-          pr.saveToMetadataTable(context);
-        }
-      } catch (Exception e) {
-        log.error("Failed to file problem report " + pr.getTableId() + " " + 
pr.getProblemType()
-            + " " + pr.getResource(), e);
-      }
-    };
-
-    try {
-      reportExecutor.execute(r);
-    } catch (RejectedExecutionException ree) {
-      log.error("Failed to report problem {} {} {} {}", pr.getTableId(), 
pr.getProblemType(),
-          pr.getResource(), ree.getMessage());
-    }
-
-  }
-
-  public void printProblems() {
-    for (ProblemReport pr : this) {
-      System.out.println(pr.getTableId() + " " + pr.getProblemType() + " " + 
pr.getResource() + " "
-          + pr.getException());
-    }
-  }
-
-  public void deleteProblemReport(TableId table, ProblemType pType, String 
resource) {
-    final ProblemReport pr = new ProblemReport(table, pType, resource, null);
-
-    Runnable r = () -> {
-      try {
-        if (isMeta(pr.getTableId())) {
-          // file report in zookeeper
-          pr.removeFromZooKeeper(context);
-        } else {
-          // file report in metadata table
-          pr.removeFromMetadataTable(context);
-        }
-      } catch (Exception e) {
-        log.error("Failed to delete problem report {} {} {}", pr.getTableId(), 
pr.getProblemType(),
-            pr.getResource(), e);
-      }
-    };
-
-    try {
-      reportExecutor.execute(r);
-    } catch (RejectedExecutionException ree) {
-      log.error("Failed to delete problem report {} {} {} {}", 
pr.getTableId(), pr.getProblemType(),
-          pr.getResource(), ree.getMessage());
-    }
-  }
-
-  private static ProblemReports instance;
-
-  public void deleteProblemReports(TableId table) throws Exception {
-
-    if (isMeta(table)) {
-      Iterator<ProblemReport> pri = iterator(table);
-      while (pri.hasNext()) {
-        pri.next().removeFromZooKeeper(context);
-      }
-      return;
-    }
-
-    Scanner scanner =
-        context.createScanner(AccumuloTable.METADATA.tableName(), 
Authorizations.EMPTY);
-    scanner.addScanIterator(new IteratorSetting(1, "keys-only", 
SortedKeyIterator.class));
-
-    scanner.setRange(new Range(ProblemSection.getRowPrefix() + table));
-
-    Mutation delMut = new Mutation(ProblemSection.getRowPrefix() + table);
-
-    boolean hasProblems = false;
-    for (Entry<Key,Value> entry : scanner) {
-      hasProblems = true;
-      delMut.putDelete(entry.getKey().getColumnFamily(), 
entry.getKey().getColumnQualifier());
-    }
-
-    if (hasProblems) {
-      try (var writer = 
context.createBatchWriter(AccumuloTable.METADATA.tableName())) {
-        writer.addMutation(delMut);
-      }
-    }
-  }
-
-  private static boolean isMeta(TableId tableId) {
-    return tableId.equals(AccumuloTable.METADATA.tableId())
-        || tableId.equals(AccumuloTable.ROOT.tableId());
-  }
-
-  public Iterator<ProblemReport> iterator(final TableId table) {
-    return new Iterator<>() {
-
-      final ZooReaderWriter zoo = context.getZooReaderWriter();
-      private int iter1Count = 0;
-      private Iterator<String> iter1;
-
-      private Iterator<String> getIter1() {
-        if (iter1 == null) {
-          try {
-            List<String> children;
-            if (table == null || isMeta(table)) {
-              children = zoo.getChildren(context.getZooKeeperRoot() + 
Constants.ZPROBLEMS);
-            } else {
-              children = Collections.emptyList();
-            }
-            iter1 = children.iterator();
-          } catch (KeeperException | InterruptedException e) {
-            throw new IllegalStateException(e);
-          }
-        }
-
-        return iter1;
-      }
-
-      private Iterator<Entry<Key,Value>> iter2;
-
-      private Iterator<Entry<Key,Value>> getIter2() {
-        if (iter2 == null) {
-          try {
-            if ((table == null || !isMeta(table)) && iter1Count == 0) {
-              Scanner scanner =
-                  context.createScanner(AccumuloTable.METADATA.tableName(), 
Authorizations.EMPTY);
-              scanner.setTimeout(3, TimeUnit.SECONDS);
-
-              if (table == null) {
-                scanner.setRange(ProblemSection.getRange());
-              } else {
-                scanner.setRange(new Range(ProblemSection.getRowPrefix() + 
table));
-              }
-
-              iter2 = scanner.iterator();
-
-            } else {
-              Map<Key,Value> m = Collections.emptyMap();
-              iter2 = m.entrySet().iterator();
-            }
-          } catch (TableNotFoundException e) {
-            throw new IllegalStateException(e);
-          }
-        }
-
-        return iter2;
-      }
-
-      @Override
-      public boolean hasNext() {
-        if (getIter1().hasNext()) {
-          return true;
-        }
-        return getIter2().hasNext();
-      }
-
-      @Override
-      public ProblemReport next() {
-        try {
-          if (getIter1().hasNext()) {
-            iter1Count++;
-            return ProblemReport.decodeZooKeeperEntry(context, 
getIter1().next());
-          }
-
-          if (getIter2().hasNext()) {
-            return ProblemReport.decodeMetadataEntry(getIter2().next());
-          }
-        } catch (IOException e) {
-          throw new UncheckedIOException(e);
-        } catch (KeeperException | InterruptedException e) {
-          throw new IllegalStateException(e);
-        }
-
-        throw new NoSuchElementException();
-      }
-
-      @Override
-      public void remove() {
-        throw new UnsupportedOperationException();
-      }
-
-    };
-  }
-
-  @Override
-  public Iterator<ProblemReport> iterator() {
-    return iterator(null);
-  }
-
-  public static synchronized ProblemReports getInstance(ServerContext context) 
{
-    if (instance == null) {
-      instance = new ProblemReports(context);
-    }
-
-    return instance;
-  }
-
-  public static void main(String[] args) {
-    var context = new ServerContext(SiteConfiguration.auto());
-    getInstance(context).printProblems();
-  }
-
-  public Map<TableId,Map<ProblemType,Integer>> summarize() {
-
-    TreeMap<TableId,Map<ProblemType,Integer>> summary = new TreeMap<>();
-
-    for (ProblemReport pr : this) {
-      Map<ProblemType,Integer> tableProblems = summary.get(pr.getTableId());
-      if (tableProblems == null) {
-        tableProblems = new EnumMap<>(ProblemType.class);
-        summary.put(pr.getTableId(), tableProblems);
-      }
-
-      Integer count = tableProblems.get(pr.getProblemType());
-      if (count == null) {
-        count = 0;
-      }
-
-      tableProblems.put(pr.getProblemType(), count + 1);
-    }
-
-    return summary;
-  }
-
-}
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemType.java
 
b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemType.java
deleted file mode 100644
index 43e3766e14..0000000000
--- 
a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemType.java
+++ /dev/null
@@ -1,23 +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
- *
- *   https://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.accumulo.server.problems;
-
-public enum ProblemType {
-  FILE_READ, FILE_WRITE, TABLET_LOAD
-}
diff --git 
a/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java
 
b/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java
deleted file mode 100644
index efbfbb07bc..0000000000
--- 
a/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java
+++ /dev/null
@@ -1,205 +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
- *
- *   https://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.accumulo.server.problems;
-
-import static org.easymock.EasyMock.aryEq;
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.eq;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.verify;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertNotEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertSame;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-
-import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.data.InstanceId;
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
-import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
-import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
-import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy;
-import org.apache.accumulo.core.metadata.AccumuloTable;
-import org.apache.accumulo.core.util.Encoding;
-import org.apache.accumulo.server.MockServerContext;
-import org.apache.accumulo.server.ServerContext;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-public class ProblemReportTest {
-  private static final TableId TABLE_ID = TableId.of("table");
-  private static final String RESOURCE = "resource";
-  private static final String SERVER = "server";
-
-  private ServerContext context;
-  private ZooReaderWriter zoorw;
-  private ProblemReport r;
-
-  @BeforeEach
-  public void setUp() {
-    context = MockServerContext.getWithZK(InstanceId.of("instance"), "", 
30_000);
-    zoorw = createMock(ZooReaderWriter.class);
-    expect(context.getZooReaderWriter()).andReturn(zoorw).anyTimes();
-    replay(context);
-  }
-
-  @Test
-  public void testGetters() {
-    long now = System.currentTimeMillis();
-    r = new ProblemReport(TABLE_ID, ProblemType.FILE_READ, RESOURCE, SERVER, 
null, now);
-    assertEquals(TABLE_ID, r.getTableId());
-    assertSame(ProblemType.FILE_READ, r.getProblemType());
-    assertEquals(RESOURCE, r.getResource());
-    assertEquals(SERVER, r.getServer());
-    assertEquals(now, r.getTime());
-    assertNull(r.getException());
-  }
-
-  @Test
-  public void testWithException() {
-    Exception e = new IllegalArgumentException("Oh noes");
-    r = new ProblemReport(TABLE_ID, ProblemType.FILE_READ, RESOURCE, SERVER, 
e);
-    assertEquals("Oh noes", r.getException());
-  }
-
-  @Test
-  public void testEquals() {
-    r = new ProblemReport(TABLE_ID, ProblemType.FILE_READ, RESOURCE, SERVER, 
null);
-    assertEquals(r, r);
-    ProblemReport r2 = new ProblemReport(TABLE_ID, ProblemType.FILE_READ, 
RESOURCE, SERVER, null);
-    assertEquals(r, r2);
-    assertEquals(r2, r);
-    ProblemReport rx1 = new ProblemReport(AccumuloTable.METADATA.tableId(), 
ProblemType.FILE_READ,
-        RESOURCE, SERVER, null);
-    assertNotEquals(r, rx1);
-    ProblemReport rx2 = new ProblemReport(TABLE_ID, ProblemType.FILE_WRITE, 
RESOURCE, SERVER, null);
-    assertNotEquals(r, rx2);
-    ProblemReport rx3 =
-        new ProblemReport(TABLE_ID, ProblemType.FILE_READ, RESOURCE + "x", 
SERVER, null);
-    assertNotEquals(r, rx3);
-    ProblemReport re1 =
-        new ProblemReport(TABLE_ID, ProblemType.FILE_READ, RESOURCE, SERVER + 
"x", null);
-    assertEquals(r, re1);
-    ProblemReport re2 = new ProblemReport(TABLE_ID, ProblemType.FILE_READ, 
RESOURCE, SERVER,
-        new IllegalArgumentException("yikes"));
-    assertEquals(r, re2);
-  }
-
-  @Test
-  public void testEqualsNull() {
-    r = new ProblemReport(TABLE_ID, ProblemType.FILE_READ, RESOURCE, SERVER, 
null);
-    assertFalse(r.equals(null));
-  }
-
-  @Test
-  public void testHashCode() {
-    r = new ProblemReport(TABLE_ID, ProblemType.FILE_READ, RESOURCE, SERVER, 
null);
-    ProblemReport r2 = new ProblemReport(TABLE_ID, ProblemType.FILE_READ, 
RESOURCE, SERVER, null);
-    assertEquals(r.hashCode(), r2.hashCode());
-    ProblemReport re1 =
-        new ProblemReport(TABLE_ID, ProblemType.FILE_READ, RESOURCE, SERVER + 
"x", null);
-    assertEquals(r.hashCode(), re1.hashCode());
-    ProblemReport re2 = new ProblemReport(TABLE_ID, ProblemType.FILE_READ, 
RESOURCE, SERVER,
-        new IllegalArgumentException("yikes"));
-    assertEquals(r.hashCode(), re2.hashCode());
-  }
-
-  private byte[] makeZPathFileName(TableId table, ProblemType problemType, 
String resource)
-      throws Exception {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-    dos.writeUTF(table.canonical());
-    dos.writeUTF(problemType.name());
-    dos.writeUTF(resource);
-    dos.close();
-    return baos.toByteArray();
-  }
-
-  private byte[] encodeReportData(long creationTime, String server, String 
exception)
-      throws Exception {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    DataOutputStream dos = new DataOutputStream(baos);
-    dos.writeLong(creationTime);
-    dos.writeBoolean(server != null);
-    if (server != null) {
-      dos.writeUTF(server);
-    }
-    dos.writeBoolean(exception != null);
-    if (exception != null) {
-      dos.writeUTF(exception);
-    }
-    dos.close();
-    return baos.toByteArray();
-  }
-
-  @Test
-  public void testRemoveFromZooKeeper() throws Exception {
-    r = new ProblemReport(TABLE_ID, ProblemType.FILE_READ, RESOURCE, SERVER, 
null);
-    byte[] zpathFileName = makeZPathFileName(TABLE_ID, ProblemType.FILE_READ, 
RESOURCE);
-    String path = ZooUtil.getRoot(InstanceId.of("instance")) + 
Constants.ZPROBLEMS + "/"
-        + Encoding.encodeAsBase64FileName(zpathFileName);
-    zoorw.recursiveDelete(path, NodeMissingPolicy.SKIP);
-    replay(zoorw);
-
-    r.removeFromZooKeeper(zoorw, context);
-    verify(zoorw);
-  }
-
-  @Test
-  public void testSaveToZooKeeper() throws Exception {
-    long now = System.currentTimeMillis();
-    r = new ProblemReport(TABLE_ID, ProblemType.FILE_READ, RESOURCE, SERVER, 
null, now);
-    byte[] zpathFileName = makeZPathFileName(TABLE_ID, ProblemType.FILE_READ, 
RESOURCE);
-    String path = ZooUtil.getRoot(InstanceId.of("instance")) + 
Constants.ZPROBLEMS + "/"
-        + Encoding.encodeAsBase64FileName(zpathFileName);
-    byte[] encoded = encodeReportData(now, SERVER, null);
-    expect(zoorw.putPersistentData(eq(path), aryEq(encoded), 
eq(NodeExistsPolicy.OVERWRITE)))
-        .andReturn(true);
-    replay(zoorw);
-
-    r.saveToZooKeeper(context);
-    verify(zoorw);
-  }
-
-  @Test
-  public void testDecodeZooKeeperEntry() throws Exception {
-    byte[] zpathFileName = makeZPathFileName(TABLE_ID, ProblemType.FILE_READ, 
RESOURCE);
-    String node = Encoding.encodeAsBase64FileName(zpathFileName);
-    long now = System.currentTimeMillis();
-    byte[] encoded = encodeReportData(now, SERVER, "excmsg");
-
-    expect(zoorw
-        .getData(ZooUtil.getRoot(InstanceId.of("instance")) + 
Constants.ZPROBLEMS + "/" + node))
-        .andReturn(encoded);
-    replay(zoorw);
-
-    r = ProblemReport.decodeZooKeeperEntry(context, node);
-    assertEquals(TABLE_ID, r.getTableId());
-    assertSame(ProblemType.FILE_READ, r.getProblemType());
-    assertEquals(RESOURCE, r.getResource());
-    assertEquals(SERVER, r.getServer());
-    assertEquals(now, r.getTime());
-    assertEquals("excmsg", r.getException());
-  }
-}
diff --git 
a/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportingIteratorTest.java
 
b/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportingIteratorTest.java
index 36f07d72ee..db041a8d9a 100644
--- 
a/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportingIteratorTest.java
+++ 
b/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportingIteratorTest.java
@@ -50,7 +50,7 @@ public class ProblemReportingIteratorTest {
   @BeforeEach
   public void setUp() {
     ii = EasyMock.createMock(InterruptibleIterator.class);
-    pri = new ProblemReportingIterator(null, TABLE_ID, RESOURCE, false, ii);
+    pri = new ProblemReportingIterator(TABLE_ID, RESOURCE, false, ii);
   }
 
   @Test
diff --git 
a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/CleanUp.java
 
b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/CleanUp.java
index 38a6f1cd5d..c45fa1e54e 100644
--- 
a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/CleanUp.java
+++ 
b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/delete/CleanUp.java
@@ -48,7 +48,6 @@ import org.apache.accumulo.manager.tableOps.ManagerRepo;
 import org.apache.accumulo.manager.tableOps.Utils;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.manager.state.MetaDataTableScanner;
-import org.apache.accumulo.server.problems.ProblemReports;
 import org.apache.accumulo.server.util.MetadataTableUtil;
 import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
@@ -165,13 +164,6 @@ class CleanUp extends ManagerRepo {
       log.error("error deleting " + tableId + " from metadata table", e);
     }
 
-    // remove any problem reports the table may have
-    try {
-      
ProblemReports.getInstance(manager.getContext()).deleteProblemReports(tableId);
-    } catch (Exception e) {
-      log.error("Failed to delete problem reports for table " + tableId, e);
-    }
-
     if (refCount == 0) {
       // delete the data files
       try {
diff --git 
a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java
 
b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java
index ffdd6ba506..704f3c3d50 100644
--- 
a/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java
+++ 
b/server/manager/src/main/java/org/apache/accumulo/manager/upgrade/Upgrader11to12.java
@@ -19,10 +19,15 @@
 package org.apache.accumulo.manager.upgrade;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.Objects.requireNonNull;
 import static org.apache.accumulo.core.metadata.RootTable.ZROOT_TABLET;
+import static 
org.apache.accumulo.core.metadata.schema.MetadataSchema.RESERVED_PREFIX;
 import static 
org.apache.accumulo.server.AccumuloDataVersion.METADATA_FILE_JSON_ENCODING;
 
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
 import java.io.IOException;
+import java.io.UncheckedIOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -43,6 +48,7 @@ import org.apache.accumulo.core.clientImpl.NamespaceMapping;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil;
 import org.apache.accumulo.core.metadata.AccumuloTable;
@@ -53,7 +59,9 @@ import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Ch
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import 
org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ExternalCompactionColumnFamily;
 import org.apache.accumulo.core.metadata.schema.RootTabletMetadata;
+import org.apache.accumulo.core.schema.Section;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.Encoding;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.init.FileSystemInitializer;
@@ -144,6 +152,9 @@ public class Upgrader11to12 implements Upgrader {
         zrw.delete(namespaceNamePath);
       }
 
+      log.info("Removing problems reports from zookeeper");
+      removeZKProblemReports(context);
+
     } catch (InterruptedException ex) {
       Thread.currentThread().interrupt();
       throw new IllegalStateException(
@@ -172,6 +183,8 @@ public class Upgrader11to12 implements Upgrader {
     upgradeTabletsMetadata(context, metaName);
     removeScanServerRange(context, metaName);
     createScanServerRefTable(context);
+    log.info("Removing problems reports from metadata table");
+    removeMetadataProblemReports(context);
   }
 
   private void upgradeTabletsMetadata(@NonNull ServerContext context, String 
metaName) {
@@ -289,4 +302,138 @@ public class Upgrader11to12 implements Upgrader {
     }
     log.info("Created ScanServerRef table");
   }
+
+  private static final String ZPROBLEMS = "/problems";
+
+  private void removeZKProblemReports(ServerContext context) {
+    String zpath = context.getZooKeeperRoot() + ZPROBLEMS;
+    try {
+      if (!context.getZooReaderWriter().exists(zpath)) {
+        // could be running a second time and the node was already deleted
+        return;
+      }
+      var children = context.getZooReaderWriter().getChildren(zpath);
+      for (var child : children) {
+        var pr = ProblemReport.decodeZooKeeperEntry(context, child);
+        logProblemDeletion(pr);
+      }
+      context.getZooReaderWriter().recursiveDelete(zpath, 
ZooUtil.NodeMissingPolicy.SKIP);
+    } catch (Exception e) {
+      throw new IllegalStateException(e);
+    }
+  }
+
+  /**
+   * Holds error message processing flags
+   */
+  private static class ProblemSection {
+    private static final Section section =
+        new Section(RESERVED_PREFIX + "err_", true, RESERVED_PREFIX + "err`", 
false);
+
+    public static Range getRange() {
+      return section.getRange();
+    }
+
+    public static String getRowPrefix() {
+      return section.getRowPrefix();
+    }
+  }
+
+  private void removeMetadataProblemReports(ServerContext context) {
+    try (
+        var scanner =
+            context.createScanner(AccumuloTable.METADATA.tableName(), 
Authorizations.EMPTY);
+        var writer = 
context.createBatchWriter(AccumuloTable.METADATA.tableName())) {
+      scanner.setRange(ProblemSection.getRange());
+      for (Map.Entry<Key,Value> entry : scanner) {
+        var pr = ProblemReport.decodeMetadataEntry(entry.getKey(), 
entry.getValue());
+        logProblemDeletion(pr);
+        Mutation m = new Mutation(entry.getKey().getRow());
+        m.putDelete(entry.getKey().getColumnFamily(), 
entry.getKey().getColumnQualifier());
+        writer.addMutation(m);
+      }
+    } catch (TableNotFoundException | MutationsRejectedException e) {
+      throw new IllegalStateException(e);
+    }
+  }
+
+  private void logProblemDeletion(ProblemReport pr) {
+    log.info(
+        "Deleting problem report tableId:{} type:{} resource:{} server:{} 
time:{} exception:{}",
+        pr.tableId, pr.problemType, pr.resource, pr.server, pr.creationTime, 
pr.exception);
+  }
+
+  public enum ProblemType {
+    FILE_READ, FILE_WRITE, TABLET_LOAD
+  }
+
+  private static class ProblemReport {
+    private final TableId tableId;
+    private final ProblemType problemType;
+    private final String resource;
+    private String exception;
+    private String server;
+    private long creationTime;
+
+    private ProblemReport(TableId table, ProblemType problemType, String 
resource, byte[] enc) {
+      requireNonNull(table, "table is null");
+      requireNonNull(problemType, "problemType is null");
+      requireNonNull(resource, "resource is null");
+      this.tableId = table;
+      this.problemType = problemType;
+      this.resource = resource;
+
+      decode(enc);
+    }
+
+    private void decode(byte[] enc) {
+      try {
+        ByteArrayInputStream bais = new ByteArrayInputStream(enc);
+        DataInputStream dis = new DataInputStream(bais);
+
+        creationTime = dis.readLong();
+
+        if (dis.readBoolean()) {
+          server = dis.readUTF();
+        } else {
+          server = null;
+        }
+
+        if (dis.readBoolean()) {
+          exception = dis.readUTF();
+        } else {
+          exception = null;
+        }
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+    }
+
+    static ProblemReport decodeZooKeeperEntry(ServerContext context, String 
node)
+        throws IOException, KeeperException, InterruptedException {
+      byte[] bytes = Encoding.decodeBase64FileName(node);
+
+      ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
+      DataInputStream dis = new DataInputStream(bais);
+
+      TableId tableId = TableId.of(dis.readUTF());
+      String problemType = dis.readUTF();
+      String resource = dis.readUTF();
+
+      String zpath = context.getZooKeeperRoot() + ZPROBLEMS + "/" + node;
+      byte[] enc = context.getZooReaderWriter().getData(zpath);
+
+      return new ProblemReport(tableId, ProblemType.valueOf(problemType), 
resource, enc);
+
+    }
+
+    public static ProblemReport decodeMetadataEntry(Key key, Value value) {
+      TableId tableId =
+          
TableId.of(key.getRow().toString().substring(ProblemSection.getRowPrefix().length()));
+      String problemType = key.getColumnFamily().toString();
+      String resource = key.getColumnQualifier().toString();
+
+      return new ProblemReport(tableId, ProblemType.valueOf(problemType), 
resource, value.get());
+    }
+  }
 }
diff --git 
a/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader11to12Test.java
 
b/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader11to12Test.java
index d0c1bbf1cf..ca5e57e7a3 100644
--- 
a/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader11to12Test.java
+++ 
b/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/Upgrader11to12Test.java
@@ -397,6 +397,8 @@ public class Upgrader11to12Test {
       expectLastCall().once();
     }
 
+    expect(zrw.exists(eq(zkRoot + "/problems"))).andReturn(false).once();
+
     replay(context, zrw);
 
     upgrader.upgradeZookeeper(context);
diff --git 
a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java 
b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
index 3b2e618456..8f0f0ea7a0 100644
--- a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
+++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java
@@ -48,7 +48,6 @@ import 
org.apache.accumulo.core.compaction.thrift.CompactionCoordinatorService;
 import org.apache.accumulo.core.compaction.thrift.TExternalCompaction;
 import org.apache.accumulo.core.compaction.thrift.TExternalCompactionList;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.core.fate.zookeeper.ZooUtil.NodeMissingPolicy;
@@ -81,8 +80,6 @@ import 
org.apache.accumulo.monitor.rest.compactions.external.RunningCompactorDet
 import org.apache.accumulo.server.AbstractServer;
 import org.apache.accumulo.server.HighlyAvailableService;
 import org.apache.accumulo.server.ServerContext;
-import org.apache.accumulo.server.problems.ProblemReports;
-import org.apache.accumulo.server.problems.ProblemType;
 import org.apache.accumulo.server.util.TableInfoUtil;
 import org.apache.zookeeper.KeeperException;
 import org.eclipse.jetty.servlet.DefaultServlet;
@@ -142,8 +139,6 @@ public class Monitor extends AbstractServer implements 
HighlyAvailableService {
 
   private final AtomicBoolean fetching = new AtomicBoolean(false);
   private ManagerMonitorInfo mmi;
-  private Map<TableId,Map<ProblemType,Integer>> problemSummary = 
Collections.emptyMap();
-  private Exception problemException;
   private GCStatus gcStatus;
   private Optional<HostAndPort> coordinatorHost = Optional.empty();
   private long coordinatorCheckNanos = 0L;
@@ -299,14 +294,6 @@ public class Monitor extends AbstractServer implements 
HighlyAvailableService {
         this.totalLookups = totalLookups;
 
       }
-      try {
-        this.problemSummary = ProblemReports.getInstance(context).summarize();
-        this.problemException = null;
-      } catch (Exception e) {
-        log.info("Failed to obtain problem reports ", e);
-        this.problemSummary = Collections.emptyMap();
-        this.problemException = e;
-      }
 
       // check for compaction coordinator host and only notify its discovery
       Optional<HostAndPort> previousHost;
@@ -874,14 +861,6 @@ public class Monitor extends AbstractServer implements 
HighlyAvailableService {
     return totalHoldTime;
   }
 
-  public Exception getProblemException() {
-    return problemException;
-  }
-
-  public Map<TableId,Map<ProblemType,Integer>> getProblemSummary() {
-    return problemSummary;
-  }
-
   public GCStatus getGcStatus() {
     return gcStatus;
   }
diff --git 
a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemDetail.java
 
b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemDetail.java
deleted file mode 100644
index 979ff53fa9..0000000000
--- 
a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemDetail.java
+++ /dev/null
@@ -1,42 +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
- *
- *   https://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.accumulo.monitor.rest.problems;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Stores a new problem detail object
- *
- * @since 2.0.0
- */
-public class ProblemDetail {
-
-  // Variable names become JSON keys
-  public List<ProblemDetailInformation> problemDetails = new ArrayList<>();
-
-  /**
-   * Adds a new problem detail to the list
-   *
-   * @param problemDetail problem detail to add
-   */
-  public void addProblemDetail(ProblemDetailInformation problemDetail) {
-    this.problemDetails.add(problemDetail);
-  }
-}
diff --git 
a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemDetailInformation.java
 
b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemDetailInformation.java
deleted file mode 100644
index 61119a321f..0000000000
--- 
a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemDetailInformation.java
+++ /dev/null
@@ -1,69 +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
- *
- *   https://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.accumulo.monitor.rest.problems;
-
-import jakarta.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.monitor.util.JaxbAbstractIdSerializer;
-
-/**
- * Generates a problem detail as a JSON object
- *
- * @since 2.0.0
- */
-public class ProblemDetailInformation {
-
-  // Variable names become JSON keys
-  public String tableName;
-
-  @XmlJavaTypeAdapter(JaxbAbstractIdSerializer.class)
-  public TableId tableID;
-
-  public String type;
-  public String server;
-
-  public Long time;
-  public String resource;
-  public String exception;
-
-  public ProblemDetailInformation() {}
-
-  /**
-   * Stores a problem detail
-   *
-   * @param tableName Table name of the problem
-   * @param tableId Table ID of the problem
-   * @param type Type of problem
-   * @param server Location of the problem
-   * @param time Time of the problem
-   * @param resource Resource with the problem
-   * @param exception Exception of the problem
-   */
-  public ProblemDetailInformation(String tableName, TableId tableId, String 
type, String server,
-      Long time, String resource, String exception) {
-    this.tableName = tableName;
-    this.tableID = tableId;
-    this.type = type;
-    this.server = server;
-    this.time = time;
-    this.resource = resource;
-    this.exception = exception;
-  }
-}
diff --git 
a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemSummary.java
 
b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemSummary.java
deleted file mode 100644
index 32e596f50c..0000000000
--- 
a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemSummary.java
+++ /dev/null
@@ -1,42 +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
- *
- *   https://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.accumulo.monitor.rest.problems;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Stores a new problem summary object
- *
- * @since 2.0.0
- */
-public class ProblemSummary {
-
-  // Variable names become JSON keys
-  public List<ProblemSummaryInformation> problemSummary = new ArrayList<>();
-
-  /**
-   * Adds a new problem summary to the list
-   *
-   * @param problemSummary problem summary to add
-   */
-  public void addProblemSummary(ProblemSummaryInformation problemSummary) {
-    this.problemSummary.add(problemSummary);
-  }
-}
diff --git 
a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemSummaryInformation.java
 
b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemSummaryInformation.java
deleted file mode 100644
index c18a1aabeb..0000000000
--- 
a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemSummaryInformation.java
+++ /dev/null
@@ -1,62 +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
- *
- *   https://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.accumulo.monitor.rest.problems;
-
-import jakarta.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.monitor.util.JaxbAbstractIdSerializer;
-
-/**
- * Generates a problem summary object
- *
- * @since 2.0.0
- */
-public class ProblemSummaryInformation {
-
-  // Variable names become JSON keys
-  public String tableName;
-
-  @XmlJavaTypeAdapter(JaxbAbstractIdSerializer.class)
-  public TableId tableID;
-
-  public Integer fileRead;
-  public Integer fileWrite;
-  public Integer tableLoad;
-
-  public ProblemSummaryInformation() {}
-
-  /**
-   * Stores a single problem summary object
-   *
-   * @param tableName Name of the table with a problem
-   * @param tableId ID of the table with a problem
-   * @param fileRead Number of files read
-   * @param fileWrite Number of files written
-   * @param tableLoad Number of table loads
-   */
-  public ProblemSummaryInformation(String tableName, TableId tableId, Integer 
fileRead,
-      Integer fileWrite, Integer tableLoad) {
-    this.tableName = tableName;
-    this.tableID = tableId;
-    this.fileRead = fileRead;
-    this.fileWrite = fileWrite;
-    this.tableLoad = tableLoad;
-  }
-}
diff --git 
a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemsResource.java
 
b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemsResource.java
deleted file mode 100644
index b53f5a0370..0000000000
--- 
a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/problems/ProblemsResource.java
+++ /dev/null
@@ -1,185 +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
- *
- *   https://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.accumulo.monitor.rest.problems;
-
-import static 
org.apache.accumulo.monitor.util.ParameterValidator.ALPHA_NUM_REGEX_TABLE_ID;
-import static 
org.apache.accumulo.monitor.util.ParameterValidator.PROBLEM_TYPE_REGEX;
-import static 
org.apache.accumulo.monitor.util.ParameterValidator.RESOURCE_REGEX;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import jakarta.inject.Inject;
-import jakarta.validation.constraints.NotNull;
-import jakarta.validation.constraints.Pattern;
-import jakarta.ws.rs.Consumes;
-import jakarta.ws.rs.GET;
-import jakarta.ws.rs.POST;
-import jakarta.ws.rs.Path;
-import jakarta.ws.rs.Produces;
-import jakarta.ws.rs.QueryParam;
-import jakarta.ws.rs.core.MediaType;
-
-import org.apache.accumulo.core.data.TableId;
-import org.apache.accumulo.monitor.Monitor;
-import org.apache.accumulo.server.problems.ProblemReport;
-import org.apache.accumulo.server.problems.ProblemReports;
-import org.apache.accumulo.server.problems.ProblemType;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Generates a problem summary and details as a JSON object
- *
- * @since 2.0.0
- */
-@Path("/problems")
-@Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
-public class ProblemsResource {
-
-  @Inject
-  private Monitor monitor;
-
-  /**
-   * Generates a list with the problem summary
-   *
-   * @return problem summary list
-   */
-  @GET
-  @Path("summary")
-  public ProblemSummary getSummary() {
-
-    ProblemSummary problems = new ProblemSummary();
-
-    if (monitor.getProblemException() == null) {
-      for (Entry<TableId,Map<ProblemType,Integer>> entry : 
monitor.getProblemSummary().entrySet()) {
-        Integer readCount = null, writeCount = null, loadCount = null;
-
-        for (ProblemType pt : ProblemType.values()) {
-          Integer pcount = entry.getValue().get(pt);
-          if (pt.equals(ProblemType.FILE_READ)) {
-            readCount = pcount;
-          } else if (pt.equals(ProblemType.FILE_WRITE)) {
-            writeCount = pcount;
-          } else if (pt.equals(ProblemType.TABLET_LOAD)) {
-            loadCount = pcount;
-          }
-        }
-
-        String tableName = 
monitor.getContext().getPrintableTableInfoFromId(entry.getKey());
-
-        problems.addProblemSummary(new ProblemSummaryInformation(tableName, 
entry.getKey(),
-            readCount, writeCount, loadCount));
-      }
-    }
-    return problems;
-  }
-
-  /**
-   * REST call to clear problem reports from a table
-   *
-   * @param tableID Table ID to clear problems
-   */
-  @POST
-  @Consumes(MediaType.TEXT_PLAIN)
-  @Path("summary")
-  public void clearTableProblems(
-      @QueryParam("s") @NotNull @Pattern(regexp = ALPHA_NUM_REGEX_TABLE_ID) 
String tableID) {
-    Logger log = LoggerFactory.getLogger(Monitor.class);
-    try {
-      
ProblemReports.getInstance(monitor.getContext()).deleteProblemReports(TableId.of(tableID));
-    } catch (Exception e) {
-      log.error("Failed to delete problem reports for table "
-          + (tableID.isEmpty() ? "" : sanitize(tableID)), e);
-    }
-  }
-
-  /**
-   * Generates a list of the problem details as a JSON object
-   *
-   * @return problem details list
-   */
-  @GET
-  @Path("details")
-  public ProblemDetail getDetails() {
-
-    ProblemDetail problems = new ProblemDetail();
-
-    if (monitor.getProblemException() == null) {
-      for (Entry<TableId,Map<ProblemType,Integer>> entry : 
monitor.getProblemSummary().entrySet()) {
-        ArrayList<ProblemReport> problemReports = new ArrayList<>();
-        Iterator<ProblemReport> iter =
-            entry.getKey() == null ? 
ProblemReports.getInstance(monitor.getContext()).iterator()
-                : 
ProblemReports.getInstance(monitor.getContext()).iterator(entry.getKey());
-        while (iter.hasNext()) {
-          problemReports.add(iter.next());
-        }
-        for (ProblemReport pr : problemReports) {
-          String tableName = 
monitor.getContext().getPrintableTableInfoFromId(pr.getTableId());
-
-          problems.addProblemDetail(
-              new ProblemDetailInformation(tableName, entry.getKey(), 
pr.getProblemType().name(),
-                  pr.getServer(), pr.getTime(), pr.getResource(), 
pr.getException()));
-        }
-      }
-    }
-    return problems;
-  }
-
-  /**
-   * REST call to clear specific problem details
-   *
-   * @param tableID Table ID to clear
-   * @param resource Resource to clear
-   * @param ptype Problem type to clear
-   */
-  @POST
-  @Consumes(MediaType.TEXT_PLAIN)
-  @Path("details")
-  public void clearDetailsProblems(
-      @QueryParam("table") @NotNull @Pattern(regexp = 
ALPHA_NUM_REGEX_TABLE_ID) String tableID,
-      @QueryParam("resource") @NotNull @Pattern(regexp = RESOURCE_REGEX) 
String resource,
-      @QueryParam("ptype") @NotNull @Pattern(regexp = PROBLEM_TYPE_REGEX) 
String ptype) {
-    Logger log = LoggerFactory.getLogger(Monitor.class);
-    try {
-      
ProblemReports.getInstance(monitor.getContext()).deleteProblemReport(TableId.of(tableID),
-          ProblemType.valueOf(ptype), resource);
-    } catch (Exception e) {
-      log.error("Failed to delete problem reports for table "
-          + (tableID.isBlank() ? "" : sanitize(tableID)), e);
-    }
-  }
-
-  /**
-   * Prevent potential CRLF injection into logs from read in user data. See the
-   * <a 
href="https://find-sec-bugs.github.io/bugs.htm#CRLF_INJECTION_LOGS";>bug 
description</a>
-   */
-  private String sanitize(String msg) {
-    return msg.replaceAll("[\r\n]", "");
-  }
-
-  @GET
-  @Path("exception")
-  public Exception getException() {
-    return monitor.getProblemException();
-  }
-
-}
diff --git 
a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusInformation.java
 
b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusInformation.java
index 867fb1d2bb..d35b3214bd 100644
--- 
a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusInformation.java
+++ 
b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusInformation.java
@@ -31,8 +31,6 @@ public class StatusInformation {
   public String tServerStatus = null;
   public String coordinatorStatus = null;
 
-  public Integer problemNumber = 0;
-
   public StatusInformation() {}
 
   /**
@@ -42,15 +40,13 @@ public class StatusInformation {
    * @param gcStatus Status for the GC
    * @param tServerStatus Status for the tserver
    * @param coordinatorStatus Status for the Compaction Coordinator
-   * @param problemNumber Number of problems per table
    */
   public StatusInformation(String managerStatus, String gcStatus, String 
tServerStatus,
-      String coordinatorStatus, Integer problemNumber) {
+      String coordinatorStatus) {
     this.managerStatus = managerStatus;
     this.gcStatus = gcStatus;
     this.tServerStatus = tServerStatus;
     this.coordinatorStatus = coordinatorStatus;
-    this.problemNumber = problemNumber;
 
   }
 }
diff --git 
a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusResource.java
 
b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusResource.java
index e94f5e2148..a9e5d3bfa3 100644
--- 
a/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusResource.java
+++ 
b/server/monitor/src/main/java/org/apache/accumulo/monitor/rest/status/StatusResource.java
@@ -97,7 +97,6 @@ public class StatusResource {
     }
 
     return new StatusInformation(managerStatus.toString(), gcStatus.toString(),
-        tServerStatus.toString(), coordinatorStatus.toString(),
-        monitor.getProblemSummary().entrySet().size());
+        tServerStatus.toString(), coordinatorStatus.toString());
   }
 }
diff --git 
a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/functions.js
 
b/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/functions.js
index 1ce6c7776a..ebc3dc7d50 100644
--- 
a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/functions.js
+++ 
b/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/functions.js
@@ -18,8 +18,7 @@
  */
 /* JSLint global definitions */
 /*global
-    $, sessionStorage, TIMER:true, NAMESPACES:true, refreshNavBar,
-    refreshProblems
+    $, sessionStorage, TIMER:true, NAMESPACES:true, refreshNavBar
 */
 "use strict";
 
@@ -444,43 +443,6 @@ function getTableServers(tableID) {
   return getJSONForTable('/rest/tables/' + tableID, 'tableServers');
 }
 
-/**
- * REST POST call to clear all table problems
- *
- * @param {string} tableID Table ID
- */
-function clearTableProblems(tableID) {
-  doLoggedPostCall('/rest/problems/summary?s=' + tableID, refresh, true);
-}
-
-/**
- * REST POST call to clear detail problems
- *
- * @param {string} table Table ID
- * @param {string} resource Resource for problem
- * @param {string} type Type of problem
- */
-function clearDetailsProblems(table, resource, type) {
-  doLoggedPostCall('/rest/problems/details?table=' + table + '&resource=' +
-    resource + '&ptype=' + type, refresh, true);
-}
-
-/**
- * REST GET call for the problems summary,
- * stores it on a sessionStorage variable
- */
-function getProblemSummary() {
-  return getJSONForTable('/rest/problems/summary', 'problemSummary');
-}
-
-/**
- * REST GET call for the problems details,
- * stores it on a sessionStorage variable
- */
-function getProblemDetails() {
-  return getJSONForTable('/rest/problems/details', 'problemDetails');
-}
-
 /**
  * REST GET call for the server status, stores it on a sessionStorage variable
  */
diff --git 
a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/navbar.js
 
b/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/navbar.js
index fb46416756..de299a81a7 100644
--- 
a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/navbar.js
+++ 
b/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/navbar.js
@@ -119,39 +119,6 @@ function updateServerNotifications(statusData) {
   });
 }
 
-/**
- * Updates the notification color for the table problems icon within the debug 
dropdown
- */
-function updateTableProblemsNotification(statusData) {
-  if (statusData.problemNumber > 0) {
-    updateElementStatus('tableProblemsNotifications', STATUS.ERROR);
-  } else {
-    updateElementStatus('tableProblemsNotifications', STATUS.OK);
-  }
-  // Number
-  var problemNumber = statusData.problemNumber > 99 ? '99+' : 
statusData.problemNumber;
-  $('#tableProblemsNotifications').html(problemNumber);
-}
-
-/**
- * Updates the notification color for the debug dropdown icon
- */
-function updateDebugDropdownNotification(statusData) {
-  if (statusData.problemNumber > 0) {
-    if (statusData.problemNumber > 0) {
-      updateElementStatus('errorsNotification', STATUS.ERROR);
-    } else {
-      updateElementStatus('errorsNotification', STATUS.WARN);
-    }
-  } else {
-    updateElementStatus('errorsNotification', STATUS.OK);
-  }
-  // Number
-  var totalNumber = statusData.problemNumber > 99 ?
-    '99+' : statusData.problemNumber;
-  $('#errorsNotification').html(totalNumber);
-}
-
 /**
  * Creates the initial sidebar
  */
@@ -184,6 +151,5 @@ function refreshSideBarNotifications() {
   const statusData = sessionStorage?.status ? 
JSON.parse(sessionStorage.status) : undefined;
 
   updateServerNotifications(statusData);
-  updateTableProblemsNotification(statusData);
   updateDebugDropdownNotification(statusData);
 }
diff --git 
a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/problems.js
 
b/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/problems.js
deleted file mode 100644
index 051a40d06e..0000000000
--- 
a/server/monitor/src/main/resources/org/apache/accumulo/monitor/resources/js/problems.js
+++ /dev/null
@@ -1,167 +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
- *
- *   https://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.
- */
-"use strict";
-
-var tableID;
-var problemSummaryTable;
-var problemDetailTable;
-$(document).ready(function () {
-  // Create a table for summary. See datatables doc for more info on the dom 
property
-  problemSummaryTable = $('#problemSummary').DataTable({
-    "ajax": {
-      "url": '/rest/problems/summary',
-      "dataSrc": "problemSummary"
-    },
-    "stateSave": true,
-    "dom": 't<"align-left"l>p',
-    "columnDefs": [{
-      "targets": "big-num",
-      "render": function (data, type, row) {
-        if (type === 'display') data = bigNumberForQuantity(data);
-        return data;
-      }
-    }],
-    "columns": [{
-        "data": "tableName",
-        "type": "html",
-        "render": function (data, type, row, meta) {
-          if (type === 'display') data = '<a href="/tables/' + row.tableID + 
'">' + row.tableName + '</a>';
-          return data;
-        }
-      },
-      {
-        "data": "fileRead"
-      },
-      {
-        "data": "fileWrite"
-      },
-      {
-        "data": "tableLoad"
-      },
-      {
-        "data": "tableID",
-        "type": "html",
-        "render": function (data, type, row, meta) {
-          if (type === 'display') data = '<a 
href="javascript:clearTableProblemsTable(\'' +
-            row.tableID + '\');">clear ALL problems with table ' + 
row.tableName + '</a>';
-          return data;
-        }
-      }
-    ]
-  });
-  // Create a table for details
-  problemDetailTable = $('#problemDetails').DataTable({
-    "ajax": {
-      "url": '/rest/problems/details',
-      "dataSrc": "problemDetails"
-    },
-    "stateSave": true,
-    "dom": 't<"align-left"l>p',
-    "columnDefs": [{
-      "targets": "date",
-      "render": function (data, type, row) {
-        if (type === 'display') data = dateFormat(data);
-        return data;
-      }
-    }],
-    "columns": [{
-        "data": "tableName",
-        "type": "html",
-        "render": function (data, type, row, meta) {
-          if (type === 'display') data = '<a href="/tables/' + row.tableID + 
'">' + row.tableName + '</a>';
-          return data;
-        }
-      },
-      {
-        "data": "type"
-      },
-      {
-        "data": "server"
-      },
-      {
-        "data": "time"
-      },
-      {
-        "data": "resource"
-      },
-      {
-        "data": "exception"
-      },
-      {
-        "data": "tableID",
-        "type": "html",
-        "render": function (data, type, row, meta) {
-          if (type === 'display') data = '<a 
href="javascript:clearDetailsProblemsTable(\'' +
-            row.tableID + '\',\'' + row.resource + '\',\'' + row.type + 
'\')">clear this problem</a>';
-          return data;
-        }
-      }
-    ]
-  });
-});
-
-/**
- * Makes the REST calls, generates the tables with the new information
- */
-function refreshProblems() {
-  refreshNavBar();
-  refreshProblemSummaryTable();
-  refreshProblemDetailsTable();
-}
-
-/**
- * Used to redraw the page
- */
-function refresh() {
-  refreshProblems();
-}
-
-/**
- * Makes REST POST call to clear the problem summary table
- *
- * @param {string} tableID Table ID to clear
- */
-function clearTableProblemsTable(tableID) {
-  clearTableProblems(tableID);
-}
-
-/**
- * Makes REST POST call to clear the problem details table
- *
- * @param {string} table Table ID to clear
- * @param {string} resource Resource of problem
- * @param {string} type Type of problem
- */
-function clearDetailsProblemsTable(table, resource, type) {
-  clearDetailsProblems(table, resource, type);
-}
-
-/**
- * Generates the problem summary table
- */
-function refreshProblemSummaryTable() {
-  if (problemSummaryTable) problemSummaryTable.ajax.reload(null, false); // 
user paging is not reset on reload
-}
-
-/**
- * Generates the problem details table
- */
-function refreshProblemDetailsTable() {
-  if (problemDetailTable) problemDetailTable.ajax.reload(null, false); // user 
paging is not reset on reload
-}
diff --git 
a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/navbar.ftl
 
b/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/navbar.ftl
index 0d9dc494fa..7135936d88 100644
--- 
a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/navbar.ftl
+++ 
b/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/navbar.ftl
@@ -58,14 +58,6 @@
                 <li><a class="dropdown-item" 
href="/ec">External&nbsp;Compactions</a></li>
               </ul>
             </li>
-            <li class="dropdown">
-              <a class="nav-link dropdown-toggle" href="#" id="navbarDropdown"
-              role="button" data-bs-toggle="dropdown" 
aria-expanded="false">Debug&nbsp;<span id="errorsNotification" 
class="badge"></span><span class="caret"></span>
-              </a>
-              <ul class="dropdown-menu">
-                <li><a class="dropdown-item" 
href="/problems">Table&nbsp;Problems&nbsp;<span id="tableProblemsNotifications" 
class="badge"></span></a></li>
-              </ul>
-            </li>
             <li class="dropdown">
               <a class="nav-link dropdown-toggle" href="#" id="navbarDropdown"
               role="button" data-bs-toggle="dropdown" 
aria-expanded="false">REST
diff --git 
a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/problems.ftl
 
b/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/problems.ftl
deleted file mode 100644
index 50afaadfa5..0000000000
--- 
a/server/monitor/src/main/resources/org/apache/accumulo/monitor/templates/problems.ftl
+++ /dev/null
@@ -1,71 +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
-
-      https://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.
-
--->
-      <script>
-        /**
-         * Creates problems initial table, passes tableID from template
-         */
-        $(document).ready(function() {
-          tableID = <#if table??>'${table}'<#else>''</#if>;
-          refreshProblems();
-        });
-      </script>
-      <div class="row">
-        <div class="col-xs-12">
-          <h3>${title}</h3>
-        </div>
-      </div>
-      <div class="row">
-        <div class="col-xs-12">
-          <table id="problemSummary" class="table caption-top table-bordered 
table-striped table-condensed">
-            <caption><span class="table-caption">Summary</span></caption>
-            <thead>
-              <tr>
-                <th class="firstcell">Table&nbsp;</th>
-                <th class="big-num">FILE_READ&nbsp;</th>
-                <th class="big-num">FILE_WRITE&nbsp;</th>
-                <th class="big-num">TABLET_LOAD&nbsp;</th>
-                <th>Operations&nbsp;</th>
-              </tr>
-            </thead>
-            <tbody></tbody>
-          </table>
-        </div>
-      </div>
-      <br />
-      <div class="row">
-        <div class="col-xs-12">
-          <table id="problemDetails" class="table caption-top table-bordered 
table-striped table-condensed">
-            <caption><span class="table-caption">Details</span></caption>
-            <thead>
-              <tr>
-                <th class="firstcell">Table&nbsp;</th>
-                <th>Problem&nbsp;Type&nbsp;</th>
-                <th>Server&nbsp;</th>
-                <th class="date">Time&nbsp;</th>
-                <th>Resource&nbsp;</th>
-                <th>Exception&nbsp;</th>
-                <th>Operations&nbsp;</th>
-              </tr>
-            </thead>
-            <tbody></tbody>
-          </table>
-        </div>
-      </div>
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java
index 552d9f40a9..1d6f1fe244 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java
@@ -19,7 +19,6 @@
 package org.apache.accumulo.tserver;
 
 import static java.util.concurrent.TimeUnit.MINUTES;
-import static org.apache.accumulo.server.problems.ProblemType.TABLET_LOAD;
 
 import java.util.Arrays;
 import java.util.Set;
@@ -27,8 +26,8 @@ import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.data.TableId;
 import org.apache.accumulo.core.dataImpl.KeyExtent;
+import org.apache.accumulo.core.logging.TabletLogger;
 import org.apache.accumulo.core.manager.thrift.TabletLoadState;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.TServerInstance;
@@ -38,8 +37,6 @@ import org.apache.accumulo.core.util.threads.ThreadPools;
 import org.apache.accumulo.core.util.threads.Threads;
 import org.apache.accumulo.server.manager.state.Assignment;
 import org.apache.accumulo.server.manager.state.TabletStateStore;
-import org.apache.accumulo.server.problems.ProblemReport;
-import org.apache.accumulo.server.problems.ProblemReports;
 import org.apache.accumulo.server.util.ManagerMetadataUtil;
 import 
org.apache.accumulo.tserver.TabletServerResourceManager.TabletResourceManager;
 import org.apache.accumulo.tserver.managermessage.TabletStatusMessage;
@@ -137,7 +134,7 @@ class AssignmentHandler implements Runnable {
         server.openingTablets.remove(extent);
         server.openingTablets.notifyAll();
       }
-      log.warn("Failed to verify tablet " + extent, e);
+      TabletLogger.tabletLoadFailed(extent, e);
       server.enqueueManagerMessage(new 
TabletStatusMessage(TabletLoadState.LOAD_FAILURE, extent));
       throw new RuntimeException(e);
     }
@@ -194,15 +191,7 @@ class AssignmentHandler implements Runnable {
       tablet = null; // release this reference
       successful = true;
     } catch (Exception e) {
-      log.warn("exception trying to assign tablet {} {}", extent, 
locationToOpen, e);
-
-      if (e.getMessage() != null) {
-        log.warn("{}", e.getMessage());
-      }
-
-      TableId tableId = extent.tableId();
-      ProblemReports.getInstance(server.getContext()).report(new 
ProblemReport(tableId, TABLET_LOAD,
-          extent.getUUID().toString(), server.getClientAddressString(), e));
+      TabletLogger.tabletLoadFailed(extent, e);
     }
 
     if (successful) {
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
index 2a3dc01809..9dc42f6386 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinorCompactor.java
@@ -35,9 +35,6 @@ import org.apache.accumulo.core.util.LocalityGroupUtil;
 import org.apache.accumulo.server.compaction.CompactionStats;
 import org.apache.accumulo.server.compaction.FileCompactor;
 import org.apache.accumulo.server.conf.TableConfiguration;
-import org.apache.accumulo.server.problems.ProblemReport;
-import org.apache.accumulo.server.problems.ProblemReports;
-import org.apache.accumulo.server.problems.ProblemType;
 import org.apache.accumulo.tserver.InMemoryMap;
 import org.apache.accumulo.tserver.MinorCompactionReason;
 import org.apache.accumulo.tserver.TabletServer;
@@ -86,7 +83,6 @@ public class MinorCompactor extends FileCompactor {
     int sleepTime = 100;
     double growthFactor = 4;
     int maxSleepTime = 1000 * 60 * 3; // 3 minutes
-    boolean reportedProblem = false;
     int retryCounter = 0;
 
     runningCompactions.add(this);
@@ -99,25 +95,15 @@ public class MinorCompactor extends FileCompactor {
           // %,d bytes ",map.size(), entriesCompacted,
           // (int)(map.size()/((t2 - t1)/1000.0)), (t2 - t1)/1000.0, 
estimatedSizeInBytes()));
 
-          if (reportedProblem) {
-            ProblemReports.getInstance(tabletServer.getContext())
-                .deleteProblemReport(getExtent().tableId(), 
ProblemType.FILE_WRITE, outputFileName);
-          }
-
           return ret;
         } catch (IOException | UnsatisfiedLinkError e) {
           log.warn("MinC failed ({}) to create {} retrying ...", 
e.getMessage(), outputFileName);
-          ProblemReports.getInstance(tabletServer.getContext()).report(
-              new ProblemReport(getExtent().tableId(), ProblemType.FILE_WRITE, 
outputFileName, e));
-          reportedProblem = true;
         } catch (RuntimeException | NoClassDefFoundError e) {
           // if this is coming from a user iterator, it is possible that the 
user could change the
           // iterator config and that the minor compaction would succeed
           // If the minor compaction stalls for too long during recovery, it 
can interfere with
           // other tables loading
           // Throw exception if this happens so assignments can be rescheduled.
-          ProblemReports.getInstance(tabletServer.getContext()).report(
-              new ProblemReport(getExtent().tableId(), ProblemType.FILE_WRITE, 
outputFileName, e));
           if (retryCounter >= 4 && 
mincReason.equals(MinorCompactionReason.RECOVERY)) {
             log.warn(
                 "MinC ({}) is stuck for too long during recovery, throwing 
error to reschedule.",
@@ -125,7 +111,6 @@ public class MinorCompactor extends FileCompactor {
             throw new RuntimeException(e);
           }
           log.warn("MinC failed ({}) to create {} retrying ...", 
e.getMessage(), outputFileName, e);
-          reportedProblem = true;
           retryCounter++;
         } catch (CompactionCanceledException e) {
           throw new IllegalStateException(e);
diff --git 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 53cba11d06..7482e75a6e 100644
--- 
a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ 
b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -99,9 +99,6 @@ import 
org.apache.accumulo.server.compaction.PausedCompactionMetrics;
 import org.apache.accumulo.server.fs.VolumeChooserEnvironmentImpl;
 import org.apache.accumulo.server.fs.VolumeUtil;
 import org.apache.accumulo.server.fs.VolumeUtil.TabletFiles;
-import org.apache.accumulo.server.problems.ProblemReport;
-import org.apache.accumulo.server.problems.ProblemReports;
-import org.apache.accumulo.server.problems.ProblemType;
 import org.apache.accumulo.server.tablets.TabletTime;
 import org.apache.accumulo.server.tablets.UniqueNameAllocator;
 import org.apache.accumulo.server.util.FileUtil;
@@ -1093,8 +1090,6 @@ public class Tablet extends TabletBase {
         }
       }
       if (err != null) {
-        ProblemReports.getInstance(context).report(new 
ProblemReport(extent.tableId(),
-            ProblemType.TABLET_LOAD, this.extent.toString(), err));
         log.error("Tablet closed consistency check has failed for {} giving up 
and closing",
             this.extent);
       }


Reply via email to