You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2014/08/08 20:25:57 UTC

[1/3] git commit: ACCUMULO-2560 fetch gc.threads.delete from configuration every time

Repository: accumulo
Updated Branches:
  refs/heads/master ab6871e16 -> 7f82a15f2


ACCUMULO-2560 fetch gc.threads.delete from configuration every time


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/6023dba2
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/6023dba2
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/6023dba2

Branch: refs/heads/master
Commit: 6023dba227d25cbfe28c8d6332e8dd6e4bdc8024
Parents: 290866f
Author: Eric C. Newton <er...@gmail.com>
Authored: Thu Aug 7 17:24:01 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Thu Aug 7 17:24:01 2014 -0400

----------------------------------------------------------------------
 .../accumulo/gc/SimpleGarbageCollector.java     | 63 +++++++-------------
 .../accumulo/gc/SimpleGarbageCollectorTest.java | 16 +++--
 2 files changed, 35 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/6023dba2/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
----------------------------------------------------------------------
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
index 378ee78..64db1db 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
@@ -127,29 +127,27 @@ public class SimpleGarbageCollector implements Iface {
   private static final Logger log = Logger.getLogger(SimpleGarbageCollector.class);
 
   private Credentials credentials;
-  private long gcStartDelay;
   private VolumeManager fs;
-  private boolean useTrash = true;
+  private AccumuloConfiguration config;
   private Opts opts = new Opts();
   private ZooLock lock;
 
   private GCStatus status = new GCStatus(new GcCycleStats(), new GcCycleStats(), new GcCycleStats(), new GcCycleStats());
 
-  private int numDeleteThreads;
-
   private Instance instance;
 
   public static void main(String[] args) throws UnknownHostException, IOException {
     SecurityUtil.serverLogin(ServerConfiguration.getSiteConfiguration());
     Instance instance = HdfsZooInstance.getInstance();
-    ServerConfiguration serverConf = new ServerConfiguration(instance);
+    ServerConfiguration conf = new ServerConfiguration(instance);
     final VolumeManager fs = VolumeManagerImpl.get();
-    Accumulo.init(fs, serverConf, "gc");
+    Accumulo.init(fs, conf, "gc");
     Opts opts = new Opts();
     opts.parseArgs("gc", args);
     SimpleGarbageCollector gc = new SimpleGarbageCollector(opts);
+    AccumuloConfiguration config = conf.getConfiguration();
 
-    gc.init(fs, instance, SystemCredentials.get(), serverConf.getConfiguration().getBoolean(Property.GC_TRASH_IGNORE));
+    gc.init(fs, instance, SystemCredentials.get(), config);
     Accumulo.enableTracing(opts.getAddress(), "gc");
     gc.run();
   }
@@ -177,7 +175,7 @@ public class SimpleGarbageCollector implements Iface {
    * @return start delay, in milliseconds
    */
   long getStartDelay() {
-    return gcStartDelay;
+    return config.getTimeInMillis(Property.GC_CYCLE_START);
   }
   /**
    * Gets the volume manager used by this GC.
@@ -194,7 +192,7 @@ public class SimpleGarbageCollector implements Iface {
    * @return true if trash is used
    */
   boolean isUsingTrash() {
-    return useTrash;
+    return !config.getBoolean(Property.GC_TRASH_IGNORE);
   }
   /**
    * Gets the options for this garbage collector.
@@ -208,7 +206,7 @@ public class SimpleGarbageCollector implements Iface {
    * @return number of delete threads
    */
   int getNumDeleteThreads() {
-    return numDeleteThreads;
+    return config.getCount(Property.GC_DELETE_THREADS);
   }
   /**
    * Gets the instance used by this GC.
@@ -220,18 +218,6 @@ public class SimpleGarbageCollector implements Iface {
   }
 
   /**
-   * Initializes this garbage collector with the current system configuration.
-   *
-   * @param fs volume manager
-   * @param instance instance
-   * @param credentials credentials
-   * @param noTrash true to not move files to trash instead of deleting
-   */
-  public void init(VolumeManager fs, Instance instance, Credentials credentials, boolean noTrash) {
-    init(fs, instance, credentials, noTrash, ServerConfiguration.getSystemConfiguration(instance));
-  }
-
-  /**
    * Initializes this garbage collector.
    *
    * @param fs volume manager
@@ -240,21 +226,18 @@ public class SimpleGarbageCollector implements Iface {
    * @param noTrash true to not move files to trash instead of deleting
    * @param systemConfig system configuration
    */
-  public void init(VolumeManager fs, Instance instance, Credentials credentials, boolean noTrash, AccumuloConfiguration systemConfig) {
+  public void init(VolumeManager fs, Instance instance, Credentials credentials, AccumuloConfiguration config) {
     this.fs = fs;
     this.credentials = credentials;
     this.instance = instance;
-
-    gcStartDelay = systemConfig.getTimeInMillis(Property.GC_CYCLE_START);
-    long gcDelay = systemConfig.getTimeInMillis(Property.GC_CYCLE_DELAY);
-    numDeleteThreads = systemConfig.getCount(Property.GC_DELETE_THREADS);
-    log.info("start delay: " + gcStartDelay + " milliseconds");
+    this.config = config;
+    long gcDelay = config.getTimeInMillis(Property.GC_CYCLE_DELAY);
+    log.info("start delay: " + getStartDelay() + " milliseconds");
     log.info("time delay: " + gcDelay + " milliseconds");
     log.info("safemode: " + opts.safeMode);
     log.info("verbose: " + opts.verbose);
     log.info("memory threshold: " + CANDIDATE_MEMORY_PERCENTAGE + " of " + Runtime.getRuntime().maxMemory() + " bytes");
-    log.info("delete threads: " + numDeleteThreads);
-    useTrash = !noTrash;
+    log.info("delete threads: " + getNumDeleteThreads());
   }
 
   private class GCEnv implements GarbageCollectionEnvironment {
@@ -376,7 +359,7 @@ public class SimpleGarbageCollector implements Iface {
 
       final BatchWriter finalWriter = writer;
 
-      ExecutorService deleteThreadPool = Executors.newFixedThreadPool(numDeleteThreads, new NamingThreadFactory("deleting"));
+      ExecutorService deleteThreadPool = Executors.newFixedThreadPool(getNumDeleteThreads(), new NamingThreadFactory("deleting"));
 
       final List<Pair<Path,Path>> replacements = ServerConstants.getVolumeReplacements();
 
@@ -521,8 +504,9 @@ public class SimpleGarbageCollector implements Iface {
     }
 
     try {
-      log.debug("Sleeping for " + gcStartDelay + " milliseconds before beginning garbage collection cycles");
-      Thread.sleep(gcStartDelay);
+      long delay = getStartDelay();
+      log.debug("Sleeping for " + delay + " milliseconds before beginning garbage collection cycles");
+      Thread.sleep(delay);
     } catch (InterruptedException e) {
       log.warn(e, e);
       return;
@@ -563,7 +547,7 @@ public class SimpleGarbageCollector implements Iface {
       // Clean up any unused write-ahead logs
       Span waLogs = Trace.start("walogs");
       try {
-        GarbageCollectWriteAheadLogs walogCollector = new GarbageCollectWriteAheadLogs(instance, fs, useTrash);
+        GarbageCollectWriteAheadLogs walogCollector = new GarbageCollectWriteAheadLogs(instance, fs, isUsingTrash());
         log.info("Beginning garbage collection of write-ahead logs");
         walogCollector.collect(status);
       } catch (Exception e) {
@@ -584,7 +568,7 @@ public class SimpleGarbageCollector implements Iface {
 
       Trace.offNoFlush();
       try {
-        long gcDelay = ServerConfiguration.getSystemConfiguration(instance).getTimeInMillis(Property.GC_CYCLE_DELAY);
+        long gcDelay = config.getTimeInMillis(Property.GC_CYCLE_DELAY);
         log.debug("Sleeping for " + gcDelay + " milliseconds");
         Thread.sleep(gcDelay);
       } catch (InterruptedException e) {
@@ -604,7 +588,7 @@ public class SimpleGarbageCollector implements Iface {
    * @throws IOException if the volume manager encountered a problem
    */
   boolean moveToTrash(Path path) throws IOException {
-    if (!useTrash)
+    if (!isUsingTrash())
       return false;
     try {
       return fs.moveToTrash(path);
@@ -646,14 +630,13 @@ public class SimpleGarbageCollector implements Iface {
 
   private HostAndPort startStatsService() throws UnknownHostException {
     Processor<Iface> processor = new Processor<Iface>(RpcWrapper.service(this));
-    AccumuloConfiguration conf = ServerConfiguration.getSystemConfiguration(instance);
-    int port = conf.getPort(Property.GC_PORT);
-    long maxMessageSize = conf.getMemoryInBytes(Property.GENERAL_MAX_MESSAGE_SIZE);
+    int port = config.getPort(Property.GC_PORT);
+    long maxMessageSize = config.getMemoryInBytes(Property.GENERAL_MAX_MESSAGE_SIZE);
     HostAndPort result = HostAndPort.fromParts(opts.getAddress(), port);
     log.debug("Starting garbage collector listening on " + result);
     try {
       return TServerUtils.startTServer(result, processor, this.getClass().getSimpleName(), "GC Monitor Service", 2, 1000, maxMessageSize,
-          SslConnectionParams.forServer(conf), 0).address;
+          SslConnectionParams.forServer(config), 0).address;
     } catch (Exception ex) {
       log.fatal(ex, ex);
       throw new RuntimeException(ex);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6023dba2/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
----------------------------------------------------------------------
diff --git a/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java b/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
index 532eeba..e3d215f 100644
--- a/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
+++ b/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
@@ -70,13 +70,15 @@ public class SimpleGarbageCollectorTest {
     expect(systemConfig.getTimeInMillis(Property.GC_CYCLE_START)).andReturn(1000L);
     expect(systemConfig.getTimeInMillis(Property.GC_CYCLE_DELAY)).andReturn(20000L);
     expect(systemConfig.getCount(Property.GC_DELETE_THREADS)).andReturn(2);
+    expect(systemConfig.getCount(Property.GC_DELETE_THREADS)).andReturn(2);
+    expect(systemConfig.getBoolean(Property.GC_TRASH_IGNORE)).andReturn(false);
     replay(systemConfig);
     return systemConfig;
   }
 
   @Test
   public void testInit() throws Exception {
-    gc.init(volMgr, instance, credentials, false, systemConfig);
+    gc.init(volMgr, instance, credentials, systemConfig);
     assertSame(volMgr, gc.getVolumeManager());
     assertSame(instance, gc.getInstance());
     assertSame(credentials, gc.getCredentials());
@@ -87,7 +89,7 @@ public class SimpleGarbageCollectorTest {
 
   @Test
   public void testMoveToTrash_UsingTrash() throws Exception {
-    gc.init(volMgr, instance, credentials, false, systemConfig);
+    gc.init(volMgr, instance, credentials, systemConfig);
     Path path = createMock(Path.class);
     expect(volMgr.moveToTrash(path)).andReturn(true);
     replay(volMgr);
@@ -97,7 +99,7 @@ public class SimpleGarbageCollectorTest {
 
   @Test
   public void testMoveToTrash_UsingTrash_VolMgrFailure() throws Exception {
-    gc.init(volMgr, instance, credentials, false, systemConfig);
+    gc.init(volMgr, instance, credentials, systemConfig);
     Path path = createMock(Path.class);
     expect(volMgr.moveToTrash(path)).andThrow(new FileNotFoundException());
     replay(volMgr);
@@ -107,7 +109,13 @@ public class SimpleGarbageCollectorTest {
 
   @Test
   public void testMoveToTrash_NotUsingTrash() throws Exception {
-    gc.init(volMgr, instance, credentials, true, systemConfig);
+    AccumuloConfiguration systemConfig = createMock(AccumuloConfiguration.class);
+    expect(systemConfig.getTimeInMillis(Property.GC_CYCLE_START)).andReturn(1000L);
+    expect(systemConfig.getTimeInMillis(Property.GC_CYCLE_DELAY)).andReturn(20000L);
+    expect(systemConfig.getCount(Property.GC_DELETE_THREADS)).andReturn(2);
+    expect(systemConfig.getBoolean(Property.GC_TRASH_IGNORE)).andReturn(true);
+    replay(systemConfig);
+    gc.init(volMgr, instance, credentials, systemConfig);
     Path path = createMock(Path.class);
     assertFalse(gc.moveToTrash(path));
   }


[3/3] git commit: ACCUMULO-2560 merge 1.6.1

Posted by ec...@apache.org.
ACCUMULO-2560 merge 1.6.1


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/7f82a15f
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/7f82a15f
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/7f82a15f

Branch: refs/heads/master
Commit: 7f82a15f243eb103da88f45a37e32e8981d05842
Parents: 44fc2a6
Author: Eric C. Newton <er...@gmail.com>
Authored: Fri Aug 8 12:04:07 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Aug 8 12:04:07 2014 -0400

----------------------------------------------------------------------
 .../java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java     | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/7f82a15f/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
----------------------------------------------------------------------
diff --git a/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java b/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
index e3d215f..30ac8ad 100644
--- a/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
+++ b/server/gc/src/test/java/org/apache/accumulo/gc/SimpleGarbageCollectorTest.java
@@ -68,6 +68,7 @@ public class SimpleGarbageCollectorTest {
   private AccumuloConfiguration mockSystemConfig() {
     AccumuloConfiguration systemConfig = createMock(AccumuloConfiguration.class);
     expect(systemConfig.getTimeInMillis(Property.GC_CYCLE_START)).andReturn(1000L);
+    expect(systemConfig.getTimeInMillis(Property.GC_CYCLE_START)).andReturn(1000L);
     expect(systemConfig.getTimeInMillis(Property.GC_CYCLE_DELAY)).andReturn(20000L);
     expect(systemConfig.getCount(Property.GC_DELETE_THREADS)).andReturn(2);
     expect(systemConfig.getCount(Property.GC_DELETE_THREADS)).andReturn(2);


[2/3] git commit: ACCUMULO-2560 merge 1.6.1

Posted by ec...@apache.org.
ACCUMULO-2560 merge 1.6.1


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/44fc2a64
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/44fc2a64
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/44fc2a64

Branch: refs/heads/master
Commit: 44fc2a646549eb6f020fce6212cff294dc30b27f
Parents: ab6871e 6023dba
Author: Eric C. Newton <er...@gmail.com>
Authored: Fri Aug 8 12:03:53 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Aug 8 12:03:53 2014 -0400

----------------------------------------------------------------------
 .../server/fs/VolumeManagerImplTest.java        |  1 -
 .../accumulo/gc/SimpleGarbageCollector.java     | 65 ++++++++------------
 .../accumulo/gc/SimpleGarbageCollectorTest.java | 16 +++--
 3 files changed, 36 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/44fc2a64/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java
----------------------------------------------------------------------
diff --cc server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java
index 5f97bf6,5f97bf6..757405a
--- a/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java
@@@ -19,7 -19,7 +19,6 @@@ package org.apache.accumulo.server.fs
  import java.util.Arrays;
  import java.util.List;
  
--import org.apache.accumulo.core.conf.AccumuloConfiguration;
  import org.apache.accumulo.core.conf.ConfigurationCopy;
  import org.apache.accumulo.core.conf.Property;
  import org.apache.accumulo.server.fs.VolumeManager.FileType;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/44fc2a64/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
----------------------------------------------------------------------
diff --cc server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
index 2466635,64db1db..120a078
--- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
@@@ -145,13 -137,11 +144,11 @@@ public class SimpleGarbageCollector imp
    private Instance instance;
  
    public static void main(String[] args) throws UnknownHostException, IOException {
 -    SecurityUtil.serverLogin(ServerConfiguration.getSiteConfiguration());
 +    SecurityUtil.serverLogin(SiteConfiguration.getInstance());
      Instance instance = HdfsZooInstance.getInstance();
-     ServerConfigurationFactory serverConf = new ServerConfigurationFactory(instance);
 -    ServerConfiguration conf = new ServerConfiguration(instance);
++    ServerConfigurationFactory conf = new ServerConfigurationFactory(instance);
      final VolumeManager fs = VolumeManagerImpl.get();
-     Accumulo.init(fs, serverConf, "gc");
+     Accumulo.init(fs, conf, "gc");
      Opts opts = new Opts();
      opts.parseArgs("gc", args);
      SimpleGarbageCollector gc = new SimpleGarbageCollector(opts);
@@@ -699,8 -635,8 +682,8 @@@
      HostAndPort result = HostAndPort.fromParts(opts.getAddress(), port);
      log.debug("Starting garbage collector listening on " + result);
      try {
-       return TServerUtils.startTServer(result, processor, this.getClass().getSimpleName(), "GC Monitor Service", 2,
-           conf.getCount(Property.GENERAL_SIMPLETIMER_THREADPOOL_SIZE), 1000, maxMessageSize, SslConnectionParams.forServer(conf), 0).address;
 -      return TServerUtils.startTServer(result, processor, this.getClass().getSimpleName(), "GC Monitor Service", 2, 1000, maxMessageSize,
 -          SslConnectionParams.forServer(config), 0).address;
++      return TServerUtils.startTServer(result, processor, this.getClass().getSimpleName(), "GC Monitor Service", 2, 
++          config.getCount(Property.GENERAL_SIMPLETIMER_THREADPOOL_SIZE), 1000, maxMessageSize, SslConnectionParams.forServer(config), 0).address;
      } catch (Exception ex) {
        log.fatal(ex, ex);
        throw new RuntimeException(ex);