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/27 00:22:10 UTC

[GitHub] [accumulo] dlmarion opened a new pull request, #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

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

   This commit:
     1. Deprecated 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.


-- 
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 #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

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

   Full IT build successful


-- 
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 #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

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


##########
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 p = Property.GC_TRASH_IGNORE;

Review Comment:
   Can keep this name:
   ```suggestion
     private final Property GC_TRASH_IGNORE = Property.GC_TRASH_IGNORE;
   ```



##########
server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java:
##########
@@ -97,7 +99,7 @@ private ConfigurationCopy createSystemConfig() {
     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(p.getKey(), "false");

Review Comment:
   ```suggestion
       conf.put(GC_TRASH_IGNORE.getKey(), "false");
   ```



##########
server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java:
##########
@@ -463,11 +463,15 @@ boolean inSafeMode() {
   boolean moveToTrash(Path path) throws IOException {
     final VolumeManager fs = context.getVolumeManager();
     if (!isUsingTrash()) {
+      log.trace("Accumulo Trash is disabled.");
       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", ex);

Review Comment:
   ```suggestion
         log.error("Error moving {} to trash", path, ex);
   ```



##########
server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java:
##########
@@ -132,7 +134,7 @@ public void testMoveToTrash_UsingTrash_VolMgrFailure() throws Exception {
 
   @Test
   public void testMoveToTrash_NotUsingTrash() throws Exception {
-    systemConfig.set(Property.GC_TRASH_IGNORE.getKey(), "true");
+    systemConfig.set(p.getKey(), "true");

Review Comment:
   ```suggestion
       systemConfig.set(GC_TRASH_IGNORE.getKey(), "true");
   ```



##########
server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java:
##########
@@ -354,7 +355,10 @@ public void bulkRename(Map<Path,Path> oldToNewPathMap, int poolSize, String pool
   @Override
   public boolean moveToTrash(Path path) throws IOException {
     FileSystem fs = getFileSystemByPath(path);
+    log.trace("fs.trash.interval: {}",
+        fs.getConf().get(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY));
     Trash trash = new Trash(fs, fs.getConf());
+    log.trace("Hadoop Trash is enabled: {}", trash.isEnabled());

Review Comment:
   ```suggestion
       log.trace("Hadoop Trash is enabled for {}: {}", path, trash.isEnabled());
   ```



##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashBase.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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;
+
+public class GarbageCollectorTrashBase extends ConfigurableMacBase {

Review Comment:
   ```suggestion
   // base class for ITs that test our legacy trash flag and adoop's trash policy with accumulo-gc
   public class GarbageCollectorTrashBase extends ConfigurableMacBase {
   ```



##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashBase.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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;
+
+public class GarbageCollectorTrashBase extends ConfigurableMacBase {
+
+  private static final Logger LOG = LoggerFactory.getLogger(GarbageCollectorTrashBase.class);
+
+  public 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;
+  }
+
+  public 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);
+  }
+
+  public boolean userTrashDirExists(FileSystem fs) {
+    return !fs.getTrashRoots(false).isEmpty();
+  }
+
+  public void makeTrashDir(FileSystem fs) throws IOException {
+    if (!userTrashDirExists(fs)) {
+      Path homeDir = fs.getHomeDirectory();
+      Path trashDir = new Path(homeDir, ".Trash");

Review Comment:
   This is okay so long as miniDFS is used. If LocalFileSystem or RawLocalFileSystem is used, this is probably going to pollute the user's `$HOME` instead of stay in the `/target/` directory.



##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledCustomPolicyIT.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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;
+
+public class GarbageCollectorTrashEnabledCustomPolicyIT extends GarbageCollectorTrashBase {

Review Comment:
   ```suggestion
   // verify trash is used if our legacy prop is set to not ignore it (the default)
   // and Hadoop is also configured to use a custom trash policy
   public class GarbageCollectorTrashEnabledCustomPolicyIT extends GarbageCollectorTrashBase {
   ```



##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.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;
+
+public class GarbageCollectorTrashDefaultIT extends GarbageCollectorTrashBase {

Review Comment:
   ```suggestion
   // verify trash is not used using by Hadoop default when Hadoop isn't configured
   // to use it (even though our legacy prop set to use it, if configured)
   public class GarbageCollectorTrashDefaultIT extends GarbageCollectorTrashBase {
   ```



##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashBase.java:
##########


Review Comment:
   Methods can be protected instead of public on this base class



##########
server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java:
##########
@@ -354,7 +355,10 @@ public void bulkRename(Map<Path,Path> oldToNewPathMap, int poolSize, String pool
   @Override
   public boolean moveToTrash(Path path) throws IOException {
     FileSystem fs = getFileSystemByPath(path);
+    log.trace("fs.trash.interval: {}",
+        fs.getConf().get(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY));

Review Comment:
   ```suggestion
       String key = CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
       log.trace("{}: {}", key, fs.getConf().get(key));
   ```



##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDisabledIT.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;
+
+public class GarbageCollectorTrashDisabledIT extends GarbageCollectorTrashBase {

Review Comment:
   ```suggestion
   // verify trash is ignored when Hadoop is configured to use it, because our
   // legacy prop is set to ignore it
   public class GarbageCollectorTrashDisabledIT extends GarbageCollectorTrashBase {
   ```



##########
server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java:
##########
@@ -463,11 +463,15 @@ boolean inSafeMode() {
   boolean moveToTrash(Path path) throws IOException {
     final VolumeManager fs = context.getVolumeManager();
     if (!isUsingTrash()) {
+      log.trace("Accumulo Trash is disabled.");

Review Comment:
   ```suggestion
         log.trace("Accumulo Trash is disabled. Skipped for {}", path);
   ```



##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledIT.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.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;
+
+public class GarbageCollectorTrashEnabledIT extends GarbageCollectorTrashBase {

Review Comment:
   ```suggestion
   // verify trash is used with Hadoop's default trash policy when our legacy prop
   // is set to not ignore
   public class GarbageCollectorTrashEnabledIT extends GarbageCollectorTrashBase {
   ```



-- 
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 #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

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

   > An argument to test the hadoop functionality - at least to some extent is to validate the the function continues to work as expected in future releases - if the "hadoop" functionality changed (or broke) we may catch that in out automated test runs and then we would be in a position to take some action - maybe we need to do something different? Submit a patch to hadoop? Or at a minimum, warn users that the function may not work as expected after x.y.z.
   > 
   > I'm not sure how extensively we are checking the behavior here, but some validation might be valuable.
   
   My argument against, as a general rule, is that regression testing for Hadoop features should be contributed to Hadoop. We cannot be the landing zone for testing all the Hadoop features we think Hadoop is not testing adequately. I suggest contributing those tests to Hadoop to make Hadoop better, and watching its build statuses/change history for the features we care about, and try to focus our tests on testing our code.
   
   I'm definitely not arguing against testing... just advocating for contributing them to the proper venue. We shouldn't treat Accumulo as a personal playground for testing other libraries we care about. We should play more nicely with those other open source projects, and contribute tests to them, if we think they need that testing.
   
   That said, everything is on a case-by-case basis, and I'm not sure exactly what, if anything, should change here once the property is dropped, with respect to this testing, until I look at it, which I will do shortly. But I think it's important to stake out a general principle, so we don't suffer from scope creep.


-- 
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 #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

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


##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.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;
+
+public class GarbageCollectorTrashDefaultIT extends GarbageCollectorTrashBase {

Review Comment:
   Implemented in 2aa689f



##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDisabledIT.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;
+
+public class GarbageCollectorTrashDisabledIT extends GarbageCollectorTrashBase {

Review Comment:
   Implemented in 2aa689f



##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledCustomPolicyIT.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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;
+
+public class GarbageCollectorTrashEnabledCustomPolicyIT extends GarbageCollectorTrashBase {

Review Comment:
   Implemented in 2aa689f



-- 
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 #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

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


##########
server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java:
##########
@@ -354,7 +355,10 @@ public void bulkRename(Map<Path,Path> oldToNewPathMap, int poolSize, String pool
   @Override
   public boolean moveToTrash(Path path) throws IOException {
     FileSystem fs = getFileSystemByPath(path);
+    log.trace("fs.trash.interval: {}",
+        fs.getConf().get(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY));
     Trash trash = new Trash(fs, fs.getConf());
+    log.trace("Hadoop Trash is enabled: {}", trash.isEnabled());

Review Comment:
   Implemented in 2aa689f



##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashBase.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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;
+
+public class GarbageCollectorTrashBase extends ConfigurableMacBase {

Review Comment:
   Implemented in 2aa689f



-- 
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 #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

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


##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledIT.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.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;
+
+public class GarbageCollectorTrashEnabledIT extends GarbageCollectorTrashBase {

Review Comment:
   Implemented in 2aa689f



-- 
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 #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

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


##########
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 p = Property.GC_TRASH_IGNORE;

Review Comment:
   Implemented in 2aa689f



##########
server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java:
##########
@@ -97,7 +99,7 @@ private ConfigurationCopy createSystemConfig() {
     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(p.getKey(), "false");

Review Comment:
   Implemented in 2aa689f



-- 
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 #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

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


##########
server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java:
##########
@@ -354,7 +355,10 @@ public void bulkRename(Map<Path,Path> oldToNewPathMap, int poolSize, String pool
   @Override
   public boolean moveToTrash(Path path) throws IOException {
     FileSystem fs = getFileSystemByPath(path);
+    log.trace("fs.trash.interval: {}",
+        fs.getConf().get(CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY));

Review Comment:
   Implemented in 2aa689f



##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashBase.java:
##########


Review Comment:
   Implemented in 2aa689f



-- 
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] keith-turner commented on a diff in pull request #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3436:
URL: https://github.com/apache/accumulo/pull/3436#discussion_r1210553664


##########
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;
+
+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);
+    ArrayList<StoredTabletFile> files = null;
+    TableId tid = null;
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
+      files = super.loadData(super.getServerContext(), c, table);
+      assertFalse(files.isEmpty());
+      c.tableOperations().compact(table, new CompactionConfig());
+      tid = TableId.of(c.tableOperations().tableIdMap().get(table));
+    }
+    Thread.sleep(10000);
+    assertEquals(files.size(), super.countFilesInTrash(fs, tid));

Review Comment:
   This could loop waiting for the condition to be true.



##########
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;
+
+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);
+    ArrayList<StoredTabletFile> files = null;
+    TableId tid = null;
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
+      files = super.loadData(super.getServerContext(), c, table);
+      assertFalse(files.isEmpty());
+      c.tableOperations().compact(table, new CompactionConfig());
+      tid = TableId.of(c.tableOperations().tableIdMap().get(table));
+    }
+    Thread.sleep(10000);

Review Comment:
   Could wait for compaction and then GC (by waiting for the files to be deleted in orig location) here also instead of sleeping.  Could put a new method in the parent class that waits for files to be gone from a table.



##########
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;
+
+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);
+    ArrayList<StoredTabletFile> files = null;
+    TableId tid = null;
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
+      files = super.loadData(super.getServerContext(), c, table);
+      assertFalse(files.isEmpty());
+      c.tableOperations().compact(table, new CompactionConfig());

Review Comment:
   Could wait for the compaction to finish
   
   ```suggestion
         c.tableOperations().compact(table, new CompactionConfig().setWait(true));
   ```



##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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 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.FileSystem;
+import org.junit.jupiter.api.Test;
+
+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

Review Comment:
   Could explicitly set this to zero in the test even thought its the default.



##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledCustomPolicyIT.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.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;
+
+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);
+    ArrayList<StoredTabletFile> files = null;
+    TableId tid = null;
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
+      files = super.loadData(super.getServerContext(), c, table);
+      assertFalse(files.isEmpty());
+      c.tableOperations().compact(table, new CompactionConfig());
+      tid = TableId.of(c.tableOperations().tableIdMap().get(table));
+    }
+    Thread.sleep(10000);

Review Comment:
   Could make this wait until the flush files in the original location are gone/GCed and then count the files in trash instead of sleeping.



##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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 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.FileSystem;
+import org.junit.jupiter.api.Test;
+
+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
+    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);
+    TableId tid = null;
+    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());
+      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
+    Thread.sleep(10000);

Review Comment:
   could also wait for compaction and files to be gone here.



-- 
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 #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

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


##########
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;
+
+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);
+    ArrayList<StoredTabletFile> files = null;
+    TableId tid = null;
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
+      files = super.loadData(super.getServerContext(), c, table);
+      assertFalse(files.isEmpty());
+      c.tableOperations().compact(table, new CompactionConfig());
+      tid = TableId.of(c.tableOperations().tableIdMap().get(table));
+    }
+    Thread.sleep(10000);
+    assertEquals(files.size(), super.countFilesInTrash(fs, tid));

Review Comment:
   Addressed in 5ec3ca8.



-- 
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] keith-turner commented on a diff in pull request #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3436:
URL: https://github.com/apache/accumulo/pull/3436#discussion_r1210571216


##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledCustomPolicyIT.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.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;
+
+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")) {

Review Comment:
   The test could be restructured a bit so that there are F and C or A files to GC.  Then we could got more positive confirmation in the test that this class is being called (we could have zero files in trash for other reasons than this class being called).  Thinking of something like the following.
   
    1. ingest data
    2. flush table creating a F file
    3. compact table creating an A file and marking the F file for GC
    4. compact table again creating another A file and marking prev A file for GC
    5. wait for the F and A files marked for GC to be gone
    6. assert trash has one file, should be the A file



-- 
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 #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

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


##########
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;
+
+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);
+    ArrayList<StoredTabletFile> files = null;
+    TableId tid = null;
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
+      files = super.loadData(super.getServerContext(), c, table);
+      assertFalse(files.isEmpty());
+      c.tableOperations().compact(table, new CompactionConfig());
+      tid = TableId.of(c.tableOperations().tableIdMap().get(table));
+    }
+    Thread.sleep(10000);

Review Comment:
   Addressed in 5ec3ca8.



##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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 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.FileSystem;
+import org.junit.jupiter.api.Test;
+
+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
+    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);
+    TableId tid = null;
+    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());
+      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
+    Thread.sleep(10000);

Review Comment:
   Addressed in 5ec3ca8.



-- 
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] EdColeman commented on pull request #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

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

   An argument to test the hadoop functionality - at least to some extent is to validate the the function continues to work as expected in future releases - if the "hadoop" functionality changed (or broke) we may catch that in out automated test runs and then we would be in a position to take some action - maybe we need to do something different? Submit a patch to hadoop?  Or at a minimum, warn users that the function may not work as expected after x.y.z.
   
   I'm not sure how extensively we are checking the behavior here, but some validation might be valuable.


-- 
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 #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

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

   Kicked off full IT build


-- 
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 #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

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


##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledCustomPolicyIT.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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;
+
+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);
+      assertTrue(files2.stream().noneMatch(stf -> stf.getPath().getName().startsWith("F")));

Review Comment:
   Added in ef414e1



-- 
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 #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

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


##########
server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java:
##########
@@ -132,7 +134,7 @@ public void testMoveToTrash_UsingTrash_VolMgrFailure() throws Exception {
 
   @Test
   public void testMoveToTrash_NotUsingTrash() throws Exception {
-    systemConfig.set(Property.GC_TRASH_IGNORE.getKey(), "true");
+    systemConfig.set(p.getKey(), "true");

Review Comment:
   Implemented in 2aa689f



##########
server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java:
##########
@@ -463,11 +463,15 @@ boolean inSafeMode() {
   boolean moveToTrash(Path path) throws IOException {
     final VolumeManager fs = context.getVolumeManager();
     if (!isUsingTrash()) {
+      log.trace("Accumulo Trash is disabled.");
       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", ex);

Review Comment:
   Implemented in 2aa689f



##########
server/gc/src/main/java/org/apache/accumulo/gc/GCRun.java:
##########
@@ -463,11 +463,15 @@ boolean inSafeMode() {
   boolean moveToTrash(Path path) throws IOException {
     final VolumeManager fs = context.getVolumeManager();
     if (!isUsingTrash()) {
+      log.trace("Accumulo Trash is disabled.");

Review Comment:
   Implemented in 2aa689f



-- 
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 #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

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


##########
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;
+
+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);
+    ArrayList<StoredTabletFile> files = null;
+    TableId tid = null;
+    try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
+      files = super.loadData(super.getServerContext(), c, table);
+      assertFalse(files.isEmpty());
+      c.tableOperations().compact(table, new CompactionConfig());

Review Comment:
   it defaults to true



-- 
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 #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

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


##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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 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.FileSystem;
+import org.junit.jupiter.api.Test;
+
+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

Review Comment:
   Addressed in 5ec3ca8.



##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledCustomPolicyIT.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.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;
+
+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")) {

Review Comment:
   Addressed in 5ec3ca8.



-- 
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] keith-turner commented on a diff in pull request #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

Posted by "keith-turner (via GitHub)" <gi...@apache.org>.
keith-turner commented on code in PR #3436:
URL: https://github.com/apache/accumulo/pull/3436#discussion_r1211706724


##########
test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledCustomPolicyIT.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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;
+
+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);
+      assertTrue(files2.stream().noneMatch(stf -> stf.getPath().getName().startsWith("F")));

Review Comment:
   A check like the one above would be good here.  Could the not empty check be more specific, do you know how many files to expect?
   
   ```suggestion
         assertFalse(files2.isEmpty());
         assertTrue(files2.stream().noneMatch(stf -> stf.getPath().getName().startsWith("F")));
   ```



-- 
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 merged pull request #3436: Deprecate GC_TRASH_IGNORE, add tests to confirm Trash works

Posted by "dlmarion (via GitHub)" <gi...@apache.org>.
dlmarion merged PR #3436:
URL: https://github.com/apache/accumulo/pull/3436


-- 
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