You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by dl...@apache.org on 2023/06/01 11:22:11 UTC

[accumulo] branch 2.1 updated: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works (#3436)

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

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


The following commit(s) were added to refs/heads/2.1 by this push:
     new d4a55591dc Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works (#3436)
d4a55591dc is described below

commit d4a55591dc83cce4eb3e53a9a293b68bb636030d
Author: Dave Marion <dl...@apache.org>
AuthorDate: Thu Jun 1 07:22:05 2023 -0400

    Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works (#3436)
    
    This commit:
      1. Deprecates GC_TRASH_IGNORE
      2. Modifies MiniAccumuloCluster so that the user can provide
         Hadoop properties for the Hadoop configuration that is used
         by the MiniDFS cluster.
      3. Adds ITs that confirm:
        a. Deleted files do not get put into the Trash in the default
           configuration (Accumulo enabled / Hadoop disabled) because
           the default value for fs.trash.interval is zero, which
           means disabled.
        b. Deleted files do get put into the Trash when the default
           Accumulo configuration is used (GC_TRASH_IGNORE = false)
           and fs.trash.interval is set to a nonzero value.
        c. Deleted files do not get put into the Trash when
           GC_TRASH_IGNORE is set to true and fs.trash.interval is
           set to a nonzero value.
        d. Deleted files do not get put into the Trash when Accumulo
           and Hadoop are enabled, but a custom Trash policy is used
           that filters file names.
---
 .../org/apache/accumulo/core/conf/Property.java    |   1 +
 .../miniclusterImpl/MiniAccumuloClusterImpl.java   |   1 +
 .../miniclusterImpl/MiniAccumuloConfigImpl.java    |  21 ++++
 .../accumulo/server/fs/VolumeManagerImpl.java      |   4 +
 .../main/java/org/apache/accumulo/gc/GCRun.java    |  10 +-
 .../apache/accumulo/gc/SimpleGarbageCollector.java |   4 +-
 .../accumulo/gc/SimpleGarbageCollectorTest.java    |   6 +-
 .../manager/upgrade/RootFilesUpgradeTest.java      |   4 +-
 .../test/functional/GarbageCollectorTrashBase.java | 125 +++++++++++++++++++++
 .../functional/GarbageCollectorTrashDefaultIT.java |  88 +++++++++++++++
 .../GarbageCollectorTrashDisabledIT.java           |  86 ++++++++++++++
 ...GarbageCollectorTrashEnabledCustomPolicyIT.java | 114 +++++++++++++++++++
 .../functional/GarbageCollectorTrashEnabledIT.java |  84 ++++++++++++++
 test/src/main/resources/log4j2-test.properties     |   3 +
 14 files changed, 545 insertions(+), 6 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index a47a0992ac..2f5a6201ea 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -873,6 +873,7 @@ public enum Property {
       "The listening port for the garbage collector's monitor service", "1.3.5"),
   GC_DELETE_THREADS("gc.threads.delete", "16", PropertyType.COUNT,
       "The number of threads used to delete RFiles and write-ahead logs", "1.3.5"),
+  @Deprecated(since = "2.1.1", forRemoval = true)
   GC_TRASH_IGNORE("gc.trash.ignore", "false", PropertyType.BOOLEAN,
       "Do not use the Trash, even if it is configured.", "1.5.0"),
   @Deprecated(since = "2.1.0", forRemoval = true)
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
index a5f8b08b08..a6470efe83 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
@@ -195,6 +195,7 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
       conf.set("dfs.support.append", "true");
       conf.set("dfs.datanode.synconclose", "true");
       conf.set("dfs.datanode.data.dir.perm", MiniDFSUtil.computeDatanodeDirectoryPermission());
+      config.getHadoopConfOverrides().forEach((k, v) -> conf.set(k, v));
       String oldTestBuildData = System.setProperty("test.build.data", dfs.getAbsolutePath());
       miniDFS.set(new MiniDFSCluster.Builder(conf).build());
       if (oldTestBuildData == null) {
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
index 69f3bc99d4..9aac388e74 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
@@ -54,6 +54,7 @@ public class MiniAccumuloConfigImpl {
 
   private File dir = null;
   private String rootPassword = null;
+  private Map<String,String> hadoopConfOverrides = new HashMap<>();
   private Map<String,String> siteConfig = new HashMap<>();
   private Map<String,String> configuredSiteConig = new HashMap<>();
   private Map<String,String> clientProps = new HashMap<>();
@@ -836,4 +837,24 @@ public class MiniAccumuloConfigImpl {
       this.preStartConfigProcessor.accept(this);
     }
   }
+
+  /**
+   * Add server-side Hadoop configuration properties
+   *
+   * @param overrides properties
+   * @since 2.1.1
+   */
+  public void setHadoopConfOverrides(Map<String,String> overrides) {
+    hadoopConfOverrides.putAll(overrides);
+  }
+
+  /**
+   * Get server-side Hadoop configuration properties
+   *
+   * @return map of properties set in prior call to {@link #setHadoopConfOverrides(Map)}
+   * @since 2.1.1
+   */
+  public Map<String,String> getHadoopConfOverrides() {
+    return new HashMap<>(hadoopConfOverrides);
+  }
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
index ab0d86f6a9..755b1e4be9 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
@@ -49,6 +49,7 @@ import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.core.volume.VolumeImpl;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -354,7 +355,10 @@ public class VolumeManagerImpl implements VolumeManager {
   @Override
   public boolean moveToTrash(Path path) throws IOException {
     FileSystem fs = getFileSystemByPath(path);
+    String key = CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
+    log.trace("{}: {}", key, fs.getConf().get(key));
     Trash trash = new Trash(fs, fs.getConf());
+    log.trace("Hadoop Trash is enabled for {}: {}", path, trash.isEnabled());
     return trash.moveToTrash(path);
   }
 
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java b/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java
index 699e793d34..b96d14b6d5 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java
@@ -463,11 +463,15 @@ public class GCRun implements GarbageCollectionEnvironment {
   boolean moveToTrash(Path path) throws IOException {
     final VolumeManager fs = context.getVolumeManager();
     if (!isUsingTrash()) {
+      log.trace("Accumulo Trash is disabled. Skipped for {}", path);
       return false;
     }
     try {
-      return fs.moveToTrash(path);
+      boolean success = fs.moveToTrash(path);
+      log.trace("Accumulo Trash enabled, moving to trash succeeded?: {}", success);
+      return success;
     } catch (FileNotFoundException ex) {
+      log.error("Error moving {} to trash", path, ex);
       return false;
     }
   }
@@ -478,7 +482,9 @@ public class GCRun implements GarbageCollectionEnvironment {
    * @return true if trash is used
    */
   boolean isUsingTrash() {
-    return !config.getBoolean(Property.GC_TRASH_IGNORE);
+    @SuppressWarnings("removal")
+    Property p = Property.GC_TRASH_IGNORE;
+    return !config.getBoolean(p);
   }
 
   /**
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
index 314dc2baf6..edb660a70c 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
@@ -116,7 +116,9 @@ public class SimpleGarbageCollector extends AbstractServer implements Iface {
    * @return true if trash is used
    */
   boolean isUsingTrash() {
-    return !getConfiguration().getBoolean(Property.GC_TRASH_IGNORE);
+    @SuppressWarnings("removal")
+    Property p = Property.GC_TRASH_IGNORE;
+    return !getConfiguration().getBoolean(p);
   }
 
   /**
diff --git a/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java b/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
index 01e26db7eb..3ba63c3da5 100644
--- a/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
+++ b/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
@@ -65,6 +65,8 @@ public class SimpleGarbageCollectorTest {
   private SimpleGarbageCollector gc;
   private ConfigurationCopy systemConfig;
   private static SiteConfiguration siteConfig = SiteConfiguration.empty().build();
+  @SuppressWarnings("removal")
+  private final Property GC_TRASH_IGNORE = Property.GC_TRASH_IGNORE;
 
   @BeforeEach
   public void setUp() {
@@ -97,7 +99,7 @@ public class SimpleGarbageCollectorTest {
     conf.put(Property.GC_CYCLE_START.getKey(), "1");
     conf.put(Property.GC_CYCLE_DELAY.getKey(), "20");
     conf.put(Property.GC_DELETE_THREADS.getKey(), "2");
-    conf.put(Property.GC_TRASH_IGNORE.getKey(), "false");
+    conf.put(GC_TRASH_IGNORE.getKey(), "false");
 
     return new ConfigurationCopy(conf);
   }
@@ -132,7 +134,7 @@ public class SimpleGarbageCollectorTest {
 
   @Test
   public void testMoveToTrash_NotUsingTrash() throws Exception {
-    systemConfig.set(Property.GC_TRASH_IGNORE.getKey(), "true");
+    systemConfig.set(GC_TRASH_IGNORE.getKey(), "true");
     Path path = createMock(Path.class);
     assertFalse(gc.moveToTrash(path));
   }
diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/RootFilesUpgradeTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/RootFilesUpgradeTest.java
index e90e4cee4e..638123b23f 100644
--- a/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/RootFilesUpgradeTest.java
+++ b/server/manager/src/test/java/org/apache/accumulo/manager/upgrade/RootFilesUpgradeTest.java
@@ -84,7 +84,9 @@ public class RootFilesUpgradeTest extends WithTestNames {
       // up later
       for (Path path : oldDatafiles) {
         Path deleteFile = new Path(location + "/delete+" + compactName + "+" + path.getName());
-        if (acuTableConf.getBoolean(Property.GC_TRASH_IGNORE) || !fs.moveToTrash(deleteFile)) {
+        @SuppressWarnings("removal")
+        Property p = Property.GC_TRASH_IGNORE;
+        if (acuTableConf.getBoolean(p) || !fs.moveToTrash(deleteFile)) {
           fs.deleteRecursively(deleteFile);
         }
       }
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashBase.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashBase.java
new file mode 100644
index 0000000000..e3172e2d0e
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashBase.java
@@ -0,0 +1,125 @@
+/*
+ * 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.test.functional;
+
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.core.metadata.TabletFile;
+import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType;
+import org.apache.accumulo.core.metadata.schema.TabletsMetadata;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.test.util.Wait;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+// base class for ITs that test our legacy trash property and Hadoop's trash policy with accumulo gc
+public class GarbageCollectorTrashBase extends ConfigurableMacBase {
+
+  private static final Logger LOG = LoggerFactory.getLogger(GarbageCollectorTrashBase.class);
+
+  protected ArrayList<StoredTabletFile> getFilesForTable(ServerContext ctx, AccumuloClient client,
+      String tableName) {
+    String tid = client.tableOperations().tableIdMap().get(tableName);
+    TabletsMetadata tms =
+        ctx.getAmple().readTablets().forTable(TableId.of(tid)).fetch(ColumnType.FILES).build();
+    ArrayList<StoredTabletFile> files = new ArrayList<>();
+    tms.forEach(tm -> {
+      files.addAll(tm.getFiles());
+    });
+    LOG.debug("Tablet files: {}", files);
+    return files;
+  }
+
+  protected ArrayList<StoredTabletFile> loadData(ServerContext ctx, AccumuloClient client,
+      String tableName) throws Exception {
+    // create some files
+    for (int i = 0; i < 5; i++) {
+      ReadWriteIT.ingest(client, 10, 10, 10, 0, tableName);
+      client.tableOperations().flush(tableName);
+    }
+    return getFilesForTable(ctx, client, tableName);
+  }
+
+  protected boolean userTrashDirExists(FileSystem fs) {
+    return !fs.getTrashRoots(false).isEmpty();
+  }
+
+  protected void makeTrashDir(FileSystem fs) throws IOException {
+    if (!userTrashDirExists(fs)) {
+      Path homeDir = fs.getHomeDirectory();
+      Path trashDir = new Path(homeDir, ".Trash");
+      assertTrue(fs.mkdirs(trashDir));
+    }
+    assertTrue(userTrashDirExists(fs));
+
+  }
+
+  protected void waitForFilesToBeGCd(final ArrayList<StoredTabletFile> files) throws Exception {
+    Wait.waitFor(() -> files.stream().noneMatch(stf -> {
+      try {
+        return super.getCluster().getMiniDfs().getFileSystem().exists(stf.getPath());
+      } catch (IOException e) {
+        throw new UncheckedIOException("error", e);
+      }
+    }));
+  }
+
+  protected long countFilesInTrash(FileSystem fs, TableId tid)
+      throws FileNotFoundException, IOException {
+    Collection<FileStatus> dirs = fs.getTrashRoots(true);
+    if (dirs.isEmpty()) {
+      return -1;
+    }
+    long count = 0;
+    Iterator<FileStatus> iter = dirs.iterator();
+    while (iter.hasNext()) {
+      FileStatus stat = iter.next();
+      LOG.debug("Trash root: {}", stat.getPath());
+      RemoteIterator<LocatedFileStatus> riter = fs.listFiles(stat.getPath(), true);
+      while (riter.hasNext()) {
+        LocatedFileStatus lfs = riter.next();
+        if (lfs.isDirectory()) {
+          continue;
+        }
+        TabletFile tf = new TabletFile(lfs.getPath());
+        LOG.debug("File in trash: {}, tableId: {}", lfs.getPath(), tf.getTableId());
+        if (tid.equals(tf.getTableId())) {
+          count++;
+        }
+      }
+    }
+    return count;
+  }
+
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java
new file mode 100644
index 0000000000..b8faff982a
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java
@@ -0,0 +1,88 @@
+/*
+ * 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.test.functional;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.admin.CompactionConfig;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
+import org.junit.jupiter.api.Test;
+
+// verify trash is not used with Hadoop default configuration as Trash is not
+// enabled by default.
+public class GarbageCollectorTrashDefaultIT extends GarbageCollectorTrashBase {
+
+  @Override
+  protected Duration defaultTimeout() {
+    return Duration.ofMinutes(5);
+  }
+
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    // By default Hadoop trash is disabled - fs.trash.interval defaults to 0
+    Map<String,String> hadoopOverrides = new HashMap<>();
+    hadoopOverrides.put(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY, "0");
+    cfg.setHadoopConfOverrides(hadoopOverrides);
+    cfg.useMiniDFS(true);
+
+    cfg.setProperty(Property.GC_CYCLE_START, "1");
+    cfg.setProperty(Property.GC_CYCLE_DELAY, "1");
+    @SuppressWarnings("removal")
+    Property p = Property.GC_TRASH_IGNORE;
+    cfg.setProperty(p, "false"); // default, use trash if configured
+    cfg.setProperty(Property.GC_PORT, "0");
+    cfg.setProperty(Property.TSERV_MAXMEM, "5K");
+    cfg.setProperty(Property.TABLE_MAJC_RATIO, "5.0");
+    cfg.setProperty(Property.TSERV_MAJC_DELAY, "1");
+  }
+
+  @Test
+  public void testTrashHadoopDisabledAccumuloEnabled() throws Exception {
+    String table = this.getUniqueNames(1)[0];
+    final FileSystem fs = super.getCluster().getFileSystem();
+    super.makeTrashDir(fs);
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
+      ArrayList<StoredTabletFile> files = super.loadData(super.getServerContext(), c, table);
+      assertFalse(files.isEmpty());
+      c.tableOperations().compact(table, new CompactionConfig());
+      TableId tid = TableId.of(c.tableOperations().tableIdMap().get(table));
+      // The default value for fs.trash.interval is 0, which means that
+      // trash is disabled in the Hadoop configuration. Enabling trash in
+      // Accumulo (GC_TRASH_IGNORE = false) still requires enabling trash in Hadoop
+      super.waitForFilesToBeGCd(files);
+      assertEquals(0, super.countFilesInTrash(fs, tid));
+    }
+
+  }
+
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDisabledIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDisabledIT.java
new file mode 100644
index 0000000000..d1985fa1df
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDisabledIT.java
@@ -0,0 +1,86 @@
+/*
+ * 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.test.functional;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.admin.CompactionConfig;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
+import org.junit.jupiter.api.Test;
+
+// verify trash is not used when Hadoop is configured to enable it and our property
+// is set to ignore it and delete the file anyway
+public class GarbageCollectorTrashDisabledIT extends GarbageCollectorTrashBase {
+
+  @Override
+  protected Duration defaultTimeout() {
+    return Duration.ofMinutes(5);
+  }
+
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+
+    Map<String,String> hadoopOverrides = new HashMap<>();
+    hadoopOverrides.put(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY, "5");
+    cfg.setHadoopConfOverrides(hadoopOverrides);
+    cfg.useMiniDFS(true);
+
+    cfg.setProperty(Property.GC_CYCLE_START, "1");
+    cfg.setProperty(Property.GC_CYCLE_DELAY, "1");
+    @SuppressWarnings("removal")
+    Property p = Property.GC_TRASH_IGNORE;
+    cfg.setProperty(p, "true"); // don't use trash if configured
+    cfg.setProperty(Property.GC_PORT, "0");
+    cfg.setProperty(Property.TSERV_MAXMEM, "5K");
+    cfg.setProperty(Property.TABLE_MAJC_RATIO, "5.0");
+    cfg.setProperty(Property.TSERV_MAJC_DELAY, "180s");
+  }
+
+  @Test
+  public void testTrashHadoopEnabledAccumuloDisabled() throws Exception {
+    String table = this.getUniqueNames(1)[0];
+    final FileSystem fs = super.getCluster().getFileSystem();
+    super.makeTrashDir(fs);
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
+      ArrayList<StoredTabletFile> files = super.loadData(super.getServerContext(), c, table);
+      assertFalse(files.isEmpty());
+      c.tableOperations().compact(table, new CompactionConfig());
+      TableId tid = TableId.of(c.tableOperations().tableIdMap().get(table));
+      super.waitForFilesToBeGCd(files);
+      // Trash is disabled in Accumulo (GC_TRASH_IGNORE = true)
+      // no files for this table should be in the trash
+      assertEquals(0, super.countFilesInTrash(fs, tid));
+    }
+  }
+
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledCustomPolicyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledCustomPolicyIT.java
new file mode 100644
index 0000000000..f2df9151a0
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledCustomPolicyIT.java
@@ -0,0 +1,114 @@
+/*
+ * 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.test.functional;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.admin.CompactionConfig;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.TrashPolicyDefault;
+import org.junit.jupiter.api.Test;
+
+// verify that trash is used if our property is set to not ignore it (the default)
+// and Hadoop Trash is configured to enable it and use a custom policy.
+public class GarbageCollectorTrashEnabledCustomPolicyIT extends GarbageCollectorTrashBase {
+
+  public static class NoFlushFilesInTrashPolicy extends TrashPolicyDefault {
+
+    @Override
+    public boolean moveToTrash(Path path) throws IOException {
+      // Don't put flush files in the Trash
+      if (!path.getName().startsWith("F")) {
+        return super.moveToTrash(path);
+      }
+      return false;
+    }
+
+  }
+
+  @Override
+  protected Duration defaultTimeout() {
+    return Duration.ofMinutes(5);
+  }
+
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+
+    Map<String,String> hadoopOverrides = new HashMap<>();
+    hadoopOverrides.put(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY, "5");
+    hadoopOverrides.put("fs.trash.classname", NoFlushFilesInTrashPolicy.class.getName());
+    cfg.setHadoopConfOverrides(hadoopOverrides);
+    cfg.useMiniDFS(true);
+
+    cfg.setProperty(Property.GC_CYCLE_START, "1");
+    cfg.setProperty(Property.GC_CYCLE_DELAY, "1");
+    @SuppressWarnings("removal")
+    Property p = Property.GC_TRASH_IGNORE;
+    cfg.setProperty(p, "false"); // default, use trash if configured
+    cfg.setProperty(Property.GC_PORT, "0");
+    cfg.setProperty(Property.TSERV_MAXMEM, "5K");
+    cfg.setProperty(Property.TABLE_MAJC_RATIO, "5.0");
+    cfg.setProperty(Property.TSERV_MAJC_DELAY, "180s");
+  }
+
+  @Test
+  public void testTrashHadoopEnabledAccumuloEnabled() throws Exception {
+    String table = this.getUniqueNames(1)[0];
+    final FileSystem fs = super.getCluster().getFileSystem();
+    super.makeTrashDir(fs);
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
+      ReadWriteIT.ingest(c, 10, 10, 10, 0, table);
+      c.tableOperations().flush(table);
+      ArrayList<StoredTabletFile> files1 = getFilesForTable(super.getServerContext(), c, table);
+      assertFalse(files1.isEmpty());
+      assertTrue(files1.stream().allMatch(stf -> stf.getPath().getName().startsWith("F")));
+      c.tableOperations().compact(table, new CompactionConfig());
+      super.waitForFilesToBeGCd(files1);
+      ArrayList<StoredTabletFile> files2 = getFilesForTable(super.getServerContext(), c, table);
+      assertFalse(files2.isEmpty());
+      assertTrue(files2.stream().noneMatch(stf -> stf.getPath().getName().startsWith("F")));
+      assertTrue(files2.stream().allMatch(stf -> stf.getPath().getName().startsWith("A")));
+      c.tableOperations().compact(table, new CompactionConfig());
+      super.waitForFilesToBeGCd(files2);
+      ArrayList<StoredTabletFile> files3 = getFilesForTable(super.getServerContext(), c, table);
+      assertTrue(files3.stream().allMatch(stf -> stf.getPath().getName().startsWith("A")));
+      assertEquals(1, files3.size());
+      TableId tid = TableId.of(c.tableOperations().tableIdMap().get(table));
+      assertEquals(1, super.countFilesInTrash(fs, tid));
+    }
+  }
+
+}
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledIT.java
new file mode 100644
index 0000000000..603d3f8abe
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledIT.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   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.test.functional;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.Accumulo;
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.admin.CompactionConfig;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.TableId;
+import org.apache.accumulo.core.metadata.StoredTabletFile;
+import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
+import org.junit.jupiter.api.Test;
+
+// verify that trash is used if our property is set to not ignore it (the default)
+// and Hadoop is configured to enable it
+public class GarbageCollectorTrashEnabledIT extends GarbageCollectorTrashBase {
+
+  @Override
+  protected Duration defaultTimeout() {
+    return Duration.ofMinutes(5);
+  }
+
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+
+    Map<String,String> hadoopOverrides = new HashMap<>();
+    hadoopOverrides.put(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY, "5");
+    cfg.setHadoopConfOverrides(hadoopOverrides);
+    cfg.useMiniDFS(true);
+
+    cfg.setProperty(Property.GC_CYCLE_START, "1");
+    cfg.setProperty(Property.GC_CYCLE_DELAY, "1");
+    @SuppressWarnings("removal")
+    Property p = Property.GC_TRASH_IGNORE;
+    cfg.setProperty(p, "false"); // default, use trash if configured
+    cfg.setProperty(Property.GC_PORT, "0");
+    cfg.setProperty(Property.TSERV_MAXMEM, "5K");
+    cfg.setProperty(Property.TABLE_MAJC_RATIO, "5.0");
+    cfg.setProperty(Property.TSERV_MAJC_DELAY, "180s");
+  }
+
+  @Test
+  public void testTrashHadoopEnabledAccumuloEnabled() throws Exception {
+    String table = this.getUniqueNames(1)[0];
+    final FileSystem fs = super.getCluster().getFileSystem();
+    super.makeTrashDir(fs);
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
+      ArrayList<StoredTabletFile> files = super.loadData(super.getServerContext(), c, table);
+      assertFalse(files.isEmpty());
+      c.tableOperations().compact(table, new CompactionConfig());
+      TableId tid = TableId.of(c.tableOperations().tableIdMap().get(table));
+      super.waitForFilesToBeGCd(files);
+      assertEquals(files.size(), super.countFilesInTrash(fs, tid));
+    }
+  }
+
+}
diff --git a/test/src/main/resources/log4j2-test.properties b/test/src/main/resources/log4j2-test.properties
index a9fd65426b..cf09c1b90a 100644
--- a/test/src/main/resources/log4j2-test.properties
+++ b/test/src/main/resources/log4j2-test.properties
@@ -146,6 +146,9 @@ logger.36.level = error
 logger.37.name = org.eclipse.jetty
 logger.37.level = warn
 
+logger.38.name = org.apache.hadoop.fs.TrashPolicyDefault
+logger.38.level = debug
+
 property.metricsFilename = ./target/test-metrics
 
 # appender.metrics.type = Console