http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/main/java/org/apache/accumulo/server/util/time/SimpleTimer.java ---------------------------------------------------------------------- diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/time/SimpleTimer.java b/server/base/src/main/java/org/apache/accumulo/server/util/time/SimpleTimer.java index bc1d652..556e6b9 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/time/SimpleTimer.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/time/SimpleTimer.java @@ -16,24 +16,26 @@ */ package org.apache.accumulo.server.util.time; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; + import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; import org.apache.log4j.Logger; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.util.concurrent.ThreadFactoryBuilder; + /** * Generic singleton timer. Don't use this if you are going to do anything that will take very long. Please use it to reduce the number of threads dedicated to * simple events. - * + * */ public class SimpleTimer { private static final Logger log = Logger.getLogger(SimpleTimer.class); - + private static class ExceptionHandler implements Thread.UncaughtExceptionHandler { public void uncaughtException(Thread t, Throwable e) { log.warn("SimpleTimer task failed", e); @@ -43,24 +45,25 @@ public class SimpleTimer { private static int instanceThreadPoolSize = -1; private static SimpleTimer instance; private ScheduledExecutorService executor; - + private static final int DEFAULT_THREAD_POOL_SIZE = 1; + /** * Gets the timer instance. * - * @deprecated Use {@link #getInstance(AccumuloConfiguration)} instead to - * get the configured number of threads. + * @deprecated Use {@link #getInstance(AccumuloConfiguration)} instead to get the configured number of threads. */ @Deprecated public static synchronized SimpleTimer getInstance() { return getInstance(null); } + /** - * Gets the timer instance. If an instance has already been created, it will - * have the number of threads supplied when it was constructed, and the size + * Gets the timer instance. If an instance has already been created, it will have the number of threads supplied when it was constructed, and the size * provided here is ignored. * - * @param threadPoolSize number of threads + * @param threadPoolSize + * number of threads */ public static synchronized SimpleTimer getInstance(int threadPoolSize) { if (instance == null) { @@ -68,20 +71,18 @@ public class SimpleTimer { SimpleTimer.instanceThreadPoolSize = threadPoolSize; } else { if (SimpleTimer.instanceThreadPoolSize != threadPoolSize) { - log.warn("Asked to create SimpleTimer with thread pool size " + - threadPoolSize + ", existing instance has " + - instanceThreadPoolSize); + log.warn("Asked to create SimpleTimer with thread pool size " + threadPoolSize + ", existing instance has " + instanceThreadPoolSize); } } return instance; } + /** - * Gets the timer instance. If an instance has already been created, it will - * have the number of threads supplied when it was constructed, and the size - * provided by the configuration here is ignored. If a null configuration is - * supplied, the number of threads defaults to 1. + * Gets the timer instance. If an instance has already been created, it will have the number of threads supplied when it was constructed, and the size + * provided by the configuration here is ignored. If a null configuration is supplied, the number of threads defaults to 1. * - * @param conf configuration from which to get the number of threads + * @param conf + * configuration from which to get the number of threads * @see Property#GENERAL_SIMPLETIMER_THREADPOOL_SIZE */ public static synchronized SimpleTimer getInstance(AccumuloConfiguration conf) { @@ -103,30 +104,34 @@ public class SimpleTimer { static int getInstanceThreadPoolSize() { return instanceThreadPoolSize; } - + private SimpleTimer(int threadPoolSize) { executor = Executors.newScheduledThreadPool(threadPoolSize, new ThreadFactoryBuilder().setNameFormat("SimpleTimer-%d").setDaemon(true) - .setUncaughtExceptionHandler(new ExceptionHandler()).build()); + .setUncaughtExceptionHandler(new ExceptionHandler()).build()); } - + /** * Schedules a task to run in the future. * - * @param task task to run - * @param delay number of milliseconds to wait before execution + * @param task + * task to run + * @param delay + * number of milliseconds to wait before execution * @return future for scheduled task */ public ScheduledFuture<?> schedule(Runnable task, long delay) { return executor.schedule(task, delay, TimeUnit.MILLISECONDS); } - + /** - * Schedules a task to run in the future with a fixed delay between repeated - * executions. + * Schedules a task to run in the future with a fixed delay between repeated executions. * - * @param task task to run - * @param delay number of milliseconds to wait before first execution - * @param period number of milliseconds to wait between executions + * @param task + * task to run + * @param delay + * number of milliseconds to wait before first execution + * @param period + * number of milliseconds to wait between executions * @return future for scheduled task */ public ScheduledFuture<?> schedule(Runnable task, long delay, long period) {
http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/main/java/org/apache/accumulo/server/util/time/SystemTime.java ---------------------------------------------------------------------- diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/time/SystemTime.java b/server/base/src/main/java/org/apache/accumulo/server/util/time/SystemTime.java index c421f5f..b8fe371 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/time/SystemTime.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/time/SystemTime.java @@ -18,13 +18,13 @@ package org.apache.accumulo.server.util.time; /** * The most obvious implementation of ProvidesTime. - * + * */ public class SystemTime implements ProvidesTime { - + @Override public long currentTime() { return System.currentTimeMillis(); } - + } http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java ---------------------------------------------------------------------- diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java index 1452aa7..412a97a 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java +++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java @@ -38,15 +38,15 @@ import org.apache.zookeeper.Watcher; /** * Provides a way to push work out to tablet servers via zookeeper and wait for that work to be done. Any tablet server can pick up a work item and process it. - * + * * Worker processes watch a zookeeper node for tasks to be performed. After getting an exclusive lock on the node, the worker will perform the task. */ public class DistributedWorkQueue { - + private static final String LOCKS_NODE = "locks"; private static final Logger log = Logger.getLogger(DistributedWorkQueue.class); - + private ThreadPoolExecutor threadPool; private ZooReaderWriter zoo = ZooReaderWriter.getInstance(); private String path; @@ -58,15 +58,15 @@ public class DistributedWorkQueue { private void lookForWork(final Processor processor, List<String> children) { if (children.size() == 0) return; - + if (numTask.get() >= threadPool.getCorePoolSize()) return; - + Random random = new Random(); Collections.shuffle(children, random); try { for (final String child : children) { - + if (child.equals(LOCKS_NODE)) continue; @@ -82,7 +82,7 @@ public class DistributedWorkQueue { } final String childPath = path + "/" + child; - + // check to see if another node processed it already if (!zoo.exists(childPath)) { zoo.recursiveDelete(lockPath, NodeMissingPolicy.SKIP); @@ -94,9 +94,9 @@ public class DistributedWorkQueue { zoo.recursiveDelete(lockPath, NodeMissingPolicy.SKIP); break; } - + log.debug("got lock for " + child); - + Runnable task = new Runnable() { @Override @@ -104,18 +104,18 @@ public class DistributedWorkQueue { try { try { processor.newProcessor().process(child, zoo.getData(childPath, null)); - + // if the task fails, then its entry in the Q is not deleted... so it will be retried try { zoo.recursiveDelete(childPath, NodeMissingPolicy.SKIP); } catch (Exception e) { log.error("Error received when trying to delete entry in zookeeper " + childPath, e); } - + } catch (Exception e) { log.warn("Failed to process work " + child, e); } - + try { zoo.recursiveDelete(lockPath, NodeMissingPolicy.SKIP); } catch (Exception e) { @@ -125,7 +125,7 @@ public class DistributedWorkQueue { } finally { numTask.decrementAndGet(); } - + try { // its important that this is called after numTask is decremented lookForWork(processor, zoo.getChildren(path)); @@ -136,7 +136,7 @@ public class DistributedWorkQueue { } } }; - + numTask.incrementAndGet(); threadPool.execute(task); @@ -151,10 +151,10 @@ public class DistributedWorkQueue { void process(String workID, byte[] data); } - + public DistributedWorkQueue(String path, AccumuloConfiguration config) { // Preserve the old delay and period - this(path, config, new Random().nextInt(60*1000), 60*1000); + this(path, config, new Random().nextInt(60 * 1000), 60 * 1000); } public DistributedWorkQueue(String path, AccumuloConfiguration config, long timerInitialDelay, long timerPeriod) { @@ -163,9 +163,9 @@ public class DistributedWorkQueue { this.timerInitialDelay = timerInitialDelay; this.timerPeriod = timerPeriod; } - + public void startProcessing(final Processor processor, ThreadPoolExecutor executorService) throws KeeperException, InterruptedException { - + threadPool = executorService; zoo.mkdirs(path); @@ -193,13 +193,13 @@ public class DistributedWorkQueue { case None: log.info("Got unexpected zookeeper event: " + event.getType() + " for " + path); break; - + } } }); - + lookForWork(processor, children); - + // Add a little jitter to avoid all the tservers slamming zookeeper at once SimpleTimer.getInstance(config).schedule(new Runnable() { @Override @@ -222,7 +222,7 @@ public class DistributedWorkQueue { public void addWork(String workId, String data) throws KeeperException, InterruptedException { addWork(workId, data.getBytes(UTF_8)); } - + public void addWork(String workId, byte[] data) throws KeeperException, InterruptedException { if (workId.equalsIgnoreCase(LOCKS_NODE)) throw new IllegalArgumentException("locks is reserved work id"); @@ -230,7 +230,7 @@ public class DistributedWorkQueue { zoo.mkdirs(path); zoo.putPersistentData(path + "/" + workId, data, NodeExistsPolicy.SKIP); } - + public List<String> getWorkQueued() throws KeeperException, InterruptedException { ArrayList<String> children = new ArrayList<String>(zoo.getChildren(path)); children.remove(LOCKS_NODE); @@ -238,9 +238,9 @@ public class DistributedWorkQueue { } public void waitUntilDone(Set<String> workIDs) throws KeeperException, InterruptedException { - + final Object condVar = new Object(); - + Watcher watcher = new Watcher() { @Override public void process(WatchedEvent event) { @@ -256,13 +256,13 @@ public class DistributedWorkQueue { case None: log.info("Got unexpected zookeeper event: " + event.getType() + " for " + path); break; - + } } }; - + List<String> children = zoo.getChildren(path, watcher); - + while (!Collections.disjoint(children, workIDs)) { synchronized (condVar) { condVar.wait(10000); http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java ---------------------------------------------------------------------- diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java index 4e0e977..0e1cdfd 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java +++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/TransactionWatcher.java @@ -27,17 +27,17 @@ import org.apache.zookeeper.KeeperException; public class TransactionWatcher extends org.apache.accumulo.fate.zookeeper.TransactionWatcher { public static class ZooArbitrator implements Arbitrator { - + Instance instance = HdfsZooInstance.getInstance(); ZooReader rdr = new ZooReader(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut()); - + @Override public boolean transactionAlive(String type, long tid) throws Exception { String path = ZooUtil.getRoot(instance) + "/" + type + "/" + tid; rdr.sync(path); return rdr.exists(path); } - + public static void start(String type, long tid) throws KeeperException, InterruptedException { Instance instance = HdfsZooInstance.getInstance(); IZooReaderWriter writer = ZooReaderWriter.getInstance(); @@ -45,13 +45,13 @@ public class TransactionWatcher extends org.apache.accumulo.fate.zookeeper.Trans writer.putPersistentData(ZooUtil.getRoot(instance) + "/" + type + "/" + tid, new byte[] {}, NodeExistsPolicy.OVERWRITE); writer.putPersistentData(ZooUtil.getRoot(instance) + "/" + type + "/" + tid + "-running", new byte[] {}, NodeExistsPolicy.OVERWRITE); } - + public static void stop(String type, long tid) throws KeeperException, InterruptedException { Instance instance = HdfsZooInstance.getInstance(); IZooReaderWriter writer = ZooReaderWriter.getInstance(); writer.recursiveDelete(ZooUtil.getRoot(instance) + "/" + type + "/" + tid, NodeMissingPolicy.SKIP); } - + public static void cleanup(String type, long tid) throws KeeperException, InterruptedException { Instance instance = HdfsZooInstance.getInstance(); IZooReaderWriter writer = ZooReaderWriter.getInstance(); @@ -66,7 +66,7 @@ public class TransactionWatcher extends org.apache.accumulo.fate.zookeeper.Trans return !rdr.exists(path); } } - + public TransactionWatcher() { super(new ZooArbitrator()); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java ---------------------------------------------------------------------- diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java index bf34ef6..aca9c82 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java +++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooCache.java @@ -24,11 +24,11 @@ public class ZooCache extends org.apache.accumulo.fate.zookeeper.ZooCache { public ZooCache() { this(null); } - + public ZooCache(Watcher watcher) { super(ZooReaderWriter.getInstance(), watcher); } - + public ZooCache(AccumuloConfiguration conf, Watcher watcher) { super(conf.get(Property.INSTANCE_ZK_HOST), (int) conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT), watcher); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooLock.java ---------------------------------------------------------------------- diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooLock.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooLock.java index dce6d38..b67b82b 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooLock.java +++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooLock.java @@ -19,15 +19,15 @@ package org.apache.accumulo.server.zookeeper; import org.apache.zookeeper.KeeperException; public class ZooLock extends org.apache.accumulo.fate.zookeeper.ZooLock { - + public ZooLock(String path) { super(new ZooCache(), ZooReaderWriter.getInstance(), path); } - + public static void deleteLock(String path) throws InterruptedException, KeeperException { deleteLock(ZooReaderWriter.getInstance(), path); } - + public static boolean deleteLock(String path, String lockData) throws InterruptedException, KeeperException { return deleteLock(ZooReaderWriter.getInstance(), path, lockData); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriter.java ---------------------------------------------------------------------- diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriter.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriter.java index 34c9070..8a3383c 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriter.java +++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/ZooReaderWriter.java @@ -26,11 +26,11 @@ public class ZooReaderWriter extends org.apache.accumulo.fate.zookeeper.ZooReade private static final String SCHEME = "digest"; private static final String USER = "accumulo"; private static ZooReaderWriter instance = null; - + public ZooReaderWriter(String string, int timeInMillis, String secret) { super(string, timeInMillis, SCHEME, (USER + ":" + secret).getBytes(UTF_8)); } - + public static synchronized ZooReaderWriter getInstance() { if (instance == null) { AccumuloConfiguration conf = SiteConfiguration.getInstance(); @@ -39,5 +39,5 @@ public class ZooReaderWriter extends org.apache.accumulo.fate.zookeeper.ZooReade } return instance; } - + } http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java b/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java index 52929ed..19b0a9b 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java @@ -108,6 +108,7 @@ public class AccumuloTest { FileUtils.deleteDirectory(confDir); } } + @Test public void testLocateLogConfig_Default() throws Exception { File confDir = new File(FileUtils.getTempDirectory(), "AccumuloTest" + System.currentTimeMillis()); @@ -121,6 +122,7 @@ public class AccumuloTest { FileUtils.deleteDirectory(confDir); } } + @Test public void testLocateLogConfig_Explicit() throws Exception { File confDir = new File(FileUtils.getTempDirectory(), "AccumuloTest" + System.currentTimeMillis()); http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/ServerConstantsTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/ServerConstantsTest.java b/server/base/src/test/java/org/apache/accumulo/server/ServerConstantsTest.java index 8c8b2f2..40d5eb8 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/ServerConstantsTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/ServerConstantsTest.java @@ -33,7 +33,7 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; /** - * + * */ public class ServerConstantsTest { @Rule http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java b/server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java index d4420aa..2c6889b 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java @@ -16,9 +16,10 @@ */ package org.apache.accumulo.server; +import static org.junit.Assert.assertEquals; + import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.*; public class ServerOptsTest { private ServerOpts opts; http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java b/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java index d12483c..fad6398 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java @@ -52,7 +52,7 @@ import org.junit.Assert; import org.junit.Test; public class BulkImporterTest { - + static final SortedSet<KeyExtent> fakeMetaData = new TreeSet<KeyExtent>(); static final Text tableId = new Text("1"); static { @@ -62,49 +62,49 @@ public class BulkImporterTest { } fakeMetaData.add(new KeyExtent(tableId, null, fakeMetaData.last().getEndRow())); } - + class MockTabletLocator extends TabletLocator { int invalidated = 0; - + @Override public TabletLocation locateTablet(ClientContext context, Text row, boolean skipRow, boolean retry) throws AccumuloException, AccumuloSecurityException, TableNotFoundException { return new TabletLocation(fakeMetaData.tailSet(new KeyExtent(tableId, row, null)).first(), "localhost", "1"); } - + @Override public <T extends Mutation> void binMutations(ClientContext context, List<T> mutations, Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures) throws AccumuloException, AccumuloSecurityException, TableNotFoundException { throw new NotImplementedException(); } - + @Override public List<Range> binRanges(ClientContext context, List<Range> ranges, Map<String,Map<KeyExtent,List<Range>>> binnedRanges) throws AccumuloException, AccumuloSecurityException, TableNotFoundException { throw new NotImplementedException(); } - + @Override public void invalidateCache(KeyExtent failedExtent) { invalidated++; } - + @Override public void invalidateCache(Collection<KeyExtent> keySet) { throw new NotImplementedException(); } - + @Override public void invalidateCache() { throw new NotImplementedException(); } - + @Override public void invalidateCache(Instance instance, String server) { throw new NotImplementedException(); } } - + @Test public void testFindOverlappingTablets() throws Exception { MockTabletLocator locator = new MockTabletLocator(); @@ -145,7 +145,7 @@ public class BulkImporterTest { Assert.assertEquals(new KeyExtent(tableId, new Text("dm"), new Text("d")), overlaps.get(2).tablet_extent); Assert.assertEquals(new KeyExtent(tableId, new Text("j"), new Text("i")), overlaps.get(3).tablet_extent); Assert.assertEquals(new KeyExtent(tableId, null, new Text("l")), overlaps.get(4).tablet_extent); - + List<TabletLocation> overlaps2 = BulkImporter.findOverlappingTablets(context, vm, locator, new Path(file), new KeyExtent(tableId, new Text("h"), new Text( "b"))); Assert.assertEquals(3, overlaps2.size()); @@ -154,5 +154,5 @@ public class BulkImporterTest { Assert.assertEquals(new KeyExtent(tableId, new Text("j"), new Text("i")), overlaps2.get(2).tablet_extent); Assert.assertEquals(locator.invalidated, 1); } - + } http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java index 628b981..6bd6424 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/conf/NamespaceConfigurationTest.java @@ -17,9 +17,6 @@ package org.apache.accumulo.server.conf; import static java.nio.charset.StandardCharsets.UTF_8; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; import static org.easymock.EasyMock.anyObject; import static org.easymock.EasyMock.createMock; import static org.easymock.EasyMock.eq; @@ -27,11 +24,15 @@ import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.expectLastCall; import static org.easymock.EasyMock.replay; import static org.easymock.EasyMock.verify; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; import java.util.Collection; import java.util.List; import java.util.Map; import java.util.UUID; + import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.Instance; import org.apache.accumulo.core.client.impl.Namespaces; @@ -86,8 +87,8 @@ public class NamespaceConfigurationTest { @Test public void testGet_InZK() { Property p = Property.INSTANCE_SECRET; - expect(zc.get(ZooUtil.getRoot(iid) + Constants.ZNAMESPACES + "/" + NSID + Constants.ZNAMESPACE_CONF + "/" + p.getKey())).andReturn( - "sekrit".getBytes(UTF_8)); + expect(zc.get(ZooUtil.getRoot(iid) + Constants.ZNAMESPACES + "/" + NSID + Constants.ZNAMESPACE_CONF + "/" + p.getKey())) + .andReturn("sekrit".getBytes(UTF_8)); replay(zc); assertEquals("sekrit", c.get(Property.INSTANCE_SECRET)); } @@ -123,10 +124,8 @@ public class NamespaceConfigurationTest { children.add("foo"); children.add("ding"); expect(zc.getChildren(ZooUtil.getRoot(iid) + Constants.ZNAMESPACES + "/" + NSID + Constants.ZNAMESPACE_CONF)).andReturn(children); - expect(zc.get(ZooUtil.getRoot(iid) + Constants.ZNAMESPACES + "/" + NSID + Constants.ZNAMESPACE_CONF + "/" + "foo")).andReturn( - "bar".getBytes(UTF_8)); - expect(zc.get(ZooUtil.getRoot(iid) + Constants.ZNAMESPACES + "/" + NSID + Constants.ZNAMESPACE_CONF + "/" + "ding")).andReturn( - "dong".getBytes(UTF_8)); + expect(zc.get(ZooUtil.getRoot(iid) + Constants.ZNAMESPACES + "/" + NSID + Constants.ZNAMESPACE_CONF + "/" + "foo")).andReturn("bar".getBytes(UTF_8)); + expect(zc.get(ZooUtil.getRoot(iid) + Constants.ZNAMESPACES + "/" + NSID + Constants.ZNAMESPACE_CONF + "/" + "ding")).andReturn("dong".getBytes(UTF_8)); replay(zc); c.getProperties(props, filter); assertEquals(2, props.size()); @@ -150,8 +149,8 @@ public class NamespaceConfigurationTest { public void testInvalidateCache() { // need to do a get so the accessor is created Property p = Property.INSTANCE_SECRET; - expect(zc.get(ZooUtil.getRoot(iid) + Constants.ZNAMESPACES + "/" + NSID + Constants.ZNAMESPACE_CONF + "/" + p.getKey())).andReturn( - "sekrit".getBytes(UTF_8)); + expect(zc.get(ZooUtil.getRoot(iid) + Constants.ZNAMESPACES + "/" + NSID + Constants.ZNAMESPACE_CONF + "/" + p.getKey())) + .andReturn("sekrit".getBytes(UTF_8)); zc.clear(); replay(zc); c.get(Property.INSTANCE_SECRET); http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/conf/TableConfigurationTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/TableConfigurationTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/TableConfigurationTest.java index 0ede571..68ee2b9 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/conf/TableConfigurationTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/conf/TableConfigurationTest.java @@ -83,8 +83,7 @@ public class TableConfigurationTest { @Test public void testGet_InZK() { Property p = Property.INSTANCE_SECRET; - expect(zc.get(ZooUtil.getRoot(iid) + Constants.ZTABLES + "/" + TID + Constants.ZTABLE_CONF + "/" + p.getKey())).andReturn( - "sekrit".getBytes(UTF_8)); + expect(zc.get(ZooUtil.getRoot(iid) + Constants.ZTABLES + "/" + TID + Constants.ZTABLE_CONF + "/" + p.getKey())).andReturn("sekrit".getBytes(UTF_8)); replay(zc); assertEquals("sekrit", c.get(Property.INSTANCE_SECRET)); } @@ -109,10 +108,8 @@ public class TableConfigurationTest { children.add("foo"); children.add("ding"); expect(zc.getChildren(ZooUtil.getRoot(iid) + Constants.ZTABLES + "/" + TID + Constants.ZTABLE_CONF)).andReturn(children); - expect(zc.get(ZooUtil.getRoot(iid) + Constants.ZTABLES + "/" + TID + Constants.ZTABLE_CONF + "/" + "foo")) - .andReturn("bar".getBytes(UTF_8)); - expect(zc.get(ZooUtil.getRoot(iid) + Constants.ZTABLES + "/" + TID + Constants.ZTABLE_CONF + "/" + "ding")).andReturn( - "dong".getBytes(UTF_8)); + expect(zc.get(ZooUtil.getRoot(iid) + Constants.ZTABLES + "/" + TID + Constants.ZTABLE_CONF + "/" + "foo")).andReturn("bar".getBytes(UTF_8)); + expect(zc.get(ZooUtil.getRoot(iid) + Constants.ZTABLES + "/" + TID + Constants.ZTABLE_CONF + "/" + "ding")).andReturn("dong".getBytes(UTF_8)); replay(zc); c.getProperties(props, filter); assertEquals(2, props.size()); @@ -136,8 +133,7 @@ public class TableConfigurationTest { public void testInvalidateCache() { // need to do a get so the accessor is created Property p = Property.INSTANCE_SECRET; - expect(zc.get(ZooUtil.getRoot(iid) + Constants.ZTABLES + "/" + TID + Constants.ZTABLE_CONF + "/" + p.getKey())).andReturn( - "sekrit".getBytes(UTF_8)); + expect(zc.get(ZooUtil.getRoot(iid) + Constants.ZTABLES + "/" + TID + Constants.ZTABLE_CONF + "/" + p.getKey())).andReturn("sekrit".getBytes(UTF_8)); zc.clear(); replay(zc); c.get(Property.INSTANCE_SECRET); http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/conf/ZooConfigurationFactoryTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/ZooConfigurationFactoryTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/ZooConfigurationFactoryTest.java index b54f3e7..b069163 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/conf/ZooConfigurationFactoryTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/conf/ZooConfigurationFactoryTest.java @@ -16,19 +16,20 @@ */ package org.apache.accumulo.server.conf; +import static org.easymock.EasyMock.createMock; +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.expectLastCall; +import static org.easymock.EasyMock.replay; +import static org.easymock.EasyMock.verify; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertSame; + import org.apache.accumulo.core.client.Instance; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.fate.zookeeper.ZooCache; import org.apache.accumulo.fate.zookeeper.ZooCacheFactory; import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertSame; -import static org.easymock.EasyMock.createMock; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.expectLastCall; -import static org.easymock.EasyMock.replay; -import static org.easymock.EasyMock.verify; public class ZooConfigurationFactoryTest { private Instance instance; http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/constraints/MetadataConstraintsTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/constraints/MetadataConstraintsTest.java b/server/base/src/test/java/org/apache/accumulo/server/constraints/MetadataConstraintsTest.java index d0b2e9e..7b6eec2 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/constraints/MetadataConstraintsTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/constraints/MetadataConstraintsTest.java @@ -28,26 +28,25 @@ import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily; import org.apache.accumulo.fate.zookeeper.TransactionWatcher.Arbitrator; -import org.apache.accumulo.server.constraints.MetadataConstraints; import org.apache.hadoop.io.Text; import org.apache.log4j.Level; import org.apache.log4j.Logger; import org.junit.Test; public class MetadataConstraintsTest { - + static class TestMetadataConstraints extends MetadataConstraints { @Override protected Arbitrator getArbitrator() { return new Arbitrator() { - + @Override public boolean transactionAlive(String type, long tid) throws Exception { if (tid == 9) throw new RuntimeException("txid 9 reserved for future use"); return tid == 5 || tid == 7; } - + @Override public boolean transactionComplete(String type, long tid) throws Exception { return tid != 5 && tid != 7; @@ -55,89 +54,89 @@ public class MetadataConstraintsTest { }; } } - + @Test public void testCheck() { Logger.getLogger(AccumuloConfiguration.class).setLevel(Level.ERROR); Mutation m = new Mutation(new Text("0;foo")); TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.put(m, new Value("1foo".getBytes())); - + MetadataConstraints mc = new MetadataConstraints(); - + List<Short> violations = mc.check(null, m); - + assertNotNull(violations); assertEquals(1, violations.size()); assertEquals(Short.valueOf((short) 3), violations.get(0)); - + m = new Mutation(new Text("0:foo")); TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.put(m, new Value("1poo".getBytes())); - + violations = mc.check(null, m); - + assertNotNull(violations); assertEquals(1, violations.size()); assertEquals(Short.valueOf((short) 4), violations.get(0)); - + m = new Mutation(new Text("0;foo")); m.put(new Text("bad_column_name"), new Text(""), new Value("e".getBytes())); - + violations = mc.check(null, m); - + assertNotNull(violations); assertEquals(1, violations.size()); assertEquals(Short.valueOf((short) 2), violations.get(0)); - + m = new Mutation(new Text("!!<")); TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.put(m, new Value("1poo".getBytes())); - + violations = mc.check(null, m); - + assertNotNull(violations); assertEquals(2, violations.size()); assertEquals(Short.valueOf((short) 4), violations.get(0)); assertEquals(Short.valueOf((short) 5), violations.get(1)); - + m = new Mutation(new Text("0;foo")); TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.put(m, new Value("".getBytes())); - + violations = mc.check(null, m); - + assertNotNull(violations); assertEquals(1, violations.size()); assertEquals(Short.valueOf((short) 6), violations.get(0)); - + m = new Mutation(new Text("0;foo")); TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.put(m, new Value("bar".getBytes())); - + violations = mc.check(null, m); - + assertEquals(null, violations); - + m = new Mutation(new Text("!0<")); TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.put(m, new Value("bar".getBytes())); - + violations = mc.check(null, m); - + assertEquals(null, violations); - + m = new Mutation(new Text("!1<")); TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.put(m, new Value("bar".getBytes())); - + violations = mc.check(null, m); - + assertNotNull(violations); assertEquals(1, violations.size()); assertEquals(Short.valueOf((short) 4), violations.get(0)); - + } - + @Test public void testBulkFileCheck() { MetadataConstraints mc = new TestMetadataConstraints(); Mutation m; List<Short> violations; - + // inactive txid m = new Mutation(new Text("0;foo")); m.put(TabletsSection.BulkFileColumnFamily.NAME, new Text("/someFile"), new Value("12345".getBytes())); @@ -146,7 +145,7 @@ public class MetadataConstraintsTest { assertNotNull(violations); assertEquals(1, violations.size()); assertEquals(Short.valueOf((short) 8), violations.get(0)); - + // txid that throws exception m = new Mutation(new Text("0;foo")); m.put(TabletsSection.BulkFileColumnFamily.NAME, new Text("/someFile"), new Value("9".getBytes())); @@ -155,14 +154,14 @@ public class MetadataConstraintsTest { assertNotNull(violations); assertEquals(1, violations.size()); assertEquals(Short.valueOf((short) 8), violations.get(0)); - + // active txid w/ file m = new Mutation(new Text("0;foo")); m.put(TabletsSection.BulkFileColumnFamily.NAME, new Text("/someFile"), new Value("5".getBytes())); m.put(DataFileColumnFamily.NAME, new Text("/someFile"), new Value("1,1".getBytes())); violations = mc.check(null, m); assertNull(violations); - + // active txid w/o file m = new Mutation(new Text("0;foo")); m.put(TabletsSection.BulkFileColumnFamily.NAME, new Text("/someFile"), new Value("5".getBytes())); @@ -170,7 +169,7 @@ public class MetadataConstraintsTest { assertNotNull(violations); assertEquals(1, violations.size()); assertEquals(Short.valueOf((short) 8), violations.get(0)); - + // two active txids w/ files m = new Mutation(new Text("0;foo")); m.put(TabletsSection.BulkFileColumnFamily.NAME, new Text("/someFile"), new Value("5".getBytes())); @@ -181,7 +180,7 @@ public class MetadataConstraintsTest { assertNotNull(violations); assertEquals(1, violations.size()); assertEquals(Short.valueOf((short) 8), violations.get(0)); - + // two files w/ one active txid m = new Mutation(new Text("0;foo")); m.put(TabletsSection.BulkFileColumnFamily.NAME, new Text("/someFile"), new Value("5".getBytes())); @@ -190,7 +189,7 @@ public class MetadataConstraintsTest { m.put(DataFileColumnFamily.NAME, new Text("/someFile2"), new Value("1,1".getBytes())); violations = mc.check(null, m); assertNull(violations); - + // two loaded w/ one active txid and one file m = new Mutation(new Text("0;foo")); m.put(TabletsSection.BulkFileColumnFamily.NAME, new Text("/someFile"), new Value("5".getBytes())); @@ -200,41 +199,41 @@ public class MetadataConstraintsTest { assertNotNull(violations); assertEquals(1, violations.size()); assertEquals(Short.valueOf((short) 8), violations.get(0)); - + // active txid, mutation that looks like split m = new Mutation(new Text("0;foo")); m.put(TabletsSection.BulkFileColumnFamily.NAME, new Text("/someFile"), new Value("5".getBytes())); TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value("/t1".getBytes())); violations = mc.check(null, m); assertNull(violations); - + // inactive txid, mutation that looks like split m = new Mutation(new Text("0;foo")); m.put(TabletsSection.BulkFileColumnFamily.NAME, new Text("/someFile"), new Value("12345".getBytes())); TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value("/t1".getBytes())); violations = mc.check(null, m); assertNull(violations); - + // active txid, mutation that looks like a load m = new Mutation(new Text("0;foo")); m.put(TabletsSection.BulkFileColumnFamily.NAME, new Text("/someFile"), new Value("5".getBytes())); m.put(TabletsSection.CurrentLocationColumnFamily.NAME, new Text("789"), new Value("127.0.0.1:9997".getBytes())); violations = mc.check(null, m); assertNull(violations); - + // inactive txid, mutation that looks like a load m = new Mutation(new Text("0;foo")); m.put(TabletsSection.BulkFileColumnFamily.NAME, new Text("/someFile"), new Value("12345".getBytes())); m.put(TabletsSection.CurrentLocationColumnFamily.NAME, new Text("789"), new Value("127.0.0.1:9997".getBytes())); violations = mc.check(null, m); assertNull(violations); - + // deleting a load flag m = new Mutation(new Text("0;foo")); m.putDelete(TabletsSection.BulkFileColumnFamily.NAME, new Text("/someFile")); violations = mc.check(null, m); assertNull(violations); - + } - + } http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/data/ServerMutationTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/data/ServerMutationTest.java b/server/base/src/test/java/org/apache/accumulo/server/data/ServerMutationTest.java index 0df27f1..52d3164 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/data/ServerMutationTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/data/ServerMutationTest.java @@ -30,50 +30,49 @@ import org.apache.hadoop.util.ReflectionUtils; import org.junit.Test; public class ServerMutationTest { - + @Test public void test() throws Exception { ServerMutation m = new ServerMutation(new Text("r1")); m.put(new Text("cf1"), new Text("cq1"), new Value("v1".getBytes())); m.put(new Text("cf2"), new Text("cq2"), 56, new Value("v2".getBytes())); m.setSystemTimestamp(42); - + List<ColumnUpdate> updates = m.getUpdates(); - + assertEquals(2, updates.size()); - + assertEquals("r1", new String(m.getRow())); ColumnUpdate cu = updates.get(0); - + assertEquals("cf1", new String(cu.getColumnFamily())); assertEquals("cq1", new String(cu.getColumnQualifier())); assertEquals("", new String(cu.getColumnVisibility())); assertFalse(cu.hasTimestamp()); assertEquals(42l, cu.getTimestamp()); - + ServerMutation m2 = new ServerMutation(); ReflectionUtils.copy(CachedConfiguration.getInstance(), m, m2); - + updates = m2.getUpdates(); - + assertEquals(2, updates.size()); assertEquals("r1", new String(m2.getRow())); - + cu = updates.get(0); assertEquals("cf1", new String(cu.getColumnFamily())); assertEquals("cq1", new String(cu.getColumnQualifier())); assertFalse(cu.hasTimestamp()); assertEquals(42l, cu.getTimestamp()); - + cu = updates.get(1); - + assertEquals("r1", new String(m2.getRow())); assertEquals("cf2", new String(cu.getColumnFamily())); assertEquals("cq2", new String(cu.getColumnQualifier())); assertTrue(cu.hasTimestamp()); assertEquals(56, cu.getTimestamp()); - - + } - + } http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/fs/FileRefTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/fs/FileRefTest.java b/server/base/src/test/java/org/apache/accumulo/server/fs/FileRefTest.java index 4ad0ea3..402f689 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/fs/FileRefTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/fs/FileRefTest.java @@ -23,9 +23,9 @@ import org.junit.Assert; import org.junit.Test; /** - * + * */ -public class FileRefTest{ +public class FileRefTest { private void testBadTableSuffix(String badPath) { try { http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/fs/FileTypeTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/fs/FileTypeTest.java b/server/base/src/test/java/org/apache/accumulo/server/fs/FileTypeTest.java index ad29c19..9ffe7d3 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/fs/FileTypeTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/fs/FileTypeTest.java @@ -22,7 +22,7 @@ import org.junit.Assert; import org.junit.Test; /** - * + * */ public class FileTypeTest { @Test http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/fs/ViewFSUtilsTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/fs/ViewFSUtilsTest.java b/server/base/src/test/java/org/apache/accumulo/server/fs/ViewFSUtilsTest.java index 52c70c0..cea35e2 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/fs/ViewFSUtilsTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/fs/ViewFSUtilsTest.java @@ -26,16 +26,16 @@ import org.junit.Assert; import org.junit.Test; /** - * + * */ public class ViewFSUtilsTest { - - private String[] shuffle(String ... inputs){ + + private String[] shuffle(String... inputs) { // code below will modify array Collections.shuffle(Arrays.asList(inputs)); return inputs; } - + @Test public void testDisjointMountPoints() throws IllegalArgumentException, IOException { if (ViewFSUtils.isViewFSSupported()) { @@ -47,8 +47,7 @@ public class ViewFSUtilsTest { String[] tablesDirs1 = shuffle("viewfs:///ns1/accumulo/tables", "viewfs:///ns2/accumulo/tables", "viewfs:///ns22/accumulo/tables", "viewfs:///ns/accumulo/tables"); - String[] tablesDirs2 = shuffle("viewfs:/ns1/accumulo/tables", "viewfs:/ns2/accumulo/tables", "viewfs:/ns22/accumulo/tables", - "viewfs:/ns/accumulo/tables"); + String[] tablesDirs2 = shuffle("viewfs:/ns1/accumulo/tables", "viewfs:/ns2/accumulo/tables", "viewfs:/ns22/accumulo/tables", "viewfs:/ns/accumulo/tables"); for (String ns : Arrays.asList("ns1", "ns2", "ns22", "ns")) { Path match = ViewFSUtils.matchingFileSystem(new Path("viewfs:/" + ns + "/bulk_import_01"), tablesDirs2, conf); http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeManagerImplTest.java ---------------------------------------------------------------------- diff --git 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 index 23829d1..3bf207a 100644 --- 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 @@ -21,7 +21,6 @@ import java.util.List; import org.apache.accumulo.core.conf.ConfigurationCopy; import org.apache.accumulo.core.conf.Property; -import org.apache.accumulo.server.fs.VolumeChooserEnvironment; import org.apache.accumulo.server.fs.VolumeManager.FileType; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.fs.Path; @@ -111,7 +110,7 @@ public class VolumeManagerImplTest { List<String> volumes = Arrays.asList("file://one/", "file://two/", "file://three/"); ConfigurationCopy conf = new ConfigurationCopy(); conf.set(INSTANCE_DFS_URI, volumes.get(0)); - conf.set(Property.INSTANCE_VOLUMES, StringUtils.join(volumes,",")); + conf.set(Property.INSTANCE_VOLUMES, StringUtils.join(volumes, ",")); conf.set(Property.GENERAL_VOLUME_CHOOSER, WrongVolumeChooser.class.getName()); VolumeManager vm = VolumeManagerImpl.get(conf); String choice = vm.choose(Optional.of("sometable"), volumes.toArray(new String[0])); http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeUtilTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeUtilTest.java b/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeUtilTest.java index 0013d04..9fcd3e0 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeUtilTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/fs/VolumeUtilTest.java @@ -33,7 +33,7 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; /** - * + * */ public class VolumeUtilTest { @@ -79,12 +79,12 @@ public class VolumeUtilTest { replacements.add(new Pair<Path,Path>(new Path("hdfs://nn1:9000/accumulo"), new Path("viewfs:/a"))); replacements.add(new Pair<Path,Path>(new Path("hdfs://nn2/accumulo"), new Path("viewfs:/b"))); - Assert.assertEquals("viewfs:/a/tables/t-00000/C000.rf", - VolumeUtil.switchVolume("hdfs://nn1/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements)); + Assert + .assertEquals("viewfs:/a/tables/t-00000/C000.rf", VolumeUtil.switchVolume("hdfs://nn1/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements)); Assert.assertEquals("viewfs:/a/tables/t-00000/C000.rf", VolumeUtil.switchVolume("hdfs://nn1:9000/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements)); - Assert.assertEquals("viewfs:/b/tables/t-00000/C000.rf", - VolumeUtil.switchVolume("hdfs://nn2/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements)); + Assert + .assertEquals("viewfs:/b/tables/t-00000/C000.rf", VolumeUtil.switchVolume("hdfs://nn2/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements)); Assert.assertNull(VolumeUtil.switchVolume("viewfs:/a/tables/t-00000/C000.rf", FileType.TABLE, replacements)); Assert.assertNull(VolumeUtil.switchVolume("file:/nn1/a/accumulo/tables/t-00000/C000.rf", FileType.TABLE, replacements)); @@ -205,9 +205,7 @@ public class VolumeUtilTest { FileType ft = FileType.TABLE; - Assert.assertEquals("file:/foo/v8/tables/+r/root_tablet", - VolumeUtil.switchVolume("file:/foo/v1/tables/+r/root_tablet", - ft, replacements)); + Assert.assertEquals("file:/foo/v8/tables/+r/root_tablet", VolumeUtil.switchVolume("file:/foo/v1/tables/+r/root_tablet", ft, replacements)); } private void writeFile(FileSystem fs, Path dir, String filename, String data) throws IOException { http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java b/server/base/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java index 4a45e99..cfdd5e9 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/iterators/MetadataBulkLoadFilterTest.java @@ -34,13 +34,12 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily; import org.apache.accumulo.core.util.ColumnFQ; import org.apache.accumulo.fate.zookeeper.TransactionWatcher.Arbitrator; -import org.apache.accumulo.server.iterators.MetadataBulkLoadFilter; import org.apache.hadoop.io.Text; import org.junit.Assert; import org.junit.Test; /** - * + * */ public class MetadataBulkLoadFilterTest { static class TestArbitrator implements Arbitrator { @@ -48,7 +47,7 @@ public class MetadataBulkLoadFilterTest { public boolean transactionAlive(String type, long tid) throws Exception { return tid == 5; } - + @Override public boolean transactionComplete(String type, long tid) throws Exception { if (tid == 9) @@ -56,19 +55,19 @@ public class MetadataBulkLoadFilterTest { return tid != 5 && tid != 7; } } - + static class TestMetadataBulkLoadFilter extends MetadataBulkLoadFilter { @Override protected Arbitrator getArbitrator() { return new TestArbitrator(); } } - + private static void put(TreeMap<Key,Value> tm, String row, ColumnFQ cfq, String val) { Key k = new Key(new Text(row), cfq.getColumnFamily(), cfq.getColumnQualifier()); tm.put(k, new Value(val.getBytes())); } - + private static void put(TreeMap<Key,Value> tm, String row, Text cf, String cq, String val) { Key k = new Key(new Text(row), cf, new Text(cq)); if (val == null) { @@ -77,12 +76,12 @@ public class MetadataBulkLoadFilterTest { } else tm.put(k, new Value(val.getBytes())); } - + @Test public void testBasic() throws IOException { TreeMap<Key,Value> tm1 = new TreeMap<Key,Value>(); TreeMap<Key,Value> expected = new TreeMap<Key,Value>(); - + // following should not be deleted by filter put(tm1, "2;m", TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN, "/t1"); put(tm1, "2;m", DataFileColumnFamily.NAME, "/t1/file1", "1,1"); @@ -95,50 +94,50 @@ public class MetadataBulkLoadFilterTest { put(tm1, "2<", TabletsSection.BulkFileColumnFamily.NAME, "/t2/file7", "7"); put(tm1, "2<", TabletsSection.BulkFileColumnFamily.NAME, "/t2/file8", "9"); put(tm1, "2<", TabletsSection.BulkFileColumnFamily.NAME, "/t2/fileC", null); - + expected.putAll(tm1); - + // the following should be deleted by filter put(tm1, "2;m", TabletsSection.BulkFileColumnFamily.NAME, "/t1/file5", "8"); put(tm1, "2<", TabletsSection.BulkFileColumnFamily.NAME, "/t2/file9", "8"); put(tm1, "2<", TabletsSection.BulkFileColumnFamily.NAME, "/t2/fileA", "2"); - + TestMetadataBulkLoadFilter iter = new TestMetadataBulkLoadFilter(); iter.init(new SortedMapIterator(tm1), new HashMap<String,String>(), new IteratorEnvironment() { - + @Override public SortedKeyValueIterator<Key,Value> reserveMapFileReader(String mapFileName) throws IOException { return null; } - + @Override public void registerSideChannel(SortedKeyValueIterator<Key,Value> iter) {} - + @Override public boolean isFullMajorCompaction() { return false; } - + @Override public IteratorScope getIteratorScope() { return IteratorScope.majc; } - + @Override public AccumuloConfiguration getConfig() { return null; } }); - + iter.seek(new Range(), new ArrayList<ByteSequence>(), false); - + TreeMap<Key,Value> actual = new TreeMap<Key,Value>(); - + while (iter.hasTop()) { actual.put(iter.getTopKey(), iter.getTopValue()); iter.next(); } - + Assert.assertEquals(expected, actual); } } http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancerTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancerTest.java index 2dff03d..1b691c7 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancerTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancerTest.java @@ -33,7 +33,6 @@ import org.apache.accumulo.core.data.KeyExtent; import org.apache.accumulo.core.master.thrift.TableInfo; import org.apache.accumulo.core.master.thrift.TabletServerStatus; import org.apache.accumulo.core.tabletserver.thrift.TabletStats; -import org.apache.accumulo.server.master.balancer.ChaoticLoadBalancer; import org.apache.accumulo.server.master.state.TServerInstance; import org.apache.accumulo.server.master.state.TabletMigration; import org.apache.hadoop.io.Text; @@ -43,10 +42,10 @@ import org.junit.Test; import com.google.common.net.HostAndPort; public class ChaoticLoadBalancerTest { - + class FakeTServer { List<KeyExtent> extents = new ArrayList<KeyExtent>(); - + TabletServerStatus getStatus(TServerInstance server) { TabletServerStatus result = new TabletServerStatus(); result.tableMap = new HashMap<String,TableInfo>(); @@ -63,11 +62,11 @@ public class ChaoticLoadBalancerTest { return result; } } - + Map<TServerInstance,FakeTServer> servers = new HashMap<TServerInstance,FakeTServer>(); - + class TestChaoticLoadBalancer extends ChaoticLoadBalancer { - + @Override public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, String table) throws ThriftSecurityException, TException { List<TabletStats> result = new ArrayList<TabletStats>(); @@ -79,7 +78,7 @@ public class ChaoticLoadBalancerTest { return result; } } - + @Test public void testAssignMigrations() { servers.clear(); @@ -99,20 +98,20 @@ public class ChaoticLoadBalancerTest { metadataTable.put(makeExtent(table, "d", "c"), null); metadataTable.put(makeExtent(table, "e", "d"), null); metadataTable.put(makeExtent(table, null, "e"), null); - + TestChaoticLoadBalancer balancer = new TestChaoticLoadBalancer(); - + SortedMap<TServerInstance,TabletServerStatus> current = new TreeMap<TServerInstance,TabletServerStatus>(); for (Entry<TServerInstance,FakeTServer> entry : servers.entrySet()) { current.put(entry.getKey(), entry.getValue().getStatus(entry.getKey())); } - + Map<KeyExtent,TServerInstance> assignments = new HashMap<KeyExtent,TServerInstance>(); balancer.getAssignments(getAssignments(servers), metadataTable, assignments); - + assertEquals(assignments.size(), metadataTable.size()); } - + SortedMap<TServerInstance,TabletServerStatus> getAssignments(Map<TServerInstance,FakeTServer> servers) { SortedMap<TServerInstance,TabletServerStatus> result = new TreeMap<TServerInstance,TabletServerStatus>(); for (Entry<TServerInstance,FakeTServer> entry : servers.entrySet()) { @@ -120,7 +119,7 @@ public class ChaoticLoadBalancerTest { } return result; } - + @Test public void testUnevenAssignment() { servers.clear(); @@ -146,22 +145,22 @@ public class ChaoticLoadBalancerTest { first.getValue().extents.add(makeExtent("newTable", "i", null)); TestChaoticLoadBalancer balancer = new TestChaoticLoadBalancer(); Set<KeyExtent> migrations = Collections.emptySet(); - + // Just want to make sure it gets some migrations, randomness prevents guarantee of a defined amount, or even expected amount List<TabletMigration> migrationsOut = new ArrayList<TabletMigration>(); while (migrationsOut.size() != 0) { balancer.balance(getAssignments(servers), migrations, migrationsOut); } } - + private static KeyExtent makeExtent(String table, String end, String prev) { return new KeyExtent(new Text(table), toText(end), toText(prev)); } - + private static Text toText(String value) { if (value != null) return new Text(value); return null; } - + } http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java index 9f99b1c..aee15bc 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancerTest.java @@ -36,7 +36,6 @@ import org.apache.accumulo.core.data.KeyExtent; import org.apache.accumulo.core.master.thrift.TableInfo; import org.apache.accumulo.core.master.thrift.TabletServerStatus; import org.apache.accumulo.core.tabletserver.thrift.TabletStats; -import org.apache.accumulo.server.master.balancer.DefaultLoadBalancer; import org.apache.accumulo.server.master.state.TServerInstance; import org.apache.accumulo.server.master.state.TabletMigration; import org.apache.hadoop.io.Text; @@ -47,10 +46,10 @@ import org.junit.Test; import com.google.common.net.HostAndPort; public class DefaultLoadBalancerTest { - + class FakeTServer { List<KeyExtent> extents = new ArrayList<KeyExtent>(); - + TabletServerStatus getStatus(TServerInstance server) { TabletServerStatus result = new TabletServerStatus(); result.tableMap = new HashMap<String,TableInfo>(); @@ -67,12 +66,12 @@ public class DefaultLoadBalancerTest { return result; } } - + Map<TServerInstance,FakeTServer> servers = new HashMap<TServerInstance,FakeTServer>(); - Map<KeyExtent, TServerInstance> last = new HashMap<KeyExtent, TServerInstance>(); - + Map<KeyExtent,TServerInstance> last = new HashMap<KeyExtent,TServerInstance>(); + class TestDefaultLoadBalancer extends DefaultLoadBalancer { - + @Override public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, String table) throws ThriftSecurityException, TException { List<TabletStats> result = new ArrayList<TabletStats>(); @@ -84,13 +83,13 @@ public class DefaultLoadBalancerTest { return result; } } - + @Before public void setUp() { last.clear(); servers.clear(); } - + @Test public void testAssignMigrations() { servers.put(new TServerInstance(HostAndPort.fromParts("127.0.0.1", 1234), "a"), new FakeTServer()); @@ -110,42 +109,42 @@ public class DefaultLoadBalancerTest { metadataTable.add(makeExtent(table, "e", "d")); metadataTable.add(makeExtent(table, null, "e")); Collections.sort(metadataTable); - + TestDefaultLoadBalancer balancer = new TestDefaultLoadBalancer(); - + SortedMap<TServerInstance,TabletServerStatus> current = new TreeMap<TServerInstance,TabletServerStatus>(); for (Entry<TServerInstance,FakeTServer> entry : servers.entrySet()) { current.put(entry.getKey(), entry.getValue().getStatus(entry.getKey())); } assignTablets(metadataTable, servers, current, balancer); - + // Verify that the counts on the tables are correct Map<String,Integer> expectedCounts = new HashMap<String,Integer>(); expectedCounts.put("t1", 1); expectedCounts.put("t2", 1); expectedCounts.put("t3", 2); checkBalance(metadataTable, servers, expectedCounts); - + // Rebalance once for (Entry<TServerInstance,FakeTServer> entry : servers.entrySet()) { current.put(entry.getKey(), entry.getValue().getStatus(entry.getKey())); } - + // Nothing should happen, we are balanced ArrayList<TabletMigration> out = new ArrayList<TabletMigration>(); balancer.getMigrations(current, out); assertEquals(out.size(), 0); - + // Take down a tabletServer TServerInstance first = current.keySet().iterator().next(); current.remove(first); FakeTServer remove = servers.remove(first); - + // reassign offline extents assignTablets(remove.extents, servers, current, balancer); checkBalance(metadataTable, servers, null); } - + private void assignTablets(List<KeyExtent> metadataTable, Map<TServerInstance,FakeTServer> servers, SortedMap<TServerInstance,TabletServerStatus> status, TestDefaultLoadBalancer balancer) { // Assign tablets @@ -157,7 +156,7 @@ public class DefaultLoadBalancerTest { last.put(extent, assignment); } } - + SortedMap<TServerInstance,TabletServerStatus> getAssignments(Map<TServerInstance,FakeTServer> servers) { SortedMap<TServerInstance,TabletServerStatus> result = new TreeMap<TServerInstance,TabletServerStatus>(); for (Entry<TServerInstance,FakeTServer> entry : servers.entrySet()) { @@ -165,7 +164,7 @@ public class DefaultLoadBalancerTest { } return result; } - + @Test public void testUnevenAssignment() { for (char c : "abcdefghijklmnopqrstuvwxyz".toCharArray()) { @@ -205,7 +204,7 @@ public class DefaultLoadBalancerTest { } assertEquals(8, moved); } - + @Test public void testUnevenAssignment2() { // make 26 servers @@ -230,7 +229,7 @@ public class DefaultLoadBalancerTest { for (int i = 0; i < 10; i++) { shortServer.getValue().extents.add(makeExtent("s" + i, null, null)); } - + TestDefaultLoadBalancer balancer = new TestDefaultLoadBalancer(); Set<KeyExtent> migrations = Collections.emptySet(); int moved = 0; @@ -251,7 +250,7 @@ public class DefaultLoadBalancerTest { // average is 58, with 2 at 59: we need 48 more moved to the short server assertEquals(48, moved); } - + private void checkBalance(List<KeyExtent> metadataTable, Map<TServerInstance,FakeTServer> servers, Map<String,Integer> expectedCounts) { // Verify they are spread evenly over the cluster int average = metadataTable.size() / servers.size(); @@ -262,7 +261,7 @@ public class DefaultLoadBalancerTest { if (diff > 1) fail("average number of tablets is " + average + " but a server has " + server.extents.size()); } - + if (expectedCounts != null) { for (FakeTServer server : servers.values()) { Map<String,Integer> counts = new HashMap<String,Integer>(); @@ -278,15 +277,15 @@ public class DefaultLoadBalancerTest { } } } - + private static KeyExtent makeExtent(String table, String end, String prev) { return new KeyExtent(new Text(table), toText(end), toText(prev)); } - + private static Text toText(String value) { if (value != null) return new Text(value); return null; } - + } http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java index f107759..8e68d84 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java @@ -48,11 +48,11 @@ import org.junit.Test; import com.google.common.net.HostAndPort; public class TableLoadBalancerTest { - + static private TServerInstance mkts(String address, String session) throws Exception { return new TServerInstance(HostAndPort.fromParts(address, 1234), session); } - + static private TabletServerStatus status(Object... config) { TabletServerStatus result = new TabletServerStatus(); result.tableMap = new HashMap<String,TableInfo>(); @@ -70,11 +70,11 @@ public class TableLoadBalancerTest { } return result; } - + static MockInstance instance = new MockInstance("mockamatic"); - + static SortedMap<TServerInstance,TabletServerStatus> state; - + static List<TabletStats> generateFakeTablets(TServerInstance tserver, String tableId) { List<TabletStats> result = new ArrayList<TabletStats>(); TabletServerStatus tableInfo = state.get(tserver); @@ -87,39 +87,39 @@ public class TableLoadBalancerTest { } return result; } - + static class DefaultLoadBalancer extends org.apache.accumulo.server.master.balancer.DefaultLoadBalancer { - + public DefaultLoadBalancer(String table) { super(table); } - + @Override public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, String tableId) throws ThriftSecurityException, TException { return generateFakeTablets(tserver, tableId); } } - + // ugh... so wish I had provided mock objects to the LoadBalancer in the master static class TableLoadBalancer extends org.apache.accumulo.server.master.balancer.TableLoadBalancer { - + TableLoadBalancer() { super(); } - + // use our new classname to test class loading @Override protected String getLoadBalancerClassNameForTable(String table) { return DefaultLoadBalancer.class.getName(); } - + // we don't have real tablet servers to ask: invent some online tablets @Override public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, String tableId) throws ThriftSecurityException, TException { return generateFakeTablets(tserver, tableId); } } - + @Test public void test() throws Exception { Connector c = instance.getConnector("user", new PasswordToken("pass")); @@ -143,14 +143,14 @@ public class TableLoadBalancerTest { state = new TreeMap<TServerInstance,TabletServerStatus>(); TServerInstance svr = mkts("10.0.0.1", "0x01020304"); state.put(svr, status(t1Id, 10, t2Id, 10, t3Id, 10)); - + Set<KeyExtent> migrations = Collections.emptySet(); List<TabletMigration> migrationsOut = new ArrayList<TabletMigration>(); TableLoadBalancer tls = new TableLoadBalancer(); tls.init(confFactory); tls.balance(state, migrations, migrationsOut); Assert.assertEquals(0, migrationsOut.size()); - + state.put(mkts("10.0.0.2", "0x02030405"), status()); tls = new TableLoadBalancer(); tls.init(confFactory); @@ -171,5 +171,5 @@ public class TableLoadBalancerTest { Assert.assertEquals(5, moved.intValue()); } } - + } http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/master/state/MergeInfoTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/state/MergeInfoTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/state/MergeInfoTest.java index 58484af..7c26f85 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/master/state/MergeInfoTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/master/state/MergeInfoTest.java @@ -16,19 +16,25 @@ */ package org.apache.accumulo.server.master.state; +import static org.easymock.EasyMock.createMock; +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.expectLastCall; +import static org.easymock.EasyMock.replay; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; + import org.apache.accumulo.core.data.KeyExtent; import org.apache.hadoop.io.Text; import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.*; -import static org.easymock.EasyMock.createMock; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.expectLastCall; -import static org.easymock.EasyMock.replay; public class MergeInfoTest { private KeyExtent keyExtent; http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/master/state/TabletLocationStateTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/state/TabletLocationStateTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/state/TabletLocationStateTest.java index c1f312d..a7b673b 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/master/state/TabletLocationStateTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/master/state/TabletLocationStateTest.java @@ -16,17 +16,23 @@ */ package org.apache.accumulo.server.master.state; +import static org.easymock.EasyMock.createMock; +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.replay; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + import java.util.Collection; import java.util.Set; + import org.apache.accumulo.core.data.KeyExtent; import org.apache.hadoop.io.Text; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; -import static org.junit.Assert.*; -import static org.easymock.EasyMock.createMock; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.replay; public class TabletLocationStateTest { private static final Collection<String> innerWalogs = new java.util.HashSet<String>(); http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java b/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java index dbad326..1ca3e8d 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java @@ -16,8 +16,21 @@ */ package org.apache.accumulo.server.problems; +import static org.easymock.EasyMock.aryEq; +import static org.easymock.EasyMock.createMock; +import static org.easymock.EasyMock.eq; +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.replay; +import static org.easymock.EasyMock.verify; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; + import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.Instance; import org.apache.accumulo.core.util.Encoding; @@ -28,13 +41,6 @@ import org.apache.accumulo.server.zookeeper.ZooReaderWriter; import org.apache.hadoop.io.Text; import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.*; -import static org.easymock.EasyMock.aryEq; -import static org.easymock.EasyMock.createMock; -import static org.easymock.EasyMock.eq; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.replay; -import static org.easymock.EasyMock.verify; public class ProblemReportTest { private static final String TABLE = "table"; http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java b/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java index a9801b0..74e5455 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java @@ -177,8 +177,8 @@ public class StatusCombinerTest { @Test public void commutativeWithMultipleUpdates() { - Status newFile = StatusUtil.fileCreated(100), update1 = StatusUtil.ingestedUntil(100), update2 = StatusUtil.ingestedUntil(200), repl1 = StatusUtil.replicated(50), repl2 = StatusUtil - .replicated(150); + Status newFile = StatusUtil.fileCreated(100), update1 = StatusUtil.ingestedUntil(100), update2 = StatusUtil.ingestedUntil(200), repl1 = StatusUtil + .replicated(50), repl2 = StatusUtil.replicated(150); Status order1 = combiner.typedReduce(key, Arrays.asList(newFile, update1, repl1, update2, repl2).iterator()); http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/replication/ZooKeeperInitializationTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/replication/ZooKeeperInitializationTest.java b/server/base/src/test/java/org/apache/accumulo/server/replication/ZooKeeperInitializationTest.java index 25e7bc8..5604242 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/replication/ZooKeeperInitializationTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/replication/ZooKeeperInitializationTest.java @@ -27,7 +27,7 @@ import org.apache.accumulo.server.zookeeper.ZooReaderWriter; import org.junit.Test; /** - * + * */ public class ZooKeeperInitializationTest { http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java b/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java index 4202a7e..01ff9ac 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java @@ -32,10 +32,10 @@ import org.junit.BeforeClass; import org.junit.Test; /** - * + * */ public class SystemCredentialsTest { - + private static MockInstance inst; @BeforeClass @@ -54,7 +54,7 @@ public class SystemCredentialsTest { testInstanceVersion.createNewFile(); } } - + /** * This is a test to ensure the string literal in {@link ConnectorImpl#ConnectorImpl(org.apache.accumulo.core.client.impl.ClientContext)} is kept up-to-date * if we move the {@link SystemToken}<br/> @@ -65,7 +65,7 @@ public class SystemCredentialsTest { assertEquals("org.apache.accumulo.server.security.SystemCredentials$SystemToken", SystemToken.class.getName()); assertEquals(SystemCredentials.get(inst).getToken().getClass(), SystemToken.class); } - + @Test public void testSystemCredentials() { Credentials a = SystemCredentials.get(inst); http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java b/server/base/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java index 2f01e6e..eb5e2f3 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/security/handler/ZKAuthenticatorTest.java @@ -19,31 +19,31 @@ package org.apache.accumulo.server.security.handler; import java.util.Set; import java.util.TreeSet; +import junit.framework.TestCase; + import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.security.SystemPermission; import org.apache.accumulo.core.security.TablePermission; import org.apache.accumulo.core.util.ByteArraySet; - -import junit.framework.TestCase; import org.apache.log4j.Logger; public class ZKAuthenticatorTest extends TestCase { private static final Logger log = Logger.getLogger(ZKAuthenticatorTest.class); - + public void testPermissionIdConversions() { for (SystemPermission s : SystemPermission.values()) assertTrue(s.equals(SystemPermission.getPermissionById(s.getId()))); - + for (TablePermission s : TablePermission.values()) assertTrue(s.equals(TablePermission.getPermissionById(s.getId()))); } - + public void testAuthorizationConversion() { ByteArraySet auths = new ByteArraySet(); for (int i = 0; i < 300; i += 3) auths.add(Integer.toString(i).getBytes()); - + Authorizations converted = new Authorizations(auths); byte[] test = ZKSecurityTool.convertAuthorizations(converted); Authorizations test2 = ZKSecurityTool.convertAuthorizations(test); @@ -52,29 +52,29 @@ public class ZKAuthenticatorTest extends TestCase { assertTrue(test2.contains(s)); } } - + public void testSystemConversion() { Set<SystemPermission> perms = new TreeSet<SystemPermission>(); for (SystemPermission s : SystemPermission.values()) perms.add(s); - + Set<SystemPermission> converted = ZKSecurityTool.convertSystemPermissions(ZKSecurityTool.convertSystemPermissions(perms)); assertTrue(perms.size() == converted.size()); for (SystemPermission s : perms) assertTrue(converted.contains(s)); } - + public void testTableConversion() { Set<TablePermission> perms = new TreeSet<TablePermission>(); for (TablePermission s : TablePermission.values()) perms.add(s); - + Set<TablePermission> converted = ZKSecurityTool.convertTablePermissions(ZKSecurityTool.convertTablePermissions(perms)); assertTrue(perms.size() == converted.size()); for (TablePermission s : perms) assertTrue(converted.contains(s)); } - + public void testEncryption() { byte[] rawPass = "myPassword".getBytes(); byte[] storedBytes; http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/tablets/MillisTimeTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/tablets/MillisTimeTest.java b/server/base/src/test/java/org/apache/accumulo/server/tablets/MillisTimeTest.java index 15c8465..3ee220e 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/tablets/MillisTimeTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/tablets/MillisTimeTest.java @@ -16,18 +16,20 @@ */ package org.apache.accumulo.server.tablets; +import static org.easymock.EasyMock.anyLong; +import static org.easymock.EasyMock.createMock; +import static org.easymock.EasyMock.replay; +import static org.easymock.EasyMock.verify; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.List; + import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.server.data.ServerMutation; import org.apache.accumulo.server.tablets.TabletTime.MillisTime; - -import java.util.List; import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.*; -import static org.easymock.EasyMock.anyLong; -import static org.easymock.EasyMock.createMock; -import static org.easymock.EasyMock.replay; -import static org.easymock.EasyMock.verify; public class MillisTimeTest { private static final long TIME = 1234L; http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/tablets/TabletTimeTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/tablets/TabletTimeTest.java b/server/base/src/test/java/org/apache/accumulo/server/tablets/TabletTimeTest.java index 38cca85..46068e3 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/tablets/TabletTimeTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/tablets/TabletTimeTest.java @@ -16,17 +16,18 @@ */ package org.apache.accumulo.server.tablets; +import static org.easymock.EasyMock.createMock; +import static org.easymock.EasyMock.replay; +import static org.easymock.EasyMock.verify; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + import org.apache.accumulo.core.client.admin.TimeType; import org.apache.accumulo.server.data.ServerMutation; import org.apache.accumulo.server.tablets.TabletTime.LogicalTime; import org.apache.accumulo.server.tablets.TabletTime.MillisTime; - import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.*; -import static org.easymock.EasyMock.createMock; -import static org.easymock.EasyMock.replay; -import static org.easymock.EasyMock.verify; public class TabletTimeTest { private static final long TIME = 1234L; http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bc67602/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java ---------------------------------------------------------------------- diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java index f95c0f1..ab799ec 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java @@ -16,8 +16,11 @@ */ package org.apache.accumulo.server.util; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; + import org.junit.Test; -import static org.junit.Assert.*; public class AdminCommandsTest { @Test