You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by "dlmarion (via GitHub)" <gi...@apache.org> on 2023/05/25 18:22:23 UTC

[GitHub] [accumulo] dlmarion opened a new pull request, #3428: Introduce new GC trash property, move duplicate methods to new class

dlmarion opened a new pull request, #3428:
URL: https://github.com/apache/accumulo/pull/3428

   This commit deprecates GC_TRASH_IGNORE in favor of GC_USE_TRASH. The description for the deprecated property and the way that it was used is confusing as it's written and used in the negative. If you want to use the Trash, you have to set it the property to "false". The new property works in the affirmative. Set it to "true" if you want to use the Trash, "false" if you don't. A third value, "bulk_imports_only" means that you only want to send bulk import files to the Trash.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on pull request #3428: Introduce new GC trash property, move duplicate methods to new class

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on PR #3428:
URL: https://github.com/apache/accumulo/pull/3428#issuecomment-1563525322

   I can likely put up a different PR for the SPI approach. Given that it's the opposite of what's here, I don't feel like this is a good starting point for it.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #3428: Introduce new GC trash property, move duplicate methods to new class

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on code in PR #3428:
URL: https://github.com/apache/accumulo/pull/3428#discussion_r1206024644


##########
server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorNewPropertyTest.java:
##########


Review Comment:
   Ah, I forgot JUnit would execute the `@Test` methods of the parent class.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on a diff in pull request #3428: Introduce new GC trash property, move duplicate methods to new class

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on code in PR #3428:
URL: https://github.com/apache/accumulo/pull/3428#discussion_r1205972659


