Author: ecn
Date: Tue Jan 8 19:00:35 2013
New Revision: 1430444
URL: http://svn.apache.org/viewvc?rev=1430444&view=rev
Log:
ACCUMULO-919 delete files using Trash
Modified:
accumulo/trunk/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java
Modified:
accumulo/trunk/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java
URL:
http://svn.apache.org/viewvc/accumulo/trunk/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java?rev=1430444&r1=1430443&r2=1430444&view=diff
==============================================================================
---
accumulo/trunk/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java
(original)
+++
accumulo/trunk/server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java
Tue Jan 8 19:00:35 2013
@@ -65,6 +65,8 @@ import org.apache.accumulo.core.gc.thrif
import org.apache.accumulo.core.master.state.tables.TableState;
import org.apache.accumulo.core.security.thrift.AuthInfo;
import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.core.util.Daemon;
+import org.apache.accumulo.core.util.LoggingRunnable;
import org.apache.accumulo.core.util.NamingThreadFactory;
import org.apache.accumulo.core.util.ServerServices;
import org.apache.accumulo.core.util.ServerServices.Service;
@@ -88,6 +90,7 @@ import org.apache.accumulo.server.zookee
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.Trash;
import org.apache.hadoop.io.Text;
import org.apache.log4j.Logger;
import org.apache.zookeeper.KeeperException;
@@ -114,11 +117,12 @@ public class SimpleGarbageCollector impl
private boolean candidateMemExceeded;
private static final Logger log =
Logger.getLogger(SimpleGarbageCollector.class);
-
+
private AuthInfo credentials;
private long gcStartDelay;
private boolean checkForBulkProcessingFiles;
private FileSystem fs;
+ private Trash trash;
private boolean safemode = false, offline = false, verbose = false;
private String address = "localhost";
private ZooLock lock;
@@ -174,8 +178,9 @@ public class SimpleGarbageCollector impl
this.address = address;
}
- public void init(FileSystem fs, Instance instance, AuthInfo credentials) {
+ public void init(FileSystem fs, Instance instance, AuthInfo credentials)
throws IOException {
this.fs = TraceFileSystem.wrap(fs);
+ this.trash = new Trash(fs, fs.getConf());
this.credentials = credentials;
this.instance = instance;
@@ -189,6 +194,8 @@ public class SimpleGarbageCollector impl
log.info("verbose: " + verbose);
log.info("memory threshold: " + CANDIDATE_MEMORY_PERCENTAGE + " of " +
Runtime.getRuntime().maxMemory() + " bytes");
log.info("delete threads: " + numDeleteThreads);
+ log.info("Starting trash emptier");
+ new Daemon(new LoggingRunnable(log, trash.getEmptier()), "Trash
Emptier").start();
}
private void run() {
@@ -590,7 +597,7 @@ public class SimpleGarbageCollector impl
Path p = new Path(ServerConstants.getTablesDir() + delete);
- if (fs.delete(p, true)) {
+ if (trash.moveToTrash(p) || fs.delete(p, true)) {
// delete succeeded, still want to delete
removeFlag = true;
synchronized (SimpleGarbageCollector.this) {