Return-Path: X-Original-To: apmail-accumulo-commits-archive@www.apache.org Delivered-To: apmail-accumulo-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id BDF0C10B3C for ; Fri, 8 Aug 2014 18:25:57 +0000 (UTC) Received: (qmail 6907 invoked by uid 500); 8 Aug 2014 18:25:57 -0000 Delivered-To: apmail-accumulo-commits-archive@accumulo.apache.org Received: (qmail 6870 invoked by uid 500); 8 Aug 2014 18:25:57 -0000 Mailing-List: contact commits-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@accumulo.apache.org Delivered-To: mailing list commits@accumulo.apache.org Received: (qmail 6858 invoked by uid 99); 8 Aug 2014 18:25:57 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 08 Aug 2014 18:25:57 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 6E79A8A1160; Fri, 8 Aug 2014 18:25:57 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: ecn@apache.org To: commits@accumulo.apache.org Date: Fri, 08 Aug 2014 18:25:57 -0000 Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: [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 Authored: Thu Aug 7 17:24:01 2014 -0400 Committer: Eric C. Newton 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> 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 processor = new Processor(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)); }