##########
server/gc/src/main/java/org/apache/accumulo/gc/FileJanitor.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.gc;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FileJanitor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileJanitor.class);
+
+  enum SendFilesToTrash {
+
+    TRUE("true"), FALSE("false"), IMPORTS_ONLY("bulk_imports_only");
+
+    final String val;
+
+    SendFilesToTrash(String val) {
+      this.val = val;
+    }
+
+    public static SendFilesToTrash fromBoolean(boolean b) {
+      if (b) {
+        return TRUE;
+      }
+      return FALSE;
+    }
+
+    public static SendFilesToTrash fromString(String val) {
+      switch (val.toLowerCase()) {

Review Comment:
   ```suggestion
         switch (val.toLowerCase().strip()) {
   ```



##########
server/gc/src/main/java/org/apache/accumulo/gc/FileJanitor.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.gc;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FileJanitor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileJanitor.class);
+
+  enum SendFilesToTrash {
+
+    TRUE("true"), FALSE("false"), IMPORTS_ONLY("bulk_imports_only");
+
+    final String val;
+
+    SendFilesToTrash(String val) {
+      this.val = val;
+    }
+
+    public static SendFilesToTrash fromBoolean(boolean b) {
+      if (b) {
+        return TRUE;
+      }
+      return FALSE;
+    }
+
+    public static SendFilesToTrash fromString(String val) {
+      switch (val.toLowerCase()) {
+        case "true":
+          return TRUE;
+        case "false":
+          return FALSE;
+        case "bulk_imports_only":
+          return IMPORTS_ONLY;
+        default:
+          throw new IllegalArgumentException("Unknown value: " + val);
+      }
+    }
+  }
+
+  private final ServerContext context;
+  private final SendFilesToTrash usingTrash;
+
+  @SuppressWarnings("removal")
+  public FileJanitor(ServerContext context) {
+    this.context = context;
+    final AccumuloConfiguration conf = this.context.getConfiguration();
+    if (conf.isPropertySet(Property.GC_TRASH_IGNORE) && conf.isPropertySet(Property.GC_USE_TRASH)) {
+      throw new IllegalStateException("Cannot specify both " + Property.GC_TRASH_IGNORE.getKey()
+          + " and " + Property.GC_USE_TRASH.getKey() + " properties.");
+    }
+    if (conf.isPropertySet(Property.GC_TRASH_IGNORE)) {
+      this.usingTrash = SendFilesToTrash.fromBoolean(!conf.getBoolean(Property.GC_TRASH_IGNORE));
+    } else if (conf.isPropertySet(Property.GC_USE_TRASH)) {
+      this.usingTrash = SendFilesToTrash.fromString(conf.get(Property.GC_USE_TRASH));
+    } else {
+      LOG.warn(
+          "Neither GC trash property was set ({} or {}). Defaulting to using trash for all files (if available).",
+          Property.GC_TRASH_IGNORE.getKey(), Property.GC_USE_TRASH.getKey());
+      this.usingTrash = SendFilesToTrash.TRUE;
+    }
+  }
+
+  public ServerContext getContext() {
+    return this.context;
+  }
+

Review Comment:
   Don't think we need this getter. If not, it should be removed. Any caller should already have a context.



##########
server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorNewPropertyTest.java:
##########


Review Comment:
   This seems like it's just a FileJanitorTest unit test for FileJanitor. I think it can be decoupled from the SGC tests.



##########
server/gc/src/main/java/org/apache/accumulo/gc/FileJanitor.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.gc;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FileJanitor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileJanitor.class);
+
+  enum SendFilesToTrash {
+
+    TRUE("true"), FALSE("false"), IMPORTS_ONLY("bulk_imports_only");
+
+    final String val;
+
+    SendFilesToTrash(String val) {
+      this.val = val;
+    }
+
+    public static SendFilesToTrash fromBoolean(boolean b) {
+      if (b) {
+        return TRUE;
+      }
+      return FALSE;
+    }
+
+    public static SendFilesToTrash fromString(String val) {
+      switch (val.toLowerCase()) {
+        case "true":
+          return TRUE;
+        case "false":
+          return FALSE;
+        case "bulk_imports_only":
+          return IMPORTS_ONLY;
+        default:
+          throw new IllegalArgumentException("Unknown value: " + val);
+      }
+    }
+  }
+
+  private final ServerContext context;
+  private final SendFilesToTrash usingTrash;
+
+  @SuppressWarnings("removal")

Review Comment:
   This warning could be more narrowly placed on the property name, as a local variable:
   
   ```java
   @SuppressWarnings("removal")
   Property GC_TRASH_IGNORE = Property.GC_TRASH_IGNORE;
   ```
   
   Now, you have a local variable that is not deprecated, that you can use everywhere you are currently using the deprecated one, and you don't need to suppress warnings on the entire method.



##########
core/src/main/java/org/apache/accumulo/core/conf/Property.java:
##########
@@ -873,8 +873,13 @@ 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"),
+  GC_USE_TRASH("gc.use.trash", "true", PropertyType.STRING,
+      "Moves a file to the Trash (if available). Valid values are true, false, and bulk_imports_only."
+          + " Mutually exclusive with gc.trash.ignore.",
+      "2.1.1"),

Review Comment:
   I like the fix of the negation. But, I'm wondering if there are use cases for use trash for `everything_except_bulk_imports`  or something different for WALs and RFiles. I am thinking we should have a more robust TrashPolicy SPI, where we give it everything we know about a file, and it returns "Trash" or "Delete". The default implementation would be "AlwaysTrash", but we could also provide a "NeverTrash" implementation, and anything else more complicated would be up to the user to provide.



##########
server/gc/src/main/java/org/apache/accumulo/gc/FileJanitor.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.gc;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FileJanitor {

Review Comment:
   Class needs a javadoc to explain its purpose, so we don't have to infer it by trying to read all the code that uses it.



##########
server/gc/src/main/java/org/apache/accumulo/gc/FileJanitor.java:
##########


Review Comment:
   I think this class is doing basically the logic of what I was describing as a "TrashPolicy" SPI. I think it should be made into a proper SPI.
   
   I do like the name "FileJanitor", though. I think this class should be renamed, though, in order to reserve the name "janitor" for eventually renaming the "accumulo-gc" service, which would be nice to do to deconflict with the Java GC.



##########
server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorNewPropertyTest.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.gc;
+
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.partialMockBuilder;
+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.assertTrue;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.gc.FileJanitor.SendFilesToTrash;
+import org.apache.hadoop.fs.Path;
+import org.junit.jupiter.api.Test;
+
+public class SimpleGarbageCollectorNewPropertyTest extends SimpleGarbageCollectorTest {
+
+  @Override
+  protected ConfigurationCopy createSystemConfig() {
+    Map<String,String> conf = new HashMap<>();
+    conf.put(Property.INSTANCE_RPC_SASL_ENABLED.getKey(), "false");
+    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_USE_TRASH.getKey(), "true");
+
+    return new ConfigurationCopy(conf);
+  }
+
+  @Test
+  public void testMoveToTrash_NotUsingTrash_importsOnlyEnabled() throws Exception {
+    systemConfig.set(Property.GC_USE_TRASH.getKey(), "bulk_imports_only");
+
+    gc = partialMockBuilder(SimpleGarbageCollector.class).addMockedMethod("getContext")
+        .addMockedMethod("getFileJanitor").createMock();

Review Comment:
   I feel like you could just call `new FileJanitor()` with a mocked context to test it, rather than trying to get it from a partially mocked SimpleGarbageCollector to rely on that object to construct it for you. You're not actually testing anything from SimpleGarbageCollector here. It's just extra noisy test code that is a long-way around getting to the code you're actually testing.



##########
server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java:
##########
@@ -436,4 +410,9 @@ public GcCycleMetrics getGcCycleMetrics() {
     return gcCycleMetrics;
   }
 
+  @VisibleForTesting
+  public FileJanitor getFileJanitor() {
+    return this.janitor;
+  }
+

Review Comment:
   This can be package-private. It doesn't need to be public for unit tests in the same package.



##########
server/gc/src/main/java/org/apache/accumulo/gc/FileJanitor.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.gc;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.server.ServerContext;
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FileJanitor {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileJanitor.class);
+
+  enum SendFilesToTrash {
+
+    TRUE("true"), FALSE("false"), IMPORTS_ONLY("bulk_imports_only");
+
+    final String val;
+
+    SendFilesToTrash(String val) {
+      this.val = val;
+    }
+
+    public static SendFilesToTrash fromBoolean(boolean b) {
+      if (b) {
+        return TRUE;
+      }
+      return FALSE;
+    }
+
+    public static SendFilesToTrash fromString(String val) {
+      switch (val.toLowerCase()) {
+        case "true":
+          return TRUE;
+        case "false":
+          return FALSE;
+        case "bulk_imports_only":
+          return IMPORTS_ONLY;
+        default:
+          throw new IllegalArgumentException("Unknown value: " + val);
+      }
+    }
+  }
+
+  private final ServerContext context;
+  private final SendFilesToTrash usingTrash;
+
+  @SuppressWarnings("removal")
+  public FileJanitor(ServerContext context) {
+    this.context = context;
+    final AccumuloConfiguration conf = this.context.getConfiguration();
+    if (conf.isPropertySet(Property.GC_TRASH_IGNORE) && conf.isPropertySet(Property.GC_USE_TRASH)) {
+      throw new IllegalStateException("Cannot specify both " + Property.GC_TRASH_IGNORE.getKey()
+          + " and " + Property.GC_USE_TRASH.getKey() + " properties.");
+    }

Review Comment:
   It's okay to set them both as long as they are consistent. They should be opposite values, though. So, instead of checking that they are set, we should check that they are not equal (assuming they are boolean, and the bulk imports option isn't used). As long as they aren't equal to each other, that's fine. Some users are very pedantic, and they set everything, because they aren't sure which property to set, or because they have set an old value in ZK, and the new one in the config file, or something similar. That's not a problem, as long as its consistent.



##########
server/manager/src/test/java/org/apache/accumulo/manager/upgrade/RootFilesUpgradeTest.java:
##########
@@ -78,6 +78,7 @@ public void renameReplacement(VolumeManager fs, Path tmpDatafile, Path newDatafi
       rename(fs, tmpDatafile, newDatafile);
     }
 
+    @SuppressWarnings("removal")

Review Comment:
   Instead of suppressing the old property, can the test be updated to use the new?



##########
server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorNewPropertyTest.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.gc;
+
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.partialMockBuilder;
+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.assertTrue;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.gc.FileJanitor.SendFilesToTrash;
+import org.apache.hadoop.fs.Path;
+import org.junit.jupiter.api.Test;
+
+public class SimpleGarbageCollectorNewPropertyTest extends SimpleGarbageCollectorTest {

Review Comment:
   I don't think you need to extend SimpleGarbageCollectorTest to test this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on pull request #3428: Introduce new GC trash property, move duplicate methods to new class

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on PR #3428:
URL: https://github.com/apache/accumulo/pull/3428#issuecomment-1564277873

   I started looking at the SPI approach this morning and found something interesting. An SPI already exists in the Hadoop code. If anything I think we could probably remove code from Accumulo and users could create their own implementation of [TrashPolicy](https://github.com/apache/hadoop/blob/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java) and set it via the what appears to be the undocumented Hadoop property `fs.trash.classname` which is used in [TrashPolicy.getInstance](https://github.com/apache/hadoop/blob/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java#L156).
   
   When the Accumulo [GCRun](https://github.com/apache/accumulo/blob/main/server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java#L439) code calls [VolumeManagerImpl.moveToTrash(Path)](https://github.com/apache/accumulo/blob/main/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java#L355), a new `Trash` object is created, which calls `TrashPolicy.getInstance` which loads the `TrashPolicy` class specified by `fs.trash.classname`. I think there is an opportunity here to make this more efficient in `VolumeManagerImpl.moveToTrash`, I don't think it needs to create a new `Trash` instance for every call.
   
   Looking at the history of TrashPolicy, it appears that support for pluggable TrashPolicy's in Hadoop was added about 12 years ago.
   
   
   @alerman @ctubbsii - Is there a reason *not* to use the Hadoop pluggable Trash Policy?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on pull request #3428: Introduce new GC trash property, move duplicate methods to new class

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on PR #3428:
URL: https://github.com/apache/accumulo/pull/3428#issuecomment-1564688595

   @dlmarion That's a great find! If it's already pluggable in Hadoop, then we just need the flag we already have for using or not using it (but we might want to still negate the property name to "use trash = true" instead of "ignore trash = false", but the change will no longer be urgent... just for convenience). Doing this using a pluggable HDFS solution aligns exactly to [my previous comment](https://github.com/apache/accumulo/pull/3140#discussion_r1126585073) about decoupling ourselves from HDFS-specific feature management.
   
   I definitely don't want to abandon this change proposal, until we can experiment and make sure that the per-volume config with a custom trash policy works, but what you found looks promising.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #3428: Introduce new GC trash property, move duplicate methods to new class

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on code in PR #3428:
URL: https://github.com/apache/accumulo/pull/3428#discussion_r1206011159


##########
server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorNewPropertyTest.java:
##########


Review Comment:
   By extending `SimpleGarbageCollectorTest` and overriding `createSystemConfig` to use the new `GC_USE_TRASH` property instead of the older `GC_TRASH_IGNORE` this test exercises all of the tests in the parent class using the new property and this one additional test in this class for the new property option.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion closed pull request #3428: Introduce new GC trash property, move duplicate methods to new class

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion closed pull request #3428: Introduce new GC trash property, move duplicate methods to new class
URL: https://github.com/apache/accumulo/pull/3428


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on a diff in pull request #3428: Introduce new GC trash property, move duplicate methods to new class

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on code in PR #3428:
URL: https://github.com/apache/accumulo/pull/3428#discussion_r1206011159


##########
server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorNewPropertyTest.java:
##########


Review Comment:
   By extending `SimpleGarbageCollectorTest` and overriding `createSystemConfig` to use the new `GC_USE_TRASH` property instead of the older `GC_TRASH_IGNORE` this test class exercises all of the tests in the parent class using the new property and this one additional test in this class for the new property option.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on pull request #3428: Introduce new GC trash property, move duplicate methods to new class

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on PR #3428:
URL: https://github.com/apache/accumulo/pull/3428#issuecomment-1563347672

   Lol. I think the negations got me and I messed up the bulk import only. Waiting for confirmation.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on pull request #3428: Introduce new GC trash property, move duplicate methods to new class

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on PR #3428:
URL: https://github.com/apache/accumulo/pull/3428#issuecomment-1564305338

   Along with https://github.com/apache/accumulo/pull/3180, added in 2.1.1, this would allow different TrashPolicy implementations per volume.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] alerman commented on pull request #3428: Introduce new GC trash property, move duplicate methods to new class

Posted by "alerman (via GitHub)" <gi...@apache.org>.
alerman commented on PR #3428:
URL: https://github.com/apache/accumulo/pull/3428#issuecomment-1564358865

   Did not know there was a pluggable trash policy. The reason we needed the change for imports not going to trash was namenode pressure when trash was enabled due to the volume. Some testing should be done to see if the trash policy accomplishes the same goal without the need for the specialized handling in accumulo.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ctubbsii commented on pull request #3428: Introduce new GC trash property, move duplicate methods to new class

Posted by "ctubbsii (via GitHub)" <gi...@apache.org>.
ctubbsii commented on PR #3428:
URL: https://github.com/apache/accumulo/pull/3428#issuecomment-1563532461

   > I can likely put up a different PR for the SPI approach. Given that it's the opposite of what's here, I don't feel like this is a good starting point for it.
   
   Agreed that it's not the same starting point. Not sure about "opposite" :smiley_cat:, but definitely independent. Let me know if you'd prefer I work on that alternative. I'm happy to do so, but don't want to step on your toes.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on pull request #3428: Introduce new GC trash property, move duplicate methods to new class

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on PR #3428:
URL: https://github.com/apache/accumulo/pull/3428#issuecomment-1564365206

   My guess is that you can extent [TrashPolicyDefault](https://github.com/apache/hadoop/blob/trunk/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java#L123) and override the `moveToTrash` method. After checking the file name prefix, just call the super class method.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on pull request #3428: Introduce new GC trash property, move duplicate methods to new class

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion commented on PR #3428:
URL: https://github.com/apache/accumulo/pull/3428#issuecomment-1565100173

   Closing this in favor of #3436 as a solution for 2.1.1.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org