From ff128a8ad5de83bc55b8cec80d7775e61816e4e6 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Mon, 27 Feb 2012 18:07:29 -0800 Subject: [PATCH 001/209] TimeBasedInconsistency resolover bug fix --- .../TimeBasedInconsistencyResolver.java | 11 +- .../voldemort/versioning/VectorClock.java | 32 +++++ .../ChainedInconsistencyResolverTest.java | 123 ++++++++++++++++++ 3 files changed, 163 insertions(+), 3 deletions(-) create mode 100644 test/unit/voldemort/versioning/ChainedInconsistencyResolverTest.java diff --git a/src/java/voldemort/versioning/TimeBasedInconsistencyResolver.java b/src/java/voldemort/versioning/TimeBasedInconsistencyResolver.java index 151dd69dfb..0b5560dc89 100644 --- a/src/java/voldemort/versioning/TimeBasedInconsistencyResolver.java +++ b/src/java/voldemort/versioning/TimeBasedInconsistencyResolver.java @@ -28,26 +28,31 @@ public class TimeBasedInconsistencyResolver implements InconsistencyResolver> { public List> resolveConflicts(List> items) { + List maxClock = null; if(items.size() <= 1) { return items; } else { Versioned max = items.get(0); long maxTime = ((VectorClock) items.get(0).getVersion()).getTimestamp(); + maxClock = ((VectorClock) items.get(0).getVersion()).getEntries(); for(Versioned versioned: items) { VectorClock clock = (VectorClock) versioned.getVersion(); if(clock.getTimestamp() > maxTime) { max = versioned; maxTime = ((VectorClock) versioned.getVersion()).getTimestamp(); } + maxClock = VectorClock.maxClockList(maxClock, clock.getEntries()); } - return Collections.singletonList(max); + Versioned maxTimeClockVersioned = new Versioned(max.getValue(), + new VectorClock(maxClock, maxTime)); + return Collections.singletonList(maxTimeClockVersioned); } } - @Override public boolean equals(Object o) { - if (this == o) return true; + if(this == o) + return true; return (o != null && getClass() == o.getClass()); } diff --git a/src/java/voldemort/versioning/VectorClock.java b/src/java/voldemort/versioning/VectorClock.java index db35c1fc88..3dce7e55e8 100644 --- a/src/java/voldemort/versioning/VectorClock.java +++ b/src/java/voldemort/versioning/VectorClock.java @@ -343,4 +343,36 @@ public List getEntries() { return this.versions; } + public static List maxClockList(List v1, List v2) { + if(v1 == null || v2 == null) + throw new IllegalArgumentException("Can't compare null vector clocks!"); + int p1 = 0; + int p2 = 0; + List maxClockList = new ArrayList(); + + /* Create a ClockEntry list with the maximum clocks of v1 and v2 */ + while(p1 < v1.size() && p2 < v2.size()) { + ClockEntry ver1 = v1.get(p1); + ClockEntry ver2 = v2.get(p2); + if(ver1.getNodeId() < ver2.getNodeId()) { + maxClockList.add(ver2); + } else { + maxClockList.add(ver1); + } + p1++; + p2++; + } + + /* Insert remaining clocks */ + while(p1 < v1.size()) { + maxClockList.add(v1.get(p1)); + p1++; + } + while(p2 < v2.size()) { + maxClockList.add(v1.get(p2)); + p2++; + } + + return maxClockList; + } } diff --git a/test/unit/voldemort/versioning/ChainedInconsistencyResolverTest.java b/test/unit/voldemort/versioning/ChainedInconsistencyResolverTest.java new file mode 100644 index 0000000000..8460a33133 --- /dev/null +++ b/test/unit/voldemort/versioning/ChainedInconsistencyResolverTest.java @@ -0,0 +1,123 @@ +package voldemort.versioning; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +import junit.framework.TestCase; + +import org.junit.After; +import org.junit.Test; + +import voldemort.ServerTestUtils; +import voldemort.TestUtils; +import voldemort.VoldemortTestConstants; +import voldemort.client.ClientConfig; +import voldemort.client.SocketStoreClientFactory; +import voldemort.client.StoreClient; +import voldemort.client.StoreClientFactory; +import voldemort.client.protocol.RequestFormatType; +import voldemort.cluster.Cluster; +import voldemort.cluster.Node; +import voldemort.server.VoldemortServer; +import voldemort.store.Store; +import voldemort.store.socket.SocketStoreFactory; +import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; +import voldemort.utils.ByteArray; + +/* + * Test to ensure that the TimeBasedInconsistencyResolver returns a max Vector + * clock with the max timestamp. + */ +public class ChainedInconsistencyResolverTest extends TestCase { + + private static final String KEY = "XYZ"; + private InconsistencyResolver> vectorResolver, timeResolver, chainedResolver; + private Versioned v1, v2, v3; + + private Node node; + private Cluster cluster; + private StoreClient defaultStoreClient; + private Store socketStore; + private List servers; + private final SocketStoreFactory socketStoreFactory = new ClientRequestExecutorPool(2, + 10000, + 100000, + 32 * 1024); + private final boolean useNio = false; + private static final String STORES_XML = "test/common/voldemort/config/single-store.xml"; + + @SuppressWarnings("unchecked") + @Override + public void setUp() throws IOException { + VoldemortTestConstants.getSingleStoreDefinitionsXml(); + this.cluster = ServerTestUtils.getLocalCluster(2); + this.node = cluster.getNodes().iterator().next(); + String bootstrapUrl = "tcp://" + node.getHost() + ":" + node.getSocketPort(); + StoreClientFactory storeClientFactory = new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrl)); + servers = new ArrayList(); + servers.add(ServerTestUtils.startVoldemortServer(socketStoreFactory, + ServerTestUtils.createServerConfig(useNio, + 0, + TestUtils.createTempDir() + .getAbsolutePath(), + null, + STORES_XML, + new Properties()), + cluster)); + servers.add(ServerTestUtils.startVoldemortServer(socketStoreFactory, + ServerTestUtils.createServerConfig(useNio, + 1, + TestUtils.createTempDir() + .getAbsolutePath(), + null, + STORES_XML, + new Properties()), + cluster)); + + vectorResolver = new VectorClockInconsistencyResolver(); + timeResolver = new TimeBasedInconsistencyResolver(); + chainedResolver = new ChainedResolver>(vectorResolver, timeResolver); + v1 = getVersioned(0, 0, 1, 1, 1, 1, 1); + v2 = getVersioned(0, 0, 0, 1, 1, 1, 1); + + defaultStoreClient = storeClientFactory.getStoreClient("test"); + socketStore = ServerTestUtils.getSocketStore(socketStoreFactory, + "test", + node.getSocketPort(), + RequestFormatType.VOLDEMORT_V1); + } + + @Override + @After + public void tearDown() throws Exception { + socketStore.close(); + socketStoreFactory.close(); + } + + private Versioned getVersioned(int... nodes) { + return new Versioned("my-value", TestUtils.getClock(nodes)); + } + + @SuppressWarnings("unchecked") + @Test + public void testVersionedPut() { + defaultStoreClient.put(KEY, v1); + defaultStoreClient.put(KEY, v2); + Versioned res = defaultStoreClient.get(KEY); + defaultStoreClient.put(KEY, res); + List> resList = socketStore.get(new ByteArray(KEY.getBytes()), null); + assertEquals(1, resList.size()); + } + + @SuppressWarnings("unchecked") + @Test + public void testNormalPut() { + defaultStoreClient.put(KEY, v1); + defaultStoreClient.put(KEY, v2); + defaultStoreClient.put(KEY, "my-value2"); + List> resList = socketStore.get(new ByteArray(KEY.getBytes()), null); + assertEquals(1, resList.size()); + } +} From dea43e29e18fbe96bc4995477cc533919de081e0 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Mon, 27 Feb 2012 19:11:38 -0800 Subject: [PATCH 002/209] Used existing merge function in VectorClock and code cleanup. --- .../TimeBasedInconsistencyResolver.java | 8 ++--- .../voldemort/versioning/VectorClock.java | 33 ------------------- .../ChainedInconsistencyResolverTest.java | 14 ++------ 3 files changed, 6 insertions(+), 49 deletions(-) diff --git a/src/java/voldemort/versioning/TimeBasedInconsistencyResolver.java b/src/java/voldemort/versioning/TimeBasedInconsistencyResolver.java index 0b5560dc89..71d986189d 100644 --- a/src/java/voldemort/versioning/TimeBasedInconsistencyResolver.java +++ b/src/java/voldemort/versioning/TimeBasedInconsistencyResolver.java @@ -28,23 +28,21 @@ public class TimeBasedInconsistencyResolver implements InconsistencyResolver> { public List> resolveConflicts(List> items) { - List maxClock = null; if(items.size() <= 1) { return items; } else { Versioned max = items.get(0); long maxTime = ((VectorClock) items.get(0).getVersion()).getTimestamp(); - maxClock = ((VectorClock) items.get(0).getVersion()).getEntries(); + VectorClock maxClock = ((VectorClock) items.get(0).getVersion()); for(Versioned versioned: items) { VectorClock clock = (VectorClock) versioned.getVersion(); if(clock.getTimestamp() > maxTime) { max = versioned; maxTime = ((VectorClock) versioned.getVersion()).getTimestamp(); } - maxClock = VectorClock.maxClockList(maxClock, clock.getEntries()); + maxClock = maxClock.merge(clock); } - Versioned maxTimeClockVersioned = new Versioned(max.getValue(), - new VectorClock(maxClock, maxTime)); + Versioned maxTimeClockVersioned = new Versioned(max.getValue(), maxClock); return Collections.singletonList(maxTimeClockVersioned); } } diff --git a/src/java/voldemort/versioning/VectorClock.java b/src/java/voldemort/versioning/VectorClock.java index 3dce7e55e8..ff0f4e0cad 100644 --- a/src/java/voldemort/versioning/VectorClock.java +++ b/src/java/voldemort/versioning/VectorClock.java @@ -342,37 +342,4 @@ public long getTimestamp() { public List getEntries() { return this.versions; } - - public static List maxClockList(List v1, List v2) { - if(v1 == null || v2 == null) - throw new IllegalArgumentException("Can't compare null vector clocks!"); - int p1 = 0; - int p2 = 0; - List maxClockList = new ArrayList(); - - /* Create a ClockEntry list with the maximum clocks of v1 and v2 */ - while(p1 < v1.size() && p2 < v2.size()) { - ClockEntry ver1 = v1.get(p1); - ClockEntry ver2 = v2.get(p2); - if(ver1.getNodeId() < ver2.getNodeId()) { - maxClockList.add(ver2); - } else { - maxClockList.add(ver1); - } - p1++; - p2++; - } - - /* Insert remaining clocks */ - while(p1 < v1.size()) { - maxClockList.add(v1.get(p1)); - p1++; - } - while(p2 < v2.size()) { - maxClockList.add(v1.get(p2)); - p2++; - } - - return maxClockList; - } } diff --git a/test/unit/voldemort/versioning/ChainedInconsistencyResolverTest.java b/test/unit/voldemort/versioning/ChainedInconsistencyResolverTest.java index 8460a33133..f557beeb65 100644 --- a/test/unit/voldemort/versioning/ChainedInconsistencyResolverTest.java +++ b/test/unit/voldemort/versioning/ChainedInconsistencyResolverTest.java @@ -33,8 +33,7 @@ public class ChainedInconsistencyResolverTest extends TestCase { private static final String KEY = "XYZ"; - private InconsistencyResolver> vectorResolver, timeResolver, chainedResolver; - private Versioned v1, v2, v3; + private Versioned v1, v2; private Node node; private Cluster cluster; @@ -48,7 +47,6 @@ public class ChainedInconsistencyResolverTest extends TestCase { private final boolean useNio = false; private static final String STORES_XML = "test/common/voldemort/config/single-store.xml"; - @SuppressWarnings("unchecked") @Override public void setUp() throws IOException { VoldemortTestConstants.getSingleStoreDefinitionsXml(); @@ -76,12 +74,8 @@ public void setUp() throws IOException { new Properties()), cluster)); - vectorResolver = new VectorClockInconsistencyResolver(); - timeResolver = new TimeBasedInconsistencyResolver(); - chainedResolver = new ChainedResolver>(vectorResolver, timeResolver); - v1 = getVersioned(0, 0, 1, 1, 1, 1, 1); - v2 = getVersioned(0, 0, 0, 1, 1, 1, 1); - + v1 = getVersioned(0, 1, 1, 1, 1, 1); + v2 = getVersioned(0, 0, 1, 1, 1, 1); defaultStoreClient = storeClientFactory.getStoreClient("test"); socketStore = ServerTestUtils.getSocketStore(socketStoreFactory, "test", @@ -100,7 +94,6 @@ private Versioned getVersioned(int... nodes) { return new Versioned("my-value", TestUtils.getClock(nodes)); } - @SuppressWarnings("unchecked") @Test public void testVersionedPut() { defaultStoreClient.put(KEY, v1); @@ -111,7 +104,6 @@ public void testVersionedPut() { assertEquals(1, resList.size()); } - @SuppressWarnings("unchecked") @Test public void testNormalPut() { defaultStoreClient.put(KEY, v1); From 2f08f0d378983ffd6f03c926f1a5504ac4da946f Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Mon, 27 Feb 2012 19:12:49 -0800 Subject: [PATCH 003/209] More code cleanup --- src/java/voldemort/versioning/VectorClock.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/java/voldemort/versioning/VectorClock.java b/src/java/voldemort/versioning/VectorClock.java index ff0f4e0cad..db35c1fc88 100644 --- a/src/java/voldemort/versioning/VectorClock.java +++ b/src/java/voldemort/versioning/VectorClock.java @@ -342,4 +342,5 @@ public long getTimestamp() { public List getEntries() { return this.versions; } + } From 3d8ecf29c559a9623b55a58ce9e9ab8ea1bdb0be Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Wed, 29 Feb 2012 18:20:22 -0800 Subject: [PATCH 004/209] Additional test in ChainedInconsistencyResolverTest for testing larger conflicts --- .../ChainedInconsistencyResolverTest.java | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/test/unit/voldemort/versioning/ChainedInconsistencyResolverTest.java b/test/unit/voldemort/versioning/ChainedInconsistencyResolverTest.java index f557beeb65..26767af8c1 100644 --- a/test/unit/voldemort/versioning/ChainedInconsistencyResolverTest.java +++ b/test/unit/voldemort/versioning/ChainedInconsistencyResolverTest.java @@ -34,6 +34,7 @@ public class ChainedInconsistencyResolverTest extends TestCase { private static final String KEY = "XYZ"; private Versioned v1, v2; + private Versioned conflict1, conflict2, conflict3, conflict4, conflict5, conflict6; private Node node; private Cluster cluster; @@ -74,8 +75,18 @@ public void setUp() throws IOException { new Properties()), cluster)); + // Initialize versioned puts for basic test v1 = getVersioned(0, 1, 1, 1, 1, 1); v2 = getVersioned(0, 0, 1, 1, 1, 1); + + // Initialize versioned puts for > 1 conflicts + conflict1 = getVersioned(0, 1, 1, 1, 1, 1, 1, 1, 1, 1); + conflict2 = getVersioned(0, 0, 1, 1, 1, 1, 1, 1, 1, 1); + conflict3 = getVersioned(0, 0, 0, 1, 1, 1, 1, 1, 1, 1); + conflict4 = getVersioned(0, 0, 0, 0, 1, 1, 1, 1, 1, 1); + conflict5 = getVersioned(0, 0, 0, 0, 0, 1, 1, 1, 1, 1); + conflict6 = getVersioned(0, 0, 0, 0, 0, 0, 1, 1, 1, 1); + defaultStoreClient = storeClientFactory.getStoreClient("test"); socketStore = ServerTestUtils.getSocketStore(socketStoreFactory, "test", @@ -112,4 +123,20 @@ public void testNormalPut() { List> resList = socketStore.get(new ByteArray(KEY.getBytes()), null); assertEquals(1, resList.size()); } + + @Test + public void testMoreConflicts() { + defaultStoreClient.put(KEY, conflict1); + defaultStoreClient.put(KEY, conflict2); + defaultStoreClient.put(KEY, conflict3); + defaultStoreClient.put(KEY, conflict4); + defaultStoreClient.put(KEY, conflict5); + defaultStoreClient.put(KEY, conflict6); + List> resList = socketStore.get(new ByteArray(KEY.getBytes()), null); + assertEquals(6, resList.size()); + Versioned res = defaultStoreClient.get(KEY); + defaultStoreClient.put(KEY, res); + resList = socketStore.get(new ByteArray(KEY.getBytes()), null); + assertEquals(1, resList.size()); + } } From b2843c8cb0a21f875ab37c8f1676f58305685801 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Tue, 6 Mar 2012 16:59:24 -0800 Subject: [PATCH 005/209] Added configurable option to interrupt service being unscheduled --- .../voldemort/server/VoldemortConfig.java | 10 ++++ .../voldemort/server/VoldemortServer.java | 3 +- .../server/scheduler/DataCleanupJob.java | 7 +-- .../server/scheduler/SchedulerService.java | 8 ++- .../scheduler/slop/BlockingSlopPusherJob.java | 8 +-- .../slop/StreamingSlopPusherJob.java | 6 +- .../voldemort/server/storage/RepairJob.java | 9 +-- .../server/storage/StorageService.java | 25 ++++++--- .../scheduled/BlockingSlopPusherTest.java | 6 +- .../scheduled/DataCleanupJobTest.java | 4 +- .../scheduled/StreamingSlopPusherTest.java | 56 ++++++++++--------- .../store/routed/HintedHandoffTest.java | 6 +- 12 files changed, 89 insertions(+), 59 deletions(-) diff --git a/src/java/voldemort/server/VoldemortConfig.java b/src/java/voldemort/server/VoldemortConfig.java index 4d1cf00759..13541ebd29 100644 --- a/src/java/voldemort/server/VoldemortConfig.java +++ b/src/java/voldemort/server/VoldemortConfig.java @@ -119,6 +119,7 @@ public class VoldemortConfig implements Serializable { private int clientMaxQueuedRequests; private int schedulerThreads; + private boolean mayInterruptService; private int numScanPermits; @@ -307,6 +308,7 @@ public VoldemortConfig(Props props) { this.slopZonesDownToTerminate = props.getInt("slop.zones.terminate", 0); this.schedulerThreads = props.getInt("scheduler.threads", 6); + this.mayInterruptService = props.getBoolean("service.interruptible", true); this.numScanPermits = props.getInt("num.scan.permits", 1); @@ -1133,6 +1135,14 @@ public void setSchedulerThreads(int schedulerThreads) { this.schedulerThreads = schedulerThreads; } + public boolean canInterruptService() { + return mayInterruptService; + } + + public void setInterruptible(boolean canInterrupt) { + this.mayInterruptService = canInterrupt; + } + public String getReadOnlyDataStorageDirectory() { return this.readOnlyStorageDir; } diff --git a/src/java/voldemort/server/VoldemortServer.java b/src/java/voldemort/server/VoldemortServer.java index 05640baf74..af82eec8a9 100644 --- a/src/java/voldemort/server/VoldemortServer.java +++ b/src/java/voldemort/server/VoldemortServer.java @@ -147,7 +147,8 @@ private List createServices() { /* Services are given in the order they must be started */ List services = new ArrayList(); SchedulerService scheduler = new SchedulerService(voldemortConfig.getSchedulerThreads(), - SystemTime.INSTANCE); + SystemTime.INSTANCE, + voldemortConfig.canInterruptService()); StorageService storageService = new StorageService(storeRepository, metadata, scheduler, diff --git a/src/java/voldemort/server/scheduler/DataCleanupJob.java b/src/java/voldemort/server/scheduler/DataCleanupJob.java index 1cef22f870..ff3dd3adc1 100644 --- a/src/java/voldemort/server/scheduler/DataCleanupJob.java +++ b/src/java/voldemort/server/scheduler/DataCleanupJob.java @@ -16,10 +16,9 @@ package voldemort.server.scheduler; -import java.util.concurrent.Semaphore; - import org.apache.log4j.Logger; +import voldemort.server.storage.ScanPermitWrapper; import voldemort.store.StorageEngine; import voldemort.utils.ClosableIterator; import voldemort.utils.EventThrottler; @@ -39,13 +38,13 @@ public class DataCleanupJob implements Runnable { private static final Logger logger = Logger.getLogger(DataCleanupJob.class); private final StorageEngine store; - private final Semaphore cleanupPermits; + private final ScanPermitWrapper cleanupPermits; private final long maxAgeMs; private final Time time; private final EventThrottler throttler; public DataCleanupJob(StorageEngine store, - Semaphore cleanupPermits, + ScanPermitWrapper cleanupPermits, long maxAgeMs, Time time, EventThrottler throttler) { diff --git a/src/java/voldemort/server/scheduler/SchedulerService.java b/src/java/voldemort/server/scheduler/SchedulerService.java index 110fff97e1..c241c4efa4 100644 --- a/src/java/voldemort/server/scheduler/SchedulerService.java +++ b/src/java/voldemort/server/scheduler/SchedulerService.java @@ -48,6 +48,7 @@ public class SchedulerService extends AbstractService { private static final Logger logger = Logger.getLogger(VoldemortService.class); + private boolean mayInterrupt; private class ScheduledRunnable { @@ -85,11 +86,16 @@ long getIntervalMs() { private final ConcurrentHashMap allJobs; public SchedulerService(int schedulerThreads, Time time) { + this(schedulerThreads, time, true); + } + + public SchedulerService(int schedulerThreads, Time time, boolean mayInterrupt) { super(ServiceType.SCHEDULER); this.time = time; this.scheduler = new SchedulerThreadPool(schedulerThreads); this.scheduledJobResults = new ConcurrentHashMap(); this.allJobs = new ConcurrentHashMap(); + this.mayInterrupt = mayInterrupt; } @Override @@ -104,7 +110,7 @@ public void stopInner() { public void disable(String id) { if(allJobs.containsKey(id) && scheduledJobResults.containsKey(id)) { ScheduledFuture future = scheduledJobResults.get(id); - boolean cancelled = future.cancel(false); + boolean cancelled = future.cancel(this.mayInterrupt); if(cancelled == true) { logger.info("Removed '" + id + "' from list of scheduled jobs"); scheduledJobResults.remove(id); diff --git a/src/java/voldemort/server/scheduler/slop/BlockingSlopPusherJob.java b/src/java/voldemort/server/scheduler/slop/BlockingSlopPusherJob.java index 0cac6b2a46..6e4b88dedf 100644 --- a/src/java/voldemort/server/scheduler/slop/BlockingSlopPusherJob.java +++ b/src/java/voldemort/server/scheduler/slop/BlockingSlopPusherJob.java @@ -18,7 +18,6 @@ import java.util.Date; import java.util.Map; -import java.util.concurrent.Semaphore; import org.apache.log4j.Level; import org.apache.log4j.Logger; @@ -28,13 +27,14 @@ import voldemort.cluster.failuredetector.FailureDetector; import voldemort.server.StoreRepository; import voldemort.server.VoldemortConfig; +import voldemort.server.storage.ScanPermitWrapper; import voldemort.store.StorageEngine; import voldemort.store.Store; import voldemort.store.UnreachableStoreException; import voldemort.store.metadata.MetadataStore; import voldemort.store.slop.Slop; -import voldemort.store.slop.SlopStorageEngine; import voldemort.store.slop.Slop.Operation; +import voldemort.store.slop.SlopStorageEngine; import voldemort.utils.ByteArray; import voldemort.utils.ClosableIterator; import voldemort.utils.EventThrottler; @@ -62,13 +62,13 @@ public class BlockingSlopPusherJob implements Runnable { private final MetadataStore metadataStore; private final FailureDetector failureDetector; private final long maxWriteBytesPerSec; - private final Semaphore repairPermits; + private final ScanPermitWrapper repairPermits; public BlockingSlopPusherJob(StoreRepository storeRepo, MetadataStore metadataStore, FailureDetector failureDetector, VoldemortConfig voldemortConfig, - Semaphore repairPermits) { + ScanPermitWrapper repairPermits) { this.storeRepo = storeRepo; this.metadataStore = metadataStore; this.repairPermits = Utils.notNull(repairPermits); diff --git a/src/java/voldemort/server/scheduler/slop/StreamingSlopPusherJob.java b/src/java/voldemort/server/scheduler/slop/StreamingSlopPusherJob.java index 63d2e538ee..bd6a5be339 100644 --- a/src/java/voldemort/server/scheduler/slop/StreamingSlopPusherJob.java +++ b/src/java/voldemort/server/scheduler/slop/StreamingSlopPusherJob.java @@ -10,7 +10,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; -import java.util.concurrent.Semaphore; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; @@ -27,6 +26,7 @@ import voldemort.cluster.failuredetector.FailureDetector; import voldemort.server.StoreRepository; import voldemort.server.VoldemortConfig; +import voldemort.server.storage.ScanPermitWrapper; import voldemort.store.StorageEngine; import voldemort.store.UnreachableStoreException; import voldemort.store.metadata.MetadataStore; @@ -68,13 +68,13 @@ public class StreamingSlopPusherJob implements Runnable { private final Map> zoneMapping; private ConcurrentHashMap attemptedByNode; private ConcurrentHashMap succeededByNode; - private final Semaphore repairPermits; + private final ScanPermitWrapper repairPermits; public StreamingSlopPusherJob(StoreRepository storeRepo, MetadataStore metadataStore, FailureDetector failureDetector, VoldemortConfig voldemortConfig, - Semaphore repairPermits) { + ScanPermitWrapper repairPermits) { this.storeRepo = storeRepo; this.metadataStore = metadataStore; this.failureDetector = failureDetector; diff --git a/src/java/voldemort/server/storage/RepairJob.java b/src/java/voldemort/server/storage/RepairJob.java index 46ee9fd0bf..39e16895a0 100644 --- a/src/java/voldemort/server/storage/RepairJob.java +++ b/src/java/voldemort/server/storage/RepairJob.java @@ -4,7 +4,6 @@ import java.util.Date; import java.util.List; import java.util.Map; -import java.util.concurrent.Semaphore; import javax.management.MBeanOperationInfo; @@ -33,14 +32,14 @@ public class RepairJob implements Runnable { public final static List blackList = Arrays.asList("mysql", "krati", "read-only"); - private final Semaphore repairPermits; + private final ScanPermitWrapper repairPermits; private final StoreRepository storeRepo; private final MetadataStore metadataStore; private final int deleteBatchSize; public RepairJob(StoreRepository storeRepo, MetadataStore metadataStore, - Semaphore repairPermits, + ScanPermitWrapper repairPermits, int deleteBatchSize) { this.storeRepo = storeRepo; this.metadataStore = metadataStore; @@ -48,7 +47,9 @@ public RepairJob(StoreRepository storeRepo, this.deleteBatchSize = deleteBatchSize; } - public RepairJob(StoreRepository storeRepo, MetadataStore metadataStore, Semaphore repairPermits) { + public RepairJob(StoreRepository storeRepo, + MetadataStore metadataStore, + ScanPermitWrapper repairPermits) { this(storeRepo, metadataStore, repairPermits, DELETE_BATCH_SIZE); } diff --git a/src/java/voldemort/server/storage/StorageService.java b/src/java/voldemort/server/storage/StorageService.java index 659a84de96..d37df573e5 100644 --- a/src/java/voldemort/server/storage/StorageService.java +++ b/src/java/voldemort/server/storage/StorageService.java @@ -30,7 +30,6 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import javax.management.MBeanOperationInfo; @@ -40,6 +39,7 @@ import org.apache.log4j.Logger; import voldemort.VoldemortException; +import voldemort.annotations.jmx.JmxGetter; import voldemort.annotations.jmx.JmxManaged; import voldemort.annotations.jmx.JmxOperation; import voldemort.client.ClientThreadPool; @@ -117,7 +117,8 @@ public class StorageService extends AbstractService { private final DynamicThrottleLimit dynThrottleLimit; // Common permit shared by all job which do a disk scan - private final Semaphore scanPermits; + // private final Semaphore scanPermits; + private final ScanPermitWrapper scanPermitWrapper; private final SocketStoreFactory storeFactory; private final ConcurrentMap storageConfigs; private final ClientThreadPool clientThreadPool; @@ -134,7 +135,9 @@ public StorageService(StoreRepository storeRepository, this.scheduler = scheduler; this.storeRepository = storeRepository; this.metadata = metadata; - this.scanPermits = new Semaphore(voldemortConfig.getNumScanPermits()); + // this.scanPermits = new + // Semaphore(voldemortConfig.getNumScanPermits()); + this.scanPermitWrapper = new ScanPermitWrapper(voldemortConfig.getNumScanPermits()); this.storageConfigs = new ConcurrentHashMap(); this.clientThreadPool = new ClientThreadPool(config.getClientMaxThreads(), config.getClientThreadIdleMs(), @@ -234,12 +237,12 @@ protected void startInner() { metadata, failureDetector, voldemortConfig, - scanPermits) + scanPermitWrapper) : new StreamingSlopPusherJob(storeRepository, metadata, failureDetector, voldemortConfig, - scanPermits), + scanPermitWrapper), nextRun, voldemortConfig.getSlopFrequencyMs()); } @@ -247,7 +250,7 @@ protected void startInner() { // Create a repair job object and register it with Store repository if(voldemortConfig.isRepairEnabled()) { logger.info("Initializing repair job."); - RepairJob job = new RepairJob(storeRepository, metadata, scanPermits); + RepairJob job = new RepairJob(storeRepository, metadata, scanPermitWrapper); JmxUtils.registerMbean(job, JmxUtils.createObjectName(job.getClass())); storeRepository.registerRepairJob(job); } @@ -586,7 +589,7 @@ private void scheduleCleanupJob(StoreDefinition storeDef, EventThrottler throttler = new EventThrottler(maxReadRate); Runnable cleanupJob = new DataCleanupJob(engine, - scanPermits, + scanPermitWrapper, storeDef.getRetentionDays() * Time.MS_PER_DAY, SystemTime.INSTANCE, @@ -713,10 +716,10 @@ public void forceCleanupOldDataThrottled(String storeName, int entryScanThrottle if(storeDef.hasRetentionPeriod()) { ExecutorService executor = Executors.newFixedThreadPool(1); try { - if(scanPermits.availablePermits() >= 1) { + if(scanPermitWrapper.availablePermits() >= 1) { executor.execute(new DataCleanupJob(engine, - scanPermits, + scanPermitWrapper, storeDef.getRetentionDays() * Time.MS_PER_DAY, SystemTime.INSTANCE, @@ -815,4 +818,8 @@ public DynamicThrottleLimit getDynThrottleLimit() { return dynThrottleLimit; } + @JmxGetter(name = "getScanPermitOwners", description = "Returns class names of services holding the scan permit") + public List getPermitOwners() { + return this.scanPermitWrapper.getPermitOwners(); + } } diff --git a/test/unit/voldemort/scheduled/BlockingSlopPusherTest.java b/test/unit/voldemort/scheduled/BlockingSlopPusherTest.java index fafe74e05a..ad047dcf35 100644 --- a/test/unit/voldemort/scheduled/BlockingSlopPusherTest.java +++ b/test/unit/voldemort/scheduled/BlockingSlopPusherTest.java @@ -21,7 +21,6 @@ import java.util.Arrays; import java.util.Date; import java.util.List; -import java.util.concurrent.Semaphore; import junit.framework.TestCase; import voldemort.ServerTestUtils; @@ -32,12 +31,13 @@ import voldemort.server.StoreRepository; import voldemort.server.VoldemortConfig; import voldemort.server.scheduler.slop.BlockingSlopPusherJob; +import voldemort.server.storage.ScanPermitWrapper; import voldemort.store.FailingStore; import voldemort.store.memory.InMemoryStorageEngine; import voldemort.store.metadata.MetadataStore; import voldemort.store.slop.Slop; -import voldemort.store.slop.SlopStorageEngine; import voldemort.store.slop.Slop.Operation; +import voldemort.store.slop.SlopStorageEngine; import voldemort.utils.ByteArray; import voldemort.utils.Props; import voldemort.versioning.Versioned; @@ -70,7 +70,7 @@ protected void setUp() throws Exception { metadataStore, new NoopFailureDetector(), new VoldemortConfig(props), - new Semaphore(1)); + new ScanPermitWrapper(1)); } private Cluster makeCluster(int numNodes) { diff --git a/test/unit/voldemort/scheduled/DataCleanupJobTest.java b/test/unit/voldemort/scheduled/DataCleanupJobTest.java index 679c29a4e3..89dfb98bce 100644 --- a/test/unit/voldemort/scheduled/DataCleanupJobTest.java +++ b/test/unit/voldemort/scheduled/DataCleanupJobTest.java @@ -17,11 +17,11 @@ package voldemort.scheduled; import java.util.List; -import java.util.concurrent.Semaphore; import junit.framework.TestCase; import voldemort.MockTime; import voldemort.server.scheduler.DataCleanupJob; +import voldemort.server.storage.ScanPermitWrapper; import voldemort.store.StorageEngine; import voldemort.store.memory.InMemoryStorageEngine; import voldemort.utils.EventThrottler; @@ -52,7 +52,7 @@ public void testCleanupCleansUp() { // now run cleanup new DataCleanupJob(engine, - new Semaphore(1), + new ScanPermitWrapper(1), Time.MS_PER_DAY, time, new EventThrottler(1)).run(); diff --git a/test/unit/voldemort/scheduled/StreamingSlopPusherTest.java b/test/unit/voldemort/scheduled/StreamingSlopPusherTest.java index 048c920b5f..f0a045face 100644 --- a/test/unit/voldemort/scheduled/StreamingSlopPusherTest.java +++ b/test/unit/voldemort/scheduled/StreamingSlopPusherTest.java @@ -10,7 +10,6 @@ import java.util.Iterator; import java.util.List; import java.util.Properties; -import java.util.concurrent.Semaphore; import org.junit.After; import org.junit.Before; @@ -26,6 +25,7 @@ import voldemort.server.VoldemortConfig; import voldemort.server.VoldemortServer; import voldemort.server.scheduler.slop.StreamingSlopPusherJob; +import voldemort.server.storage.ScanPermitWrapper; import voldemort.store.StorageEngine; import voldemort.store.metadata.MetadataStore; import voldemort.store.slop.Slop; @@ -141,7 +141,7 @@ public void testFailedServer() throws IOException, InterruptedException { metadataStore, configs[0]))), configs[0], - new Semaphore(1)); + new ScanPermitWrapper(1)); pusher.run(); @@ -156,8 +156,9 @@ public void testFailedServer() throws IOException, InterruptedException { StorageEngine store = getVoldemortServer(2).getStoreRepository() .getStorageEngine(nextSlop.getStoreName()); if(nextSlop.getOperation().equals(Slop.Operation.PUT)) { - assertNotSame("entry should be present at store", 0, store.get(nextSlop.getKey(), - null).size()); + assertNotSame("entry should be present at store", + 0, + store.get(nextSlop.getKey(), null).size()); assertEquals("entry value should match", new String(nextSlop.getValue()), new String(store.get(nextSlop.getKey(), null).get(0).getValue())); @@ -274,7 +275,7 @@ public void testOutOfOrder() throws InterruptedException, IOException { metadataStore, configs[0]))), configs[0], - new Semaphore(1)); + new ScanPermitWrapper(1)); pusher.run(); @@ -325,7 +326,7 @@ public void testNormalPush() throws InterruptedException, IOException { metadataStore, configs[0]))), configs[0], - new Semaphore(1)); + new ScanPermitWrapper(1)); pusher.run(); @@ -340,8 +341,9 @@ public void testNormalPush() throws InterruptedException, IOException { StorageEngine store = getVoldemortServer(1).getStoreRepository() .getStorageEngine(nextSlop.getStoreName()); if(nextSlop.getOperation().equals(Slop.Operation.PUT)) { - assertNotSame("entry should be present at store", 0, store.get(nextSlop.getKey(), - null).size()); + assertNotSame("entry should be present at store", + 0, + store.get(nextSlop.getKey(), null).size()); assertEquals("entry value should match", new String(nextSlop.getValue()), new String(store.get(nextSlop.getKey(), null).get(0).getValue())); @@ -440,14 +442,14 @@ public void testNormalPushBothWays() throws InterruptedException, IOException { metadataStore, configs[0]))), configs[0], - new Semaphore(1)), pusher1 = new StreamingSlopPusherJob(getVoldemortServer(1).getStoreRepository(), - getVoldemortServer(1).getMetadataStore(), - new BannagePeriodFailureDetector(new FailureDetectorConfig().setNodes(cluster.getNodes()) - .setStoreVerifier(new ServerStoreVerifier(socketStoreFactory, - metadataStore, - configs[1]))), - configs[1], - new Semaphore(1)); + new ScanPermitWrapper(1)), pusher1 = new StreamingSlopPusherJob(getVoldemortServer(1).getStoreRepository(), + getVoldemortServer(1).getMetadataStore(), + new BannagePeriodFailureDetector(new FailureDetectorConfig().setNodes(cluster.getNodes()) + .setStoreVerifier(new ServerStoreVerifier(socketStoreFactory, + metadataStore, + configs[1]))), + configs[1], + new ScanPermitWrapper(1)); pusher0.run(); pusher1.run(); @@ -463,8 +465,9 @@ public void testNormalPushBothWays() throws InterruptedException, IOException { StorageEngine store = getVoldemortServer(1).getStoreRepository() .getStorageEngine(nextSlop.getStoreName()); if(nextSlop.getOperation().equals(Slop.Operation.PUT)) { - assertNotSame("entry should be present at store", 0, store.get(nextSlop.getKey(), - null).size()); + assertNotSame("entry should be present at store", + 0, + store.get(nextSlop.getKey(), null).size()); assertEquals("entry value should match", new String(nextSlop.getValue()), new String(store.get(nextSlop.getKey(), null).get(0).getValue())); @@ -484,8 +487,9 @@ public void testNormalPushBothWays() throws InterruptedException, IOException { StorageEngine store = getVoldemortServer(0).getStoreRepository() .getStorageEngine(nextSlop.getStoreName()); if(nextSlop.getOperation().equals(Slop.Operation.PUT)) { - assertNotSame("entry should be present at store", 0, store.get(nextSlop.getKey(), - null).size()); + assertNotSame("entry should be present at store", + 0, + store.get(nextSlop.getKey(), null).size()); assertEquals("entry value should match", new String(nextSlop.getValue()), new String(store.get(nextSlop.getKey(), null).get(0).getValue())); @@ -547,7 +551,7 @@ public void testServerReplacementWithoutBounce() throws IOException, Interrupted metadataStore, configs[0]))), configs[0], - new Semaphore(1)); + new ScanPermitWrapper(1)); pusher.run(); @@ -562,8 +566,9 @@ public void testServerReplacementWithoutBounce() throws IOException, Interrupted StorageEngine store = getVoldemortServer(2).getStoreRepository() .getStorageEngine(nextSlop.getStoreName()); if(nextSlop.getOperation().equals(Slop.Operation.PUT)) { - assertNotSame("entry should be present at store", 0, store.get(nextSlop.getKey(), - null).size()); + assertNotSame("entry should be present at store", + 0, + store.get(nextSlop.getKey(), null).size()); assertEquals("entry value should match", new String(nextSlop.getValue()), new String(store.get(nextSlop.getKey(), null).get(0).getValue())); @@ -619,8 +624,9 @@ public void testServerReplacementWithoutBounce() throws IOException, Interrupted StorageEngine store = getVoldemortServer(1).getStoreRepository() .getStorageEngine(nextSlop.getStoreName()); if(nextSlop.getOperation().equals(Slop.Operation.PUT)) { - assertNotSame("entry should be present at store", 0, store.get(nextSlop.getKey(), - null).size()); + assertNotSame("entry should be present at store", + 0, + store.get(nextSlop.getKey(), null).size()); assertEquals("entry value should match", new String(nextSlop.getValue()), new String(store.get(nextSlop.getKey(), null).get(0).getValue())); diff --git a/test/unit/voldemort/store/routed/HintedHandoffTest.java b/test/unit/voldemort/store/routed/HintedHandoffTest.java index dab6ee37b2..64d1662192 100644 --- a/test/unit/voldemort/store/routed/HintedHandoffTest.java +++ b/test/unit/voldemort/store/routed/HintedHandoffTest.java @@ -17,7 +17,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.Semaphore; import org.apache.log4j.Logger; import org.junit.After; @@ -27,7 +26,6 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import voldemort.cluster.failuredetector.MutableStoreVerifier; import voldemort.ServerTestUtils; import voldemort.TestUtils; import voldemort.VoldemortException; @@ -39,12 +37,14 @@ import voldemort.cluster.failuredetector.FailureDetector; import voldemort.cluster.failuredetector.FailureDetectorConfig; import voldemort.cluster.failuredetector.FailureDetectorUtils; +import voldemort.cluster.failuredetector.MutableStoreVerifier; import voldemort.routing.RoutingStrategy; import voldemort.routing.RoutingStrategyFactory; import voldemort.routing.RoutingStrategyType; import voldemort.serialization.SerializerDefinition; import voldemort.server.StoreRepository; import voldemort.server.scheduler.slop.StreamingSlopPusherJob; +import voldemort.server.storage.ScanPermitWrapper; import voldemort.store.ForceFailStore; import voldemort.store.StorageEngine; import voldemort.store.Store; @@ -200,7 +200,7 @@ public void setUp() throws Exception { cluster, Lists.newArrayList(storeDef), new Properties()), - new Semaphore(1)); + new ScanPermitWrapper(1)); slopPusherJobs.add(pusher); } From 91a0dc27a3d8dfc8399c0a3175a22aa0d56cd2dc Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Tue, 6 Mar 2012 18:16:00 -0800 Subject: [PATCH 006/209] Track number of active client connections --- .../server/niosocket/AsyncRequestHandler.java | 12 +++++++++++- .../server/niosocket/NioSelectorManager.java | 19 +++++++++++++++++-- .../server/niosocket/NioSocketService.java | 10 ++++++++++ 3 files changed, 38 insertions(+), 3 deletions(-) diff --git a/src/java/voldemort/server/niosocket/AsyncRequestHandler.java b/src/java/voldemort/server/niosocket/AsyncRequestHandler.java index 55a65fd106..3eae171fcb 100644 --- a/src/java/voldemort/server/niosocket/AsyncRequestHandler.java +++ b/src/java/voldemort/server/niosocket/AsyncRequestHandler.java @@ -25,6 +25,7 @@ import java.nio.channels.Selector; import java.nio.channels.SocketChannel; +import org.apache.commons.lang.mutable.MutableInt; import org.apache.log4j.Level; import voldemort.VoldemortException; @@ -59,12 +60,16 @@ public class AsyncRequestHandler extends SelectorManagerWorker { private StreamRequestHandler streamRequestHandler; + private MutableInt serverConnectionCount; + public AsyncRequestHandler(Selector selector, SocketChannel socketChannel, RequestHandlerFactory requestHandlerFactory, - int socketBufferSize) { + int socketBufferSize, + MutableInt serverConnectionCount) { super(selector, socketChannel, socketBufferSize); this.requestHandlerFactory = requestHandlerFactory; + this.serverConnectionCount = serverConnectionCount; } @Override @@ -345,4 +350,9 @@ private boolean initRequestHandler(SelectionKey selectionKey) { } } + @Override + public void close() { + serverConnectionCount.decrement(); + super.close(); + } } diff --git a/src/java/voldemort/server/niosocket/NioSelectorManager.java b/src/java/voldemort/server/niosocket/NioSelectorManager.java index 86ef6aef30..fe9df047bd 100644 --- a/src/java/voldemort/server/niosocket/NioSelectorManager.java +++ b/src/java/voldemort/server/niosocket/NioSelectorManager.java @@ -23,6 +23,7 @@ import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; +import org.apache.commons.lang.mutable.MutableInt; import org.apache.log4j.Level; import voldemort.server.protocol.RequestHandlerFactory; @@ -99,6 +100,8 @@ public class NioSelectorManager extends SelectorManager { private final int socketBufferSize; + private MutableInt numActiveConnections; + public NioSelectorManager(InetSocketAddress endpoint, RequestHandlerFactory requestHandlerFactory, int socketBufferSize) { @@ -106,6 +109,7 @@ public NioSelectorManager(InetSocketAddress endpoint, this.socketChannelQueue = new ConcurrentLinkedQueue(); this.requestHandlerFactory = requestHandlerFactory; this.socketBufferSize = socketBufferSize; + this.numActiveConnections = new MutableInt(0); } public void accept(SocketChannel socketChannel) { @@ -155,10 +159,13 @@ protected void processEvents() { AsyncRequestHandler attachment = new AsyncRequestHandler(selector, socketChannel, requestHandlerFactory, - socketBufferSize); + socketBufferSize, + numActiveConnections); - if(!isClosed.get()) + if(!isClosed.get()) { socketChannel.register(selector, SelectionKey.OP_READ, attachment); + numActiveConnections.increment(); + } } catch(ClosedSelectorException e) { if(logger.isDebugEnabled()) logger.debug("Selector is closed, exiting"); @@ -177,4 +184,12 @@ protected void processEvents() { } } + /** + * Returns the number of active connections for this selector manager + * + * @return + */ + public Integer getNumActiveConnections() { + return numActiveConnections.toInteger(); + } } diff --git a/src/java/voldemort/server/niosocket/NioSocketService.java b/src/java/voldemort/server/niosocket/NioSocketService.java index 6f3b223d8a..d36ecaa221 100644 --- a/src/java/voldemort/server/niosocket/NioSocketService.java +++ b/src/java/voldemort/server/niosocket/NioSocketService.java @@ -31,6 +31,7 @@ import org.apache.log4j.Logger; import voldemort.VoldemortException; +import voldemort.annotations.jmx.JmxGetter; import voldemort.server.AbstractSocketService; import voldemort.server.ServiceType; import voldemort.server.StatusManager; @@ -258,4 +259,13 @@ public void run() { } + @JmxGetter(name = "numActiveConnections", description = "total number of active connections across selector managers") + public final int getNumActiveConnections() { + int sum = 0; + for(NioSelectorManager manager: selectorManagers) { + sum += manager.getNumActiveConnections(); + } + return sum; + } + } From 7c65ab72ee9f5fb4224f90db53f2cab10a1891fa Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 14 Mar 2012 10:27:19 -0700 Subject: [PATCH 007/209] Unit test + jmx to track pending connections --- .../server/niosocket/AsyncRequestHandler.java | 5 +- .../server/niosocket/NioSelectorManager.java | 9 ++ .../server/niosocket/NioSocketService.java | 9 ++ src/java/voldemort/utils/Props.java | 4 + test/common/voldemort/ServerTestUtils.java | 24 ++-- .../server/socket/NioStatsJmxTest.java | 115 ++++++++++++++++++ 6 files changed, 154 insertions(+), 12 deletions(-) create mode 100644 test/unit/voldemort/server/socket/NioStatsJmxTest.java diff --git a/src/java/voldemort/server/niosocket/AsyncRequestHandler.java b/src/java/voldemort/server/niosocket/AsyncRequestHandler.java index 3eae171fcb..d4df1254d7 100644 --- a/src/java/voldemort/server/niosocket/AsyncRequestHandler.java +++ b/src/java/voldemort/server/niosocket/AsyncRequestHandler.java @@ -352,7 +352,10 @@ private boolean initRequestHandler(SelectionKey selectionKey) { @Override public void close() { + if(!isClosed.compareAndSet(false, true)) + return; + serverConnectionCount.decrement(); - super.close(); + closeInternal(); } } diff --git a/src/java/voldemort/server/niosocket/NioSelectorManager.java b/src/java/voldemort/server/niosocket/NioSelectorManager.java index fe9df047bd..c24327336a 100644 --- a/src/java/voldemort/server/niosocket/NioSelectorManager.java +++ b/src/java/voldemort/server/niosocket/NioSelectorManager.java @@ -192,4 +192,13 @@ protected void processEvents() { public Integer getNumActiveConnections() { return numActiveConnections.toInteger(); } + + /** + * Returns the number of connections queued for registration + * + * @return + */ + public Integer getNumQueuedConnections() { + return socketChannelQueue.size(); + } } diff --git a/src/java/voldemort/server/niosocket/NioSocketService.java b/src/java/voldemort/server/niosocket/NioSocketService.java index d36ecaa221..b292fba739 100644 --- a/src/java/voldemort/server/niosocket/NioSocketService.java +++ b/src/java/voldemort/server/niosocket/NioSocketService.java @@ -268,4 +268,13 @@ public final int getNumActiveConnections() { return sum; } + @JmxGetter(name = "numQueuedConnections", description = "total number of connections pending for registration with selector managers") + public final int getNumQueuedConnections() { + int sum = 0; + for(NioSelectorManager manager: selectorManagers) { + sum += manager.getNumQueuedConnections(); + } + return sum; + } + } diff --git a/src/java/voldemort/utils/Props.java b/src/java/voldemort/utils/Props.java index 4c1920b0c2..c440e127dc 100644 --- a/src/java/voldemort/utils/Props.java +++ b/src/java/voldemort/utils/Props.java @@ -67,6 +67,10 @@ public Props(Map... props) { public Props(Properties... properties) { this.props = new HashMap(); + loadProperties(properties); + } + + public void loadProperties(Properties... properties) { for(int i = properties.length - 1; i >= 0; i--) for(Entry e: properties[i].entrySet()) this.props.put((String) e.getKey(), (String) e.getValue()); diff --git a/test/common/voldemort/ServerTestUtils.java b/test/common/voldemort/ServerTestUtils.java index c1e6d48db0..9dfe1418a8 100644 --- a/test/common/voldemort/ServerTestUtils.java +++ b/test/common/voldemort/ServerTestUtils.java @@ -1,12 +1,12 @@ /* * Copyright 2008-2009 LinkedIn, Inc - * + * * Licensed under the Apache License, Version 2.0 (the "License"); you may not * use this file except in compliance with the License. You may obtain a copy of * the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the @@ -33,7 +33,6 @@ import org.apache.commons.io.FileUtils; import org.apache.http.client.HttpClient; -import org.apache.http.impl.client.DefaultHttpClient; import org.mortbay.jetty.Server; import org.mortbay.jetty.servlet.Context; import org.mortbay.jetty.servlet.ServletHolder; @@ -82,8 +81,8 @@ /** * Helper functions for testing with real server implementations - * - * + * + * */ public class ServerTestUtils { @@ -211,7 +210,9 @@ public static Context getJettyServer(String clusterXml, return context; } - public static HttpStore getHttpStore(String storeName, RequestFormatType format, int port, + public static HttpStore getHttpStore(String storeName, + RequestFormatType format, + int port, final HttpClient httpClient) { return new HttpStore(storeName, "localhost", @@ -288,7 +289,7 @@ public static Cluster getLocalCluster(int numberOfNodes, int[] ports, int[][] pa /** * Update a cluster by replacing the specified server with a new host, i.e. * new ports since they are all localhost - * + * * @param original The original cluster to be updated * @param serverIds The ids of the server to be replaced with new hosts * @return updated cluster @@ -328,7 +329,7 @@ public static Cluster updateClusterWithNewHost(Cluster original, int... serverId /** * Returns a list of zones with their proximity list being in increasing * order - * + * * @param numberOfZones The number of zones to return * @return List of zones */ @@ -352,7 +353,7 @@ public static List getZones(int numberOfZones) { * Returns a cluster with numberOfNodes nodes in numberOfZones * zones. It is important that numberOfNodes be divisible by * numberOfZones - * + * * @param numberOfNodes Number of nodes in the cluster * @param partitionsPerNode Number of partitions in one node * @param numberOfZones Number of zones @@ -595,7 +596,7 @@ public static VoldemortConfig createServerConfig(boolean useNio, String clusterFile, String storeFile, Properties properties) throws IOException { - Props props = new Props(properties); + Props props = new Props(); props.put("node.id", nodeId); props.put("voldemort.home", baseDir + "/node-" + nodeId); props.put("bdb.cache.size", 1 * 1024 * 1024); @@ -603,6 +604,7 @@ public static VoldemortConfig createServerConfig(boolean useNio, props.put("bdb.flush.transactions", "true"); props.put("jmx.enable", "false"); props.put("enable.mysql.engine", "true"); + props.loadProperties(properties); VoldemortConfig config = new VoldemortConfig(props); config.setMysqlDatabaseName("voldemort"); diff --git a/test/unit/voldemort/server/socket/NioStatsJmxTest.java b/test/unit/voldemort/server/socket/NioStatsJmxTest.java new file mode 100644 index 0000000000..f287737ba1 --- /dev/null +++ b/test/unit/voldemort/server/socket/NioStatsJmxTest.java @@ -0,0 +1,115 @@ +/* + * Copyright 2008-2009 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package voldemort.server.socket; + +import java.lang.management.ManagementFactory; +import java.util.Properties; +import java.util.Random; +import java.util.concurrent.TimeUnit; + +import javax.management.MBeanServer; +import javax.management.ObjectName; + +import junit.framework.TestCase; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import voldemort.ServerTestUtils; +import voldemort.TestUtils; +import voldemort.client.ClientConfig; +import voldemort.cluster.Cluster; +import voldemort.cluster.Node; +import voldemort.server.VoldemortConfig; +import voldemort.server.VoldemortServer; +import voldemort.server.niosocket.NioSocketService; +import voldemort.store.Store; +import voldemort.store.socket.SocketStoreFactory; +import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; +import voldemort.utils.ByteArray; +import voldemort.utils.JmxUtils; +import voldemort.versioning.Versioned; + +/** + * Unit test for NIO selector connection stats + * + */ +public class NioStatsJmxTest extends TestCase { + + private VoldemortServer server; + private Store socketStore; + private static final int MAX_TRAFFIC_TIME_MS = 2000; + + @Override + @Before + public void setUp() throws Exception { + String storeDefinitionFile = "test/common/voldemort/config/single-store.xml"; + ClientConfig clientConfig = new ClientConfig().setMaxConnectionsPerNode(1).setMaxThreads(1); + SocketStoreFactory socketStoreFactory = new ClientRequestExecutorPool(clientConfig.getSelectors(), + clientConfig.getMaxConnectionsPerNode(), + clientConfig.getConnectionTimeout(TimeUnit.MILLISECONDS), + clientConfig.getSocketTimeout(TimeUnit.MILLISECONDS), + clientConfig.getSocketBufferSize(), + clientConfig.getSocketKeepAlive()); + Cluster cluster = ServerTestUtils.getLocalCluster(1); + Properties props = new Properties(); + props.put("jmx.enable", "true"); + VoldemortConfig config = ServerTestUtils.createServerConfig(true, + 0, + TestUtils.createTempDir() + .getAbsolutePath(), + null, + storeDefinitionFile, + props); + server = ServerTestUtils.startVoldemortServer(socketStoreFactory, config, cluster); + for(Node node: cluster.getNodes()) { + socketStore = ServerTestUtils.getSocketStore(socketStoreFactory, + "test", + node.getSocketPort(), + clientConfig.getRequestFormatType()); + } + } + + @Test + public void testActiveConnectionCount() throws Exception { + // generate some traffic, + Random dataGen = new Random(); + long start = System.currentTimeMillis(); + long now = 0; + + byte[] data = new byte[256]; + while(((now = System.currentTimeMillis()) - start) <= MAX_TRAFFIC_TIME_MS) { + dataGen.nextBytes(data); + ByteArray key = new ByteArray(data); + socketStore.put(key, new Versioned(data), null); + } + + // has to be 1, since we configure client with 1 connection and do + // atleast one operation + MBeanServer beanserver = ManagementFactory.getPlatformMBeanServer(); + ObjectName name = JmxUtils.createObjectName(JmxUtils.getPackageName(NioSocketService.class), + "nio-socket-server"); + assertEquals(1, beanserver.getAttribute(name, "numActiveConnections")); + } + + @Override + @After + public void tearDown() { + server.stop(); + } +} From 72667991cd5b654703f613109d40f3fa1c416b35 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Wed, 14 Mar 2012 17:42:33 -0700 Subject: [PATCH 008/209] Using Histogram conditionally (only with StoreStats). Updated histogram bucket size to 10000 --- .../voldemort/store/stats/RequestCounter.java | 104 +++++++++++++----- .../voldemort/store/stats/StoreStats.java | 43 +++++--- 2 files changed, 103 insertions(+), 44 deletions(-) diff --git a/src/java/voldemort/store/stats/RequestCounter.java b/src/java/voldemort/store/stats/RequestCounter.java index 08b654c22a..89933f2b48 100644 --- a/src/java/voldemort/store/stats/RequestCounter.java +++ b/src/java/voldemort/store/stats/RequestCounter.java @@ -19,25 +19,43 @@ public class RequestCounter { private final Histogram histogram; private volatile int q95LatencyMs; private volatile int q99LatencyMs; + private boolean withHistogram; /** * @param durationMS specifies for how long you want to maintain this * counter (in milliseconds). */ public RequestCounter(int durationMS) { - this(durationMS, SystemTime.INSTANCE); + this(durationMS, SystemTime.INSTANCE, false); + } + + /** + * @param durationMS specifies for how long you want to maintain this + * counter (in milliseconds). withHistogram indicates that this + * counter should also use a histogram. + */ + public RequestCounter(int durationMS, boolean withHistogram) { + this(durationMS, SystemTime.INSTANCE, withHistogram); } /** * For testing request expiration via an injected time provider */ RequestCounter(int durationMS, Time time) { + this(durationMS, time, false); + } + + RequestCounter(int durationMS, Time time, boolean withHistogram) { this.time = time; this.values = new AtomicReference(new Accumulator()); this.durationMS = durationMS; - this.histogram = new Histogram(65535, 1); this.q95LatencyMs = 0; this.q99LatencyMs = 0; + this.withHistogram = withHistogram; + if(this.withHistogram) + this.histogram = new Histogram(10000, 1); + else + this.histogram = null; } public long getCount() { @@ -50,8 +68,7 @@ public long getTotalCount() { public float getThroughput() { Accumulator oldv = getValidAccumulator(); - double elapsed = (time.getMilliseconds() - oldv.startTimeMS) - / (double) Time.MS_PER_SECOND; + double elapsed = (time.getMilliseconds() - oldv.startTimeMS) / (double) Time.MS_PER_SECOND; if(elapsed > 0f) { return (float) (oldv.count / elapsed); } else { @@ -61,8 +78,7 @@ public float getThroughput() { public float getThroughputInBytes() { Accumulator oldv = getValidAccumulator(); - double elapsed = (time.getMilliseconds() - oldv.startTimeMS) - / (double) Time.MS_PER_SECOND; + double elapsed = (time.getMilliseconds() - oldv.startTimeMS) / (double) Time.MS_PER_SECOND; if(elapsed > 0f) { return (float) (oldv.totalBytes / elapsed); } else { @@ -91,10 +107,12 @@ public long getMaxLatencyInMs() { } private void maybeResetHistogram() { + if(!this.withHistogram) + return; Accumulator accum = values.get(); long now = time.getMilliseconds(); if(now - accum.startTimeMS > durationMS) { - // Reset the histogram + // Reset the histogram q95LatencyMs = histogram.getQuantile(0.95); q99LatencyMs = histogram.getQuantile(0.99); histogram.reset(); @@ -140,17 +158,24 @@ public void addRequest(long timeNS) { } /** - * @see #addRequest(long) - * Detailed request to track additionald data about PUT, GET and GET_ALL - * - * @param numEmptyResponses For GET and GET_ALL, how many keys were no values found + * @see #addRequest(long) Detailed request to track additionald data about + * PUT, GET and GET_ALL + * + * @param numEmptyResponses For GET and GET_ALL, how many keys were no + * values found * @param bytes Total number of bytes across all versions of values' bytes - * @param getAllAggregatedCount Total number of keys returned for getAll calls + * @param getAllAggregatedCount Total number of keys returned for getAll + * calls */ - public void addRequest(long timeNS, long numEmptyResponses, long bytes, long getAllAggregatedCount) { + public void addRequest(long timeNS, + long numEmptyResponses, + long bytes, + long getAllAggregatedCount) { int timeMs = (int) timeNS / (int) Time.NS_PER_MS; - histogram.insert(timeMs); - maybeResetHistogram(); + if(this.withHistogram) { + histogram.insert(timeMs); + maybeResetHistogram(); + } for(int i = 0; i < 3; i++) { Accumulator oldv = getValidAccumulator(); Accumulator newv = new Accumulator(oldv.startTimeMS, @@ -168,28 +193,32 @@ public void addRequest(long timeNS, long numEmptyResponses, long bytes, long get } /** - * Return the number of requests that have returned returned no value for the requested key. Tracked only for GET. + * Return the number of requests that have returned returned no value for + * the requested key. Tracked only for GET. */ public long getNumEmptyResponses() { return getValidAccumulator().numEmptyResponses; } /** - * Return the size of the largest response or request in bytes returned. Tracked only for GET, GET_ALL and PUT. + * Return the size of the largest response or request in bytes returned. + * Tracked only for GET, GET_ALL and PUT. */ public long getMaxSizeInBytes() { return getValidAccumulator().maxBytes; } /** - * Return the average size of all the versioned values returned. Tracked only for GET, GET_ALL and PUT. + * Return the average size of all the versioned values returned. Tracked + * only for GET, GET_ALL and PUT. */ public double getAverageSizeInBytes() { return getValidAccumulator().getAverageBytes(); } /** - * Return the aggregated number of keys returned across all getAll calls, taking into account multiple values returned per call. + * Return the aggregated number of keys returned across all getAll calls, + * taking into account multiple values returned per call. */ public long getGetAllAggregatedCount() { return getValidAccumulator().getAllAggregatedCount; @@ -198,32 +227,51 @@ public long getGetAllAggregatedCount() { public int getQ95LatencyMs() { return q95LatencyMs; } - + public int getQ99LatencyMs() { return q99LatencyMs; - } - + } + private class Accumulator { final long startTimeMS; final long count; final long totalTimeNS; final long total; - final long numEmptyResponses; // GET and GET_ALL: number of empty responses that have been returned - final long getAllAggregatedCount; // GET_ALL: a single call to GET_ALL can return multiple k-v pairs. Track total returned. + final long numEmptyResponses; // GET and GET_ALL: number of empty + // responses that have been returned + final long getAllAggregatedCount; // GET_ALL: a single call to GET_ALL + // can return multiple k-v pairs. + // Track total returned. final long maxLatencyNS; - final long maxBytes; // Maximum single value - final long totalBytes; // Sum of all the values + final long maxBytes; // Maximum single value + final long totalBytes; // Sum of all the values public Accumulator() { this(RequestCounter.this.time.getMilliseconds(), 0, 0, 0, 0, 0, 0, 0, 0); } public Accumulator newWithTotal() { - return new Accumulator(RequestCounter.this.time.getMilliseconds(), 0, 0, total, 0, 0, 0, 0, 0); + return new Accumulator(RequestCounter.this.time.getMilliseconds(), + 0, + 0, + total, + 0, + 0, + 0, + 0, + 0); } - public Accumulator(long startTimeMS, long count, long totalTimeNS, long total, long numEmptyResponses, long maxLatencyNS, long totalBytes, long maxBytes, long getAllAggregatedCount) { + public Accumulator(long startTimeMS, + long count, + long totalTimeNS, + long total, + long numEmptyResponses, + long maxLatencyNS, + long totalBytes, + long maxBytes, + long getAllAggregatedCount) { this.startTimeMS = startTimeMS; this.count = count; this.totalTimeNS = totalTimeNS; diff --git a/src/java/voldemort/store/stats/StoreStats.java b/src/java/voldemort/store/stats/StoreStats.java index 863a82466e..07bd452cf7 100644 --- a/src/java/voldemort/store/stats/StoreStats.java +++ b/src/java/voldemort/store/stats/StoreStats.java @@ -26,36 +26,40 @@ public StoreStats(StoreStats parent) { counters = new EnumMap(Tracked.class); for(Tracked tracked: Tracked.values()) { - counters.put(tracked, new RequestCounter(300000)); + counters.put(tracked, new RequestCounter(300000, true)); } this.parent = parent; } /** - * Record the duration of specified op. For PUT, GET and GET_ALL use specific methods for those ops. + * Record the duration of specified op. For PUT, GET and GET_ALL use + * specific methods for those ops. */ public void recordTime(Tracked op, long timeNS) { recordTime(op, timeNS, 0, 0, 0); } /** - * Record the duration of a put operation, along with the size of the values returned. + * Record the duration of a put operation, along with the size of the values + * returned. */ public void recordPutTimeAndSize(long timeNS, long size) { - recordTime(Tracked.PUT, timeNS, 0, size, 0); + recordTime(Tracked.PUT, timeNS, 0, size, 0); } /** - * Record the duration of a get operation, along with whether or not an empty response (ie no values matched) - * and the size of the values returned. + * Record the duration of a get operation, along with whether or not an + * empty response (ie no values matched) and the size of the values + * returned. */ public void recordGetTime(long timeNS, boolean emptyResponse, long totalBytes) { recordTime(Tracked.GET, timeNS, emptyResponse ? 1 : 0, totalBytes, 0); } /** - * Record the duration of a get_all operation, along with how many values were requested, how may were actually - * returned and the size of the values returned. + * Record the duration of a get_all operation, along with how many values + * were requested, how may were actually returned and the size of the values + * returned. */ public void recordGetAllTime(long timeNS, int requested, int returned, long totalBytes) { recordTime(Tracked.GET_ALL, timeNS, requested - returned, totalBytes, requested); @@ -63,14 +67,21 @@ public void recordGetAllTime(long timeNS, int requested, int returned, long tota /** * Method to service public recording APIs - * - * @param op Operation being tracked - * @param timeNS Duration of operation - * @param numEmptyResponses GET and GET_ALL: number of empty responses being sent back, ie requested keys for which there were no values - * @param size Total size of response payload, ie sum of lengths of bytes in all versions of values - * @param getAllAggregateRequests Total of key-values requested in aggregatee from get_all operations + * + * @param op Operation being tracked + * @param timeNS Duration of operation + * @param numEmptyResponses GET and GET_ALL: number of empty responses being + * sent back, ie requested keys for which there were no values + * @param size Total size of response payload, ie sum of lengths of bytes in + * all versions of values + * @param getAllAggregateRequests Total of key-values requested in + * aggregatee from get_all operations */ - private void recordTime(Tracked op, long timeNS, long numEmptyResponses, long size, long getAllAggregateRequests) { + private void recordTime(Tracked op, + long timeNS, + long numEmptyResponses, + long size, + long getAllAggregateRequests) { counters.get(op).addRequest(timeNS, numEmptyResponses, size, getAllAggregateRequests); if(parent != null) parent.recordTime(op, timeNS, numEmptyResponses, size, getAllAggregateRequests); @@ -103,7 +114,7 @@ public long getMaxLatencyInMs(Tracked op) { public long getQ95LatencyInMs(Tracked op) { return counters.get(op).getQ95LatencyMs(); } - + public long getQ99LatencyInMs(Tracked op) { return counters.get(op).getQ99LatencyMs(); } From ddded7cf0af82cc1a789ca8426ef6c48744bf445 Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Thu, 15 Mar 2012 12:40:44 -0700 Subject: [PATCH 009/209] add clientId for voldemort client --- .../client/AbstractStoreClientFactory.java | 77 +++- .../client/CachingStoreClientFactory.java | 32 +- src/java/voldemort/client/ClientConfig.java | 17 +- .../voldemort/client/DefaultStoreClient.java | 29 +- .../client/MockStoreClientFactory.java | 9 +- .../voldemort/client/StoreClientFactory.java | 17 +- .../voldemort/StaticStoreClientFactory.java | 10 +- test/unit/voldemort/client/ClientJmxTest.java | 341 ++++++++++++++++++ 8 files changed, 505 insertions(+), 27 deletions(-) create mode 100644 test/unit/voldemort/client/ClientJmxTest.java diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index 94664d08b0..c35f17c124 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -16,12 +16,17 @@ package voldemort.client; +import java.io.File; +import java.io.IOException; import java.io.StringReader; +import java.net.InetAddress; import java.net.URI; import java.net.URISyntaxException; +import java.net.UnknownHostException; import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -98,6 +103,8 @@ public abstract class AbstractStoreClientFactory implements StoreClientFactory { private final ClientConfig config; private final RoutedStoreFactory routedStoreFactory; private final int clientZoneId; + private final String clientContextName; + private final AtomicInteger sequencer; public AbstractStoreClientFactory(ClientConfig config) { this.config = config; @@ -112,18 +119,24 @@ public AbstractStoreClientFactory(ClientConfig config) { this.maxBootstrapRetries = config.getMaxBootstrapRetries(); this.stats = new StoreStats(); this.clientZoneId = config.getClientZoneId(); + this.clientContextName = (null == config.getClientContextName() ? "" + : config.getClientContextName()); this.routedStoreFactory = new RoutedStoreFactory(config.isPipelineRoutedStoreEnabled(), threadPool, config.getRoutingTimeout(TimeUnit.MILLISECONDS)); + this.sequencer = new AtomicInteger(0); if(this.isJmxEnabled) { JmxUtils.registerMbean(threadPool, JmxUtils.createObjectName(JmxUtils.getPackageName(threadPool.getClass()), JmxUtils.getClassName(threadPool.getClass()) + + "." + + clientContextName + jmxId())); JmxUtils.registerMbean(new StoreStatsJmx(stats), JmxUtils.createObjectName("voldemort.store.stats.aggregate", - "aggregate-perf" + jmxId())); + clientContextName + ".aggregate-perf" + + jmxId())); } } @@ -133,12 +146,18 @@ public StoreClient getStoreClient(String storeName) { public StoreClient getStoreClient(String storeName, InconsistencyResolver> resolver) { - return new DefaultStoreClient(storeName, resolver, this, 3); + return new DefaultStoreClient(storeName, + resolver, + this, + 3, + clientContextName, + sequencer.getAndIncrement()); } @SuppressWarnings("unchecked") public Store getRawStore(String storeName, - InconsistencyResolver> resolver) { + InconsistencyResolver> resolver, + UUID clientId) { logger.info("Client zone-id [" + clientZoneId + "] Attempting to obtain metadata for store [" + storeName + "] "); @@ -222,7 +241,13 @@ public Store getRawStore(String storeName, store = statStore; JmxUtils.registerMbean(new StoreStatsJmx(statStore.getStats()), JmxUtils.createObjectName(JmxUtils.getPackageName(store.getClass()), - store.getName() + jmxId())); + clientContextName + + "." + + store.getName() + + jmxId() + + (null == clientId ? "" + : "." + + clientId.toString()))); } if(storeDef.getKeySerializer().hasCompression() @@ -256,6 +281,11 @@ public Store getRawStore(String storeName, return serializedStore; } + public Store getRawStore(String storeName, + InconsistencyResolver> resolver) { + return getRawStore(storeName, resolver, null); + } + protected ClientConfig getConfig() { return config; } @@ -400,7 +430,44 @@ public void close() { /* Give a unique id to avoid jmx clashes */ private String jmxId() { - return jmxId == 0 ? "" : Integer.toString(jmxId); + return jmxId == 0 ? "" : "." + Integer.toString(jmxId); } + /** + * Generate a unique client ID based on: 0. clientContext, if specified; 1. + * storeName 2. run path 3. client sequence + * + * @param storeName the name of the store the client is created for + * @param contextName the name of the client context + * @param clientSequence the client sequence number + * @return unique client ID + */ + public static UUID generateClientId(String storeName, String contextName, int clientSequence) { + String newLine = System.getProperty("line.separator"); + StringBuilder context = new StringBuilder(contextName == null ? "" : contextName); + context.append(0 == clientSequence ? "" : ("." + clientSequence)); + context.append(".").append(storeName); + + try { + InetAddress host = InetAddress.getLocalHost(); + context.append("@").append(host.getHostName()).append(":"); + } catch(UnknownHostException e) { + logger.info("Unable to obtain client hostname."); + logger.info(e.getMessage()); + } + + try { + String currentPath = new File(".").getCanonicalPath(); + context.append(currentPath).append(newLine); + } catch(IOException e) { + logger.info("Unable to obtain client run path."); + logger.info(e.getMessage()); + } + + if(logger.isDebugEnabled()) { + logger.debug(context.toString()); + } + + return UUID.nameUUIDFromBytes(context.toString().getBytes()); + } } diff --git a/src/java/voldemort/client/CachingStoreClientFactory.java b/src/java/voldemort/client/CachingStoreClientFactory.java index 993ff7c3a5..5fd2953da3 100644 --- a/src/java/voldemort/client/CachingStoreClientFactory.java +++ b/src/java/voldemort/client/CachingStoreClientFactory.java @@ -1,12 +1,12 @@ /* * Copyright 2008-2010 LinkedIn, Inc - * + * * Licensed under the Apache License, Version 2.0 (the "License"); you may not * use this file except in compliance with the License. You may obtain a copy of * the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the @@ -16,8 +16,13 @@ package voldemort.client; -import com.google.common.collect.ImmutableList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + import org.apache.log4j.Logger; + import voldemort.annotations.jmx.JmxManaged; import voldemort.annotations.jmx.JmxOperation; import voldemort.cluster.failuredetector.FailureDetector; @@ -26,14 +31,12 @@ import voldemort.versioning.InconsistencyResolver; import voldemort.versioning.Versioned; -import java.util.List; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; +import com.google.common.collect.ImmutableList; /** - * A wrapper for a store {@link StoreClientFactory} which caches requests - * to getStoreClient - * + * A wrapper for a store {@link StoreClientFactory} which caches requests to + * getStoreClient + * */ @JmxManaged(description = "A StoreClientFactory which caches clients") public class CachingStoreClientFactory implements StoreClientFactory { @@ -48,7 +51,6 @@ public CachingStoreClientFactory(StoreClientFactory inner) { this.cache = new ConcurrentHashMap, StoreClient>(); } - @SuppressWarnings("unchecked") public StoreClient getStoreClient(String storeName) { Pair key = Pair.create(storeName, null); @@ -74,7 +76,13 @@ public StoreClient getStoreClient(String storeName, public Store getRawStore(String storeName, InconsistencyResolver> resolver) { - return inner.getRawStore(storeName, resolver); + return getRawStore(storeName, resolver, null); + } + + public Store getRawStore(String storeName, + InconsistencyResolver> resolver, + UUID clientId) { + return inner.getRawStore(storeName, resolver, clientId); } public void close() { diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index f74f31c04d..30e4ad9dad 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -76,6 +76,7 @@ public class ClientConfig { private long failureDetectorRequestLengthThreshold = socketTimeoutMs; private volatile int maxBootstrapRetries = 2; + private volatile String clientContextName = "default"; public ClientConfig() {} @@ -110,6 +111,7 @@ public ClientConfig() {} public static final String FAILUREDETECTOR_CATASTROPHIC_ERROR_TYPES_PROPERTY = "failuredetector_catastrophic_error_types"; public static final String FAILUREDETECTOR_REQUEST_LENGTH_THRESHOLD_PROPERTY = "failuredetector_request_length_threshold"; public static final String MAX_BOOTSTRAP_RETRIES = "max_bootstrap_retries"; + public static final String CLIENT_CONTEXT_NAME = "voldemort_client_context"; /** * Instantiate the client config using a properties file @@ -238,6 +240,10 @@ private void setProperties(Properties properties) { if(props.containsKey(MAX_BOOTSTRAP_RETRIES)) this.setMaxBootstrapRetries(props.getInt(MAX_BOOTSTRAP_RETRIES)); + + if(props.containsKey(CLIENT_CONTEXT_NAME)) { + this.setClientContextName(props.getString(CLIENT_CONTEXT_NAME, null)); + } } public int getMaxConnectionsPerNode() { @@ -517,7 +523,7 @@ public boolean isLazyEnabled() { /** * Enable lazy initialization of clients? - * + * * @param enableLazy If true clients will be lazily initialized */ public ClientConfig setEnableLazy(boolean enableLazy) { @@ -634,4 +640,13 @@ public ClientConfig setMaxBootstrapRetries(int maxBootstrapRetries) { return this; } + public String getClientContextName() { + return clientContextName; + } + + public ClientConfig setClientContextName(String clientContextName) { + this.clientContextName = clientContextName; + return this; + } + } diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index 832339b243..0f90c04a31 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -18,6 +18,7 @@ import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.Map.Entry; import org.apache.log4j.Logger; @@ -63,29 +64,45 @@ public class DefaultStoreClient implements StoreClient { private final String storeName; private final InconsistencyResolver> resolver; private volatile Store store; + private final UUID clientId; public DefaultStoreClient(String storeName, InconsistencyResolver> resolver, StoreClientFactory storeFactory, int maxMetadataRefreshAttempts) { + this(storeName, resolver, storeFactory, maxMetadataRefreshAttempts, null, 0); + } + + public DefaultStoreClient(String storeName, + InconsistencyResolver> resolver, + StoreClientFactory storeFactory, + int maxMetadataRefreshAttempts, + String clientContext, + int clientSequence) { + this.storeName = Utils.notNull(storeName); this.resolver = resolver; this.storeFactory = Utils.notNull(storeFactory); this.metadataRefreshAttempts = maxMetadataRefreshAttempts; - + this.clientId = AbstractStoreClientFactory.generateClientId(storeName, + clientContext, + clientSequence); // Registering self to be able to bootstrap client dynamically via JMX JmxUtils.registerMbean(this, JmxUtils.createObjectName(JmxUtils.getPackageName(this.getClass()), JmxUtils.getClassName(this.getClass()) - + "." + storeName)); - + + "." + clientContext + "." + + storeName + "." + + clientId.toString())); bootStrap(); + logger.info("Voldemort client created: clientContext=" + clientContext + " clientSequence=" + + clientSequence + " clientId=" + clientId.toString()); } @JmxOperation(description = "bootstrap metadata from the cluster.") public void bootStrap() { logger.info("Bootstrapping metadata for store " + this.storeName); - this.store = storeFactory.getRawStore(storeName, resolver); + this.store = storeFactory.getRawStore(storeName, resolver, clientId); } public boolean delete(K key) { @@ -355,4 +372,8 @@ else if(versions.size() == 1) return put(key, versioned, transforms); } + + public UUID getClientId() { + return clientId; + } } diff --git a/src/java/voldemort/client/MockStoreClientFactory.java b/src/java/voldemort/client/MockStoreClientFactory.java index 80613b653b..32a2997842 100644 --- a/src/java/voldemort/client/MockStoreClientFactory.java +++ b/src/java/voldemort/client/MockStoreClientFactory.java @@ -18,6 +18,7 @@ import java.io.StringReader; import java.util.List; +import java.util.UUID; import voldemort.cluster.failuredetector.FailureDetector; import voldemort.cluster.failuredetector.NoopFailureDetector; @@ -106,7 +107,8 @@ public StoreClient getStoreClient(String storeName, } public Store getRawStore(String storeName, - InconsistencyResolver> resolver) { + InconsistencyResolver> resolver, + UUID clientId) { if(this.storesXml != null) return getRawStore(storeName); @@ -131,6 +133,11 @@ public Store getRawStore(String storeName, return consistentStore; } + public Store getRawStore(String storeName, + InconsistencyResolver> resolver) { + return getRawStore(storeName, resolver, null); + } + private Store getRawStore(String storeName) { List storeDefs = storeMapper.readStoreList(new StringReader(storesXml)); StoreDefinition storeDef = null; diff --git a/src/java/voldemort/client/StoreClientFactory.java b/src/java/voldemort/client/StoreClientFactory.java index 166eac0c1d..f8fa8e710e 100644 --- a/src/java/voldemort/client/StoreClientFactory.java +++ b/src/java/voldemort/client/StoreClientFactory.java @@ -16,6 +16,8 @@ package voldemort.client; +import java.util.UUID; + import voldemort.cluster.failuredetector.FailureDetector; import voldemort.store.Store; import voldemort.versioning.InconsistencyResolver; @@ -65,9 +67,6 @@ public StoreClient getStoreClient(String storeName, /** * Get the underlying store, not the public StoreClient interface * - * @param The key type - * @param The value type - * @param The transform type * @param storeName The name of the store * @param resolver The inconsistency resolver * @return The appropriate store @@ -75,6 +74,18 @@ public StoreClient getStoreClient(String storeName, Store getRawStore(String storeName, InconsistencyResolver> resolver); + /** + * Get the underlying store, not the public StoreClient interface + * + * @param storeName The name of the store + * @param resolver The inconsistency resolver + * @param clientId The unique id of the client + * @return The appropriate store + */ + Store getRawStore(String storeName, + InconsistencyResolver> resolver, + UUID clientId); + /** * Close the store client */ diff --git a/test/common/voldemort/StaticStoreClientFactory.java b/test/common/voldemort/StaticStoreClientFactory.java index b93d1d73c0..c1ec4c4513 100644 --- a/test/common/voldemort/StaticStoreClientFactory.java +++ b/test/common/voldemort/StaticStoreClientFactory.java @@ -2,6 +2,7 @@ import java.util.Arrays; import java.util.List; +import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import voldemort.client.DefaultStoreClient; @@ -42,10 +43,17 @@ public StaticStoreClientFactory(Store... stores) { @SuppressWarnings("unchecked") public Store getRawStore(String storeName, - InconsistencyResolver> resolver) { + InconsistencyResolver> resolver, + UUID clientId) { return (Store) stores.get(Math.max(current.getAndIncrement(), stores.size() - 1)); } + @SuppressWarnings("unchecked") + public Store getRawStore(String storeName, + InconsistencyResolver> resolver) { + return getRawStore(storeName, resolver, null); + } + @SuppressWarnings("unchecked") public StoreClient getStoreClient(String storeName) { return new DefaultStoreClient(storeName, null, this, 3); diff --git a/test/unit/voldemort/client/ClientJmxTest.java b/test/unit/voldemort/client/ClientJmxTest.java new file mode 100644 index 0000000000..1c215580f8 --- /dev/null +++ b/test/unit/voldemort/client/ClientJmxTest.java @@ -0,0 +1,341 @@ +package voldemort.client; + +import java.lang.management.ManagementFactory; +import java.net.URISyntaxException; + +import javax.management.InstanceNotFoundException; +import javax.management.MBeanServer; +import javax.management.ObjectName; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import voldemort.ServerTestUtils; +import voldemort.serialization.SerializerFactory; +import voldemort.server.AbstractSocketService; +import voldemort.utils.JmxUtils; + +public class ClientJmxTest extends AbstractStoreClientFactoryTest { + + private static String STATS_DOMAIN = "voldemort.store.stats"; + private static String AGGREGATE_STATS_DOMAIN = "voldemort.store.stats.aggregate"; + private static String CLIENT_DOMAIN = "voldemort.client"; + private static String CLUSTER_FAILUREDETECTOR_DOMAIN = "voldemort.cluster.failuredetector"; + private static String CLIENT_REQUEST_DOMAIN = "voldemort.store.socket.clientrequest"; + + private AbstractSocketService socketService; + private MBeanServer mbServer = null; + + private static int factoryJmxId = 0; + + public ClientJmxTest() { + super(); + } + + private static String getAndIncrementJmxId() { + int current = factoryJmxId; + factoryJmxId++; + return (0 == current ? "" : "." + current); + } + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + socketService = ServerTestUtils.getSocketService(true, + getClusterXml(), + getStoreDefXml(), + getValidStoreName(), + getLocalNode().getSocketPort()); + socketService.start(); + mbServer = ManagementFactory.getPlatformMBeanServer(); + } + + @Override + @After + public void tearDown() throws Exception { + mbServer = null; + super.tearDown(); + socketService.stop(); + } + + @Override + protected StoreClientFactory getFactory(String... bootstrapUrls) { + return new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrls) + .setEnableLazy(false) + .setEnableJmx(true)); + } + + protected StoreClientFactory getFactoryWithClientContext(String clientContext, + String... bootstrapUrls) { + return new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrls) + .setEnableLazy(false) + .setClientContextName(clientContext) + .setEnableJmx(true)); + } + + @Test + public void testTwoClientContextOnJmx() throws Exception { + String clientContext1 = "clientA"; + String clientContext2 = "clientB"; + String jmxId1 = getAndIncrementJmxId(); + String jmxId2 = getAndIncrementJmxId(); + + StoreClient c1 = getFactoryWithClientContext(clientContext1, + getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + StoreClient c2 = getFactoryWithClientContext(clientContext2, + getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + + // checking for aggregate stats + ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext1 + + ".aggregate-perf" + + jmxId1); + ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext2 + + ".aggregate-perf" + + jmxId2); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + + // checking for per store stats + String c1type = clientContext1 + ".test" + jmxId1; + String c2type = clientContext2 + ".test" + jmxId2; + c1Name = JmxUtils.createObjectName(STATS_DOMAIN, + c1type + + "." + + ((DefaultStoreClient) c1).getClientId() + .toString()); + c2Name = JmxUtils.createObjectName(STATS_DOMAIN, + c2type + + "." + + ((DefaultStoreClient) c2).getClientId() + .toString()); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + } + + @Test + public void testSameContextOnJmx() throws Exception { + String clientContext = "clientContext"; + String jmxId1 = getAndIncrementJmxId(); + String jmxId2 = getAndIncrementJmxId(); + + StoreClient[] clients = new StoreClient[2]; + for(int i = 0; i < 2; i++) { + clients[i] = getFactoryWithClientContext(clientContext, getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + } + + // checking for aggregate stats + ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext + + ".aggregate-perf" + + jmxId1); + ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext + + ".aggregate-perf" + + jmxId2); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + + // checking for per store stats + String c1type = clientContext + ".test" + jmxId1; + String c2type = clientContext + ".test" + jmxId2; + c1Name = JmxUtils.createObjectName(STATS_DOMAIN, + c1type + + "." + + ((DefaultStoreClient) clients[0]).getClientId() + .toString()); + c2Name = JmxUtils.createObjectName(STATS_DOMAIN, + c2type + + "." + + ((DefaultStoreClient) clients[1]).getClientId() + .toString()); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + } + + @Test + public void testTwoClientNoContextOnJmx() throws Exception { + String clientContextCompare = "default"; + String jmxId1 = getAndIncrementJmxId(); + String jmxId2 = getAndIncrementJmxId(); + + StoreClient c1 = getFactory(getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + StoreClient c2 = getFactory(getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + + // checking for aggregate stats + ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare + + ".aggregate-perf" + + jmxId1); + ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare + + ".aggregate-perf" + + jmxId2); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + + // checking for per store stats + String c1type = clientContextCompare + ".test" + jmxId1; + String c2type = clientContextCompare + ".test" + jmxId2; + c1Name = JmxUtils.createObjectName(STATS_DOMAIN, + c1type + + "." + + ((DefaultStoreClient) c1).getClientId() + .toString()); + c2Name = JmxUtils.createObjectName(STATS_DOMAIN, + c2type + + "." + + ((DefaultStoreClient) c2).getClientId() + .toString()); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + } + + @Test + public void testTwoClientNullContextOnJmx() throws Exception { + String clientContextCompare = ""; + String jmxId1 = getAndIncrementJmxId(); + String jmxId2 = getAndIncrementJmxId(); + + StoreClient c1 = getFactoryWithClientContext(null, getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + StoreClient c2 = getFactoryWithClientContext(null, getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + + // checking for aggregate stats + ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare + + ".aggregate-perf" + + jmxId1); + ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare + + ".aggregate-perf" + + jmxId2); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + + // checking for per store stats + String c1type = clientContextCompare + ".test" + jmxId1; + String c2type = clientContextCompare + ".test" + jmxId2; + c1Name = JmxUtils.createObjectName(STATS_DOMAIN, + c1type + + "." + + ((DefaultStoreClient) c1).getClientId() + .toString()); + c2Name = JmxUtils.createObjectName(STATS_DOMAIN, + c2type + + "." + + ((DefaultStoreClient) c2).getClientId() + .toString()); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + } + + @Test + public void testSameContextAndFactory() throws Exception { + String clientContext = "clientContext"; + String jmxId = getAndIncrementJmxId(); + StoreClientFactory factory = getFactoryWithClientContext(clientContext, + getValidBootstrapUrl()); + + StoreClient[] clients = new StoreClient[2]; + for(int i = 0; i < 2; i++) { + clients[i] = factory.getStoreClient(getValidStoreName()); + } + + ObjectName cName = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext + + ".aggregate-perf" + + jmxId); + checkForMbeanFound(cName); + mbServer.unregisterMBean(cName); + + // checking for per store stats + String ctype = clientContext + ".test" + jmxId; + ObjectName c1Name = JmxUtils.createObjectName(STATS_DOMAIN, + ctype + + "." + + ((DefaultStoreClient) clients[0]).getClientId() + .toString()); + ObjectName c2Name = JmxUtils.createObjectName(STATS_DOMAIN, + ctype + + "." + + ((DefaultStoreClient) clients[1]).getClientId() + .toString()); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + } + + @Test + public void testDifferentId() throws Exception { + String clientContext = "clientContext"; + String jmxId = getAndIncrementJmxId(); + StoreClientFactory factory = getFactoryWithClientContext(clientContext, + getValidBootstrapUrl()); + + StoreClient[] clients = new StoreClient[2]; + clients[0] = factory.getStoreClient(getValidStoreName()); + clients[1] = factory.getStoreClient(getValidStoreName()); + + ObjectName cName = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext + + ".aggregate-perf" + + jmxId); + checkForMbeanFound(cName); + mbServer.unregisterMBean(cName); + + // checking for per store stats + String ctype = clientContext + ".test" + jmxId; + ObjectName c1Name = JmxUtils.createObjectName(STATS_DOMAIN, + ctype + + "." + + ((DefaultStoreClient) clients[0]).getClientId() + .toString()); + ObjectName c2Name = JmxUtils.createObjectName(STATS_DOMAIN, + ctype + + "." + + ((DefaultStoreClient) clients[1]).getClientId() + .toString()); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + assertTrue(!c1Name.equals(c2Name)); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + } + + private void checkForMbeanFound(ObjectName name) { + try { + mbServer.getMBeanInfo(name); + } catch(InstanceNotFoundException e) { + fail("MBean not found on the JMX Server: " + name.toString()); + } catch(Exception e) { + fail("Test failed: " + e.getMessage()); + } + } + + @Override + protected StoreClientFactory getFactoryWithSerializer(SerializerFactory factory, + String... bootstrapUrls) { + return new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrls) + .setEnableLazy(false) + .setSerializerFactory(factory)); + } + + @Override + protected String getValidBootstrapUrl() throws URISyntaxException { + return getLocalNode().getSocketUrl().toString(); + } + + @Override + protected String getValidScheme() { + return SocketStoreClientFactory.URL_SCHEME; + } +} From ea2c4f03650d2f0d161b26d3fcf90a681903c1b3 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Thu, 15 Mar 2012 13:22:45 -0700 Subject: [PATCH 010/209] Changed withHistogram to useHistogram --- .../voldemort/store/stats/RequestCounter.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/java/voldemort/store/stats/RequestCounter.java b/src/java/voldemort/store/stats/RequestCounter.java index 89933f2b48..9926170a1d 100644 --- a/src/java/voldemort/store/stats/RequestCounter.java +++ b/src/java/voldemort/store/stats/RequestCounter.java @@ -19,7 +19,7 @@ public class RequestCounter { private final Histogram histogram; private volatile int q95LatencyMs; private volatile int q99LatencyMs; - private boolean withHistogram; + private boolean useHistogram; /** * @param durationMS specifies for how long you want to maintain this @@ -31,11 +31,11 @@ public RequestCounter(int durationMS) { /** * @param durationMS specifies for how long you want to maintain this - * counter (in milliseconds). withHistogram indicates that this + * counter (in milliseconds). useHistogram indicates that this * counter should also use a histogram. */ - public RequestCounter(int durationMS, boolean withHistogram) { - this(durationMS, SystemTime.INSTANCE, withHistogram); + public RequestCounter(int durationMS, boolean useHistogram) { + this(durationMS, SystemTime.INSTANCE, useHistogram); } /** @@ -45,14 +45,14 @@ public RequestCounter(int durationMS, boolean withHistogram) { this(durationMS, time, false); } - RequestCounter(int durationMS, Time time, boolean withHistogram) { + RequestCounter(int durationMS, Time time, boolean useHistogram) { this.time = time; this.values = new AtomicReference(new Accumulator()); this.durationMS = durationMS; this.q95LatencyMs = 0; this.q99LatencyMs = 0; - this.withHistogram = withHistogram; - if(this.withHistogram) + this.useHistogram = useHistogram; + if(this.useHistogram) this.histogram = new Histogram(10000, 1); else this.histogram = null; @@ -107,7 +107,7 @@ public long getMaxLatencyInMs() { } private void maybeResetHistogram() { - if(!this.withHistogram) + if(!this.useHistogram) return; Accumulator accum = values.get(); long now = time.getMilliseconds(); @@ -172,7 +172,7 @@ public void addRequest(long timeNS, long bytes, long getAllAggregatedCount) { int timeMs = (int) timeNS / (int) Time.NS_PER_MS; - if(this.withHistogram) { + if(this.useHistogram) { histogram.insert(timeMs); maybeResetHistogram(); } From 8b53cc0d5938afe5eae793dd44e02b129fd8a73c Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Tue, 20 Mar 2012 20:23:08 -0700 Subject: [PATCH 011/209] fix a bug in histogram where long shall be used instead of int for tracking each call time --- src/java/voldemort/store/stats/Histogram.java | 34 +++++++++---------- .../voldemort/store/stats/RequestCounter.java | 2 +- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/src/java/voldemort/store/stats/Histogram.java b/src/java/voldemort/store/stats/Histogram.java index 5d5c2270f6..6031c841d5 100644 --- a/src/java/voldemort/store/stats/Histogram.java +++ b/src/java/voldemort/store/stats/Histogram.java @@ -1,20 +1,20 @@ package voldemort.store.stats; +import java.util.Arrays; + import voldemort.VoldemortException; import voldemort.annotations.concurrency.Threadsafe; -import java.util.Arrays; - /** * A class for computing percentiles based on a histogram. Values are bucketed * by a configurable bound (e.g., 0-1, 1-2, 2-3). When a value is inserted, * perform a binary search to find the correct bucket. - * - * + * + * */ @Threadsafe public class Histogram { - + private final int nBuckets; private final int step; private final int[] buckets; @@ -23,7 +23,7 @@ public class Histogram { /** * Initialize an empty histogram - * + * * @param nBuckets The number of buckets to use * @param step The size of each bucket */ @@ -34,7 +34,7 @@ public Histogram(int nBuckets, int step) { this.bounds = new int[nBuckets]; init(); } - + protected void init() { int bound = 0; for(int i = 0; i < nBuckets; i++, bound += step) { @@ -54,10 +54,10 @@ public synchronized void reset() { /** * Insert a value into the right bucket of the histogram. If the value is * larger than any bound, insert into the last bucket - * + * * @param data The value to insert into the histogram */ - public synchronized void insert(int data) { + public synchronized void insert(long data) { int index = findBucket(data); if(index == -1) { throw new VoldemortException(data + " can't be bucketed, is invalid!"); @@ -67,9 +67,9 @@ public synchronized void insert(int data) { } /** - * Find the a value n such that the percentile falls within - * [n, n + step) - * + * Find the a value n such that the percentile falls within [ + * n, n + step) + * * @param quantile The percentile to find * @return Lower bound associated with the percentile */ @@ -84,9 +84,9 @@ public synchronized int getQuantile(double quantile) { } return 0; } - - private int findBucket(int needle) { - int max = step * nBuckets; + + private int findBucket(long needle) { + long max = step * nBuckets; if(needle > max) { return nBuckets - 1; } @@ -105,8 +105,8 @@ private int findBucket(int needle) { } return -1; } - - private int compareToBucket(int bucket, int needle) { + + private int compareToBucket(int bucket, long needle) { int low = bounds[bucket]; int high = low + step; if(low <= needle && high > needle) { diff --git a/src/java/voldemort/store/stats/RequestCounter.java b/src/java/voldemort/store/stats/RequestCounter.java index 9926170a1d..19b0aeb85a 100644 --- a/src/java/voldemort/store/stats/RequestCounter.java +++ b/src/java/voldemort/store/stats/RequestCounter.java @@ -171,7 +171,7 @@ public void addRequest(long timeNS, long numEmptyResponses, long bytes, long getAllAggregatedCount) { - int timeMs = (int) timeNS / (int) Time.NS_PER_MS; + long timeMs = timeNS / Time.NS_PER_MS; if(this.useHistogram) { histogram.insert(timeMs); maybeResetHistogram(); From fd3367629ce2d5c950a227f0723e45e5db974e7b Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Wed, 21 Mar 2012 15:30:42 -0700 Subject: [PATCH 012/209] Added a log instead of Voldemort exception in Histogram and added RequestCounterTest --- src/java/voldemort/store/stats/Histogram.java | 6 ++-- .../store/stats/RequestCounterTest.java | 28 +++++++++++++++++++ 2 files changed, 32 insertions(+), 2 deletions(-) create mode 100644 test/unit/voldemort/store/stats/RequestCounterTest.java diff --git a/src/java/voldemort/store/stats/Histogram.java b/src/java/voldemort/store/stats/Histogram.java index 6031c841d5..fe46c1c08a 100644 --- a/src/java/voldemort/store/stats/Histogram.java +++ b/src/java/voldemort/store/stats/Histogram.java @@ -2,7 +2,8 @@ import java.util.Arrays; -import voldemort.VoldemortException; +import org.apache.log4j.Logger; + import voldemort.annotations.concurrency.Threadsafe; /** @@ -20,6 +21,7 @@ public class Histogram { private final int[] buckets; private final int[] bounds; private int size; + private static final Logger logger = Logger.getLogger(Histogram.class); /** * Initialize an empty histogram @@ -60,7 +62,7 @@ public synchronized void reset() { public synchronized void insert(long data) { int index = findBucket(data); if(index == -1) { - throw new VoldemortException(data + " can't be bucketed, is invalid!"); + logger.error(data + " can't be bucketed, is invalid!"); } buckets[index]++; size++; diff --git a/test/unit/voldemort/store/stats/RequestCounterTest.java b/test/unit/voldemort/store/stats/RequestCounterTest.java new file mode 100644 index 0000000000..3e31723794 --- /dev/null +++ b/test/unit/voldemort/store/stats/RequestCounterTest.java @@ -0,0 +1,28 @@ +package voldemort.store.stats; + +import org.junit.Before; +import org.junit.Test; + +public class RequestCounterTest { + + private RequestCounter requestCounter; + + @Before + public void setUp() { + // Initialize the RequestCounter with a histogram + requestCounter = new RequestCounter(10000, true); + } + + @Test + public void test() { + long val = 234; + requestCounter.addRequest(val); + } + + @Test + public void testLargeValues() { + long val = 999999992342756424l; + requestCounter.addRequest(val); + } + +} From e72630c2582b70b0d67c975b165084e39dd20d03 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Wed, 21 Mar 2012 15:33:52 -0700 Subject: [PATCH 013/209] Small fix with error log in Histogram --- src/java/voldemort/store/stats/Histogram.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/java/voldemort/store/stats/Histogram.java b/src/java/voldemort/store/stats/Histogram.java index fe46c1c08a..88a9a7b768 100644 --- a/src/java/voldemort/store/stats/Histogram.java +++ b/src/java/voldemort/store/stats/Histogram.java @@ -63,6 +63,7 @@ public synchronized void insert(long data) { int index = findBucket(data); if(index == -1) { logger.error(data + " can't be bucketed, is invalid!"); + return; } buckets[index]++; size++; From 78ee7c142605ded927f2a5b97a1450347afccb91 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Thu, 22 Mar 2012 17:27:28 -0700 Subject: [PATCH 014/209] Failure detector fix for host swap : DDS-2536 --- src/java/voldemort/cluster/Node.java | 6 ++++++ .../AbstractFailureDetector.java | 17 ++++++++++++++++- .../failuredetector/FailureDetectorConfig.java | 7 ++++++- 3 files changed, 28 insertions(+), 2 deletions(-) diff --git a/src/java/voldemort/cluster/Node.java b/src/java/voldemort/cluster/Node.java index 22b0c3bc29..b1b7e1d4ea 100644 --- a/src/java/voldemort/cluster/Node.java +++ b/src/java/voldemort/cluster/Node.java @@ -153,4 +153,10 @@ public int hashCode() { public int compareTo(Node other) { return Integer.valueOf(this.id).compareTo(other.getId()); } + + public boolean isEqualState(Node other) { + return id == other.getId() && host.equalsIgnoreCase(other.getHost()) + && httpPort == other.getHttpPort() && socketPort == other.getSocketPort() + && adminPort == other.getAdminPort() && zoneId == other.getZoneId(); + } } \ No newline at end of file diff --git a/src/java/voldemort/cluster/failuredetector/AbstractFailureDetector.java b/src/java/voldemort/cluster/failuredetector/AbstractFailureDetector.java index 437b890ea9..25a2ebc27c 100644 --- a/src/java/voldemort/cluster/failuredetector/AbstractFailureDetector.java +++ b/src/java/voldemort/cluster/failuredetector/AbstractFailureDetector.java @@ -45,6 +45,9 @@ public abstract class AbstractFailureDetector implements FailureDetector { protected final Map nodeStatusMap; + // Also maintain the list of nodes by IDs (in order to handle host swaps) + protected final Map nodeMap; + protected final Logger logger = Logger.getLogger(getClass().getName()); protected AbstractFailureDetector(FailureDetectorConfig failureDetectorConfig) { @@ -54,10 +57,12 @@ protected AbstractFailureDetector(FailureDetectorConfig failureDetectorConfig) { this.failureDetectorConfig = failureDetectorConfig; listeners = new ConcurrentHashMap(); nodeStatusMap = new ConcurrentHashMap(); + nodeMap = new ConcurrentHashMap(); for(Node node: failureDetectorConfig.getNodes()) { nodeStatusMap.put(node, createNodeStatus(failureDetectorConfig.getTime() .getMilliseconds())); + nodeMap.put(node.getId(), node); } } @@ -212,14 +217,24 @@ protected void setUnavailable(Node node, UnreachableStoreException e) { protected NodeStatus getNodeStatus(Node node) { NodeStatus nodeStatus = nodeStatusMap.get(node); + Node currentTrackedNode = nodeMap.get(node.getId()); - if(nodeStatus == null) { + if(nodeStatus == null || !currentTrackedNode.isEqualState(node)) { if(logger.isEnabledFor(Level.WARN)) logger.warn("creating new node status for node " + node.getId() + " for failure detector"); + // If the host is being replaced, remove old tracking information + if(nodeStatus != null) { + nodeStatusMap.remove(currentTrackedNode); + nodeMap.remove(currentTrackedNode); + failureDetectorConfig.removeNode(currentTrackedNode); + } + nodeStatus = createNodeStatus(failureDetectorConfig.getTime().getMilliseconds()); nodeStatusMap.put(node, nodeStatus); + nodeMap.put(node.getId(), node); + if(!failureDetectorConfig.getNodes().contains(node)) { failureDetectorConfig.addNode(node); } diff --git a/src/java/voldemort/cluster/failuredetector/FailureDetectorConfig.java b/src/java/voldemort/cluster/failuredetector/FailureDetectorConfig.java index 0882a4fb7e..a751844f6c 100644 --- a/src/java/voldemort/cluster/failuredetector/FailureDetectorConfig.java +++ b/src/java/voldemort/cluster/failuredetector/FailureDetectorConfig.java @@ -23,7 +23,6 @@ import java.util.HashSet; import java.util.List; -import com.google.common.collect.ImmutableSet; import voldemort.client.ClientConfig; import voldemort.cluster.Node; import voldemort.server.VoldemortConfig; @@ -32,6 +31,7 @@ import voldemort.utils.Utils; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; /** * FailureDetectorConfig simply holds all the data that was available to it upon @@ -567,6 +567,11 @@ public synchronized void addNode(Node node) { nodes.add(node); } + public synchronized void removeNode(Node node) { + Utils.notNull(node); + nodes.remove(node); + } + public StoreVerifier getStoreVerifier() { return storeVerifier; } From d2fae73cb9ee1e69d611d2ec6ae6b870f529e76e Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Thu, 22 Mar 2012 19:09:49 -0700 Subject: [PATCH 015/209] Combined node and status in a composite map in AbstractFailureDetector --- .../AbstractFailureDetector.java | 59 +++++++++++++------ 1 file changed, 40 insertions(+), 19 deletions(-) diff --git a/src/java/voldemort/cluster/failuredetector/AbstractFailureDetector.java b/src/java/voldemort/cluster/failuredetector/AbstractFailureDetector.java index 25a2ebc27c..e3328cbe0d 100644 --- a/src/java/voldemort/cluster/failuredetector/AbstractFailureDetector.java +++ b/src/java/voldemort/cluster/failuredetector/AbstractFailureDetector.java @@ -43,10 +43,9 @@ public abstract class AbstractFailureDetector implements FailureDetector { // simply a wrapper around a ConcurrentHashMap anyway :( protected final ConcurrentHashMap listeners; - protected final Map nodeStatusMap; - - // Also maintain the list of nodes by IDs (in order to handle host swaps) - protected final Map nodeMap; + // Maintain the list of nodes and their status by IDs (in order to handle + // host swaps) + protected final Map idNodeStatusMap; protected final Logger logger = Logger.getLogger(getClass().getName()); @@ -56,13 +55,13 @@ protected AbstractFailureDetector(FailureDetectorConfig failureDetectorConfig) { this.failureDetectorConfig = failureDetectorConfig; listeners = new ConcurrentHashMap(); - nodeStatusMap = new ConcurrentHashMap(); - nodeMap = new ConcurrentHashMap(); + idNodeStatusMap = new ConcurrentHashMap(); for(Node node: failureDetectorConfig.getNodes()) { - nodeStatusMap.put(node, createNodeStatus(failureDetectorConfig.getTime() - .getMilliseconds())); - nodeMap.put(node.getId(), node); + idNodeStatusMap.put(node.getId(), + new CompositeNodeStatus(node, + createNodeStatus(failureDetectorConfig.getTime() + .getMilliseconds()))); } } @@ -216,29 +215,28 @@ protected void setUnavailable(Node node, UnreachableStoreException e) { } protected NodeStatus getNodeStatus(Node node) { - NodeStatus nodeStatus = nodeStatusMap.get(node); - Node currentTrackedNode = nodeMap.get(node.getId()); + NodeStatus nodeStatus = null; + CompositeNodeStatus currentNodeStatus = idNodeStatusMap.get(node.getId()); - if(nodeStatus == null || !currentTrackedNode.isEqualState(node)) { + if(currentNodeStatus == null || !currentNodeStatus.getNode().isEqualState(node)) { if(logger.isEnabledFor(Level.WARN)) logger.warn("creating new node status for node " + node.getId() + " for failure detector"); // If the host is being replaced, remove old tracking information - if(nodeStatus != null) { - nodeStatusMap.remove(currentTrackedNode); - nodeMap.remove(currentTrackedNode); - failureDetectorConfig.removeNode(currentTrackedNode); + if(currentNodeStatus != null) { + idNodeStatusMap.remove(currentNodeStatus); + failureDetectorConfig.removeNode(currentNodeStatus.getNode()); } nodeStatus = createNodeStatus(failureDetectorConfig.getTime().getMilliseconds()); - nodeStatusMap.put(node, nodeStatus); - nodeMap.put(node.getId(), node); + idNodeStatusMap.put(node.getId(), new CompositeNodeStatus(node, nodeStatus)); if(!failureDetectorConfig.getNodes().contains(node)) { failureDetectorConfig.addNode(node); } - } + } else + nodeStatus = currentNodeStatus.getStatus(); return nodeStatus; } @@ -276,4 +274,27 @@ private boolean setAvailable(NodeStatus nodeStatus, boolean isAvailable) { } } + private class CompositeNodeStatus { + + private Node node; + private NodeStatus status; + + CompositeNodeStatus(Node node, NodeStatus status) { + this.node = node; + this.status = status; + } + + public void setValues(Node node, NodeStatus status) { + this.node = node; + this.status = status; + } + + public Node getNode() { + return this.node; + } + + public NodeStatus getStatus() { + return this.status; + } + } } From b470708d52e3b4256f98adc4e67e009d7950af15 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Thu, 29 Mar 2012 15:00:06 -0700 Subject: [PATCH 016/209] Fix race condition in client socket close --- src/java/voldemort/store/socket/SocketStore.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/java/voldemort/store/socket/SocketStore.java b/src/java/voldemort/store/socket/SocketStore.java index 4c44292bcd..0886341521 100644 --- a/src/java/voldemort/store/socket/SocketStore.java +++ b/src/java/voldemort/store/socket/SocketStore.java @@ -240,10 +240,9 @@ public void close() throws VoldemortException { private T request(ClientRequest delegate, String operationName) { ClientRequestExecutor clientRequestExecutor = pool.checkout(destination); - + BlockingClientRequest blockingClientRequest = null; try { - BlockingClientRequest blockingClientRequest = new BlockingClientRequest(delegate, - timeoutMs); + blockingClientRequest = new BlockingClientRequest(delegate, timeoutMs); clientRequestExecutor.addClientRequest(blockingClientRequest, timeoutMs); blockingClientRequest.await(); return blockingClientRequest.getResult(); @@ -255,6 +254,10 @@ private T request(ClientRequest delegate, String operationName) { throw new UnreachableStoreException("Failure in " + operationName + " on " + destination + ": " + e.getMessage(), e); } finally { + if(blockingClientRequest != null && !blockingClientRequest.isComplete()) { + // close the executor if we timed out + clientRequestExecutor.close(); + } pool.checkin(destination, clientRequestExecutor); } } @@ -262,7 +265,7 @@ private T request(ClientRequest delegate, String operationName) { /** * This method handles submitting and then waiting for the request from the * server. It uses the ClientRequest API to actually write the request and - * then read back the response. This implementation will block for a + * then read back the response. This implementation will not block for a * response from the server. * * @param Return type From 1f057d7e3c68dc4d67387a5485ac6793d3feb8b7 Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Sat, 7 Apr 2012 11:47:00 -0700 Subject: [PATCH 017/209] fix two concurrency bugs during the termination of donor-based rebalancing --- .../admin/AdminServiceRequestHandler.java | 102 ++++++++------- .../server/rebalance/Rebalancer.java | 2 +- .../DonorBasedRebalanceAsyncOperation.java | 57 ++++++--- .../async/DonorBasedRebalancePusherSlave.java | 121 +++++++++--------- .../rebalance/AbstractRebalanceTest.java | 1 + 5 files changed, 151 insertions(+), 132 deletions(-) diff --git a/src/java/voldemort/server/protocol/admin/AdminServiceRequestHandler.java b/src/java/voldemort/server/protocol/admin/AdminServiceRequestHandler.java index 0254877d93..76263f9fb8 100644 --- a/src/java/voldemort/server/protocol/admin/AdminServiceRequestHandler.java +++ b/src/java/voldemort/server/protocol/admin/AdminServiceRequestHandler.java @@ -268,70 +268,74 @@ public StreamRequestHandler handleRequest(final DataInputStream inputStream, private VAdminProto.DeleteStoreRebalanceStateResponse handleDeleteStoreRebalanceState(VAdminProto.DeleteStoreRebalanceStateRequest request) { VAdminProto.DeleteStoreRebalanceStateResponse.Builder response = VAdminProto.DeleteStoreRebalanceStateResponse.newBuilder(); + synchronized(rebalancer) { + try { - try { - - int nodeId = request.getNodeId(); - String storeName = request.getStoreName(); + int nodeId = request.getNodeId(); + String storeName = request.getStoreName(); - logger.info("Removing rebalancing state for donor node " + nodeId + " and store " - + storeName); - RebalancePartitionsInfo info = metadataStore.getRebalancerState().find(nodeId); - if(info == null) { - throw new VoldemortException("Could not find state for donor node " + nodeId); - } + logger.info("Removing rebalancing state for donor node " + nodeId + " and store " + + storeName + " from stealer node " + metadataStore.getNodeId()); + RebalancePartitionsInfo info = metadataStore.getRebalancerState().find(nodeId); + if(info == null) { + throw new VoldemortException("Could not find state for donor node " + nodeId); + } - HashMap> replicaToPartition = info.getReplicaToAddPartitionList(storeName); - if(replicaToPartition == null) { - throw new VoldemortException("Could not find state for donor node " + nodeId - + " and store " + storeName); - } + HashMap> replicaToPartition = info.getReplicaToAddPartitionList(storeName); + if(replicaToPartition == null) { + throw new VoldemortException("Could not find state for donor node " + nodeId + + " and store " + storeName); + } - info.removeStore(storeName); - logger.info("Removed rebalancing state for donor node " + nodeId + " and store " - + storeName); + info.removeStore(storeName); + logger.info("Removed rebalancing state for donor node " + nodeId + " and store " + + storeName + " from stealer node " + metadataStore.getNodeId()); - if(info.getUnbalancedStoreList().isEmpty()) { - metadataStore.deleteRebalancingState(info); - logger.info("Removed entire rebalancing state for donor node " + nodeId); + if(info.getUnbalancedStoreList().isEmpty()) { + metadataStore.deleteRebalancingState(info); + logger.info("Removed entire rebalancing state for donor node " + nodeId + + " from stealer node " + metadataStore.getNodeId()); + } + } catch(VoldemortException e) { + response.setError(ProtoUtils.encodeError(errorCodeMapper, e)); + logger.error("handleDeleteStoreRebalanceState failed for request(" + + request.toString() + ")", e); } - } catch(VoldemortException e) { - response.setError(ProtoUtils.encodeError(errorCodeMapper, e)); - logger.error("handleDeleteStoreRebalanceState failed for request(" + request.toString() - + ")", e); } return response.build(); } public VAdminProto.RebalanceStateChangeResponse handleRebalanceStateChange(VAdminProto.RebalanceStateChangeRequest request) { - VAdminProto.RebalanceStateChangeResponse.Builder response = VAdminProto.RebalanceStateChangeResponse.newBuilder(); - try { - // Retrieve all values first - List rebalancePartitionsInfo = Lists.newArrayList(); - for(RebalancePartitionInfoMap map: request.getRebalancePartitionInfoListList()) { - rebalancePartitionsInfo.add(ProtoUtils.decodeRebalancePartitionInfoMap(map)); - } - - Cluster cluster = new ClusterMapper().readCluster(new StringReader(request.getClusterString())); - - boolean swapRO = request.getSwapRo(); - boolean changeClusterMetadata = request.getChangeClusterMetadata(); - boolean changeRebalanceState = request.getChangeRebalanceState(); - boolean rollback = request.getRollback(); + synchronized(rebalancer) { + try { + // Retrieve all values first + List rebalancePartitionsInfo = Lists.newArrayList(); + for(RebalancePartitionInfoMap map: request.getRebalancePartitionInfoListList()) { + rebalancePartitionsInfo.add(ProtoUtils.decodeRebalancePartitionInfoMap(map)); + } - rebalancer.rebalanceStateChange(cluster, - rebalancePartitionsInfo, - swapRO, - changeClusterMetadata, - changeRebalanceState, - rollback); - } catch(VoldemortException e) { - response.setError(ProtoUtils.encodeError(errorCodeMapper, e)); - logger.error("handleRebalanceStateChange failed for request(" + request.toString() - + ")", e); + Cluster cluster = new ClusterMapper().readCluster(new StringReader(request.getClusterString())); + + boolean swapRO = request.getSwapRo(); + boolean changeClusterMetadata = request.getChangeClusterMetadata(); + boolean changeRebalanceState = request.getChangeRebalanceState(); + boolean rollback = request.getRollback(); + + rebalancer.rebalanceStateChange(cluster, + rebalancePartitionsInfo, + swapRO, + changeClusterMetadata, + changeRebalanceState, + rollback); + } catch(VoldemortException e) { + response.setError(ProtoUtils.encodeError(errorCodeMapper, e)); + logger.error("handleRebalanceStateChange failed for request(" + request.toString() + + ")", e); + } } + return response.build(); } diff --git a/src/java/voldemort/server/rebalance/Rebalancer.java b/src/java/voldemort/server/rebalance/Rebalancer.java index 96e412fbbb..13425f45c0 100644 --- a/src/java/voldemort/server/rebalance/Rebalancer.java +++ b/src/java/voldemort/server/rebalance/Rebalancer.java @@ -350,7 +350,7 @@ public int rebalanceNodeOnDonor(final List stealInfos) // Get a lock for the stealer node if(!acquireRebalancingPermit(stealerNodeId)) { throw new VoldemortException("Node " + metadataStore.getNodeId() - + " is already trying to steal from " + + " is already trying to push to stealer node " + stealerNodeId); } diff --git a/src/java/voldemort/server/rebalance/async/DonorBasedRebalanceAsyncOperation.java b/src/java/voldemort/server/rebalance/async/DonorBasedRebalanceAsyncOperation.java index b15882a635..ec20b36139 100644 --- a/src/java/voldemort/server/rebalance/async/DonorBasedRebalanceAsyncOperation.java +++ b/src/java/voldemort/server/rebalance/async/DonorBasedRebalanceAsyncOperation.java @@ -21,11 +21,15 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Set; +import java.util.Map.Entry; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import voldemort.VoldemortException; @@ -34,7 +38,6 @@ import voldemort.cluster.Cluster; import voldemort.server.StoreRepository; import voldemort.server.VoldemortConfig; -import voldemort.server.protocol.admin.AsyncOperationService; import voldemort.server.rebalance.Rebalancer; import voldemort.server.rebalance.VoldemortRebalancingException; import voldemort.store.StorageEngine; @@ -59,9 +62,12 @@ */ public class DonorBasedRebalanceAsyncOperation extends RebalanceAsyncOperation { - public final static Pair> END = Pair.create(null, null); + public static final Pair> END = Pair.create(new ByteArray("END".getBytes()), + new Versioned("END".getBytes())); + public static final Pair> BREAK = Pair.create(new ByteArray("BREAK".getBytes()), + new Versioned("BREAK".getBytes())); - // Batch 500 entries for each fetchUpdate call. + // Batch 1000 entries for each fetchUpdate call. private static final int FETCHUPDATE_BATCH_SIZE = 1000; // Print scanned entries every 100k private static final int SCAN_PROGRESS_COUNT = 100000; @@ -75,7 +81,7 @@ public class DonorBasedRebalanceAsyncOperation extends RebalanceAsyncOperation { private final HashMultimap>>> storeToNodePartitionMapping; - private final AsyncOperationService pushSlavesExecutor; + private final ExecutorService pushSlavesExecutor; private Map> updatePushSlavePool; private HashMultimap>>> groupByStores(List stealInfos) { @@ -107,7 +113,14 @@ public DonorBasedRebalanceAsyncOperation(Rebalancer rebalancer, // Group the plans by the store names this.storeToNodePartitionMapping = groupByStores(stealInfos); - pushSlavesExecutor = rebalancer.getAsyncOperationService(); + pushSlavesExecutor = Executors.newCachedThreadPool(new ThreadFactory() { + + public Thread newThread(Runnable r) { + Thread thread = new Thread(r); + thread.setName(r.getClass().getName()); + return thread; + } + }); updatePushSlavePool = Collections.synchronizedMap(new HashMap>()); } @@ -268,18 +281,15 @@ private void rebalanceStore(final String storeName, final SynchronousQueue>> queue = new SynchronousQueue>>(); nodeToQueue.put(tuple.getFirst(), queue); - int jobId = pushSlavesExecutor.getUniqueRequestId(); String jobName = "DonorBasedRebalancePusherSlave for store " + storeName + " on node " + tuple.getFirst(); - DonorBasedRebalancePusherSlave updatePushSlave = new DonorBasedRebalancePusherSlave(jobId, - jobName, - tuple.getFirst(), + DonorBasedRebalancePusherSlave updatePushSlave = new DonorBasedRebalancePusherSlave(tuple.getFirst(), queue, storeName, adminClient); storePushSlaves.add(updatePushSlave); - pushSlavesExecutor.submitOperation(jobId, updatePushSlave); - logger.info("Submitted donor-based pusher job: id=" + jobId + " name=" + jobName); + pushSlavesExecutor.execute(updatePushSlave); + logger.info("Started a thread for " + jobName); } fetchEntriesForStealers(storageEngine, @@ -339,7 +349,7 @@ private void putAll(List dests, fetched[nodeId]++; nodeToQueue.get(nodeId).put(Pair.create(key, value)); if(0 == fetched[nodeId] % FETCHUPDATE_BATCH_SIZE) { - nodeToQueue.get(nodeId).put(END); + nodeToQueue.get(nodeId).put(BREAK); } } } @@ -371,18 +381,27 @@ private void terminateAllSlaves(List updatePushS it.next().requestCompletion(); } - // wait for all async slave to finish - for(Iterator it = updatePushSlavePool.iterator(); it.hasNext();) { - it.next().waitCompletion(); + // signal and wait for all slaves to finish + pushSlavesExecutor.shutdown(); + try { + if(pushSlavesExecutor.awaitTermination(30, TimeUnit.MINUTES)) { + logger.info("All DonorBasedRebalancePushSlaves terminated successfully."); + } else { + logger.warn("Timed out while waiting for pusher slaves to shutdown!!!"); + } + } catch(InterruptedException e) { + logger.warn("Interrupted while waiting for pusher slaves to shutdown!!!"); } - logger.info("All DonorBasedRebalancePushSlaves terminated successfully."); + logger.info("DonorBasedRebalancingOperation existed."); } private void terminateAllSlavesAsync(List updatePushSlavePool) { logger.info("Terminating DonorBasedRebalancePushSlaves asynchronously"); for(Iterator it = updatePushSlavePool.iterator(); it.hasNext();) { - it.next().setCompletion(); + it.next().requestCompletion(); } + pushSlavesExecutor.shutdownNow(); + logger.info("DonorBasedRebalancingAsyncOperation existed."); } @Override @@ -395,4 +414,4 @@ public void stop() { } executors.shutdownNow(); } -} +} \ No newline at end of file diff --git a/src/java/voldemort/server/rebalance/async/DonorBasedRebalancePusherSlave.java b/src/java/voldemort/server/rebalance/async/DonorBasedRebalancePusherSlave.java index fddedbed9c..240205d06c 100644 --- a/src/java/voldemort/server/rebalance/async/DonorBasedRebalancePusherSlave.java +++ b/src/java/voldemort/server/rebalance/async/DonorBasedRebalancePusherSlave.java @@ -4,13 +4,11 @@ import java.util.ArrayList; import java.util.NoSuchElementException; import java.util.concurrent.BlockingQueue; -import java.util.concurrent.atomic.AtomicBoolean; import org.apache.log4j.Logger; import voldemort.VoldemortException; import voldemort.client.protocol.admin.AdminClient; -import voldemort.server.protocol.admin.AsyncOperation; import voldemort.utils.ByteArray; import voldemort.utils.ClosableIterator; import voldemort.utils.Pair; @@ -18,7 +16,7 @@ import com.google.common.collect.Lists; -public class DonorBasedRebalancePusherSlave extends AsyncOperation { +public class DonorBasedRebalancePusherSlave implements Runnable { protected final static Logger logger = Logger.getLogger(DonorBasedRebalancePusherSlave.class); @@ -26,28 +24,24 @@ public class DonorBasedRebalancePusherSlave extends AsyncOperation { private BlockingQueue>> queue; private String storeName; private AdminClient adminClient; - private ResumableIterator>> nodeIterator = new ResumableIterator>>(); - private AtomicBoolean stopRequested; + private ResumableIterator>> nodeIterator; - public DonorBasedRebalancePusherSlave(int id, - String description, - int nodeId, + public DonorBasedRebalancePusherSlave(int nodeId, BlockingQueue>> queue, String storeName, AdminClient adminClient) { - super(id, description); this.nodeId = nodeId; this.queue = queue; this.storeName = storeName; this.adminClient = adminClient; - this.stopRequested = new AtomicBoolean(false); + nodeIterator = new ResumableIterator>>(); } - @Override - public void operate() throws Exception { + public void run() throws VoldemortException { + boolean needWait = false; logger.info("DonorBasedRebalancePusherSlave begains to send partitions for store " + storeName + " to node " + nodeId); - while(!isStopRequest()) { + while(!nodeIterator.done) { try { nodeIterator.reset(); adminClient.updateEntries(nodeId, storeName, nodeIterator, null); @@ -61,41 +55,38 @@ public void operate() throws Exception { + " to remote node " + nodeId + ". Will retry again after 5 minutes"); logger.error(e.getCause()); - Thread.sleep(30000); + needWait = true; } else { throw e; } } + + if(needWait) { + try { + // sleep for 5 minutes if exception occur while communicate + // with remote node + logger.info("waiting for 5 minutes for the remote node to recover"); + Thread.sleep(30000); + needWait = false; + } catch(InterruptedException e) { + // continue + } + } } - setCompletion(); + logger.info("DonorBasedRebalancePusherSlave finished sending partitions for store " + storeName + " to node " + nodeId); } - public void setStopRequest() { - stopRequested.set(true); - } - - public boolean isStopRequest() { - return stopRequested.get(); - } - - @Override - public void stop() { - requestCompletion(); - } - /** - * This function will set the request for stop first; Then insert 'END' into - * the queue so slave will return from updateEntries. Noted that this order - * shall not be changed or the slave will enter updateEntries again. + * This function inserts 'END' into the queue so slave will return from + * updateEntries. * * @param immediateTerminate * @param notifySlave */ - public synchronized void requestCompletion() { + public void requestCompletion() { try { - setStopRequest(); queue.put(DonorBasedRebalanceAsyncOperation.END); } catch(InterruptedException e) { logger.info("Unable to send termination message to pusher slave for node " + nodeId @@ -103,28 +94,10 @@ public synchronized void requestCompletion() { } } - public synchronized void setCompletion() { - getStatus().setComplete(true); - notifyAll(); - } - - public synchronized void waitCompletion() { - while(!getStatus().isComplete()) { - try { - logger.info("Waiting for the completion, with 10s timeout, of pusher slave for " - + getStatus().getDescription() + " with id=" + getStatus().getId()); - // check for status every 10 seconds - wait(10000); - } catch(InterruptedException e) { - logger.info("Existing wait loop due to interrupt."); - break; - } - } - } - // It will always Iterator through 'tentativeList' before iterating 'queue' class ResumableIterator implements ClosableIterator>> { + private boolean done = false; private boolean recoveryModeOn = false; private int recoveryPosition = 0; private Pair> currentElem = null; @@ -153,19 +126,30 @@ public void reset() { this.currentElem = null; } - // return when something is available, blocked otherwise public boolean hasNext() { boolean hasNext = false; - while(null == currentElem) { - try { - currentElem = getNextElem(); - } catch(InterruptedException e) { - logger.info("hasNext is interrupted while waiting for the next elem, existing..."); - break; + if(!done) { + while(null == currentElem) { + try { + currentElem = getNextElem(); + } catch(InterruptedException e) { + logger.info("hasNext is interrupted while waiting for the next elem, existing..."); + break; + } + } + + // regular event + if(null != currentElem + && !currentElem.equals(DonorBasedRebalanceAsyncOperation.END) + && !currentElem.equals(DonorBasedRebalanceAsyncOperation.BREAK)) { + hasNext = true; + } + + // this is the last element returned by this iterator + if(currentElem != null && currentElem.equals(DonorBasedRebalanceAsyncOperation.END)) { + done = true; + hasNext = false; } - } - if(null != currentElem && !currentElem.equals(DonorBasedRebalanceAsyncOperation.END)) { - hasNext = true; } return hasNext; } @@ -173,6 +157,10 @@ public boolean hasNext() { // return the element when one or more is available, blocked // otherwise public Pair> next() { + if(done) { + throw new NoSuchElementException(); + } + while(null == currentElem) { try { currentElem = getNextElem(); @@ -180,10 +168,17 @@ public Pair> next() { logger.info("next is interrupted while waiting for the next elem, existing..."); break; } - if(null == currentElem || currentElem.equals(DonorBasedRebalanceAsyncOperation.END)) { + if(null == currentElem || currentElem.equals(DonorBasedRebalanceAsyncOperation.END) + || currentElem.equals(DonorBasedRebalanceAsyncOperation.BREAK)) { throw new NoSuchElementException(); } } + + // this is the last element returned by this iterator + if(currentElem != null && currentElem.equals(DonorBasedRebalanceAsyncOperation.END)) { + done = true; + } + Pair> returnValue = currentElem; currentElem = null; return returnValue; diff --git a/test/unit/voldemort/client/rebalance/AbstractRebalanceTest.java b/test/unit/voldemort/client/rebalance/AbstractRebalanceTest.java index ddbceb1a66..4eea8e554c 100644 --- a/test/unit/voldemort/client/rebalance/AbstractRebalanceTest.java +++ b/test/unit/voldemort/client/rebalance/AbstractRebalanceTest.java @@ -666,6 +666,7 @@ public void testRWRebalanceFourNodes() throws Exception { config.setDeleteAfterRebalancingEnabled(true); config.setStealerBasedRebalancing(!useDonorBased()); config.setPrimaryPartitionBatchSize(100); + config.setMaxParallelRebalancing(5); RebalanceController rebalanceClient = new RebalanceController(getBootstrapUrl(currentCluster, 0), config); From a22e6f073a68a6821f28da913160ac8b99c6fa6f Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Tue, 10 Apr 2012 13:52:12 -0700 Subject: [PATCH 018/209] Added rollback capability to the Admin tool --- src/java/voldemort/VoldemortAdminTool.java | 36 +++++++++++++++++++++- 1 file changed, 35 insertions(+), 1 deletion(-) diff --git a/src/java/voldemort/VoldemortAdminTool.java b/src/java/voldemort/VoldemortAdminTool.java index 22d61febeb..195df6dec2 100644 --- a/src/java/voldemort/VoldemortAdminTool.java +++ b/src/java/voldemort/VoldemortAdminTool.java @@ -208,6 +208,14 @@ public static void main(String[] args) throws Exception { parser.accepts("backup-incremental", "Perform an incremental backup for point-in-time recovery." + " By default backup has latest consistent snapshot."); + parser.accepts("rollback", "rollback a store") + .withRequiredArg() + .describedAs("store-name") + .ofType(String.class); + parser.accepts("version", "version of store to rollback to") + .withRequiredArg() + .describedAs("version") + .ofType(Long.class); OptionSet options = parser.parse(args); @@ -224,7 +232,7 @@ public static void main(String[] args) throws Exception { || options.has("ro-metadata") || options.has("set-metadata") || options.has("get-metadata") || options.has("check-metadata") || options.has("key-distribution")) || options.has("truncate") || options.has("clear-rebalancing-metadata") - || options.has("async") || options.has("native-backup"))) { + || options.has("async") || options.has("native-backup") || options.has("rollback"))) { System.err.println("Missing required arguments: " + Joiner.on(", ").join(missing)); printHelp(System.err, parser); System.exit(1); @@ -292,6 +300,12 @@ public static void main(String[] args) throws Exception { } ops += "n"; } + if(options.has("rollback")) { + if(!options.has("version")) { + Utils.croak("A version of the read-only store must be specified with rollback option"); + } + ops += "o"; + } if(ops.length() < 1) { Utils.croak("At least one of (delete-partitions, restore, add-node, fetch-entries, " + "fetch-keys, add-stores, delete-store, update-entries, get-metadata, ro-metadata, " @@ -456,12 +470,30 @@ public static void main(String[] args) throws Exception { options.has("backup-verify"), options.has("backup-incremental")); } + if(ops.contains("o")) { + String storeName = (String) options.valueOf("rollback"); + long pushVersion = (Long) options.valueOf("version"); + executeRollback(nodeId, storeName, pushVersion, adminClient); + } } catch(Exception e) { e.printStackTrace(); Utils.croak(e.getMessage()); } } + private static void executeRollback(Integer nodeId, + String storeName, + long pushVersion, + AdminClient adminClient) { + if(nodeId < 0) { + for(Node node: adminClient.getAdminClientCluster().getNodes()) { + adminClient.rollbackStore(node.getId(), storeName, pushVersion); + } + } else { + adminClient.rollbackStore(nodeId, storeName, pushVersion); + } + } + private static void executeRepairJob(Integer nodeId, AdminClient adminClient) { if(nodeId < 0) { for(Node node: adminClient.getAdminClientCluster().getNodes()) { @@ -573,6 +605,8 @@ public static void printHelp(PrintStream stream, OptionParser parser) throws IOE stream.println("\t5) Backup bdb data natively"); stream.println("\t\t./bin/voldemort-admin-tool.sh --native-backup [store] --backup-dir [outdir] " + "--backup-timeout [mins] [--backup-verify] [--backup-incremental] --url [url] --node [node-id]"); + stream.println("\t6) Rollback a read-only store for the specified version"); + stream.println("\t\t./bin/voldemort-admin-tool.sh --rollback [store-name] --url [url] --node [node-id] --version [version-num] "); parser.printHelpOn(stream); } From 136473c31f5c18a021d5028e631322fab2122ec1 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 11 Apr 2012 12:52:12 -0700 Subject: [PATCH 019/209] Error count and max getall count jmx end points --- .../client/AbstractStoreClientFactory.java | 3 +- .../voldemort/store/routed/PipelineData.java | 14 +++ .../store/routed/PipelineRoutedStats.java | 103 ++++++++++++++++++ .../store/routed/PipelineRoutedStore.java | 65 +++++++++-- .../store/routed/RoutedStoreFactory.java | 26 ++++- .../action/PerformParallelDeleteRequests.java | 1 + .../action/PerformParallelGetAllRequests.java | 1 + .../action/PerformParallelPutRequests.java | 1 + .../action/PerformParallelRequests.java | 1 + .../voldemort/store/stats/RequestCounter.java | 22 +++- .../voldemort/store/stats/StoreStats.java | 4 + .../voldemort/store/stats/StoreStatsJmx.java | 34 +++--- 12 files changed, 247 insertions(+), 28 deletions(-) create mode 100644 src/java/voldemort/store/routed/PipelineRoutedStats.java diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index 94664d08b0..fb82936c50 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -214,7 +214,8 @@ public Store getRawStore(String storeName, nonblockingSlopStores, repairReads, clientZoneId, - getFailureDetector()); + getFailureDetector(), + isJmxEnabled); store = new LoggingStore(store); if(isJmxEnabled) { diff --git a/src/java/voldemort/store/routed/PipelineData.java b/src/java/voldemort/store/routed/PipelineData.java index 8f6ea313a0..432c8904e7 100644 --- a/src/java/voldemort/store/routed/PipelineData.java +++ b/src/java/voldemort/store/routed/PipelineData.java @@ -73,6 +73,12 @@ public abstract class PipelineData { protected List replicationSet; + protected PipelineRoutedStats stats; + + public void setStats(PipelineRoutedStats stats) { + this.stats = stats; + } + public List getReplicationSet() { return replicationSet; } @@ -117,6 +123,7 @@ public VoldemortException getFatalError() { } public void setFatalError(VoldemortException fatalError) { + reportException(fatalError); this.fatalError = fatalError; } @@ -139,6 +146,7 @@ public List getFailures() { */ public void recordFailure(Exception e) { + reportException(e); this.failures.add(e); } @@ -157,4 +165,10 @@ public String getStoreName() { public void setStoreName(String storeName) { this.storeName = storeName; } + + public void reportException(Exception e) { + if(stats != null) { + stats.reportException(e); + } + } } diff --git a/src/java/voldemort/store/routed/PipelineRoutedStats.java b/src/java/voldemort/store/routed/PipelineRoutedStats.java new file mode 100644 index 0000000000..8f02c86641 --- /dev/null +++ b/src/java/voldemort/store/routed/PipelineRoutedStats.java @@ -0,0 +1,103 @@ +package voldemort.store.routed; + +import java.util.Iterator; +import java.util.Map.Entry; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; + +import voldemort.annotations.jmx.JmxGetter; +import voldemort.store.InsufficientOperationalNodesException; +import voldemort.store.InsufficientZoneResponsesException; +import voldemort.store.InvalidMetadataException; +import voldemort.store.StoreTimeoutException; +import voldemort.store.UnreachableStoreException; + +/** + * Tracks all the exceptions we see, down at the routing layer also including + * ones that will be eventually propagated up to the client from the routing + * layer + * + */ +public class PipelineRoutedStats { + + private ConcurrentHashMap, AtomicLong> errCountMap; + private AtomicLong severeExceptionCount; + private AtomicLong benignExceptionCount; + + PipelineRoutedStats() { + errCountMap = new ConcurrentHashMap, AtomicLong>(); + errCountMap.put(InvalidMetadataException.class, new AtomicLong(0)); + errCountMap.put(InsufficientOperationalNodesException.class, new AtomicLong(0)); + errCountMap.put(InsufficientZoneResponsesException.class, new AtomicLong(0)); + errCountMap.put(UnreachableStoreException.class, new AtomicLong(0)); + errCountMap.put(StoreTimeoutException.class, new AtomicLong(0)); + + severeExceptionCount = new AtomicLong(0); + benignExceptionCount = new AtomicLong(0); + } + + @JmxGetter(name = "numSevereExceptions", description = "Number of exceptions considered serious errors") + public long getNumSevereExceptions() { + return severeExceptionCount.get(); + } + + @JmxGetter(name = "numBenignExceptions", description = "Number of exceptions considered benign") + public long getNumBenignExceptions() { + return benignExceptionCount.get(); + } + + @JmxGetter(name = "numInsufficientOperationalNodesExceptions", description = "Number of client operations failed due to sufficient nodes not being up") + public long getNumInsufficientOperationalNodesExceptions() { + return errCountMap.get(InsufficientOperationalNodesException.class).get(); + } + + @JmxGetter(name = "numInsufficientZoneResponsesExceptions", description = "Number of client operations failed due to sufficient nodes not up across zones") + public long getNumInsufficientZoneResponsesExceptions() { + return errCountMap.get(InsufficientZoneResponsesException.class).get(); + } + + @JmxGetter(name = "numInvalidMetadataExceptions", description = "Number of times the metadata was invalid at the client") + public long getNumInvalidMetadataExceptions() { + return errCountMap.get(InvalidMetadataException.class).get(); + } + + @JmxGetter(name = "numUnreachableStoreExceptions", description = "Number of requests incomplete since some server could not be reached") + public long getNumUnreachableStoreExceptions() { + return errCountMap.get(UnreachableStoreException.class).get(); + } + + @JmxGetter(name = "numStoreTimeoutExceptions", description = "Number of requests timed out since some server was overloaded/unavailable") + public long getNumStoreTimeoutExceptions() { + return errCountMap.get(StoreTimeoutException.class).get(); + } + + @JmxGetter(name = "getExceptionCountsAsString", description = "Returns counts of all the Exceptions seen so far as a string") + public String getExceptionCountsAsString() { + StringBuilder result = new StringBuilder(); + Iterator, AtomicLong>> itr = errCountMap.entrySet() + .iterator(); + while(itr.hasNext()) { + Entry, AtomicLong> pair = itr.next(); + result.append(pair.getKey().getName() + ":" + pair.getValue().get() + "\n"); + } + return result.toString(); + } + + public void reportException(Exception e) { + if(isSevere(e)) + severeExceptionCount.incrementAndGet(); + else + benignExceptionCount.incrementAndGet(); + errCountMap.putIfAbsent(e.getClass(), new AtomicLong(0)); + errCountMap.get(e.getClass()).incrementAndGet(); + } + + private boolean isSevere(Exception ve) { + if(ve instanceof InsufficientOperationalNodesException + || ve instanceof InsufficientZoneResponsesException + || ve instanceof InvalidMetadataException) + return true; + else + return false; + } +} diff --git a/src/java/voldemort/store/routed/PipelineRoutedStore.java b/src/java/voldemort/store/routed/PipelineRoutedStore.java index 61125e5edb..c5aa1ba06b 100644 --- a/src/java/voldemort/store/routed/PipelineRoutedStore.java +++ b/src/java/voldemort/store/routed/PipelineRoutedStore.java @@ -21,6 +21,7 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import voldemort.VoldemortException; import voldemort.cluster.Cluster; @@ -55,6 +56,7 @@ import voldemort.store.slop.strategy.HintedHandoffStrategyFactory; import voldemort.utils.ByteArray; import voldemort.utils.ByteUtils; +import voldemort.utils.JmxUtils; import voldemort.utils.SystemTime; import voldemort.versioning.Version; import voldemort.versioning.Versioned; @@ -66,12 +68,16 @@ */ public class PipelineRoutedStore extends RoutedStore { + private static AtomicInteger jmxIdCounter = new AtomicInteger(0); + private final Map nonblockingStores; private final Map> slopStores; private final Map nonblockingSlopStores; private final HintedHandoffStrategy handoffStrategy; private Zone clientZone; private boolean zoneRoutingEnabled; + private PipelineRoutedStats stats; + private final int jmxId; /** * Create a PipelineRoutedStore @@ -97,7 +103,8 @@ public PipelineRoutedStore(String name, boolean repairReads, int clientZoneId, long timeoutMs, - FailureDetector failureDetector) { + FailureDetector failureDetector, + boolean jmxEnabled) { super(name, innerStores, cluster, @@ -113,7 +120,7 @@ public PipelineRoutedStore(String name, } else { zoneRoutingEnabled = false; } - + this.jmxId = jmxIdCounter.getAndIncrement(); this.nonblockingStores = new ConcurrentHashMap(nonblockingStores); this.slopStores = slopStores; if(storeDef.hasHintedHandoffStrategyType()) { @@ -123,6 +130,13 @@ public PipelineRoutedStore(String name, } else { this.handoffStrategy = null; } + + if(jmxEnabled) { + stats = new PipelineRoutedStats(); + JmxUtils.registerMbean(stats, + JmxUtils.createObjectName(JmxUtils.getPackageName(stats.getClass()), + getName() + jmxId())); + } } public List> get(final ByteArray key, final byte[] transforms) { @@ -133,6 +147,7 @@ public List> get(final ByteArray key, final byte[] transforms) pipelineData.setZonesRequired(storeDef.getZoneCountReads()); else pipelineData.setZonesRequired(null); + pipelineData.setStats(stats); final Pipeline pipeline = new Pipeline(Operation.GET, timeoutMs, TimeUnit.MILLISECONDS); boolean allowReadRepair = repairReads && transforms == null; @@ -204,7 +219,12 @@ public List> request(Store store) { + ByteUtils.toHexString(key.get())); } - pipeline.execute(); + try { + pipeline.execute(); + } catch(VoldemortException e) { + stats.reportException(e); + throw e; + } if(pipelineData.getFatalError() != null) throw pipelineData.getFatalError(); @@ -233,8 +253,9 @@ public Map>> getAll(Iterable keys, pipelineData.setZonesRequired(storeDef.getZoneCountReads()); else pipelineData.setZonesRequired(null); - Pipeline pipeline = new Pipeline(Operation.GET_ALL, timeoutMs, TimeUnit.MILLISECONDS); + pipelineData.setStats(stats); + Pipeline pipeline = new Pipeline(Operation.GET_ALL, timeoutMs, TimeUnit.MILLISECONDS); pipeline.addEventAction(Event.STARTED, new GetAllConfigureNodes(pipelineData, Event.CONFIGURED, @@ -280,7 +301,12 @@ public Map>> getAll(Iterable keys, logger.debug("Operation " + pipeline.getOperation().getSimpleName() + " Keys " + keyStr.toString()); } - pipeline.execute(); + try { + pipeline.execute(); + } catch(VoldemortException e) { + stats.reportException(e); + throw e; + } if(pipelineData.getFatalError() != null) throw pipelineData.getFatalError(); @@ -296,6 +322,7 @@ public List getVersions(final ByteArray key) { pipelineData.setZonesRequired(storeDef.getZoneCountReads()); else pipelineData.setZonesRequired(null); + pipelineData.setStats(stats); Pipeline pipeline = new Pipeline(Operation.GET_VERSIONS, timeoutMs, TimeUnit.MILLISECONDS); StoreRequest> blockingStoreRequest = new StoreRequest>() { @@ -352,7 +379,12 @@ public List request(Store store) { logger.debug("Operation " + pipeline.getOperation().getSimpleName() + "Key " + ByteUtils.toHexString(key.get())); } - pipeline.execute(); + try { + pipeline.execute(); + } catch(VoldemortException e) { + stats.reportException(e); + throw e; + } if(pipelineData.getFatalError() != null) throw pipelineData.getFatalError(); @@ -374,6 +406,8 @@ public boolean delete(final ByteArray key, final Version version) throws Voldemo else pipelineData.setZonesRequired(null); pipelineData.setStoreName(name); + pipelineData.setStats(stats); + Pipeline pipeline = new Pipeline(Operation.DELETE, timeoutMs, TimeUnit.MILLISECONDS); pipeline.setEnableHintedHandoff(isHintedHandoffEnabled()); @@ -428,7 +462,12 @@ public boolean delete(final ByteArray key, final Version version) throws Voldemo logger.debug("Operation " + pipeline.getOperation().getSimpleName() + " Key " + ByteUtils.toHexString(key.get())); } - pipeline.execute(); + try { + pipeline.execute(); + } catch(VoldemortException e) { + stats.reportException(e); + throw e; + } if(pipelineData.getFatalError() != null) throw pipelineData.getFatalError(); @@ -455,6 +494,7 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) pipelineData.setZonesRequired(null); pipelineData.setStartTimeNs(System.nanoTime()); pipelineData.setStoreName(name); + pipelineData.setStats(stats); Pipeline pipeline = new Pipeline(Operation.PUT, timeoutMs, TimeUnit.MILLISECONDS); pipeline.setEnableHintedHandoff(isHintedHandoffEnabled()); @@ -531,7 +571,12 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) logger.debug("Operation " + pipeline.getOperation().getSimpleName() + " Key " + ByteUtils.toHexString(key.get())); } - pipeline.execute(); + try { + pipeline.execute(); + } catch(VoldemortException e) { + stats.reportException(e); + throw e; + } if(pipelineData.getFatalError() != null) throw pipelineData.getFatalError(); @@ -555,4 +600,8 @@ public void close() { super.close(); } + /* Give a unique id to avoid jmx clashes */ + private String jmxId() { + return jmxId == 0 ? "" : Integer.toString(jmxId); + } } diff --git a/src/java/voldemort/store/routed/RoutedStoreFactory.java b/src/java/voldemort/store/routed/RoutedStoreFactory.java index 52093be256..0a608bac4d 100644 --- a/src/java/voldemort/store/routed/RoutedStoreFactory.java +++ b/src/java/voldemort/store/routed/RoutedStoreFactory.java @@ -58,6 +58,28 @@ public RoutedStore create(Cluster cluster, boolean repairReads, int clientZoneId, FailureDetector failureDetector) { + return create(cluster, + storeDefinition, + nodeStores, + nonblockingStores, + slopStores, + nonblockingSlopStores, + repairReads, + clientZoneId, + failureDetector, + false); + } + + public RoutedStore create(Cluster cluster, + StoreDefinition storeDefinition, + Map> nodeStores, + Map nonblockingStores, + Map> slopStores, + Map nonblockingSlopStores, + boolean repairReads, + int clientZoneId, + FailureDetector failureDetector, + boolean jmxEnabled) { if(isPipelineRoutedStoreEnabled) { return new PipelineRoutedStore(storeDefinition.getName(), nodeStores, @@ -69,7 +91,8 @@ public RoutedStore create(Cluster cluster, repairReads, clientZoneId, routingTimeoutMs, - failureDetector); + failureDetector, + jmxEnabled); } else { if(storeDefinition.getRoutingStrategyType() .compareTo(RoutingStrategyType.ZONE_STRATEGY) == 0) { @@ -104,7 +127,6 @@ public RoutedStore create(Cluster cluster, for(Map.Entry> entry: nodeStores.entrySet()) nonblockingStores.put(entry.getKey(), toNonblockingStore(entry.getValue())); - return create(cluster, storeDefinition, nodeStores, diff --git a/src/java/voldemort/store/routed/action/PerformParallelDeleteRequests.java b/src/java/voldemort/store/routed/action/PerformParallelDeleteRequests.java index 890bcac55a..7f30c90a9b 100644 --- a/src/java/voldemort/store/routed/action/PerformParallelDeleteRequests.java +++ b/src/java/voldemort/store/routed/action/PerformParallelDeleteRequests.java @@ -128,6 +128,7 @@ public void requestComplete(Object result, long requestTime) { if(pipeline.isFinished() && response.getValue() instanceof Exception && !(response.getValue() instanceof ObsoleteVersionException)) { if(response.getValue() instanceof InvalidMetadataException) { + pipelineData.reportException((InvalidMetadataException) response.getValue()); logger.warn("Received invalid metadata problem after a successful " + pipeline.getOperation().getSimpleName() + " call on node " + node.getId() + ", store '" diff --git a/src/java/voldemort/store/routed/action/PerformParallelGetAllRequests.java b/src/java/voldemort/store/routed/action/PerformParallelGetAllRequests.java index c9c5a85fd1..a6bc581a7d 100644 --- a/src/java/voldemort/store/routed/action/PerformParallelGetAllRequests.java +++ b/src/java/voldemort/store/routed/action/PerformParallelGetAllRequests.java @@ -98,6 +98,7 @@ public void requestComplete(Object result, long requestTime) { // responses below. if(pipeline.isFinished() && response.getValue() instanceof Exception) if(response.getValue() instanceof InvalidMetadataException) { + pipelineData.reportException((InvalidMetadataException) response.getValue()); logger.warn("Received invalid metadata problem after a successful " + pipeline.getOperation().getSimpleName() + " call on node " + node.getId() + ", store '" diff --git a/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java b/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java index ebaa9d4fd7..c459057589 100644 --- a/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java +++ b/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java @@ -156,6 +156,7 @@ public void requestComplete(Object result, long requestTime) { if(pipeline.isFinished() && response.getValue() instanceof Exception && !(response.getValue() instanceof ObsoleteVersionException)) { if(response.getValue() instanceof InvalidMetadataException) { + pipelineData.reportException((InvalidMetadataException) response.getValue()); logger.warn("Received invalid metadata problem after a successful " + pipeline.getOperation().getSimpleName() + " call on node " + node.getId() + ", store '" diff --git a/src/java/voldemort/store/routed/action/PerformParallelRequests.java b/src/java/voldemort/store/routed/action/PerformParallelRequests.java index 5a5096bbd5..a5a0ef9aa6 100644 --- a/src/java/voldemort/store/routed/action/PerformParallelRequests.java +++ b/src/java/voldemort/store/routed/action/PerformParallelRequests.java @@ -115,6 +115,7 @@ public void requestComplete(Object result, long requestTime) { // responses below. if(pipeline.isFinished() && response.getValue() instanceof Exception) { if(response.getValue() instanceof InvalidMetadataException) { + pipelineData.reportException((InvalidMetadataException) response.getValue()); logger.warn("Received invalid metadata problem after a successful " + pipeline.getOperation().getSimpleName() + " call on node " + node.getId() + ", store '" diff --git a/src/java/voldemort/store/stats/RequestCounter.java b/src/java/voldemort/store/stats/RequestCounter.java index 19b0aeb85a..96c17bbf17 100644 --- a/src/java/voldemort/store/stats/RequestCounter.java +++ b/src/java/voldemort/store/stats/RequestCounter.java @@ -186,7 +186,9 @@ public void addRequest(long timeNS, Math.max(timeNS, oldv.maxLatencyNS), oldv.totalBytes + bytes, Math.max(oldv.maxBytes, bytes), - oldv.getAllAggregatedCount + getAllAggregatedCount); + oldv.getAllAggregatedCount + getAllAggregatedCount, + getAllAggregatedCount > oldv.getAllMaxCount ? getAllAggregatedCount + : oldv.getAllMaxCount); if(values.compareAndSet(oldv, newv)) return; } @@ -224,6 +226,13 @@ public long getGetAllAggregatedCount() { return getValidAccumulator().getAllAggregatedCount; } + /** + * Return the maximum number of keys returned across all getAll calls. + */ + public long getGetAllMaxCount() { + return getValidAccumulator().getAllMaxCount; + } + public int getQ95LatencyMs() { return q95LatencyMs; } @@ -242,13 +251,15 @@ private class Accumulator { // responses that have been returned final long getAllAggregatedCount; // GET_ALL: a single call to GET_ALL // can return multiple k-v pairs. - // Track total returned. + // Track total requested. + final long getAllMaxCount; // GET_ALL : track max number of keys + // requesed final long maxLatencyNS; final long maxBytes; // Maximum single value final long totalBytes; // Sum of all the values public Accumulator() { - this(RequestCounter.this.time.getMilliseconds(), 0, 0, 0, 0, 0, 0, 0, 0); + this(RequestCounter.this.time.getMilliseconds(), 0, 0, 0, 0, 0, 0, 0, 0, 0); } public Accumulator newWithTotal() { @@ -260,6 +271,7 @@ public Accumulator newWithTotal() { 0, 0, 0, + 0, 0); } @@ -271,7 +283,8 @@ public Accumulator(long startTimeMS, long maxLatencyNS, long totalBytes, long maxBytes, - long getAllAggregatedCount) { + long getAllAggregatedCount, + long getAllMaxCount) { this.startTimeMS = startTimeMS; this.count = count; this.totalTimeNS = totalTimeNS; @@ -281,6 +294,7 @@ public Accumulator(long startTimeMS, this.totalBytes = totalBytes; this.maxBytes = maxBytes; this.getAllAggregatedCount = getAllAggregatedCount; + this.getAllMaxCount = getAllMaxCount; } public double getAverageTimeNS() { diff --git a/src/java/voldemort/store/stats/StoreStats.java b/src/java/voldemort/store/stats/StoreStats.java index 07bd452cf7..e6cf34c99b 100644 --- a/src/java/voldemort/store/stats/StoreStats.java +++ b/src/java/voldemort/store/stats/StoreStats.java @@ -139,4 +139,8 @@ public double getGetAllAverageCount() { public long getGetAllAggregatedCount() { return counters.get(Tracked.GET_ALL).getGetAllAggregatedCount(); } + + public long getGetAllMaxCount() { + return counters.get(Tracked.GET_ALL).getGetAllMaxCount(); + } } diff --git a/src/java/voldemort/store/stats/StoreStatsJmx.java b/src/java/voldemort/store/stats/StoreStatsJmx.java index 4c05a2933f..6469ff2d4b 100644 --- a/src/java/voldemort/store/stats/StoreStatsJmx.java +++ b/src/java/voldemort/store/stats/StoreStatsJmx.java @@ -44,6 +44,11 @@ public double getAverageGetAllCount() { return stats.getGetAllAverageCount(); } + @JmxGetter(name = "maxGetAllCount", description = "The max number of keys in a GET_ALL request.") + public long getMaxGetAllCount() { + return stats.getGetAllMaxCount(); + } + @JmxGetter(name = "numberOfCallsToGet", description = "The number of calls to GET since the last reset.") public long getNumberOfCallsToGet() { return stats.getCount(Tracked.GET); @@ -134,22 +139,25 @@ public double getOperationThroughput() { @JmxGetter(name = "AllOperationThroughputInBytes", description = "Throughput of all operations in bytes.") public double getOperationThroghputInBytes() { - return stats.getThroughputInBytes(Tracked.GET) + stats.getThroughputInBytes(Tracked.GET_ALL) - + stats.getThroughputInBytes(Tracked.PUT); + return stats.getThroughputInBytes(Tracked.GET) + + stats.getThroughputInBytes(Tracked.GET_ALL) + + stats.getThroughputInBytes(Tracked.PUT); } @JmxGetter(name = "percentGetReturningEmptyResponse", description = "The percentage of calls to GET for which no value was found.") public double getPercentGetReturningEmptyResponse() { - return numEmptyResponses(stats.getNumEmptyResponses(Tracked.GET), stats.getCount(Tracked.GET)); + return numEmptyResponses(stats.getNumEmptyResponses(Tracked.GET), + stats.getCount(Tracked.GET)); } @JmxGetter(name = "percentGetAllReturningEmptyResponse", description = "The percentage of calls to GET_ALL for which no value was found, taking into account multiple returned key-values.") public double getPercentGetAllReturningEmptyResponse() { - return numEmptyResponses(stats.getNumEmptyResponses(Tracked.GET_ALL), stats.getGetAllAggregatedCount()); + return numEmptyResponses(stats.getNumEmptyResponses(Tracked.GET_ALL), + stats.getGetAllAggregatedCount()); } private double numEmptyResponses(long numEmpty, long total) { - return total == 0 ? 0.0d : numEmpty / (float)total; + return total == 0 ? 0.0d : numEmpty / (float) total; } @JmxGetter(name = "maxPutLatencyInMs", description = "Maximum latency in ms of PUT") @@ -172,42 +180,42 @@ public long getMaxDeleteLatency() { return stats.getMaxLatencyInMs(Tracked.DELETE); } - @JmxGetter(name = "q95PutLatencyInMs", description="") + @JmxGetter(name = "q95PutLatencyInMs", description = "") public long getQ95PutLatency() { return stats.getQ95LatencyInMs(Tracked.PUT); } - @JmxGetter(name = "q95GetLatencyInMs", description="") + @JmxGetter(name = "q95GetLatencyInMs", description = "") public long getQ95GetLatency() { return stats.getQ95LatencyInMs(Tracked.GET); } - @JmxGetter(name = "q95GetAllLatencyInMs", description="") + @JmxGetter(name = "q95GetAllLatencyInMs", description = "") public long getQ95GetAllLatency() { return stats.getQ95LatencyInMs(Tracked.GET_ALL); } - @JmxGetter(name = "q95DeleteLatencyInMs", description="") + @JmxGetter(name = "q95DeleteLatencyInMs", description = "") public long getQ95DeleteLatency() { return stats.getQ95LatencyInMs(Tracked.DELETE); } - @JmxGetter(name = "q99PutLatencyInMs", description="") + @JmxGetter(name = "q99PutLatencyInMs", description = "") public long getQ99PutLatency() { return stats.getQ99LatencyInMs(Tracked.PUT); } - @JmxGetter(name = "q99GetLatencyInMs", description="") + @JmxGetter(name = "q99GetLatencyInMs", description = "") public long getQ99GetLatency() { return stats.getQ99LatencyInMs(Tracked.GET); } - @JmxGetter(name = "q99GetAllLatencyInMs", description="") + @JmxGetter(name = "q99GetAllLatencyInMs", description = "") public long getQ99GetAllLatency() { return stats.getQ99LatencyInMs(Tracked.GET_ALL); } - @JmxGetter(name = "q99DeleteLatencyInMs", description="") + @JmxGetter(name = "q99DeleteLatencyInMs", description = "") public long getQ99DeleteLatency() { return stats.getQ99LatencyInMs(Tracked.DELETE); } From 9b93e92dcd3fdfc82dbad9d1e7ece128f93cb0f6 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 11 Apr 2012 16:37:06 -0700 Subject: [PATCH 020/209] Include ObsoleteVersionException --- src/java/voldemort/store/routed/PipelineRoutedStats.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/java/voldemort/store/routed/PipelineRoutedStats.java b/src/java/voldemort/store/routed/PipelineRoutedStats.java index 8f02c86641..77918d28c6 100644 --- a/src/java/voldemort/store/routed/PipelineRoutedStats.java +++ b/src/java/voldemort/store/routed/PipelineRoutedStats.java @@ -11,6 +11,7 @@ import voldemort.store.InvalidMetadataException; import voldemort.store.StoreTimeoutException; import voldemort.store.UnreachableStoreException; +import voldemort.versioning.ObsoleteVersionException; /** * Tracks all the exceptions we see, down at the routing layer also including @@ -31,6 +32,7 @@ public class PipelineRoutedStats { errCountMap.put(InsufficientZoneResponsesException.class, new AtomicLong(0)); errCountMap.put(UnreachableStoreException.class, new AtomicLong(0)); errCountMap.put(StoreTimeoutException.class, new AtomicLong(0)); + errCountMap.put(ObsoleteVersionException.class, new AtomicLong(0)); severeExceptionCount = new AtomicLong(0); benignExceptionCount = new AtomicLong(0); @@ -71,6 +73,11 @@ public long getNumStoreTimeoutExceptions() { return errCountMap.get(StoreTimeoutException.class).get(); } + @JmxGetter(name = "numObsoleteVersionExceptions", description = "Number of requests that got a ObsoleteVersionException as response") + public long getNumObsoleteVersionExceptions() { + return errCountMap.get(ObsoleteVersionException.class).get(); + } + @JmxGetter(name = "getExceptionCountsAsString", description = "Returns counts of all the Exceptions seen so far as a string") public String getExceptionCountsAsString() { StringBuilder result = new StringBuilder(); From a78111dde17c6f9e7ee4ba68d81a6a86e47fdf37 Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Wed, 18 Apr 2012 00:03:28 -0700 Subject: [PATCH 021/209] fixed a donor-based rebalancing bug and added a unit test - all DonorBasedRebalanceAsyncOperation no longer share a same executor service so one async operation won't shutdown the executor service while the other async operation is still using it. --- .../DonorBasedRebalanceAsyncOperation.java | 68 ++++++++++++------- .../rebalance/AbstractRebalanceTest.java | 62 +++++++++++++++++ 2 files changed, 104 insertions(+), 26 deletions(-) diff --git a/src/java/voldemort/server/rebalance/async/DonorBasedRebalanceAsyncOperation.java b/src/java/voldemort/server/rebalance/async/DonorBasedRebalanceAsyncOperation.java index ec20b36139..8f6a595bb9 100644 --- a/src/java/voldemort/server/rebalance/async/DonorBasedRebalanceAsyncOperation.java +++ b/src/java/voldemort/server/rebalance/async/DonorBasedRebalanceAsyncOperation.java @@ -81,8 +81,9 @@ public class DonorBasedRebalanceAsyncOperation extends RebalanceAsyncOperation { private final HashMultimap>>> storeToNodePartitionMapping; - private final ExecutorService pushSlavesExecutor; - private Map> updatePushSlavePool; + // each table being rebalanced is associated with one executor service and a + // pool of threads + private Map>> updatePushSlavePool; private HashMultimap>>> groupByStores(List stealInfos) { @@ -112,17 +113,7 @@ public DonorBasedRebalanceAsyncOperation(Rebalancer rebalancer, // Group the plans by the store names this.storeToNodePartitionMapping = groupByStores(stealInfos); - - pushSlavesExecutor = Executors.newCachedThreadPool(new ThreadFactory() { - - public Thread newThread(Runnable r) { - Thread thread = new Thread(r); - thread.setName(r.getClass().getName()); - return thread; - } - }); - - updatePushSlavePool = Collections.synchronizedMap(new HashMap>()); + updatePushSlavePool = Collections.synchronizedMap(new HashMap>>()); } @Override @@ -241,7 +232,17 @@ private void rebalanceStore(final String storeName, StorageEngine storageEngine = storeRepository.getStorageEngine(storeName); StoreDefinition storeDef = metadataStore.getStoreDef(storeName); List storePushSlaves = Lists.newArrayList(); - updatePushSlavePool.put(storeName, storePushSlaves); + ExecutorService pushSlavesExecutor = Executors.newCachedThreadPool(new ThreadFactory() { + + public Thread newThread(Runnable r) { + Thread thread = new Thread(r); + thread.setName(r.getClass().getName()); + return thread; + } + }); + updatePushSlavePool.put(storeName, + new Pair>(pushSlavesExecutor, + storePushSlaves)); if(isReadOnlyStore) { @@ -330,10 +331,10 @@ private void fetchEntriesForStealers(StorageEngine st printProgress(scanned, fetched, startTime, storeName); } } - terminateAllSlaves(updatePushSlavePool.get(storeName)); + terminateAllSlaves(storeName); } catch(InterruptedException e) { logger.info("InterruptedException received while sending entries to remote nodes, the process is terminating..."); - terminateAllSlavesAsync(updatePushSlavePool.get(storeName)); + terminateAllSlavesAsync(storeName); } finally { close(keys, storeName, scanned, fetched, startTime); } @@ -374,10 +375,13 @@ private void close(ClosableIterator keys, keys.close(); } - private void terminateAllSlaves(List updatePushSlavePool) { + private void terminateAllSlaves(String storeName) { // Everything is done, put the terminator in logger.info("Terminating DonorBasedRebalancePushSlaves..."); - for(Iterator it = updatePushSlavePool.iterator(); it.hasNext();) { + ExecutorService pushSlavesExecutor = updatePushSlavePool.get(storeName).getFirst(); + List pushSlaves = updatePushSlavePool.get(storeName) + .getSecond(); + for(Iterator it = pushSlaves.iterator(); it.hasNext();) { it.next().requestCompletion(); } @@ -395,12 +399,26 @@ private void terminateAllSlaves(List updatePushS logger.info("DonorBasedRebalancingOperation existed."); } - private void terminateAllSlavesAsync(List updatePushSlavePool) { - logger.info("Terminating DonorBasedRebalancePushSlaves asynchronously"); - for(Iterator it = updatePushSlavePool.iterator(); it.hasNext();) { - it.next().requestCompletion(); + private void terminateAllSlavesAsync(String storeName) { + logger.info("Terminating DonorBasedRebalancePushSlaves asynchronously."); + if(null == storeName) { + for(Pair> pair: updatePushSlavePool.values()) { + ExecutorService pushSlavesExecutor = pair.getFirst(); + List pushSlaves = pair.getSecond(); + for(Iterator it = pushSlaves.iterator(); it.hasNext();) { + it.next().requestCompletion(); + } + pushSlavesExecutor.shutdownNow(); + } + } else { + ExecutorService pushSlavesExecutor = updatePushSlavePool.get(storeName).getFirst(); + List pushSlaves = updatePushSlavePool.get(storeName) + .getSecond(); + for(Iterator it = pushSlaves.iterator(); it.hasNext();) { + it.next().requestCompletion(); + } + pushSlavesExecutor.shutdownNow(); } - pushSlavesExecutor.shutdownNow(); logger.info("DonorBasedRebalancingAsyncOperation existed."); } @@ -409,9 +427,7 @@ public void stop() { running.set(false); updateStatus(getHeader(stealInfos) + "Stop called on donor-based rebalance operation"); logger.info(getHeader(stealInfos) + "Stop called on donor-based rebalance operation"); - for(List storePushSlaves: updatePushSlavePool.values()) { - terminateAllSlavesAsync(storePushSlaves); - } + terminateAllSlavesAsync(null); executors.shutdownNow(); } } \ No newline at end of file diff --git a/test/unit/voldemort/client/rebalance/AbstractRebalanceTest.java b/test/unit/voldemort/client/rebalance/AbstractRebalanceTest.java index 4eea8e554c..d3b8fb3b47 100644 --- a/test/unit/voldemort/client/rebalance/AbstractRebalanceTest.java +++ b/test/unit/voldemort/client/rebalance/AbstractRebalanceTest.java @@ -696,6 +696,68 @@ public void testRWRebalanceFourNodes() throws Exception { } } + @Test + public void testRWRebalanceSerial() throws Exception { + Cluster currentCluster = ServerTestUtils.getLocalCluster(4, new int[][] { + { 0, 1, 4, 7, 9 }, { 2, 3, 5, 6, 8 }, {}, {} }); + + ArrayList nodes = Lists.newArrayList(currentCluster.getNodes()); + int totalPortNum = nodes.size() * 3; + int[] ports = new int[totalPortNum]; + for(int i = 0; i < nodes.size(); i++) { + ports[i * 3] = nodes.get(i).getHttpPort(); + ports[i * 3 + 1] = nodes.get(i).getSocketPort(); + ports[i * 3 + 2] = nodes.get(i).getAdminPort(); + } + + Cluster targetCluster = ServerTestUtils.getLocalCluster(4, ports, new int[][] { + { 0, 4, 7 }, { 2, 8 }, { 1, 6 }, { 3, 5, 9 } }); + + // start servers + Map serverProps = new HashMap(); + serverProps.put("max.parallel.stores.rebalancing", String.valueOf(1)); + List serverList = Arrays.asList(0, 1, 2, 3); + currentCluster = startServers(currentCluster, + rwTwoStoreDefFileWithReplication, + serverList, + serverProps); + // Update the cluster information based on the node information + targetCluster = updateCluster(targetCluster); + + RebalanceClientConfig config = new RebalanceClientConfig(); + config.setDeleteAfterRebalancingEnabled(true); + config.setStealerBasedRebalancing(!useDonorBased()); + config.setPrimaryPartitionBatchSize(100); + config.setMaxParallelRebalancing(5); + RebalanceController rebalanceClient = new RebalanceController(getBootstrapUrl(currentCluster, + 0), + config); + try { + populateData(currentCluster, + rwStoreDefWithReplication, + rebalanceClient.getAdminClient(), + false); + + populateData(currentCluster, + rwStoreDefWithReplication2, + rebalanceClient.getAdminClient(), + false); + + rebalanceAndCheck(currentCluster, + targetCluster, + Lists.newArrayList(rwStoreDefWithReplication, + rwStoreDefWithReplication2), + rebalanceClient, + serverList); + checkConsistentMetadata(targetCluster, serverList); + } catch(Exception e) { + fail(e.getMessage()); + } finally { + // stop servers + stopServer(serverList); + } + } + @Test public void testProxyGetDuringRebalancing() throws Exception { final Cluster currentCluster = ServerTestUtils.getLocalCluster(2, new int[][] { From 24dbf6165a7bf777edbfe76fe1a3ed51bac02ddc Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Wed, 2 May 2012 11:21:24 -0700 Subject: [PATCH 022/209] Fixed the confusing display messages for Admin tool rollback service --- src/java/voldemort/VoldemortAdminTool.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/java/voldemort/VoldemortAdminTool.java b/src/java/voldemort/VoldemortAdminTool.java index 195df6dec2..899e0b70fc 100644 --- a/src/java/voldemort/VoldemortAdminTool.java +++ b/src/java/voldemort/VoldemortAdminTool.java @@ -212,7 +212,7 @@ public static void main(String[] args) throws Exception { .withRequiredArg() .describedAs("store-name") .ofType(String.class); - parser.accepts("version", "version of store to rollback to") + parser.accepts("version", "Push version of store to rollback to") .withRequiredArg() .describedAs("version") .ofType(Long.class); @@ -302,7 +302,7 @@ public static void main(String[] args) throws Exception { } if(options.has("rollback")) { if(!options.has("version")) { - Utils.croak("A version of the read-only store must be specified with rollback option"); + Utils.croak("A read-only push version must be specified with rollback option"); } ops += "o"; } @@ -605,7 +605,7 @@ public static void printHelp(PrintStream stream, OptionParser parser) throws IOE stream.println("\t5) Backup bdb data natively"); stream.println("\t\t./bin/voldemort-admin-tool.sh --native-backup [store] --backup-dir [outdir] " + "--backup-timeout [mins] [--backup-verify] [--backup-incremental] --url [url] --node [node-id]"); - stream.println("\t6) Rollback a read-only store for the specified version"); + stream.println("\t6) Rollback a read-only store to the specified push version"); stream.println("\t\t./bin/voldemort-admin-tool.sh --rollback [store-name] --url [url] --node [node-id] --version [version-num] "); parser.printHelpOn(stream); From 787f4126a3070766e8847f77a72e876a5e005151 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Wed, 2 May 2012 14:52:19 -0700 Subject: [PATCH 023/209] Added ScanPermitWrapper class and general cleanup --- .../server/storage/ScanPermitWrapper.java | 45 +++++++++++++++++++ .../server/storage/StorageService.java | 3 -- 2 files changed, 45 insertions(+), 3 deletions(-) create mode 100644 src/java/voldemort/server/storage/ScanPermitWrapper.java diff --git a/src/java/voldemort/server/storage/ScanPermitWrapper.java b/src/java/voldemort/server/storage/ScanPermitWrapper.java new file mode 100644 index 0000000000..42368a97d4 --- /dev/null +++ b/src/java/voldemort/server/storage/ScanPermitWrapper.java @@ -0,0 +1,45 @@ +package voldemort.server.storage; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Semaphore; + +import voldemort.annotations.jmx.JmxManaged; + +@JmxManaged(description = "Wrapper for Scan permit.") +public class ScanPermitWrapper { + + private final Semaphore scanPermits; + private List permitOwners; + + public ScanPermitWrapper(int numPermits) { + scanPermits = new Semaphore(numPermits); + permitOwners = new ArrayList(); + } + + public synchronized void acquire() throws InterruptedException { + this.scanPermits.acquire(); + permitOwners.add(Thread.currentThread().getStackTrace()[2].getClassName()); + } + + public synchronized void release() { + this.scanPermits.release(); + String className = Thread.currentThread().getStackTrace()[2].getClassName(); + permitOwners.remove(className); + } + + public List getPermitOwners() { + return this.permitOwners; + } + + public synchronized boolean tryAcquire() { + boolean gotPermit = this.scanPermits.tryAcquire(); + if(gotPermit) + permitOwners.add(Thread.currentThread().getStackTrace()[2].getClassName()); + return gotPermit; + } + + public synchronized int availablePermits() { + return this.scanPermits.availablePermits(); + } +} diff --git a/src/java/voldemort/server/storage/StorageService.java b/src/java/voldemort/server/storage/StorageService.java index d37df573e5..5a26c3e86d 100644 --- a/src/java/voldemort/server/storage/StorageService.java +++ b/src/java/voldemort/server/storage/StorageService.java @@ -117,7 +117,6 @@ public class StorageService extends AbstractService { private final DynamicThrottleLimit dynThrottleLimit; // Common permit shared by all job which do a disk scan - // private final Semaphore scanPermits; private final ScanPermitWrapper scanPermitWrapper; private final SocketStoreFactory storeFactory; private final ConcurrentMap storageConfigs; @@ -135,8 +134,6 @@ public StorageService(StoreRepository storeRepository, this.scheduler = scheduler; this.storeRepository = storeRepository; this.metadata = metadata; - // this.scanPermits = new - // Semaphore(voldemortConfig.getNumScanPermits()); this.scanPermitWrapper = new ScanPermitWrapper(voldemortConfig.getNumScanPermits()); this.storageConfigs = new ConcurrentHashMap(); this.clientThreadPool = new ClientThreadPool(config.getClientMaxThreads(), From 220cdf3b73186babce873eee08e4fd0c849228c4 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Wed, 2 May 2012 15:04:40 -0700 Subject: [PATCH 024/209] Removed jmx managed for scanpermitwrapper --- src/java/voldemort/server/storage/ScanPermitWrapper.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/java/voldemort/server/storage/ScanPermitWrapper.java b/src/java/voldemort/server/storage/ScanPermitWrapper.java index 42368a97d4..02f8a963c2 100644 --- a/src/java/voldemort/server/storage/ScanPermitWrapper.java +++ b/src/java/voldemort/server/storage/ScanPermitWrapper.java @@ -4,9 +4,6 @@ import java.util.List; import java.util.concurrent.Semaphore; -import voldemort.annotations.jmx.JmxManaged; - -@JmxManaged(description = "Wrapper for Scan permit.") public class ScanPermitWrapper { private final Semaphore scanPermits; From 6bca2887223d9d15eea7cc2fba963ac1d46d1630 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Wed, 2 May 2012 16:25:14 -0700 Subject: [PATCH 025/209] Created a separate scheduler operation to terminate jobs. Disable will simply unschedule and not force kill. --- .../voldemort/server/scheduler/SchedulerService.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/src/java/voldemort/server/scheduler/SchedulerService.java b/src/java/voldemort/server/scheduler/SchedulerService.java index c241c4efa4..e5c2941097 100644 --- a/src/java/voldemort/server/scheduler/SchedulerService.java +++ b/src/java/voldemort/server/scheduler/SchedulerService.java @@ -108,6 +108,18 @@ public void stopInner() { @JmxOperation(description = "Disable a particular scheduled job", impact = MBeanOperationInfo.ACTION) public void disable(String id) { + if(allJobs.containsKey(id) && scheduledJobResults.containsKey(id)) { + ScheduledFuture future = scheduledJobResults.get(id); + boolean cancelled = future.cancel(false); + if(cancelled == true) { + logger.info("Removed '" + id + "' from list of scheduled jobs"); + scheduledJobResults.remove(id); + } + } + } + + @JmxOperation(description = "Terminate a particular scheduled job", impact = MBeanOperationInfo.ACTION) + public void terminate(String id) { if(allJobs.containsKey(id) && scheduledJobResults.containsKey(id)) { ScheduledFuture future = scheduledJobResults.get(id); boolean cancelled = future.cancel(this.mayInterrupt); From d59d3918e6fcc9bc4b31e4b8fed672c473330f6d Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Fri, 4 May 2012 11:39:13 -0700 Subject: [PATCH 026/209] Fixed a potential deadlock issue in ScanPermitWrapper --- .../server/storage/ScanPermitWrapper.java | 38 +++++++++++++------ 1 file changed, 26 insertions(+), 12 deletions(-) diff --git a/src/java/voldemort/server/storage/ScanPermitWrapper.java b/src/java/voldemort/server/storage/ScanPermitWrapper.java index 02f8a963c2..058973845a 100644 --- a/src/java/voldemort/server/storage/ScanPermitWrapper.java +++ b/src/java/voldemort/server/storage/ScanPermitWrapper.java @@ -1,6 +1,8 @@ package voldemort.server.storage; import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.concurrent.Semaphore; @@ -9,34 +11,46 @@ public class ScanPermitWrapper { private final Semaphore scanPermits; private List permitOwners; - public ScanPermitWrapper(int numPermits) { + public ScanPermitWrapper(final int numPermits) { scanPermits = new Semaphore(numPermits); - permitOwners = new ArrayList(); + permitOwners = Collections.synchronizedList(new ArrayList()); } - public synchronized void acquire() throws InterruptedException { + public void acquire() throws InterruptedException { this.scanPermits.acquire(); - permitOwners.add(Thread.currentThread().getStackTrace()[2].getClassName()); + synchronized(permitOwners) { + permitOwners.add(Thread.currentThread().getStackTrace()[2].getClassName()); + } } - public synchronized void release() { + public void release() { this.scanPermits.release(); - String className = Thread.currentThread().getStackTrace()[2].getClassName(); - permitOwners.remove(className); + synchronized(permitOwners) { + permitOwners.remove(Thread.currentThread().getStackTrace()[2].getClassName()); + } } public List getPermitOwners() { - return this.permitOwners; + List ownerList = new ArrayList(); + synchronized(permitOwners) { + Iterator i = this.permitOwners.iterator(); + while(i.hasNext()) + ownerList.add(i.next()); + } + return ownerList; } - public synchronized boolean tryAcquire() { + public boolean tryAcquire() { boolean gotPermit = this.scanPermits.tryAcquire(); - if(gotPermit) - permitOwners.add(Thread.currentThread().getStackTrace()[2].getClassName()); + if(gotPermit) { + synchronized(permitOwners) { + permitOwners.add(Thread.currentThread().getStackTrace()[2].getClassName()); + } + } return gotPermit; } - public synchronized int availablePermits() { + public int availablePermits() { return this.scanPermits.availablePermits(); } } From 425d99d186fbfd179759f2ebac557f6348583b40 Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Thu, 10 May 2012 13:08:37 -0700 Subject: [PATCH 027/209] add zoned option for restore from replicas --- src/java/voldemort/VoldemortAdminTool.java | 7 +- .../client/protocol/admin/AdminClient.java | 152 +++++++++++++----- .../client/rebalance/RebalanceController.java | 3 + 3 files changed, 123 insertions(+), 39 deletions(-) diff --git a/src/java/voldemort/VoldemortAdminTool.java b/src/java/voldemort/VoldemortAdminTool.java index 22d61febeb..669074ff43 100644 --- a/src/java/voldemort/VoldemortAdminTool.java +++ b/src/java/voldemort/VoldemortAdminTool.java @@ -208,6 +208,10 @@ public static void main(String[] args) throws Exception { parser.accepts("backup-incremental", "Perform an incremental backup for point-in-time recovery." + " By default backup has latest consistent snapshot."); + parser.accepts("zone", "zone id") + .withRequiredArg() + .describedAs("zone-id") + .ofType(Integer.class); OptionSet options = parser.parse(args); @@ -234,6 +238,7 @@ public static void main(String[] args) throws Exception { String url = (String) options.valueOf("url"); Integer nodeId = CmdUtils.valueOf(options, "node", -1); int parallelism = CmdUtils.valueOf(options, "restore", 5); + Integer zoneId = CmdUtils.valueOf(options, "zone", -1); AdminClient adminClient = new AdminClient(url, new AdminClientConfig()); @@ -324,7 +329,7 @@ public static void main(String[] args) throws Exception { System.exit(1); } System.out.println("Starting restore"); - adminClient.restoreDataFromReplications(nodeId, parallelism); + adminClient.restoreDataFromReplications(nodeId, parallelism, zoneId); System.out.println("Finished restore"); } if(ops.contains("k")) { diff --git a/src/java/voldemort/client/protocol/admin/AdminClient.java b/src/java/voldemort/client/protocol/admin/AdminClient.java index 3b0dd469b6..0398462105 100644 --- a/src/java/voldemort/client/protocol/admin/AdminClient.java +++ b/src/java/voldemort/client/protocol/admin/AdminClient.java @@ -647,6 +647,22 @@ public ByteArray computeNext() { * @throws InterruptedException */ public void restoreDataFromReplications(int nodeId, int parallelTransfers) { + restoreDataFromReplications(nodeId, parallelTransfers, -1); + } + + /** + * RestoreData from copies on other machines for the given nodeId + *

+ * Recovery mechanism to recover and restore data actively from replicated + * copies in the cluster.
+ * + * @param nodeId Id of the node to restoreData + * @param parallelTransfers number of transfers + * @param zoneId zone from which the nodes are chosen from, -1 means no zone + * preference + * @throws InterruptedException + */ + public void restoreDataFromReplications(int nodeId, int parallelTransfers, int zoneId) { ExecutorService executors = Executors.newFixedThreadPool(parallelTransfers, new ThreadFactory() { @@ -676,7 +692,7 @@ public Thread newThread(Runnable r) { } } for(StoreDefinition def: writableStores) { - restoreStoreFromReplication(nodeId, cluster, def, executors); + restoreStoreFromReplication(nodeId, cluster, def, executors, zoneId); } } finally { executors.shutdown(); @@ -703,8 +719,26 @@ public Thread newThread(Runnable r) { public Map>> getReplicationMapping(int restoringNode, Cluster cluster, StoreDefinition storeDef) { + return getReplicationMapping(restoringNode, cluster, storeDef, -1); + } + + /** + * For a particular node, finds out all the [replica, partition] tuples it + * needs to steal in order to be brought back to normal state + * + * @param restoringNode The id of the node which needs to be restored + * @param cluster The cluster definition + * @param storeDef The store definition to use + * @param zoneId zone from which nodes are chosen, -1 means no zone + * preference + * @return Map of node id to map of replica type and corresponding partition + * list + */ + public Map>> getReplicationMapping(int restoringNode, + Cluster cluster, + StoreDefinition storeDef, + int zoneId) { - Map partitionToNodeId = RebalanceUtils.getCurrentPartitionMapping(cluster); Map>> returnMap = Maps.newHashMap(); RoutingStrategy strategy = new RoutingStrategyFactory().updateRoutingStrategy(storeDef, @@ -731,28 +765,12 @@ public Map>> getReplicationMapping(int r + "being left in replicating list"); } - // Pick the first element and find its position in the - // origin replicating list - int replicaType = extraCopyReplicatingPartitions.indexOf(replicatingPartitions.get(0)); - int partition = extraCopyReplicatingPartitions.get(0); - int nodeId = partitionToNodeId.get(replicatingPartitions.get(0)); - - HashMap> replicaToPartitionList = null; - if(returnMap.containsKey(nodeId)) { - replicaToPartitionList = returnMap.get(nodeId); - } else { - replicaToPartitionList = Maps.newHashMap(); - returnMap.put(nodeId, replicaToPartitionList); - } - - List partitions = null; - if(replicaToPartitionList.containsKey(replicaType)) { - partitions = replicaToPartitionList.get(replicaType); - } else { - partitions = Lists.newArrayList(); - replicaToPartitionList.put(replicaType, partitions); - } - partitions.add(partition); + addDonorWithZonePreference(replicatingPartitions, + extraCopyReplicatingPartitions, + returnMap, + zoneId, + cluster, + storeDef); } } @@ -760,6 +778,70 @@ public Map>> getReplicationMapping(int r return returnMap; } + /** + * For each partition that need to be restored, find a donor node that owns + * the partition AND has the same zone ID as requested. -1 means no zone + * preference required when finding a donor node needs to steal in order to + * + * @param remainderPartitions The replicating partitions without the one + * needed by the restore node + * @param originalPartitions The entire replicating partition list + * (including the one needed by the restore node) + * @param donorMap All donor nodes that will be fetched from + * @param zondId The zone from which donor nodes will be chosen from; -1 + * means all zones are fine + * @param cluster The cluster metadata + * @param storeDef The store to be restored + * @return + */ + private void addDonorWithZonePreference(List remainderPartitions, + List originalPartitions, + Map>> donorMap, + int zoneId, + Cluster cluster, + StoreDefinition storeDef) { + Map partitionToNodeId = RebalanceUtils.getCurrentPartitionMapping(cluster); + int nodeId = -1; + int replicaType = -1; + int partition = -1; + boolean found = false; + int index = 0; + + while(!found && index < remainderPartitions.size()) { + replicaType = originalPartitions.indexOf(remainderPartitions.get(index)); + partition = originalPartitions.get(0); + nodeId = partitionToNodeId.get(remainderPartitions.get(index)); + if(-1 == zoneId || cluster.getNodeById(nodeId).getZoneId() == zoneId) { + found = true; + } else { + index++; + } + } + + if(!found) { + throw new VoldemortException("unable to find a node to fetch partition " + partition + + " of replica type " + replicaType + " for store " + + storeDef.getName()); + } + + HashMap> replicaToPartitionList = null; + if(donorMap.containsKey(nodeId)) { + replicaToPartitionList = donorMap.get(nodeId); + } else { + replicaToPartitionList = Maps.newHashMap(); + donorMap.put(nodeId, replicaToPartitionList); + } + + List partitions = null; + if(replicaToPartitionList.containsKey(replicaType)) { + partitions = replicaToPartitionList.get(replicaType); + } else { + partitions = Lists.newArrayList(); + replicaToPartitionList.put(replicaType, partitions); + } + partitions.add(partition); + } + /** * For a particular store and node, runs the replication job. This works * only for read-write stores @@ -772,13 +854,15 @@ public Map>> getReplicationMapping(int r private void restoreStoreFromReplication(final int restoringNodeId, final Cluster cluster, final StoreDefinition storeDef, - final ExecutorService executorService) { + final ExecutorService executorService, + final int zoneId) { logger.info("Restoring data for store " + storeDef.getName() + " on node " + restoringNodeId); Map>> restoreMapping = getReplicationMapping(restoringNodeId, cluster, - storeDef); + storeDef, + zoneId); // migrate partition for(final Entry>> replicationEntry: restoreMapping.entrySet()) { final int donorNodeId = replicationEntry.getKey(); @@ -790,18 +874,10 @@ public void run() { + restoringNodeId + " from node " + replicationEntry.getKey() + " partitions:" + replicationEntry.getValue()); - int migrateAsyncId = migratePartitions(donorNodeId, - restoringNodeId, - storeDef.getName(), - replicationEntry.getValue(), - null, - null, - false); - waitForCompletion(restoringNodeId, - migrateAsyncId, - adminClientConfig.getRestoreDataTimeoutSec(), - TimeUnit.SECONDS); - + int migrateAsyncId = migratePartitions(donorNodeId, restoringNodeId, storeDef.getName(), replicationEntry.getValue(), null, null, false); + + waitForCompletion(restoringNodeId, migrateAsyncId, adminClientConfig.getRestoreDataTimeoutSec(), TimeUnit.SECONDS); + logger.info("Restoring data for store:" + storeDef.getName() + " from node " + donorNodeId + " completed."); } catch(Exception e) { diff --git a/src/java/voldemort/client/rebalance/RebalanceController.java b/src/java/voldemort/client/rebalance/RebalanceController.java index f7c7bfb774..ed0a2206a3 100644 --- a/src/java/voldemort/client/rebalance/RebalanceController.java +++ b/src/java/voldemort/client/rebalance/RebalanceController.java @@ -718,6 +718,9 @@ private List executeTasks(final int taskId, HashMap> donorNodeBasedPartitionsInfo = RebalanceUtils.groupPartitionsInfoByNode(rebalancePartitionPlanList, false); for(Entry> entries: donorNodeBasedPartitionsInfo.entrySet()) { + try { + Thread.sleep(10000); + } catch (InterruptedException e) {} DonorBasedRebalanceTask rebalanceTask = new DonorBasedRebalanceTask(taskId, entries.getValue(), rebalanceConfig, From 5555d37e79aed4d1f0c8d2b4d7b31c5f3b864787 Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Thu, 10 May 2012 18:51:13 -0700 Subject: [PATCH 028/209] add tests for zoned restore --- .../client/AdminServiceBasicTest.java | 147 +++++++++++++++++- 1 file changed, 142 insertions(+), 5 deletions(-) diff --git a/test/unit/voldemort/client/AdminServiceBasicTest.java b/test/unit/voldemort/client/AdminServiceBasicTest.java index 9623e89ecf..8343bd6855 100644 --- a/test/unit/voldemort/client/AdminServiceBasicTest.java +++ b/test/unit/voldemort/client/AdminServiceBasicTest.java @@ -27,9 +27,9 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Properties; import java.util.Set; -import java.util.Map.Entry; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -601,6 +601,141 @@ public void testReplicationMapping() { } } + @Test + public void testReplicationMappingWithZonePreference() { + List nodes = Lists.newArrayList(); + nodes.add(new Node(0, "localhost", 1, 2, 3, 0, Lists.newArrayList(0, 4, 8))); + nodes.add(new Node(1, "localhost", 1, 2, 3, 0, Lists.newArrayList(1, 5, 9))); + nodes.add(new Node(2, "localhost", 1, 2, 3, 1, Lists.newArrayList(2, 6, 10))); + nodes.add(new Node(3, "localhost", 1, 2, 3, 1, Lists.newArrayList(3, 7, 11))); + + // Test 0 - With rep-factor 1; zone 1 + StoreDefinition storeDef = ServerTestUtils.getStoreDef("consistent", + 1, + 1, + 1, + 1, + 1, + RoutingStrategyType.CONSISTENT_STRATEGY); + Cluster newCluster = new Cluster("single_zone_cluster", nodes); + + try { + adminClient.getReplicationMapping(0, newCluster, storeDef, 1); + fail("Should have thrown an exception since rep-factor = 1"); + } catch(VoldemortException e) {} + + // With rep-factor 1; zone 0 + storeDef = ServerTestUtils.getStoreDef("consistent", + 1, + 1, + 1, + 1, + 1, + RoutingStrategyType.CONSISTENT_STRATEGY); + newCluster = new Cluster("single_zone_cluster", nodes); + + try { + adminClient.getReplicationMapping(0, newCluster, storeDef, 0); + fail("Should have thrown an exception since rep-factor = 1"); + } catch(VoldemortException e) {} + + // Test 1 - With consistent routing strategy + storeDef = ServerTestUtils.getStoreDef("consistent", + 4, + 1, + 1, + 1, + 1, + RoutingStrategyType.CONSISTENT_STRATEGY); + + // On node 0; zone id 1 + Map>> replicationMapping = adminClient.getReplicationMapping(0, + newCluster, + storeDef, + 1); + { + HashMap>> expectedMapping = Maps.newHashMap(); + HashMap> partitionTuple = Maps.newHashMap(); + partitionTuple.put(0, Lists.newArrayList(2, 6, 10)); + partitionTuple.put(1, Lists.newArrayList(1, 5, 9)); + partitionTuple.put(2, Lists.newArrayList(0, 4, 8)); + expectedMapping.put(2, partitionTuple); + HashMap> partitionTuple2 = Maps.newHashMap(); + partitionTuple2.put(0, Lists.newArrayList(3, 7, 11)); + expectedMapping.put(3, partitionTuple2); + // {2={0=[2, 6, 10], 1=[1, 5, 9], 2=[0, 4, 8]}, 3={0=[3, 7, 11]}} + assertEquals(replicationMapping, expectedMapping); + } + + // On node 0; zone id 0 + replicationMapping = adminClient.getReplicationMapping(0, newCluster, storeDef, 0); + { + HashMap>> expectedMapping = Maps.newHashMap(); + HashMap> partitionTuple = Maps.newHashMap(); + partitionTuple.clear(); + partitionTuple.put(0, Lists.newArrayList(1, 5, 9)); + partitionTuple.put(1, Lists.newArrayList(0, 4, 8)); + partitionTuple.put(2, Lists.newArrayList(3, 7, 11)); + partitionTuple.put(3, Lists.newArrayList(2, 6, 10)); + expectedMapping.put(1, partitionTuple); + // {1={0=[1, 5, 9], 1=[0, 4, 8]}, 2=[3, 7, 11], 3=[2, 6, 10]} + assertEquals(replicationMapping, expectedMapping); + } + + // Test 2 - With zone routing strategy, and zone replication factor 1 + List zones = ServerTestUtils.getZones(2); + HashMap zoneReplicationFactors = Maps.newHashMap(); + for(int zoneIds = 0; zoneIds < 2; zoneIds++) { + zoneReplicationFactors.put(zoneIds, 1); + } + storeDef = ServerTestUtils.getStoreDef("zone", + 2, + 1, + 1, + 1, + 0, + 0, + zoneReplicationFactors, + HintedHandoffStrategyType.PROXIMITY_STRATEGY, + RoutingStrategyType.ZONE_STRATEGY); + newCluster = new Cluster("multi_zone_cluster", nodes, zones); + + { + // On node 0, zone 0 - failure case since zoneReplicationFactor is 1 + + try { + replicationMapping = adminClient.getReplicationMapping(0, newCluster, storeDef, 0); + fail("Should have thrown an exception since zoneReplicationFactor is 1"); + } catch(VoldemortException e) {} + } + + { + // On node 0, zone 1 + replicationMapping = adminClient.getReplicationMapping(0, newCluster, storeDef, 1); + HashMap>> expectedMapping = Maps.newHashMap(); + HashMap> partitionTuple = Maps.newHashMap(); + partitionTuple.put(0, Lists.newArrayList(2, 6, 10)); + partitionTuple.put(1, Lists.newArrayList(0, 4, 8)); + expectedMapping.put(2, partitionTuple); + HashMap> partitionTuple2 = Maps.newHashMap(); + partitionTuple2.put(0, Lists.newArrayList(3, 7, 11)); + expectedMapping.put(3, partitionTuple2); + // {2={0=[2, 6, 10], 1=[0, 4, 8]}, 3={0=[3, 7, 11]}}} + assertEquals(replicationMapping, expectedMapping); + } + + { + // On node 1, zone 1 + replicationMapping = adminClient.getReplicationMapping(1, newCluster, storeDef, 1); + HashMap>> expectedMapping = Maps.newHashMap(); + HashMap> partitionTuple = Maps.newHashMap(); + partitionTuple.put(1, Lists.newArrayList(1, 5, 9)); + expectedMapping.put(2, partitionTuple); + // {2={1=[1, 5, 9]}} + assertEquals(replicationMapping, expectedMapping); + } + } + @Test public void testDeleteStore() throws Exception { AdminClient adminClient = getAdminClient(); @@ -721,8 +856,9 @@ public void testDeletePartitionEntries() { store = getStore(0, testStoreName); for(Entry entry: entrySet.entrySet()) { if(isKeyPartition(entry.getKey(), 0, testStoreName, deletePartitionsList)) { - assertEquals("deleted partitions should be missing.", 0, store.get(entry.getKey(), - null).size()); + assertEquals("deleted partitions should be missing.", + 0, + store.get(entry.getKey(), null).size()); } } } @@ -1174,8 +1310,9 @@ public void testUpdateSlops() { Store store = getStore(0, nextSlop.getStoreName()); if(nextSlop.getOperation().equals(Slop.Operation.PUT)) { - assertNotSame("entry should be present at store", 0, store.get(nextSlop.getKey(), - null).size()); + assertNotSame("entry should be present at store", + 0, + store.get(nextSlop.getKey(), null).size()); assertEquals("entry value should match", new String(nextSlop.getValue()), new String(store.get(nextSlop.getKey(), null).get(0).getValue())); From 39690ac5d635c93e9b508d5ae2d75c747be6c8bc Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Thu, 10 May 2012 18:52:17 -0700 Subject: [PATCH 029/209] Minor change to in getReplicationMapping --- .../client/protocol/admin/AdminClient.java | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/src/java/voldemort/client/protocol/admin/AdminClient.java b/src/java/voldemort/client/protocol/admin/AdminClient.java index 0398462105..94203296a5 100644 --- a/src/java/voldemort/client/protocol/admin/AdminClient.java +++ b/src/java/voldemort/client/protocol/admin/AdminClient.java @@ -809,7 +809,6 @@ private void addDonorWithZonePreference(List remainderPartitions, while(!found && index < remainderPartitions.size()) { replicaType = originalPartitions.indexOf(remainderPartitions.get(index)); - partition = originalPartitions.get(0); nodeId = partitionToNodeId.get(remainderPartitions.get(index)); if(-1 == zoneId || cluster.getNodeById(nodeId).getZoneId() == zoneId) { found = true; @@ -824,6 +823,7 @@ private void addDonorWithZonePreference(List remainderPartitions, + storeDef.getName()); } + partition = originalPartitions.get(0); HashMap> replicaToPartitionList = null; if(donorMap.containsKey(nodeId)) { replicaToPartitionList = donorMap.get(nodeId); @@ -874,10 +874,19 @@ public void run() { + restoringNodeId + " from node " + replicationEntry.getKey() + " partitions:" + replicationEntry.getValue()); - int migrateAsyncId = migratePartitions(donorNodeId, restoringNodeId, storeDef.getName(), replicationEntry.getValue(), null, null, false); - - waitForCompletion(restoringNodeId, migrateAsyncId, adminClientConfig.getRestoreDataTimeoutSec(), TimeUnit.SECONDS); - + int migrateAsyncId = migratePartitions(donorNodeId, + restoringNodeId, + storeDef.getName(), + replicationEntry.getValue(), + null, + null, + false); + + waitForCompletion(restoringNodeId, + migrateAsyncId, + adminClientConfig.getRestoreDataTimeoutSec(), + TimeUnit.SECONDS); + logger.info("Restoring data for store:" + storeDef.getName() + " from node " + donorNodeId + " completed."); } catch(Exception e) { From c8ab434ef7376a7e502b1c7be877044dc8b58de8 Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Fri, 11 May 2012 15:01:52 -0700 Subject: [PATCH 030/209] Revert "add clientId for voldemort client" This reverts commit ddded7cf0af82cc1a789ca8426ef6c48744bf445. ClientId implementation will be released after we do one more open-source release before our next Major open-source release. --- .../client/AbstractStoreClientFactory.java | 77 +--- .../client/CachingStoreClientFactory.java | 32 +- src/java/voldemort/client/ClientConfig.java | 17 +- .../voldemort/client/DefaultStoreClient.java | 29 +- .../client/MockStoreClientFactory.java | 9 +- .../voldemort/client/StoreClientFactory.java | 17 +- .../voldemort/StaticStoreClientFactory.java | 10 +- test/unit/voldemort/client/ClientJmxTest.java | 341 ------------------ 8 files changed, 27 insertions(+), 505 deletions(-) delete mode 100644 test/unit/voldemort/client/ClientJmxTest.java diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index e5072be68b..fb82936c50 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -16,17 +16,12 @@ package voldemort.client; -import java.io.File; -import java.io.IOException; import java.io.StringReader; -import java.net.InetAddress; import java.net.URI; import java.net.URISyntaxException; -import java.net.UnknownHostException; import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -103,8 +98,6 @@ public abstract class AbstractStoreClientFactory implements StoreClientFactory { private final ClientConfig config; private final RoutedStoreFactory routedStoreFactory; private final int clientZoneId; - private final String clientContextName; - private final AtomicInteger sequencer; public AbstractStoreClientFactory(ClientConfig config) { this.config = config; @@ -119,24 +112,18 @@ public AbstractStoreClientFactory(ClientConfig config) { this.maxBootstrapRetries = config.getMaxBootstrapRetries(); this.stats = new StoreStats(); this.clientZoneId = config.getClientZoneId(); - this.clientContextName = (null == config.getClientContextName() ? "" - : config.getClientContextName()); this.routedStoreFactory = new RoutedStoreFactory(config.isPipelineRoutedStoreEnabled(), threadPool, config.getRoutingTimeout(TimeUnit.MILLISECONDS)); - this.sequencer = new AtomicInteger(0); if(this.isJmxEnabled) { JmxUtils.registerMbean(threadPool, JmxUtils.createObjectName(JmxUtils.getPackageName(threadPool.getClass()), JmxUtils.getClassName(threadPool.getClass()) - + "." - + clientContextName + jmxId())); JmxUtils.registerMbean(new StoreStatsJmx(stats), JmxUtils.createObjectName("voldemort.store.stats.aggregate", - clientContextName + ".aggregate-perf" - + jmxId())); + "aggregate-perf" + jmxId())); } } @@ -146,18 +133,12 @@ public StoreClient getStoreClient(String storeName) { public StoreClient getStoreClient(String storeName, InconsistencyResolver> resolver) { - return new DefaultStoreClient(storeName, - resolver, - this, - 3, - clientContextName, - sequencer.getAndIncrement()); + return new DefaultStoreClient(storeName, resolver, this, 3); } @SuppressWarnings("unchecked") public Store getRawStore(String storeName, - InconsistencyResolver> resolver, - UUID clientId) { + InconsistencyResolver> resolver) { logger.info("Client zone-id [" + clientZoneId + "] Attempting to obtain metadata for store [" + storeName + "] "); @@ -242,13 +223,7 @@ public Store getRawStore(String storeName, store = statStore; JmxUtils.registerMbean(new StoreStatsJmx(statStore.getStats()), JmxUtils.createObjectName(JmxUtils.getPackageName(store.getClass()), - clientContextName - + "." - + store.getName() - + jmxId() - + (null == clientId ? "" - : "." - + clientId.toString()))); + store.getName() + jmxId())); } if(storeDef.getKeySerializer().hasCompression() @@ -282,11 +257,6 @@ public Store getRawStore(String storeName, return serializedStore; } - public Store getRawStore(String storeName, - InconsistencyResolver> resolver) { - return getRawStore(storeName, resolver, null); - } - protected ClientConfig getConfig() { return config; } @@ -431,44 +401,7 @@ public void close() { /* Give a unique id to avoid jmx clashes */ private String jmxId() { - return jmxId == 0 ? "" : "." + Integer.toString(jmxId); + return jmxId == 0 ? "" : Integer.toString(jmxId); } - /** - * Generate a unique client ID based on: 0. clientContext, if specified; 1. - * storeName 2. run path 3. client sequence - * - * @param storeName the name of the store the client is created for - * @param contextName the name of the client context - * @param clientSequence the client sequence number - * @return unique client ID - */ - public static UUID generateClientId(String storeName, String contextName, int clientSequence) { - String newLine = System.getProperty("line.separator"); - StringBuilder context = new StringBuilder(contextName == null ? "" : contextName); - context.append(0 == clientSequence ? "" : ("." + clientSequence)); - context.append(".").append(storeName); - - try { - InetAddress host = InetAddress.getLocalHost(); - context.append("@").append(host.getHostName()).append(":"); - } catch(UnknownHostException e) { - logger.info("Unable to obtain client hostname."); - logger.info(e.getMessage()); - } - - try { - String currentPath = new File(".").getCanonicalPath(); - context.append(currentPath).append(newLine); - } catch(IOException e) { - logger.info("Unable to obtain client run path."); - logger.info(e.getMessage()); - } - - if(logger.isDebugEnabled()) { - logger.debug(context.toString()); - } - - return UUID.nameUUIDFromBytes(context.toString().getBytes()); - } } diff --git a/src/java/voldemort/client/CachingStoreClientFactory.java b/src/java/voldemort/client/CachingStoreClientFactory.java index 5fd2953da3..993ff7c3a5 100644 --- a/src/java/voldemort/client/CachingStoreClientFactory.java +++ b/src/java/voldemort/client/CachingStoreClientFactory.java @@ -1,12 +1,12 @@ /* * Copyright 2008-2010 LinkedIn, Inc - * + * * Licensed under the Apache License, Version 2.0 (the "License"); you may not * use this file except in compliance with the License. You may obtain a copy of * the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the @@ -16,13 +16,8 @@ package voldemort.client; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - +import com.google.common.collect.ImmutableList; import org.apache.log4j.Logger; - import voldemort.annotations.jmx.JmxManaged; import voldemort.annotations.jmx.JmxOperation; import voldemort.cluster.failuredetector.FailureDetector; @@ -31,12 +26,14 @@ import voldemort.versioning.InconsistencyResolver; import voldemort.versioning.Versioned; -import com.google.common.collect.ImmutableList; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; /** - * A wrapper for a store {@link StoreClientFactory} which caches requests to - * getStoreClient - * + * A wrapper for a store {@link StoreClientFactory} which caches requests + * to getStoreClient + * */ @JmxManaged(description = "A StoreClientFactory which caches clients") public class CachingStoreClientFactory implements StoreClientFactory { @@ -51,6 +48,7 @@ public CachingStoreClientFactory(StoreClientFactory inner) { this.cache = new ConcurrentHashMap, StoreClient>(); } + @SuppressWarnings("unchecked") public StoreClient getStoreClient(String storeName) { Pair key = Pair.create(storeName, null); @@ -76,13 +74,7 @@ public StoreClient getStoreClient(String storeName, public Store getRawStore(String storeName, InconsistencyResolver> resolver) { - return getRawStore(storeName, resolver, null); - } - - public Store getRawStore(String storeName, - InconsistencyResolver> resolver, - UUID clientId) { - return inner.getRawStore(storeName, resolver, clientId); + return inner.getRawStore(storeName, resolver); } public void close() { diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index 30e4ad9dad..f74f31c04d 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -76,7 +76,6 @@ public class ClientConfig { private long failureDetectorRequestLengthThreshold = socketTimeoutMs; private volatile int maxBootstrapRetries = 2; - private volatile String clientContextName = "default"; public ClientConfig() {} @@ -111,7 +110,6 @@ public ClientConfig() {} public static final String FAILUREDETECTOR_CATASTROPHIC_ERROR_TYPES_PROPERTY = "failuredetector_catastrophic_error_types"; public static final String FAILUREDETECTOR_REQUEST_LENGTH_THRESHOLD_PROPERTY = "failuredetector_request_length_threshold"; public static final String MAX_BOOTSTRAP_RETRIES = "max_bootstrap_retries"; - public static final String CLIENT_CONTEXT_NAME = "voldemort_client_context"; /** * Instantiate the client config using a properties file @@ -240,10 +238,6 @@ private void setProperties(Properties properties) { if(props.containsKey(MAX_BOOTSTRAP_RETRIES)) this.setMaxBootstrapRetries(props.getInt(MAX_BOOTSTRAP_RETRIES)); - - if(props.containsKey(CLIENT_CONTEXT_NAME)) { - this.setClientContextName(props.getString(CLIENT_CONTEXT_NAME, null)); - } } public int getMaxConnectionsPerNode() { @@ -523,7 +517,7 @@ public boolean isLazyEnabled() { /** * Enable lazy initialization of clients? - * + * * @param enableLazy If true clients will be lazily initialized */ public ClientConfig setEnableLazy(boolean enableLazy) { @@ -640,13 +634,4 @@ public ClientConfig setMaxBootstrapRetries(int maxBootstrapRetries) { return this; } - public String getClientContextName() { - return clientContextName; - } - - public ClientConfig setClientContextName(String clientContextName) { - this.clientContextName = clientContextName; - return this; - } - } diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index 0f90c04a31..832339b243 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -18,7 +18,6 @@ import java.util.List; import java.util.Map; -import java.util.UUID; import java.util.Map.Entry; import org.apache.log4j.Logger; @@ -64,45 +63,29 @@ public class DefaultStoreClient implements StoreClient { private final String storeName; private final InconsistencyResolver> resolver; private volatile Store store; - private final UUID clientId; public DefaultStoreClient(String storeName, InconsistencyResolver> resolver, StoreClientFactory storeFactory, int maxMetadataRefreshAttempts) { - this(storeName, resolver, storeFactory, maxMetadataRefreshAttempts, null, 0); - } - - public DefaultStoreClient(String storeName, - InconsistencyResolver> resolver, - StoreClientFactory storeFactory, - int maxMetadataRefreshAttempts, - String clientContext, - int clientSequence) { - this.storeName = Utils.notNull(storeName); this.resolver = resolver; this.storeFactory = Utils.notNull(storeFactory); this.metadataRefreshAttempts = maxMetadataRefreshAttempts; - this.clientId = AbstractStoreClientFactory.generateClientId(storeName, - clientContext, - clientSequence); + // Registering self to be able to bootstrap client dynamically via JMX JmxUtils.registerMbean(this, JmxUtils.createObjectName(JmxUtils.getPackageName(this.getClass()), JmxUtils.getClassName(this.getClass()) - + "." + clientContext + "." - + storeName + "." - + clientId.toString())); + + "." + storeName)); + bootStrap(); - logger.info("Voldemort client created: clientContext=" + clientContext + " clientSequence=" - + clientSequence + " clientId=" + clientId.toString()); } @JmxOperation(description = "bootstrap metadata from the cluster.") public void bootStrap() { logger.info("Bootstrapping metadata for store " + this.storeName); - this.store = storeFactory.getRawStore(storeName, resolver, clientId); + this.store = storeFactory.getRawStore(storeName, resolver); } public boolean delete(K key) { @@ -372,8 +355,4 @@ else if(versions.size() == 1) return put(key, versioned, transforms); } - - public UUID getClientId() { - return clientId; - } } diff --git a/src/java/voldemort/client/MockStoreClientFactory.java b/src/java/voldemort/client/MockStoreClientFactory.java index 32a2997842..80613b653b 100644 --- a/src/java/voldemort/client/MockStoreClientFactory.java +++ b/src/java/voldemort/client/MockStoreClientFactory.java @@ -18,7 +18,6 @@ import java.io.StringReader; import java.util.List; -import java.util.UUID; import voldemort.cluster.failuredetector.FailureDetector; import voldemort.cluster.failuredetector.NoopFailureDetector; @@ -107,8 +106,7 @@ public StoreClient getStoreClient(String storeName, } public Store getRawStore(String storeName, - InconsistencyResolver> resolver, - UUID clientId) { + InconsistencyResolver> resolver) { if(this.storesXml != null) return getRawStore(storeName); @@ -133,11 +131,6 @@ public Store getRawStore(String storeName, return consistentStore; } - public Store getRawStore(String storeName, - InconsistencyResolver> resolver) { - return getRawStore(storeName, resolver, null); - } - private Store getRawStore(String storeName) { List storeDefs = storeMapper.readStoreList(new StringReader(storesXml)); StoreDefinition storeDef = null; diff --git a/src/java/voldemort/client/StoreClientFactory.java b/src/java/voldemort/client/StoreClientFactory.java index f8fa8e710e..166eac0c1d 100644 --- a/src/java/voldemort/client/StoreClientFactory.java +++ b/src/java/voldemort/client/StoreClientFactory.java @@ -16,8 +16,6 @@ package voldemort.client; -import java.util.UUID; - import voldemort.cluster.failuredetector.FailureDetector; import voldemort.store.Store; import voldemort.versioning.InconsistencyResolver; @@ -67,6 +65,9 @@ public StoreClient getStoreClient(String storeName, /** * Get the underlying store, not the public StoreClient interface * + * @param The key type + * @param The value type + * @param The transform type * @param storeName The name of the store * @param resolver The inconsistency resolver * @return The appropriate store @@ -74,18 +75,6 @@ public StoreClient getStoreClient(String storeName, Store getRawStore(String storeName, InconsistencyResolver> resolver); - /** - * Get the underlying store, not the public StoreClient interface - * - * @param storeName The name of the store - * @param resolver The inconsistency resolver - * @param clientId The unique id of the client - * @return The appropriate store - */ - Store getRawStore(String storeName, - InconsistencyResolver> resolver, - UUID clientId); - /** * Close the store client */ diff --git a/test/common/voldemort/StaticStoreClientFactory.java b/test/common/voldemort/StaticStoreClientFactory.java index c1ec4c4513..b93d1d73c0 100644 --- a/test/common/voldemort/StaticStoreClientFactory.java +++ b/test/common/voldemort/StaticStoreClientFactory.java @@ -2,7 +2,6 @@ import java.util.Arrays; import java.util.List; -import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import voldemort.client.DefaultStoreClient; @@ -41,17 +40,10 @@ public StaticStoreClientFactory(Store... stores) { failureDetector = new NoopFailureDetector(); } - @SuppressWarnings("unchecked") - public Store getRawStore(String storeName, - InconsistencyResolver> resolver, - UUID clientId) { - return (Store) stores.get(Math.max(current.getAndIncrement(), stores.size() - 1)); - } - @SuppressWarnings("unchecked") public Store getRawStore(String storeName, InconsistencyResolver> resolver) { - return getRawStore(storeName, resolver, null); + return (Store) stores.get(Math.max(current.getAndIncrement(), stores.size() - 1)); } @SuppressWarnings("unchecked") diff --git a/test/unit/voldemort/client/ClientJmxTest.java b/test/unit/voldemort/client/ClientJmxTest.java deleted file mode 100644 index 1c215580f8..0000000000 --- a/test/unit/voldemort/client/ClientJmxTest.java +++ /dev/null @@ -1,341 +0,0 @@ -package voldemort.client; - -import java.lang.management.ManagementFactory; -import java.net.URISyntaxException; - -import javax.management.InstanceNotFoundException; -import javax.management.MBeanServer; -import javax.management.ObjectName; - -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import voldemort.ServerTestUtils; -import voldemort.serialization.SerializerFactory; -import voldemort.server.AbstractSocketService; -import voldemort.utils.JmxUtils; - -public class ClientJmxTest extends AbstractStoreClientFactoryTest { - - private static String STATS_DOMAIN = "voldemort.store.stats"; - private static String AGGREGATE_STATS_DOMAIN = "voldemort.store.stats.aggregate"; - private static String CLIENT_DOMAIN = "voldemort.client"; - private static String CLUSTER_FAILUREDETECTOR_DOMAIN = "voldemort.cluster.failuredetector"; - private static String CLIENT_REQUEST_DOMAIN = "voldemort.store.socket.clientrequest"; - - private AbstractSocketService socketService; - private MBeanServer mbServer = null; - - private static int factoryJmxId = 0; - - public ClientJmxTest() { - super(); - } - - private static String getAndIncrementJmxId() { - int current = factoryJmxId; - factoryJmxId++; - return (0 == current ? "" : "." + current); - } - - @Override - @Before - public void setUp() throws Exception { - super.setUp(); - socketService = ServerTestUtils.getSocketService(true, - getClusterXml(), - getStoreDefXml(), - getValidStoreName(), - getLocalNode().getSocketPort()); - socketService.start(); - mbServer = ManagementFactory.getPlatformMBeanServer(); - } - - @Override - @After - public void tearDown() throws Exception { - mbServer = null; - super.tearDown(); - socketService.stop(); - } - - @Override - protected StoreClientFactory getFactory(String... bootstrapUrls) { - return new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrls) - .setEnableLazy(false) - .setEnableJmx(true)); - } - - protected StoreClientFactory getFactoryWithClientContext(String clientContext, - String... bootstrapUrls) { - return new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrls) - .setEnableLazy(false) - .setClientContextName(clientContext) - .setEnableJmx(true)); - } - - @Test - public void testTwoClientContextOnJmx() throws Exception { - String clientContext1 = "clientA"; - String clientContext2 = "clientB"; - String jmxId1 = getAndIncrementJmxId(); - String jmxId2 = getAndIncrementJmxId(); - - StoreClient c1 = getFactoryWithClientContext(clientContext1, - getValidBootstrapUrl()).getStoreClient(getValidStoreName()); - StoreClient c2 = getFactoryWithClientContext(clientContext2, - getValidBootstrapUrl()).getStoreClient(getValidStoreName()); - - // checking for aggregate stats - ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext1 - + ".aggregate-perf" - + jmxId1); - ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext2 - + ".aggregate-perf" - + jmxId2); - checkForMbeanFound(c1Name); - checkForMbeanFound(c2Name); - mbServer.unregisterMBean(c1Name); - mbServer.unregisterMBean(c2Name); - - // checking for per store stats - String c1type = clientContext1 + ".test" + jmxId1; - String c2type = clientContext2 + ".test" + jmxId2; - c1Name = JmxUtils.createObjectName(STATS_DOMAIN, - c1type - + "." - + ((DefaultStoreClient) c1).getClientId() - .toString()); - c2Name = JmxUtils.createObjectName(STATS_DOMAIN, - c2type - + "." - + ((DefaultStoreClient) c2).getClientId() - .toString()); - checkForMbeanFound(c1Name); - checkForMbeanFound(c2Name); - mbServer.unregisterMBean(c1Name); - mbServer.unregisterMBean(c2Name); - } - - @Test - public void testSameContextOnJmx() throws Exception { - String clientContext = "clientContext"; - String jmxId1 = getAndIncrementJmxId(); - String jmxId2 = getAndIncrementJmxId(); - - StoreClient[] clients = new StoreClient[2]; - for(int i = 0; i < 2; i++) { - clients[i] = getFactoryWithClientContext(clientContext, getValidBootstrapUrl()).getStoreClient(getValidStoreName()); - } - - // checking for aggregate stats - ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext - + ".aggregate-perf" - + jmxId1); - ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext - + ".aggregate-perf" - + jmxId2); - checkForMbeanFound(c1Name); - checkForMbeanFound(c2Name); - mbServer.unregisterMBean(c1Name); - mbServer.unregisterMBean(c2Name); - - // checking for per store stats - String c1type = clientContext + ".test" + jmxId1; - String c2type = clientContext + ".test" + jmxId2; - c1Name = JmxUtils.createObjectName(STATS_DOMAIN, - c1type - + "." - + ((DefaultStoreClient) clients[0]).getClientId() - .toString()); - c2Name = JmxUtils.createObjectName(STATS_DOMAIN, - c2type - + "." - + ((DefaultStoreClient) clients[1]).getClientId() - .toString()); - checkForMbeanFound(c1Name); - checkForMbeanFound(c2Name); - mbServer.unregisterMBean(c1Name); - mbServer.unregisterMBean(c2Name); - } - - @Test - public void testTwoClientNoContextOnJmx() throws Exception { - String clientContextCompare = "default"; - String jmxId1 = getAndIncrementJmxId(); - String jmxId2 = getAndIncrementJmxId(); - - StoreClient c1 = getFactory(getValidBootstrapUrl()).getStoreClient(getValidStoreName()); - StoreClient c2 = getFactory(getValidBootstrapUrl()).getStoreClient(getValidStoreName()); - - // checking for aggregate stats - ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare - + ".aggregate-perf" - + jmxId1); - ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare - + ".aggregate-perf" - + jmxId2); - checkForMbeanFound(c1Name); - checkForMbeanFound(c2Name); - mbServer.unregisterMBean(c1Name); - mbServer.unregisterMBean(c2Name); - - // checking for per store stats - String c1type = clientContextCompare + ".test" + jmxId1; - String c2type = clientContextCompare + ".test" + jmxId2; - c1Name = JmxUtils.createObjectName(STATS_DOMAIN, - c1type - + "." - + ((DefaultStoreClient) c1).getClientId() - .toString()); - c2Name = JmxUtils.createObjectName(STATS_DOMAIN, - c2type - + "." - + ((DefaultStoreClient) c2).getClientId() - .toString()); - checkForMbeanFound(c1Name); - checkForMbeanFound(c2Name); - mbServer.unregisterMBean(c1Name); - mbServer.unregisterMBean(c2Name); - } - - @Test - public void testTwoClientNullContextOnJmx() throws Exception { - String clientContextCompare = ""; - String jmxId1 = getAndIncrementJmxId(); - String jmxId2 = getAndIncrementJmxId(); - - StoreClient c1 = getFactoryWithClientContext(null, getValidBootstrapUrl()).getStoreClient(getValidStoreName()); - StoreClient c2 = getFactoryWithClientContext(null, getValidBootstrapUrl()).getStoreClient(getValidStoreName()); - - // checking for aggregate stats - ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare - + ".aggregate-perf" - + jmxId1); - ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare - + ".aggregate-perf" - + jmxId2); - checkForMbeanFound(c1Name); - checkForMbeanFound(c2Name); - mbServer.unregisterMBean(c1Name); - mbServer.unregisterMBean(c2Name); - - // checking for per store stats - String c1type = clientContextCompare + ".test" + jmxId1; - String c2type = clientContextCompare + ".test" + jmxId2; - c1Name = JmxUtils.createObjectName(STATS_DOMAIN, - c1type - + "." - + ((DefaultStoreClient) c1).getClientId() - .toString()); - c2Name = JmxUtils.createObjectName(STATS_DOMAIN, - c2type - + "." - + ((DefaultStoreClient) c2).getClientId() - .toString()); - checkForMbeanFound(c1Name); - checkForMbeanFound(c2Name); - mbServer.unregisterMBean(c1Name); - mbServer.unregisterMBean(c2Name); - } - - @Test - public void testSameContextAndFactory() throws Exception { - String clientContext = "clientContext"; - String jmxId = getAndIncrementJmxId(); - StoreClientFactory factory = getFactoryWithClientContext(clientContext, - getValidBootstrapUrl()); - - StoreClient[] clients = new StoreClient[2]; - for(int i = 0; i < 2; i++) { - clients[i] = factory.getStoreClient(getValidStoreName()); - } - - ObjectName cName = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext - + ".aggregate-perf" - + jmxId); - checkForMbeanFound(cName); - mbServer.unregisterMBean(cName); - - // checking for per store stats - String ctype = clientContext + ".test" + jmxId; - ObjectName c1Name = JmxUtils.createObjectName(STATS_DOMAIN, - ctype - + "." - + ((DefaultStoreClient) clients[0]).getClientId() - .toString()); - ObjectName c2Name = JmxUtils.createObjectName(STATS_DOMAIN, - ctype - + "." - + ((DefaultStoreClient) clients[1]).getClientId() - .toString()); - checkForMbeanFound(c1Name); - checkForMbeanFound(c2Name); - mbServer.unregisterMBean(c1Name); - } - - @Test - public void testDifferentId() throws Exception { - String clientContext = "clientContext"; - String jmxId = getAndIncrementJmxId(); - StoreClientFactory factory = getFactoryWithClientContext(clientContext, - getValidBootstrapUrl()); - - StoreClient[] clients = new StoreClient[2]; - clients[0] = factory.getStoreClient(getValidStoreName()); - clients[1] = factory.getStoreClient(getValidStoreName()); - - ObjectName cName = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext - + ".aggregate-perf" - + jmxId); - checkForMbeanFound(cName); - mbServer.unregisterMBean(cName); - - // checking for per store stats - String ctype = clientContext + ".test" + jmxId; - ObjectName c1Name = JmxUtils.createObjectName(STATS_DOMAIN, - ctype - + "." - + ((DefaultStoreClient) clients[0]).getClientId() - .toString()); - ObjectName c2Name = JmxUtils.createObjectName(STATS_DOMAIN, - ctype - + "." - + ((DefaultStoreClient) clients[1]).getClientId() - .toString()); - checkForMbeanFound(c1Name); - checkForMbeanFound(c2Name); - assertTrue(!c1Name.equals(c2Name)); - mbServer.unregisterMBean(c1Name); - mbServer.unregisterMBean(c2Name); - } - - private void checkForMbeanFound(ObjectName name) { - try { - mbServer.getMBeanInfo(name); - } catch(InstanceNotFoundException e) { - fail("MBean not found on the JMX Server: " + name.toString()); - } catch(Exception e) { - fail("Test failed: " + e.getMessage()); - } - } - - @Override - protected StoreClientFactory getFactoryWithSerializer(SerializerFactory factory, - String... bootstrapUrls) { - return new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrls) - .setEnableLazy(false) - .setSerializerFactory(factory)); - } - - @Override - protected String getValidBootstrapUrl() throws URISyntaxException { - return getLocalNode().getSocketUrl().toString(); - } - - @Override - protected String getValidScheme() { - return SocketStoreClientFactory.URL_SCHEME; - } -} From 1c0a49af1dfb4d710749f07f11a4460ebd7c350a Mon Sep 17 00:00:00 2001 From: shingon Date: Tue, 15 May 2012 15:23:54 +0900 Subject: [PATCH 031/209] fix wrong file name in generate_cluster_xml.py --- bin/generate_cluster_xml.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/generate_cluster_xml.py b/bin/generate_cluster_xml.py index 09baf35894..3cd56644d7 100644 --- a/bin/generate_cluster_xml.py +++ b/bin/generate_cluster_xml.py @@ -2,7 +2,7 @@ import random if len(sys.argv) != 3: - print >> sys.stderr, "USAGE: python generate_partitions.py " + print >> sys.stderr, "USAGE: python generate_cluster_xml.py " sys.exit() FORMAT_WIDTH = 10 From be89c224a792eef2283a548ed007edadada05911 Mon Sep 17 00:00:00 2001 From: shingon Date: Wed, 16 May 2012 16:11:54 +0900 Subject: [PATCH 032/209] fix a typo. boostrap -> bootstrap --- src/java/voldemort/client/AbstractStoreClientFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index fb82936c50..ab0c7d998e 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -305,7 +305,7 @@ public String bootstrapMetadataWithRetries(String key, URI[] urls) { } } - throw new BootstrapFailureException("No available boostrap servers found!"); + throw new BootstrapFailureException("No available bootstrap servers found!"); } public String bootstrapMetadataWithRetries(String key) { From 53a90199026a10474fc6552d0c81e0453fa90aab Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Thu, 17 May 2012 01:04:31 -0700 Subject: [PATCH 033/209] add a java file to define all system store constants, including system store defs --- .../server/SystemStoreConstants.java | 58 +++++ .../protocol/AbstractRequestHandler.java | 1 + .../server/storage/StorageService.java | 200 ++++++++++++++++++ src/java/voldemort/store/StoreDefinition.java | 4 +- .../voldemort/xml/StoreDefinitionsMapper.java | 4 +- 5 files changed, 265 insertions(+), 2 deletions(-) create mode 100644 src/java/voldemort/server/SystemStoreConstants.java diff --git a/src/java/voldemort/server/SystemStoreConstants.java b/src/java/voldemort/server/SystemStoreConstants.java new file mode 100644 index 0000000000..820037a920 --- /dev/null +++ b/src/java/voldemort/server/SystemStoreConstants.java @@ -0,0 +1,58 @@ +package voldemort.server; + +/** + * The various system stores + */ +public class SystemStoreConstants { + + private static final String NAME_PREFIX = "voldsys$_"; + + public static enum SystemStoreName { + voldsys$_client_registry, + voldsys$_client_store_definition; + } + + public static final String SYSTEM_STORE_SCHEMA = "" + + " " + + " voldsys$_client_registry" + + " zone-routing" + + " proximity-handoff" + + " memory" + + " client" + + " 4" + + " " + + " 2" + + " 2" + + " " + + " 1" + + " 1" + + " " + + " string" + + " " + + " " + + " string" + + " " + + " 7" + + " " + + " " + + " voldsys$_client_store_definition" + + " zone-routing" + + " proximity-handoff" + + " memory" + + " client" + + " 1" + + " 1" + + " 1" + + " " + + " string" + + " " + + " " + + " string" + + " " + + " 7" + + " " + ""; + + public static boolean isSystemStore(String storeName) { + return (null == storeName ? false : storeName.startsWith(NAME_PREFIX)); + } +} \ No newline at end of file diff --git a/src/java/voldemort/server/protocol/AbstractRequestHandler.java b/src/java/voldemort/server/protocol/AbstractRequestHandler.java index b549a346b9..c9605abe7e 100644 --- a/src/java/voldemort/server/protocol/AbstractRequestHandler.java +++ b/src/java/voldemort/server/protocol/AbstractRequestHandler.java @@ -32,6 +32,7 @@ protected StoreRepository getStoreRepository() { } protected Store getStore(String name, RequestRoutingType type) { + switch(type) { case ROUTED: return storeRepository.getRoutedStore(name); diff --git a/src/java/voldemort/server/storage/StorageService.java b/src/java/voldemort/server/storage/StorageService.java index 5a26c3e86d..22e85079a2 100644 --- a/src/java/voldemort/server/storage/StorageService.java +++ b/src/java/voldemort/server/storage/StorageService.java @@ -18,6 +18,7 @@ import static voldemort.cluster.failuredetector.FailureDetectorUtils.create; +import java.io.StringReader; import java.lang.management.ManagementFactory; import java.util.ArrayList; import java.util.Calendar; @@ -54,6 +55,7 @@ import voldemort.server.RequestRoutingType; import voldemort.server.ServiceType; import voldemort.server.StoreRepository; +import voldemort.server.SystemStoreConstants; import voldemort.server.VoldemortConfig; import voldemort.server.scheduler.DataCleanupJob; import voldemort.server.scheduler.SchedulerService; @@ -65,6 +67,7 @@ import voldemort.store.StoreDefinition; import voldemort.store.invalidmetadata.InvalidMetadataCheckingStore; import voldemort.store.logging.LoggingStore; +import voldemort.store.memory.InMemoryStorageConfiguration; import voldemort.store.metadata.MetadataStore; import voldemort.store.metadata.MetadataStoreListener; import voldemort.store.nonblockingstore.NonblockingStore; @@ -97,6 +100,7 @@ import voldemort.versioning.VectorClock; import voldemort.versioning.VectorClockInconsistencyResolver; import voldemort.versioning.Versioned; +import voldemort.xml.StoreDefinitionsMapper; /** * The service responsible for managing all storage types @@ -170,6 +174,7 @@ public StorageService(StoreRepository storeRepository, } private void initStorageConfig(String configClassName) { + // add the configurations of the storage engines needed by user stores try { Class configClass = ReflectUtils.loadClass(configClassName); StorageConfiguration configuration = (StorageConfiguration) ReflectUtils.callConstructor(configClass, @@ -187,6 +192,37 @@ private void initStorageConfig(String configClassName) { if(storageConfigs.size() == 0) throw new ConfigurationException("No storage engine has been enabled!"); + + // now, add the configurations of the storage engines needed by system + // stores, if not yet exist + initSystemStorageConfig(); + } + + private void initSystemStorageConfig() { + // add InMemoryStorage used by voldsys$_client_registry + if(!storageConfigs.containsKey(InMemoryStorageConfiguration.TYPE_NAME)) { + storageConfigs.put(InMemoryStorageConfiguration.TYPE_NAME, + new InMemoryStorageConfiguration()); + } + + // add FileStorage config here + } + + private void initSystemStores() { + List storesDefs = (new StoreDefinitionsMapper()).readStoreList(new StringReader(SystemStoreConstants.SYSTEM_STORE_SCHEMA)); + + // TODO: replication factor can't now be determined unless the + // cluster.xml is made available to the server at runtime. So we need to + // set them here after load they are loaded + updateRepFactor(storesDefs); + + for(StoreDefinition storeDef: storesDefs) { + openSystemStore(storeDef); + } + } + + private void updateRepFactor(List storesDefs) { + // need impl } @Override @@ -203,6 +239,9 @@ protected void startInner() { metadata.getStoreDefList(), storeRepository)); + /* Initialize system stores */ + initSystemStores(); + /* Register slop store */ if(voldemortConfig.isSlopEnabled()) { @@ -282,6 +321,167 @@ protected void startInner() { logger.info("All stores initialized."); } + public void openSystemStore(StoreDefinition storeDef) { + + logger.info("Opening system store '" + storeDef.getName() + "' (" + storeDef.getType() + + ")."); + + StorageConfiguration config = storageConfigs.get(storeDef.getType()); + if(config == null) + throw new ConfigurationException("Attempt to open system store " + storeDef.getName() + + " but " + storeDef.getType() + + " storage engine has not been enabled."); + + final StorageEngine engine = config.getStore(storeDef.getName()); + + // Noted that there is no read-only processing as for user stores. + + // openStore() should have atomic semantics + try { + registerSystemEngine(engine); + + if(voldemortConfig.isServerRoutingEnabled()) + registerNodeStores(storeDef, metadata.getCluster(), voldemortConfig.getNodeId()); + + if(storeDef.hasRetentionPeriod()) + scheduleCleanupJob(storeDef, engine); + } catch(Exception e) { + unregisterSystemEngine(engine); + throw new VoldemortException(e); + } + } + + public void registerSystemEngine(StorageEngine engine) { + + Cluster cluster = this.metadata.getCluster(); + storeRepository.addStorageEngine(engine); + + /* Now add any store wrappers that are enabled */ + Store store = engine; + + if(voldemortConfig.isVerboseLoggingEnabled()) + store = new LoggingStore(store, + cluster.getName(), + SystemTime.INSTANCE); + /* TODO: Do we really need rebalancing for system stores? */ + if(voldemortConfig.isEnableRebalanceService()) { + store = new RedirectingStore(store, + metadata, + storeRepository, + failureDetector, + storeFactory); + if(voldemortConfig.isJmxEnabled()) { + MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer(); + ObjectName name = null; + if(this.voldemortConfig.isEnableJmxClusterName()) + name = JmxUtils.createObjectName(cluster.getName() + + "." + + JmxUtils.getPackageName(RedirectingStore.class), + store.getName()); + else + name = JmxUtils.createObjectName(JmxUtils.getPackageName(RedirectingStore.class), + store.getName()); + + synchronized(mbeanServer) { + if(mbeanServer.isRegistered(name)) + JmxUtils.unregisterMbean(mbeanServer, name); + JmxUtils.registerMbean(mbeanServer, JmxUtils.createModelMBean(store), name); + } + + } + } + + if(voldemortConfig.isMetadataCheckingEnabled()) + store = new InvalidMetadataCheckingStore(metadata.getNodeId(), store, metadata); + + if(voldemortConfig.isStatTrackingEnabled()) { + StatTrackingStore statStore = new StatTrackingStore(store, this.storeStats); + store = statStore; + if(voldemortConfig.isJmxEnabled()) { + + MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer(); + ObjectName name = null; + if(this.voldemortConfig.isEnableJmxClusterName()) + name = JmxUtils.createObjectName(metadata.getCluster().getName() + + "." + + JmxUtils.getPackageName(store.getClass()), + store.getName()); + else + name = JmxUtils.createObjectName(JmxUtils.getPackageName(store.getClass()), + store.getName()); + + synchronized(mbeanServer) { + if(mbeanServer.isRegistered(name)) + JmxUtils.unregisterMbean(mbeanServer, name); + + JmxUtils.registerMbean(mbeanServer, + JmxUtils.createModelMBean(new StoreStatsJmx(statStore.getStats())), + name); + } + } + } + + storeRepository.addLocalStore(store); + } + + public void unregisterSystemEngine(StorageEngine engine) { + String storeName = engine.getName(); + Store store = storeRepository.removeLocalStore(storeName); + + if(store != null) { + if(voldemortConfig.isJmxEnabled()) { + MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer(); + + if(voldemortConfig.isEnableRebalanceService()) { + + ObjectName name = null; + if(this.voldemortConfig.isEnableJmxClusterName()) + name = JmxUtils.createObjectName(metadata.getCluster().getName() + + "." + + JmxUtils.getPackageName(RedirectingStore.class), + store.getName()); + else + name = JmxUtils.createObjectName(JmxUtils.getPackageName(RedirectingStore.class), + store.getName()); + + synchronized(mbeanServer) { + if(mbeanServer.isRegistered(name)) + JmxUtils.unregisterMbean(mbeanServer, name); + } + + } + + if(voldemortConfig.isStatTrackingEnabled()) { + ObjectName name = null; + if(this.voldemortConfig.isEnableJmxClusterName()) + name = JmxUtils.createObjectName(metadata.getCluster().getName() + + "." + + JmxUtils.getPackageName(store.getClass()), + store.getName()); + else + name = JmxUtils.createObjectName(JmxUtils.getPackageName(store.getClass()), + store.getName()); + + synchronized(mbeanServer) { + if(mbeanServer.isRegistered(name)) + JmxUtils.unregisterMbean(mbeanServer, name); + } + + } + } + if(voldemortConfig.isServerRoutingEnabled()) { + this.storeRepository.removeRoutedStore(storeName); + for(Node node: metadata.getCluster().getNodes()) + this.storeRepository.removeNodeStore(storeName, node.getId()); + } + } + + storeRepository.removeStorageEngine(storeName); + // engine.truncate(); why truncate here when unregister? Isn't close + // good enough? + engine.close(); + } + public void openStore(StoreDefinition storeDef) { logger.info("Opening store '" + storeDef.getName() + "' (" + storeDef.getType() + ")."); diff --git a/src/java/voldemort/store/StoreDefinition.java b/src/java/voldemort/store/StoreDefinition.java index 0df48fcf84..52d4dda7f0 100644 --- a/src/java/voldemort/store/StoreDefinition.java +++ b/src/java/voldemort/store/StoreDefinition.java @@ -22,6 +22,7 @@ import voldemort.client.RoutingTier; import voldemort.serialization.SerializerDefinition; +import voldemort.server.SystemStoreConstants; import voldemort.store.slop.strategy.HintedHandoffStrategyType; import voldemort.utils.Utils; @@ -139,7 +140,8 @@ else if(requiredWrites > replicationFactor) if(retentionPeriodDays != null && retentionPeriodDays < 0) throw new IllegalArgumentException("Retention days must be non-negative."); - if(zoneReplicationFactor != null && zoneReplicationFactor.size() != 0) { + if(!SystemStoreConstants.isSystemStore(name) && zoneReplicationFactor != null + && zoneReplicationFactor.size() != 0) { if(zoneCountReads == null || zoneCountReads < 0) throw new IllegalArgumentException("Zone Counts reads must be non-negative / non-null"); diff --git a/src/java/voldemort/xml/StoreDefinitionsMapper.java b/src/java/voldemort/xml/StoreDefinitionsMapper.java index c51913d1a5..d1d4948b24 100644 --- a/src/java/voldemort/xml/StoreDefinitionsMapper.java +++ b/src/java/voldemort/xml/StoreDefinitionsMapper.java @@ -46,6 +46,7 @@ import voldemort.routing.RoutingStrategyType; import voldemort.serialization.Compression; import voldemort.serialization.SerializerDefinition; +import voldemort.server.SystemStoreConstants; import voldemort.store.StoreDefinition; import voldemort.store.StoreDefinitionBuilder; import voldemort.store.StoreUtils; @@ -227,7 +228,8 @@ private StoreDefinition readStore(Element store) { retentionThrottleRate = Integer.parseInt(throttleRate.getText()); } - if(routingStrategyType.compareTo(RoutingStrategyType.ZONE_STRATEGY) == 0) { + if(routingStrategyType.compareTo(RoutingStrategyType.ZONE_STRATEGY) == 0 + && !SystemStoreConstants.isSystemStore(name)) { if(zoneCountReads == null || zoneCountWrites == null || zoneReplicationFactor == null) { throw new MappingException("Have not set one of the following correctly for store '" + name From dc71868e4e27a0e602669e72e1c9c54d301f6058 Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Thu, 15 Mar 2012 12:40:44 -0700 Subject: [PATCH 034/209] add clientId for voldemort client --- .../client/AbstractStoreClientFactory.java | 77 +++- .../client/CachingStoreClientFactory.java | 32 +- src/java/voldemort/client/ClientConfig.java | 17 +- .../voldemort/client/DefaultStoreClient.java | 29 +- .../client/MockStoreClientFactory.java | 9 +- .../voldemort/client/StoreClientFactory.java | 17 +- .../voldemort/StaticStoreClientFactory.java | 10 +- test/unit/voldemort/client/ClientJmxTest.java | 341 ++++++++++++++++++ 8 files changed, 505 insertions(+), 27 deletions(-) create mode 100644 test/unit/voldemort/client/ClientJmxTest.java diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index fb82936c50..e5072be68b 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -16,12 +16,17 @@ package voldemort.client; +import java.io.File; +import java.io.IOException; import java.io.StringReader; +import java.net.InetAddress; import java.net.URI; import java.net.URISyntaxException; +import java.net.UnknownHostException; import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -98,6 +103,8 @@ public abstract class AbstractStoreClientFactory implements StoreClientFactory { private final ClientConfig config; private final RoutedStoreFactory routedStoreFactory; private final int clientZoneId; + private final String clientContextName; + private final AtomicInteger sequencer; public AbstractStoreClientFactory(ClientConfig config) { this.config = config; @@ -112,18 +119,24 @@ public AbstractStoreClientFactory(ClientConfig config) { this.maxBootstrapRetries = config.getMaxBootstrapRetries(); this.stats = new StoreStats(); this.clientZoneId = config.getClientZoneId(); + this.clientContextName = (null == config.getClientContextName() ? "" + : config.getClientContextName()); this.routedStoreFactory = new RoutedStoreFactory(config.isPipelineRoutedStoreEnabled(), threadPool, config.getRoutingTimeout(TimeUnit.MILLISECONDS)); + this.sequencer = new AtomicInteger(0); if(this.isJmxEnabled) { JmxUtils.registerMbean(threadPool, JmxUtils.createObjectName(JmxUtils.getPackageName(threadPool.getClass()), JmxUtils.getClassName(threadPool.getClass()) + + "." + + clientContextName + jmxId())); JmxUtils.registerMbean(new StoreStatsJmx(stats), JmxUtils.createObjectName("voldemort.store.stats.aggregate", - "aggregate-perf" + jmxId())); + clientContextName + ".aggregate-perf" + + jmxId())); } } @@ -133,12 +146,18 @@ public StoreClient getStoreClient(String storeName) { public StoreClient getStoreClient(String storeName, InconsistencyResolver> resolver) { - return new DefaultStoreClient(storeName, resolver, this, 3); + return new DefaultStoreClient(storeName, + resolver, + this, + 3, + clientContextName, + sequencer.getAndIncrement()); } @SuppressWarnings("unchecked") public Store getRawStore(String storeName, - InconsistencyResolver> resolver) { + InconsistencyResolver> resolver, + UUID clientId) { logger.info("Client zone-id [" + clientZoneId + "] Attempting to obtain metadata for store [" + storeName + "] "); @@ -223,7 +242,13 @@ public Store getRawStore(String storeName, store = statStore; JmxUtils.registerMbean(new StoreStatsJmx(statStore.getStats()), JmxUtils.createObjectName(JmxUtils.getPackageName(store.getClass()), - store.getName() + jmxId())); + clientContextName + + "." + + store.getName() + + jmxId() + + (null == clientId ? "" + : "." + + clientId.toString()))); } if(storeDef.getKeySerializer().hasCompression() @@ -257,6 +282,11 @@ public Store getRawStore(String storeName, return serializedStore; } + public Store getRawStore(String storeName, + InconsistencyResolver> resolver) { + return getRawStore(storeName, resolver, null); + } + protected ClientConfig getConfig() { return config; } @@ -401,7 +431,44 @@ public void close() { /* Give a unique id to avoid jmx clashes */ private String jmxId() { - return jmxId == 0 ? "" : Integer.toString(jmxId); + return jmxId == 0 ? "" : "." + Integer.toString(jmxId); } + /** + * Generate a unique client ID based on: 0. clientContext, if specified; 1. + * storeName 2. run path 3. client sequence + * + * @param storeName the name of the store the client is created for + * @param contextName the name of the client context + * @param clientSequence the client sequence number + * @return unique client ID + */ + public static UUID generateClientId(String storeName, String contextName, int clientSequence) { + String newLine = System.getProperty("line.separator"); + StringBuilder context = new StringBuilder(contextName == null ? "" : contextName); + context.append(0 == clientSequence ? "" : ("." + clientSequence)); + context.append(".").append(storeName); + + try { + InetAddress host = InetAddress.getLocalHost(); + context.append("@").append(host.getHostName()).append(":"); + } catch(UnknownHostException e) { + logger.info("Unable to obtain client hostname."); + logger.info(e.getMessage()); + } + + try { + String currentPath = new File(".").getCanonicalPath(); + context.append(currentPath).append(newLine); + } catch(IOException e) { + logger.info("Unable to obtain client run path."); + logger.info(e.getMessage()); + } + + if(logger.isDebugEnabled()) { + logger.debug(context.toString()); + } + + return UUID.nameUUIDFromBytes(context.toString().getBytes()); + } } diff --git a/src/java/voldemort/client/CachingStoreClientFactory.java b/src/java/voldemort/client/CachingStoreClientFactory.java index 993ff7c3a5..5fd2953da3 100644 --- a/src/java/voldemort/client/CachingStoreClientFactory.java +++ b/src/java/voldemort/client/CachingStoreClientFactory.java @@ -1,12 +1,12 @@ /* * Copyright 2008-2010 LinkedIn, Inc - * + * * Licensed under the Apache License, Version 2.0 (the "License"); you may not * use this file except in compliance with the License. You may obtain a copy of * the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the @@ -16,8 +16,13 @@ package voldemort.client; -import com.google.common.collect.ImmutableList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + import org.apache.log4j.Logger; + import voldemort.annotations.jmx.JmxManaged; import voldemort.annotations.jmx.JmxOperation; import voldemort.cluster.failuredetector.FailureDetector; @@ -26,14 +31,12 @@ import voldemort.versioning.InconsistencyResolver; import voldemort.versioning.Versioned; -import java.util.List; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; +import com.google.common.collect.ImmutableList; /** - * A wrapper for a store {@link StoreClientFactory} which caches requests - * to getStoreClient - * + * A wrapper for a store {@link StoreClientFactory} which caches requests to + * getStoreClient + * */ @JmxManaged(description = "A StoreClientFactory which caches clients") public class CachingStoreClientFactory implements StoreClientFactory { @@ -48,7 +51,6 @@ public CachingStoreClientFactory(StoreClientFactory inner) { this.cache = new ConcurrentHashMap, StoreClient>(); } - @SuppressWarnings("unchecked") public StoreClient getStoreClient(String storeName) { Pair key = Pair.create(storeName, null); @@ -74,7 +76,13 @@ public StoreClient getStoreClient(String storeName, public Store getRawStore(String storeName, InconsistencyResolver> resolver) { - return inner.getRawStore(storeName, resolver); + return getRawStore(storeName, resolver, null); + } + + public Store getRawStore(String storeName, + InconsistencyResolver> resolver, + UUID clientId) { + return inner.getRawStore(storeName, resolver, clientId); } public void close() { diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index f74f31c04d..30e4ad9dad 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -76,6 +76,7 @@ public class ClientConfig { private long failureDetectorRequestLengthThreshold = socketTimeoutMs; private volatile int maxBootstrapRetries = 2; + private volatile String clientContextName = "default"; public ClientConfig() {} @@ -110,6 +111,7 @@ public ClientConfig() {} public static final String FAILUREDETECTOR_CATASTROPHIC_ERROR_TYPES_PROPERTY = "failuredetector_catastrophic_error_types"; public static final String FAILUREDETECTOR_REQUEST_LENGTH_THRESHOLD_PROPERTY = "failuredetector_request_length_threshold"; public static final String MAX_BOOTSTRAP_RETRIES = "max_bootstrap_retries"; + public static final String CLIENT_CONTEXT_NAME = "voldemort_client_context"; /** * Instantiate the client config using a properties file @@ -238,6 +240,10 @@ private void setProperties(Properties properties) { if(props.containsKey(MAX_BOOTSTRAP_RETRIES)) this.setMaxBootstrapRetries(props.getInt(MAX_BOOTSTRAP_RETRIES)); + + if(props.containsKey(CLIENT_CONTEXT_NAME)) { + this.setClientContextName(props.getString(CLIENT_CONTEXT_NAME, null)); + } } public int getMaxConnectionsPerNode() { @@ -517,7 +523,7 @@ public boolean isLazyEnabled() { /** * Enable lazy initialization of clients? - * + * * @param enableLazy If true clients will be lazily initialized */ public ClientConfig setEnableLazy(boolean enableLazy) { @@ -634,4 +640,13 @@ public ClientConfig setMaxBootstrapRetries(int maxBootstrapRetries) { return this; } + public String getClientContextName() { + return clientContextName; + } + + public ClientConfig setClientContextName(String clientContextName) { + this.clientContextName = clientContextName; + return this; + } + } diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index 832339b243..0f90c04a31 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -18,6 +18,7 @@ import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.Map.Entry; import org.apache.log4j.Logger; @@ -63,29 +64,45 @@ public class DefaultStoreClient implements StoreClient { private final String storeName; private final InconsistencyResolver> resolver; private volatile Store store; + private final UUID clientId; public DefaultStoreClient(String storeName, InconsistencyResolver> resolver, StoreClientFactory storeFactory, int maxMetadataRefreshAttempts) { + this(storeName, resolver, storeFactory, maxMetadataRefreshAttempts, null, 0); + } + + public DefaultStoreClient(String storeName, + InconsistencyResolver> resolver, + StoreClientFactory storeFactory, + int maxMetadataRefreshAttempts, + String clientContext, + int clientSequence) { + this.storeName = Utils.notNull(storeName); this.resolver = resolver; this.storeFactory = Utils.notNull(storeFactory); this.metadataRefreshAttempts = maxMetadataRefreshAttempts; - + this.clientId = AbstractStoreClientFactory.generateClientId(storeName, + clientContext, + clientSequence); // Registering self to be able to bootstrap client dynamically via JMX JmxUtils.registerMbean(this, JmxUtils.createObjectName(JmxUtils.getPackageName(this.getClass()), JmxUtils.getClassName(this.getClass()) - + "." + storeName)); - + + "." + clientContext + "." + + storeName + "." + + clientId.toString())); bootStrap(); + logger.info("Voldemort client created: clientContext=" + clientContext + " clientSequence=" + + clientSequence + " clientId=" + clientId.toString()); } @JmxOperation(description = "bootstrap metadata from the cluster.") public void bootStrap() { logger.info("Bootstrapping metadata for store " + this.storeName); - this.store = storeFactory.getRawStore(storeName, resolver); + this.store = storeFactory.getRawStore(storeName, resolver, clientId); } public boolean delete(K key) { @@ -355,4 +372,8 @@ else if(versions.size() == 1) return put(key, versioned, transforms); } + + public UUID getClientId() { + return clientId; + } } diff --git a/src/java/voldemort/client/MockStoreClientFactory.java b/src/java/voldemort/client/MockStoreClientFactory.java index 80613b653b..32a2997842 100644 --- a/src/java/voldemort/client/MockStoreClientFactory.java +++ b/src/java/voldemort/client/MockStoreClientFactory.java @@ -18,6 +18,7 @@ import java.io.StringReader; import java.util.List; +import java.util.UUID; import voldemort.cluster.failuredetector.FailureDetector; import voldemort.cluster.failuredetector.NoopFailureDetector; @@ -106,7 +107,8 @@ public StoreClient getStoreClient(String storeName, } public Store getRawStore(String storeName, - InconsistencyResolver> resolver) { + InconsistencyResolver> resolver, + UUID clientId) { if(this.storesXml != null) return getRawStore(storeName); @@ -131,6 +133,11 @@ public Store getRawStore(String storeName, return consistentStore; } + public Store getRawStore(String storeName, + InconsistencyResolver> resolver) { + return getRawStore(storeName, resolver, null); + } + private Store getRawStore(String storeName) { List storeDefs = storeMapper.readStoreList(new StringReader(storesXml)); StoreDefinition storeDef = null; diff --git a/src/java/voldemort/client/StoreClientFactory.java b/src/java/voldemort/client/StoreClientFactory.java index 166eac0c1d..f8fa8e710e 100644 --- a/src/java/voldemort/client/StoreClientFactory.java +++ b/src/java/voldemort/client/StoreClientFactory.java @@ -16,6 +16,8 @@ package voldemort.client; +import java.util.UUID; + import voldemort.cluster.failuredetector.FailureDetector; import voldemort.store.Store; import voldemort.versioning.InconsistencyResolver; @@ -65,9 +67,6 @@ public StoreClient getStoreClient(String storeName, /** * Get the underlying store, not the public StoreClient interface * - * @param The key type - * @param The value type - * @param The transform type * @param storeName The name of the store * @param resolver The inconsistency resolver * @return The appropriate store @@ -75,6 +74,18 @@ public StoreClient getStoreClient(String storeName, Store getRawStore(String storeName, InconsistencyResolver> resolver); + /** + * Get the underlying store, not the public StoreClient interface + * + * @param storeName The name of the store + * @param resolver The inconsistency resolver + * @param clientId The unique id of the client + * @return The appropriate store + */ + Store getRawStore(String storeName, + InconsistencyResolver> resolver, + UUID clientId); + /** * Close the store client */ diff --git a/test/common/voldemort/StaticStoreClientFactory.java b/test/common/voldemort/StaticStoreClientFactory.java index b93d1d73c0..c1ec4c4513 100644 --- a/test/common/voldemort/StaticStoreClientFactory.java +++ b/test/common/voldemort/StaticStoreClientFactory.java @@ -2,6 +2,7 @@ import java.util.Arrays; import java.util.List; +import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import voldemort.client.DefaultStoreClient; @@ -42,10 +43,17 @@ public StaticStoreClientFactory(Store... stores) { @SuppressWarnings("unchecked") public Store getRawStore(String storeName, - InconsistencyResolver> resolver) { + InconsistencyResolver> resolver, + UUID clientId) { return (Store) stores.get(Math.max(current.getAndIncrement(), stores.size() - 1)); } + @SuppressWarnings("unchecked") + public Store getRawStore(String storeName, + InconsistencyResolver> resolver) { + return getRawStore(storeName, resolver, null); + } + @SuppressWarnings("unchecked") public StoreClient getStoreClient(String storeName) { return new DefaultStoreClient(storeName, null, this, 3); diff --git a/test/unit/voldemort/client/ClientJmxTest.java b/test/unit/voldemort/client/ClientJmxTest.java new file mode 100644 index 0000000000..1c215580f8 --- /dev/null +++ b/test/unit/voldemort/client/ClientJmxTest.java @@ -0,0 +1,341 @@ +package voldemort.client; + +import java.lang.management.ManagementFactory; +import java.net.URISyntaxException; + +import javax.management.InstanceNotFoundException; +import javax.management.MBeanServer; +import javax.management.ObjectName; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import voldemort.ServerTestUtils; +import voldemort.serialization.SerializerFactory; +import voldemort.server.AbstractSocketService; +import voldemort.utils.JmxUtils; + +public class ClientJmxTest extends AbstractStoreClientFactoryTest { + + private static String STATS_DOMAIN = "voldemort.store.stats"; + private static String AGGREGATE_STATS_DOMAIN = "voldemort.store.stats.aggregate"; + private static String CLIENT_DOMAIN = "voldemort.client"; + private static String CLUSTER_FAILUREDETECTOR_DOMAIN = "voldemort.cluster.failuredetector"; + private static String CLIENT_REQUEST_DOMAIN = "voldemort.store.socket.clientrequest"; + + private AbstractSocketService socketService; + private MBeanServer mbServer = null; + + private static int factoryJmxId = 0; + + public ClientJmxTest() { + super(); + } + + private static String getAndIncrementJmxId() { + int current = factoryJmxId; + factoryJmxId++; + return (0 == current ? "" : "." + current); + } + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + socketService = ServerTestUtils.getSocketService(true, + getClusterXml(), + getStoreDefXml(), + getValidStoreName(), + getLocalNode().getSocketPort()); + socketService.start(); + mbServer = ManagementFactory.getPlatformMBeanServer(); + } + + @Override + @After + public void tearDown() throws Exception { + mbServer = null; + super.tearDown(); + socketService.stop(); + } + + @Override + protected StoreClientFactory getFactory(String... bootstrapUrls) { + return new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrls) + .setEnableLazy(false) + .setEnableJmx(true)); + } + + protected StoreClientFactory getFactoryWithClientContext(String clientContext, + String... bootstrapUrls) { + return new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrls) + .setEnableLazy(false) + .setClientContextName(clientContext) + .setEnableJmx(true)); + } + + @Test + public void testTwoClientContextOnJmx() throws Exception { + String clientContext1 = "clientA"; + String clientContext2 = "clientB"; + String jmxId1 = getAndIncrementJmxId(); + String jmxId2 = getAndIncrementJmxId(); + + StoreClient c1 = getFactoryWithClientContext(clientContext1, + getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + StoreClient c2 = getFactoryWithClientContext(clientContext2, + getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + + // checking for aggregate stats + ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext1 + + ".aggregate-perf" + + jmxId1); + ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext2 + + ".aggregate-perf" + + jmxId2); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + + // checking for per store stats + String c1type = clientContext1 + ".test" + jmxId1; + String c2type = clientContext2 + ".test" + jmxId2; + c1Name = JmxUtils.createObjectName(STATS_DOMAIN, + c1type + + "." + + ((DefaultStoreClient) c1).getClientId() + .toString()); + c2Name = JmxUtils.createObjectName(STATS_DOMAIN, + c2type + + "." + + ((DefaultStoreClient) c2).getClientId() + .toString()); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + } + + @Test + public void testSameContextOnJmx() throws Exception { + String clientContext = "clientContext"; + String jmxId1 = getAndIncrementJmxId(); + String jmxId2 = getAndIncrementJmxId(); + + StoreClient[] clients = new StoreClient[2]; + for(int i = 0; i < 2; i++) { + clients[i] = getFactoryWithClientContext(clientContext, getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + } + + // checking for aggregate stats + ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext + + ".aggregate-perf" + + jmxId1); + ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext + + ".aggregate-perf" + + jmxId2); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + + // checking for per store stats + String c1type = clientContext + ".test" + jmxId1; + String c2type = clientContext + ".test" + jmxId2; + c1Name = JmxUtils.createObjectName(STATS_DOMAIN, + c1type + + "." + + ((DefaultStoreClient) clients[0]).getClientId() + .toString()); + c2Name = JmxUtils.createObjectName(STATS_DOMAIN, + c2type + + "." + + ((DefaultStoreClient) clients[1]).getClientId() + .toString()); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + } + + @Test + public void testTwoClientNoContextOnJmx() throws Exception { + String clientContextCompare = "default"; + String jmxId1 = getAndIncrementJmxId(); + String jmxId2 = getAndIncrementJmxId(); + + StoreClient c1 = getFactory(getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + StoreClient c2 = getFactory(getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + + // checking for aggregate stats + ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare + + ".aggregate-perf" + + jmxId1); + ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare + + ".aggregate-perf" + + jmxId2); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + + // checking for per store stats + String c1type = clientContextCompare + ".test" + jmxId1; + String c2type = clientContextCompare + ".test" + jmxId2; + c1Name = JmxUtils.createObjectName(STATS_DOMAIN, + c1type + + "." + + ((DefaultStoreClient) c1).getClientId() + .toString()); + c2Name = JmxUtils.createObjectName(STATS_DOMAIN, + c2type + + "." + + ((DefaultStoreClient) c2).getClientId() + .toString()); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + } + + @Test + public void testTwoClientNullContextOnJmx() throws Exception { + String clientContextCompare = ""; + String jmxId1 = getAndIncrementJmxId(); + String jmxId2 = getAndIncrementJmxId(); + + StoreClient c1 = getFactoryWithClientContext(null, getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + StoreClient c2 = getFactoryWithClientContext(null, getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + + // checking for aggregate stats + ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare + + ".aggregate-perf" + + jmxId1); + ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare + + ".aggregate-perf" + + jmxId2); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + + // checking for per store stats + String c1type = clientContextCompare + ".test" + jmxId1; + String c2type = clientContextCompare + ".test" + jmxId2; + c1Name = JmxUtils.createObjectName(STATS_DOMAIN, + c1type + + "." + + ((DefaultStoreClient) c1).getClientId() + .toString()); + c2Name = JmxUtils.createObjectName(STATS_DOMAIN, + c2type + + "." + + ((DefaultStoreClient) c2).getClientId() + .toString()); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + } + + @Test + public void testSameContextAndFactory() throws Exception { + String clientContext = "clientContext"; + String jmxId = getAndIncrementJmxId(); + StoreClientFactory factory = getFactoryWithClientContext(clientContext, + getValidBootstrapUrl()); + + StoreClient[] clients = new StoreClient[2]; + for(int i = 0; i < 2; i++) { + clients[i] = factory.getStoreClient(getValidStoreName()); + } + + ObjectName cName = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext + + ".aggregate-perf" + + jmxId); + checkForMbeanFound(cName); + mbServer.unregisterMBean(cName); + + // checking for per store stats + String ctype = clientContext + ".test" + jmxId; + ObjectName c1Name = JmxUtils.createObjectName(STATS_DOMAIN, + ctype + + "." + + ((DefaultStoreClient) clients[0]).getClientId() + .toString()); + ObjectName c2Name = JmxUtils.createObjectName(STATS_DOMAIN, + ctype + + "." + + ((DefaultStoreClient) clients[1]).getClientId() + .toString()); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + } + + @Test + public void testDifferentId() throws Exception { + String clientContext = "clientContext"; + String jmxId = getAndIncrementJmxId(); + StoreClientFactory factory = getFactoryWithClientContext(clientContext, + getValidBootstrapUrl()); + + StoreClient[] clients = new StoreClient[2]; + clients[0] = factory.getStoreClient(getValidStoreName()); + clients[1] = factory.getStoreClient(getValidStoreName()); + + ObjectName cName = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext + + ".aggregate-perf" + + jmxId); + checkForMbeanFound(cName); + mbServer.unregisterMBean(cName); + + // checking for per store stats + String ctype = clientContext + ".test" + jmxId; + ObjectName c1Name = JmxUtils.createObjectName(STATS_DOMAIN, + ctype + + "." + + ((DefaultStoreClient) clients[0]).getClientId() + .toString()); + ObjectName c2Name = JmxUtils.createObjectName(STATS_DOMAIN, + ctype + + "." + + ((DefaultStoreClient) clients[1]).getClientId() + .toString()); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + assertTrue(!c1Name.equals(c2Name)); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + } + + private void checkForMbeanFound(ObjectName name) { + try { + mbServer.getMBeanInfo(name); + } catch(InstanceNotFoundException e) { + fail("MBean not found on the JMX Server: " + name.toString()); + } catch(Exception e) { + fail("Test failed: " + e.getMessage()); + } + } + + @Override + protected StoreClientFactory getFactoryWithSerializer(SerializerFactory factory, + String... bootstrapUrls) { + return new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrls) + .setEnableLazy(false) + .setSerializerFactory(factory)); + } + + @Override + protected String getValidBootstrapUrl() throws URISyntaxException { + return getLocalNode().getSocketUrl().toString(); + } + + @Override + protected String getValidScheme() { + return SocketStoreClientFactory.URL_SCHEME; + } +} From e6abad150ca0343a4718838d2071e04871661c87 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Tue, 22 May 2012 19:33:25 -0700 Subject: [PATCH 035/209] Changing the enable.nio.connector to true by default --- src/java/voldemort/server/VoldemortConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/voldemort/server/VoldemortConfig.java b/src/java/voldemort/server/VoldemortConfig.java index 13541ebd29..119c37040c 100644 --- a/src/java/voldemort/server/VoldemortConfig.java +++ b/src/java/voldemort/server/VoldemortConfig.java @@ -263,7 +263,7 @@ public VoldemortConfig(Props props) { this.socketBufferSize = (int) props.getBytes("socket.buffer.size", 64 * 1024); this.socketKeepAlive = props.getBoolean("socket.keepalive", false); - this.useNioConnector = props.getBoolean("enable.nio.connector", false); + this.useNioConnector = props.getBoolean("enable.nio.connector", true); this.nioConnectorSelectors = props.getInt("nio.connector.selectors", Math.max(8, Runtime.getRuntime() .availableProcessors())); From cccb8c3dab15bc9a0b3ca1cfab2fbe4551a71ad7 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Wed, 23 May 2012 18:55:56 -0700 Subject: [PATCH 036/209] To solve a issue that will return a null if a concurrent read is being done in the middle of a write of same entry. --- src/java/voldemort/store/bdb/BdbStorageEngine.java | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/java/voldemort/store/bdb/BdbStorageEngine.java b/src/java/voldemort/store/bdb/BdbStorageEngine.java index e07b438baf..61d809ee96 100644 --- a/src/java/voldemort/store/bdb/BdbStorageEngine.java +++ b/src/java/voldemort/store/bdb/BdbStorageEngine.java @@ -212,7 +212,15 @@ private List get(ByteArray key, Cursor cursor = null; try { cursor = getBdbDatabase().openCursor(null, null); - return get(cursor, key, lockMode, serializer); + List result = get(cursor, key, lockMode, serializer); + + // If null, try again in different locking mode to + // avoid null result due to gap between delete and new write + if(result.size() == 0 && lockMode != LockMode.DEFAULT) { + return get(cursor, key, LockMode.DEFAULT, serializer); + } else { + return result; + } } catch(DatabaseException e) { logger.error(e); throw new PersistenceFailureException(e); From 581d97a475e3015f8784901665d8a16a92390565 Mon Sep 17 00:00:00 2001 From: Guillaume Gardey Date: Fri, 25 May 2012 16:19:59 +0100 Subject: [PATCH 037/209] Handle disconnection On disconnection, raise an explicit VoldemortException. That may be easier for clients to handle recovery/reconnection rather than checking for the struct.error raised by unpack. --- clients/python/voldemort/client.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/clients/python/voldemort/client.py b/clients/python/voldemort/client.py index 830d38bbfb..d8a83e4c2a 100644 --- a/clients/python/voldemort/client.py +++ b/clients/python/voldemort/client.py @@ -239,6 +239,9 @@ def _send_request(self, connection, req_bytes): ## read a response from the connection def _receive_response(self, connection): size_bytes = connection.recv(4) + if not size_bytes: + raise VoldemortException('Connection closed') + size = struct.unpack('>i', size_bytes)[0] bytes_read = 0 @@ -252,6 +255,7 @@ def _receive_response(self, connection): return ''.join(data) + ## Bootstrap cluster metadata from a list of urls of nodes in the cluster. ## The urls are tuples in the form (host, port). ## A dictionary of node_id => node is returned. From b558e3b2487eaee017a35ca9bcd8bc42059c7047 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Wed, 30 May 2012 18:56:26 -0700 Subject: [PATCH 038/209] Adding System store functionality --- .../client/AbstractStoreClientFactory.java | 12 ++- .../client/CachingStoreClientFactory.java | 7 ++ .../voldemort/client/DefaultStoreClient.java | 10 ++- .../client/MockStoreClientFactory.java | 7 ++ .../client/SocketStoreClientFactory.java | 20 ++++- .../voldemort/client/StoreClientFactory.java | 14 +++ src/java/voldemort/client/SystemStore.java | 88 +++++++++++++++++++ .../server/SystemStoreConstants.java | 2 +- .../store/metadata/MetadataStore.java | 49 ++++++++++- .../voldemort/StaticStoreClientFactory.java | 7 ++ .../client/DefaultSocketStoreClientTest.java | 63 +++++++++++++ 11 files changed, 270 insertions(+), 9 deletions(-) create mode 100644 src/java/voldemort/client/SystemStore.java create mode 100644 test/unit/voldemort/client/DefaultSocketStoreClientTest.java diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index e5072be68b..18d016e9f5 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -158,6 +158,14 @@ public StoreClient getStoreClient(String storeName, public Store getRawStore(String storeName, InconsistencyResolver> resolver, UUID clientId) { + return getRawStore(storeName, resolver, clientId, null); + } + + @SuppressWarnings("unchecked") + public Store getRawStore(String storeName, + InconsistencyResolver> resolver, + UUID clientId, + String customStoresXml) { logger.info("Client zone-id [" + clientZoneId + "] Attempting to obtain metadata for store [" + storeName + "] "); @@ -169,7 +177,9 @@ public Store getRawStore(String storeName, // Get cluster and store metadata String clusterXml = bootstrapMetadataWithRetries(MetadataStore.CLUSTER_KEY, bootstrapUrls); Cluster cluster = clusterMapper.readCluster(new StringReader(clusterXml), false); - String storesXml = bootstrapMetadataWithRetries(MetadataStore.STORES_KEY, bootstrapUrls); + String storesXml = customStoresXml; + if(storesXml == null) + storesXml = bootstrapMetadataWithRetries(MetadataStore.STORES_KEY, bootstrapUrls); if(logger.isDebugEnabled()) { logger.debug("Obtained cluster metadata xml" + clusterXml); diff --git a/src/java/voldemort/client/CachingStoreClientFactory.java b/src/java/voldemort/client/CachingStoreClientFactory.java index 5fd2953da3..dc8f94ef8f 100644 --- a/src/java/voldemort/client/CachingStoreClientFactory.java +++ b/src/java/voldemort/client/CachingStoreClientFactory.java @@ -123,4 +123,11 @@ else if(client instanceof LazyStoreClient) { logger.warn("Exception during bootstrapAllClients", e); } } + + public Store getRawStore(String storeName, + InconsistencyResolver> resolver, + UUID clientId, + String storesXml) { + return inner.getRawStore(storeName, resolver, clientId, storesXml); + } } diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index 0f90c04a31..c056f5ea10 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -18,8 +18,8 @@ import java.util.List; import java.util.Map; -import java.util.UUID; import java.util.Map.Entry; +import java.util.UUID; import org.apache.log4j.Logger; @@ -65,6 +65,7 @@ public class DefaultStoreClient implements StoreClient { private final InconsistencyResolver> resolver; private volatile Store store; private final UUID clientId; + private SystemStore sysStore; public DefaultStoreClient(String storeName, InconsistencyResolver> resolver, @@ -103,6 +104,13 @@ public DefaultStoreClient(String storeName, public void bootStrap() { logger.info("Bootstrapping metadata for store " + this.storeName); this.store = storeFactory.getRawStore(storeName, resolver, clientId); + + logger.info("Creating System store"); + String systemKey = storeName + "-client"; + this.sysStore = new SystemStore("voldsys$_client_registry", + this.storeFactory); + sysStore.putSysStore(systemKey, "Registered"); + logger.info("Getting value - " + sysStore.getSysStore(systemKey)); } public boolean delete(K key) { diff --git a/src/java/voldemort/client/MockStoreClientFactory.java b/src/java/voldemort/client/MockStoreClientFactory.java index 32a2997842..0cf6c4a8f8 100644 --- a/src/java/voldemort/client/MockStoreClientFactory.java +++ b/src/java/voldemort/client/MockStoreClientFactory.java @@ -213,4 +213,11 @@ public FailureDetector getFailureDetector() { return failureDetector; } + public Store getRawStore(String storeName, + InconsistencyResolver> resolver, + UUID clientId, + String storesXml) { + return null; + } + } diff --git a/src/java/voldemort/client/SocketStoreClientFactory.java b/src/java/voldemort/client/SocketStoreClientFactory.java index e86ae31322..ef5de8b224 100644 --- a/src/java/voldemort/client/SocketStoreClientFactory.java +++ b/src/java/voldemort/client/SocketStoreClientFactory.java @@ -32,6 +32,7 @@ import voldemort.cluster.failuredetector.FailureDetectorConfig; import voldemort.cluster.failuredetector.FailureDetectorListener; import voldemort.server.RequestRoutingType; +import voldemort.server.SystemStoreConstants; import voldemort.store.Store; import voldemort.store.metadata.MetadataStore; import voldemort.store.socket.SocketDestination; @@ -57,6 +58,7 @@ public class SocketStoreClientFactory extends AbstractStoreClientFactory { private final SocketStoreFactory storeFactory; private FailureDetectorListener failureDetectorListener; private final RequestRoutingType requestRoutingType; + private final ClientConfig _config; public SocketStoreClientFactory(ClientConfig config) { super(config); @@ -71,6 +73,7 @@ public SocketStoreClientFactory(ClientConfig config) { config.getSocketKeepAlive()); if(config.isJmxEnabled()) JmxUtils.registerMbean(storeFactory, JmxUtils.createObjectName(storeFactory.getClass())); + _config = config; } @Override @@ -87,7 +90,8 @@ public StoreClient call() throws Exception { return getParentStoreClient(storeName, resolver); } - private StoreClient getParentStoreClient(String storeName, InconsistencyResolver> resolver) { + private StoreClient getParentStoreClient(String storeName, + InconsistencyResolver> resolver) { return super.getStoreClient(storeName, resolver); } @@ -96,7 +100,8 @@ protected List> getRemoteMetadata(String key, URI url) { try { return super.getRemoteMetadata(key, url); } catch(VoldemortException e) { - // Fix SNA-4227: When an error occurs during bootstrap, close the socket + // Fix SNA-4227: When an error occurs during bootstrap, close the + // socket SocketDestination destination = new SocketDestination(url.getHost(), url.getPort(), getRequestFormatType()); @@ -158,6 +163,14 @@ protected int getPort(Node node) { return node.getSocketPort(); } + public String[] getBootstrapURL() { + return _config.getBootstrapUrls(); + } + + public int getClientZoneID() { + return _config.getClientZoneId(); + } + @Override protected void validateUrl(URI url) { if(!URL_SCHEME.equals(url.getScheme())) @@ -177,4 +190,7 @@ public void close() { super.close(); } + public Store getSystemStore(String storeName) { + return getRawStore(storeName, null, null, SystemStoreConstants.SYSTEM_STORE_SCHEMA); + } } diff --git a/src/java/voldemort/client/StoreClientFactory.java b/src/java/voldemort/client/StoreClientFactory.java index f8fa8e710e..ee008d98af 100644 --- a/src/java/voldemort/client/StoreClientFactory.java +++ b/src/java/voldemort/client/StoreClientFactory.java @@ -86,6 +86,20 @@ Store getRawStore(String storeName, InconsistencyResolver> resolver, UUID clientId); + /** + * Get the underlying store, not the public StoreClient interface + * + * @param storeName The name of the store + * @param resolver The inconsistency resolver + * @param clientId The unique id of the client + * @param storesXml Custom set of stores containing storeName + * @return The appropriate store + */ + Store getRawStore(String storeName, + InconsistencyResolver> resolver, + UUID clientId, + String customStoresXml); + /** * Close the store client */ diff --git a/src/java/voldemort/client/SystemStore.java b/src/java/voldemort/client/SystemStore.java new file mode 100644 index 0000000000..f8e9b960f6 --- /dev/null +++ b/src/java/voldemort/client/SystemStore.java @@ -0,0 +1,88 @@ +package voldemort.client; + +import java.util.List; +import java.util.concurrent.TimeUnit; + +import org.apache.log4j.Logger; + +import voldemort.VoldemortException; +import voldemort.server.SystemStoreConstants; +import voldemort.store.Store; +import voldemort.versioning.InconsistentDataException; +import voldemort.versioning.VectorClock; +import voldemort.versioning.Versioned; + +public class SystemStore { + + private final Logger logger = Logger.getLogger(DefaultStoreClient.class); + private final SocketStoreClientFactory systemStoreFactory; + private final String storeName; + private volatile Store sysStore; + + SystemStore(String storeName, StoreClientFactory factory) { + String prefix = storeName.substring(0, SystemStoreConstants.NAME_PREFIX.length()); + if(!SystemStoreConstants.NAME_PREFIX.equals(prefix)) + throw new VoldemortException("Illegal system store : " + storeName); + if(!(factory instanceof SocketStoreClientFactory)) + throw new VoldemortException("System store cannot be created without a Socket store client factory"); + + SocketStoreClientFactory clientFactory = (SocketStoreClientFactory) factory; + ClientConfig config = new ClientConfig(); + config.setSelectors(1) + .setBootstrapUrls(clientFactory.getBootstrapURL()) + .setMaxConnectionsPerNode(2) + .setConnectionTimeout(1500, TimeUnit.MILLISECONDS) + .setSocketTimeout(5000, TimeUnit.MILLISECONDS) + .setRoutingTimeout(5000, TimeUnit.MILLISECONDS) + .setEnableJmx(false) + .setEnablePipelineRoutedStore(true) + .setClientZoneId(clientFactory.getClientZoneID()); + this.systemStoreFactory = new SocketStoreClientFactory(config); + this.storeName = storeName; + this.sysStore = this.systemStoreFactory.getSystemStore(this.storeName); + } + + public void putSysStore(K key, V value) throws VoldemortException { + logger.info("Invoking Put for key : " + key + " on store name : " + this.storeName); + Versioned versioned = getSysStore(key); + if(versioned == null) + versioned = Versioned.value(value, new VectorClock()); + else + versioned.setObject(value); + this.sysStore.put(key, versioned, null); + } + + public void putSysStore(K key, Versioned value) throws VoldemortException { + logger.info("Invoking Put for key : " + key + " on store name : " + this.storeName); + this.sysStore.put(key, value, null); + } + + public Versioned getSysStore(K key) throws VoldemortException { + logger.info("Invoking Get for key : " + key + " on store name : " + this.storeName); + Versioned versioned = null; + List> items = this.sysStore.get(key, null); + if(items.size() == 1) + versioned = items.get(0); + else if(items.size() > 1) + throw new InconsistentDataException("Unresolved versions returned from get(" + key + + ") = " + items, items); + if(versioned != null) + logger.info("Value for key : " + key + " = " + versioned.getValue() + + " on store name : " + this.storeName); + else + logger.info("Got null value"); + return versioned; + } + + public V getValueSysStore(K key) throws VoldemortException { + logger.info("Invoking Get for key : " + key + " on store name : " + this.storeName); + Versioned versioned = getSysStore(key); + if(versioned != null) { + logger.info("Value for key : " + key + " = " + versioned.getValue() + + " on store name : " + this.storeName); + return versioned.getValue(); + } + return null; + } + +} diff --git a/src/java/voldemort/server/SystemStoreConstants.java b/src/java/voldemort/server/SystemStoreConstants.java index 820037a920..cb048c66d1 100644 --- a/src/java/voldemort/server/SystemStoreConstants.java +++ b/src/java/voldemort/server/SystemStoreConstants.java @@ -5,7 +5,7 @@ */ public class SystemStoreConstants { - private static final String NAME_PREFIX = "voldsys$_"; + public static final String NAME_PREFIX = "voldsys$_"; public static enum SystemStoreName { voldsys$_client_registry, diff --git a/src/java/voldemort/store/metadata/MetadataStore.java b/src/java/voldemort/store/metadata/MetadataStore.java index 5a82acd99e..6d692ffe3f 100644 --- a/src/java/voldemort/store/metadata/MetadataStore.java +++ b/src/java/voldemort/store/metadata/MetadataStore.java @@ -41,6 +41,7 @@ import voldemort.routing.RouteToAllStrategy; import voldemort.routing.RoutingStrategy; import voldemort.routing.RoutingStrategyFactory; +import voldemort.server.SystemStoreConstants; import voldemort.server.rebalance.RebalancerState; import voldemort.store.StorageEngine; import voldemort.store.Store; @@ -73,6 +74,7 @@ public class MetadataStore implements StorageEngine { public static final String CLUSTER_KEY = "cluster.xml"; public static final String STORES_KEY = "stores.xml"; + public static final String SYSTEM_STORES_KEY = "systemStores"; public static final String SERVER_STATE_KEY = "server.state"; public static final String NODE_ID_KEY = "node.id"; public static final String REBALANCING_STEAL_INFO = "rebalancing.steal.info.key"; @@ -92,6 +94,7 @@ public class MetadataStore implements StorageEngine { // helper keys for metadataCacheOnly private static final String ROUTING_STRATEGY_KEY = "routing.strategy"; + private static final String SYSTEM_ROUTING_STRATEGY_KEY = "system.routing.strategy"; public static enum VoldemortState { NORMAL_SERVER, @@ -174,7 +177,8 @@ public synchronized void put(String key, Versioned value) { updateRoutingStrategies((Cluster) value.getValue(), getStoreDefList()); } else if(STORES_KEY.equals(key)) { updateRoutingStrategies(getCluster(), (List) value.getValue()); - } + } else if(SYSTEM_STORES_KEY.equals(key)) + throw new VoldemortException("Cannot overwrite system store definitions"); } else { throw new VoldemortException("Unhandled Key:" + key + " for MetadataStore put()"); @@ -191,8 +195,9 @@ public synchronized void put(String key, Versioned value) { public void put(String key, Object value) { if(METADATA_KEYS.contains(key)) { VectorClock version = (VectorClock) get(key, null).get(0).getVersion(); - put(key, new Versioned(value, version.incremented(getNodeId(), - System.currentTimeMillis()))); + put(key, + new Versioned(value, version.incremented(getNodeId(), + System.currentTimeMillis()))); } else { throw new VoldemortException("Unhandled Key:" + key + " for MetadataStore put()"); } @@ -297,6 +302,11 @@ public List getStoreDefList() { return (List) metadataCache.get(STORES_KEY).getValue(); } + @SuppressWarnings("unchecked") + public List getSystemStoreDefList() { + return (List) metadataCache.get(SYSTEM_STORES_KEY).getValue(); + } + public int getNodeId() { return (Integer) (metadataCache.get(NODE_ID_KEY).getValue()); } @@ -319,11 +329,21 @@ public RebalancerState getRebalancerState() { return (RebalancerState) metadataCache.get(REBALANCING_STEAL_INFO).getValue(); } + /* + * First check in the map of regular stores. If not present, check in the + * system stores map. + */ @SuppressWarnings("unchecked") public RoutingStrategy getRoutingStrategy(String storeName) { Map routingStrategyMap = (Map) metadataCache.get(ROUTING_STRATEGY_KEY) .getValue(); - return routingStrategyMap.get(storeName); + RoutingStrategy strategy = routingStrategyMap.get(storeName); + if(strategy == null) { + Map systemRoutingStrategyMap = (Map) metadataCache.get(SYSTEM_ROUTING_STRATEGY_KEY) + .getValue(); + strategy = systemRoutingStrategyMap.get(storeName); + } + return strategy; } /** @@ -362,6 +382,17 @@ private void updateRoutingStrategies(Cluster cluster, List stor } } + /* + * Initialize the routing strategy map for system stores. This is used + * during get / put on system stores. + */ + private void initSystemRoutingStrategies(Cluster cluster) { + HashMap routingStrategyMap = createRoutingStrategyMap(cluster, + getSystemStoreDefList()); + this.metadataCache.put(SYSTEM_ROUTING_STRATEGY_KEY, + new Versioned(routingStrategyMap)); + } + /** * Add the steal information to the rebalancer state * @@ -452,6 +483,10 @@ private void init(int nodeId) { initCache(CLUSTER_KEY); initCache(STORES_KEY); + // Initialize system store in the metadata cache + initSystemCache(); + initSystemRoutingStrategies(getCluster()); + initCache(NODE_ID_KEY, nodeId); if(getNodeId() != nodeId) throw new RuntimeException("Attempt to start previous node:" @@ -473,6 +508,12 @@ private synchronized void initCache(String key) { metadataCache.put(key, convertStringToObject(key, getInnerValue(key))); } + // Initialize the metadata cache with system store list + private synchronized void initSystemCache() { + List value = storeMapper.readStoreList(new StringReader(SystemStoreConstants.SYSTEM_STORE_SCHEMA)); + metadataCache.put(SYSTEM_STORES_KEY, new Versioned(value)); + } + private void initCache(String key, Object defaultValue) { try { initCache(key); diff --git a/test/common/voldemort/StaticStoreClientFactory.java b/test/common/voldemort/StaticStoreClientFactory.java index c1ec4c4513..913b75b188 100644 --- a/test/common/voldemort/StaticStoreClientFactory.java +++ b/test/common/voldemort/StaticStoreClientFactory.java @@ -74,4 +74,11 @@ public FailureDetector getFailureDetector() { return failureDetector; } + public Store getRawStore(String storeName, + InconsistencyResolver> resolver, + UUID clientId, + String storesXml) { + return null; + } + } diff --git a/test/unit/voldemort/client/DefaultSocketStoreClientTest.java b/test/unit/voldemort/client/DefaultSocketStoreClientTest.java new file mode 100644 index 0000000000..07b8db82f5 --- /dev/null +++ b/test/unit/voldemort/client/DefaultSocketStoreClientTest.java @@ -0,0 +1,63 @@ +package voldemort.client; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.fail; + +import org.junit.Before; +import org.junit.Test; + +import voldemort.utils.SystemTime; +import voldemort.utils.Time; +import voldemort.versioning.ObsoleteVersionException; +import voldemort.versioning.VectorClock; +import voldemort.versioning.Versioned; + +public class DefaultSocketStoreClientTest { + + protected StoreClient client; + protected int nodeId; + protected Time time; + + @Before + public void setUp() throws Exception { + String socketUrl = "tcp://localhost:6667"; + String storeName = "test"; + ClientConfig clientConfig = new ClientConfig().setMaxThreads(4) + .setMaxTotalConnections(4) + .setMaxConnectionsPerNode(4) + .setBootstrapUrls(socketUrl); + SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); + this.client = socketFactory.getStoreClient(storeName); + this.nodeId = 0; + this.time = SystemTime.INSTANCE; + } + + @Test + public void test() { + client.put("k", Versioned.value("v")); + Versioned v = client.get("k"); + assertEquals("GET should return the version set by PUT.", "v", v.getValue()); + VectorClock expected = new VectorClock(); + expected.incrementVersion(nodeId, time.getMilliseconds()); + assertEquals("The version should be incremented after a put.", expected, v.getVersion()); + try { + client.put("k", Versioned.value("v")); + fail("Put of obsolete version should throw exception."); + } catch(ObsoleteVersionException e) { + // this is good + } + // PUT of a concurrent version should succeed + client.put("k", + new Versioned("v2", + new VectorClock().incremented(nodeId + 1, + time.getMilliseconds()))); + assertEquals("GET should return the new value set by PUT.", "v2", client.getValue("k")); + assertEquals("GET should return the new version set by PUT.", + expected.incremented(nodeId + 1, time.getMilliseconds()), + client.get("k").getVersion()); + client.delete("k"); + assertNull("After a successful delete(k), get(k) should return null.", client.get("k")); + } + +} From 20778b0ee130c42ed03e25e1837f73666c331849 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Thu, 31 May 2012 18:03:02 -0700 Subject: [PATCH 039/209] Added 99th wait time for connection for ClientRequestExecutorPool.java --- .../clientrequest/ClientRequestExecutorPool.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java index ce914fde4f..2d83b0fe16 100644 --- a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java +++ b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java @@ -30,6 +30,7 @@ import voldemort.store.socket.SocketDestination; import voldemort.store.socket.SocketStore; import voldemort.store.socket.SocketStoreFactory; +import voldemort.store.stats.Histogram; import voldemort.utils.Time; import voldemort.utils.Utils; import voldemort.utils.pool.KeyedResourcePool; @@ -53,6 +54,7 @@ public class ClientRequestExecutorPool implements SocketStoreFactory { private final AtomicInteger checkouts; private final AtomicLong waitNs; private final AtomicLong avgWaitNs; + private final Histogram histogramWaitMs; private final KeyedResourcePool pool; private final ClientRequestExecutorFactory factory; @@ -76,6 +78,7 @@ public ClientRequestExecutorPool(int selectors, this.checkouts = new AtomicInteger(0); this.waitNs = new AtomicLong(0); this.avgWaitNs = new AtomicLong(0); + this.histogramWaitMs = new HistogramArray(10000, 1); } public ClientRequestExecutorPool(int maxConnectionsPerNode, @@ -130,13 +133,14 @@ private void updateStats(long checkoutTimeNs) { long wait = waitNs.getAndAdd(checkoutTimeNs); int count = checkouts.getAndIncrement(); - // reset reporting inverval if we have used up the current interval + // reset reporting interval if we have used up the current interval int interval = this.monitoringInterval.get(); if(count % interval == interval - 1) { // harmless race condition: waitNs.set(0); checkouts.set(0); avgWaitNs.set(wait / count); + histogramWaitMs.insert(checkoutTimeNs / Time.NS_PER_MS); } } @@ -193,6 +197,11 @@ public double getAvgWaitTimeMs() { return this.avgWaitNs.doubleValue() / Time.NS_PER_MS; } + @JmxGetter(name = "99thWaitTimeMs", description = "The 99th percentile ms of wait time to acquire a connection.") + public double get99thWaitTimeMs() { + return this.histogramWaitMs.getQuantile(0.99); + } + @JmxSetter(name = "monitoringInterval", description = "The number of checkouts over which performance statistics are calculated.") public void setMonitoringInterval(int count) { if(count <= 0) From 73e397a41f91491cb143a51e938d9440a83e958c Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Fri, 1 Jun 2012 14:18:49 -0700 Subject: [PATCH 040/209] Fixed bug and verified monitoring feature of q99th wait time --- .../socket/clientrequest/ClientRequestExecutorPool.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java index 2d83b0fe16..cebcb720f0 100644 --- a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java +++ b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java @@ -54,7 +54,7 @@ public class ClientRequestExecutorPool implements SocketStoreFactory { private final AtomicInteger checkouts; private final AtomicLong waitNs; private final AtomicLong avgWaitNs; - private final Histogram histogramWaitMs; + private final Histogram histogramWaitNs; private final KeyedResourcePool pool; private final ClientRequestExecutorFactory factory; @@ -78,7 +78,7 @@ public ClientRequestExecutorPool(int selectors, this.checkouts = new AtomicInteger(0); this.waitNs = new AtomicLong(0); this.avgWaitNs = new AtomicLong(0); - this.histogramWaitMs = new HistogramArray(10000, 1); + this.histogramWaitNs = new Histogram(10000, 1000); } public ClientRequestExecutorPool(int maxConnectionsPerNode, @@ -140,7 +140,7 @@ private void updateStats(long checkoutTimeNs) { waitNs.set(0); checkouts.set(0); avgWaitNs.set(wait / count); - histogramWaitMs.insert(checkoutTimeNs / Time.NS_PER_MS); + histogramWaitNs.insert(checkoutTimeNs); } } @@ -199,7 +199,7 @@ public double getAvgWaitTimeMs() { @JmxGetter(name = "99thWaitTimeMs", description = "The 99th percentile ms of wait time to acquire a connection.") public double get99thWaitTimeMs() { - return this.histogramWaitMs.getQuantile(0.99); + return (double) (this.histogramWaitNs.getQuantile(0.99)) / Time.NS_PER_MS; } @JmxSetter(name = "monitoringInterval", description = "The number of checkouts over which performance statistics are calculated.") From 1d2bca87e911c73f0754855ad9ebc93ec9bbfb2c Mon Sep 17 00:00:00 2001 From: Martin Kleppmann Date: Sat, 2 Jun 2012 01:06:47 -0700 Subject: [PATCH 041/209] Fix NPE in listing read-only store versions ReadOnlyUtils.getVersionDirs returns null if storeDir does not exist. I don't yet know how it could get into a state where it thinks the directory doesn't exist, but the NPE is obscuring the actual cause, so I'm fixing that first. --- src/java/voldemort/store/readonly/ReadOnlyStorageEngine.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/java/voldemort/store/readonly/ReadOnlyStorageEngine.java b/src/java/voldemort/store/readonly/ReadOnlyStorageEngine.java index 40f229c275..71faa3e40a 100644 --- a/src/java/voldemort/store/readonly/ReadOnlyStorageEngine.java +++ b/src/java/voldemort/store/readonly/ReadOnlyStorageEngine.java @@ -350,7 +350,7 @@ public void swapFiles(String newStoreDirectory) { */ private void deleteBackups() { File[] storeDirList = ReadOnlyUtils.getVersionDirs(storeDir, 0L, currentVersionId); - if(storeDirList.length > (numBackups + 1)) { + if(storeDirList != null && storeDirList.length > (numBackups + 1)) { // delete ALL old directories asynchronously File[] extraBackups = ReadOnlyUtils.findKthVersionedDir(storeDirList, 0, @@ -424,7 +424,7 @@ public void rollback(File rollbackToDir) { throw new VoldemortException("Cannot parse version id"); File[] backUpDirs = ReadOnlyUtils.getVersionDirs(storeDir, versionId, Long.MAX_VALUE); - if(backUpDirs.length <= 1) { + if(backUpDirs == null || backUpDirs.length <= 1) { logger.warn("No rollback performed since there are no back-up directories"); return; } From 552f1de98d52312252ba930b1e0fd6612e45d968 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Tue, 5 Jun 2012 15:59:35 -0700 Subject: [PATCH 042/209] Added the Voldemort Client automated re-bootstrap mechanism --- src/java/voldemort/VoldemortAdminTool.java | 31 ++++ .../client/AbstractStoreClientFactory.java | 3 +- .../client/AsyncMetadataVersionManager.java | 125 +++++++++++++++ src/java/voldemort/client/ClientConfig.java | 14 ++ .../voldemort/client/DefaultStoreClient.java | 64 ++++++-- src/java/voldemort/client/SystemStore.java | 27 ++-- .../routing/RouteToAllLocalPrefStrategy.java | 17 ++ .../routing/RoutingStrategyFactory.java | 2 + .../routing/RoutingStrategyType.java | 1 + .../server/SystemStoreConstants.java | 21 ++- .../store/routed/PipelineRoutedStore.java | 87 ++++++++++- .../routed/action/ConfigureNodesByZone.java | 147 ++++++++++++++++++ .../routed/action/ConfigureNodesDefault.java | 71 +++++++++ .../action/ConfigureNodesLocalHost.java | 71 +++++++++ .../action/ConfigureNodesLocalHostByZone.java | 72 +++++++++ 15 files changed, 718 insertions(+), 35 deletions(-) create mode 100644 src/java/voldemort/client/AsyncMetadataVersionManager.java create mode 100644 src/java/voldemort/routing/RouteToAllLocalPrefStrategy.java create mode 100644 src/java/voldemort/store/routed/action/ConfigureNodesByZone.java create mode 100644 src/java/voldemort/store/routed/action/ConfigureNodesDefault.java create mode 100644 src/java/voldemort/store/routed/action/ConfigureNodesLocalHost.java create mode 100644 src/java/voldemort/store/routed/action/ConfigureNodesLocalHostByZone.java diff --git a/src/java/voldemort/VoldemortAdminTool.java b/src/java/voldemort/VoldemortAdminTool.java index bc22f8ab56..5629691e97 100644 --- a/src/java/voldemort/VoldemortAdminTool.java +++ b/src/java/voldemort/VoldemortAdminTool.java @@ -45,6 +45,7 @@ import org.codehaus.jackson.JsonGenerator; import org.codehaus.jackson.map.ObjectMapper; +import voldemort.client.SystemStore; import voldemort.client.protocol.admin.AdminClient; import voldemort.client.protocol.admin.AdminClientConfig; import voldemort.cluster.Cluster; @@ -85,6 +86,7 @@ public class VoldemortAdminTool { private static final String ALL_METADATA = "all"; + private static SystemStore sysStoreVersion = null; @SuppressWarnings("unchecked") public static void main(String[] args) throws Exception { @@ -250,6 +252,13 @@ public static void main(String[] args) throws Exception { AdminClient adminClient = new AdminClient(url, new AdminClientConfig()); + // Initialize the system store for stores.xml version + String[] bootstrapUrls = new String[1]; + bootstrapUrls[0] = url; + sysStoreVersion = new SystemStore("voldsys$_metadata_version", + bootstrapUrls, + 0); + String ops = ""; if(options.has("delete-partitions")) { ops += "d"; @@ -433,6 +442,10 @@ public static void main(String[] args) throws Exception { adminClient, MetadataStore.STORES_KEY, mapper.writeStoreList(storeDefs)); + + // Update the store metadata version + updateStoreMetadataversion(); + } else if(metadataKey.compareTo(MetadataStore.REBALANCING_STEAL_INFO) == 0) { if(!Utils.isReadableFile(metadataValue)) throw new VoldemortException("Rebalancing steal info file path incorrect"); @@ -722,6 +735,24 @@ private static void executeCheckMetadata(AdminClient adminClient, String metadat } } + /* + * TODO: For now write one version for the entire stores.xml When we split + * the stores.xml, make this more granular + */ + private static void updateStoreMetadataversion() { + String versionKey = "stores.xml"; + Versioned storesVersion = sysStoreVersion.getSysStore(versionKey); + if(storesVersion == null) { + System.err.println("Current version is null. Assuming version 0."); + storesVersion = new Versioned((long) 1); + } else { + System.out.println("Version obtained = " + storesVersion.getValue()); + long newValue = storesVersion.getValue() + 1; + storesVersion.setObject(newValue); + } + sysStoreVersion.putSysStore(versionKey, storesVersion); + } + private static void executeSetMetadata(Integer nodeId, AdminClient adminClient, String key, diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index 18d016e9f5..f084352af4 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -151,7 +151,8 @@ public StoreClient getStoreClient(String storeName, this, 3, clientContextName, - sequencer.getAndIncrement()); + sequencer.getAndIncrement(), + config); } @SuppressWarnings("unchecked") diff --git a/src/java/voldemort/client/AsyncMetadataVersionManager.java b/src/java/voldemort/client/AsyncMetadataVersionManager.java new file mode 100644 index 0000000000..a0ce826e8b --- /dev/null +++ b/src/java/voldemort/client/AsyncMetadataVersionManager.java @@ -0,0 +1,125 @@ +package voldemort.client; + +import java.lang.Thread.UncaughtExceptionHandler; +import java.util.Random; +import java.util.concurrent.Callable; + +import org.apache.log4j.Level; +import org.apache.log4j.Logger; + +import voldemort.versioning.Versioned; + +/* + * The AsyncMetadataVersionManager is used to track the Metadata version on the + * cluster and if necessary Re-bootstrap the client. + * + * During initialization, it will retrieve the current version of the store (or + * the entire stores.xml depending upon granularity) and then periodically check + * whether this has been updated. During init if the initial version turns out + * to be null, it means that no change has been done to that store since it was + * created. In this case, we assume version '0'. + */ + +public class AsyncMetadataVersionManager implements Runnable { + + private final Logger logger = Logger.getLogger(this.getClass()); + private Versioned currentVersion; + private final SystemStore sysStore; + private final String systemKey = "stores.xml"; + private volatile boolean isRunning; + private final Callable storeClientThunk; + private long asyncMetadataCheckInterval; + + // Random delta generator + final int DELTA_MAX = 1000; + Random randomGenerator = new Random(System.currentTimeMillis()); + + public AsyncMetadataVersionManager(SystemStore systemStore, + long asyncMetadataCheckInterval, + Callable storeClientThunk) { + this(null, systemStore, asyncMetadataCheckInterval, storeClientThunk); + } + + public AsyncMetadataVersionManager(Versioned initialVersion, + SystemStore systemStore, + long asyncMetadataCheckInterval, + Callable storeClientThunk) { + this.sysStore = systemStore; + if(initialVersion == null) { + this.currentVersion = sysStore.getSysStore("stores.xml"); + + // If the received store version is null, assume version 0 + if(currentVersion == null) + currentVersion = new Versioned((long) 0); + } else { + currentVersion = initialVersion; + } + + // Initialize and start the background check thread + isRunning = true; + + Thread checkVersionThread = new Thread(this, "AsyncVersionCheckThread"); + checkVersionThread.setDaemon(true); + checkVersionThread.setUncaughtExceptionHandler(new UncaughtExceptionHandler() { + + public void uncaughtException(Thread t, Throwable e) { + if(logger.isEnabledFor(Level.ERROR)) + logger.error("Uncaught exception in Metadata Version check thread:", e); + } + }); + + this.storeClientThunk = storeClientThunk; + this.asyncMetadataCheckInterval = asyncMetadataCheckInterval; + checkVersionThread.start(); + + } + + public void destroy() { + isRunning = false; + } + + public void run() { + while(!Thread.currentThread().isInterrupted() && isRunning) { + try { + Thread.sleep(asyncMetadataCheckInterval); + } catch(InterruptedException e) { + break; + } + + Versioned newVersion = this.sysStore.getSysStore(systemKey); + + // If version obtained is null, the store is untouched. Continue + if(newVersion == null) { + logger.info("Metadata unchanged after creation ..."); + continue; + } + + logger.info("MetadataVersion check => Obtained " + systemKey + " version : " + + newVersion); + + if(!newVersion.equals(currentVersion)) { + logger.info("Metadata version mismatch detected."); + + // Determine a random delta delay between 0 to 1000 (ms) + int delta = randomGenerator.nextInt(DELTA_MAX); + + try { + logger.info("Sleeping for delta : " + delta + " (ms) before re-bootstrapping."); + Thread.sleep(delta); + } catch(InterruptedException e) { + break; + } + + // Invoke callback for bootstrap + try { + this.storeClientThunk.call(); + } catch(Exception e) { + e.printStackTrace(); + } + + // Update the current version + currentVersion = newVersion; + } + } + } +} diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index 30e4ad9dad..8cfa432539 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -77,6 +77,7 @@ public class ClientConfig { private volatile int maxBootstrapRetries = 2; private volatile String clientContextName = "default"; + private volatile long asyncCheckMetadataInterval = 5000; public ClientConfig() {} @@ -112,6 +113,7 @@ public ClientConfig() {} public static final String FAILUREDETECTOR_REQUEST_LENGTH_THRESHOLD_PROPERTY = "failuredetector_request_length_threshold"; public static final String MAX_BOOTSTRAP_RETRIES = "max_bootstrap_retries"; public static final String CLIENT_CONTEXT_NAME = "voldemort_client_context"; + public static final String ASYNC_CHECK_METADATA_INTERVAL = "check_metadata_interval"; /** * Instantiate the client config using a properties file @@ -244,6 +246,10 @@ private void setProperties(Properties properties) { if(props.containsKey(CLIENT_CONTEXT_NAME)) { this.setClientContextName(props.getString(CLIENT_CONTEXT_NAME, null)); } + + if(props.containsKey(ASYNC_CHECK_METADATA_INTERVAL)) { + this.setAsyncCheckMetadataInterval(props.getLong(ASYNC_CHECK_METADATA_INTERVAL, 5000)); + } } public int getMaxConnectionsPerNode() { @@ -649,4 +655,12 @@ public ClientConfig setClientContextName(String clientContextName) { return this; } + public long getAsyncCheckMetadataInterval() { + return asyncCheckMetadataInterval; + } + + public void setAsyncCheckMetadataInterval(long asyncCheckMetadataInterval) { + this.asyncCheckMetadataInterval = asyncCheckMetadataInterval; + } + } diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index c056f5ea10..436fcfd292 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -16,10 +16,12 @@ package voldemort.client; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.UUID; +import java.util.concurrent.Callable; import org.apache.log4j.Logger; @@ -60,26 +62,37 @@ public class DefaultStoreClient implements StoreClient { private final Logger logger = Logger.getLogger(DefaultStoreClient.class); private final StoreClientFactory storeFactory; + private final ClientConfig config; private final int metadataRefreshAttempts; private final String storeName; private final InconsistencyResolver> resolver; private volatile Store store; private final UUID clientId; - private SystemStore sysStore; + private final Map sysStoreMap; + private AsyncMetadataVersionManager asyncCheckMetadata; + + // Enumerate all the system stores + private final String METADATA_VERSION_STORE = "voldsys$_metadata_version"; + private final String CLIENT_REGISTRY_STORE = "voldsys$_client_registry"; + private final String STORE_DEFINITION_STORE = "voldsys$_client_store_definition"; + private final String[] systemStoreNames = { METADATA_VERSION_STORE, CLIENT_REGISTRY_STORE, + STORE_DEFINITION_STORE }; public DefaultStoreClient(String storeName, InconsistencyResolver> resolver, StoreClientFactory storeFactory, int maxMetadataRefreshAttempts) { - this(storeName, resolver, storeFactory, maxMetadataRefreshAttempts, null, 0); + this(storeName, resolver, storeFactory, maxMetadataRefreshAttempts, null, 0, null); } + @SuppressWarnings("unchecked") public DefaultStoreClient(String storeName, InconsistencyResolver> resolver, StoreClientFactory storeFactory, int maxMetadataRefreshAttempts, String clientContext, - int clientSequence) { + int clientSequence, + ClientConfig config) { this.storeName = Utils.notNull(storeName); this.resolver = resolver; @@ -88,6 +101,8 @@ public DefaultStoreClient(String storeName, this.clientId = AbstractStoreClientFactory.generateClientId(storeName, clientContext, clientSequence); + this.config = config; + // Registering self to be able to bootstrap client dynamically via JMX JmxUtils.registerMbean(this, JmxUtils.createObjectName(JmxUtils.getPackageName(this.getClass()), @@ -96,21 +111,50 @@ public DefaultStoreClient(String storeName, + storeName + "." + clientId.toString())); bootStrap(); + + // Initialize all the system stores + sysStoreMap = new HashMap(); + initializeSystemStores(); + + // Initialize the background thread for checking metadata version + if(config != null) { + SystemStore versionStore = this.sysStoreMap.get(METADATA_VERSION_STORE); + if(versionStore == null) + logger.info("Metadata version system store not found. Cannot run Metadata version check thread."); + else { + Callable bootstrapCallback = new Callable() { + + public Void call() throws Exception { + bootStrap(); + return null; + } + }; + + asyncCheckMetadata = new AsyncMetadataVersionManager(versionStore, + config.getAsyncCheckMetadataInterval(), + bootstrapCallback); + logger.info("Metadata version check thread started. Frequency = Every " + + config.getAsyncCheckMetadataInterval() + " ms"); + } + } + logger.info("Voldemort client created: clientContext=" + clientContext + " clientSequence=" + clientSequence + " clientId=" + clientId.toString()); } + public void initializeSystemStores() { + for(String storeName: systemStoreNames) { + SystemStore sysStore = new SystemStore(storeName, + config.getBootstrapUrls(), + config.getClientZoneId()); + this.sysStoreMap.put(storeName, sysStore); + } + } + @JmxOperation(description = "bootstrap metadata from the cluster.") public void bootStrap() { logger.info("Bootstrapping metadata for store " + this.storeName); this.store = storeFactory.getRawStore(storeName, resolver, clientId); - - logger.info("Creating System store"); - String systemKey = storeName + "-client"; - this.sysStore = new SystemStore("voldsys$_client_registry", - this.storeFactory); - sysStore.putSysStore(systemKey, "Registered"); - logger.info("Getting value - " + sysStore.getSysStore(systemKey)); } public boolean delete(K key) { diff --git a/src/java/voldemort/client/SystemStore.java b/src/java/voldemort/client/SystemStore.java index f8e9b960f6..cf1289ebd1 100644 --- a/src/java/voldemort/client/SystemStore.java +++ b/src/java/voldemort/client/SystemStore.java @@ -19,31 +19,28 @@ public class SystemStore { private final String storeName; private volatile Store sysStore; - SystemStore(String storeName, StoreClientFactory factory) { + public SystemStore(String storeName, String[] bootstrapUrls, int clientZoneID) { String prefix = storeName.substring(0, SystemStoreConstants.NAME_PREFIX.length()); if(!SystemStoreConstants.NAME_PREFIX.equals(prefix)) throw new VoldemortException("Illegal system store : " + storeName); - if(!(factory instanceof SocketStoreClientFactory)) - throw new VoldemortException("System store cannot be created without a Socket store client factory"); - SocketStoreClientFactory clientFactory = (SocketStoreClientFactory) factory; ClientConfig config = new ClientConfig(); config.setSelectors(1) - .setBootstrapUrls(clientFactory.getBootstrapURL()) + .setBootstrapUrls(bootstrapUrls) .setMaxConnectionsPerNode(2) .setConnectionTimeout(1500, TimeUnit.MILLISECONDS) .setSocketTimeout(5000, TimeUnit.MILLISECONDS) .setRoutingTimeout(5000, TimeUnit.MILLISECONDS) .setEnableJmx(false) .setEnablePipelineRoutedStore(true) - .setClientZoneId(clientFactory.getClientZoneID()); + .setClientZoneId(clientZoneID); this.systemStoreFactory = new SocketStoreClientFactory(config); this.storeName = storeName; this.sysStore = this.systemStoreFactory.getSystemStore(this.storeName); } public void putSysStore(K key, V value) throws VoldemortException { - logger.info("Invoking Put for key : " + key + " on store name : " + this.storeName); + logger.debug("Invoking Put for key : " + key + " on store name : " + this.storeName); Versioned versioned = getSysStore(key); if(versioned == null) versioned = Versioned.value(value, new VectorClock()); @@ -53,12 +50,12 @@ public void putSysStore(K key, V value) throws VoldemortException { } public void putSysStore(K key, Versioned value) throws VoldemortException { - logger.info("Invoking Put for key : " + key + " on store name : " + this.storeName); + logger.debug("Invoking Put for key : " + key + " on store name : " + this.storeName); this.sysStore.put(key, value, null); } public Versioned getSysStore(K key) throws VoldemortException { - logger.info("Invoking Get for key : " + key + " on store name : " + this.storeName); + logger.debug("Invoking Get for key : " + key + " on store name : " + this.storeName); Versioned versioned = null; List> items = this.sysStore.get(key, null); if(items.size() == 1) @@ -67,19 +64,19 @@ else if(items.size() > 1) throw new InconsistentDataException("Unresolved versions returned from get(" + key + ") = " + items, items); if(versioned != null) - logger.info("Value for key : " + key + " = " + versioned.getValue() - + " on store name : " + this.storeName); + logger.debug("Value for key : " + key + " = " + versioned.getValue() + + " on store name : " + this.storeName); else - logger.info("Got null value"); + logger.debug("Got null value"); return versioned; } public V getValueSysStore(K key) throws VoldemortException { - logger.info("Invoking Get for key : " + key + " on store name : " + this.storeName); + logger.debug("Invoking Get for key : " + key + " on store name : " + this.storeName); Versioned versioned = getSysStore(key); if(versioned != null) { - logger.info("Value for key : " + key + " = " + versioned.getValue() - + " on store name : " + this.storeName); + logger.debug("Value for key : " + key + " = " + versioned.getValue() + + " on store name : " + this.storeName); return versioned.getValue(); } return null; diff --git a/src/java/voldemort/routing/RouteToAllLocalPrefStrategy.java b/src/java/voldemort/routing/RouteToAllLocalPrefStrategy.java new file mode 100644 index 0000000000..240aff1d49 --- /dev/null +++ b/src/java/voldemort/routing/RouteToAllLocalPrefStrategy.java @@ -0,0 +1,17 @@ +package voldemort.routing; + +import java.util.Collection; + +import voldemort.cluster.Node; + +public class RouteToAllLocalPrefStrategy extends RouteToAllStrategy { + + public RouteToAllLocalPrefStrategy(Collection nodes) { + super(nodes); + } + + @Override + public String getType() { + return RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY; + } +} diff --git a/src/java/voldemort/routing/RoutingStrategyFactory.java b/src/java/voldemort/routing/RoutingStrategyFactory.java index c8b8fe0158..b4901ba40c 100644 --- a/src/java/voldemort/routing/RoutingStrategyFactory.java +++ b/src/java/voldemort/routing/RoutingStrategyFactory.java @@ -24,6 +24,8 @@ public RoutingStrategy updateRoutingStrategy(StoreDefinition storeDef, Cluster c return new ZoneRoutingStrategy(cluster.getNodes(), storeDef.getZoneReplicationFactor(), storeDef.getReplicationFactor()); + } else if(RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY.equals(storeDef.getRoutingStrategyType())) { + return new RouteToAllLocalPrefStrategy(cluster.getNodes()); } else { throw new VoldemortException("RoutingStrategyType:" + storeDef.getRoutingStrategyType() + " not handled by " + this.getClass()); diff --git a/src/java/voldemort/routing/RoutingStrategyType.java b/src/java/voldemort/routing/RoutingStrategyType.java index 92ead1b1ae..60c1a49733 100644 --- a/src/java/voldemort/routing/RoutingStrategyType.java +++ b/src/java/voldemort/routing/RoutingStrategyType.java @@ -10,6 +10,7 @@ public class RoutingStrategyType { public final static String CONSISTENT_STRATEGY = "consistent-routing"; public final static String TO_ALL_STRATEGY = "all-routing"; public final static String ZONE_STRATEGY = "zone-routing"; + public final static String TO_ALL_LOCAL_PREF_STRATEGY = "local-pref-all-routing"; private final String name; diff --git a/src/java/voldemort/server/SystemStoreConstants.java b/src/java/voldemort/server/SystemStoreConstants.java index cb048c66d1..a7c546681c 100644 --- a/src/java/voldemort/server/SystemStoreConstants.java +++ b/src/java/voldemort/server/SystemStoreConstants.java @@ -34,6 +34,7 @@ public static enum SystemStoreName { + " " + " 7" + " " + + " " + " voldsys$_client_store_definition" + " zone-routing" @@ -50,7 +51,25 @@ public static enum SystemStoreName { + " string" + " " + " 7" - + " " + ""; + + " " + + + " " + + " voldsys$_metadata_version" + + " local-pref-all-routing" + + " proximity-handoff" + + " memory" + + " client" + + " 1" + + " 1" + + " 1" + + " " + + " string" + + " " + + " " + + " java-serialization" + + " " + " " + + + ""; public static boolean isSystemStore(String storeName) { return (null == storeName ? false : storeName.startsWith(NAME_PREFIX)); diff --git a/src/java/voldemort/store/routed/PipelineRoutedStore.java b/src/java/voldemort/store/routed/PipelineRoutedStore.java index c5aa1ba06b..e14f0aae53 100644 --- a/src/java/voldemort/store/routed/PipelineRoutedStore.java +++ b/src/java/voldemort/store/routed/PipelineRoutedStore.java @@ -35,7 +35,12 @@ import voldemort.store.nonblockingstore.NonblockingStore; import voldemort.store.routed.Pipeline.Event; import voldemort.store.routed.Pipeline.Operation; +import voldemort.store.routed.action.AbstractConfigureNodes; import voldemort.store.routed.action.ConfigureNodes; +import voldemort.store.routed.action.ConfigureNodesByZone; +import voldemort.store.routed.action.ConfigureNodesDefault; +import voldemort.store.routed.action.ConfigureNodesLocalHost; +import voldemort.store.routed.action.ConfigureNodesLocalHostByZone; import voldemort.store.routed.action.GetAllConfigureNodes; import voldemort.store.routed.action.GetAllReadRepair; import voldemort.store.routed.action.IncrementClock; @@ -79,6 +84,13 @@ public class PipelineRoutedStore extends RoutedStore { private PipelineRoutedStats stats; private final int jmxId; + private enum ConfigureNodesType { + DEFAULT, + BYZONE, + DEFAULT_LOCAL, + BYZONE_LOCAL + } + /** * Create a PipelineRoutedStore * @@ -139,6 +151,66 @@ public PipelineRoutedStore(String name, } } + private ConfigureNodesType getNodeConfigurationType(BasicPipelineData>> pipelineData) { + // If Zone and local preference required + if(pipelineData.getZonesRequired() != null + && routingStrategy.getType().equals(RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY)) + return ConfigureNodesType.BYZONE_LOCAL; + + // If only local preference required + else if(pipelineData.getZonesRequired() == null + && routingStrategy.getType().equals(RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY)) + return ConfigureNodesType.DEFAULT_LOCAL; + + // If only Zone required + else if(pipelineData.getZonesRequired() != null + && !routingStrategy.getType() + .equals(RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY)) + return ConfigureNodesType.BYZONE; + + // Default case + return ConfigureNodesType.DEFAULT; + } + + private AbstractConfigureNodes>, BasicPipelineData>>> getNodeConfiguration(BasicPipelineData>> pipelineData, + ByteArray key) { + switch(getNodeConfigurationType(pipelineData)) { + case DEFAULT: + return new ConfigureNodesDefault>, BasicPipelineData>>>(pipelineData, + Event.CONFIGURED, + failureDetector, + storeDef.getRequiredReads(), + routingStrategy, + key); + case BYZONE: + return new ConfigureNodesByZone>, BasicPipelineData>>>(pipelineData, + Event.CONFIGURED, + failureDetector, + storeDef.getRequiredReads(), + routingStrategy, + key, + clientZone); + case DEFAULT_LOCAL: + return new ConfigureNodesLocalHost>, BasicPipelineData>>>(pipelineData, + Event.CONFIGURED, + failureDetector, + storeDef.getRequiredReads(), + routingStrategy, + key); + case BYZONE_LOCAL: + return new ConfigureNodesLocalHostByZone>, BasicPipelineData>>>(pipelineData, + Event.CONFIGURED, + failureDetector, + storeDef.getRequiredReads(), + routingStrategy, + key, + clientZone); + default: + return null; + } + + } + public List> get(final ByteArray key, final byte[] transforms) { StoreUtils.assertValidKey(key); @@ -160,14 +232,13 @@ public List> request(Store store) { }; - pipeline.addEventAction(Event.STARTED, - new ConfigureNodes>, BasicPipelineData>>>(pipelineData, - Event.CONFIGURED, - failureDetector, - storeDef.getRequiredReads(), - routingStrategy, - key, - clientZone)); + // Get the correct type of configure nodes action depending on the store + // requirements + AbstractConfigureNodes>, BasicPipelineData>>> configureNodes = getNodeConfiguration(pipelineData, + key); + + pipeline.addEventAction(Event.STARTED, configureNodes); + pipeline.addEventAction(Event.CONFIGURED, new PerformParallelRequests>, BasicPipelineData>>>(pipelineData, allowReadRepair ? Event.RESPONSES_RECEIVED diff --git a/src/java/voldemort/store/routed/action/ConfigureNodesByZone.java b/src/java/voldemort/store/routed/action/ConfigureNodesByZone.java new file mode 100644 index 0000000000..7a194f649b --- /dev/null +++ b/src/java/voldemort/store/routed/action/ConfigureNodesByZone.java @@ -0,0 +1,147 @@ +/* + * Copyright 2010 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package voldemort.store.routed.action; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +import voldemort.VoldemortException; +import voldemort.cluster.Node; +import voldemort.cluster.Zone; +import voldemort.cluster.failuredetector.FailureDetector; +import voldemort.routing.RoutingStrategy; +import voldemort.store.routed.BasicPipelineData; +import voldemort.store.routed.Pipeline; +import voldemort.store.routed.Pipeline.Event; +import voldemort.store.routed.Pipeline.Operation; +import voldemort.utils.ByteArray; +import voldemort.utils.ByteUtils; + +/* + * Configure the Nodes obtained via the routing strategy based on the zone + * information. Local zone nodes first, followed by the corresponding nodes from + * each of the other zones, ordered by proximity. + */ +public class ConfigureNodesByZone> extends + AbstractConfigureNodes { + + private final ByteArray key; + + private final Zone clientZone; + + public ConfigureNodesByZone(PD pipelineData, + Event completeEvent, + FailureDetector failureDetector, + int required, + RoutingStrategy routingStrategy, + ByteArray key, + Zone clientZone) { + super(pipelineData, completeEvent, failureDetector, required, routingStrategy); + this.key = key; + this.clientZone = clientZone; + } + + public List getNodes(ByteArray key, Operation op) { + List nodes = null; + try { + nodes = super.getNodes(key); + } catch(VoldemortException e) { + pipelineData.setFatalError(e); + return null; + } + + if(logger.isDebugEnabled()) + logger.debug("Adding " + nodes.size() + " node(s) to preference list"); + + if(pipelineData.getZonesRequired() > this.clientZone.getProximityList().size()) { + throw new VoldemortException("Number of zones required should be less than the total number of zones"); + } + + if(pipelineData.getZonesRequired() > required) { + throw new VoldemortException("Number of zones required should be less than the required number of " + + op.getSimpleName() + "s"); + } + + // Create zone id to node mapping + Map> zoneIdToNode = new HashMap>(); + for(Node node: nodes) { + List nodesList = null; + if(zoneIdToNode.containsKey(node.getZoneId())) { + nodesList = zoneIdToNode.get(node.getZoneId()); + } else { + nodesList = new ArrayList(); + zoneIdToNode.put(node.getZoneId(), nodesList); + } + nodesList.add(node); + } + + nodes = new ArrayList(); + LinkedList zoneProximityList = this.clientZone.getProximityList(); + if(op != Operation.PUT) { + // GET, GET_VERSIONS, DELETE + + // Add a node from every zone, upto a max of + // zoneCountReads/zoneCountWrites. + for(int index = 0; index < pipelineData.getZonesRequired(); index++) { + List zoneNodes = zoneIdToNode.get(zoneProximityList.get(index)); + if(zoneNodes != null && zoneNodes.size() > 0) { + nodes.add(zoneNodes.remove(0)); + } + } + + } + + // Add the rest, starting with client zone... + List clientZoneNodes = zoneIdToNode.get(clientZone.getId()); + if(clientZoneNodes != null && clientZoneNodes.size() > 0) + nodes.addAll(clientZoneNodes); + // ...followed by other zones sorted by proximity list + for(int index = 0; index < zoneProximityList.size(); index++) { + List zoneNodes = zoneIdToNode.get(zoneProximityList.get(index)); + if(zoneNodes != null && zoneNodes.size() > 0) { + nodes.addAll(zoneNodes); + } + } + + return nodes; + } + + public void execute(Pipeline pipeline) { + List nodes = null; + + nodes = getNodes(key, pipeline.getOperation()); + if(nodes == null) { + pipeline.abort(); + return; + } + + if(logger.isDebugEnabled()) { + StringBuilder nodeStr = new StringBuilder(); + for(Node node: nodes) { + nodeStr.append(node.getId() + ","); + } + logger.debug("Key " + ByteUtils.toHexString(key.get()) + + " final preference list to contact " + nodeStr); + } + pipelineData.setNodes(nodes); + pipeline.addEvent(completeEvent); + } + +} diff --git a/src/java/voldemort/store/routed/action/ConfigureNodesDefault.java b/src/java/voldemort/store/routed/action/ConfigureNodesDefault.java new file mode 100644 index 0000000000..b05379d74f --- /dev/null +++ b/src/java/voldemort/store/routed/action/ConfigureNodesDefault.java @@ -0,0 +1,71 @@ +package voldemort.store.routed.action; + +import java.util.List; + +import voldemort.VoldemortException; +import voldemort.cluster.Node; +import voldemort.cluster.failuredetector.FailureDetector; +import voldemort.routing.RoutingStrategy; +import voldemort.store.routed.BasicPipelineData; +import voldemort.store.routed.Pipeline; +import voldemort.store.routed.Pipeline.Event; +import voldemort.utils.ByteArray; +import voldemort.utils.ByteUtils; + +/* + * Default Configure Nodes that does not reorder the list of nodes obtained via + * the routing strategy + */ +public class ConfigureNodesDefault> extends + AbstractConfigureNodes { + + private final ByteArray key; + + public ConfigureNodesDefault(PD pipelineData, + Event completeEvent, + FailureDetector failureDetector, + int required, + RoutingStrategy routingStrategy, + ByteArray key) { + super(pipelineData, completeEvent, failureDetector, required, routingStrategy); + this.key = key; + } + + @Override + public List getNodes(ByteArray key) { + List nodes = null; + + try { + nodes = super.getNodes(key); + } catch(VoldemortException e) { + pipelineData.setFatalError(e); + return null; + } + return nodes; + } + + public void execute(Pipeline pipeline) { + List nodes = null; + + nodes = getNodes(key); + if(nodes == null) { + pipeline.abort(); + return; + } + + if(logger.isDebugEnabled()) + logger.debug("Adding " + nodes.size() + " node(s) to preference list"); + + if(logger.isDebugEnabled()) { + StringBuilder nodeStr = new StringBuilder(); + for(Node node: nodes) { + nodeStr.append(node.getId() + ","); + } + logger.debug("Key " + ByteUtils.toHexString(key.get()) + + " final preference list to contact " + nodeStr); + } + pipelineData.setNodes(nodes); + pipeline.addEvent(completeEvent); + } + +} diff --git a/src/java/voldemort/store/routed/action/ConfigureNodesLocalHost.java b/src/java/voldemort/store/routed/action/ConfigureNodesLocalHost.java new file mode 100644 index 0000000000..4eb2237380 --- /dev/null +++ b/src/java/voldemort/store/routed/action/ConfigureNodesLocalHost.java @@ -0,0 +1,71 @@ +package voldemort.store.routed.action; + +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.log4j.Logger; + +import voldemort.VoldemortException; +import voldemort.cluster.Node; +import voldemort.cluster.failuredetector.FailureDetector; +import voldemort.routing.RoutingStrategy; +import voldemort.store.routed.BasicPipelineData; +import voldemort.store.routed.Pipeline.Event; +import voldemort.utils.ByteArray; + +/* + * Use the default node list returned via the routing strategy. However give + * preference to the current node, if it is part of the preflist returned from + * the routing strategy. + */ + +public class ConfigureNodesLocalHost> extends + ConfigureNodesDefault { + + @SuppressWarnings("hiding") + private final Logger logger = Logger.getLogger(this.getClass()); + + public ConfigureNodesLocalHost(PD pipelineData, + Event completeEvent, + FailureDetector failureDetector, + int required, + RoutingStrategy routingStrategy, + ByteArray key) { + super(pipelineData, completeEvent, failureDetector, required, routingStrategy, key); + } + + /* + * If the current node exists in the nodes list, bring it to the front + */ + @Override + public List getNodes(ByteArray key) { + logger.debug("Giving pref to localhost ! "); + List nodes = null; + List reorderedNodes = new ArrayList(); + + try { + nodes = super.getNodes(key); + String currentHost = InetAddress.getLocalHost().getHostName(); + for(Node n: nodes) { + if(currentHost.contains(n.getHost()) || n.getHost().contains(currentHost)) { + logger.debug("Found localhost ! "); + reorderedNodes.add(n); + nodes.remove(n); + break; + } + } + reorderedNodes.addAll(nodes); + nodes = reorderedNodes; + } catch(VoldemortException e) { + pipelineData.setFatalError(e); + return null; + } catch(UnknownHostException e) { + e.printStackTrace(); + return null; + } + return nodes; + } + +} diff --git a/src/java/voldemort/store/routed/action/ConfigureNodesLocalHostByZone.java b/src/java/voldemort/store/routed/action/ConfigureNodesLocalHostByZone.java new file mode 100644 index 0000000000..ceb5e688eb --- /dev/null +++ b/src/java/voldemort/store/routed/action/ConfigureNodesLocalHostByZone.java @@ -0,0 +1,72 @@ +package voldemort.store.routed.action; + +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.List; + +import voldemort.VoldemortException; +import voldemort.cluster.Node; +import voldemort.cluster.Zone; +import voldemort.cluster.failuredetector.FailureDetector; +import voldemort.routing.RoutingStrategy; +import voldemort.store.routed.BasicPipelineData; +import voldemort.store.routed.Pipeline.Event; +import voldemort.store.routed.Pipeline.Operation; +import voldemort.utils.ByteArray; + +/* + * Use the zone aware node list returned via the routing strategy. However give + * preference to the current node, if it is part of the preflist returned from + * the routing strategy. + */ + +public class ConfigureNodesLocalHostByZone> extends + ConfigureNodesByZone { + + public ConfigureNodesLocalHostByZone(PD pipelineData, + Event completeEvent, + FailureDetector failureDetector, + int required, + RoutingStrategy routingStrategy, + ByteArray key, + Zone clientZone) { + super(pipelineData, + completeEvent, + failureDetector, + required, + routingStrategy, + key, + clientZone); + } + + /* + * If the current node exists in the nodes list, bring it to the front + */ + @Override + public List getNodes(ByteArray key, Operation op) { + List nodes = null; + List reorderedNodes = new ArrayList(); + + try { + nodes = super.getNodes(key, op); + String currentHost = InetAddress.getLocalHost().getHostName(); + for(Node n: nodes) { + if(currentHost.contains(n.getHost()) || n.getHost().contains(currentHost)) { + reorderedNodes.add(n); + nodes.remove(n); + break; + } + } + reorderedNodes.addAll(nodes); + nodes = reorderedNodes; + } catch(VoldemortException e) { + pipelineData.setFatalError(e); + return null; + } catch(UnknownHostException e) { + e.printStackTrace(); + return null; + } + return nodes; + } +} From c41962885a66212e96d1b3221b3ebf5e63e58f8e Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Wed, 6 Jun 2012 14:38:45 -0700 Subject: [PATCH 043/209] Added per socket monitoring for client executor pool. These monitoring reveals stats including averag, 50th, 99th wait ms for socket checkout; number of checkout in an interval etc --- .../client/SocketStoreClientFactory.java | 12 +- .../ClientRequestExecutorFactory.java | 12 + .../ClientRequestExecutorPool.java | 86 ++----- .../store/stats/ClientSocketStats.java | 230 ++++++++++++++++++ .../store/stats/ClientSocketStatsJmx.java | 101 ++++++++ .../socket/ClientRequestExecutorPoolTest.java | 13 +- .../store/stats/ClientSocketStatsTest.java | 174 +++++++++++++ 7 files changed, 549 insertions(+), 79 deletions(-) create mode 100644 src/java/voldemort/store/stats/ClientSocketStats.java create mode 100644 src/java/voldemort/store/stats/ClientSocketStatsJmx.java create mode 100644 test/unit/voldemort/store/stats/ClientSocketStatsTest.java diff --git a/src/java/voldemort/client/SocketStoreClientFactory.java b/src/java/voldemort/client/SocketStoreClientFactory.java index e86ae31322..8da1bd7135 100644 --- a/src/java/voldemort/client/SocketStoreClientFactory.java +++ b/src/java/voldemort/client/SocketStoreClientFactory.java @@ -38,7 +38,6 @@ import voldemort.store.socket.SocketStoreFactory; import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; import voldemort.utils.ByteArray; -import voldemort.utils.JmxUtils; import voldemort.versioning.InconsistencyResolver; import voldemort.versioning.Versioned; @@ -69,8 +68,9 @@ public SocketStoreClientFactory(ClientConfig config) { config.getSocketTimeout(TimeUnit.MILLISECONDS), config.getSocketBufferSize(), config.getSocketKeepAlive()); - if(config.isJmxEnabled()) - JmxUtils.registerMbean(storeFactory, JmxUtils.createObjectName(storeFactory.getClass())); + if(config.isJmxEnabled()) { + ((ClientRequestExecutorPool) storeFactory).registerJmx(); + } } @Override @@ -87,7 +87,8 @@ public StoreClient call() throws Exception { return getParentStoreClient(storeName, resolver); } - private StoreClient getParentStoreClient(String storeName, InconsistencyResolver> resolver) { + private StoreClient getParentStoreClient(String storeName, + InconsistencyResolver> resolver) { return super.getStoreClient(storeName, resolver); } @@ -96,7 +97,8 @@ protected List> getRemoteMetadata(String key, URI url) { try { return super.getRemoteMetadata(key, url); } catch(VoldemortException e) { - // Fix SNA-4227: When an error occurs during bootstrap, close the socket + // Fix SNA-4227: When an error occurs during bootstrap, close the + // socket SocketDestination destination = new SocketDestination(url.getHost(), url.getPort(), getRequestFormatType()); diff --git a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorFactory.java b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorFactory.java index ba2b68a25d..74f33df12b 100644 --- a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorFactory.java +++ b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorFactory.java @@ -37,6 +37,7 @@ import org.apache.log4j.Logger; import voldemort.store.socket.SocketDestination; +import voldemort.store.stats.ClientSocketStats; import voldemort.utils.DaemonThreadFactory; import voldemort.utils.SelectorManager; import voldemort.utils.Time; @@ -61,6 +62,7 @@ public class ClientRequestExecutorFactory implements private final AtomicInteger counter = new AtomicInteger(); private final Map lastClosedTimestamps; private final Logger logger = Logger.getLogger(getClass()); + private ClientSocketStats stats; public ClientRequestExecutorFactory(int selectors, int connectTimeoutMs, @@ -94,6 +96,9 @@ public void destroy(SocketDestination dest, ClientRequestExecutor clientRequestE throws Exception { clientRequestExecutor.close(); int numDestroyed = destroyed.incrementAndGet(); + if(stats != null) { + stats.connectionDestroy(dest); + } if(logger.isDebugEnabled()) logger.debug("Destroyed socket " + numDestroyed + " connection to " + dest.getHost() @@ -108,6 +113,9 @@ public void destroy(SocketDestination dest, ClientRequestExecutor clientRequestE public ClientRequestExecutor create(SocketDestination dest) throws Exception { int numCreated = created.incrementAndGet(); + if(stats != null) { + stats.connectionCreate(dest); + } if(logger.isDebugEnabled()) logger.debug("Creating socket " + numCreated + " for " + dest.getHost() + ":" @@ -426,4 +434,8 @@ public void setLastClosedTimestamp(SocketDestination socketDestination) { lastClosedTimestamps.put(socketDestination, System.nanoTime()); } + public void setStats(ClientSocketStats stats) { + this.stats = stats; + } + } diff --git a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java index cebcb720f0..6cc35dacb6 100644 --- a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java +++ b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java @@ -17,20 +17,17 @@ package voldemort.store.socket.clientrequest; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; import voldemort.VoldemortException; -import voldemort.annotations.jmx.JmxGetter; -import voldemort.annotations.jmx.JmxManaged; -import voldemort.annotations.jmx.JmxSetter; import voldemort.client.protocol.RequestFormatType; import voldemort.server.RequestRoutingType; import voldemort.store.UnreachableStoreException; import voldemort.store.socket.SocketDestination; import voldemort.store.socket.SocketStore; import voldemort.store.socket.SocketStoreFactory; -import voldemort.store.stats.Histogram; +import voldemort.store.stats.ClientSocketStats; +import voldemort.store.stats.ClientSocketStatsJmx; +import voldemort.utils.JmxUtils; import voldemort.utils.Time; import voldemort.utils.Utils; import voldemort.utils.pool.KeyedResourcePool; @@ -47,16 +44,11 @@ * terminated upon calling {@link #close()}. */ -@JmxManaged(description = "Voldemort socket pool.") public class ClientRequestExecutorPool implements SocketStoreFactory { - private final AtomicInteger monitoringInterval = new AtomicInteger(10000); - private final AtomicInteger checkouts; - private final AtomicLong waitNs; - private final AtomicLong avgWaitNs; - private final Histogram histogramWaitNs; private final KeyedResourcePool pool; private final ClientRequestExecutorFactory factory; + private ClientSocketStats stats; public ClientRequestExecutorPool(int selectors, int maxConnectionsPerNode, @@ -75,10 +67,8 @@ public ClientRequestExecutorPool(int selectors, socketBufferSize, socketKeepAlive); this.pool = new KeyedResourcePool(factory, config); - this.checkouts = new AtomicInteger(0); - this.waitNs = new AtomicLong(0); - this.avgWaitNs = new AtomicLong(0); - this.histogramWaitNs = new Histogram(10000, 1000); + this.stats = new ClientSocketStats(pool); + ((ClientRequestExecutorFactory) factory).setStats(stats); } public ClientRequestExecutorPool(int maxConnectionsPerNode, @@ -108,6 +98,13 @@ public SocketStore create(String storeName, requestRoutingType); } + public void registerJmx() { + stats.enableJmx(); + JmxUtils.registerMbean(new ClientSocketStatsJmx(stats), + JmxUtils.createObjectName("voldemort.store.socket.clientrequest", + "aggregated")); + } + /** * Checkout a socket from the pool * @@ -120,7 +117,10 @@ public ClientRequestExecutor checkout(SocketDestination destination) { // time checkout long start = System.nanoTime(); ClientRequestExecutor clientRequestExecutor = pool.checkout(destination); - updateStats(System.nanoTime() - start); + long end = System.nanoTime(); + if(stats != null) { + stats.recordCheckoutTimeUs(destination, (end - start) / Time.NS_PER_US); + } return clientRequestExecutor; } catch(Exception e) { @@ -129,21 +129,6 @@ public ClientRequestExecutor checkout(SocketDestination destination) { } } - private void updateStats(long checkoutTimeNs) { - long wait = waitNs.getAndAdd(checkoutTimeNs); - int count = checkouts.getAndIncrement(); - - // reset reporting interval if we have used up the current interval - int interval = this.monitoringInterval.get(); - if(count % interval == interval - 1) { - // harmless race condition: - waitNs.set(0); - checkouts.set(0); - avgWaitNs.set(wait / count); - histogramWaitNs.insert(checkoutTimeNs); - } - } - /** * Check the socket back into the pool. * @@ -172,41 +157,8 @@ public void close() { pool.close(); } - @JmxGetter(name = "socketsCreated", description = "The total number of sockets created by this pool.") - public int getNumberSocketsCreated() { - return this.factory.getNumberCreated(); - } - - @JmxGetter(name = "socketsDestroyed", description = "The total number of sockets destroyed by this pool.") - public int getNumberSocketsDestroyed() { - return this.factory.getNumberDestroyed(); - } - - @JmxGetter(name = "numberOfConnections", description = "The number of active connections.") - public int getNumberOfActiveConnections() { - return this.pool.getTotalResourceCount(); - } - - @JmxGetter(name = "numberOfIdleConnections", description = "The number of idle connections.") - public int getNumberOfCheckedInConnections() { - return this.pool.getCheckedInResourceCount(); - } - - @JmxGetter(name = "avgWaitTimeMs", description = "The avg. ms of wait time to acquire a connection.") - public double getAvgWaitTimeMs() { - return this.avgWaitNs.doubleValue() / Time.NS_PER_MS; - } - - @JmxGetter(name = "99thWaitTimeMs", description = "The 99th percentile ms of wait time to acquire a connection.") - public double get99thWaitTimeMs() { - return (double) (this.histogramWaitNs.getQuantile(0.99)) / Time.NS_PER_MS; - } - - @JmxSetter(name = "monitoringInterval", description = "The number of checkouts over which performance statistics are calculated.") - public void setMonitoringInterval(int count) { - if(count <= 0) - throw new IllegalArgumentException("Monitoring interval must be a positive number."); - this.monitoringInterval.set(count); + public ClientSocketStats getStats() { + return stats; } } diff --git a/src/java/voldemort/store/stats/ClientSocketStats.java b/src/java/voldemort/store/stats/ClientSocketStats.java new file mode 100644 index 0000000000..048c340b1b --- /dev/null +++ b/src/java/voldemort/store/stats/ClientSocketStats.java @@ -0,0 +1,230 @@ +/* + * Copyright 2008-2011 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package voldemort.store.stats; + +import java.util.Iterator; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +import voldemort.store.socket.SocketDestination; +import voldemort.store.socket.clientrequest.ClientRequestExecutor; +import voldemort.utils.JmxUtils; +import voldemort.utils.pool.KeyedResourcePool; + +/** + * Some convenient statistics to track about the client requests + * + * + */ +public class ClientSocketStats { + + private final ClientSocketStats parent; + private final ConcurrentMap statsMap; + private final SocketDestination destination; + private final KeyedResourcePool pool; + + private final AtomicInteger monitoringInterval = new AtomicInteger(10000); + private final Histogram checkoutTimeUsHistogram = new Histogram(20000, 100); + private final AtomicLong totalCheckoutTimeUs = new AtomicLong(0); + private final AtomicLong avgCheckoutTimeUs = new AtomicLong(0); + private final AtomicInteger connectionsCreated = new AtomicInteger(0); + private final AtomicInteger connectionsDestroyed = new AtomicInteger(0); + private final AtomicInteger connectionsCheckedout = new AtomicInteger(0); + private boolean jmxEnable = false; + + // private final AtomicInteger connectionsCheckedin = new AtomicInteger(0); + + /** + * To construct a per node stats object + * + * @param parent An optional parent stats object that will maintain + * aggregate data across many sockets + * @param destination The destination object that defines the node + * @param pool The socket pool that will give out connection information + */ + public ClientSocketStats(ClientSocketStats parent, + SocketDestination destination, + KeyedResourcePool pool) { + this.parent = parent; + this.statsMap = null; + this.destination = destination; + this.pool = pool; + } + + /** + * Construction of a new aggregate stats object + * + * @param pool The socket pool that will give out connection information + */ + public ClientSocketStats(KeyedResourcePool pool) { + this.parent = null; + this.statsMap = new ConcurrentHashMap(); + this.destination = null; + this.pool = pool; + } + + /* get per node stats, create one if not exist */ + private ClientSocketStats getOrCreateNodeStats(SocketDestination destination) { + if(destination == null) { + return null; + } + ClientSocketStats stats = statsMap.get(destination); + if(stats == null) { + stats = new ClientSocketStats(this, destination, pool); + statsMap.putIfAbsent(destination, stats); + stats = statsMap.get(destination); + if(this.jmxEnable) { + JmxUtils.registerMbean(new ClientSocketStatsJmx(stats), + JmxUtils.createObjectName("voldemort.store.socket.clientrequest", + "stats_" + + destination.toString() + .replace(':', + '_'))); + } + } + return stats; + } + + /** + * Record the checkout wait time in us + * + * @param dest Destination of the socket to checkout. Will actually record + * if null. Otherwise will call this on self and corresponding child + * with this param null. + * @param checkoutTimeUs The number of us to wait before getting a socket + */ + public void recordCheckoutTimeUs(SocketDestination dest, long checkoutTimeUs) { + if(dest != null) { + getOrCreateNodeStats(dest).recordCheckoutTimeUs(null, checkoutTimeUs); + recordCheckoutTimeUs(null, checkoutTimeUs); + } else { + this.totalCheckoutTimeUs.getAndAdd(checkoutTimeUs); + checkoutTimeUsHistogram.insert(checkoutTimeUs); + int checkouts = this.connectionsCheckedout.getAndIncrement(); + + // reset aggregated stats and all the node stats for new interval + if(parent == null && statsMap != null) { + int interval = this.monitoringInterval.get(); + if(checkouts % interval == interval - 1) { + // reset all children + Iterator it = statsMap.keySet().iterator(); + while(it.hasNext()) { + ClientSocketStats stats = statsMap.get(it.next()); + stats.resetForInterval(); + } + // reset itself + resetForInterval(); + } + } + } + } + + /** + * Calculate the average and reset the stats + */ + public void resetForInterval() { + // harmless race condition: + this.totalCheckoutTimeUs.set(0); + this.connectionsCheckedout.set(0); + checkoutTimeUsHistogram.reset(); + } + + public void connectionCreate(SocketDestination dest) { + if(dest != null) { + getOrCreateNodeStats(dest).connectionCreate(null); + connectionCreate(null); + } else { + this.connectionsCreated.getAndIncrement(); + } + } + + public void connectionDestroy(SocketDestination dest) { + if(dest != null) { + getOrCreateNodeStats(dest).connectionDestroy(null); + connectionDestroy(null); + } else { + this.connectionsDestroyed.getAndIncrement(); + } + } + + /* getters */ + + public int getConnectionsCreated() { + return connectionsCreated.intValue(); + } + + public int getConnectionsDestroyed() { + return connectionsDestroyed.intValue(); + } + + // public int getConnectionsCheckedin() { + // return connectionsCheckedin.intValue(); + // } + + public int getConnectionsCheckedout() { + return connectionsCheckedout.intValue(); + } + + public Histogram getWaitHistogram() { + return this.checkoutTimeUsHistogram; + } + + public long getAveWaitUs() { + long ns = totalCheckoutTimeUs.get(); + int count = connectionsCheckedout.get(); + this.avgCheckoutTimeUs.set(count > 0 ? (ns / count) : -1); + return this.avgCheckoutTimeUs.longValue(); + } + + public int getConnectionsActive(SocketDestination destination) { + if(destination == null) { + return pool.getTotalResourceCount(); + } else { + return pool.getTotalResourceCount(destination); + } + } + + public int getConnectionsInPool(SocketDestination destination) { + if(destination == null) { + return pool.getCheckedInResourceCount(); + } else { + return pool.getCheckedInResourcesCount(destination); + } + } + + public void setMonitoringInterval(int count) { + this.monitoringInterval.set(count); + } + + public int getMonitoringInterval() { + return this.monitoringInterval.get(); + } + + public void enableJmx() { + jmxEnable = true; + } + + ConcurrentMap getStatsMap() { + return statsMap; + } + + SocketDestination getDestination() { + return destination; + } +} diff --git a/src/java/voldemort/store/stats/ClientSocketStatsJmx.java b/src/java/voldemort/store/stats/ClientSocketStatsJmx.java new file mode 100644 index 0000000000..ed96d6ec56 --- /dev/null +++ b/src/java/voldemort/store/stats/ClientSocketStatsJmx.java @@ -0,0 +1,101 @@ +/* + * Copyright 2008-2011 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package voldemort.store.stats; + +import voldemort.annotations.jmx.JmxGetter; +import voldemort.annotations.jmx.JmxManaged; +import voldemort.annotations.jmx.JmxSetter; +import voldemort.utils.Time; + +/** + * + * A wrapper class to expose client socket stats via JMX + * + */ + +@JmxManaged(description = "Voldemort socket pool.") +public class ClientSocketStatsJmx { + + private final ClientSocketStats stats; + + /** + * Class for JMX + */ + public ClientSocketStatsJmx(ClientSocketStats stats) { + this.stats = stats; + } + + @JmxGetter(name = "socketsCreated", description = "Number of sockets created.") + public int getConnectionsCreated() { + return stats.getConnectionsCreated(); + } + + @JmxGetter(name = "socketsDestroyed", description = "Number of sockets destroyed.") + public int getConnectionsDestroyed() { + return stats.getConnectionsDestroyed(); + } + + @JmxGetter(name = "socketsCheckedout", description = "Number of sockets checked out.") + public int getConnectionsCheckinout() { + return stats.getConnectionsCheckedout(); + } + + @JmxGetter(name = "waitMsAverage", description = "Average ms to wait to get a socket.") + public double getWaitMsAverage() { + return (double) stats.getAveWaitUs() / Time.US_PER_MS; + } + + @JmxGetter(name = "waitMsQ50th", description = "50th percentile wait time to get a connection.") + public double get() { + return (double) stats.getWaitHistogram().getQuantile(0.5) / Time.US_PER_MS; + } + + @JmxGetter(name = "waitMsQ99th", description = "99th percentile wait time to get a connection.") + public double getWaitMsQ99th() { + return (double) stats.getWaitHistogram().getQuantile(0.99) / Time.US_PER_MS; + } + + @JmxGetter(name = "socketsActive", description = "Total number of sockets, checkedin and checkout.") + public int getConnActive() { + int result = -1; + try { + result = stats.getConnectionsActive(stats.getDestination()); + } catch(Exception e) {} + return result; + } + + @JmxGetter(name = "socketsInPool", description = "Total number of sockets in the pool.") + public int getConnAvailable() { + int result = -1; + try { + result = stats.getConnectionsInPool(stats.getDestination()); + } catch(Exception e) {} + return result; + } + + @JmxGetter(name = "monitoringInterval", description = "The number of checkouts over which performance statics are calculated.") + public int getMonitoringInterval() { + return stats.getMonitoringInterval(); + } + + @JmxSetter(name = "monitoringInterval", description = "The number of checkouts over which performance statics are calculated.") + public void setMonitoringInterval(int count) { + if(count <= 0) + throw new IllegalArgumentException("Monitoring interval must be a positive number."); + stats.setMonitoringInterval(count); + } +} diff --git a/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java b/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java index b81037c0cf..3a500aadae 100644 --- a/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java +++ b/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java @@ -112,20 +112,19 @@ public void testCloseWithInFlightSockets() throws Exception { for(int i = 0; i < maxConnectionsPerNode; i++) list.add(pool.checkout(dest1)); - assertEquals(list.size(), pool.getNumberSocketsCreated()); - assertEquals(list.size(), pool.getNumberOfActiveConnections()); + assertEquals(list.size(), pool.getStats().getConnectionsCreated()); + assertEquals(list.size(), pool.getStats().getConnectionsActive(null)); pool.close(dest1); - assertEquals(list.size(), pool.getNumberOfActiveConnections()); - assertEquals(0, pool.getNumberSocketsDestroyed()); + assertEquals(list.size(), pool.getStats().getConnectionsActive(null)); + assertEquals(0, pool.getStats().getConnectionsDestroyed()); for(ClientRequestExecutor sas: list) pool.checkin(dest1, sas); - assertEquals(0, pool.getNumberOfActiveConnections()); - assertEquals(list.size(), pool.getNumberSocketsDestroyed()); - assertEquals(0, pool.getNumberOfCheckedInConnections()); + assertEquals(0, pool.getStats().getConnectionsActive(null)); + assertEquals(list.size(), pool.getStats().getConnectionsCreated()); } @Test diff --git a/test/unit/voldemort/store/stats/ClientSocketStatsTest.java b/test/unit/voldemort/store/stats/ClientSocketStatsTest.java new file mode 100644 index 0000000000..aefd1caeb2 --- /dev/null +++ b/test/unit/voldemort/store/stats/ClientSocketStatsTest.java @@ -0,0 +1,174 @@ +/* + * Copyright 2008-2011 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package voldemort.store.stats; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import org.junit.Before; +import org.junit.Test; + +import voldemort.ServerTestUtils; +import voldemort.client.protocol.RequestFormatType; +import voldemort.store.socket.SocketDestination; +import voldemort.store.socket.clientrequest.ClientRequestExecutor; +import voldemort.utils.pool.KeyedResourcePool; + +public class ClientSocketStatsTest { + + private ClientSocketStats masterStats; + private int port; + private SocketDestination dest1; + private SocketDestination dest2; + private KeyedResourcePool pool; + + @Before + public void setUp() throws Exception { + this.port = ServerTestUtils.findFreePort(); + this.dest1 = new SocketDestination("localhost", port, RequestFormatType.VOLDEMORT_V1); + this.dest2 = new SocketDestination("localhost", port + 1, RequestFormatType.VOLDEMORT_V1); + this.masterStats = new ClientSocketStats(null); + pool = null; + } + + @Test + public void testNewNodeStatsObject() { + ClientSocketStats stats = new ClientSocketStats(masterStats, dest1, pool); + assertNotNull(stats); + } + + @Test + public void testNewAggrNodeStatsObject() { + ClientSocketStats stats = masterStats; + assertNotNull(stats); + assertEquals(0, stats.getConnectionsCreated()); + assertEquals(0, stats.getConnectionsDestroyed()); + assertEquals(0, stats.getConnectionsCheckedout()); + assertEquals(-1, stats.getAveWaitUs()); + } + + @Test + public void testConnectionCreate() { + ClientSocketStats stats = masterStats; + stats.connectionCreate(dest1); + stats.connectionCreate(dest2); + stats.connectionCreate(dest1); + assertEquals(3, stats.getConnectionsCreated()); + assertEquals(2, stats.getStatsMap().get(dest1).getConnectionsCreated()); + assertEquals(1, stats.getStatsMap().get(dest2).getConnectionsCreated()); + } + + @Test + public void testConnectionDestroy() { + ClientSocketStats stats = masterStats; + stats.connectionDestroy(dest1); + stats.connectionDestroy(dest2); + stats.connectionDestroy(dest1); + assertEquals(3, stats.getConnectionsDestroyed()); + assertEquals(2, stats.getStatsMap().get(dest1).getConnectionsDestroyed()); + assertEquals(1, stats.getStatsMap().get(dest2).getConnectionsDestroyed()); + } + + @Test + public void testRecordCheckoutTimeNsOnce() { + ClientSocketStats stats = masterStats; + assertEquals(0, stats.getConnectionsCheckedout()); + + stats.recordCheckoutTimeUs(dest1, 100); + // check parent + assertEquals(1, stats.getConnectionsCheckedout()); + assertEquals(100, stats.getWaitHistogram().getQuantile(0.99)); + + // check child + ClientSocketStats child = stats.getStatsMap().get(dest1); + assertNotNull(child); + assertEquals(1, child.getConnectionsCheckedout()); + assertEquals(100, child.getWaitHistogram().getQuantile(0.99)); + } + + @Test + public void testRecordCheckoutTimeNsMultiple() { + ClientSocketStats stats = masterStats; + assertEquals(0, stats.getConnectionsCheckedout()); + + stats.recordCheckoutTimeUs(dest1, 100); + stats.recordCheckoutTimeUs(dest1, 200); + stats.recordCheckoutTimeUs(dest1, 300); + stats.recordCheckoutTimeUs(dest1, 400); + stats.recordCheckoutTimeUs(dest2, 500); + stats.recordCheckoutTimeUs(dest2, 600); + stats.recordCheckoutTimeUs(dest1, 700); + stats.recordCheckoutTimeUs(dest1, 800); + stats.recordCheckoutTimeUs(dest2, 900); + + // check parent + assertEquals(9, stats.getConnectionsCheckedout()); + assertEquals(900, stats.getWaitHistogram().getQuantile(0.99)); + + // check child1 + ClientSocketStats child1 = stats.getStatsMap().get(dest1); + assertNotNull(child1); + assertEquals(6, child1.getConnectionsCheckedout()); + assertEquals(100, child1.getWaitHistogram().getQuantile(0.1)); + assertEquals(300, child1.getWaitHistogram().getQuantile(0.5)); + assertEquals(800, child1.getWaitHistogram().getQuantile(0.99)); + + // check child2 + ClientSocketStats child2 = stats.getStatsMap().get(dest2); + assertNotNull(child2); + assertEquals(3, child2.getConnectionsCheckedout()); + assertEquals(500, child2.getWaitHistogram().getQuantile(0.1)); + assertEquals(600, child2.getWaitHistogram().getQuantile(0.5)); + assertEquals(900, child2.getWaitHistogram().getQuantile(0.99)); + } + + @Test + public void testSetMonitoringInterval() { + ClientSocketStats stats = masterStats; + stats.setMonitoringInterval(9); + stats.recordCheckoutTimeUs(dest1, 100); + stats.recordCheckoutTimeUs(dest1, 200); + stats.recordCheckoutTimeUs(dest1, 300); + stats.recordCheckoutTimeUs(dest1, 400); + stats.recordCheckoutTimeUs(dest1, 500); + stats.recordCheckoutTimeUs(dest1, 600); + stats.recordCheckoutTimeUs(dest2, 700); + stats.recordCheckoutTimeUs(dest2, 800); + // before interval based reset + // check parent + assertEquals(8, stats.getConnectionsCheckedout()); + assertEquals(450, stats.getAveWaitUs()); + // check child + ClientSocketStats child1 = stats.getStatsMap().get(dest1); + ClientSocketStats child2 = stats.getStatsMap().get(dest2); + assertEquals(6, child1.getConnectionsCheckedout()); + assertEquals(2, child2.getConnectionsCheckedout()); + assertEquals(350, child1.getAveWaitUs()); + assertEquals(750, child2.getAveWaitUs()); + + // after interval based reset + stats.recordCheckoutTimeUs(dest2, 900000); + // check parent + assertEquals(-1, stats.getAveWaitUs()); + assertEquals(0, stats.getConnectionsCheckedout()); + // check child + assertEquals(-1, child1.getAveWaitUs()); + assertEquals(0, child1.getConnectionsCheckedout()); + assertEquals(-1, child2.getAveWaitUs()); + assertEquals(0, child2.getConnectionsCheckedout()); + } +} From f76d8b526ce3cf16c5310227b4525126860215e1 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Thu, 7 Jun 2012 11:40:25 -0700 Subject: [PATCH 044/209] Modified the test and build file to shrink the large tests. The large tests are copied to the test/large To run short tests for sanity check purpose: ant junit To run short and long tests: ant junit-long To run short, long and contrib tests: ant junit-all --- build.properties | 1 + build.xml | 23 +++- .../client/rebalance/RebalanceLargeTest.java | 119 ++++++++++++++++++ .../rebalance/AbstractRebalanceTest.java | 17 ++- 4 files changed, 153 insertions(+), 7 deletions(-) create mode 100644 test/large/voldemort/client/rebalance/RebalanceLargeTest.java diff --git a/build.properties b/build.properties index 3bc09bc26c..933629c18e 100644 --- a/build.properties +++ b/build.properties @@ -9,6 +9,7 @@ classes.dir=dist/classes resources.dir=dist/resources commontestsrc.dir=test/common unittestsrc.dir=test/unit +largetestsrc.dir=test/large inttestsrc.dir=test/integration testclasses.dir=dist/testclasses testreport.dir=dist/junit-reports diff --git a/build.xml b/build.xml index 1f9a535f22..1c4d4c6136 100644 --- a/build.xml +++ b/build.xml @@ -103,6 +103,7 @@ + @@ -387,6 +388,26 @@ + + + + + + + + + + + + + + + + + + + + @@ -394,7 +415,7 @@ - + diff --git a/test/large/voldemort/client/rebalance/RebalanceLargeTest.java b/test/large/voldemort/client/rebalance/RebalanceLargeTest.java new file mode 100644 index 0000000000..16d5b5091f --- /dev/null +++ b/test/large/voldemort/client/rebalance/RebalanceLargeTest.java @@ -0,0 +1,119 @@ +package voldemort.client.rebalance; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +import voldemort.ServerTestUtils; +import voldemort.TestUtils; +import voldemort.VoldemortException; +import voldemort.cluster.Cluster; +import voldemort.server.VoldemortConfig; +import voldemort.server.VoldemortServer; +import voldemort.store.metadata.MetadataStore.VoldemortState; + +/** + * Start VoldemortServer locally using ServerTestUtils and run rebalancing + * tests. + * + * + */ +@RunWith(Parameterized.class) +public class RebalanceLargeTest extends AbstractRebalanceTest { + + Map serverMap = new HashMap(); + private final boolean useNio; + private final boolean useDonorBased; + protected static int NUM_MANY_KEYS = 10100; + + public RebalanceLargeTest(boolean useNio, boolean useDonorBased) { + this.useNio = useNio; + this.useDonorBased = useDonorBased; + } + + @Parameters + public static Collection configs() { + return Arrays.asList(new Object[][] { { true, true }, { true, false }, { false, true }, + { false, false } }); + } + + @Override + protected int getNumKeys() { + return NUM_MANY_KEYS; + } + + @Override + protected VoldemortState getCurrentState(int nodeId) { + VoldemortServer server = serverMap.get(nodeId); + if(server == null) { + throw new VoldemortException("Node id " + nodeId + " does not exist"); + } else { + return server.getMetadataStore().getServerState(); + } + } + + @Override + protected Cluster getCurrentCluster(int nodeId) { + VoldemortServer server = serverMap.get(nodeId); + if(server == null) { + throw new VoldemortException("Node id " + nodeId + " does not exist"); + } else { + return server.getMetadataStore().getCluster(); + } + } + + @Override + protected Cluster startServers(Cluster cluster, + String storeXmlFile, + List nodeToStart, + Map configProps) throws IOException { + for(int node: nodeToStart) { + Properties properties = new Properties(); + if(null != configProps) { + for(Entry property: configProps.entrySet()) { + properties.put(property.getKey(), property.getValue()); + } + } + + VoldemortConfig config = ServerTestUtils.createServerConfig(useNio, + node, + TestUtils.createTempDir() + .getAbsolutePath(), + null, + storeXmlFile, + properties); + + VoldemortServer server = ServerTestUtils.startVoldemortServer(socketStoreFactory, + config, + cluster); + serverMap.put(node, server); + } + + return cluster; + } + + @Override + protected void stopServer(List nodesToStop) throws IOException { + for(int node: nodesToStop) { + try { + ServerTestUtils.stopVoldemortServer(serverMap.get(node)); + } catch(VoldemortException e) { + // ignore these at stop time + } + } + } + + @Override + protected boolean useDonorBased() { + return this.useDonorBased; + } +} diff --git a/test/unit/voldemort/client/rebalance/AbstractRebalanceTest.java b/test/unit/voldemort/client/rebalance/AbstractRebalanceTest.java index d3b8fb3b47..cde3ee5638 100644 --- a/test/unit/voldemort/client/rebalance/AbstractRebalanceTest.java +++ b/test/unit/voldemort/client/rebalance/AbstractRebalanceTest.java @@ -29,8 +29,8 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.Map.Entry; +import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -86,7 +86,7 @@ public abstract class AbstractRebalanceTest { - protected static int NUM_KEYS = 10100; + protected static int NUM_KEYS = 20; protected static int NUM_RO_CHUNKS_PER_BUCKET = 10; protected static String testStoreNameRW = "test"; protected static String testStoreNameRW2 = "test2"; @@ -111,7 +111,7 @@ public abstract class AbstractRebalanceTest { @Before public void setUp() throws IOException { - testEntries = ServerTestUtils.createRandomKeyValueString(NUM_KEYS); + testEntries = ServerTestUtils.createRandomKeyValueString(getNumKeys()); socketStoreFactory = new ClientRequestExecutorPool(2, 10000, 100000, 32 * 1024); // First without replication @@ -258,6 +258,10 @@ public void checkConsistentMetadata(Cluster targetCluster, List serverL } } + protected int getNumKeys() { + return NUM_KEYS; + } + @Test public void testRORWRebalance() throws Exception { Cluster currentCluster = ServerTestUtils.getLocalCluster(2, new int[][] { @@ -1053,9 +1057,10 @@ protected void populateData(Cluster cluster, // Create SocketStores for each Node first Map> storeMap = new HashMap>(); for(Node node: cluster.getNodes()) { - storeMap.put(node.getId(), getSocketStore(storeDef.getName(), - node.getHost(), - node.getSocketPort())); + storeMap.put(node.getId(), + getSocketStore(storeDef.getName(), + node.getHost(), + node.getSocketPort())); } From eb00c3dbc1499d56e53f2ec8b987d2d42bbcbbc8 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Thu, 7 Jun 2012 14:48:27 -0700 Subject: [PATCH 045/209] renamed large as long to make files consistent --- build.properties | 2 +- build.xml | 4 ++-- .../voldemort/client/rebalance/RebalanceLongTest.java} | 0 3 files changed, 3 insertions(+), 3 deletions(-) rename test/{large/voldemort/client/rebalance/RebalanceLargeTest.java => long/voldemort/client/rebalance/RebalanceLongTest.java} (100%) diff --git a/build.properties b/build.properties index 933629c18e..3cc0b8cef7 100644 --- a/build.properties +++ b/build.properties @@ -9,7 +9,7 @@ classes.dir=dist/classes resources.dir=dist/resources commontestsrc.dir=test/common unittestsrc.dir=test/unit -largetestsrc.dir=test/large +longtestsrc.dir=test/long inttestsrc.dir=test/integration testclasses.dir=dist/testclasses testreport.dir=dist/junit-reports diff --git a/build.xml b/build.xml index 1c4d4c6136..d11d89938d 100644 --- a/build.xml +++ b/build.xml @@ -103,7 +103,7 @@ - + @@ -395,7 +395,7 @@ - + diff --git a/test/large/voldemort/client/rebalance/RebalanceLargeTest.java b/test/long/voldemort/client/rebalance/RebalanceLongTest.java similarity index 100% rename from test/large/voldemort/client/rebalance/RebalanceLargeTest.java rename to test/long/voldemort/client/rebalance/RebalanceLongTest.java From d37df6b76c914ae09da48f5ddeb00dea810d563a Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Thu, 7 Jun 2012 16:14:13 -0700 Subject: [PATCH 046/209] corrected a class name mistake --- test/long/voldemort/client/rebalance/RebalanceLongTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/long/voldemort/client/rebalance/RebalanceLongTest.java b/test/long/voldemort/client/rebalance/RebalanceLongTest.java index 16d5b5091f..eecba30bb0 100644 --- a/test/long/voldemort/client/rebalance/RebalanceLongTest.java +++ b/test/long/voldemort/client/rebalance/RebalanceLongTest.java @@ -28,14 +28,14 @@ * */ @RunWith(Parameterized.class) -public class RebalanceLargeTest extends AbstractRebalanceTest { +public class RebalanceLongTest extends AbstractRebalanceTest { Map serverMap = new HashMap(); private final boolean useNio; private final boolean useDonorBased; protected static int NUM_MANY_KEYS = 10100; - public RebalanceLargeTest(boolean useNio, boolean useDonorBased) { + public RebalanceLongTest(boolean useNio, boolean useDonorBased) { this.useNio = useNio; this.useDonorBased = useDonorBased; } From bcf42c7fac85b0de9901561752f3ca9f6d25cfe3 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Thu, 7 Jun 2012 22:35:52 -0700 Subject: [PATCH 047/209] distinguish test report locations between normal tests and long tests --- build.properties | 2 ++ build.xml | 12 ++++++------ 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/build.properties b/build.properties index 3cc0b8cef7..f363ea1bff 100644 --- a/build.properties +++ b/build.properties @@ -14,6 +14,8 @@ inttestsrc.dir=test/integration testclasses.dir=dist/testclasses testreport.dir=dist/junit-reports testhtml.dir=dist/junit-reports/html +longtestreport.dir=dist/junit-long-reports +longtesthtml.dir=dist/junit-long-reports/html ## Contrib contrib.root.dir=contrib diff --git a/build.xml b/build.xml index d11d89938d..149472494d 100644 --- a/build.xml +++ b/build.xml @@ -389,22 +389,22 @@ - - + + - + - - + + - + From 366cd06a26f4e893422fbed077d4f0d31d935a4c Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Tue, 12 Jun 2012 13:49:20 -0700 Subject: [PATCH 048/209] client registry impl --- src/java/voldemort/VoldemortAdminTool.java | 25 ++++- .../client/AbstractStoreClientFactory.java | 34 ++---- .../client/AsyncMetadataVersionManager.java | 2 +- src/java/voldemort/client/ClientInfo.java | 106 ++++++++++++++++++ .../voldemort/client/DefaultStoreClient.java | 89 +++++++++------ .../client/SocketStoreClientFactory.java | 2 +- src/java/voldemort/client/SystemStore.java | 86 ++++++++------ .../client/protocol/admin/AdminClient.java | 11 +- .../admin/FetchStreamRequestHandler.java | 13 ++- .../server/storage/StorageService.java | 6 +- src/java/voldemort/store/StoreDefinition.java | 2 +- .../store/metadata/MetadataStore.java | 2 +- .../store/system/SystemStoreConstants.java | 104 +++++++++++++++++ .../voldemort/xml/StoreDefinitionsMapper.java | 2 +- .../client/DefaultSocketStoreClientTest.java | 78 +++++++++++-- 15 files changed, 441 insertions(+), 121 deletions(-) create mode 100644 src/java/voldemort/client/ClientInfo.java create mode 100644 src/java/voldemort/store/system/SystemStoreConstants.java diff --git a/src/java/voldemort/VoldemortAdminTool.java b/src/java/voldemort/VoldemortAdminTool.java index 5629691e97..c3ce8353df 100644 --- a/src/java/voldemort/VoldemortAdminTool.java +++ b/src/java/voldemort/VoldemortAdminTool.java @@ -32,6 +32,7 @@ import java.io.PrintStream; import java.io.StringReader; import java.io.StringWriter; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -61,6 +62,7 @@ import voldemort.store.metadata.MetadataStore; import voldemort.store.metadata.MetadataStore.VoldemortState; import voldemort.store.readonly.ReadOnlyStorageConfiguration; +import voldemort.store.system.SystemStoreConstants; import voldemort.utils.ByteArray; import voldemort.utils.ByteUtils; import voldemort.utils.CmdUtils; @@ -792,9 +794,8 @@ private static void executeSetMetadata(Integer nodeId, + adminClient.getAdminClientCluster() .getNodeById(currentNodeId) .getId()); - adminClient.updateRemoteMetadata(currentNodeId, - key, - Versioned.value(value.toString(), updatedVersion)); + adminClient.updateRemoteMetadata(currentNodeId, key, Versioned.value(value.toString(), + updatedVersion)); } } @@ -972,7 +973,7 @@ private static void executeFetchEntries(Integer nodeId, List storeDefinitionList = adminClient.getRemoteStoreDefList(nodeId) .getValue(); - Map storeDefinitionMap = Maps.newHashMap(); + HashMap storeDefinitionMap = Maps.newHashMap(); for(StoreDefinition storeDefinition: storeDefinitionList) { storeDefinitionMap.put(storeDefinition.getName(), storeDefinition); } @@ -986,8 +987,14 @@ private static void executeFetchEntries(Integer nodeId, } List stores = storeNames; if(stores == null) { + // when no stores specified, all user defined store will be fetched, + // but not system stores. stores = Lists.newArrayList(); stores.addAll(storeDefinitionMap.keySet()); + } else { + // add system store to the map so they can be fetched when specified + // explicitly + storeDefinitionMap.putAll(getSystemStoreDef()); } // Pick up all the partitions @@ -1003,6 +1010,7 @@ private static void executeFetchEntries(Integer nodeId, storeDefinition = storeDefinitionMap.get(store); if(null == storeDefinition) { + System.out.println("No store found under the name \'" + store + "\'"); continue; } else { @@ -1031,6 +1039,15 @@ private static void executeFetchEntries(Integer nodeId, } } + private static Map getSystemStoreDef() { + Map sysStoreDef = Maps.newHashMap(); + List storesDefs = SystemStoreConstants.getAllSystemStoreDefs(); + for(StoreDefinition def: storesDefs) { + sysStoreDef.put(def.getName(), def); + } + return sysStoreDef; + } + private static void executeUpdateEntries(Integer nodeId, AdminClient adminClient, List storeNames, diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index f084352af4..ac6ba95287 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -16,13 +16,9 @@ package voldemort.client; -import java.io.File; -import java.io.IOException; import java.io.StringReader; -import java.net.InetAddress; import java.net.URI; import java.net.URISyntaxException; -import java.net.UnknownHostException; import java.util.Collection; import java.util.List; import java.util.Map; @@ -193,8 +189,10 @@ public Store getRawStore(String storeName, for(StoreDefinition d: storeDefs) if(d.getName().equals(storeName)) storeDef = d; - if(storeDef == null) + if(storeDef == null) { + logger.error("Bootstrap - unknown store: " + storeName); throw new BootstrapFailureException("Unknown store '" + storeName + "'."); + } if(logger.isDebugEnabled()) { logger.debug(cluster.toString(true)); @@ -454,27 +452,16 @@ private String jmxId() { * @param clientSequence the client sequence number * @return unique client ID */ - public static UUID generateClientId(String storeName, String contextName, int clientSequence) { + public static UUID generateClientId(ClientInfo clientInfo) { + String contextName = clientInfo.getContext(); + int clientSequence = clientInfo.getClientSequence(); + String newLine = System.getProperty("line.separator"); StringBuilder context = new StringBuilder(contextName == null ? "" : contextName); context.append(0 == clientSequence ? "" : ("." + clientSequence)); - context.append(".").append(storeName); - - try { - InetAddress host = InetAddress.getLocalHost(); - context.append("@").append(host.getHostName()).append(":"); - } catch(UnknownHostException e) { - logger.info("Unable to obtain client hostname."); - logger.info(e.getMessage()); - } - - try { - String currentPath = new File(".").getCanonicalPath(); - context.append(currentPath).append(newLine); - } catch(IOException e) { - logger.info("Unable to obtain client run path."); - logger.info(e.getMessage()); - } + context.append(".").append(clientInfo.getStoreName()); + context.append("@").append(clientInfo.getLocalHostName()).append(":"); + context.append(clientInfo.getDeploymentPath()).append(newLine); if(logger.isDebugEnabled()) { logger.debug(context.toString()); @@ -482,4 +469,5 @@ public static UUID generateClientId(String storeName, String contextName, int cl return UUID.nameUUIDFromBytes(context.toString().getBytes()); } + } diff --git a/src/java/voldemort/client/AsyncMetadataVersionManager.java b/src/java/voldemort/client/AsyncMetadataVersionManager.java index a0ce826e8b..fa0df578c4 100644 --- a/src/java/voldemort/client/AsyncMetadataVersionManager.java +++ b/src/java/voldemort/client/AsyncMetadataVersionManager.java @@ -90,7 +90,7 @@ public void run() { // If version obtained is null, the store is untouched. Continue if(newVersion == null) { - logger.info("Metadata unchanged after creation ..."); + logger.debug("Metadata unchanged after creation ..."); continue; } diff --git a/src/java/voldemort/client/ClientInfo.java b/src/java/voldemort/client/ClientInfo.java new file mode 100644 index 0000000000..08886ad7d0 --- /dev/null +++ b/src/java/voldemort/client/ClientInfo.java @@ -0,0 +1,106 @@ +package voldemort.client; + +import java.io.File; +import java.io.IOException; +import java.io.Serializable; +import java.net.InetAddress; +import java.net.UnknownHostException; + +import org.apache.log4j.Logger; + +public class ClientInfo implements Serializable { + + /** + * + */ + private static final long serialVersionUID = 1L; + + protected static final Logger logger = Logger.getLogger(ClientInfo.class); + + private long bootstrapTime; + private String storeName; + private String context; + private int sequence; + private String localHostName; + private String deploymentPath; + + public ClientInfo(String storeName, String clientContext, int clientSequence, long bootstrapTime) { + this.bootstrapTime = bootstrapTime; + this.storeName = storeName; + this.context = clientContext; + this.sequence = clientSequence; + this.localHostName = createHostName(); + this.deploymentPath = createDeploymentPath(); + } + + private String createDeploymentPath() { + String currentPath = null; + try { + currentPath = new File(".").getCanonicalPath(); + } catch(IOException e) { + logger.warn("Unable to obtain client deployment path due to the following error:"); + logger.warn(e.getMessage()); + } + return currentPath; + } + + private String createHostName() { + String hostName = null; + try { + InetAddress host = InetAddress.getLocalHost(); + hostName = host.getHostName(); + } catch(UnknownHostException e) { + logger.warn("Unable to obtain client hostname due to the following error:"); + logger.warn(e.getMessage()); + } + return hostName; + } + + public void setStoreName(String storeName) { + this.storeName = storeName; + } + + public String getStoreName() { + return storeName; + } + + public void setBootstrapTime(long bootstrapTime) { + this.bootstrapTime = bootstrapTime; + } + + public long getBootstrapTime() { + return bootstrapTime; + } + + public void setContext(String clientContext) { + this.context = clientContext; + } + + public String getContext() { + return context; + } + + public void setClientSequence(int clientSequence) { + this.sequence = clientSequence; + } + + public int getClientSequence() { + return sequence; + } + + public void setDeploymentPath(String deploymentPath) { + this.deploymentPath = deploymentPath; + } + + public String getDeploymentPath() { + return deploymentPath; + } + + public void setLocalHostName(String localHostName) { + this.localHostName = localHostName; + } + + public String getLocalHostName() { + return localHostName; + } +} diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index 436fcfd292..37737c66e1 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -19,8 +19,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.UUID; +import java.util.Map.Entry; import java.util.concurrent.Callable; import org.apache.log4j.Logger; @@ -35,6 +35,7 @@ import voldemort.store.InvalidMetadataException; import voldemort.store.Store; import voldemort.store.StoreCapabilityType; +import voldemort.store.system.SystemStoreConstants; import voldemort.utils.JmxUtils; import voldemort.utils.Utils; import voldemort.versioning.InconsistencyResolver; @@ -70,13 +71,7 @@ public class DefaultStoreClient implements StoreClient { private final UUID clientId; private final Map sysStoreMap; private AsyncMetadataVersionManager asyncCheckMetadata; - - // Enumerate all the system stores - private final String METADATA_VERSION_STORE = "voldsys$_metadata_version"; - private final String CLIENT_REGISTRY_STORE = "voldsys$_client_registry"; - private final String STORE_DEFINITION_STORE = "voldsys$_client_store_definition"; - private final String[] systemStoreNames = { METADATA_VERSION_STORE, CLIENT_REGISTRY_STORE, - STORE_DEFINITION_STORE }; + private ClientInfo clientInfo; public DefaultStoreClient(String storeName, InconsistencyResolver> resolver, @@ -98,9 +93,11 @@ public DefaultStoreClient(String storeName, this.resolver = resolver; this.storeFactory = Utils.notNull(storeFactory); this.metadataRefreshAttempts = maxMetadataRefreshAttempts; - this.clientId = AbstractStoreClientFactory.generateClientId(storeName, - clientContext, - clientSequence); + this.clientInfo = new ClientInfo(storeName, + clientContext, + clientSequence, + System.currentTimeMillis()); + this.clientId = AbstractStoreClientFactory.generateClientId(clientInfo); this.config = config; // Registering self to be able to bootstrap client dynamically via JMX @@ -113,42 +110,64 @@ public DefaultStoreClient(String storeName, bootStrap(); // Initialize all the system stores - sysStoreMap = new HashMap(); - initializeSystemStores(); + sysStoreMap = createSystemStores(); // Initialize the background thread for checking metadata version if(config != null) { - SystemStore versionStore = this.sysStoreMap.get(METADATA_VERSION_STORE); - if(versionStore == null) - logger.info("Metadata version system store not found. Cannot run Metadata version check thread."); - else { - Callable bootstrapCallback = new Callable() { - - public Void call() throws Exception { - bootStrap(); - return null; - } - }; - - asyncCheckMetadata = new AsyncMetadataVersionManager(versionStore, - config.getAsyncCheckMetadataInterval(), - bootstrapCallback); - logger.info("Metadata version check thread started. Frequency = Every " - + config.getAsyncCheckMetadataInterval() + " ms"); - } + asyncCheckMetadata = createMetadataChecker(); } + registerClient(); logger.info("Voldemort client created: clientContext=" + clientContext + " clientSequence=" + clientSequence + " clientId=" + clientId.toString()); } - public void initializeSystemStores() { - for(String storeName: systemStoreNames) { - SystemStore sysStore = new SystemStore(storeName, + private void registerClient() { + String name = SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(); + SystemStore clientRegistry = sysStoreMap.get(name); + if(null != clientRegistry) { + try { + clientRegistry.putSysStore(clientId.toString(), clientInfo); + } catch(Exception e) { + logger.warn("Unable to register with the cluster due to the following error:", e); + } + } else { + logger.warn(name + "not found. Unable to registry with voldemort cluster."); + } + } + + private Map createSystemStores() { + Map systemStores = new HashMap(); + for(SystemStoreConstants.SystemStoreName storeName: SystemStoreConstants.SystemStoreName.values()) { + SystemStore sysStore = new SystemStore(storeName.name(), config.getBootstrapUrls(), config.getClientZoneId()); - this.sysStoreMap.put(storeName, sysStore); + systemStores.put(storeName.name(), sysStore); + } + return systemStores; + } + + private AsyncMetadataVersionManager createMetadataChecker() { + AsyncMetadataVersionManager asyncCheckMetadata = null; + SystemStore versionStore = this.sysStoreMap.get(SystemStoreConstants.SystemStoreName.voldsys$_metadata_version.name()); + if(versionStore == null) + logger.warn("Metadata version system store not found. Cannot run Metadata version check thread."); + else { + Callable bootstrapCallback = new Callable() { + + public Void call() throws Exception { + bootStrap(); + return null; + } + }; + + asyncCheckMetadata = new AsyncMetadataVersionManager(versionStore, + config.getAsyncCheckMetadataInterval(), + bootstrapCallback); + logger.info("Metadata version check thread started. Frequency = Every " + + config.getAsyncCheckMetadataInterval() + " ms"); } + return asyncCheckMetadata; } @JmxOperation(description = "bootstrap metadata from the cluster.") diff --git a/src/java/voldemort/client/SocketStoreClientFactory.java b/src/java/voldemort/client/SocketStoreClientFactory.java index ef5de8b224..a9b6b59808 100644 --- a/src/java/voldemort/client/SocketStoreClientFactory.java +++ b/src/java/voldemort/client/SocketStoreClientFactory.java @@ -32,12 +32,12 @@ import voldemort.cluster.failuredetector.FailureDetectorConfig; import voldemort.cluster.failuredetector.FailureDetectorListener; import voldemort.server.RequestRoutingType; -import voldemort.server.SystemStoreConstants; import voldemort.store.Store; import voldemort.store.metadata.MetadataStore; import voldemort.store.socket.SocketDestination; import voldemort.store.socket.SocketStoreFactory; import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; +import voldemort.store.system.SystemStoreConstants; import voldemort.utils.ByteArray; import voldemort.utils.JmxUtils; import voldemort.versioning.InconsistencyResolver; diff --git a/src/java/voldemort/client/SystemStore.java b/src/java/voldemort/client/SystemStore.java index cf1289ebd1..0e68529cf4 100644 --- a/src/java/voldemort/client/SystemStore.java +++ b/src/java/voldemort/client/SystemStore.java @@ -6,8 +6,8 @@ import org.apache.log4j.Logger; import voldemort.VoldemortException; -import voldemort.server.SystemStoreConstants; import voldemort.store.Store; +import voldemort.store.system.SystemStoreConstants; import voldemort.versioning.InconsistentDataException; import voldemort.versioning.VectorClock; import voldemort.versioning.Versioned; @@ -39,47 +39,67 @@ public SystemStore(String storeName, String[] bootstrapUrls, int clientZoneID) { this.sysStore = this.systemStoreFactory.getSystemStore(this.storeName); } - public void putSysStore(K key, V value) throws VoldemortException { - logger.debug("Invoking Put for key : " + key + " on store name : " + this.storeName); - Versioned versioned = getSysStore(key); - if(versioned == null) - versioned = Versioned.value(value, new VectorClock()); - else - versioned.setObject(value); - this.sysStore.put(key, versioned, null); + public void putSysStore(K key, V value) { + try { + logger.debug("Invoking Put for key : " + key + " on store name : " + this.storeName); + Versioned versioned = getSysStore(key); + if(versioned == null) + versioned = Versioned.value(value, new VectorClock()); + else + versioned.setObject(value); + this.sysStore.put(key, versioned, null); + } catch(Exception e) { + logger.info("Exception caught during putSysStore:"); + e.printStackTrace(); + } } - public void putSysStore(K key, Versioned value) throws VoldemortException { - logger.debug("Invoking Put for key : " + key + " on store name : " + this.storeName); - this.sysStore.put(key, value, null); + public void putSysStore(K key, Versioned value) { + try { + logger.debug("Invoking Put for key : " + key + " on store name : " + this.storeName); + this.sysStore.put(key, value, null); + } catch(Exception e) { + logger.info("Exception caught during putSysStore:"); + e.printStackTrace(); + } } - public Versioned getSysStore(K key) throws VoldemortException { + public Versioned getSysStore(K key) { logger.debug("Invoking Get for key : " + key + " on store name : " + this.storeName); Versioned versioned = null; - List> items = this.sysStore.get(key, null); - if(items.size() == 1) - versioned = items.get(0); - else if(items.size() > 1) - throw new InconsistentDataException("Unresolved versions returned from get(" + key - + ") = " + items, items); - if(versioned != null) - logger.debug("Value for key : " + key + " = " + versioned.getValue() - + " on store name : " + this.storeName); - else - logger.debug("Got null value"); + try { + List> items = this.sysStore.get(key, null); + if(items.size() == 1) + versioned = items.get(0); + else if(items.size() > 1) + throw new InconsistentDataException("Unresolved versions returned from get(" + key + + ") = " + items, items); + if(versioned != null) + logger.debug("Value for key : " + key + " = " + versioned.getValue() + + " on store name : " + this.storeName); + else + logger.debug("Got null value"); + } catch(Exception e) { + logger.info("Exception caught during getSysStore:"); + e.printStackTrace(); + } return versioned; } - public V getValueSysStore(K key) throws VoldemortException { - logger.debug("Invoking Get for key : " + key + " on store name : " + this.storeName); - Versioned versioned = getSysStore(key); - if(versioned != null) { - logger.debug("Value for key : " + key + " = " + versioned.getValue() - + " on store name : " + this.storeName); - return versioned.getValue(); + public V getValueSysStore(K key) { + V value = null; + try { + logger.debug("Invoking Get for key : " + key + " on store name : " + this.storeName); + Versioned versioned = getSysStore(key); + if(versioned != null) { + logger.debug("Value for key : " + key + " = " + versioned.getValue() + + " on store name : " + this.storeName); + value = versioned.getValue(); + } + } catch(Exception e) { + logger.info("Exception caught during getSysStore:"); + e.printStackTrace(); } - return null; + return value; } - } diff --git a/src/java/voldemort/client/protocol/admin/AdminClient.java b/src/java/voldemort/client/protocol/admin/AdminClient.java index 94203296a5..e4059ce061 100644 --- a/src/java/voldemort/client/protocol/admin/AdminClient.java +++ b/src/java/voldemort/client/protocol/admin/AdminClient.java @@ -31,8 +31,8 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Set; +import java.util.Map.Entry; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; @@ -49,8 +49,8 @@ import voldemort.client.protocol.VoldemortFilter; import voldemort.client.protocol.pb.ProtoUtils; import voldemort.client.protocol.pb.VAdminProto; -import voldemort.client.protocol.pb.VAdminProto.RebalancePartitionInfoMap; import voldemort.client.protocol.pb.VProto; +import voldemort.client.protocol.pb.VAdminProto.RebalancePartitionInfoMap; import voldemort.client.protocol.pb.VProto.RequestType; import voldemort.client.rebalance.RebalancePartitionsInfo; import voldemort.cluster.Cluster; @@ -72,6 +72,7 @@ import voldemort.store.slop.Slop; import voldemort.store.slop.Slop.Operation; import voldemort.store.socket.SocketDestination; +import voldemort.store.system.SystemStoreConstants; import voldemort.store.views.ViewStorageConfiguration; import voldemort.utils.ByteArray; import voldemort.utils.ByteUtils; @@ -962,9 +963,9 @@ public int rebalanceNode(RebalancePartitionsInfo stealInfo) { private HashMap> getReplicaToPartitionMap(int nodeId, String storeName, List partitions) { - - StoreDefinition def = RebalanceUtils.getStoreDefinitionWithName(getRemoteStoreDefList(nodeId).getValue(), - storeName); + List allStoreDefs = getRemoteStoreDefList(nodeId).getValue(); + allStoreDefs.addAll(SystemStoreConstants.getAllSystemStoreDefs()); + StoreDefinition def = RebalanceUtils.getStoreDefinitionWithName(allStoreDefs, storeName); HashMap> replicaToPartitionList = Maps.newHashMap(); for(int replicaNum = 0; replicaNum < def.getReplicationFactor(); replicaNum++) { replicaToPartitionList.put(replicaNum, partitions); diff --git a/src/java/voldemort/server/protocol/admin/FetchStreamRequestHandler.java b/src/java/voldemort/server/protocol/admin/FetchStreamRequestHandler.java index b73eda56ca..177fb5db95 100644 --- a/src/java/voldemort/server/protocol/admin/FetchStreamRequestHandler.java +++ b/src/java/voldemort/server/protocol/admin/FetchStreamRequestHandler.java @@ -23,6 +23,7 @@ import voldemort.store.metadata.MetadataStore; import voldemort.store.stats.StreamStats; import voldemort.store.stats.StreamStats.Handle; +import voldemort.store.system.SystemStoreConstants; import voldemort.utils.ByteArray; import voldemort.utils.ClosableIterator; import voldemort.utils.EventThrottler; @@ -81,7 +82,7 @@ protected FetchStreamRequestHandler(VAdminProto.FetchPartitionEntriesRequest req this.handle = stats.makeHandle(operation, replicaToPartitionList); this.storageEngine = AdminServiceRequestHandler.getStorageEngine(storeRepository, request.getStore()); - this.storeDef = metadataStore.getStoreDef(request.getStore()); + this.storeDef = getStoreDef(request.getStore(), metadataStore); if(request.hasInitialCluster()) { this.initialCluster = new ClusterMapper().readCluster(new StringReader(request.getInitialCluster())); } else { @@ -105,6 +106,16 @@ protected FetchStreamRequestHandler(VAdminProto.FetchPartitionEntriesRequest req } } + private StoreDefinition getStoreDef(String store, MetadataStore metadataStore) { + StoreDefinition def = null; + if(SystemStoreConstants.isSystemStore(store)) { + def = SystemStoreConstants.getSystemStoreDef(store); + } else { + def = metadataStore.getStoreDef(request.getStore()); + } + return def; + } + public final StreamRequestDirection getDirection() { return StreamRequestDirection.WRITING; } diff --git a/src/java/voldemort/server/storage/StorageService.java b/src/java/voldemort/server/storage/StorageService.java index 22e85079a2..eb677be6d3 100644 --- a/src/java/voldemort/server/storage/StorageService.java +++ b/src/java/voldemort/server/storage/StorageService.java @@ -18,7 +18,6 @@ import static voldemort.cluster.failuredetector.FailureDetectorUtils.create; -import java.io.StringReader; import java.lang.management.ManagementFactory; import java.util.ArrayList; import java.util.Calendar; @@ -55,7 +54,6 @@ import voldemort.server.RequestRoutingType; import voldemort.server.ServiceType; import voldemort.server.StoreRepository; -import voldemort.server.SystemStoreConstants; import voldemort.server.VoldemortConfig; import voldemort.server.scheduler.DataCleanupJob; import voldemort.server.scheduler.SchedulerService; @@ -84,6 +82,7 @@ import voldemort.store.stats.StatTrackingStore; import voldemort.store.stats.StoreStats; import voldemort.store.stats.StoreStatsJmx; +import voldemort.store.system.SystemStoreConstants; import voldemort.store.versioned.InconsistencyResolvingStore; import voldemort.store.views.ViewStorageConfiguration; import voldemort.store.views.ViewStorageEngine; @@ -100,7 +99,6 @@ import voldemort.versioning.VectorClock; import voldemort.versioning.VectorClockInconsistencyResolver; import voldemort.versioning.Versioned; -import voldemort.xml.StoreDefinitionsMapper; /** * The service responsible for managing all storage types @@ -209,7 +207,7 @@ private void initSystemStorageConfig() { } private void initSystemStores() { - List storesDefs = (new StoreDefinitionsMapper()).readStoreList(new StringReader(SystemStoreConstants.SYSTEM_STORE_SCHEMA)); + List storesDefs = SystemStoreConstants.getAllSystemStoreDefs(); // TODO: replication factor can't now be determined unless the // cluster.xml is made available to the server at runtime. So we need to diff --git a/src/java/voldemort/store/StoreDefinition.java b/src/java/voldemort/store/StoreDefinition.java index 52d4dda7f0..15944f327a 100644 --- a/src/java/voldemort/store/StoreDefinition.java +++ b/src/java/voldemort/store/StoreDefinition.java @@ -22,8 +22,8 @@ import voldemort.client.RoutingTier; import voldemort.serialization.SerializerDefinition; -import voldemort.server.SystemStoreConstants; import voldemort.store.slop.strategy.HintedHandoffStrategyType; +import voldemort.store.system.SystemStoreConstants; import voldemort.utils.Utils; import com.google.common.base.Objects; diff --git a/src/java/voldemort/store/metadata/MetadataStore.java b/src/java/voldemort/store/metadata/MetadataStore.java index 6d692ffe3f..e38aa7a785 100644 --- a/src/java/voldemort/store/metadata/MetadataStore.java +++ b/src/java/voldemort/store/metadata/MetadataStore.java @@ -41,7 +41,6 @@ import voldemort.routing.RouteToAllStrategy; import voldemort.routing.RoutingStrategy; import voldemort.routing.RoutingStrategyFactory; -import voldemort.server.SystemStoreConstants; import voldemort.server.rebalance.RebalancerState; import voldemort.store.StorageEngine; import voldemort.store.Store; @@ -49,6 +48,7 @@ import voldemort.store.StoreDefinition; import voldemort.store.StoreUtils; import voldemort.store.configuration.ConfigurationStorageEngine; +import voldemort.store.system.SystemStoreConstants; import voldemort.utils.ByteArray; import voldemort.utils.ByteUtils; import voldemort.utils.ClosableIterator; diff --git a/src/java/voldemort/store/system/SystemStoreConstants.java b/src/java/voldemort/store/system/SystemStoreConstants.java new file mode 100644 index 0000000000..32a1d2e5be --- /dev/null +++ b/src/java/voldemort/store/system/SystemStoreConstants.java @@ -0,0 +1,104 @@ +package voldemort.store.system; + +import java.io.StringReader; +import java.util.List; + +import voldemort.store.StoreDefinition; +import voldemort.xml.StoreDefinitionsMapper; + +/** + * The various system stores + */ +public class SystemStoreConstants { + + public static final String NAME_PREFIX = "voldsys$_"; + + public static enum SystemStoreName { + voldsys$_client_registry, + voldsys$_client_store_definition, + voldsys$_metadata_version; + } + + public static final String SYSTEM_STORE_SCHEMA = "" + + " " + + " voldsys$_client_registry" + + " zone-routing" + + " proximity-handoff" + + " memory" + + " client" + + " 4" + + " " + + " 2" + + " 2" + + " " + + " 1" + + " 1" + + " " + + " string" + + " utf8" + + " " + + " " + // + + // " avro-specific" + // + + // " java=voldemort.client.ClientInfo" + + " java-serialization" + + " " + + " 7" + + " " + + + " " + + " voldsys$_client_store_definition" + + " zone-routing" + + " proximity-handoff" + + " memory" + + " client" + + " 1" + + " 1" + + " 1" + + " " + + " string" + + " " + + " " + + " string" + + " " + + " 7" + + " " + + + " " + + " voldsys$_metadata_version" + + " local-pref-all-routing" + + " proximity-handoff" + + " memory" + + " client" + + " 1" + + " 1" + + " 1" + + " " + + " string" + + " " + + " " + + " java-serialization" + + " " + " " + + + ""; + + public static boolean isSystemStore(String storeName) { + return (null == storeName ? false : storeName.startsWith(NAME_PREFIX)); + } + + public static List getAllSystemStoreDefs() { + return (new StoreDefinitionsMapper()).readStoreList(new StringReader(SystemStoreConstants.SYSTEM_STORE_SCHEMA)); + } + + public static StoreDefinition getSystemStoreDef(String name) { + StoreDefinition storeDef = null; + List allDefs = getAllSystemStoreDefs(); + for(StoreDefinition def: allDefs) { + if(name.equals(def.getName())) { + storeDef = def; + } + } + return storeDef; + } +} \ No newline at end of file diff --git a/src/java/voldemort/xml/StoreDefinitionsMapper.java b/src/java/voldemort/xml/StoreDefinitionsMapper.java index d1d4948b24..4fe6d9983d 100644 --- a/src/java/voldemort/xml/StoreDefinitionsMapper.java +++ b/src/java/voldemort/xml/StoreDefinitionsMapper.java @@ -46,11 +46,11 @@ import voldemort.routing.RoutingStrategyType; import voldemort.serialization.Compression; import voldemort.serialization.SerializerDefinition; -import voldemort.server.SystemStoreConstants; import voldemort.store.StoreDefinition; import voldemort.store.StoreDefinitionBuilder; import voldemort.store.StoreUtils; import voldemort.store.slop.strategy.HintedHandoffStrategyType; +import voldemort.store.system.SystemStoreConstants; import voldemort.store.views.ViewStorageConfiguration; import voldemort.utils.Utils; diff --git a/test/unit/voldemort/client/DefaultSocketStoreClientTest.java b/test/unit/voldemort/client/DefaultSocketStoreClientTest.java index 07b8db82f5..46945a76c2 100644 --- a/test/unit/voldemort/client/DefaultSocketStoreClientTest.java +++ b/test/unit/voldemort/client/DefaultSocketStoreClientTest.java @@ -4,33 +4,74 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.fail; +import java.io.File; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; + import org.junit.Before; import org.junit.Test; -import voldemort.utils.SystemTime; +import voldemort.ServerTestUtils; +import voldemort.TestUtils; +import voldemort.client.protocol.admin.AdminClient; +import voldemort.cluster.Cluster; +import voldemort.server.VoldemortServer; +import voldemort.store.StoreDefinition; +import voldemort.store.socket.SocketStoreFactory; +import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; import voldemort.utils.Time; import voldemort.versioning.ObsoleteVersionException; import voldemort.versioning.VectorClock; import voldemort.versioning.Versioned; +import voldemort.xml.StoreDefinitionsMapper; public class DefaultSocketStoreClientTest { + private static String testStoreName = "test-replication-memory"; + private static String storesXmlfile = "test/common/voldemort/config/stores.xml"; + private SocketStoreFactory socketStoreFactory = new ClientRequestExecutorPool(2, + 10000, + 100000, + 32 * 1024); + private static AtomicBoolean running = new AtomicBoolean(true); + private List storeDefs; + private VoldemortServer[] servers; + private Cluster cluster; + private AdminClient adminClient; + + public static String socketUrl = "tcp://localhost:6667"; protected StoreClient client; protected int nodeId; protected Time time; @Before public void setUp() throws Exception { - String socketUrl = "tcp://localhost:6667"; - String storeName = "test"; - ClientConfig clientConfig = new ClientConfig().setMaxThreads(4) - .setMaxTotalConnections(4) - .setMaxConnectionsPerNode(4) - .setBootstrapUrls(socketUrl); - SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); - this.client = socketFactory.getStoreClient(storeName); - this.nodeId = 0; - this.time = SystemTime.INSTANCE; + cluster = ServerTestUtils.getLocalCluster(2, new int[][] { { 0, 1, 2, 3 }, { 4, 5, 6, 7 } }); + + servers = new VoldemortServer[2]; + storeDefs = new StoreDefinitionsMapper().readStoreList(new File(storesXmlfile)); + + servers[0] = ServerTestUtils.startVoldemortServer(socketStoreFactory, + ServerTestUtils.createServerConfig(useNio, + 0, + TestUtils.createTempDir() + .getAbsolutePath(), + null, + storesXmlfile, + new Properties()), + cluster); + servers[1] = ServerTestUtils.startVoldemortServer(socketStoreFactory, + ServerTestUtils.createServerConfig(useNio, + 1, + TestUtils.createTempDir() + .getAbsolutePath(), + null, + storesXmlfile, + new Properties()), + cluster); + + adminClient = ServerTestUtils.getAdminClient(cluster); } @Test @@ -60,4 +101,19 @@ public void test() { assertNull("After a successful delete(k), get(k) should return null.", client.get("k")); } + @Test + public void testClientRegistryHappyPath() { + ClientConfig clientConfig = new ClientConfig().setMaxThreads(4) + .setMaxTotalConnections(4) + .setMaxConnectionsPerNode(4) + .setBootstrapUrls(socketUrl) + .setClientContextName("testClientRegistryHappyPath"); + SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); + this.client = socketFactory.getStoreClient(testStoreName); + client.put("k", "v"); + adminClient.fetchEntries(0, testStoreName, null, null, false); + adminClient.fetchEntries(1, testStoreName, null, null, false); + // TODO: verify that the values in registry are correct. + } + } From d1b9bb82115addcddfc60c000064162085025cf8 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Mon, 18 Jun 2012 13:10:58 -0700 Subject: [PATCH 049/209] Adding jmxId to the Mbean name for failureDetector and storefactory --- .../voldemort/client/AbstractStoreClientFactory.java | 9 +++++++-- .../voldemort/client/SocketStoreClientFactory.java | 11 ++++++++--- 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index fb82936c50..9a1463c290 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -274,8 +274,13 @@ public FailureDetector getFailureDetector() { synchronized(this) { // second check: avoids double initialization result = failureDetector; - if(result == null) + if(result == null) { failureDetector = result = initFailureDetector(config, cluster.getNodes()); + JmxUtils.registerMbean(failureDetector, + JmxUtils.createObjectName(JmxUtils.getPackageName(failureDetector.getClass()), + JmxUtils.getClassName(failureDetector.getClass()) + + jmxId())); + } } } @@ -400,7 +405,7 @@ public void close() { } /* Give a unique id to avoid jmx clashes */ - private String jmxId() { + public String jmxId() { return jmxId == 0 ? "" : Integer.toString(jmxId); } diff --git a/src/java/voldemort/client/SocketStoreClientFactory.java b/src/java/voldemort/client/SocketStoreClientFactory.java index e86ae31322..09591eedb0 100644 --- a/src/java/voldemort/client/SocketStoreClientFactory.java +++ b/src/java/voldemort/client/SocketStoreClientFactory.java @@ -70,7 +70,10 @@ public SocketStoreClientFactory(ClientConfig config) { config.getSocketBufferSize(), config.getSocketKeepAlive()); if(config.isJmxEnabled()) - JmxUtils.registerMbean(storeFactory, JmxUtils.createObjectName(storeFactory.getClass())); + JmxUtils.registerMbean(storeFactory, + JmxUtils.createObjectName(JmxUtils.getPackageName(storeFactory.getClass()), + JmxUtils.getClassName(storeFactory.getClass()) + + jmxId())); } @Override @@ -87,7 +90,8 @@ public StoreClient call() throws Exception { return getParentStoreClient(storeName, resolver); } - private StoreClient getParentStoreClient(String storeName, InconsistencyResolver> resolver) { + private StoreClient getParentStoreClient(String storeName, + InconsistencyResolver> resolver) { return super.getStoreClient(storeName, resolver); } @@ -96,7 +100,8 @@ protected List> getRemoteMetadata(String key, URI url) { try { return super.getRemoteMetadata(key, url); } catch(VoldemortException e) { - // Fix SNA-4227: When an error occurs during bootstrap, close the socket + // Fix SNA-4227: When an error occurs during bootstrap, close the + // socket SocketDestination destination = new SocketDestination(url.getHost(), url.getPort(), getRequestFormatType()); From 3f1a91e7d71e976d63f06a3a32d19fb33a972255 Mon Sep 17 00:00:00 2001 From: shingon Date: Tue, 15 May 2012 15:23:54 +0900 Subject: [PATCH 050/209] fix wrong file name in generate_cluster_xml.py --- bin/generate_cluster_xml.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/generate_cluster_xml.py b/bin/generate_cluster_xml.py index 09baf35894..3cd56644d7 100644 --- a/bin/generate_cluster_xml.py +++ b/bin/generate_cluster_xml.py @@ -2,7 +2,7 @@ import random if len(sys.argv) != 3: - print >> sys.stderr, "USAGE: python generate_partitions.py " + print >> sys.stderr, "USAGE: python generate_cluster_xml.py " sys.exit() FORMAT_WIDTH = 10 From 1ec9694f2d9394ae96e48e796eaf4e15f3357abe Mon Sep 17 00:00:00 2001 From: shingon Date: Wed, 16 May 2012 16:11:54 +0900 Subject: [PATCH 051/209] fix a typo. boostrap -> bootstrap --- src/java/voldemort/client/AbstractStoreClientFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index fb82936c50..ab0c7d998e 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -305,7 +305,7 @@ public String bootstrapMetadataWithRetries(String key, URI[] urls) { } } - throw new BootstrapFailureException("No available boostrap servers found!"); + throw new BootstrapFailureException("No available bootstrap servers found!"); } public String bootstrapMetadataWithRetries(String key) { From 0d09d22f158fe5cf464591d386c75a6d20933f64 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Thu, 31 May 2012 18:03:02 -0700 Subject: [PATCH 052/209] Added 99th wait time for connection for ClientRequestExecutorPool.java --- .../clientrequest/ClientRequestExecutorPool.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java index ce914fde4f..2d83b0fe16 100644 --- a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java +++ b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java @@ -30,6 +30,7 @@ import voldemort.store.socket.SocketDestination; import voldemort.store.socket.SocketStore; import voldemort.store.socket.SocketStoreFactory; +import voldemort.store.stats.Histogram; import voldemort.utils.Time; import voldemort.utils.Utils; import voldemort.utils.pool.KeyedResourcePool; @@ -53,6 +54,7 @@ public class ClientRequestExecutorPool implements SocketStoreFactory { private final AtomicInteger checkouts; private final AtomicLong waitNs; private final AtomicLong avgWaitNs; + private final Histogram histogramWaitMs; private final KeyedResourcePool pool; private final ClientRequestExecutorFactory factory; @@ -76,6 +78,7 @@ public ClientRequestExecutorPool(int selectors, this.checkouts = new AtomicInteger(0); this.waitNs = new AtomicLong(0); this.avgWaitNs = new AtomicLong(0); + this.histogramWaitMs = new HistogramArray(10000, 1); } public ClientRequestExecutorPool(int maxConnectionsPerNode, @@ -130,13 +133,14 @@ private void updateStats(long checkoutTimeNs) { long wait = waitNs.getAndAdd(checkoutTimeNs); int count = checkouts.getAndIncrement(); - // reset reporting inverval if we have used up the current interval + // reset reporting interval if we have used up the current interval int interval = this.monitoringInterval.get(); if(count % interval == interval - 1) { // harmless race condition: waitNs.set(0); checkouts.set(0); avgWaitNs.set(wait / count); + histogramWaitMs.insert(checkoutTimeNs / Time.NS_PER_MS); } } @@ -193,6 +197,11 @@ public double getAvgWaitTimeMs() { return this.avgWaitNs.doubleValue() / Time.NS_PER_MS; } + @JmxGetter(name = "99thWaitTimeMs", description = "The 99th percentile ms of wait time to acquire a connection.") + public double get99thWaitTimeMs() { + return this.histogramWaitMs.getQuantile(0.99); + } + @JmxSetter(name = "monitoringInterval", description = "The number of checkouts over which performance statistics are calculated.") public void setMonitoringInterval(int count) { if(count <= 0) From 97bfa6751eba8afb13bfc801afa47d2b6bd9af1d Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Fri, 1 Jun 2012 14:18:49 -0700 Subject: [PATCH 053/209] Fixed bug and verified monitoring feature of q99th wait time --- .../socket/clientrequest/ClientRequestExecutorPool.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java index 2d83b0fe16..cebcb720f0 100644 --- a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java +++ b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java @@ -54,7 +54,7 @@ public class ClientRequestExecutorPool implements SocketStoreFactory { private final AtomicInteger checkouts; private final AtomicLong waitNs; private final AtomicLong avgWaitNs; - private final Histogram histogramWaitMs; + private final Histogram histogramWaitNs; private final KeyedResourcePool pool; private final ClientRequestExecutorFactory factory; @@ -78,7 +78,7 @@ public ClientRequestExecutorPool(int selectors, this.checkouts = new AtomicInteger(0); this.waitNs = new AtomicLong(0); this.avgWaitNs = new AtomicLong(0); - this.histogramWaitMs = new HistogramArray(10000, 1); + this.histogramWaitNs = new Histogram(10000, 1000); } public ClientRequestExecutorPool(int maxConnectionsPerNode, @@ -140,7 +140,7 @@ private void updateStats(long checkoutTimeNs) { waitNs.set(0); checkouts.set(0); avgWaitNs.set(wait / count); - histogramWaitMs.insert(checkoutTimeNs / Time.NS_PER_MS); + histogramWaitNs.insert(checkoutTimeNs); } } @@ -199,7 +199,7 @@ public double getAvgWaitTimeMs() { @JmxGetter(name = "99thWaitTimeMs", description = "The 99th percentile ms of wait time to acquire a connection.") public double get99thWaitTimeMs() { - return this.histogramWaitMs.getQuantile(0.99); + return (double) (this.histogramWaitNs.getQuantile(0.99)) / Time.NS_PER_MS; } @JmxSetter(name = "monitoringInterval", description = "The number of checkouts over which performance statistics are calculated.") From fcf7ed12d1c6a757c2260a433d0a4ad92dc40189 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Wed, 6 Jun 2012 14:38:45 -0700 Subject: [PATCH 054/209] Added per socket monitoring for client executor pool. These monitoring reveals stats including averag, 50th, 99th wait ms for socket checkout; number of checkout in an interval etc --- .../client/SocketStoreClientFactory.java | 12 +- .../ClientRequestExecutorFactory.java | 12 + .../ClientRequestExecutorPool.java | 86 ++----- .../store/stats/ClientSocketStats.java | 230 ++++++++++++++++++ .../store/stats/ClientSocketStatsJmx.java | 101 ++++++++ .../socket/ClientRequestExecutorPoolTest.java | 13 +- .../store/stats/ClientSocketStatsTest.java | 174 +++++++++++++ 7 files changed, 549 insertions(+), 79 deletions(-) create mode 100644 src/java/voldemort/store/stats/ClientSocketStats.java create mode 100644 src/java/voldemort/store/stats/ClientSocketStatsJmx.java create mode 100644 test/unit/voldemort/store/stats/ClientSocketStatsTest.java diff --git a/src/java/voldemort/client/SocketStoreClientFactory.java b/src/java/voldemort/client/SocketStoreClientFactory.java index e86ae31322..8da1bd7135 100644 --- a/src/java/voldemort/client/SocketStoreClientFactory.java +++ b/src/java/voldemort/client/SocketStoreClientFactory.java @@ -38,7 +38,6 @@ import voldemort.store.socket.SocketStoreFactory; import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; import voldemort.utils.ByteArray; -import voldemort.utils.JmxUtils; import voldemort.versioning.InconsistencyResolver; import voldemort.versioning.Versioned; @@ -69,8 +68,9 @@ public SocketStoreClientFactory(ClientConfig config) { config.getSocketTimeout(TimeUnit.MILLISECONDS), config.getSocketBufferSize(), config.getSocketKeepAlive()); - if(config.isJmxEnabled()) - JmxUtils.registerMbean(storeFactory, JmxUtils.createObjectName(storeFactory.getClass())); + if(config.isJmxEnabled()) { + ((ClientRequestExecutorPool) storeFactory).registerJmx(); + } } @Override @@ -87,7 +87,8 @@ public StoreClient call() throws Exception { return getParentStoreClient(storeName, resolver); } - private StoreClient getParentStoreClient(String storeName, InconsistencyResolver> resolver) { + private StoreClient getParentStoreClient(String storeName, + InconsistencyResolver> resolver) { return super.getStoreClient(storeName, resolver); } @@ -96,7 +97,8 @@ protected List> getRemoteMetadata(String key, URI url) { try { return super.getRemoteMetadata(key, url); } catch(VoldemortException e) { - // Fix SNA-4227: When an error occurs during bootstrap, close the socket + // Fix SNA-4227: When an error occurs during bootstrap, close the + // socket SocketDestination destination = new SocketDestination(url.getHost(), url.getPort(), getRequestFormatType()); diff --git a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorFactory.java b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorFactory.java index ba2b68a25d..74f33df12b 100644 --- a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorFactory.java +++ b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorFactory.java @@ -37,6 +37,7 @@ import org.apache.log4j.Logger; import voldemort.store.socket.SocketDestination; +import voldemort.store.stats.ClientSocketStats; import voldemort.utils.DaemonThreadFactory; import voldemort.utils.SelectorManager; import voldemort.utils.Time; @@ -61,6 +62,7 @@ public class ClientRequestExecutorFactory implements private final AtomicInteger counter = new AtomicInteger(); private final Map lastClosedTimestamps; private final Logger logger = Logger.getLogger(getClass()); + private ClientSocketStats stats; public ClientRequestExecutorFactory(int selectors, int connectTimeoutMs, @@ -94,6 +96,9 @@ public void destroy(SocketDestination dest, ClientRequestExecutor clientRequestE throws Exception { clientRequestExecutor.close(); int numDestroyed = destroyed.incrementAndGet(); + if(stats != null) { + stats.connectionDestroy(dest); + } if(logger.isDebugEnabled()) logger.debug("Destroyed socket " + numDestroyed + " connection to " + dest.getHost() @@ -108,6 +113,9 @@ public void destroy(SocketDestination dest, ClientRequestExecutor clientRequestE public ClientRequestExecutor create(SocketDestination dest) throws Exception { int numCreated = created.incrementAndGet(); + if(stats != null) { + stats.connectionCreate(dest); + } if(logger.isDebugEnabled()) logger.debug("Creating socket " + numCreated + " for " + dest.getHost() + ":" @@ -426,4 +434,8 @@ public void setLastClosedTimestamp(SocketDestination socketDestination) { lastClosedTimestamps.put(socketDestination, System.nanoTime()); } + public void setStats(ClientSocketStats stats) { + this.stats = stats; + } + } diff --git a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java index cebcb720f0..6cc35dacb6 100644 --- a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java +++ b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java @@ -17,20 +17,17 @@ package voldemort.store.socket.clientrequest; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; import voldemort.VoldemortException; -import voldemort.annotations.jmx.JmxGetter; -import voldemort.annotations.jmx.JmxManaged; -import voldemort.annotations.jmx.JmxSetter; import voldemort.client.protocol.RequestFormatType; import voldemort.server.RequestRoutingType; import voldemort.store.UnreachableStoreException; import voldemort.store.socket.SocketDestination; import voldemort.store.socket.SocketStore; import voldemort.store.socket.SocketStoreFactory; -import voldemort.store.stats.Histogram; +import voldemort.store.stats.ClientSocketStats; +import voldemort.store.stats.ClientSocketStatsJmx; +import voldemort.utils.JmxUtils; import voldemort.utils.Time; import voldemort.utils.Utils; import voldemort.utils.pool.KeyedResourcePool; @@ -47,16 +44,11 @@ * terminated upon calling {@link #close()}. */ -@JmxManaged(description = "Voldemort socket pool.") public class ClientRequestExecutorPool implements SocketStoreFactory { - private final AtomicInteger monitoringInterval = new AtomicInteger(10000); - private final AtomicInteger checkouts; - private final AtomicLong waitNs; - private final AtomicLong avgWaitNs; - private final Histogram histogramWaitNs; private final KeyedResourcePool pool; private final ClientRequestExecutorFactory factory; + private ClientSocketStats stats; public ClientRequestExecutorPool(int selectors, int maxConnectionsPerNode, @@ -75,10 +67,8 @@ public ClientRequestExecutorPool(int selectors, socketBufferSize, socketKeepAlive); this.pool = new KeyedResourcePool(factory, config); - this.checkouts = new AtomicInteger(0); - this.waitNs = new AtomicLong(0); - this.avgWaitNs = new AtomicLong(0); - this.histogramWaitNs = new Histogram(10000, 1000); + this.stats = new ClientSocketStats(pool); + ((ClientRequestExecutorFactory) factory).setStats(stats); } public ClientRequestExecutorPool(int maxConnectionsPerNode, @@ -108,6 +98,13 @@ public SocketStore create(String storeName, requestRoutingType); } + public void registerJmx() { + stats.enableJmx(); + JmxUtils.registerMbean(new ClientSocketStatsJmx(stats), + JmxUtils.createObjectName("voldemort.store.socket.clientrequest", + "aggregated")); + } + /** * Checkout a socket from the pool * @@ -120,7 +117,10 @@ public ClientRequestExecutor checkout(SocketDestination destination) { // time checkout long start = System.nanoTime(); ClientRequestExecutor clientRequestExecutor = pool.checkout(destination); - updateStats(System.nanoTime() - start); + long end = System.nanoTime(); + if(stats != null) { + stats.recordCheckoutTimeUs(destination, (end - start) / Time.NS_PER_US); + } return clientRequestExecutor; } catch(Exception e) { @@ -129,21 +129,6 @@ public ClientRequestExecutor checkout(SocketDestination destination) { } } - private void updateStats(long checkoutTimeNs) { - long wait = waitNs.getAndAdd(checkoutTimeNs); - int count = checkouts.getAndIncrement(); - - // reset reporting interval if we have used up the current interval - int interval = this.monitoringInterval.get(); - if(count % interval == interval - 1) { - // harmless race condition: - waitNs.set(0); - checkouts.set(0); - avgWaitNs.set(wait / count); - histogramWaitNs.insert(checkoutTimeNs); - } - } - /** * Check the socket back into the pool. * @@ -172,41 +157,8 @@ public void close() { pool.close(); } - @JmxGetter(name = "socketsCreated", description = "The total number of sockets created by this pool.") - public int getNumberSocketsCreated() { - return this.factory.getNumberCreated(); - } - - @JmxGetter(name = "socketsDestroyed", description = "The total number of sockets destroyed by this pool.") - public int getNumberSocketsDestroyed() { - return this.factory.getNumberDestroyed(); - } - - @JmxGetter(name = "numberOfConnections", description = "The number of active connections.") - public int getNumberOfActiveConnections() { - return this.pool.getTotalResourceCount(); - } - - @JmxGetter(name = "numberOfIdleConnections", description = "The number of idle connections.") - public int getNumberOfCheckedInConnections() { - return this.pool.getCheckedInResourceCount(); - } - - @JmxGetter(name = "avgWaitTimeMs", description = "The avg. ms of wait time to acquire a connection.") - public double getAvgWaitTimeMs() { - return this.avgWaitNs.doubleValue() / Time.NS_PER_MS; - } - - @JmxGetter(name = "99thWaitTimeMs", description = "The 99th percentile ms of wait time to acquire a connection.") - public double get99thWaitTimeMs() { - return (double) (this.histogramWaitNs.getQuantile(0.99)) / Time.NS_PER_MS; - } - - @JmxSetter(name = "monitoringInterval", description = "The number of checkouts over which performance statistics are calculated.") - public void setMonitoringInterval(int count) { - if(count <= 0) - throw new IllegalArgumentException("Monitoring interval must be a positive number."); - this.monitoringInterval.set(count); + public ClientSocketStats getStats() { + return stats; } } diff --git a/src/java/voldemort/store/stats/ClientSocketStats.java b/src/java/voldemort/store/stats/ClientSocketStats.java new file mode 100644 index 0000000000..048c340b1b --- /dev/null +++ b/src/java/voldemort/store/stats/ClientSocketStats.java @@ -0,0 +1,230 @@ +/* + * Copyright 2008-2011 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package voldemort.store.stats; + +import java.util.Iterator; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +import voldemort.store.socket.SocketDestination; +import voldemort.store.socket.clientrequest.ClientRequestExecutor; +import voldemort.utils.JmxUtils; +import voldemort.utils.pool.KeyedResourcePool; + +/** + * Some convenient statistics to track about the client requests + * + * + */ +public class ClientSocketStats { + + private final ClientSocketStats parent; + private final ConcurrentMap statsMap; + private final SocketDestination destination; + private final KeyedResourcePool pool; + + private final AtomicInteger monitoringInterval = new AtomicInteger(10000); + private final Histogram checkoutTimeUsHistogram = new Histogram(20000, 100); + private final AtomicLong totalCheckoutTimeUs = new AtomicLong(0); + private final AtomicLong avgCheckoutTimeUs = new AtomicLong(0); + private final AtomicInteger connectionsCreated = new AtomicInteger(0); + private final AtomicInteger connectionsDestroyed = new AtomicInteger(0); + private final AtomicInteger connectionsCheckedout = new AtomicInteger(0); + private boolean jmxEnable = false; + + // private final AtomicInteger connectionsCheckedin = new AtomicInteger(0); + + /** + * To construct a per node stats object + * + * @param parent An optional parent stats object that will maintain + * aggregate data across many sockets + * @param destination The destination object that defines the node + * @param pool The socket pool that will give out connection information + */ + public ClientSocketStats(ClientSocketStats parent, + SocketDestination destination, + KeyedResourcePool pool) { + this.parent = parent; + this.statsMap = null; + this.destination = destination; + this.pool = pool; + } + + /** + * Construction of a new aggregate stats object + * + * @param pool The socket pool that will give out connection information + */ + public ClientSocketStats(KeyedResourcePool pool) { + this.parent = null; + this.statsMap = new ConcurrentHashMap(); + this.destination = null; + this.pool = pool; + } + + /* get per node stats, create one if not exist */ + private ClientSocketStats getOrCreateNodeStats(SocketDestination destination) { + if(destination == null) { + return null; + } + ClientSocketStats stats = statsMap.get(destination); + if(stats == null) { + stats = new ClientSocketStats(this, destination, pool); + statsMap.putIfAbsent(destination, stats); + stats = statsMap.get(destination); + if(this.jmxEnable) { + JmxUtils.registerMbean(new ClientSocketStatsJmx(stats), + JmxUtils.createObjectName("voldemort.store.socket.clientrequest", + "stats_" + + destination.toString() + .replace(':', + '_'))); + } + } + return stats; + } + + /** + * Record the checkout wait time in us + * + * @param dest Destination of the socket to checkout. Will actually record + * if null. Otherwise will call this on self and corresponding child + * with this param null. + * @param checkoutTimeUs The number of us to wait before getting a socket + */ + public void recordCheckoutTimeUs(SocketDestination dest, long checkoutTimeUs) { + if(dest != null) { + getOrCreateNodeStats(dest).recordCheckoutTimeUs(null, checkoutTimeUs); + recordCheckoutTimeUs(null, checkoutTimeUs); + } else { + this.totalCheckoutTimeUs.getAndAdd(checkoutTimeUs); + checkoutTimeUsHistogram.insert(checkoutTimeUs); + int checkouts = this.connectionsCheckedout.getAndIncrement(); + + // reset aggregated stats and all the node stats for new interval + if(parent == null && statsMap != null) { + int interval = this.monitoringInterval.get(); + if(checkouts % interval == interval - 1) { + // reset all children + Iterator it = statsMap.keySet().iterator(); + while(it.hasNext()) { + ClientSocketStats stats = statsMap.get(it.next()); + stats.resetForInterval(); + } + // reset itself + resetForInterval(); + } + } + } + } + + /** + * Calculate the average and reset the stats + */ + public void resetForInterval() { + // harmless race condition: + this.totalCheckoutTimeUs.set(0); + this.connectionsCheckedout.set(0); + checkoutTimeUsHistogram.reset(); + } + + public void connectionCreate(SocketDestination dest) { + if(dest != null) { + getOrCreateNodeStats(dest).connectionCreate(null); + connectionCreate(null); + } else { + this.connectionsCreated.getAndIncrement(); + } + } + + public void connectionDestroy(SocketDestination dest) { + if(dest != null) { + getOrCreateNodeStats(dest).connectionDestroy(null); + connectionDestroy(null); + } else { + this.connectionsDestroyed.getAndIncrement(); + } + } + + /* getters */ + + public int getConnectionsCreated() { + return connectionsCreated.intValue(); + } + + public int getConnectionsDestroyed() { + return connectionsDestroyed.intValue(); + } + + // public int getConnectionsCheckedin() { + // return connectionsCheckedin.intValue(); + // } + + public int getConnectionsCheckedout() { + return connectionsCheckedout.intValue(); + } + + public Histogram getWaitHistogram() { + return this.checkoutTimeUsHistogram; + } + + public long getAveWaitUs() { + long ns = totalCheckoutTimeUs.get(); + int count = connectionsCheckedout.get(); + this.avgCheckoutTimeUs.set(count > 0 ? (ns / count) : -1); + return this.avgCheckoutTimeUs.longValue(); + } + + public int getConnectionsActive(SocketDestination destination) { + if(destination == null) { + return pool.getTotalResourceCount(); + } else { + return pool.getTotalResourceCount(destination); + } + } + + public int getConnectionsInPool(SocketDestination destination) { + if(destination == null) { + return pool.getCheckedInResourceCount(); + } else { + return pool.getCheckedInResourcesCount(destination); + } + } + + public void setMonitoringInterval(int count) { + this.monitoringInterval.set(count); + } + + public int getMonitoringInterval() { + return this.monitoringInterval.get(); + } + + public void enableJmx() { + jmxEnable = true; + } + + ConcurrentMap getStatsMap() { + return statsMap; + } + + SocketDestination getDestination() { + return destination; + } +} diff --git a/src/java/voldemort/store/stats/ClientSocketStatsJmx.java b/src/java/voldemort/store/stats/ClientSocketStatsJmx.java new file mode 100644 index 0000000000..ed96d6ec56 --- /dev/null +++ b/src/java/voldemort/store/stats/ClientSocketStatsJmx.java @@ -0,0 +1,101 @@ +/* + * Copyright 2008-2011 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package voldemort.store.stats; + +import voldemort.annotations.jmx.JmxGetter; +import voldemort.annotations.jmx.JmxManaged; +import voldemort.annotations.jmx.JmxSetter; +import voldemort.utils.Time; + +/** + * + * A wrapper class to expose client socket stats via JMX + * + */ + +@JmxManaged(description = "Voldemort socket pool.") +public class ClientSocketStatsJmx { + + private final ClientSocketStats stats; + + /** + * Class for JMX + */ + public ClientSocketStatsJmx(ClientSocketStats stats) { + this.stats = stats; + } + + @JmxGetter(name = "socketsCreated", description = "Number of sockets created.") + public int getConnectionsCreated() { + return stats.getConnectionsCreated(); + } + + @JmxGetter(name = "socketsDestroyed", description = "Number of sockets destroyed.") + public int getConnectionsDestroyed() { + return stats.getConnectionsDestroyed(); + } + + @JmxGetter(name = "socketsCheckedout", description = "Number of sockets checked out.") + public int getConnectionsCheckinout() { + return stats.getConnectionsCheckedout(); + } + + @JmxGetter(name = "waitMsAverage", description = "Average ms to wait to get a socket.") + public double getWaitMsAverage() { + return (double) stats.getAveWaitUs() / Time.US_PER_MS; + } + + @JmxGetter(name = "waitMsQ50th", description = "50th percentile wait time to get a connection.") + public double get() { + return (double) stats.getWaitHistogram().getQuantile(0.5) / Time.US_PER_MS; + } + + @JmxGetter(name = "waitMsQ99th", description = "99th percentile wait time to get a connection.") + public double getWaitMsQ99th() { + return (double) stats.getWaitHistogram().getQuantile(0.99) / Time.US_PER_MS; + } + + @JmxGetter(name = "socketsActive", description = "Total number of sockets, checkedin and checkout.") + public int getConnActive() { + int result = -1; + try { + result = stats.getConnectionsActive(stats.getDestination()); + } catch(Exception e) {} + return result; + } + + @JmxGetter(name = "socketsInPool", description = "Total number of sockets in the pool.") + public int getConnAvailable() { + int result = -1; + try { + result = stats.getConnectionsInPool(stats.getDestination()); + } catch(Exception e) {} + return result; + } + + @JmxGetter(name = "monitoringInterval", description = "The number of checkouts over which performance statics are calculated.") + public int getMonitoringInterval() { + return stats.getMonitoringInterval(); + } + + @JmxSetter(name = "monitoringInterval", description = "The number of checkouts over which performance statics are calculated.") + public void setMonitoringInterval(int count) { + if(count <= 0) + throw new IllegalArgumentException("Monitoring interval must be a positive number."); + stats.setMonitoringInterval(count); + } +} diff --git a/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java b/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java index b81037c0cf..3a500aadae 100644 --- a/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java +++ b/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java @@ -112,20 +112,19 @@ public void testCloseWithInFlightSockets() throws Exception { for(int i = 0; i < maxConnectionsPerNode; i++) list.add(pool.checkout(dest1)); - assertEquals(list.size(), pool.getNumberSocketsCreated()); - assertEquals(list.size(), pool.getNumberOfActiveConnections()); + assertEquals(list.size(), pool.getStats().getConnectionsCreated()); + assertEquals(list.size(), pool.getStats().getConnectionsActive(null)); pool.close(dest1); - assertEquals(list.size(), pool.getNumberOfActiveConnections()); - assertEquals(0, pool.getNumberSocketsDestroyed()); + assertEquals(list.size(), pool.getStats().getConnectionsActive(null)); + assertEquals(0, pool.getStats().getConnectionsDestroyed()); for(ClientRequestExecutor sas: list) pool.checkin(dest1, sas); - assertEquals(0, pool.getNumberOfActiveConnections()); - assertEquals(list.size(), pool.getNumberSocketsDestroyed()); - assertEquals(0, pool.getNumberOfCheckedInConnections()); + assertEquals(0, pool.getStats().getConnectionsActive(null)); + assertEquals(list.size(), pool.getStats().getConnectionsCreated()); } @Test diff --git a/test/unit/voldemort/store/stats/ClientSocketStatsTest.java b/test/unit/voldemort/store/stats/ClientSocketStatsTest.java new file mode 100644 index 0000000000..aefd1caeb2 --- /dev/null +++ b/test/unit/voldemort/store/stats/ClientSocketStatsTest.java @@ -0,0 +1,174 @@ +/* + * Copyright 2008-2011 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package voldemort.store.stats; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import org.junit.Before; +import org.junit.Test; + +import voldemort.ServerTestUtils; +import voldemort.client.protocol.RequestFormatType; +import voldemort.store.socket.SocketDestination; +import voldemort.store.socket.clientrequest.ClientRequestExecutor; +import voldemort.utils.pool.KeyedResourcePool; + +public class ClientSocketStatsTest { + + private ClientSocketStats masterStats; + private int port; + private SocketDestination dest1; + private SocketDestination dest2; + private KeyedResourcePool pool; + + @Before + public void setUp() throws Exception { + this.port = ServerTestUtils.findFreePort(); + this.dest1 = new SocketDestination("localhost", port, RequestFormatType.VOLDEMORT_V1); + this.dest2 = new SocketDestination("localhost", port + 1, RequestFormatType.VOLDEMORT_V1); + this.masterStats = new ClientSocketStats(null); + pool = null; + } + + @Test + public void testNewNodeStatsObject() { + ClientSocketStats stats = new ClientSocketStats(masterStats, dest1, pool); + assertNotNull(stats); + } + + @Test + public void testNewAggrNodeStatsObject() { + ClientSocketStats stats = masterStats; + assertNotNull(stats); + assertEquals(0, stats.getConnectionsCreated()); + assertEquals(0, stats.getConnectionsDestroyed()); + assertEquals(0, stats.getConnectionsCheckedout()); + assertEquals(-1, stats.getAveWaitUs()); + } + + @Test + public void testConnectionCreate() { + ClientSocketStats stats = masterStats; + stats.connectionCreate(dest1); + stats.connectionCreate(dest2); + stats.connectionCreate(dest1); + assertEquals(3, stats.getConnectionsCreated()); + assertEquals(2, stats.getStatsMap().get(dest1).getConnectionsCreated()); + assertEquals(1, stats.getStatsMap().get(dest2).getConnectionsCreated()); + } + + @Test + public void testConnectionDestroy() { + ClientSocketStats stats = masterStats; + stats.connectionDestroy(dest1); + stats.connectionDestroy(dest2); + stats.connectionDestroy(dest1); + assertEquals(3, stats.getConnectionsDestroyed()); + assertEquals(2, stats.getStatsMap().get(dest1).getConnectionsDestroyed()); + assertEquals(1, stats.getStatsMap().get(dest2).getConnectionsDestroyed()); + } + + @Test + public void testRecordCheckoutTimeNsOnce() { + ClientSocketStats stats = masterStats; + assertEquals(0, stats.getConnectionsCheckedout()); + + stats.recordCheckoutTimeUs(dest1, 100); + // check parent + assertEquals(1, stats.getConnectionsCheckedout()); + assertEquals(100, stats.getWaitHistogram().getQuantile(0.99)); + + // check child + ClientSocketStats child = stats.getStatsMap().get(dest1); + assertNotNull(child); + assertEquals(1, child.getConnectionsCheckedout()); + assertEquals(100, child.getWaitHistogram().getQuantile(0.99)); + } + + @Test + public void testRecordCheckoutTimeNsMultiple() { + ClientSocketStats stats = masterStats; + assertEquals(0, stats.getConnectionsCheckedout()); + + stats.recordCheckoutTimeUs(dest1, 100); + stats.recordCheckoutTimeUs(dest1, 200); + stats.recordCheckoutTimeUs(dest1, 300); + stats.recordCheckoutTimeUs(dest1, 400); + stats.recordCheckoutTimeUs(dest2, 500); + stats.recordCheckoutTimeUs(dest2, 600); + stats.recordCheckoutTimeUs(dest1, 700); + stats.recordCheckoutTimeUs(dest1, 800); + stats.recordCheckoutTimeUs(dest2, 900); + + // check parent + assertEquals(9, stats.getConnectionsCheckedout()); + assertEquals(900, stats.getWaitHistogram().getQuantile(0.99)); + + // check child1 + ClientSocketStats child1 = stats.getStatsMap().get(dest1); + assertNotNull(child1); + assertEquals(6, child1.getConnectionsCheckedout()); + assertEquals(100, child1.getWaitHistogram().getQuantile(0.1)); + assertEquals(300, child1.getWaitHistogram().getQuantile(0.5)); + assertEquals(800, child1.getWaitHistogram().getQuantile(0.99)); + + // check child2 + ClientSocketStats child2 = stats.getStatsMap().get(dest2); + assertNotNull(child2); + assertEquals(3, child2.getConnectionsCheckedout()); + assertEquals(500, child2.getWaitHistogram().getQuantile(0.1)); + assertEquals(600, child2.getWaitHistogram().getQuantile(0.5)); + assertEquals(900, child2.getWaitHistogram().getQuantile(0.99)); + } + + @Test + public void testSetMonitoringInterval() { + ClientSocketStats stats = masterStats; + stats.setMonitoringInterval(9); + stats.recordCheckoutTimeUs(dest1, 100); + stats.recordCheckoutTimeUs(dest1, 200); + stats.recordCheckoutTimeUs(dest1, 300); + stats.recordCheckoutTimeUs(dest1, 400); + stats.recordCheckoutTimeUs(dest1, 500); + stats.recordCheckoutTimeUs(dest1, 600); + stats.recordCheckoutTimeUs(dest2, 700); + stats.recordCheckoutTimeUs(dest2, 800); + // before interval based reset + // check parent + assertEquals(8, stats.getConnectionsCheckedout()); + assertEquals(450, stats.getAveWaitUs()); + // check child + ClientSocketStats child1 = stats.getStatsMap().get(dest1); + ClientSocketStats child2 = stats.getStatsMap().get(dest2); + assertEquals(6, child1.getConnectionsCheckedout()); + assertEquals(2, child2.getConnectionsCheckedout()); + assertEquals(350, child1.getAveWaitUs()); + assertEquals(750, child2.getAveWaitUs()); + + // after interval based reset + stats.recordCheckoutTimeUs(dest2, 900000); + // check parent + assertEquals(-1, stats.getAveWaitUs()); + assertEquals(0, stats.getConnectionsCheckedout()); + // check child + assertEquals(-1, child1.getAveWaitUs()); + assertEquals(0, child1.getConnectionsCheckedout()); + assertEquals(-1, child2.getAveWaitUs()); + assertEquals(0, child2.getConnectionsCheckedout()); + } +} From a3965e78dcc06c3908ce019c23cb5607fe560ed6 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Thu, 7 Jun 2012 11:40:25 -0700 Subject: [PATCH 055/209] Modified the test and build file to shrink the large tests. The large tests are copied to the test/large To run short tests for sanity check purpose: ant junit To run short and long tests: ant junit-long To run short, long and contrib tests: ant junit-all --- build.properties | 1 + build.xml | 23 +++- .../client/rebalance/RebalanceLargeTest.java | 119 ++++++++++++++++++ .../rebalance/AbstractRebalanceTest.java | 17 ++- 4 files changed, 153 insertions(+), 7 deletions(-) create mode 100644 test/large/voldemort/client/rebalance/RebalanceLargeTest.java diff --git a/build.properties b/build.properties index 3bc09bc26c..933629c18e 100644 --- a/build.properties +++ b/build.properties @@ -9,6 +9,7 @@ classes.dir=dist/classes resources.dir=dist/resources commontestsrc.dir=test/common unittestsrc.dir=test/unit +largetestsrc.dir=test/large inttestsrc.dir=test/integration testclasses.dir=dist/testclasses testreport.dir=dist/junit-reports diff --git a/build.xml b/build.xml index 1f9a535f22..1c4d4c6136 100644 --- a/build.xml +++ b/build.xml @@ -103,6 +103,7 @@ + @@ -387,6 +388,26 @@ + + + + + + + + + + + + + + + + + + + + @@ -394,7 +415,7 @@ - + diff --git a/test/large/voldemort/client/rebalance/RebalanceLargeTest.java b/test/large/voldemort/client/rebalance/RebalanceLargeTest.java new file mode 100644 index 0000000000..16d5b5091f --- /dev/null +++ b/test/large/voldemort/client/rebalance/RebalanceLargeTest.java @@ -0,0 +1,119 @@ +package voldemort.client.rebalance; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; + +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +import voldemort.ServerTestUtils; +import voldemort.TestUtils; +import voldemort.VoldemortException; +import voldemort.cluster.Cluster; +import voldemort.server.VoldemortConfig; +import voldemort.server.VoldemortServer; +import voldemort.store.metadata.MetadataStore.VoldemortState; + +/** + * Start VoldemortServer locally using ServerTestUtils and run rebalancing + * tests. + * + * + */ +@RunWith(Parameterized.class) +public class RebalanceLargeTest extends AbstractRebalanceTest { + + Map serverMap = new HashMap(); + private final boolean useNio; + private final boolean useDonorBased; + protected static int NUM_MANY_KEYS = 10100; + + public RebalanceLargeTest(boolean useNio, boolean useDonorBased) { + this.useNio = useNio; + this.useDonorBased = useDonorBased; + } + + @Parameters + public static Collection configs() { + return Arrays.asList(new Object[][] { { true, true }, { true, false }, { false, true }, + { false, false } }); + } + + @Override + protected int getNumKeys() { + return NUM_MANY_KEYS; + } + + @Override + protected VoldemortState getCurrentState(int nodeId) { + VoldemortServer server = serverMap.get(nodeId); + if(server == null) { + throw new VoldemortException("Node id " + nodeId + " does not exist"); + } else { + return server.getMetadataStore().getServerState(); + } + } + + @Override + protected Cluster getCurrentCluster(int nodeId) { + VoldemortServer server = serverMap.get(nodeId); + if(server == null) { + throw new VoldemortException("Node id " + nodeId + " does not exist"); + } else { + return server.getMetadataStore().getCluster(); + } + } + + @Override + protected Cluster startServers(Cluster cluster, + String storeXmlFile, + List nodeToStart, + Map configProps) throws IOException { + for(int node: nodeToStart) { + Properties properties = new Properties(); + if(null != configProps) { + for(Entry property: configProps.entrySet()) { + properties.put(property.getKey(), property.getValue()); + } + } + + VoldemortConfig config = ServerTestUtils.createServerConfig(useNio, + node, + TestUtils.createTempDir() + .getAbsolutePath(), + null, + storeXmlFile, + properties); + + VoldemortServer server = ServerTestUtils.startVoldemortServer(socketStoreFactory, + config, + cluster); + serverMap.put(node, server); + } + + return cluster; + } + + @Override + protected void stopServer(List nodesToStop) throws IOException { + for(int node: nodesToStop) { + try { + ServerTestUtils.stopVoldemortServer(serverMap.get(node)); + } catch(VoldemortException e) { + // ignore these at stop time + } + } + } + + @Override + protected boolean useDonorBased() { + return this.useDonorBased; + } +} diff --git a/test/unit/voldemort/client/rebalance/AbstractRebalanceTest.java b/test/unit/voldemort/client/rebalance/AbstractRebalanceTest.java index d3b8fb3b47..cde3ee5638 100644 --- a/test/unit/voldemort/client/rebalance/AbstractRebalanceTest.java +++ b/test/unit/voldemort/client/rebalance/AbstractRebalanceTest.java @@ -29,8 +29,8 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.Map.Entry; +import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -86,7 +86,7 @@ public abstract class AbstractRebalanceTest { - protected static int NUM_KEYS = 10100; + protected static int NUM_KEYS = 20; protected static int NUM_RO_CHUNKS_PER_BUCKET = 10; protected static String testStoreNameRW = "test"; protected static String testStoreNameRW2 = "test2"; @@ -111,7 +111,7 @@ public abstract class AbstractRebalanceTest { @Before public void setUp() throws IOException { - testEntries = ServerTestUtils.createRandomKeyValueString(NUM_KEYS); + testEntries = ServerTestUtils.createRandomKeyValueString(getNumKeys()); socketStoreFactory = new ClientRequestExecutorPool(2, 10000, 100000, 32 * 1024); // First without replication @@ -258,6 +258,10 @@ public void checkConsistentMetadata(Cluster targetCluster, List serverL } } + protected int getNumKeys() { + return NUM_KEYS; + } + @Test public void testRORWRebalance() throws Exception { Cluster currentCluster = ServerTestUtils.getLocalCluster(2, new int[][] { @@ -1053,9 +1057,10 @@ protected void populateData(Cluster cluster, // Create SocketStores for each Node first Map> storeMap = new HashMap>(); for(Node node: cluster.getNodes()) { - storeMap.put(node.getId(), getSocketStore(storeDef.getName(), - node.getHost(), - node.getSocketPort())); + storeMap.put(node.getId(), + getSocketStore(storeDef.getName(), + node.getHost(), + node.getSocketPort())); } From b81f2a987dbc4ef2407b90fac10da028bc9e9e5b Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Thu, 7 Jun 2012 14:48:27 -0700 Subject: [PATCH 056/209] renamed large as long to make files consistent --- build.properties | 2 +- build.xml | 4 ++-- .../voldemort/client/rebalance/RebalanceLongTest.java} | 0 3 files changed, 3 insertions(+), 3 deletions(-) rename test/{large/voldemort/client/rebalance/RebalanceLargeTest.java => long/voldemort/client/rebalance/RebalanceLongTest.java} (100%) diff --git a/build.properties b/build.properties index 933629c18e..3cc0b8cef7 100644 --- a/build.properties +++ b/build.properties @@ -9,7 +9,7 @@ classes.dir=dist/classes resources.dir=dist/resources commontestsrc.dir=test/common unittestsrc.dir=test/unit -largetestsrc.dir=test/large +longtestsrc.dir=test/long inttestsrc.dir=test/integration testclasses.dir=dist/testclasses testreport.dir=dist/junit-reports diff --git a/build.xml b/build.xml index 1c4d4c6136..d11d89938d 100644 --- a/build.xml +++ b/build.xml @@ -103,7 +103,7 @@ - + @@ -395,7 +395,7 @@ - + diff --git a/test/large/voldemort/client/rebalance/RebalanceLargeTest.java b/test/long/voldemort/client/rebalance/RebalanceLongTest.java similarity index 100% rename from test/large/voldemort/client/rebalance/RebalanceLargeTest.java rename to test/long/voldemort/client/rebalance/RebalanceLongTest.java From cc1cc7013ff50641a70a08a78d7e22d328dcbaca Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Thu, 7 Jun 2012 16:14:13 -0700 Subject: [PATCH 057/209] corrected a class name mistake --- test/long/voldemort/client/rebalance/RebalanceLongTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/long/voldemort/client/rebalance/RebalanceLongTest.java b/test/long/voldemort/client/rebalance/RebalanceLongTest.java index 16d5b5091f..eecba30bb0 100644 --- a/test/long/voldemort/client/rebalance/RebalanceLongTest.java +++ b/test/long/voldemort/client/rebalance/RebalanceLongTest.java @@ -28,14 +28,14 @@ * */ @RunWith(Parameterized.class) -public class RebalanceLargeTest extends AbstractRebalanceTest { +public class RebalanceLongTest extends AbstractRebalanceTest { Map serverMap = new HashMap(); private final boolean useNio; private final boolean useDonorBased; protected static int NUM_MANY_KEYS = 10100; - public RebalanceLargeTest(boolean useNio, boolean useDonorBased) { + public RebalanceLongTest(boolean useNio, boolean useDonorBased) { this.useNio = useNio; this.useDonorBased = useDonorBased; } From 8b0a7c292dce025a4469c5ae22613f187ba863ae Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Thu, 7 Jun 2012 22:35:52 -0700 Subject: [PATCH 058/209] distinguish test report locations between normal tests and long tests --- build.properties | 2 ++ build.xml | 12 ++++++------ 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/build.properties b/build.properties index 3cc0b8cef7..f363ea1bff 100644 --- a/build.properties +++ b/build.properties @@ -14,6 +14,8 @@ inttestsrc.dir=test/integration testclasses.dir=dist/testclasses testreport.dir=dist/junit-reports testhtml.dir=dist/junit-reports/html +longtestreport.dir=dist/junit-long-reports +longtesthtml.dir=dist/junit-long-reports/html ## Contrib contrib.root.dir=contrib diff --git a/build.xml b/build.xml index d11d89938d..149472494d 100644 --- a/build.xml +++ b/build.xml @@ -389,22 +389,22 @@ - - + + - + - - + + - + From 20dfa29fd377b3a759a1506bd5af5f2ac6014456 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 6 Jun 2012 13:27:43 -0700 Subject: [PATCH 059/209] prevent preferred reads from crossing zones. --- .../routed/action/PerformSerialRequests.java | 27 ++++++++++++++++++- 1 file changed, 26 insertions(+), 1 deletion(-) diff --git a/src/java/voldemort/store/routed/action/PerformSerialRequests.java b/src/java/voldemort/store/routed/action/PerformSerialRequests.java index 24de5236c7..70c8fa563a 100644 --- a/src/java/voldemort/store/routed/action/PerformSerialRequests.java +++ b/src/java/voldemort/store/routed/action/PerformSerialRequests.java @@ -66,6 +66,20 @@ public PerformSerialRequests(PD pipelineData, this.insufficientSuccessesEvent = insufficientSuccessesEvent; } + /** + * Checks whether every property except 'preferred' is satisfied + * + * @return + */ + private boolean isSatisfied() { + if(pipelineData.getZonesRequired() != null) { + return ((pipelineData.getSuccesses() >= required) && (pipelineData.getZoneResponses() + .size() >= (pipelineData.getZonesRequired() + 1))); + } else { + return pipelineData.getSuccesses() >= required; + } + } + public void execute(Pipeline pipeline) { List nodes = pipelineData.getNodes(); @@ -95,6 +109,10 @@ public void execute(Pipeline pipeline) { return; } + // break out if we have satisfied everything + if(isSatisfied()) + break; + pipelineData.incrementNodeIndex(); } @@ -123,6 +141,14 @@ public void execute(Pipeline pipeline) { if(zonesSatisfied >= (pipelineData.getZonesRequired() + 1)) { pipeline.addEvent(completeEvent); } else { + // if you run with zoneCountReads > 0, we could frequently + // run into this exception since our preference list for + // zone routing is laid out thus : , , , ,... + // #preferred number of reads may not be able to satisfy + // zoneCountReads, if the original read to a remote node + // fails in the parallel stage pipelineData.setFatalError(new InsufficientZoneResponsesException((pipelineData.getZonesRequired() + 1) + " " + pipeline.getOperation() @@ -139,5 +165,4 @@ public void execute(Pipeline pipeline) { } } } - } From d63c2ed4b62977e0b428e3d50989ca6dcc51e6bb Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Mon, 11 Jun 2012 10:51:15 -0700 Subject: [PATCH 060/209] further improvement on per node socket pool monitoring --- .../client/SocketStoreClientFactory.java | 21 ++++++--- .../ClientRequestExecutorFactory.java | 21 ++++----- .../ClientRequestExecutorPool.java | 43 ++++++++++++------- .../store/stats/ClientSocketStats.java | 30 +++++-------- .../store/stats/ClientSocketStatsJmx.java | 2 +- .../store/stats/ClientSocketStatsTest.java | 2 +- 6 files changed, 64 insertions(+), 55 deletions(-) diff --git a/src/java/voldemort/client/SocketStoreClientFactory.java b/src/java/voldemort/client/SocketStoreClientFactory.java index 8da1bd7135..dd7e1ff399 100644 --- a/src/java/voldemort/client/SocketStoreClientFactory.java +++ b/src/java/voldemort/client/SocketStoreClientFactory.java @@ -1,5 +1,5 @@ /* - * Copyright 2008-2010 LinkedIn, Inc + * Copyright 2008-2012 LinkedIn, Inc * * Licensed under the Apache License, Version 2.0 (the "License"); you may not * use this file except in compliance with the License. You may obtain a copy of @@ -37,7 +37,10 @@ import voldemort.store.socket.SocketDestination; import voldemort.store.socket.SocketStoreFactory; import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; +import voldemort.store.stats.ClientSocketStats; +import voldemort.store.stats.ClientSocketStatsJmx; import voldemort.utils.ByteArray; +import voldemort.utils.JmxUtils; import voldemort.versioning.InconsistencyResolver; import voldemort.versioning.Versioned; @@ -56,21 +59,27 @@ public class SocketStoreClientFactory extends AbstractStoreClientFactory { private final SocketStoreFactory storeFactory; private FailureDetectorListener failureDetectorListener; private final RequestRoutingType requestRoutingType; + private final ClientSocketStats stats; public SocketStoreClientFactory(ClientConfig config) { super(config); this.requestRoutingType = RequestRoutingType.getRequestRoutingType(RoutingTier.SERVER.equals(config.getRoutingTier()), false); - + if(config.isJmxEnabled()) { + stats = new ClientSocketStats(); + } else { + stats = null; + } this.storeFactory = new ClientRequestExecutorPool(config.getSelectors(), config.getMaxConnectionsPerNode(), config.getConnectionTimeout(TimeUnit.MILLISECONDS), config.getSocketTimeout(TimeUnit.MILLISECONDS), config.getSocketBufferSize(), - config.getSocketKeepAlive()); - if(config.isJmxEnabled()) { - ((ClientRequestExecutorPool) storeFactory).registerJmx(); - } + config.getSocketKeepAlive(), + stats); + JmxUtils.registerMbean(new ClientSocketStatsJmx(stats), + JmxUtils.createObjectName(JmxUtils.getPackageName(storeFactory.getClass()), + "aggregated")); } @Override diff --git a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorFactory.java b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorFactory.java index 74f33df12b..0309cce369 100644 --- a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorFactory.java +++ b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorFactory.java @@ -1,5 +1,5 @@ /* - * Copyright 2008-2010 LinkedIn, Inc + * Copyright 2008-2012 LinkedIn, Inc * * Licensed under the Apache License, Version 2.0 (the "License"); you may not * use this file except in compliance with the License. You may obtain a copy of @@ -62,19 +62,21 @@ public class ClientRequestExecutorFactory implements private final AtomicInteger counter = new AtomicInteger(); private final Map lastClosedTimestamps; private final Logger logger = Logger.getLogger(getClass()); - private ClientSocketStats stats; + private final ClientSocketStats stats; public ClientRequestExecutorFactory(int selectors, int connectTimeoutMs, int soTimeoutMs, int socketBufferSize, - boolean socketKeepAlive) { + boolean socketKeepAlive, + ClientSocketStats stats) { this.connectTimeoutMs = connectTimeoutMs; this.soTimeoutMs = soTimeoutMs; this.created = new AtomicInteger(0); this.destroyed = new AtomicInteger(0); this.socketBufferSize = socketBufferSize; this.socketKeepAlive = socketKeepAlive; + this.stats = stats; this.selectorManagers = new ClientRequestSelectorManager[selectors]; this.selectorManagerThreadPool = Executors.newFixedThreadPool(selectorManagers.length, @@ -113,10 +115,6 @@ public void destroy(SocketDestination dest, ClientRequestExecutor clientRequestE public ClientRequestExecutor create(SocketDestination dest) throws Exception { int numCreated = created.incrementAndGet(); - if(stats != null) { - stats.connectionCreate(dest); - } - if(logger.isDebugEnabled()) logger.debug("Creating socket " + numCreated + " for " + dest.getHost() + ":" + dest.getPort() + " using protocol " @@ -224,6 +222,10 @@ public ClientRequestExecutor create(SocketDestination dest) throws Exception { throw e; } + if(stats != null) { + stats.connectionCreate(dest); + } + return clientRequestExecutor; } @@ -433,9 +435,4 @@ private long getLastClosedTimestamp(SocketDestination socketDestination) { public void setLastClosedTimestamp(SocketDestination socketDestination) { lastClosedTimestamps.put(socketDestination, System.nanoTime()); } - - public void setStats(ClientSocketStats stats) { - this.stats = stats; - } - } diff --git a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java index 6cc35dacb6..cbd51f9976 100644 --- a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java +++ b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java @@ -1,5 +1,5 @@ /* - * Copyright 2008-2010 LinkedIn, Inc + * Copyright 2008-2012 LinkedIn, Inc * * Licensed under the Apache License, Version 2.0 (the "License"); you may not * use this file except in compliance with the License. You may obtain a copy of @@ -26,8 +26,6 @@ import voldemort.store.socket.SocketStore; import voldemort.store.socket.SocketStoreFactory; import voldemort.store.stats.ClientSocketStats; -import voldemort.store.stats.ClientSocketStatsJmx; -import voldemort.utils.JmxUtils; import voldemort.utils.Time; import voldemort.utils.Utils; import voldemort.utils.pool.KeyedResourcePool; @@ -48,27 +46,48 @@ public class ClientRequestExecutorPool implements SocketStoreFactory { private final KeyedResourcePool pool; private final ClientRequestExecutorFactory factory; - private ClientSocketStats stats; + private final ClientSocketStats stats; public ClientRequestExecutorPool(int selectors, int maxConnectionsPerNode, int connectionTimeoutMs, int soTimeoutMs, int socketBufferSize, - boolean socketKeepAlive) { + boolean socketKeepAlive, + ClientSocketStats stats) { ResourcePoolConfig config = new ResourcePoolConfig().setIsFair(true) .setMaxPoolSize(maxConnectionsPerNode) .setMaxInvalidAttempts(maxConnectionsPerNode) .setTimeout(connectionTimeoutMs, TimeUnit.MILLISECONDS); + + this.stats = stats; this.factory = new ClientRequestExecutorFactory(selectors, connectionTimeoutMs, soTimeoutMs, socketBufferSize, - socketKeepAlive); + socketKeepAlive, + stats); this.pool = new KeyedResourcePool(factory, config); - this.stats = new ClientSocketStats(pool); - ((ClientRequestExecutorFactory) factory).setStats(stats); + if(stats != null) { + this.stats.setPool(pool); + } + } + + // JMX bean is disabled by default + public ClientRequestExecutorPool(int selectors, + int maxConnectionsPerNode, + int connectionTimeoutMs, + int soTimeoutMs, + int socketBufferSize, + boolean socketKeepAlive) { + this(selectors, + maxConnectionsPerNode, + connectionTimeoutMs, + soTimeoutMs, + socketBufferSize, + socketKeepAlive, + null); } public ClientRequestExecutorPool(int maxConnectionsPerNode, @@ -98,13 +117,6 @@ public SocketStore create(String storeName, requestRoutingType); } - public void registerJmx() { - stats.enableJmx(); - JmxUtils.registerMbean(new ClientSocketStatsJmx(stats), - JmxUtils.createObjectName("voldemort.store.socket.clientrequest", - "aggregated")); - } - /** * Checkout a socket from the pool * @@ -155,6 +167,7 @@ public void close(SocketDestination destination) { public void close() { factory.close(); pool.close(); + } public ClientSocketStats getStats() { diff --git a/src/java/voldemort/store/stats/ClientSocketStats.java b/src/java/voldemort/store/stats/ClientSocketStats.java index 048c340b1b..151587ca8b 100644 --- a/src/java/voldemort/store/stats/ClientSocketStats.java +++ b/src/java/voldemort/store/stats/ClientSocketStats.java @@ -37,7 +37,7 @@ public class ClientSocketStats { private final ClientSocketStats parent; private final ConcurrentMap statsMap; private final SocketDestination destination; - private final KeyedResourcePool pool; + private KeyedResourcePool pool; private final AtomicInteger monitoringInterval = new AtomicInteger(10000); private final Histogram checkoutTimeUsHistogram = new Histogram(20000, 100); @@ -46,9 +46,6 @@ public class ClientSocketStats { private final AtomicInteger connectionsCreated = new AtomicInteger(0); private final AtomicInteger connectionsDestroyed = new AtomicInteger(0); private final AtomicInteger connectionsCheckedout = new AtomicInteger(0); - private boolean jmxEnable = false; - - // private final AtomicInteger connectionsCheckedin = new AtomicInteger(0); /** * To construct a per node stats object @@ -72,11 +69,11 @@ public ClientSocketStats(ClientSocketStats parent, * * @param pool The socket pool that will give out connection information */ - public ClientSocketStats(KeyedResourcePool pool) { + public ClientSocketStats() { this.parent = null; this.statsMap = new ConcurrentHashMap(); this.destination = null; - this.pool = pool; + this.pool = null; } /* get per node stats, create one if not exist */ @@ -89,14 +86,11 @@ private ClientSocketStats getOrCreateNodeStats(SocketDestination destination) { stats = new ClientSocketStats(this, destination, pool); statsMap.putIfAbsent(destination, stats); stats = statsMap.get(destination); - if(this.jmxEnable) { - JmxUtils.registerMbean(new ClientSocketStatsJmx(stats), - JmxUtils.createObjectName("voldemort.store.socket.clientrequest", - "stats_" - + destination.toString() - .replace(':', - '_'))); - } + JmxUtils.registerMbean(new ClientSocketStatsJmx(stats), + JmxUtils.createObjectName(JmxUtils.getPackageName(ClientRequestExecutor.class), + "stats_" + + destination.toString() + .replace(':', '_'))); } return stats; } @@ -173,10 +167,6 @@ public int getConnectionsDestroyed() { return connectionsDestroyed.intValue(); } - // public int getConnectionsCheckedin() { - // return connectionsCheckedin.intValue(); - // } - public int getConnectionsCheckedout() { return connectionsCheckedout.intValue(); } @@ -216,8 +206,8 @@ public int getMonitoringInterval() { return this.monitoringInterval.get(); } - public void enableJmx() { - jmxEnable = true; + public void setPool(KeyedResourcePool pool) { + this.pool = pool; } ConcurrentMap getStatsMap() { diff --git a/src/java/voldemort/store/stats/ClientSocketStatsJmx.java b/src/java/voldemort/store/stats/ClientSocketStatsJmx.java index ed96d6ec56..e7ba57d6bc 100644 --- a/src/java/voldemort/store/stats/ClientSocketStatsJmx.java +++ b/src/java/voldemort/store/stats/ClientSocketStatsJmx.java @@ -60,7 +60,7 @@ public double getWaitMsAverage() { } @JmxGetter(name = "waitMsQ50th", description = "50th percentile wait time to get a connection.") - public double get() { + public double getWaitMsQ50th() { return (double) stats.getWaitHistogram().getQuantile(0.5) / Time.US_PER_MS; } diff --git a/test/unit/voldemort/store/stats/ClientSocketStatsTest.java b/test/unit/voldemort/store/stats/ClientSocketStatsTest.java index aefd1caeb2..bd89713c93 100644 --- a/test/unit/voldemort/store/stats/ClientSocketStatsTest.java +++ b/test/unit/voldemort/store/stats/ClientSocketStatsTest.java @@ -41,7 +41,7 @@ public void setUp() throws Exception { this.port = ServerTestUtils.findFreePort(); this.dest1 = new SocketDestination("localhost", port, RequestFormatType.VOLDEMORT_V1); this.dest2 = new SocketDestination("localhost", port + 1, RequestFormatType.VOLDEMORT_V1); - this.masterStats = new ClientSocketStats(null); + this.masterStats = new ClientSocketStats(); pool = null; } From 1197e9b3f3bf8659c94ca7b244134d0681344d5b Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Mon, 11 Jun 2012 13:03:33 -0700 Subject: [PATCH 061/209] added unregister MBeans when closing pool --- .../ClientRequestExecutorPool.java | 18 ++++++++++++++++++ .../store/stats/ClientSocketStats.java | 2 +- 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java index cbd51f9976..72297a6d2c 100644 --- a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java +++ b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java @@ -16,6 +16,7 @@ package voldemort.store.socket.clientrequest; +import java.util.Iterator; import java.util.concurrent.TimeUnit; import voldemort.VoldemortException; @@ -26,6 +27,7 @@ import voldemort.store.socket.SocketStore; import voldemort.store.socket.SocketStoreFactory; import voldemort.store.stats.ClientSocketStats; +import voldemort.utils.JmxUtils; import voldemort.utils.Time; import voldemort.utils.Utils; import voldemort.utils.pool.KeyedResourcePool; @@ -165,6 +167,22 @@ public void close(SocketDestination destination) { * Close the socket pool */ public void close() { + // unregister MBeans + if(stats != null) { + Iterator it = stats.getStatsMap().keySet().iterator(); + while(it.hasNext()) { + try { + SocketDestination destination = it.next(); + JmxUtils.unregisterMbean(JmxUtils.createObjectName(JmxUtils.getPackageName(ClientRequestExecutor.class), + "stats_" + + destination.toString() + .replace(':', + '_'))); + } catch(Exception e) {} + } + JmxUtils.unregisterMbean(JmxUtils.createObjectName(JmxUtils.getPackageName(ClientRequestExecutor.class), + "aggregated")); + } factory.close(); pool.close(); diff --git a/src/java/voldemort/store/stats/ClientSocketStats.java b/src/java/voldemort/store/stats/ClientSocketStats.java index 151587ca8b..93b06e151c 100644 --- a/src/java/voldemort/store/stats/ClientSocketStats.java +++ b/src/java/voldemort/store/stats/ClientSocketStats.java @@ -210,7 +210,7 @@ public void setPool(KeyedResourcePool this.pool = pool; } - ConcurrentMap getStatsMap() { + public ConcurrentMap getStatsMap() { return statsMap; } From 9c520a9ea5edeef36a76e1e70e5055fafa14f511 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Mon, 11 Jun 2012 13:04:34 -0700 Subject: [PATCH 062/209] registers MBean in ClientRequestExecutorPoolTest.java so that ClientSocketStats object will exist for testing --- .../server/socket/ClientRequestExecutorPoolTest.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java b/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java index 3a500aadae..cec6d3ec97 100644 --- a/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java +++ b/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java @@ -38,6 +38,7 @@ import voldemort.store.socket.SocketDestination; import voldemort.store.socket.clientrequest.ClientRequestExecutor; import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; +import voldemort.store.stats.ClientSocketStats; /** * Tests for the socket pooling @@ -52,6 +53,7 @@ public class ClientRequestExecutorPoolTest extends TestCase { private ClientRequestExecutorPool pool; private SocketDestination dest1; private AbstractSocketService server; + private ClientSocketStats stats; private final boolean useNio; @@ -67,8 +69,15 @@ public static Collection configs() { @Override @Before public void setUp() { + this.stats = new ClientSocketStats(); this.port = ServerTestUtils.findFreePort(); - this.pool = new ClientRequestExecutorPool(maxConnectionsPerNode, 1000, 1000, 32 * 1024); + this.pool = new ClientRequestExecutorPool(2, + maxConnectionsPerNode, + 1000, + 1000, + 32 * 1024, + false, + stats); this.dest1 = new SocketDestination("localhost", port, RequestFormatType.VOLDEMORT_V1); RequestHandlerFactory handlerFactory = ServerTestUtils.getSocketRequestHandlerFactory(new StoreRepository()); this.server = ServerTestUtils.getSocketService(useNio, From 320203f2e769c53b01156391f158ba13d6994b84 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Mon, 11 Jun 2012 18:22:39 -0700 Subject: [PATCH 063/209] moved registering JMX inside ClientRequestExecutorPool --- .../client/SocketStoreClientFactory.java | 14 +------------- .../ClientRequestExecutorPool.java | 17 ++++++++++++----- .../socket/ClientRequestExecutorPoolTest.java | 3 +-- 3 files changed, 14 insertions(+), 20 deletions(-) diff --git a/src/java/voldemort/client/SocketStoreClientFactory.java b/src/java/voldemort/client/SocketStoreClientFactory.java index dd7e1ff399..c5c7336f5d 100644 --- a/src/java/voldemort/client/SocketStoreClientFactory.java +++ b/src/java/voldemort/client/SocketStoreClientFactory.java @@ -37,10 +37,7 @@ import voldemort.store.socket.SocketDestination; import voldemort.store.socket.SocketStoreFactory; import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; -import voldemort.store.stats.ClientSocketStats; -import voldemort.store.stats.ClientSocketStatsJmx; import voldemort.utils.ByteArray; -import voldemort.utils.JmxUtils; import voldemort.versioning.InconsistencyResolver; import voldemort.versioning.Versioned; @@ -59,27 +56,18 @@ public class SocketStoreClientFactory extends AbstractStoreClientFactory { private final SocketStoreFactory storeFactory; private FailureDetectorListener failureDetectorListener; private final RequestRoutingType requestRoutingType; - private final ClientSocketStats stats; public SocketStoreClientFactory(ClientConfig config) { super(config); this.requestRoutingType = RequestRoutingType.getRequestRoutingType(RoutingTier.SERVER.equals(config.getRoutingTier()), false); - if(config.isJmxEnabled()) { - stats = new ClientSocketStats(); - } else { - stats = null; - } this.storeFactory = new ClientRequestExecutorPool(config.getSelectors(), config.getMaxConnectionsPerNode(), config.getConnectionTimeout(TimeUnit.MILLISECONDS), config.getSocketTimeout(TimeUnit.MILLISECONDS), config.getSocketBufferSize(), config.getSocketKeepAlive(), - stats); - JmxUtils.registerMbean(new ClientSocketStatsJmx(stats), - JmxUtils.createObjectName(JmxUtils.getPackageName(storeFactory.getClass()), - "aggregated")); + config.isJmxEnabled()); } @Override diff --git a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java index 72297a6d2c..4cf3ff88e2 100644 --- a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java +++ b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java @@ -27,6 +27,7 @@ import voldemort.store.socket.SocketStore; import voldemort.store.socket.SocketStoreFactory; import voldemort.store.stats.ClientSocketStats; +import voldemort.store.stats.ClientSocketStatsJmx; import voldemort.utils.JmxUtils; import voldemort.utils.Time; import voldemort.utils.Utils; @@ -56,14 +57,20 @@ public ClientRequestExecutorPool(int selectors, int soTimeoutMs, int socketBufferSize, boolean socketKeepAlive, - ClientSocketStats stats) { + boolean enableJmx) { ResourcePoolConfig config = new ResourcePoolConfig().setIsFair(true) .setMaxPoolSize(maxConnectionsPerNode) .setMaxInvalidAttempts(maxConnectionsPerNode) .setTimeout(connectionTimeoutMs, TimeUnit.MILLISECONDS); - - this.stats = stats; + if(enableJmx) { + stats = new ClientSocketStats(); + JmxUtils.registerMbean(new ClientSocketStatsJmx(stats), + JmxUtils.createObjectName(JmxUtils.getPackageName(this.getClass()), + "aggregated")); + } else { + stats = null; + } this.factory = new ClientRequestExecutorFactory(selectors, connectionTimeoutMs, soTimeoutMs, @@ -76,20 +83,20 @@ public ClientRequestExecutorPool(int selectors, } } - // JMX bean is disabled by default public ClientRequestExecutorPool(int selectors, int maxConnectionsPerNode, int connectionTimeoutMs, int soTimeoutMs, int socketBufferSize, boolean socketKeepAlive) { + // JMX bean is disabled by default this(selectors, maxConnectionsPerNode, connectionTimeoutMs, soTimeoutMs, socketBufferSize, socketKeepAlive, - null); + false); } public ClientRequestExecutorPool(int maxConnectionsPerNode, diff --git a/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java b/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java index cec6d3ec97..091a6073f8 100644 --- a/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java +++ b/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java @@ -69,7 +69,6 @@ public static Collection configs() { @Override @Before public void setUp() { - this.stats = new ClientSocketStats(); this.port = ServerTestUtils.findFreePort(); this.pool = new ClientRequestExecutorPool(2, maxConnectionsPerNode, @@ -77,7 +76,7 @@ public void setUp() { 1000, 32 * 1024, false, - stats); + true); this.dest1 = new SocketDestination("localhost", port, RequestFormatType.VOLDEMORT_V1); RequestHandlerFactory handlerFactory = ServerTestUtils.getSocketRequestHandlerFactory(new StoreRepository()); this.server = ServerTestUtils.getSocketService(useNio, From f527daa2d631931145f82fa08c2cf344e525307e Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Tue, 12 Jun 2012 11:36:33 -0700 Subject: [PATCH 064/209] moved unregistering of the child stats beans into the parent stats --- .../ClientRequestExecutorPool.java | 20 +++++-------------- .../store/stats/ClientSocketStats.java | 17 ++++++++++++++++ 2 files changed, 22 insertions(+), 15 deletions(-) diff --git a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java index 4cf3ff88e2..6d015a2d54 100644 --- a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java +++ b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java @@ -16,7 +16,6 @@ package voldemort.store.socket.clientrequest; -import java.util.Iterator; import java.util.concurrent.TimeUnit; import voldemort.VoldemortException; @@ -176,23 +175,14 @@ public void close(SocketDestination destination) { public void close() { // unregister MBeans if(stats != null) { - Iterator it = stats.getStatsMap().keySet().iterator(); - while(it.hasNext()) { - try { - SocketDestination destination = it.next(); - JmxUtils.unregisterMbean(JmxUtils.createObjectName(JmxUtils.getPackageName(ClientRequestExecutor.class), - "stats_" - + destination.toString() - .replace(':', - '_'))); - } catch(Exception e) {} - } - JmxUtils.unregisterMbean(JmxUtils.createObjectName(JmxUtils.getPackageName(ClientRequestExecutor.class), - "aggregated")); + try { + JmxUtils.unregisterMbean(JmxUtils.createObjectName(JmxUtils.getPackageName(ClientRequestExecutor.class), + "aggregated")); + } catch(Exception e) {} + stats.close(); } factory.close(); pool.close(); - } public ClientSocketStats getStats() { diff --git a/src/java/voldemort/store/stats/ClientSocketStats.java b/src/java/voldemort/store/stats/ClientSocketStats.java index 93b06e151c..721f67a802 100644 --- a/src/java/voldemort/store/stats/ClientSocketStats.java +++ b/src/java/voldemort/store/stats/ClientSocketStats.java @@ -217,4 +217,21 @@ public ConcurrentMap getStatsMap() { SocketDestination getDestination() { return destination; } + + /** + * Unregister all MBeans + */ + public void close() { + Iterator it = getStatsMap().keySet().iterator(); + while(it.hasNext()) { + try { + SocketDestination destination = it.next(); + JmxUtils.unregisterMbean(JmxUtils.createObjectName(JmxUtils.getPackageName(ClientRequestExecutor.class), + "stats_" + + destination.toString() + .replace(':', + '_'))); + } catch(Exception e) {} + } + } } From be224413bb0c53a4339c72f9c5c0c09687034ac3 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Wed, 13 Jun 2012 14:22:26 -0700 Subject: [PATCH 065/209] changed checkout record code to make sure timeouts are recorded --- .../ClientRequestExecutorPool.java | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java index 6d015a2d54..1c7ee64bc3 100644 --- a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java +++ b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java @@ -133,20 +133,21 @@ public SocketStore create(String storeName, */ public ClientRequestExecutor checkout(SocketDestination destination) { + // time checkout + long start = System.nanoTime(); + ClientRequestExecutor clientRequestExecutor; try { - // time checkout - long start = System.nanoTime(); - ClientRequestExecutor clientRequestExecutor = pool.checkout(destination); + clientRequestExecutor = pool.checkout(destination); + } catch(Exception e) { + throw new UnreachableStoreException("Failure while checking out socket for " + + destination + ": ", e); + } finally { long end = System.nanoTime(); if(stats != null) { stats.recordCheckoutTimeUs(destination, (end - start) / Time.NS_PER_US); } - - return clientRequestExecutor; - } catch(Exception e) { - throw new UnreachableStoreException("Failure while checking out socket for " - + destination + ": ", e); } + return clientRequestExecutor; } /** From 32aa7d58d3fd7c2132034b9cbd3d80ebf4b7d3e2 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Tue, 19 Jun 2012 15:04:33 -0700 Subject: [PATCH 066/209] Added concurrent tests for Histogram related code --- .../store/stats/ClientSocketStatsTest.java | 59 +++++++++++++++++++ 1 file changed, 59 insertions(+) diff --git a/test/unit/voldemort/store/stats/ClientSocketStatsTest.java b/test/unit/voldemort/store/stats/ClientSocketStatsTest.java index bd89713c93..9f0a5ae3e8 100644 --- a/test/unit/voldemort/store/stats/ClientSocketStatsTest.java +++ b/test/unit/voldemort/store/stats/ClientSocketStatsTest.java @@ -171,4 +171,63 @@ public void testSetMonitoringInterval() { assertEquals(-1, child2.getAveWaitUs()); assertEquals(0, child2.getConnectionsCheckedout()); } + + @Test + public void concurrentTest() { + SocketDestination dest3 = new SocketDestination("localhost", + dest1.getPort() + 2, + RequestFormatType.VOLDEMORT_V1); + SocketDestination dest4 = new SocketDestination("localhost", + dest1.getPort() + 3, + RequestFormatType.VOLDEMORT_V1); + class TestThread implements Runnable { + + SocketDestination dest; + + public TestThread(SocketDestination dest) { + this.dest = dest; + } + + public void run() { + masterStats.recordCheckoutTimeUs(dest, 1000); + masterStats.recordCheckoutTimeUs(dest, 2000); + masterStats.recordCheckoutTimeUs(dest, 3000); + masterStats.recordCheckoutTimeUs(dest, 4000); + masterStats.recordCheckoutTimeUs(dest, 5000); + masterStats.recordCheckoutTimeUs(dest, 6000); + masterStats.recordCheckoutTimeUs(dest, 7000); + masterStats.recordCheckoutTimeUs(dest, 8000); + masterStats.recordCheckoutTimeUs(dest, 9000); + } + } + Thread t1 = new Thread(new TestThread(dest1)); + Thread t1_1 = new Thread(new TestThread(dest1)); + Thread t1_2 = new Thread(new TestThread(dest1)); + Thread t1_3 = new Thread(new TestThread(dest1)); + Thread t2 = new Thread(new TestThread(dest2)); + Thread t2_1 = new Thread(new TestThread(dest2)); + t1.start(); + t2.start(); + t2_1.start(); + t1_1.start(); + t1_2.start(); + t1_3.start(); + try { + t1.join(); + t2.join(); + t2_1.join(); + t1_1.join(); + t1_2.join(); + t1_3.join(); + } catch(Exception e) {} + assertEquals(masterStats.getWaitHistogram().getQuantile(0.01), 1000); + assertEquals(masterStats.getWaitHistogram().getQuantile(0.5), 5000); + + assertEquals(masterStats.getStatsMap().get(dest1).getWaitHistogram().getQuantile(0.01), + 1000); + assertEquals(masterStats.getStatsMap().get(dest1).getWaitHistogram().getQuantile(0.5), 5000); + assertEquals(masterStats.getStatsMap().get(dest2).getWaitHistogram().getQuantile(0.01), + 1000); + assertEquals(masterStats.getStatsMap().get(dest2).getWaitHistogram().getQuantile(0.5), 5000); + } } From 10211c71f35b358ffc38bdbf7d75d6f0cf1c1480 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Fri, 1 Jun 2012 14:48:26 -0700 Subject: [PATCH 067/209] finer timeouts and partial getalls. --- .../client/AbstractStoreClientFactory.java | 2 +- src/java/voldemort/client/ClientConfig.java | 60 ++++++- .../voldemort/server/VoldemortConfig.java | 25 +++ .../server/storage/StorageService.java | 2 +- .../store/routed/PipelineRoutedStore.java | 46 +++-- .../voldemort/store/routed/RoutedStore.java | 7 +- .../store/routed/RoutedStoreFactory.java | 11 +- .../store/routed/ThreadPoolRoutedStore.java | 31 +++- .../action/PerformParallelGetAllRequests.java | 2 +- .../action/PerformSerialGetAllRequests.java | 34 ++-- src/java/voldemort/utils/TimeoutConfig.java | 114 ++++++++++++ .../ClientConnectionStressTest.java | 39 ++-- .../RoutedStoreParallelismTest.java | 10 +- .../store/routed/HintedHandoffTest.java | 5 +- .../store/routed/ReadRepairerTest.java | 15 +- .../store/routed/RoutedStoreTest.java | 167 ++++++++++++++++-- 16 files changed, 474 insertions(+), 96 deletions(-) create mode 100644 src/java/voldemort/utils/TimeoutConfig.java diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index ab0c7d998e..bf523cfdc6 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -114,7 +114,7 @@ public AbstractStoreClientFactory(ClientConfig config) { this.clientZoneId = config.getClientZoneId(); this.routedStoreFactory = new RoutedStoreFactory(config.isPipelineRoutedStoreEnabled(), threadPool, - config.getRoutingTimeout(TimeUnit.MILLISECONDS)); + config.getTimeoutConfig()); if(this.isJmxEnabled) { JmxUtils.registerMbean(threadPool, diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index f74f31c04d..19ab3bde3a 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -36,6 +36,7 @@ import voldemort.utils.ConfigurationException; import voldemort.utils.Props; import voldemort.utils.ReflectUtils; +import voldemort.utils.TimeoutConfig; import voldemort.utils.Utils; /** @@ -55,6 +56,7 @@ public class ClientConfig { private volatile boolean socketKeepAlive = false; private volatile int selectors = 8; private volatile long routingTimeoutMs = 15000; + private volatile TimeoutConfig timeoutConfig = new TimeoutConfig(routingTimeoutMs, false); private volatile int socketBufferSize = 64 * 1024; private volatile SerializerFactory serializerFactory = new DefaultSerializerFactory(); private volatile List bootstrapUrls = null; @@ -91,6 +93,12 @@ public ClientConfig() {} public static final String SOCKET_KEEPALIVE_PROPERTY = "socket_keepalive"; public static final String SELECTORS_PROPERTY = "selectors"; public static final String ROUTING_TIMEOUT_MS_PROPERTY = "routing_timeout_ms"; + public static final String GETALL_ROUTING_TIMEOUT_MS_PROPERTY = "getall_routing_timeout_ms"; + public static final String PUT_ROUTING_TIMEOUT_MS_PROPERTY = "put_routing_timeout_ms"; + public static final String GET_ROUTING_TIMEOUT_MS_PROPERTY = "get_routing_timeout_ms"; + public static final String GET_VERSIONS_ROUTING_TIMEOUT_MS_PROPERTY = "getversions_routing_timeout_ms"; + public static final String DELETE_ROUTING_TIMEOUT_MS_PROPERTY = "delete_routing_timeout_ms"; + public static final String ALLOW_PARTIAL_GETALLS_PROPERTY = "allow_partial_getalls"; public static final String NODE_BANNAGE_MS_PROPERTY = "node_bannage_ms"; public static final String SOCKET_BUFFER_SIZE_PROPERTY = "socket_buffer_size"; public static final String SERIALIZER_FACTORY_CLASS_PROPERTY = "serializer_factory_class"; @@ -174,6 +182,36 @@ private void setProperties(Properties properties) { if(props.containsKey(ROUTING_TIMEOUT_MS_PROPERTY)) this.setRoutingTimeout(props.getInt(ROUTING_TIMEOUT_MS_PROPERTY), TimeUnit.MILLISECONDS); + // By default, make all the timeouts equal to routing timeout + timeoutConfig = new TimeoutConfig(routingTimeoutMs, false); + + if(props.containsKey(GETALL_ROUTING_TIMEOUT_MS_PROPERTY)) + timeoutConfig.getAllTimeoutMs(props.getInt(GETALL_ROUTING_TIMEOUT_MS_PROPERTY), + TimeUnit.MILLISECONDS); + + if(props.containsKey(GET_ROUTING_TIMEOUT_MS_PROPERTY)) + timeoutConfig.getTimeoutMs(props.getInt(GET_ROUTING_TIMEOUT_MS_PROPERTY), + TimeUnit.MILLISECONDS); + + if(props.containsKey(PUT_ROUTING_TIMEOUT_MS_PROPERTY)) { + long putTimeoutMs = props.getInt(PUT_ROUTING_TIMEOUT_MS_PROPERTY); + timeoutConfig.putTimeoutMs(putTimeoutMs, TimeUnit.MILLISECONDS); + // By default, use the same thing for getVersions() also + timeoutConfig.getVersionsTimeoutMs(putTimeoutMs, TimeUnit.MILLISECONDS); + } + + // of course, if someone overrides it, we will respect that + if(props.containsKey(GET_VERSIONS_ROUTING_TIMEOUT_MS_PROPERTY)) + timeoutConfig.getVersionsTimeoutMs(props.getInt(GET_VERSIONS_ROUTING_TIMEOUT_MS_PROPERTY), + TimeUnit.MILLISECONDS); + + if(props.containsKey(DELETE_ROUTING_TIMEOUT_MS_PROPERTY)) + timeoutConfig.deleteTimeoutMs(props.getInt(DELETE_ROUTING_TIMEOUT_MS_PROPERTY), + TimeUnit.MILLISECONDS); + + if(props.containsKey(ALLOW_PARTIAL_GETALLS_PROPERTY)) + timeoutConfig.setPartialGetAllAllowed(props.getBoolean(ALLOW_PARTIAL_GETALLS_PROPERTY)); + if(props.containsKey(SOCKET_BUFFER_SIZE_PROPERTY)) this.setSocketBufferSize(props.getInt(SOCKET_BUFFER_SIZE_PROPERTY)); @@ -324,6 +362,26 @@ public ClientConfig setRoutingTimeout(int routingTimeout, TimeUnit unit) { return this; } + /** + * Set the timeout configuration for the voldemort operations + * + * @param tConfig + * @return + */ + public ClientConfig setTimeoutConfig(TimeoutConfig tConfig) { + this.timeoutConfig = tConfig; + return this; + } + + /** + * Get the timeouts for voldemort operations + * + * @return + */ + public TimeoutConfig getTimeoutConfig() { + return timeoutConfig; + } + /** * @deprecated Use {@link #getFailureDetectorBannagePeriod()} instead */ @@ -517,7 +575,7 @@ public boolean isLazyEnabled() { /** * Enable lazy initialization of clients? - * + * * @param enableLazy If true clients will be lazily initialized */ public ClientConfig setEnableLazy(boolean enableLazy) { diff --git a/src/java/voldemort/server/VoldemortConfig.java b/src/java/voldemort/server/VoldemortConfig.java index 119c37040c..ae1991d6fa 100644 --- a/src/java/voldemort/server/VoldemortConfig.java +++ b/src/java/voldemort/server/VoldemortConfig.java @@ -21,6 +21,7 @@ import java.io.Serializable; import java.util.List; import java.util.Properties; +import java.util.concurrent.TimeUnit; import voldemort.client.protocol.RequestFormatType; import voldemort.cluster.failuredetector.FailureDetectorConfig; @@ -34,6 +35,7 @@ import voldemort.utils.ConfigurationException; import voldemort.utils.Props; import voldemort.utils.Time; +import voldemort.utils.TimeoutConfig; import voldemort.utils.UndefinedPropertyException; import voldemort.utils.Utils; @@ -112,6 +114,7 @@ public class VoldemortConfig implements Serializable { private int clientSelectors; private int clientRoutingTimeoutMs; + private TimeoutConfig clientTimeoutConfig; private int clientMaxConnectionsPerNode; private int clientConnectionTimeoutMs; private int clientMaxThreads; @@ -275,6 +278,24 @@ public VoldemortConfig(Props props) { this.clientMaxConnectionsPerNode = props.getInt("client.max.connections.per.node", 50); this.clientConnectionTimeoutMs = props.getInt("client.connection.timeout.ms", 500); this.clientRoutingTimeoutMs = props.getInt("client.routing.timeout.ms", 15000); + this.clientTimeoutConfig = new TimeoutConfig(this.clientRoutingTimeoutMs, false); + this.clientTimeoutConfig.getTimeoutMs(props.getInt("client.routing.get.timeout.ms", + this.clientRoutingTimeoutMs), + TimeUnit.MILLISECONDS); + this.clientTimeoutConfig.getAllTimeoutMs(props.getInt("client.routing.getall.timeout.ms", + this.clientRoutingTimeoutMs), + TimeUnit.MILLISECONDS); + this.clientTimeoutConfig.putTimeoutMs(props.getInt("client.routing.put.timeout.ms", + this.clientRoutingTimeoutMs), + TimeUnit.MILLISECONDS); + this.clientTimeoutConfig.getVersionsTimeoutMs(props.getLong("client.routing.getversions.timeout.ms", + this.clientTimeoutConfig.putTimeoutMs()), + TimeUnit.MILLISECONDS); + this.clientTimeoutConfig.deleteTimeoutMs(props.getInt("client.routing.delete.timeout.ms", + this.clientRoutingTimeoutMs), + TimeUnit.MILLISECONDS); + this.clientTimeoutConfig.setPartialGetAllAllowed(props.getBoolean("client.routing.allow.partial.getall", + false)); this.clientMaxThreads = props.getInt("client.max.threads", 500); this.clientThreadIdleMs = props.getInt("client.thread.idle.ms", 100000); this.clientMaxQueuedRequests = props.getInt("client.max.queued.requests", 1000); @@ -997,6 +1018,10 @@ public void setClientRoutingTimeoutMs(int routingTimeoutMs) { this.clientRoutingTimeoutMs = routingTimeoutMs; } + public TimeoutConfig getTimeoutConfig() { + return this.clientTimeoutConfig; + } + public int getClientMaxConnectionsPerNode() { return clientMaxConnectionsPerNode; } diff --git a/src/java/voldemort/server/storage/StorageService.java b/src/java/voldemort/server/storage/StorageService.java index 5a26c3e86d..8bb3919c6f 100644 --- a/src/java/voldemort/server/storage/StorageService.java +++ b/src/java/voldemort/server/storage/StorageService.java @@ -155,7 +155,7 @@ public StorageService(StoreRepository storeRepository, this.storeStats = new StoreStats(); this.routedStoreFactory = new RoutedStoreFactory(voldemortConfig.isPipelineRoutedStoreEnabled(), this.clientThreadPool, - voldemortConfig.getClientRoutingTimeoutMs()); + voldemortConfig.getTimeoutConfig()); /* * Initialize the dynamic throttle limit based on the per node limit diff --git a/src/java/voldemort/store/routed/PipelineRoutedStore.java b/src/java/voldemort/store/routed/PipelineRoutedStore.java index c5aa1ba06b..9ed094c03c 100644 --- a/src/java/voldemort/store/routed/PipelineRoutedStore.java +++ b/src/java/voldemort/store/routed/PipelineRoutedStore.java @@ -58,6 +58,7 @@ import voldemort.utils.ByteUtils; import voldemort.utils.JmxUtils; import voldemort.utils.SystemTime; +import voldemort.utils.TimeoutConfig; import voldemort.versioning.Version; import voldemort.versioning.Versioned; @@ -102,7 +103,7 @@ public PipelineRoutedStore(String name, StoreDefinition storeDef, boolean repairReads, int clientZoneId, - long timeoutMs, + TimeoutConfig timeoutConfig, FailureDetector failureDetector, boolean jmxEnabled) { super(name, @@ -110,7 +111,7 @@ public PipelineRoutedStore(String name, cluster, storeDef, repairReads, - timeoutMs, + timeoutConfig, failureDetector, SystemTime.INSTANCE); this.nonblockingSlopStores = nonblockingSlopStores; @@ -149,7 +150,9 @@ public List> get(final ByteArray key, final byte[] transforms) pipelineData.setZonesRequired(null); pipelineData.setStats(stats); - final Pipeline pipeline = new Pipeline(Operation.GET, timeoutMs, TimeUnit.MILLISECONDS); + final Pipeline pipeline = new Pipeline(Operation.GET, + timeoutConfig.getTimeoutMs(), + TimeUnit.MILLISECONDS); boolean allowReadRepair = repairReads && transforms == null; StoreRequest>> blockingStoreRequest = new StoreRequest>>() { @@ -177,7 +180,7 @@ public List> request(Store store) { failureDetector, storeDef.getPreferredReads(), storeDef.getRequiredReads(), - timeoutMs, + timeoutConfig.getTimeoutMs(), nonblockingStores, Event.INSUFFICIENT_SUCCESSES, Event.INSUFFICIENT_ZONES)); @@ -198,7 +201,7 @@ public List> request(Store store) { new ReadRepair>>>(pipelineData, Event.COMPLETED, storeDef.getPreferredReads(), - timeoutMs, + timeoutConfig.getTimeoutMs(), nonblockingStores, readRepairer)); @@ -255,7 +258,9 @@ public Map>> getAll(Iterable keys, pipelineData.setZonesRequired(null); pipelineData.setStats(stats); - Pipeline pipeline = new Pipeline(Operation.GET_ALL, timeoutMs, TimeUnit.MILLISECONDS); + Pipeline pipeline = new Pipeline(Operation.GET_ALL, + timeoutConfig.getAllTimeoutMs(), + TimeUnit.MILLISECONDS); pipeline.addEventAction(Event.STARTED, new GetAllConfigureNodes(pipelineData, Event.CONFIGURED, @@ -270,7 +275,7 @@ public Map>> getAll(Iterable keys, new PerformParallelGetAllRequests(pipelineData, Event.INSUFFICIENT_SUCCESSES, failureDetector, - timeoutMs, + timeoutConfig.getAllTimeoutMs(), nonblockingStores)); pipeline.addEventAction(Event.INSUFFICIENT_SUCCESSES, new PerformSerialGetAllRequests(pipelineData, @@ -280,14 +285,15 @@ public Map>> getAll(Iterable keys, failureDetector, innerStores, storeDef.getPreferredReads(), - storeDef.getRequiredReads())); + storeDef.getRequiredReads(), + timeoutConfig.isPartialGetAllAllowed())); if(allowReadRepair) pipeline.addEventAction(Event.RESPONSES_RECEIVED, new GetAllReadRepair(pipelineData, Event.COMPLETED, storeDef.getPreferredReads(), - timeoutMs, + timeoutConfig.getAllTimeoutMs(), nonblockingStores, readRepairer)); @@ -323,7 +329,9 @@ public List getVersions(final ByteArray key) { else pipelineData.setZonesRequired(null); pipelineData.setStats(stats); - Pipeline pipeline = new Pipeline(Operation.GET_VERSIONS, timeoutMs, TimeUnit.MILLISECONDS); + Pipeline pipeline = new Pipeline(Operation.GET_VERSIONS, + timeoutConfig.getVersionsTimeoutMs(), + TimeUnit.MILLISECONDS); StoreRequest> blockingStoreRequest = new StoreRequest>() { @@ -349,7 +357,7 @@ public List request(Store store) { failureDetector, storeDef.getPreferredReads(), storeDef.getRequiredReads(), - timeoutMs, + timeoutConfig.getVersionsTimeoutMs(), nonblockingStores, Event.INSUFFICIENT_SUCCESSES, Event.INSUFFICIENT_ZONES)); @@ -408,7 +416,9 @@ public boolean delete(final ByteArray key, final Version version) throws Voldemo pipelineData.setStoreName(name); pipelineData.setStats(stats); - Pipeline pipeline = new Pipeline(Operation.DELETE, timeoutMs, TimeUnit.MILLISECONDS); + Pipeline pipeline = new Pipeline(Operation.DELETE, + timeoutConfig.deleteTimeoutMs(), + TimeUnit.MILLISECONDS); pipeline.setEnableHintedHandoff(isHintedHandoffEnabled()); HintedHandoff hintedHandoff = null; @@ -419,7 +429,7 @@ public boolean delete(final ByteArray key, final Version version) throws Voldemo nonblockingSlopStores, handoffStrategy, pipelineData.getFailedNodes(), - timeoutMs); + timeoutConfig.deleteTimeoutMs()); pipeline.addEventAction(Event.STARTED, new ConfigureNodes>(pipelineData, @@ -437,7 +447,7 @@ public boolean delete(final ByteArray key, final Version version) throws Voldemo failureDetector, storeDef.getPreferredWrites(), storeDef.getRequiredWrites(), - timeoutMs, + timeoutConfig.deleteTimeoutMs(), nonblockingStores, hintedHandoff, version)); @@ -496,7 +506,9 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) pipelineData.setStoreName(name); pipelineData.setStats(stats); - Pipeline pipeline = new Pipeline(Operation.PUT, timeoutMs, TimeUnit.MILLISECONDS); + Pipeline pipeline = new Pipeline(Operation.PUT, + timeoutConfig.putTimeoutMs(), + TimeUnit.MILLISECONDS); pipeline.setEnableHintedHandoff(isHintedHandoffEnabled()); HintedHandoff hintedHandoff = null; @@ -507,7 +519,7 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) nonblockingSlopStores, handoffStrategy, pipelineData.getFailedNodes(), - timeoutMs); + timeoutConfig.putTimeoutMs()); pipeline.addEventAction(Event.STARTED, new ConfigureNodes(pipelineData, @@ -537,7 +549,7 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) failureDetector, storeDef.getPreferredWrites(), storeDef.getRequiredWrites(), - timeoutMs, + timeoutConfig.putTimeoutMs(), nonblockingStores, hintedHandoff)); if(isHintedHandoffEnabled()) { diff --git a/src/java/voldemort/store/routed/RoutedStore.java b/src/java/voldemort/store/routed/RoutedStore.java index 613f195bf8..ad2eb4a14d 100644 --- a/src/java/voldemort/store/routed/RoutedStore.java +++ b/src/java/voldemort/store/routed/RoutedStore.java @@ -32,6 +32,7 @@ import voldemort.store.StoreDefinition; import voldemort.utils.ByteArray; import voldemort.utils.Time; +import voldemort.utils.TimeoutConfig; import voldemort.utils.Utils; /** @@ -45,7 +46,7 @@ public abstract class RoutedStore implements Store { protected final Map> innerStores; protected final boolean repairReads; protected final ReadRepairer readRepairer; - protected final long timeoutMs; + protected final TimeoutConfig timeoutConfig; protected final Time time; protected final StoreDefinition storeDef; protected final FailureDetector failureDetector; @@ -57,7 +58,7 @@ protected RoutedStore(String name, Cluster cluster, StoreDefinition storeDef, boolean repairReads, - long timeoutMs, + TimeoutConfig timeoutConfig, FailureDetector failureDetector, Time time) { if(storeDef.getRequiredReads() < 1) @@ -77,7 +78,7 @@ protected RoutedStore(String name, this.innerStores = new ConcurrentHashMap>(innerStores); this.repairReads = repairReads; this.readRepairer = new ReadRepairer(); - this.timeoutMs = timeoutMs; + this.timeoutConfig = timeoutConfig; this.time = Utils.notNull(time); this.storeDef = storeDef; this.failureDetector = failureDetector; diff --git a/src/java/voldemort/store/routed/RoutedStoreFactory.java b/src/java/voldemort/store/routed/RoutedStoreFactory.java index 0a608bac4d..270e959fa0 100644 --- a/src/java/voldemort/store/routed/RoutedStoreFactory.java +++ b/src/java/voldemort/store/routed/RoutedStoreFactory.java @@ -18,6 +18,7 @@ import voldemort.store.slop.Slop; import voldemort.utils.ByteArray; import voldemort.utils.SystemTime; +import voldemort.utils.TimeoutConfig; import com.google.common.collect.Maps; @@ -27,16 +28,16 @@ public class RoutedStoreFactory { private final ExecutorService threadPool; - private final long routingTimeoutMs; + private final TimeoutConfig timeoutConfig; private final Logger logger = Logger.getLogger(getClass()); public RoutedStoreFactory(boolean isPipelineRoutedStoreEnabled, ExecutorService threadPool, - long routingTimeoutMs) { + TimeoutConfig timeoutConfig) { this.isPipelineRoutedStoreEnabled = isPipelineRoutedStoreEnabled; this.threadPool = threadPool; - this.routingTimeoutMs = routingTimeoutMs; + this.timeoutConfig = timeoutConfig; } public NonblockingStore toNonblockingStore(Store store) { @@ -90,7 +91,7 @@ public RoutedStore create(Cluster cluster, storeDefinition, repairReads, clientZoneId, - routingTimeoutMs, + timeoutConfig, failureDetector, jmxEnabled); } else { @@ -111,7 +112,7 @@ public RoutedStore create(Cluster cluster, storeDefinition, repairReads, threadPool, - routingTimeoutMs, + timeoutConfig, failureDetector, SystemTime.INSTANCE); } diff --git a/src/java/voldemort/store/routed/ThreadPoolRoutedStore.java b/src/java/voldemort/store/routed/ThreadPoolRoutedStore.java index 9fec120cd2..5a8a667dff 100644 --- a/src/java/voldemort/store/routed/ThreadPoolRoutedStore.java +++ b/src/java/voldemort/store/routed/ThreadPoolRoutedStore.java @@ -48,6 +48,7 @@ import voldemort.utils.ByteUtils; import voldemort.utils.SystemTime; import voldemort.utils.Time; +import voldemort.utils.TimeoutConfig; import voldemort.versioning.ObsoleteVersionException; import voldemort.versioning.VectorClock; import voldemort.versioning.Version; @@ -102,7 +103,7 @@ public ThreadPoolRoutedStore(String name, StoreDefinition storeDef, int numberOfThreads, boolean repairReads, - long timeoutMs, + TimeoutConfig timeoutConfig, FailureDetector failureDetector) { this(name, innerStores, @@ -110,7 +111,7 @@ public ThreadPoolRoutedStore(String name, storeDef, repairReads, Executors.newFixedThreadPool(numberOfThreads), - timeoutMs, + timeoutConfig, failureDetector, SystemTime.INSTANCE); } @@ -134,10 +135,17 @@ public ThreadPoolRoutedStore(String name, StoreDefinition storeDef, boolean repairReads, ExecutorService threadPool, - long timeoutMs, + TimeoutConfig timeoutConfig, FailureDetector failureDetector, Time time) { - super(name, innerStores, cluster, storeDef, repairReads, timeoutMs, failureDetector, time); + super(name, + innerStores, + cluster, + storeDef, + repairReads, + timeoutConfig, + failureDetector, + time); this.executor = threadPool; } @@ -184,7 +192,8 @@ public void run() { } catch(Exception e) { failures.add(e); logger.warn("Error in DELETE on node " + node.getId() + "(" - + node.getHost() + ")", e); + + node.getHost() + ")", + e); } finally { // signal that the operation is complete semaphore.release(); @@ -199,6 +208,7 @@ public void run() { } else { for(int i = 0; i < numNodes; i++) { try { + long timeoutMs = timeoutConfig.deleteTimeoutMs(); boolean acquired = semaphore.tryAcquire(timeoutMs, TimeUnit.MILLISECONDS); if(!acquired) logger.warn("Delete operation timed out waiting for operation " + i @@ -292,6 +302,7 @@ public Map>> getAll(Iterable keys, keyToSuccessCount.put(key, new MutableInt(0)); List> futures; + long timeoutMs = timeoutConfig.getAllTimeoutMs(); try { // TODO What to do about timeouts? They should be longer as getAll // is likely to @@ -377,7 +388,8 @@ public Map>> getAll(Iterable keys, throw e; } catch(Exception e) { logger.warn("Error in GET_ALL on node " + node.getId() + "(" - + node.getHost() + ")", e); + + node.getHost() + ")", + e); failures.add(e); } } @@ -453,6 +465,8 @@ private List get(final ByteArray key, } List>> futures; + long timeoutMs = (fetcher == VERSION_OP) ? timeoutConfig.getVersionsTimeoutMs() + : timeoutConfig.getTimeoutMs(); try { futures = executor.invokeAll(callables, timeoutMs, TimeUnit.MILLISECONDS); } catch(InterruptedException e) { @@ -498,8 +512,7 @@ private List get(final ByteArray key, key, fetcher.execute(innerStores.get(node.getId()), key, - transforms), - null)); + transforms), null)); ++successes; recordSuccess(node, startNs); } catch(UnreachableStoreException e) { @@ -760,7 +773,7 @@ private boolean blockOnPut(long startNs, for(int i = startingIndex; i < blockCount; i++) { try { long ellapsedNs = System.nanoTime() - startNs; - long remainingNs = (timeoutMs * Time.NS_PER_MS) - ellapsedNs; + long remainingNs = (timeoutConfig.putTimeoutMs() * Time.NS_PER_MS) - ellapsedNs; boolean acquiredPermit = semaphore.tryAcquire(Math.max(remainingNs, 0), TimeUnit.NANOSECONDS); if(!acquiredPermit) { diff --git a/src/java/voldemort/store/routed/action/PerformParallelGetAllRequests.java b/src/java/voldemort/store/routed/action/PerformParallelGetAllRequests.java index a6bc581a7d..444fdd930e 100644 --- a/src/java/voldemort/store/routed/action/PerformParallelGetAllRequests.java +++ b/src/java/voldemort/store/routed/action/PerformParallelGetAllRequests.java @@ -119,7 +119,7 @@ public void requestComplete(Object result, long requestTime) { } try { - latch.await(timeoutMs * 3, TimeUnit.MILLISECONDS); + latch.await(timeoutMs, TimeUnit.MILLISECONDS); } catch(InterruptedException e) { if(logger.isEnabledFor(Level.WARN)) logger.warn(e, e); diff --git a/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java b/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java index bc8cf957e6..059545e90b 100644 --- a/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java +++ b/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java @@ -52,19 +52,23 @@ public class PerformSerialGetAllRequests private final int required; + private final boolean allowPartial; + public PerformSerialGetAllRequests(GetAllPipelineData pipelineData, Event completeEvent, Iterable keys, FailureDetector failureDetector, Map> stores, int preferred, - int required) { + int required, + boolean allowPartial) { super(pipelineData, completeEvent); this.keys = keys; this.failureDetector = failureDetector; this.stores = stores; this.preferred = preferred; this.required = required; + this.allowPartial = allowPartial; } public void execute(Pipeline pipeline) { @@ -156,17 +160,23 @@ public void execute(Pipeline pipeline) { MutableInt successCount = pipelineData.getSuccessCount(key); if(successCount.intValue() < required) { - pipelineData.setFatalError(new InsufficientOperationalNodesException(required - + " " - + pipeline.getOperation() - .getSimpleName() - + "s required, but " - + successCount.intValue() - + " succeeded. Failing nodes : " - + pipelineData.getFailedNodes(), - pipelineData.getFailures())); - pipeline.addEvent(Event.ERROR); - return; + // if we allow partial results, then just remove keys that did + // not meet 'required' guarantee; else raise error + if(allowPartial) { + result.remove(key); + } else { + pipelineData.setFatalError(new InsufficientOperationalNodesException(required + + " " + + pipeline.getOperation() + .getSimpleName() + + "s required, but " + + successCount.intValue() + + " succeeded. Failing nodes : " + + pipelineData.getFailedNodes(), + pipelineData.getFailures())); + pipeline.addEvent(Event.ERROR); + return; + } } } diff --git a/src/java/voldemort/utils/TimeoutConfig.java b/src/java/voldemort/utils/TimeoutConfig.java new file mode 100644 index 0000000000..1752d1eab2 --- /dev/null +++ b/src/java/voldemort/utils/TimeoutConfig.java @@ -0,0 +1,114 @@ +package voldemort.utils; + +import java.util.concurrent.TimeUnit; + +/** + * Encapsulates the timeouts for various voldemort operations + * + */ +public class TimeoutConfig { + + private long getTimeoutMs; + + private long putTimeoutMs; + + private long getAllTimeoutMs; + + private long deleteTimeoutMs; + + private long getVersionsTimeoutMs; + + private boolean partialGetAllAllowed; + + public TimeoutConfig(long globalTimeout, boolean allowPartialGetAlls) { + this(globalTimeout, + globalTimeout, + globalTimeout, + globalTimeout, + globalTimeout, + allowPartialGetAlls); + } + + public TimeoutConfig(long getTimeout, + long putTimeout, + long deleteTimeout, + long getAllTimeout, + long getVersionsTimeout, + boolean allowPartialGetAlls) { + getTimeoutMs(getTimeout, TimeUnit.MILLISECONDS); + putTimeoutMs(putTimeout, TimeUnit.MILLISECONDS); + deleteTimeoutMs(deleteTimeout, TimeUnit.MILLISECONDS); + getAllTimeoutMs(getAllTimeout, TimeUnit.MILLISECONDS); + getVersionsTimeoutMs(getVersionsTimeout, TimeUnit.MILLISECONDS); + setPartialGetAllAllowed(allowPartialGetAlls); + } + + public long getTimeoutMs(TimeUnit unit) { + return unit.convert(getTimeoutMs, TimeUnit.MILLISECONDS); + } + + public long getTimeoutMs() { + return getTimeoutMs; + } + + public void getTimeoutMs(long getTimeoutMs, TimeUnit unit) { + this.getTimeoutMs = unit.toMillis(getTimeoutMs); + } + + public long getVersionsTimeoutMs(TimeUnit unit) { + return unit.convert(getVersionsTimeoutMs, TimeUnit.MILLISECONDS); + } + + public long getVersionsTimeoutMs() { + return getVersionsTimeoutMs; + } + + public void getVersionsTimeoutMs(long getTimeoutMs, TimeUnit unit) { + this.getVersionsTimeoutMs = unit.toMillis(getTimeoutMs); + } + + public long putTimeoutMs(TimeUnit unit) { + return unit.convert(putTimeoutMs, TimeUnit.MILLISECONDS); + } + + public long putTimeoutMs() { + return putTimeoutMs; + } + + public void putTimeoutMs(long putTimeoutMs, TimeUnit unit) { + this.putTimeoutMs = unit.toMillis(putTimeoutMs); + } + + public long getAllTimeoutMs(TimeUnit unit) { + return unit.convert(getAllTimeoutMs, TimeUnit.MILLISECONDS); + } + + public long getAllTimeoutMs() { + return getAllTimeoutMs; + } + + public void getAllTimeoutMs(long getAllTimeoutMs, TimeUnit unit) { + this.getAllTimeoutMs = unit.toMillis(getAllTimeoutMs); + } + + public long deleteTimeoutMs(TimeUnit unit) { + return unit.convert(deleteTimeoutMs, TimeUnit.MILLISECONDS); + } + + public long deleteTimeoutMs() { + return deleteTimeoutMs; + } + + public void deleteTimeoutMs(long deleteTimeoutMs, TimeUnit unit) { + this.deleteTimeoutMs = unit.toMillis(deleteTimeoutMs); + } + + public boolean isPartialGetAllAllowed() { + return partialGetAllAllowed; + } + + public void setPartialGetAllAllowed(boolean allowPartialGetAlls) { + this.partialGetAllAllowed = allowPartialGetAlls; + } + +} diff --git a/test/integration/voldemort/performance/ClientConnectionStressTest.java b/test/integration/voldemort/performance/ClientConnectionStressTest.java index a26beb984e..6bf7e23ada 100644 --- a/test/integration/voldemort/performance/ClientConnectionStressTest.java +++ b/test/integration/voldemort/performance/ClientConnectionStressTest.java @@ -1,12 +1,12 @@ /* * Copyright 2008-2010 LinkedIn, Inc - * + * * Licensed under the Apache License, Version 2.0 (the "License"); you may not * use this file except in compliance with the License. You may obtain a copy of * the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the @@ -16,6 +16,12 @@ package voldemort.performance; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + import joptsimple.OptionParser; import joptsimple.OptionSet; import voldemort.client.ClientConfig; @@ -23,12 +29,7 @@ import voldemort.client.StoreClient; import voldemort.client.StoreClientFactory; import voldemort.utils.CmdUtils; - -import java.util.List; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; +import voldemort.utils.TimeoutConfig; /** * Stress tests the client. Intended to diagnose issues such as connection leaks @@ -92,8 +93,6 @@ public void run() { executor.shutdown(); } - - public static void main(String[] args) throws Exception { OptionParser parser = new OptionParser(); @@ -112,18 +111,14 @@ public static void main(String[] args) throws Exception { parser.accepts(MAX_CONNECTIONS_TOTAL, "Max total connections") .withRequiredArg() .ofType(Integer.class); - parser.accepts(MAX_THREADS, "Max threads") - .withRequiredArg() - .ofType(Integer.class); + parser.accepts(MAX_THREADS, "Max threads").withRequiredArg().ofType(Integer.class); parser.accepts(SELECTORS, "Number of NIO selectors") .withRequiredArg() .ofType(Integer.class); parser.accepts(SOCKET_BUFFER_SIZE, "Socket buffer size") - .withRequiredArg() - .ofType(Integer.class); - parser.accepts(REQS, "Requests per session") .withRequiredArg() .ofType(Integer.class); + parser.accepts(REQS, "Requests per session").withRequiredArg().ofType(Integer.class); parser.accepts(CONNECTIONS, "Total connections to make") .withRequiredArg() .ofType(Integer.class); @@ -144,11 +139,15 @@ public static void main(String[] args) throws Exception { ClientConfig config = new ClientConfig(); if(options.has(CONNECTION_TIMEOUT)) - config.setConnectionTimeout((Integer) options.valueOf(CONNECTION_TIMEOUT), TimeUnit.MILLISECONDS); + config.setConnectionTimeout((Integer) options.valueOf(CONNECTION_TIMEOUT), + TimeUnit.MILLISECONDS); if(options.has(ROUTING_TIMEOUT)) - config.setRoutingTimeout((Integer) options.valueOf(ROUTING_TIMEOUT), TimeUnit.MILLISECONDS); + config.setTimeoutConfig(new TimeoutConfig(TimeUnit.MILLISECONDS.toMillis((Integer) options.valueOf(ROUTING_TIMEOUT)), + false)); + if(options.has(SOCKET_TIMEOUT)) - config.setSocketTimeout((Integer) options.valueOf(SOCKET_TIMEOUT), TimeUnit.MILLISECONDS); + config.setSocketTimeout((Integer) options.valueOf(SOCKET_TIMEOUT), + TimeUnit.MILLISECONDS); if(options.has(MAX_CONNECTIONS)) config.setMaxConnectionsPerNode((Integer) options.valueOf(MAX_CONNECTIONS)); if(options.has(MAX_THREADS)) diff --git a/test/integration/voldemort/performance/RoutedStoreParallelismTest.java b/test/integration/voldemort/performance/RoutedStoreParallelismTest.java index 0bbbddb76b..0397416613 100644 --- a/test/integration/voldemort/performance/RoutedStoreParallelismTest.java +++ b/test/integration/voldemort/performance/RoutedStoreParallelismTest.java @@ -28,7 +28,6 @@ import joptsimple.OptionParser; import joptsimple.OptionSet; -import voldemort.cluster.failuredetector.MutableStoreVerifier; import voldemort.ServerTestUtils; import voldemort.TestUtils; import voldemort.VoldemortException; @@ -39,6 +38,7 @@ import voldemort.cluster.failuredetector.FailureDetector; import voldemort.cluster.failuredetector.FailureDetectorConfig; import voldemort.cluster.failuredetector.FailureDetectorUtils; +import voldemort.cluster.failuredetector.MutableStoreVerifier; import voldemort.server.StoreRepository; import voldemort.server.VoldemortConfig; import voldemort.server.VoldemortServer; @@ -101,7 +101,9 @@ public static void main(String[] args) throws Throwable { .ofType(Integer.class); parser.accepts("num-clients", "The number of threads to make requests concurrently Default = " - + DEFAULT_NUM_CLIENTS).withRequiredArg().ofType(Integer.class); + + DEFAULT_NUM_CLIENTS) + .withRequiredArg() + .ofType(Integer.class); parser.accepts("routed-store-type", "Type of routed store, either \"" + THREAD_POOL_ROUTED_STORE + "\" or \"" + PIPELINE_ROUTED_STORE + "\" Default = " @@ -201,7 +203,7 @@ public static void main(String[] args) throws Throwable { RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(routedStoreType.trim() .equalsIgnoreCase(PIPELINE_ROUTED_STORE), routedStoreThreadPool, - clientConfig.getRoutingTimeout(TimeUnit.MILLISECONDS)); + clientConfig.getTimeoutConfig()); final RoutedStore routedStore = routedStoreFactory.create(cluster, storeDefinition, @@ -223,7 +225,7 @@ public void run() { try { routedStore.get(key, null); } catch(VoldemortException e) { - // + // } } } diff --git a/test/unit/voldemort/store/routed/HintedHandoffTest.java b/test/unit/voldemort/store/routed/HintedHandoffTest.java index 64d1662192..653ccb2d70 100644 --- a/test/unit/voldemort/store/routed/HintedHandoffTest.java +++ b/test/unit/voldemort/store/routed/HintedHandoffTest.java @@ -61,6 +61,7 @@ import voldemort.store.slop.strategy.HintedHandoffStrategyType; import voldemort.utils.ByteArray; import voldemort.utils.ByteUtils; +import voldemort.utils.TimeoutConfig; import voldemort.versioning.Version; import voldemort.versioning.Versioned; @@ -168,7 +169,9 @@ public void setUp() throws Exception { setFailureDetector(subStores); routedStoreThreadPool = Executors.newFixedThreadPool(NUM_THREADS); - routedStoreFactory = new RoutedStoreFactory(true, routedStoreThreadPool, 1500L); + routedStoreFactory = new RoutedStoreFactory(true, + routedStoreThreadPool, + new TimeoutConfig(1500L, false)); strategy = new RoutingStrategyFactory().updateRoutingStrategy(storeDef, cluster); Map nonblockingSlopStores = Maps.newHashMap(); diff --git a/test/unit/voldemort/store/routed/ReadRepairerTest.java b/test/unit/voldemort/store/routed/ReadRepairerTest.java index 3c0958f09b..7adfde569c 100644 --- a/test/unit/voldemort/store/routed/ReadRepairerTest.java +++ b/test/unit/voldemort/store/routed/ReadRepairerTest.java @@ -21,9 +21,9 @@ import static org.junit.Assert.assertEquals; import static voldemort.FailureDetectorTestUtils.recordException; import static voldemort.FailureDetectorTestUtils.recordSuccess; -import static voldemort.cluster.failuredetector.MutableStoreVerifier.create; import static voldemort.TestUtils.getClock; import static voldemort.cluster.failuredetector.FailureDetectorUtils.create; +import static voldemort.cluster.failuredetector.MutableStoreVerifier.create; import java.util.ArrayList; import java.util.Arrays; @@ -56,6 +56,7 @@ import voldemort.store.memory.InMemoryStorageEngine; import voldemort.utils.ByteArray; import voldemort.utils.Time; +import voldemort.utils.TimeoutConfig; import voldemort.versioning.Versioned; import com.google.common.collect.Iterables; @@ -157,7 +158,8 @@ public void testMissingKeysAreAddedToNodeWhenDoingReadRepair() throws Exception RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(isPipelineRoutedStoreEnabled, routedStoreThreadPool, - 1000L); + new TimeoutConfig(1000L, + false)); RoutedStore store = routedStoreFactory.create(cluster, storeDef, @@ -198,8 +200,8 @@ public void testNoDuplicates() throws Exception { public void testSingleSuccessor() throws Exception { assertVariationsEqual(singletonList(getValue(1, 1, new int[] { 1, 1 })), - asList(getValue(1, 1, new int[] { 1 }), getValue(2, 1, new int[] { 1, - 1 }))); + asList(getValue(1, 1, new int[] { 1 }), + getValue(2, 1, new int[] { 1, 1 }))); } public void testAllConcurrent() throws Exception { @@ -257,8 +259,9 @@ public void testConcurrentToOneDoesNotImplyConcurrentToAll() throws Exception { getValue(1, 1, new int[] { 1, 2 }), getValue(2, 1, new int[] { 1, 3, 3 }), getValue(3, 1, new int[] { 1, 2 })), - asList(getValue(1, 1, new int[] { 3, 3 }), getValue(2, 1, new int[] { - 1, 2 }), getValue(3, 1, new int[] { 1, 3, 3 }))); + asList(getValue(1, 1, new int[] { 3, 3 }), + getValue(2, 1, new int[] { 1, 2 }), + getValue(3, 1, new int[] { 1, 3, 3 }))); } public void testLotsOfVersions() throws Exception { diff --git a/test/unit/voldemort/store/routed/RoutedStoreTest.java b/test/unit/voldemort/store/routed/RoutedStoreTest.java index 4ec00e514b..8da982f487 100644 --- a/test/unit/voldemort/store/routed/RoutedStoreTest.java +++ b/test/unit/voldemort/store/routed/RoutedStoreTest.java @@ -18,10 +18,10 @@ import static voldemort.FailureDetectorTestUtils.recordException; import static voldemort.FailureDetectorTestUtils.recordSuccess; -import static voldemort.cluster.failuredetector.MutableStoreVerifier.create; import static voldemort.TestUtils.getClock; import static voldemort.VoldemortTestConstants.getNineNodeCluster; import static voldemort.cluster.failuredetector.FailureDetectorUtils.create; +import static voldemort.cluster.failuredetector.MutableStoreVerifier.create; import java.util.ArrayList; import java.util.Arrays; @@ -33,6 +33,7 @@ import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import org.junit.After; import org.junit.Before; @@ -73,6 +74,7 @@ import voldemort.utils.ByteArray; import voldemort.utils.ByteUtils; import voldemort.utils.Time; +import voldemort.utils.TimeoutConfig; import voldemort.utils.Utils; import voldemort.versioning.Occurred; import voldemort.versioning.VectorClock; @@ -202,7 +204,8 @@ else if(count < failing + sleepy) routedStoreThreadPool = Executors.newFixedThreadPool(threads); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(isPipelineRoutedStoreEnabled, routedStoreThreadPool, - 1000L); + new TimeoutConfig(1000L, + false)); return routedStoreFactory.create(cluster, storeDef, subStores, true, failureDetector); } @@ -252,7 +255,8 @@ else if(sleepy != null && sleepy.contains(n.getId())) routedStoreThreadPool = Executors.newFixedThreadPool(threads); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(true, routedStoreThreadPool, - timeOutMs); + new TimeoutConfig(timeOutMs, + false)); return routedStoreFactory.create(cluster, storeDef, subStores, true, failureDetector); } @@ -376,7 +380,7 @@ public void testPutIncrementsVersion() throws Exception { @Test public void testObsoleteMasterFails() { - // write me + // write me } @Test @@ -779,6 +783,75 @@ public void testGetAllWithNodeDown() throws Exception { } } + /** + * Tests that getAll returns partial results + */ + @Test + public void testPartialGetAll() throws Exception { + // create a store with rf=1 i.e disjoint partitions + StoreDefinition definition = new StoreDefinitionBuilder().setName("test") + .setType("foo") + .setKeySerializer(new SerializerDefinition("test")) + .setValueSerializer(new SerializerDefinition("test")) + .setRoutingPolicy(RoutingTier.CLIENT) + .setRoutingStrategyType(RoutingStrategyType.CONSISTENT_STRATEGY) + .setReplicationFactor(1) + .setPreferredReads(1) + .setRequiredReads(1) + .setPreferredWrites(1) + .setRequiredWrites(1) + .build(); + + Map> stores = new HashMap>(); + List nodes = new ArrayList(); + // create nodes with varying speeds - 100ms, 200ms, 300ms + for(int i = 0; i < 3; i++) { + Store store = new SleepyStore(100 * (i + 1), + new InMemoryStorageEngine("test")); + stores.put(i, store); + List partitions = Arrays.asList(i); + nodes.add(new Node(i, "none", 0, 0, 0, partitions)); + } + setFailureDetector(stores); + + routedStoreThreadPool = Executors.newFixedThreadPool(3); + + TimeoutConfig timeoutConfig = new TimeoutConfig(1500, true); + // This means, the getall will only succeed on two of the nodes + timeoutConfig.getAllTimeoutMs(250, TimeUnit.MILLISECONDS); + RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(true, + routedStoreThreadPool, + timeoutConfig); + + RoutedStore routedStore = routedStoreFactory.create(new Cluster("test", nodes), + definition, + stores, + true, + failureDetector); + /* do some puts so we have some data to test getalls */ + Map expectedValues = Maps.newHashMap(); + for(byte i = 1; i < 11; ++i) { + ByteArray key = new ByteArray(new byte[] { i }); + byte[] value = new byte[] { (byte) (i + 50) }; + routedStore.put(key, Versioned.value(value), null); + expectedValues.put(key, value); + } + + /* 1. positive test; if partial is on, should get something back */ + Map>> all = routedStore.getAll(expectedValues.keySet(), + null); + assert (expectedValues.size() > all.size()); + + /* 2. negative test; if partial is off, should fail the whole operation */ + timeoutConfig.setPartialGetAllAllowed(false); + try { + all = routedStore.getAll(expectedValues.keySet(), null); + fail("Should have failed"); + } catch(Exception e) { + + } + } + @Test public void testGetAllWithFailingStore() throws Exception { cluster = VoldemortTestConstants.getTwoNodeCluster(); @@ -802,7 +875,8 @@ public void testGetAllWithFailingStore() throws Exception { routedStoreThreadPool = Executors.newFixedThreadPool(1); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(isPipelineRoutedStoreEnabled, routedStoreThreadPool, - 1000L); + new TimeoutConfig(1000L, + false)); RoutedStore routedStore = routedStoreFactory.create(cluster, storeDef, @@ -858,7 +932,8 @@ public void testGetAllWithMorePreferredReadsThanNodes() throws Exception { routedStoreThreadPool = Executors.newFixedThreadPool(1); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(isPipelineRoutedStoreEnabled, routedStoreThreadPool, - 1000L); + new TimeoutConfig(1000L, + false)); RoutedStore routedStore = routedStoreFactory.create(cluster, storeDef, @@ -965,7 +1040,8 @@ public void testPutWithOneNodeDownAndOneNodeSlow() throws Exception { routedStoreThreadPool = Executors.newFixedThreadPool(1); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(isPipelineRoutedStoreEnabled, routedStoreThreadPool, - 1000L); + new TimeoutConfig(1000L, + false)); RoutedStore routedStore = routedStoreFactory.create(cluster, storeDef, @@ -1011,7 +1087,8 @@ public void testPutTimeout() throws Exception { routedStoreThreadPool = Executors.newFixedThreadPool(3); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(isPipelineRoutedStoreEnabled, routedStoreThreadPool, - timeout); + new TimeoutConfig(timeout, + false)); RoutedStore routedStore = routedStoreFactory.create(new Cluster("test", nodes), definition, @@ -1064,7 +1141,8 @@ public void testGetTimeout() throws Exception { routedStoreThreadPool = Executors.newFixedThreadPool(3); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(true, routedStoreThreadPool, - timeout); + new TimeoutConfig(timeout, + false)); RoutedStore routedStore = routedStoreFactory.create(new Cluster("test", nodes), definition, @@ -1082,6 +1160,62 @@ public void testGetTimeout() throws Exception { } } + @Test + public void testOperationSpecificTimeouts() throws Exception { + StoreDefinition definition = new StoreDefinitionBuilder().setName("test") + .setType("foo") + .setKeySerializer(new SerializerDefinition("test")) + .setValueSerializer(new SerializerDefinition("test")) + .setRoutingPolicy(RoutingTier.CLIENT) + .setRoutingStrategyType(RoutingStrategyType.CONSISTENT_STRATEGY) + .setReplicationFactor(3) + .setPreferredReads(3) + .setRequiredReads(3) + .setPreferredWrites(3) + .setRequiredWrites(3) + .build(); + Map> stores = new HashMap>(); + List nodes = new ArrayList(); + for(int i = 0; i < 3; i++) { + Store store = new SleepyStore(200, + new InMemoryStorageEngine("test")); + stores.put(i, store); + List partitions = Arrays.asList(i); + nodes.add(new Node(i, "none", 0, 0, 0, partitions)); + } + + setFailureDetector(stores); + + routedStoreThreadPool = Executors.newFixedThreadPool(3); + // with a 500ms general timeout and a 100ms get timeout, only get should + // fail + TimeoutConfig timeoutConfig = new TimeoutConfig(1500, false); + timeoutConfig.getTimeoutMs(100, TimeUnit.MILLISECONDS); + RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(true, + routedStoreThreadPool, + timeoutConfig); + + RoutedStore routedStore = routedStoreFactory.create(new Cluster("test", nodes), + definition, + stores, + true, + failureDetector); + try { + routedStore.put(new ByteArray("test".getBytes()), + new Versioned(new byte[] { 1 }), + null); + } catch(InsufficientOperationalNodesException e) { + fail("Should not have failed"); + } + + try { + routedStore.get(new ByteArray("test".getBytes()), null); + fail("Should have thrown"); + } catch(InsufficientOperationalNodesException e) { + + } + } + /** * See Issue #211: Unnecessary read repairs during getAll with more than one * key @@ -1113,7 +1247,8 @@ public void testNoReadRepair() throws Exception { routedStoreThreadPool = Executors.newFixedThreadPool(1); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(isPipelineRoutedStoreEnabled, routedStoreThreadPool, - 1000L); + new TimeoutConfig(1000L, + false)); RoutedStore routedStore = routedStoreFactory.create(cluster, storeDef, @@ -1164,7 +1299,8 @@ public void testTardyResponsesNotIncludedInResult() throws Exception { routedStoreThreadPool = Executors.newFixedThreadPool(cluster.getNumberOfNodes()); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(isPipelineRoutedStoreEnabled, routedStoreThreadPool, - 10000L); + new TimeoutConfig(10000L, + false)); RoutedStore routedStore = routedStoreFactory.create(cluster, storeDef, @@ -1176,7 +1312,7 @@ public void testTardyResponsesNotIncludedInResult() throws Exception { routedStoreFactory = new RoutedStoreFactory(isPipelineRoutedStoreEnabled, routedStoreThreadPool, - sleepTimeMs / 2); + new TimeoutConfig(sleepTimeMs / 2, false)); routedStore = routedStoreFactory.create(cluster, storeDef, subStores, true, failureDetector); @@ -1218,7 +1354,8 @@ public void testSlowStoreDowngradesFromPreferredToRequired() throws Exception { routedStoreThreadPool = Executors.newFixedThreadPool(cluster.getNumberOfNodes()); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(isPipelineRoutedStoreEnabled, routedStoreThreadPool, - 10000L); + new TimeoutConfig(10000L, + false)); RoutedStore routedStore = routedStoreFactory.create(cluster, storeDef, @@ -1230,7 +1367,7 @@ public void testSlowStoreDowngradesFromPreferredToRequired() throws Exception { routedStoreFactory = new RoutedStoreFactory(isPipelineRoutedStoreEnabled, routedStoreThreadPool, - sleepTimeMs / 2); + new TimeoutConfig(sleepTimeMs / 2, false)); routedStore = routedStoreFactory.create(cluster, storeDef, subStores, true, failureDetector); @@ -1279,7 +1416,7 @@ public void testPutDeleteZoneRouting() throws Exception { routedStoreThreadPool = Executors.newFixedThreadPool(8); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(true, routedStoreThreadPool, - 60); + new TimeoutConfig(60, false)); Store s1 = routedStoreFactory.create(cluster, storeDef, From 35f56811cf23b5f04326a5f22be4824b1705b6bb Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 20 Jun 2012 10:14:49 -0700 Subject: [PATCH 068/209] Refactor TimeoutConfig --- src/java/voldemort/client/ClientConfig.java | 21 ++-- src/java/voldemort/client/TimeoutConfig.java | 56 +++++++++ .../voldemort/client/VoldemortOperation.java | 14 +++ .../voldemort/server/VoldemortConfig.java | 34 +++--- .../store/routed/PipelineRoutedStore.java | 31 ++--- .../voldemort/store/routed/RoutedStore.java | 2 +- .../store/routed/RoutedStoreFactory.java | 2 +- .../store/routed/ThreadPoolRoutedStore.java | 14 ++- .../action/PerformSerialGetAllRequests.java | 5 + src/java/voldemort/utils/TimeoutConfig.java | 114 ------------------ .../ClientConnectionStressTest.java | 2 +- .../store/routed/HintedHandoffTest.java | 2 +- .../store/routed/ReadRepairerTest.java | 2 +- .../store/routed/RoutedStoreTest.java | 8 +- 14 files changed, 135 insertions(+), 172 deletions(-) create mode 100644 src/java/voldemort/client/TimeoutConfig.java create mode 100644 src/java/voldemort/client/VoldemortOperation.java delete mode 100644 src/java/voldemort/utils/TimeoutConfig.java diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index 19ab3bde3a..91c6c630dc 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -36,7 +36,6 @@ import voldemort.utils.ConfigurationException; import voldemort.utils.Props; import voldemort.utils.ReflectUtils; -import voldemort.utils.TimeoutConfig; import voldemort.utils.Utils; /** @@ -186,28 +185,28 @@ private void setProperties(Properties properties) { timeoutConfig = new TimeoutConfig(routingTimeoutMs, false); if(props.containsKey(GETALL_ROUTING_TIMEOUT_MS_PROPERTY)) - timeoutConfig.getAllTimeoutMs(props.getInt(GETALL_ROUTING_TIMEOUT_MS_PROPERTY), - TimeUnit.MILLISECONDS); + timeoutConfig.setOperationTimeout(VoldemortOperation.GETALL, + props.getInt(GETALL_ROUTING_TIMEOUT_MS_PROPERTY)); if(props.containsKey(GET_ROUTING_TIMEOUT_MS_PROPERTY)) - timeoutConfig.getTimeoutMs(props.getInt(GET_ROUTING_TIMEOUT_MS_PROPERTY), - TimeUnit.MILLISECONDS); + timeoutConfig.setOperationTimeout(VoldemortOperation.GET, + props.getInt(GET_ROUTING_TIMEOUT_MS_PROPERTY)); if(props.containsKey(PUT_ROUTING_TIMEOUT_MS_PROPERTY)) { long putTimeoutMs = props.getInt(PUT_ROUTING_TIMEOUT_MS_PROPERTY); - timeoutConfig.putTimeoutMs(putTimeoutMs, TimeUnit.MILLISECONDS); + timeoutConfig.setOperationTimeout(VoldemortOperation.PUT, putTimeoutMs); // By default, use the same thing for getVersions() also - timeoutConfig.getVersionsTimeoutMs(putTimeoutMs, TimeUnit.MILLISECONDS); + timeoutConfig.setOperationTimeout(VoldemortOperation.GETVERSIONS, putTimeoutMs); } // of course, if someone overrides it, we will respect that if(props.containsKey(GET_VERSIONS_ROUTING_TIMEOUT_MS_PROPERTY)) - timeoutConfig.getVersionsTimeoutMs(props.getInt(GET_VERSIONS_ROUTING_TIMEOUT_MS_PROPERTY), - TimeUnit.MILLISECONDS); + timeoutConfig.setOperationTimeout(VoldemortOperation.GETVERSIONS, + props.getInt(GET_VERSIONS_ROUTING_TIMEOUT_MS_PROPERTY)); if(props.containsKey(DELETE_ROUTING_TIMEOUT_MS_PROPERTY)) - timeoutConfig.deleteTimeoutMs(props.getInt(DELETE_ROUTING_TIMEOUT_MS_PROPERTY), - TimeUnit.MILLISECONDS); + timeoutConfig.setOperationTimeout(VoldemortOperation.DELETE, + props.getInt(DELETE_ROUTING_TIMEOUT_MS_PROPERTY)); if(props.containsKey(ALLOW_PARTIAL_GETALLS_PROPERTY)) timeoutConfig.setPartialGetAllAllowed(props.getBoolean(ALLOW_PARTIAL_GETALLS_PROPERTY)); diff --git a/src/java/voldemort/client/TimeoutConfig.java b/src/java/voldemort/client/TimeoutConfig.java new file mode 100644 index 0000000000..f316ae113f --- /dev/null +++ b/src/java/voldemort/client/TimeoutConfig.java @@ -0,0 +1,56 @@ +package voldemort.client; + +import java.util.HashMap; + +/** + * Encapsulates the timeouts for various voldemort operations + * + */ +public class TimeoutConfig { + + private HashMap timeoutMap; + + private boolean partialGetAllAllowed; + + public TimeoutConfig(long globalTimeout, boolean allowPartialGetAlls) { + this(globalTimeout, + globalTimeout, + globalTimeout, + globalTimeout, + globalTimeout, + allowPartialGetAlls); + } + + public TimeoutConfig(long getTimeout, + long putTimeout, + long deleteTimeout, + long getAllTimeout, + long getVersionsTimeout, + boolean allowPartialGetAlls) { + timeoutMap = new HashMap(); + timeoutMap.put(VoldemortOperation.GET, getTimeout); + timeoutMap.put(VoldemortOperation.PUT, putTimeout); + timeoutMap.put(VoldemortOperation.DELETE, deleteTimeout); + timeoutMap.put(VoldemortOperation.GETALL, getAllTimeout); + timeoutMap.put(VoldemortOperation.GETVERSIONS, getVersionsTimeout); + setPartialGetAllAllowed(allowPartialGetAlls); + } + + public long getOperationTimeout(VoldemortOperation operation) { + assert timeoutMap.containsKey(operation); + return timeoutMap.get(operation); + } + + public void setOperationTimeout(VoldemortOperation operation, long timeoutMs) { + timeoutMap.put(operation, timeoutMs); + } + + public boolean isPartialGetAllAllowed() { + return partialGetAllAllowed; + } + + public void setPartialGetAllAllowed(boolean allowPartialGetAlls) { + this.partialGetAllAllowed = allowPartialGetAlls; + } + +} diff --git a/src/java/voldemort/client/VoldemortOperation.java b/src/java/voldemort/client/VoldemortOperation.java new file mode 100644 index 0000000000..3c508ea351 --- /dev/null +++ b/src/java/voldemort/client/VoldemortOperation.java @@ -0,0 +1,14 @@ +package voldemort.client; + +/** + * + * Set of operations supported by Voldemort. + * + */ +public enum VoldemortOperation { + GET, + PUT, + GETALL, + DELETE, + GETVERSIONS +} diff --git a/src/java/voldemort/server/VoldemortConfig.java b/src/java/voldemort/server/VoldemortConfig.java index ae1991d6fa..29c3a8bf49 100644 --- a/src/java/voldemort/server/VoldemortConfig.java +++ b/src/java/voldemort/server/VoldemortConfig.java @@ -21,8 +21,9 @@ import java.io.Serializable; import java.util.List; import java.util.Properties; -import java.util.concurrent.TimeUnit; +import voldemort.client.TimeoutConfig; +import voldemort.client.VoldemortOperation; import voldemort.client.protocol.RequestFormatType; import voldemort.cluster.failuredetector.FailureDetectorConfig; import voldemort.server.scheduler.slop.StreamingSlopPusherJob; @@ -35,7 +36,6 @@ import voldemort.utils.ConfigurationException; import voldemort.utils.Props; import voldemort.utils.Time; -import voldemort.utils.TimeoutConfig; import voldemort.utils.UndefinedPropertyException; import voldemort.utils.Utils; @@ -279,21 +279,21 @@ public VoldemortConfig(Props props) { this.clientConnectionTimeoutMs = props.getInt("client.connection.timeout.ms", 500); this.clientRoutingTimeoutMs = props.getInt("client.routing.timeout.ms", 15000); this.clientTimeoutConfig = new TimeoutConfig(this.clientRoutingTimeoutMs, false); - this.clientTimeoutConfig.getTimeoutMs(props.getInt("client.routing.get.timeout.ms", - this.clientRoutingTimeoutMs), - TimeUnit.MILLISECONDS); - this.clientTimeoutConfig.getAllTimeoutMs(props.getInt("client.routing.getall.timeout.ms", - this.clientRoutingTimeoutMs), - TimeUnit.MILLISECONDS); - this.clientTimeoutConfig.putTimeoutMs(props.getInt("client.routing.put.timeout.ms", - this.clientRoutingTimeoutMs), - TimeUnit.MILLISECONDS); - this.clientTimeoutConfig.getVersionsTimeoutMs(props.getLong("client.routing.getversions.timeout.ms", - this.clientTimeoutConfig.putTimeoutMs()), - TimeUnit.MILLISECONDS); - this.clientTimeoutConfig.deleteTimeoutMs(props.getInt("client.routing.delete.timeout.ms", - this.clientRoutingTimeoutMs), - TimeUnit.MILLISECONDS); + this.clientTimeoutConfig.setOperationTimeout(VoldemortOperation.GET, + props.getInt("client.routing.get.timeout.ms", + this.clientRoutingTimeoutMs)); + this.clientTimeoutConfig.setOperationTimeout(VoldemortOperation.GETALL, + props.getInt("client.routing.getall.timeout.ms", + this.clientRoutingTimeoutMs)); + this.clientTimeoutConfig.setOperationTimeout(VoldemortOperation.PUT, + props.getInt("client.routing.put.timeout.ms", + this.clientRoutingTimeoutMs)); + this.clientTimeoutConfig.setOperationTimeout(VoldemortOperation.GETVERSIONS, + props.getLong("client.routing.getversions.timeout.ms", + this.clientTimeoutConfig.getOperationTimeout(VoldemortOperation.PUT))); + this.clientTimeoutConfig.setOperationTimeout(VoldemortOperation.DELETE, + props.getInt("client.routing.delete.timeout.ms", + this.clientRoutingTimeoutMs)); this.clientTimeoutConfig.setPartialGetAllAllowed(props.getBoolean("client.routing.allow.partial.getall", false)); this.clientMaxThreads = props.getInt("client.max.threads", 500); diff --git a/src/java/voldemort/store/routed/PipelineRoutedStore.java b/src/java/voldemort/store/routed/PipelineRoutedStore.java index 9ed094c03c..1ca6517796 100644 --- a/src/java/voldemort/store/routed/PipelineRoutedStore.java +++ b/src/java/voldemort/store/routed/PipelineRoutedStore.java @@ -24,6 +24,8 @@ import java.util.concurrent.atomic.AtomicInteger; import voldemort.VoldemortException; +import voldemort.client.TimeoutConfig; +import voldemort.client.VoldemortOperation; import voldemort.cluster.Cluster; import voldemort.cluster.Zone; import voldemort.cluster.failuredetector.FailureDetector; @@ -58,7 +60,6 @@ import voldemort.utils.ByteUtils; import voldemort.utils.JmxUtils; import voldemort.utils.SystemTime; -import voldemort.utils.TimeoutConfig; import voldemort.versioning.Version; import voldemort.versioning.Versioned; @@ -151,7 +152,7 @@ public List> get(final ByteArray key, final byte[] transforms) pipelineData.setStats(stats); final Pipeline pipeline = new Pipeline(Operation.GET, - timeoutConfig.getTimeoutMs(), + timeoutConfig.getOperationTimeout(VoldemortOperation.GET), TimeUnit.MILLISECONDS); boolean allowReadRepair = repairReads && transforms == null; @@ -180,7 +181,7 @@ public List> request(Store store) { failureDetector, storeDef.getPreferredReads(), storeDef.getRequiredReads(), - timeoutConfig.getTimeoutMs(), + timeoutConfig.getOperationTimeout(VoldemortOperation.GET), nonblockingStores, Event.INSUFFICIENT_SUCCESSES, Event.INSUFFICIENT_ZONES)); @@ -201,7 +202,7 @@ public List> request(Store store) { new ReadRepair>>>(pipelineData, Event.COMPLETED, storeDef.getPreferredReads(), - timeoutConfig.getTimeoutMs(), + timeoutConfig.getOperationTimeout(VoldemortOperation.GET), nonblockingStores, readRepairer)); @@ -259,7 +260,7 @@ public Map>> getAll(Iterable keys, pipelineData.setStats(stats); Pipeline pipeline = new Pipeline(Operation.GET_ALL, - timeoutConfig.getAllTimeoutMs(), + timeoutConfig.getOperationTimeout(VoldemortOperation.GETALL), TimeUnit.MILLISECONDS); pipeline.addEventAction(Event.STARTED, new GetAllConfigureNodes(pipelineData, @@ -275,7 +276,7 @@ public Map>> getAll(Iterable keys, new PerformParallelGetAllRequests(pipelineData, Event.INSUFFICIENT_SUCCESSES, failureDetector, - timeoutConfig.getAllTimeoutMs(), + timeoutConfig.getOperationTimeout(VoldemortOperation.GETALL), nonblockingStores)); pipeline.addEventAction(Event.INSUFFICIENT_SUCCESSES, new PerformSerialGetAllRequests(pipelineData, @@ -293,7 +294,7 @@ public Map>> getAll(Iterable keys, new GetAllReadRepair(pipelineData, Event.COMPLETED, storeDef.getPreferredReads(), - timeoutConfig.getAllTimeoutMs(), + timeoutConfig.getOperationTimeout(VoldemortOperation.GETALL), nonblockingStores, readRepairer)); @@ -330,7 +331,7 @@ public List getVersions(final ByteArray key) { pipelineData.setZonesRequired(null); pipelineData.setStats(stats); Pipeline pipeline = new Pipeline(Operation.GET_VERSIONS, - timeoutConfig.getVersionsTimeoutMs(), + timeoutConfig.getOperationTimeout(VoldemortOperation.GETVERSIONS), TimeUnit.MILLISECONDS); StoreRequest> blockingStoreRequest = new StoreRequest>() { @@ -357,7 +358,7 @@ public List request(Store store) { failureDetector, storeDef.getPreferredReads(), storeDef.getRequiredReads(), - timeoutConfig.getVersionsTimeoutMs(), + timeoutConfig.getOperationTimeout(VoldemortOperation.GETVERSIONS), nonblockingStores, Event.INSUFFICIENT_SUCCESSES, Event.INSUFFICIENT_ZONES)); @@ -417,7 +418,7 @@ public boolean delete(final ByteArray key, final Version version) throws Voldemo pipelineData.setStats(stats); Pipeline pipeline = new Pipeline(Operation.DELETE, - timeoutConfig.deleteTimeoutMs(), + timeoutConfig.getOperationTimeout(VoldemortOperation.DELETE), TimeUnit.MILLISECONDS); pipeline.setEnableHintedHandoff(isHintedHandoffEnabled()); @@ -429,7 +430,7 @@ public boolean delete(final ByteArray key, final Version version) throws Voldemo nonblockingSlopStores, handoffStrategy, pipelineData.getFailedNodes(), - timeoutConfig.deleteTimeoutMs()); + timeoutConfig.getOperationTimeout(VoldemortOperation.DELETE)); pipeline.addEventAction(Event.STARTED, new ConfigureNodes>(pipelineData, @@ -447,7 +448,7 @@ public boolean delete(final ByteArray key, final Version version) throws Voldemo failureDetector, storeDef.getPreferredWrites(), storeDef.getRequiredWrites(), - timeoutConfig.deleteTimeoutMs(), + timeoutConfig.getOperationTimeout(VoldemortOperation.DELETE), nonblockingStores, hintedHandoff, version)); @@ -507,7 +508,7 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) pipelineData.setStats(stats); Pipeline pipeline = new Pipeline(Operation.PUT, - timeoutConfig.putTimeoutMs(), + timeoutConfig.getOperationTimeout(VoldemortOperation.PUT), TimeUnit.MILLISECONDS); pipeline.setEnableHintedHandoff(isHintedHandoffEnabled()); @@ -519,7 +520,7 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) nonblockingSlopStores, handoffStrategy, pipelineData.getFailedNodes(), - timeoutConfig.putTimeoutMs()); + timeoutConfig.getOperationTimeout(VoldemortOperation.PUT)); pipeline.addEventAction(Event.STARTED, new ConfigureNodes(pipelineData, @@ -549,7 +550,7 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) failureDetector, storeDef.getPreferredWrites(), storeDef.getRequiredWrites(), - timeoutConfig.putTimeoutMs(), + timeoutConfig.getOperationTimeout(VoldemortOperation.PUT), nonblockingStores, hintedHandoff)); if(isHintedHandoffEnabled()) { diff --git a/src/java/voldemort/store/routed/RoutedStore.java b/src/java/voldemort/store/routed/RoutedStore.java index ad2eb4a14d..a634b16447 100644 --- a/src/java/voldemort/store/routed/RoutedStore.java +++ b/src/java/voldemort/store/routed/RoutedStore.java @@ -22,6 +22,7 @@ import org.apache.log4j.Logger; import voldemort.VoldemortException; +import voldemort.client.TimeoutConfig; import voldemort.cluster.Cluster; import voldemort.cluster.failuredetector.FailureDetector; import voldemort.routing.RoutingStrategy; @@ -32,7 +33,6 @@ import voldemort.store.StoreDefinition; import voldemort.utils.ByteArray; import voldemort.utils.Time; -import voldemort.utils.TimeoutConfig; import voldemort.utils.Utils; /** diff --git a/src/java/voldemort/store/routed/RoutedStoreFactory.java b/src/java/voldemort/store/routed/RoutedStoreFactory.java index 270e959fa0..24b9210c82 100644 --- a/src/java/voldemort/store/routed/RoutedStoreFactory.java +++ b/src/java/voldemort/store/routed/RoutedStoreFactory.java @@ -7,6 +7,7 @@ import org.apache.log4j.Logger; import voldemort.VoldemortException; +import voldemort.client.TimeoutConfig; import voldemort.cluster.Cluster; import voldemort.cluster.Zone; import voldemort.cluster.failuredetector.FailureDetector; @@ -18,7 +19,6 @@ import voldemort.store.slop.Slop; import voldemort.utils.ByteArray; import voldemort.utils.SystemTime; -import voldemort.utils.TimeoutConfig; import com.google.common.collect.Maps; diff --git a/src/java/voldemort/store/routed/ThreadPoolRoutedStore.java b/src/java/voldemort/store/routed/ThreadPoolRoutedStore.java index 5a8a667dff..eb0e4b20cc 100644 --- a/src/java/voldemort/store/routed/ThreadPoolRoutedStore.java +++ b/src/java/voldemort/store/routed/ThreadPoolRoutedStore.java @@ -36,6 +36,8 @@ import voldemort.VoldemortApplicationException; import voldemort.VoldemortException; +import voldemort.client.TimeoutConfig; +import voldemort.client.VoldemortOperation; import voldemort.cluster.Cluster; import voldemort.cluster.Node; import voldemort.cluster.failuredetector.FailureDetector; @@ -48,7 +50,6 @@ import voldemort.utils.ByteUtils; import voldemort.utils.SystemTime; import voldemort.utils.Time; -import voldemort.utils.TimeoutConfig; import voldemort.versioning.ObsoleteVersionException; import voldemort.versioning.VectorClock; import voldemort.versioning.Version; @@ -208,7 +209,7 @@ public void run() { } else { for(int i = 0; i < numNodes; i++) { try { - long timeoutMs = timeoutConfig.deleteTimeoutMs(); + long timeoutMs = timeoutConfig.getOperationTimeout(VoldemortOperation.DELETE); boolean acquired = semaphore.tryAcquire(timeoutMs, TimeUnit.MILLISECONDS); if(!acquired) logger.warn("Delete operation timed out waiting for operation " + i @@ -302,7 +303,7 @@ public Map>> getAll(Iterable keys, keyToSuccessCount.put(key, new MutableInt(0)); List> futures; - long timeoutMs = timeoutConfig.getAllTimeoutMs(); + long timeoutMs = timeoutConfig.getOperationTimeout(VoldemortOperation.GETALL); try { // TODO What to do about timeouts? They should be longer as getAll // is likely to @@ -465,8 +466,8 @@ private List get(final ByteArray key, } List>> futures; - long timeoutMs = (fetcher == VERSION_OP) ? timeoutConfig.getVersionsTimeoutMs() - : timeoutConfig.getTimeoutMs(); + long timeoutMs = (fetcher == VERSION_OP) ? timeoutConfig.getOperationTimeout(VoldemortOperation.GETVERSIONS) + : timeoutConfig.getOperationTimeout(VoldemortOperation.GET); try { futures = executor.invokeAll(callables, timeoutMs, TimeUnit.MILLISECONDS); } catch(InterruptedException e) { @@ -773,7 +774,8 @@ private boolean blockOnPut(long startNs, for(int i = startingIndex; i < blockCount; i++) { try { long ellapsedNs = System.nanoTime() - startNs; - long remainingNs = (timeoutConfig.putTimeoutMs() * Time.NS_PER_MS) - ellapsedNs; + long remainingNs = (timeoutConfig.getOperationTimeout(VoldemortOperation.PUT) * Time.NS_PER_MS) + - ellapsedNs; boolean acquiredPermit = semaphore.tryAcquire(Math.max(remainingNs, 0), TimeUnit.NANOSECONDS); if(!acquiredPermit) { diff --git a/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java b/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java index 059545e90b..2f5d07e7a2 100644 --- a/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java +++ b/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java @@ -33,6 +33,7 @@ import voldemort.store.routed.Pipeline.Event; import voldemort.store.routed.Response; import voldemort.utils.ByteArray; +import voldemort.utils.ByteUtils; import voldemort.utils.Time; import voldemort.versioning.Versioned; @@ -163,6 +164,10 @@ public void execute(Pipeline pipeline) { // if we allow partial results, then just remove keys that did // not meet 'required' guarantee; else raise error if(allowPartial) { + if(logger.isDebugEnabled()) { + logger.debug("Excluding Key " + ByteUtils.toHexString(key.get()) + + " from partial get_all result"); + } result.remove(key); } else { pipelineData.setFatalError(new InsufficientOperationalNodesException(required diff --git a/src/java/voldemort/utils/TimeoutConfig.java b/src/java/voldemort/utils/TimeoutConfig.java deleted file mode 100644 index 1752d1eab2..0000000000 --- a/src/java/voldemort/utils/TimeoutConfig.java +++ /dev/null @@ -1,114 +0,0 @@ -package voldemort.utils; - -import java.util.concurrent.TimeUnit; - -/** - * Encapsulates the timeouts for various voldemort operations - * - */ -public class TimeoutConfig { - - private long getTimeoutMs; - - private long putTimeoutMs; - - private long getAllTimeoutMs; - - private long deleteTimeoutMs; - - private long getVersionsTimeoutMs; - - private boolean partialGetAllAllowed; - - public TimeoutConfig(long globalTimeout, boolean allowPartialGetAlls) { - this(globalTimeout, - globalTimeout, - globalTimeout, - globalTimeout, - globalTimeout, - allowPartialGetAlls); - } - - public TimeoutConfig(long getTimeout, - long putTimeout, - long deleteTimeout, - long getAllTimeout, - long getVersionsTimeout, - boolean allowPartialGetAlls) { - getTimeoutMs(getTimeout, TimeUnit.MILLISECONDS); - putTimeoutMs(putTimeout, TimeUnit.MILLISECONDS); - deleteTimeoutMs(deleteTimeout, TimeUnit.MILLISECONDS); - getAllTimeoutMs(getAllTimeout, TimeUnit.MILLISECONDS); - getVersionsTimeoutMs(getVersionsTimeout, TimeUnit.MILLISECONDS); - setPartialGetAllAllowed(allowPartialGetAlls); - } - - public long getTimeoutMs(TimeUnit unit) { - return unit.convert(getTimeoutMs, TimeUnit.MILLISECONDS); - } - - public long getTimeoutMs() { - return getTimeoutMs; - } - - public void getTimeoutMs(long getTimeoutMs, TimeUnit unit) { - this.getTimeoutMs = unit.toMillis(getTimeoutMs); - } - - public long getVersionsTimeoutMs(TimeUnit unit) { - return unit.convert(getVersionsTimeoutMs, TimeUnit.MILLISECONDS); - } - - public long getVersionsTimeoutMs() { - return getVersionsTimeoutMs; - } - - public void getVersionsTimeoutMs(long getTimeoutMs, TimeUnit unit) { - this.getVersionsTimeoutMs = unit.toMillis(getTimeoutMs); - } - - public long putTimeoutMs(TimeUnit unit) { - return unit.convert(putTimeoutMs, TimeUnit.MILLISECONDS); - } - - public long putTimeoutMs() { - return putTimeoutMs; - } - - public void putTimeoutMs(long putTimeoutMs, TimeUnit unit) { - this.putTimeoutMs = unit.toMillis(putTimeoutMs); - } - - public long getAllTimeoutMs(TimeUnit unit) { - return unit.convert(getAllTimeoutMs, TimeUnit.MILLISECONDS); - } - - public long getAllTimeoutMs() { - return getAllTimeoutMs; - } - - public void getAllTimeoutMs(long getAllTimeoutMs, TimeUnit unit) { - this.getAllTimeoutMs = unit.toMillis(getAllTimeoutMs); - } - - public long deleteTimeoutMs(TimeUnit unit) { - return unit.convert(deleteTimeoutMs, TimeUnit.MILLISECONDS); - } - - public long deleteTimeoutMs() { - return deleteTimeoutMs; - } - - public void deleteTimeoutMs(long deleteTimeoutMs, TimeUnit unit) { - this.deleteTimeoutMs = unit.toMillis(deleteTimeoutMs); - } - - public boolean isPartialGetAllAllowed() { - return partialGetAllAllowed; - } - - public void setPartialGetAllAllowed(boolean allowPartialGetAlls) { - this.partialGetAllAllowed = allowPartialGetAlls; - } - -} diff --git a/test/integration/voldemort/performance/ClientConnectionStressTest.java b/test/integration/voldemort/performance/ClientConnectionStressTest.java index 6bf7e23ada..810b94f9c8 100644 --- a/test/integration/voldemort/performance/ClientConnectionStressTest.java +++ b/test/integration/voldemort/performance/ClientConnectionStressTest.java @@ -28,8 +28,8 @@ import voldemort.client.SocketStoreClientFactory; import voldemort.client.StoreClient; import voldemort.client.StoreClientFactory; +import voldemort.client.TimeoutConfig; import voldemort.utils.CmdUtils; -import voldemort.utils.TimeoutConfig; /** * Stress tests the client. Intended to diagnose issues such as connection leaks diff --git a/test/unit/voldemort/store/routed/HintedHandoffTest.java b/test/unit/voldemort/store/routed/HintedHandoffTest.java index 653ccb2d70..7f6d7d695e 100644 --- a/test/unit/voldemort/store/routed/HintedHandoffTest.java +++ b/test/unit/voldemort/store/routed/HintedHandoffTest.java @@ -30,6 +30,7 @@ import voldemort.TestUtils; import voldemort.VoldemortException; import voldemort.client.RoutingTier; +import voldemort.client.TimeoutConfig; import voldemort.cluster.Cluster; import voldemort.cluster.Node; import voldemort.cluster.Zone; @@ -61,7 +62,6 @@ import voldemort.store.slop.strategy.HintedHandoffStrategyType; import voldemort.utils.ByteArray; import voldemort.utils.ByteUtils; -import voldemort.utils.TimeoutConfig; import voldemort.versioning.Version; import voldemort.versioning.Versioned; diff --git a/test/unit/voldemort/store/routed/ReadRepairerTest.java b/test/unit/voldemort/store/routed/ReadRepairerTest.java index 7adfde569c..c7d64e2e9a 100644 --- a/test/unit/voldemort/store/routed/ReadRepairerTest.java +++ b/test/unit/voldemort/store/routed/ReadRepairerTest.java @@ -46,6 +46,7 @@ import voldemort.ServerTestUtils; import voldemort.TestUtils; import voldemort.VoldemortTestConstants; +import voldemort.client.TimeoutConfig; import voldemort.cluster.Cluster; import voldemort.cluster.failuredetector.BannagePeriodFailureDetector; import voldemort.cluster.failuredetector.FailureDetector; @@ -56,7 +57,6 @@ import voldemort.store.memory.InMemoryStorageEngine; import voldemort.utils.ByteArray; import voldemort.utils.Time; -import voldemort.utils.TimeoutConfig; import voldemort.versioning.Versioned; import com.google.common.collect.Iterables; diff --git a/test/unit/voldemort/store/routed/RoutedStoreTest.java b/test/unit/voldemort/store/routed/RoutedStoreTest.java index 8da982f487..13dc3e96b9 100644 --- a/test/unit/voldemort/store/routed/RoutedStoreTest.java +++ b/test/unit/voldemort/store/routed/RoutedStoreTest.java @@ -33,7 +33,6 @@ import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; import org.junit.After; import org.junit.Before; @@ -47,6 +46,8 @@ import voldemort.VoldemortException; import voldemort.VoldemortTestConstants; import voldemort.client.RoutingTier; +import voldemort.client.TimeoutConfig; +import voldemort.client.VoldemortOperation; import voldemort.cluster.Cluster; import voldemort.cluster.Node; import voldemort.cluster.failuredetector.BannagePeriodFailureDetector; @@ -74,7 +75,6 @@ import voldemort.utils.ByteArray; import voldemort.utils.ByteUtils; import voldemort.utils.Time; -import voldemort.utils.TimeoutConfig; import voldemort.utils.Utils; import voldemort.versioning.Occurred; import voldemort.versioning.VectorClock; @@ -818,7 +818,7 @@ public void testPartialGetAll() throws Exception { TimeoutConfig timeoutConfig = new TimeoutConfig(1500, true); // This means, the getall will only succeed on two of the nodes - timeoutConfig.getAllTimeoutMs(250, TimeUnit.MILLISECONDS); + timeoutConfig.setOperationTimeout(VoldemortOperation.GETALL, 250); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(true, routedStoreThreadPool, timeoutConfig); @@ -1190,7 +1190,7 @@ public void testOperationSpecificTimeouts() throws Exception { // with a 500ms general timeout and a 100ms get timeout, only get should // fail TimeoutConfig timeoutConfig = new TimeoutConfig(1500, false); - timeoutConfig.getTimeoutMs(100, TimeUnit.MILLISECONDS); + timeoutConfig.setOperationTimeout(VoldemortOperation.GET, 100); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(true, routedStoreThreadPool, timeoutConfig); From f70b3c2cf2df49cf2353b660780e33b5eeb7ced8 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Thu, 21 Jun 2012 10:20:38 -0700 Subject: [PATCH 069/209] Reuse VoldemortOpCode --- src/java/voldemort/client/ClientConfig.java | 13 ++++---- src/java/voldemort/client/TimeoutConfig.java | 26 ++++++++-------- .../VoldemortNativeClientRequestFormat.java | 2 +- .../VoldemortOpCode.java | 2 +- .../serialization/VoldemortOperation.java | 1 + .../voldemort/server/VoldemortConfig.java | 14 ++++----- .../vold/VoldemortNativeRequestHandler.java | 2 +- .../store/routed/PipelineRoutedStore.java | 30 +++++++++---------- .../store/routed/ThreadPoolRoutedStore.java | 12 ++++---- .../store/routed/RoutedStoreTest.java | 6 ++-- 10 files changed, 56 insertions(+), 52 deletions(-) rename src/java/voldemort/{serialization => common}/VoldemortOpCode.java (97%) diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index 91c6c630dc..bc44a29830 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -31,6 +31,7 @@ import voldemort.client.protocol.RequestFormatType; import voldemort.cluster.Zone; import voldemort.cluster.failuredetector.FailureDetectorConfig; +import voldemort.common.VoldemortOpCode; import voldemort.serialization.DefaultSerializerFactory; import voldemort.serialization.SerializerFactory; import voldemort.utils.ConfigurationException; @@ -185,27 +186,27 @@ private void setProperties(Properties properties) { timeoutConfig = new TimeoutConfig(routingTimeoutMs, false); if(props.containsKey(GETALL_ROUTING_TIMEOUT_MS_PROPERTY)) - timeoutConfig.setOperationTimeout(VoldemortOperation.GETALL, + timeoutConfig.setOperationTimeout(VoldemortOpCode.GET_ALL_OP_CODE, props.getInt(GETALL_ROUTING_TIMEOUT_MS_PROPERTY)); if(props.containsKey(GET_ROUTING_TIMEOUT_MS_PROPERTY)) - timeoutConfig.setOperationTimeout(VoldemortOperation.GET, + timeoutConfig.setOperationTimeout(VoldemortOpCode.GET_OP_CODE, props.getInt(GET_ROUTING_TIMEOUT_MS_PROPERTY)); if(props.containsKey(PUT_ROUTING_TIMEOUT_MS_PROPERTY)) { long putTimeoutMs = props.getInt(PUT_ROUTING_TIMEOUT_MS_PROPERTY); - timeoutConfig.setOperationTimeout(VoldemortOperation.PUT, putTimeoutMs); + timeoutConfig.setOperationTimeout(VoldemortOpCode.PUT_OP_CODE, putTimeoutMs); // By default, use the same thing for getVersions() also - timeoutConfig.setOperationTimeout(VoldemortOperation.GETVERSIONS, putTimeoutMs); + timeoutConfig.setOperationTimeout(VoldemortOpCode.GET_VERSION_OP_CODE, putTimeoutMs); } // of course, if someone overrides it, we will respect that if(props.containsKey(GET_VERSIONS_ROUTING_TIMEOUT_MS_PROPERTY)) - timeoutConfig.setOperationTimeout(VoldemortOperation.GETVERSIONS, + timeoutConfig.setOperationTimeout(VoldemortOpCode.GET_VERSION_OP_CODE, props.getInt(GET_VERSIONS_ROUTING_TIMEOUT_MS_PROPERTY)); if(props.containsKey(DELETE_ROUTING_TIMEOUT_MS_PROPERTY)) - timeoutConfig.setOperationTimeout(VoldemortOperation.DELETE, + timeoutConfig.setOperationTimeout(VoldemortOpCode.DELETE_OP_CODE, props.getInt(DELETE_ROUTING_TIMEOUT_MS_PROPERTY)); if(props.containsKey(ALLOW_PARTIAL_GETALLS_PROPERTY)) diff --git a/src/java/voldemort/client/TimeoutConfig.java b/src/java/voldemort/client/TimeoutConfig.java index f316ae113f..cafe7712e9 100644 --- a/src/java/voldemort/client/TimeoutConfig.java +++ b/src/java/voldemort/client/TimeoutConfig.java @@ -2,13 +2,15 @@ import java.util.HashMap; +import voldemort.common.VoldemortOpCode; + /** * Encapsulates the timeouts for various voldemort operations * */ public class TimeoutConfig { - private HashMap timeoutMap; + private HashMap timeoutMap; private boolean partialGetAllAllowed; @@ -27,22 +29,22 @@ public TimeoutConfig(long getTimeout, long getAllTimeout, long getVersionsTimeout, boolean allowPartialGetAlls) { - timeoutMap = new HashMap(); - timeoutMap.put(VoldemortOperation.GET, getTimeout); - timeoutMap.put(VoldemortOperation.PUT, putTimeout); - timeoutMap.put(VoldemortOperation.DELETE, deleteTimeout); - timeoutMap.put(VoldemortOperation.GETALL, getAllTimeout); - timeoutMap.put(VoldemortOperation.GETVERSIONS, getVersionsTimeout); + timeoutMap = new HashMap(); + timeoutMap.put(VoldemortOpCode.GET_OP_CODE, getTimeout); + timeoutMap.put(VoldemortOpCode.PUT_OP_CODE, putTimeout); + timeoutMap.put(VoldemortOpCode.DELETE_OP_CODE, deleteTimeout); + timeoutMap.put(VoldemortOpCode.GET_ALL_OP_CODE, getAllTimeout); + timeoutMap.put(VoldemortOpCode.GET_VERSION_OP_CODE, getVersionsTimeout); setPartialGetAllAllowed(allowPartialGetAlls); } - public long getOperationTimeout(VoldemortOperation operation) { - assert timeoutMap.containsKey(operation); - return timeoutMap.get(operation); + public long getOperationTimeout(Byte opCode) { + assert timeoutMap.containsKey(opCode); + return timeoutMap.get(opCode); } - public void setOperationTimeout(VoldemortOperation operation, long timeoutMs) { - timeoutMap.put(operation, timeoutMs); + public void setOperationTimeout(Byte opCode, long timeoutMs) { + timeoutMap.put(opCode, timeoutMs); } public boolean isPartialGetAllAllowed() { diff --git a/src/java/voldemort/client/protocol/vold/VoldemortNativeClientRequestFormat.java b/src/java/voldemort/client/protocol/vold/VoldemortNativeClientRequestFormat.java index 1516c15247..4953f05b0f 100644 --- a/src/java/voldemort/client/protocol/vold/VoldemortNativeClientRequestFormat.java +++ b/src/java/voldemort/client/protocol/vold/VoldemortNativeClientRequestFormat.java @@ -29,7 +29,7 @@ import voldemort.VoldemortException; import voldemort.client.protocol.RequestFormat; -import voldemort.serialization.VoldemortOpCode; +import voldemort.common.VoldemortOpCode; import voldemort.server.RequestRoutingType; import voldemort.store.ErrorCodeMapper; import voldemort.store.StoreUtils; diff --git a/src/java/voldemort/serialization/VoldemortOpCode.java b/src/java/voldemort/common/VoldemortOpCode.java similarity index 97% rename from src/java/voldemort/serialization/VoldemortOpCode.java rename to src/java/voldemort/common/VoldemortOpCode.java index e07b6bd09e..a38e103125 100644 --- a/src/java/voldemort/serialization/VoldemortOpCode.java +++ b/src/java/voldemort/common/VoldemortOpCode.java @@ -14,7 +14,7 @@ * the License. */ -package voldemort.serialization; +package voldemort.common; public class VoldemortOpCode { diff --git a/src/java/voldemort/serialization/VoldemortOperation.java b/src/java/voldemort/serialization/VoldemortOperation.java index 4549782640..3026b25695 100644 --- a/src/java/voldemort/serialization/VoldemortOperation.java +++ b/src/java/voldemort/serialization/VoldemortOperation.java @@ -22,6 +22,7 @@ import java.io.DataOutputStream; import java.io.IOException; +import voldemort.common.VoldemortOpCode; import voldemort.utils.ByteUtils; import voldemort.versioning.VectorClock; import voldemort.versioning.Version; diff --git a/src/java/voldemort/server/VoldemortConfig.java b/src/java/voldemort/server/VoldemortConfig.java index 29c3a8bf49..61e11c0df2 100644 --- a/src/java/voldemort/server/VoldemortConfig.java +++ b/src/java/voldemort/server/VoldemortConfig.java @@ -23,9 +23,9 @@ import java.util.Properties; import voldemort.client.TimeoutConfig; -import voldemort.client.VoldemortOperation; import voldemort.client.protocol.RequestFormatType; import voldemort.cluster.failuredetector.FailureDetectorConfig; +import voldemort.common.VoldemortOpCode; import voldemort.server.scheduler.slop.StreamingSlopPusherJob; import voldemort.store.bdb.BdbStorageConfiguration; import voldemort.store.memory.CacheStorageConfiguration; @@ -279,19 +279,19 @@ public VoldemortConfig(Props props) { this.clientConnectionTimeoutMs = props.getInt("client.connection.timeout.ms", 500); this.clientRoutingTimeoutMs = props.getInt("client.routing.timeout.ms", 15000); this.clientTimeoutConfig = new TimeoutConfig(this.clientRoutingTimeoutMs, false); - this.clientTimeoutConfig.setOperationTimeout(VoldemortOperation.GET, + this.clientTimeoutConfig.setOperationTimeout(VoldemortOpCode.GET_OP_CODE, props.getInt("client.routing.get.timeout.ms", this.clientRoutingTimeoutMs)); - this.clientTimeoutConfig.setOperationTimeout(VoldemortOperation.GETALL, + this.clientTimeoutConfig.setOperationTimeout(VoldemortOpCode.GET_ALL_OP_CODE, props.getInt("client.routing.getall.timeout.ms", this.clientRoutingTimeoutMs)); - this.clientTimeoutConfig.setOperationTimeout(VoldemortOperation.PUT, + this.clientTimeoutConfig.setOperationTimeout(VoldemortOpCode.PUT_OP_CODE, props.getInt("client.routing.put.timeout.ms", this.clientRoutingTimeoutMs)); - this.clientTimeoutConfig.setOperationTimeout(VoldemortOperation.GETVERSIONS, + this.clientTimeoutConfig.setOperationTimeout(VoldemortOpCode.GET_VERSION_OP_CODE, props.getLong("client.routing.getversions.timeout.ms", - this.clientTimeoutConfig.getOperationTimeout(VoldemortOperation.PUT))); - this.clientTimeoutConfig.setOperationTimeout(VoldemortOperation.DELETE, + this.clientTimeoutConfig.getOperationTimeout(VoldemortOpCode.PUT_OP_CODE))); + this.clientTimeoutConfig.setOperationTimeout(VoldemortOpCode.DELETE_OP_CODE, props.getInt("client.routing.delete.timeout.ms", this.clientRoutingTimeoutMs)); this.clientTimeoutConfig.setPartialGetAllAllowed(props.getBoolean("client.routing.allow.partial.getall", diff --git a/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java b/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java index b552a73b77..b74b60a920 100644 --- a/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java +++ b/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java @@ -12,7 +12,7 @@ import org.apache.log4j.Logger; import voldemort.VoldemortException; -import voldemort.serialization.VoldemortOpCode; +import voldemort.common.VoldemortOpCode; import voldemort.server.RequestRoutingType; import voldemort.server.StoreRepository; import voldemort.server.protocol.AbstractRequestHandler; diff --git a/src/java/voldemort/store/routed/PipelineRoutedStore.java b/src/java/voldemort/store/routed/PipelineRoutedStore.java index 1ca6517796..38ff90517d 100644 --- a/src/java/voldemort/store/routed/PipelineRoutedStore.java +++ b/src/java/voldemort/store/routed/PipelineRoutedStore.java @@ -25,10 +25,10 @@ import voldemort.VoldemortException; import voldemort.client.TimeoutConfig; -import voldemort.client.VoldemortOperation; import voldemort.cluster.Cluster; import voldemort.cluster.Zone; import voldemort.cluster.failuredetector.FailureDetector; +import voldemort.common.VoldemortOpCode; import voldemort.routing.RoutingStrategyType; import voldemort.store.Store; import voldemort.store.StoreDefinition; @@ -152,7 +152,7 @@ public List> get(final ByteArray key, final byte[] transforms) pipelineData.setStats(stats); final Pipeline pipeline = new Pipeline(Operation.GET, - timeoutConfig.getOperationTimeout(VoldemortOperation.GET), + timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_OP_CODE), TimeUnit.MILLISECONDS); boolean allowReadRepair = repairReads && transforms == null; @@ -181,7 +181,7 @@ public List> request(Store store) { failureDetector, storeDef.getPreferredReads(), storeDef.getRequiredReads(), - timeoutConfig.getOperationTimeout(VoldemortOperation.GET), + timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_OP_CODE), nonblockingStores, Event.INSUFFICIENT_SUCCESSES, Event.INSUFFICIENT_ZONES)); @@ -202,7 +202,7 @@ public List> request(Store store) { new ReadRepair>>>(pipelineData, Event.COMPLETED, storeDef.getPreferredReads(), - timeoutConfig.getOperationTimeout(VoldemortOperation.GET), + timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_OP_CODE), nonblockingStores, readRepairer)); @@ -260,7 +260,7 @@ public Map>> getAll(Iterable keys, pipelineData.setStats(stats); Pipeline pipeline = new Pipeline(Operation.GET_ALL, - timeoutConfig.getOperationTimeout(VoldemortOperation.GETALL), + timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_ALL_OP_CODE), TimeUnit.MILLISECONDS); pipeline.addEventAction(Event.STARTED, new GetAllConfigureNodes(pipelineData, @@ -276,7 +276,7 @@ public Map>> getAll(Iterable keys, new PerformParallelGetAllRequests(pipelineData, Event.INSUFFICIENT_SUCCESSES, failureDetector, - timeoutConfig.getOperationTimeout(VoldemortOperation.GETALL), + timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_ALL_OP_CODE), nonblockingStores)); pipeline.addEventAction(Event.INSUFFICIENT_SUCCESSES, new PerformSerialGetAllRequests(pipelineData, @@ -294,7 +294,7 @@ public Map>> getAll(Iterable keys, new GetAllReadRepair(pipelineData, Event.COMPLETED, storeDef.getPreferredReads(), - timeoutConfig.getOperationTimeout(VoldemortOperation.GETALL), + timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_ALL_OP_CODE), nonblockingStores, readRepairer)); @@ -331,7 +331,7 @@ public List getVersions(final ByteArray key) { pipelineData.setZonesRequired(null); pipelineData.setStats(stats); Pipeline pipeline = new Pipeline(Operation.GET_VERSIONS, - timeoutConfig.getOperationTimeout(VoldemortOperation.GETVERSIONS), + timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_VERSION_OP_CODE), TimeUnit.MILLISECONDS); StoreRequest> blockingStoreRequest = new StoreRequest>() { @@ -358,7 +358,7 @@ public List request(Store store) { failureDetector, storeDef.getPreferredReads(), storeDef.getRequiredReads(), - timeoutConfig.getOperationTimeout(VoldemortOperation.GETVERSIONS), + timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_VERSION_OP_CODE), nonblockingStores, Event.INSUFFICIENT_SUCCESSES, Event.INSUFFICIENT_ZONES)); @@ -418,7 +418,7 @@ public boolean delete(final ByteArray key, final Version version) throws Voldemo pipelineData.setStats(stats); Pipeline pipeline = new Pipeline(Operation.DELETE, - timeoutConfig.getOperationTimeout(VoldemortOperation.DELETE), + timeoutConfig.getOperationTimeout(VoldemortOpCode.DELETE_OP_CODE), TimeUnit.MILLISECONDS); pipeline.setEnableHintedHandoff(isHintedHandoffEnabled()); @@ -430,7 +430,7 @@ public boolean delete(final ByteArray key, final Version version) throws Voldemo nonblockingSlopStores, handoffStrategy, pipelineData.getFailedNodes(), - timeoutConfig.getOperationTimeout(VoldemortOperation.DELETE)); + timeoutConfig.getOperationTimeout(VoldemortOpCode.DELETE_OP_CODE)); pipeline.addEventAction(Event.STARTED, new ConfigureNodes>(pipelineData, @@ -448,7 +448,7 @@ public boolean delete(final ByteArray key, final Version version) throws Voldemo failureDetector, storeDef.getPreferredWrites(), storeDef.getRequiredWrites(), - timeoutConfig.getOperationTimeout(VoldemortOperation.DELETE), + timeoutConfig.getOperationTimeout(VoldemortOpCode.DELETE_OP_CODE), nonblockingStores, hintedHandoff, version)); @@ -508,7 +508,7 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) pipelineData.setStats(stats); Pipeline pipeline = new Pipeline(Operation.PUT, - timeoutConfig.getOperationTimeout(VoldemortOperation.PUT), + timeoutConfig.getOperationTimeout(VoldemortOpCode.PUT_OP_CODE), TimeUnit.MILLISECONDS); pipeline.setEnableHintedHandoff(isHintedHandoffEnabled()); @@ -520,7 +520,7 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) nonblockingSlopStores, handoffStrategy, pipelineData.getFailedNodes(), - timeoutConfig.getOperationTimeout(VoldemortOperation.PUT)); + timeoutConfig.getOperationTimeout(VoldemortOpCode.PUT_OP_CODE)); pipeline.addEventAction(Event.STARTED, new ConfigureNodes(pipelineData, @@ -550,7 +550,7 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) failureDetector, storeDef.getPreferredWrites(), storeDef.getRequiredWrites(), - timeoutConfig.getOperationTimeout(VoldemortOperation.PUT), + timeoutConfig.getOperationTimeout(VoldemortOpCode.PUT_OP_CODE), nonblockingStores, hintedHandoff)); if(isHintedHandoffEnabled()) { diff --git a/src/java/voldemort/store/routed/ThreadPoolRoutedStore.java b/src/java/voldemort/store/routed/ThreadPoolRoutedStore.java index eb0e4b20cc..947b44f42b 100644 --- a/src/java/voldemort/store/routed/ThreadPoolRoutedStore.java +++ b/src/java/voldemort/store/routed/ThreadPoolRoutedStore.java @@ -37,10 +37,10 @@ import voldemort.VoldemortApplicationException; import voldemort.VoldemortException; import voldemort.client.TimeoutConfig; -import voldemort.client.VoldemortOperation; import voldemort.cluster.Cluster; import voldemort.cluster.Node; import voldemort.cluster.failuredetector.FailureDetector; +import voldemort.common.VoldemortOpCode; import voldemort.store.InsufficientOperationalNodesException; import voldemort.store.Store; import voldemort.store.StoreDefinition; @@ -209,7 +209,7 @@ public void run() { } else { for(int i = 0; i < numNodes; i++) { try { - long timeoutMs = timeoutConfig.getOperationTimeout(VoldemortOperation.DELETE); + long timeoutMs = timeoutConfig.getOperationTimeout(VoldemortOpCode.DELETE_OP_CODE); boolean acquired = semaphore.tryAcquire(timeoutMs, TimeUnit.MILLISECONDS); if(!acquired) logger.warn("Delete operation timed out waiting for operation " + i @@ -303,7 +303,7 @@ public Map>> getAll(Iterable keys, keyToSuccessCount.put(key, new MutableInt(0)); List> futures; - long timeoutMs = timeoutConfig.getOperationTimeout(VoldemortOperation.GETALL); + long timeoutMs = timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_ALL_OP_CODE); try { // TODO What to do about timeouts? They should be longer as getAll // is likely to @@ -466,8 +466,8 @@ private List get(final ByteArray key, } List>> futures; - long timeoutMs = (fetcher == VERSION_OP) ? timeoutConfig.getOperationTimeout(VoldemortOperation.GETVERSIONS) - : timeoutConfig.getOperationTimeout(VoldemortOperation.GET); + long timeoutMs = (fetcher == VERSION_OP) ? timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_VERSION_OP_CODE) + : timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_OP_CODE); try { futures = executor.invokeAll(callables, timeoutMs, TimeUnit.MILLISECONDS); } catch(InterruptedException e) { @@ -774,7 +774,7 @@ private boolean blockOnPut(long startNs, for(int i = startingIndex; i < blockCount; i++) { try { long ellapsedNs = System.nanoTime() - startNs; - long remainingNs = (timeoutConfig.getOperationTimeout(VoldemortOperation.PUT) * Time.NS_PER_MS) + long remainingNs = (timeoutConfig.getOperationTimeout(VoldemortOpCode.PUT_OP_CODE) * Time.NS_PER_MS) - ellapsedNs; boolean acquiredPermit = semaphore.tryAcquire(Math.max(remainingNs, 0), TimeUnit.NANOSECONDS); diff --git a/test/unit/voldemort/store/routed/RoutedStoreTest.java b/test/unit/voldemort/store/routed/RoutedStoreTest.java index 13dc3e96b9..1b1ed1428e 100644 --- a/test/unit/voldemort/store/routed/RoutedStoreTest.java +++ b/test/unit/voldemort/store/routed/RoutedStoreTest.java @@ -47,12 +47,12 @@ import voldemort.VoldemortTestConstants; import voldemort.client.RoutingTier; import voldemort.client.TimeoutConfig; -import voldemort.client.VoldemortOperation; import voldemort.cluster.Cluster; import voldemort.cluster.Node; import voldemort.cluster.failuredetector.BannagePeriodFailureDetector; import voldemort.cluster.failuredetector.FailureDetector; import voldemort.cluster.failuredetector.FailureDetectorConfig; +import voldemort.common.VoldemortOpCode; import voldemort.routing.RoutingStrategy; import voldemort.routing.RoutingStrategyFactory; import voldemort.routing.RoutingStrategyType; @@ -818,7 +818,7 @@ public void testPartialGetAll() throws Exception { TimeoutConfig timeoutConfig = new TimeoutConfig(1500, true); // This means, the getall will only succeed on two of the nodes - timeoutConfig.setOperationTimeout(VoldemortOperation.GETALL, 250); + timeoutConfig.setOperationTimeout(VoldemortOpCode.GET_ALL_OP_CODE, 250); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(true, routedStoreThreadPool, timeoutConfig); @@ -1190,7 +1190,7 @@ public void testOperationSpecificTimeouts() throws Exception { // with a 500ms general timeout and a 100ms get timeout, only get should // fail TimeoutConfig timeoutConfig = new TimeoutConfig(1500, false); - timeoutConfig.setOperationTimeout(VoldemortOperation.GET, 100); + timeoutConfig.setOperationTimeout(VoldemortOpCode.GET_OP_CODE, 100); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(true, routedStoreThreadPool, timeoutConfig); From cd8b94278cc5b9851325d495672c033320ca02cc Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Thu, 17 May 2012 01:04:31 -0700 Subject: [PATCH 070/209] add a java file to define all system store constants, including system store defs --- .../server/SystemStoreConstants.java | 58 +++++ .../protocol/AbstractRequestHandler.java | 1 + .../server/storage/StorageService.java | 200 ++++++++++++++++++ src/java/voldemort/store/StoreDefinition.java | 4 +- .../voldemort/xml/StoreDefinitionsMapper.java | 4 +- 5 files changed, 265 insertions(+), 2 deletions(-) create mode 100644 src/java/voldemort/server/SystemStoreConstants.java diff --git a/src/java/voldemort/server/SystemStoreConstants.java b/src/java/voldemort/server/SystemStoreConstants.java new file mode 100644 index 0000000000..820037a920 --- /dev/null +++ b/src/java/voldemort/server/SystemStoreConstants.java @@ -0,0 +1,58 @@ +package voldemort.server; + +/** + * The various system stores + */ +public class SystemStoreConstants { + + private static final String NAME_PREFIX = "voldsys$_"; + + public static enum SystemStoreName { + voldsys$_client_registry, + voldsys$_client_store_definition; + } + + public static final String SYSTEM_STORE_SCHEMA = "" + + " " + + " voldsys$_client_registry" + + " zone-routing" + + " proximity-handoff" + + " memory" + + " client" + + " 4" + + " " + + " 2" + + " 2" + + " " + + " 1" + + " 1" + + " " + + " string" + + " " + + " " + + " string" + + " " + + " 7" + + " " + + " " + + " voldsys$_client_store_definition" + + " zone-routing" + + " proximity-handoff" + + " memory" + + " client" + + " 1" + + " 1" + + " 1" + + " " + + " string" + + " " + + " " + + " string" + + " " + + " 7" + + " " + ""; + + public static boolean isSystemStore(String storeName) { + return (null == storeName ? false : storeName.startsWith(NAME_PREFIX)); + } +} \ No newline at end of file diff --git a/src/java/voldemort/server/protocol/AbstractRequestHandler.java b/src/java/voldemort/server/protocol/AbstractRequestHandler.java index b549a346b9..c9605abe7e 100644 --- a/src/java/voldemort/server/protocol/AbstractRequestHandler.java +++ b/src/java/voldemort/server/protocol/AbstractRequestHandler.java @@ -32,6 +32,7 @@ protected StoreRepository getStoreRepository() { } protected Store getStore(String name, RequestRoutingType type) { + switch(type) { case ROUTED: return storeRepository.getRoutedStore(name); diff --git a/src/java/voldemort/server/storage/StorageService.java b/src/java/voldemort/server/storage/StorageService.java index 8bb3919c6f..3027b6e237 100644 --- a/src/java/voldemort/server/storage/StorageService.java +++ b/src/java/voldemort/server/storage/StorageService.java @@ -18,6 +18,7 @@ import static voldemort.cluster.failuredetector.FailureDetectorUtils.create; +import java.io.StringReader; import java.lang.management.ManagementFactory; import java.util.ArrayList; import java.util.Calendar; @@ -54,6 +55,7 @@ import voldemort.server.RequestRoutingType; import voldemort.server.ServiceType; import voldemort.server.StoreRepository; +import voldemort.server.SystemStoreConstants; import voldemort.server.VoldemortConfig; import voldemort.server.scheduler.DataCleanupJob; import voldemort.server.scheduler.SchedulerService; @@ -65,6 +67,7 @@ import voldemort.store.StoreDefinition; import voldemort.store.invalidmetadata.InvalidMetadataCheckingStore; import voldemort.store.logging.LoggingStore; +import voldemort.store.memory.InMemoryStorageConfiguration; import voldemort.store.metadata.MetadataStore; import voldemort.store.metadata.MetadataStoreListener; import voldemort.store.nonblockingstore.NonblockingStore; @@ -97,6 +100,7 @@ import voldemort.versioning.VectorClock; import voldemort.versioning.VectorClockInconsistencyResolver; import voldemort.versioning.Versioned; +import voldemort.xml.StoreDefinitionsMapper; /** * The service responsible for managing all storage types @@ -170,6 +174,7 @@ public StorageService(StoreRepository storeRepository, } private void initStorageConfig(String configClassName) { + // add the configurations of the storage engines needed by user stores try { Class configClass = ReflectUtils.loadClass(configClassName); StorageConfiguration configuration = (StorageConfiguration) ReflectUtils.callConstructor(configClass, @@ -187,6 +192,37 @@ private void initStorageConfig(String configClassName) { if(storageConfigs.size() == 0) throw new ConfigurationException("No storage engine has been enabled!"); + + // now, add the configurations of the storage engines needed by system + // stores, if not yet exist + initSystemStorageConfig(); + } + + private void initSystemStorageConfig() { + // add InMemoryStorage used by voldsys$_client_registry + if(!storageConfigs.containsKey(InMemoryStorageConfiguration.TYPE_NAME)) { + storageConfigs.put(InMemoryStorageConfiguration.TYPE_NAME, + new InMemoryStorageConfiguration()); + } + + // add FileStorage config here + } + + private void initSystemStores() { + List storesDefs = (new StoreDefinitionsMapper()).readStoreList(new StringReader(SystemStoreConstants.SYSTEM_STORE_SCHEMA)); + + // TODO: replication factor can't now be determined unless the + // cluster.xml is made available to the server at runtime. So we need to + // set them here after load they are loaded + updateRepFactor(storesDefs); + + for(StoreDefinition storeDef: storesDefs) { + openSystemStore(storeDef); + } + } + + private void updateRepFactor(List storesDefs) { + // need impl } @Override @@ -203,6 +239,9 @@ protected void startInner() { metadata.getStoreDefList(), storeRepository)); + /* Initialize system stores */ + initSystemStores(); + /* Register slop store */ if(voldemortConfig.isSlopEnabled()) { @@ -282,6 +321,167 @@ protected void startInner() { logger.info("All stores initialized."); } + public void openSystemStore(StoreDefinition storeDef) { + + logger.info("Opening system store '" + storeDef.getName() + "' (" + storeDef.getType() + + ")."); + + StorageConfiguration config = storageConfigs.get(storeDef.getType()); + if(config == null) + throw new ConfigurationException("Attempt to open system store " + storeDef.getName() + + " but " + storeDef.getType() + + " storage engine has not been enabled."); + + final StorageEngine engine = config.getStore(storeDef.getName()); + + // Noted that there is no read-only processing as for user stores. + + // openStore() should have atomic semantics + try { + registerSystemEngine(engine); + + if(voldemortConfig.isServerRoutingEnabled()) + registerNodeStores(storeDef, metadata.getCluster(), voldemortConfig.getNodeId()); + + if(storeDef.hasRetentionPeriod()) + scheduleCleanupJob(storeDef, engine); + } catch(Exception e) { + unregisterSystemEngine(engine); + throw new VoldemortException(e); + } + } + + public void registerSystemEngine(StorageEngine engine) { + + Cluster cluster = this.metadata.getCluster(); + storeRepository.addStorageEngine(engine); + + /* Now add any store wrappers that are enabled */ + Store store = engine; + + if(voldemortConfig.isVerboseLoggingEnabled()) + store = new LoggingStore(store, + cluster.getName(), + SystemTime.INSTANCE); + /* TODO: Do we really need rebalancing for system stores? */ + if(voldemortConfig.isEnableRebalanceService()) { + store = new RedirectingStore(store, + metadata, + storeRepository, + failureDetector, + storeFactory); + if(voldemortConfig.isJmxEnabled()) { + MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer(); + ObjectName name = null; + if(this.voldemortConfig.isEnableJmxClusterName()) + name = JmxUtils.createObjectName(cluster.getName() + + "." + + JmxUtils.getPackageName(RedirectingStore.class), + store.getName()); + else + name = JmxUtils.createObjectName(JmxUtils.getPackageName(RedirectingStore.class), + store.getName()); + + synchronized(mbeanServer) { + if(mbeanServer.isRegistered(name)) + JmxUtils.unregisterMbean(mbeanServer, name); + JmxUtils.registerMbean(mbeanServer, JmxUtils.createModelMBean(store), name); + } + + } + } + + if(voldemortConfig.isMetadataCheckingEnabled()) + store = new InvalidMetadataCheckingStore(metadata.getNodeId(), store, metadata); + + if(voldemortConfig.isStatTrackingEnabled()) { + StatTrackingStore statStore = new StatTrackingStore(store, this.storeStats); + store = statStore; + if(voldemortConfig.isJmxEnabled()) { + + MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer(); + ObjectName name = null; + if(this.voldemortConfig.isEnableJmxClusterName()) + name = JmxUtils.createObjectName(metadata.getCluster().getName() + + "." + + JmxUtils.getPackageName(store.getClass()), + store.getName()); + else + name = JmxUtils.createObjectName(JmxUtils.getPackageName(store.getClass()), + store.getName()); + + synchronized(mbeanServer) { + if(mbeanServer.isRegistered(name)) + JmxUtils.unregisterMbean(mbeanServer, name); + + JmxUtils.registerMbean(mbeanServer, + JmxUtils.createModelMBean(new StoreStatsJmx(statStore.getStats())), + name); + } + } + } + + storeRepository.addLocalStore(store); + } + + public void unregisterSystemEngine(StorageEngine engine) { + String storeName = engine.getName(); + Store store = storeRepository.removeLocalStore(storeName); + + if(store != null) { + if(voldemortConfig.isJmxEnabled()) { + MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer(); + + if(voldemortConfig.isEnableRebalanceService()) { + + ObjectName name = null; + if(this.voldemortConfig.isEnableJmxClusterName()) + name = JmxUtils.createObjectName(metadata.getCluster().getName() + + "." + + JmxUtils.getPackageName(RedirectingStore.class), + store.getName()); + else + name = JmxUtils.createObjectName(JmxUtils.getPackageName(RedirectingStore.class), + store.getName()); + + synchronized(mbeanServer) { + if(mbeanServer.isRegistered(name)) + JmxUtils.unregisterMbean(mbeanServer, name); + } + + } + + if(voldemortConfig.isStatTrackingEnabled()) { + ObjectName name = null; + if(this.voldemortConfig.isEnableJmxClusterName()) + name = JmxUtils.createObjectName(metadata.getCluster().getName() + + "." + + JmxUtils.getPackageName(store.getClass()), + store.getName()); + else + name = JmxUtils.createObjectName(JmxUtils.getPackageName(store.getClass()), + store.getName()); + + synchronized(mbeanServer) { + if(mbeanServer.isRegistered(name)) + JmxUtils.unregisterMbean(mbeanServer, name); + } + + } + } + if(voldemortConfig.isServerRoutingEnabled()) { + this.storeRepository.removeRoutedStore(storeName); + for(Node node: metadata.getCluster().getNodes()) + this.storeRepository.removeNodeStore(storeName, node.getId()); + } + } + + storeRepository.removeStorageEngine(storeName); + // engine.truncate(); why truncate here when unregister? Isn't close + // good enough? + engine.close(); + } + public void openStore(StoreDefinition storeDef) { logger.info("Opening store '" + storeDef.getName() + "' (" + storeDef.getType() + ")."); diff --git a/src/java/voldemort/store/StoreDefinition.java b/src/java/voldemort/store/StoreDefinition.java index 0df48fcf84..52d4dda7f0 100644 --- a/src/java/voldemort/store/StoreDefinition.java +++ b/src/java/voldemort/store/StoreDefinition.java @@ -22,6 +22,7 @@ import voldemort.client.RoutingTier; import voldemort.serialization.SerializerDefinition; +import voldemort.server.SystemStoreConstants; import voldemort.store.slop.strategy.HintedHandoffStrategyType; import voldemort.utils.Utils; @@ -139,7 +140,8 @@ else if(requiredWrites > replicationFactor) if(retentionPeriodDays != null && retentionPeriodDays < 0) throw new IllegalArgumentException("Retention days must be non-negative."); - if(zoneReplicationFactor != null && zoneReplicationFactor.size() != 0) { + if(!SystemStoreConstants.isSystemStore(name) && zoneReplicationFactor != null + && zoneReplicationFactor.size() != 0) { if(zoneCountReads == null || zoneCountReads < 0) throw new IllegalArgumentException("Zone Counts reads must be non-negative / non-null"); diff --git a/src/java/voldemort/xml/StoreDefinitionsMapper.java b/src/java/voldemort/xml/StoreDefinitionsMapper.java index c51913d1a5..d1d4948b24 100644 --- a/src/java/voldemort/xml/StoreDefinitionsMapper.java +++ b/src/java/voldemort/xml/StoreDefinitionsMapper.java @@ -46,6 +46,7 @@ import voldemort.routing.RoutingStrategyType; import voldemort.serialization.Compression; import voldemort.serialization.SerializerDefinition; +import voldemort.server.SystemStoreConstants; import voldemort.store.StoreDefinition; import voldemort.store.StoreDefinitionBuilder; import voldemort.store.StoreUtils; @@ -227,7 +228,8 @@ private StoreDefinition readStore(Element store) { retentionThrottleRate = Integer.parseInt(throttleRate.getText()); } - if(routingStrategyType.compareTo(RoutingStrategyType.ZONE_STRATEGY) == 0) { + if(routingStrategyType.compareTo(RoutingStrategyType.ZONE_STRATEGY) == 0 + && !SystemStoreConstants.isSystemStore(name)) { if(zoneCountReads == null || zoneCountWrites == null || zoneReplicationFactor == null) { throw new MappingException("Have not set one of the following correctly for store '" + name From b59973e8b14a9093762a0048d92c74ce49665485 Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Thu, 15 Mar 2012 12:40:44 -0700 Subject: [PATCH 071/209] add clientId for voldemort client --- .../client/AbstractStoreClientFactory.java | 75 +++- .../client/CachingStoreClientFactory.java | 32 +- src/java/voldemort/client/ClientConfig.java | 15 + .../voldemort/client/DefaultStoreClient.java | 29 +- .../client/MockStoreClientFactory.java | 9 +- .../voldemort/client/StoreClientFactory.java | 17 +- .../voldemort/StaticStoreClientFactory.java | 10 +- test/unit/voldemort/client/ClientJmxTest.java | 341 ++++++++++++++++++ 8 files changed, 503 insertions(+), 25 deletions(-) create mode 100644 test/unit/voldemort/client/ClientJmxTest.java diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index 55d41f5e6f..9998846679 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -16,12 +16,17 @@ package voldemort.client; +import java.io.File; +import java.io.IOException; import java.io.StringReader; +import java.net.InetAddress; import java.net.URI; import java.net.URISyntaxException; +import java.net.UnknownHostException; import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -98,6 +103,8 @@ public abstract class AbstractStoreClientFactory implements StoreClientFactory { private final ClientConfig config; private final RoutedStoreFactory routedStoreFactory; private final int clientZoneId; + private final String clientContextName; + private final AtomicInteger sequencer; public AbstractStoreClientFactory(ClientConfig config) { this.config = config; @@ -112,18 +119,24 @@ public AbstractStoreClientFactory(ClientConfig config) { this.maxBootstrapRetries = config.getMaxBootstrapRetries(); this.stats = new StoreStats(); this.clientZoneId = config.getClientZoneId(); + this.clientContextName = (null == config.getClientContextName() ? "" + : config.getClientContextName()); this.routedStoreFactory = new RoutedStoreFactory(config.isPipelineRoutedStoreEnabled(), threadPool, config.getTimeoutConfig()); + this.sequencer = new AtomicInteger(0); if(this.isJmxEnabled) { JmxUtils.registerMbean(threadPool, JmxUtils.createObjectName(JmxUtils.getPackageName(threadPool.getClass()), JmxUtils.getClassName(threadPool.getClass()) + + "." + + clientContextName + jmxId())); JmxUtils.registerMbean(new StoreStatsJmx(stats), JmxUtils.createObjectName("voldemort.store.stats.aggregate", - "aggregate-perf" + jmxId())); + clientContextName + ".aggregate-perf" + + jmxId())); } } @@ -133,12 +146,18 @@ public StoreClient getStoreClient(String storeName) { public StoreClient getStoreClient(String storeName, InconsistencyResolver> resolver) { - return new DefaultStoreClient(storeName, resolver, this, 3); + return new DefaultStoreClient(storeName, + resolver, + this, + 3, + clientContextName, + sequencer.getAndIncrement()); } @SuppressWarnings("unchecked") public Store getRawStore(String storeName, - InconsistencyResolver> resolver) { + InconsistencyResolver> resolver, + UUID clientId) { logger.info("Client zone-id [" + clientZoneId + "] Attempting to obtain metadata for store [" + storeName + "] "); @@ -223,7 +242,13 @@ public Store getRawStore(String storeName, store = statStore; JmxUtils.registerMbean(new StoreStatsJmx(statStore.getStats()), JmxUtils.createObjectName(JmxUtils.getPackageName(store.getClass()), - store.getName() + jmxId())); + clientContextName + + "." + + store.getName() + + jmxId() + + (null == clientId ? "" + : "." + + clientId.toString()))); } if(storeDef.getKeySerializer().hasCompression() @@ -257,6 +282,11 @@ public Store getRawStore(String storeName, return serializedStore; } + public Store getRawStore(String storeName, + InconsistencyResolver> resolver) { + return getRawStore(storeName, resolver, null); + } + protected ClientConfig getConfig() { return config; } @@ -409,4 +439,41 @@ public String jmxId() { return jmxId == 0 ? "" : Integer.toString(jmxId); } + /** + * Generate a unique client ID based on: 0. clientContext, if specified; 1. + * storeName 2. run path 3. client sequence + * + * @param storeName the name of the store the client is created for + * @param contextName the name of the client context + * @param clientSequence the client sequence number + * @return unique client ID + */ + public static UUID generateClientId(String storeName, String contextName, int clientSequence) { + String newLine = System.getProperty("line.separator"); + StringBuilder context = new StringBuilder(contextName == null ? "" : contextName); + context.append(0 == clientSequence ? "" : ("." + clientSequence)); + context.append(".").append(storeName); + + try { + InetAddress host = InetAddress.getLocalHost(); + context.append("@").append(host.getHostName()).append(":"); + } catch(UnknownHostException e) { + logger.info("Unable to obtain client hostname."); + logger.info(e.getMessage()); + } + + try { + String currentPath = new File(".").getCanonicalPath(); + context.append(currentPath).append(newLine); + } catch(IOException e) { + logger.info("Unable to obtain client run path."); + logger.info(e.getMessage()); + } + + if(logger.isDebugEnabled()) { + logger.debug(context.toString()); + } + + return UUID.nameUUIDFromBytes(context.toString().getBytes()); + } } diff --git a/src/java/voldemort/client/CachingStoreClientFactory.java b/src/java/voldemort/client/CachingStoreClientFactory.java index 993ff7c3a5..5fd2953da3 100644 --- a/src/java/voldemort/client/CachingStoreClientFactory.java +++ b/src/java/voldemort/client/CachingStoreClientFactory.java @@ -1,12 +1,12 @@ /* * Copyright 2008-2010 LinkedIn, Inc - * + * * Licensed under the Apache License, Version 2.0 (the "License"); you may not * use this file except in compliance with the License. You may obtain a copy of * the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the @@ -16,8 +16,13 @@ package voldemort.client; -import com.google.common.collect.ImmutableList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + import org.apache.log4j.Logger; + import voldemort.annotations.jmx.JmxManaged; import voldemort.annotations.jmx.JmxOperation; import voldemort.cluster.failuredetector.FailureDetector; @@ -26,14 +31,12 @@ import voldemort.versioning.InconsistencyResolver; import voldemort.versioning.Versioned; -import java.util.List; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; +import com.google.common.collect.ImmutableList; /** - * A wrapper for a store {@link StoreClientFactory} which caches requests - * to getStoreClient - * + * A wrapper for a store {@link StoreClientFactory} which caches requests to + * getStoreClient + * */ @JmxManaged(description = "A StoreClientFactory which caches clients") public class CachingStoreClientFactory implements StoreClientFactory { @@ -48,7 +51,6 @@ public CachingStoreClientFactory(StoreClientFactory inner) { this.cache = new ConcurrentHashMap, StoreClient>(); } - @SuppressWarnings("unchecked") public StoreClient getStoreClient(String storeName) { Pair key = Pair.create(storeName, null); @@ -74,7 +76,13 @@ public StoreClient getStoreClient(String storeName, public Store getRawStore(String storeName, InconsistencyResolver> resolver) { - return inner.getRawStore(storeName, resolver); + return getRawStore(storeName, resolver, null); + } + + public Store getRawStore(String storeName, + InconsistencyResolver> resolver, + UUID clientId) { + return inner.getRawStore(storeName, resolver, clientId); } public void close() { diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index bc44a29830..4fb0011e2b 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -78,6 +78,7 @@ public class ClientConfig { private long failureDetectorRequestLengthThreshold = socketTimeoutMs; private volatile int maxBootstrapRetries = 2; + private volatile String clientContextName = "default"; public ClientConfig() {} @@ -118,6 +119,7 @@ public ClientConfig() {} public static final String FAILUREDETECTOR_CATASTROPHIC_ERROR_TYPES_PROPERTY = "failuredetector_catastrophic_error_types"; public static final String FAILUREDETECTOR_REQUEST_LENGTH_THRESHOLD_PROPERTY = "failuredetector_request_length_threshold"; public static final String MAX_BOOTSTRAP_RETRIES = "max_bootstrap_retries"; + public static final String CLIENT_CONTEXT_NAME = "voldemort_client_context"; /** * Instantiate the client config using a properties file @@ -276,6 +278,10 @@ private void setProperties(Properties properties) { if(props.containsKey(MAX_BOOTSTRAP_RETRIES)) this.setMaxBootstrapRetries(props.getInt(MAX_BOOTSTRAP_RETRIES)); + + if(props.containsKey(CLIENT_CONTEXT_NAME)) { + this.setClientContextName(props.getString(CLIENT_CONTEXT_NAME, null)); + } } public int getMaxConnectionsPerNode() { @@ -692,4 +698,13 @@ public ClientConfig setMaxBootstrapRetries(int maxBootstrapRetries) { return this; } + public String getClientContextName() { + return clientContextName; + } + + public ClientConfig setClientContextName(String clientContextName) { + this.clientContextName = clientContextName; + return this; + } + } diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index 832339b243..0f90c04a31 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -18,6 +18,7 @@ import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.Map.Entry; import org.apache.log4j.Logger; @@ -63,29 +64,45 @@ public class DefaultStoreClient implements StoreClient { private final String storeName; private final InconsistencyResolver> resolver; private volatile Store store; + private final UUID clientId; public DefaultStoreClient(String storeName, InconsistencyResolver> resolver, StoreClientFactory storeFactory, int maxMetadataRefreshAttempts) { + this(storeName, resolver, storeFactory, maxMetadataRefreshAttempts, null, 0); + } + + public DefaultStoreClient(String storeName, + InconsistencyResolver> resolver, + StoreClientFactory storeFactory, + int maxMetadataRefreshAttempts, + String clientContext, + int clientSequence) { + this.storeName = Utils.notNull(storeName); this.resolver = resolver; this.storeFactory = Utils.notNull(storeFactory); this.metadataRefreshAttempts = maxMetadataRefreshAttempts; - + this.clientId = AbstractStoreClientFactory.generateClientId(storeName, + clientContext, + clientSequence); // Registering self to be able to bootstrap client dynamically via JMX JmxUtils.registerMbean(this, JmxUtils.createObjectName(JmxUtils.getPackageName(this.getClass()), JmxUtils.getClassName(this.getClass()) - + "." + storeName)); - + + "." + clientContext + "." + + storeName + "." + + clientId.toString())); bootStrap(); + logger.info("Voldemort client created: clientContext=" + clientContext + " clientSequence=" + + clientSequence + " clientId=" + clientId.toString()); } @JmxOperation(description = "bootstrap metadata from the cluster.") public void bootStrap() { logger.info("Bootstrapping metadata for store " + this.storeName); - this.store = storeFactory.getRawStore(storeName, resolver); + this.store = storeFactory.getRawStore(storeName, resolver, clientId); } public boolean delete(K key) { @@ -355,4 +372,8 @@ else if(versions.size() == 1) return put(key, versioned, transforms); } + + public UUID getClientId() { + return clientId; + } } diff --git a/src/java/voldemort/client/MockStoreClientFactory.java b/src/java/voldemort/client/MockStoreClientFactory.java index 80613b653b..32a2997842 100644 --- a/src/java/voldemort/client/MockStoreClientFactory.java +++ b/src/java/voldemort/client/MockStoreClientFactory.java @@ -18,6 +18,7 @@ import java.io.StringReader; import java.util.List; +import java.util.UUID; import voldemort.cluster.failuredetector.FailureDetector; import voldemort.cluster.failuredetector.NoopFailureDetector; @@ -106,7 +107,8 @@ public StoreClient getStoreClient(String storeName, } public Store getRawStore(String storeName, - InconsistencyResolver> resolver) { + InconsistencyResolver> resolver, + UUID clientId) { if(this.storesXml != null) return getRawStore(storeName); @@ -131,6 +133,11 @@ public Store getRawStore(String storeName, return consistentStore; } + public Store getRawStore(String storeName, + InconsistencyResolver> resolver) { + return getRawStore(storeName, resolver, null); + } + private Store getRawStore(String storeName) { List storeDefs = storeMapper.readStoreList(new StringReader(storesXml)); StoreDefinition storeDef = null; diff --git a/src/java/voldemort/client/StoreClientFactory.java b/src/java/voldemort/client/StoreClientFactory.java index 166eac0c1d..f8fa8e710e 100644 --- a/src/java/voldemort/client/StoreClientFactory.java +++ b/src/java/voldemort/client/StoreClientFactory.java @@ -16,6 +16,8 @@ package voldemort.client; +import java.util.UUID; + import voldemort.cluster.failuredetector.FailureDetector; import voldemort.store.Store; import voldemort.versioning.InconsistencyResolver; @@ -65,9 +67,6 @@ public StoreClient getStoreClient(String storeName, /** * Get the underlying store, not the public StoreClient interface * - * @param The key type - * @param The value type - * @param The transform type * @param storeName The name of the store * @param resolver The inconsistency resolver * @return The appropriate store @@ -75,6 +74,18 @@ public StoreClient getStoreClient(String storeName, Store getRawStore(String storeName, InconsistencyResolver> resolver); + /** + * Get the underlying store, not the public StoreClient interface + * + * @param storeName The name of the store + * @param resolver The inconsistency resolver + * @param clientId The unique id of the client + * @return The appropriate store + */ + Store getRawStore(String storeName, + InconsistencyResolver> resolver, + UUID clientId); + /** * Close the store client */ diff --git a/test/common/voldemort/StaticStoreClientFactory.java b/test/common/voldemort/StaticStoreClientFactory.java index b93d1d73c0..c1ec4c4513 100644 --- a/test/common/voldemort/StaticStoreClientFactory.java +++ b/test/common/voldemort/StaticStoreClientFactory.java @@ -2,6 +2,7 @@ import java.util.Arrays; import java.util.List; +import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import voldemort.client.DefaultStoreClient; @@ -42,10 +43,17 @@ public StaticStoreClientFactory(Store... stores) { @SuppressWarnings("unchecked") public Store getRawStore(String storeName, - InconsistencyResolver> resolver) { + InconsistencyResolver> resolver, + UUID clientId) { return (Store) stores.get(Math.max(current.getAndIncrement(), stores.size() - 1)); } + @SuppressWarnings("unchecked") + public Store getRawStore(String storeName, + InconsistencyResolver> resolver) { + return getRawStore(storeName, resolver, null); + } + @SuppressWarnings("unchecked") public StoreClient getStoreClient(String storeName) { return new DefaultStoreClient(storeName, null, this, 3); diff --git a/test/unit/voldemort/client/ClientJmxTest.java b/test/unit/voldemort/client/ClientJmxTest.java new file mode 100644 index 0000000000..1c215580f8 --- /dev/null +++ b/test/unit/voldemort/client/ClientJmxTest.java @@ -0,0 +1,341 @@ +package voldemort.client; + +import java.lang.management.ManagementFactory; +import java.net.URISyntaxException; + +import javax.management.InstanceNotFoundException; +import javax.management.MBeanServer; +import javax.management.ObjectName; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import voldemort.ServerTestUtils; +import voldemort.serialization.SerializerFactory; +import voldemort.server.AbstractSocketService; +import voldemort.utils.JmxUtils; + +public class ClientJmxTest extends AbstractStoreClientFactoryTest { + + private static String STATS_DOMAIN = "voldemort.store.stats"; + private static String AGGREGATE_STATS_DOMAIN = "voldemort.store.stats.aggregate"; + private static String CLIENT_DOMAIN = "voldemort.client"; + private static String CLUSTER_FAILUREDETECTOR_DOMAIN = "voldemort.cluster.failuredetector"; + private static String CLIENT_REQUEST_DOMAIN = "voldemort.store.socket.clientrequest"; + + private AbstractSocketService socketService; + private MBeanServer mbServer = null; + + private static int factoryJmxId = 0; + + public ClientJmxTest() { + super(); + } + + private static String getAndIncrementJmxId() { + int current = factoryJmxId; + factoryJmxId++; + return (0 == current ? "" : "." + current); + } + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + socketService = ServerTestUtils.getSocketService(true, + getClusterXml(), + getStoreDefXml(), + getValidStoreName(), + getLocalNode().getSocketPort()); + socketService.start(); + mbServer = ManagementFactory.getPlatformMBeanServer(); + } + + @Override + @After + public void tearDown() throws Exception { + mbServer = null; + super.tearDown(); + socketService.stop(); + } + + @Override + protected StoreClientFactory getFactory(String... bootstrapUrls) { + return new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrls) + .setEnableLazy(false) + .setEnableJmx(true)); + } + + protected StoreClientFactory getFactoryWithClientContext(String clientContext, + String... bootstrapUrls) { + return new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrls) + .setEnableLazy(false) + .setClientContextName(clientContext) + .setEnableJmx(true)); + } + + @Test + public void testTwoClientContextOnJmx() throws Exception { + String clientContext1 = "clientA"; + String clientContext2 = "clientB"; + String jmxId1 = getAndIncrementJmxId(); + String jmxId2 = getAndIncrementJmxId(); + + StoreClient c1 = getFactoryWithClientContext(clientContext1, + getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + StoreClient c2 = getFactoryWithClientContext(clientContext2, + getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + + // checking for aggregate stats + ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext1 + + ".aggregate-perf" + + jmxId1); + ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext2 + + ".aggregate-perf" + + jmxId2); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + + // checking for per store stats + String c1type = clientContext1 + ".test" + jmxId1; + String c2type = clientContext2 + ".test" + jmxId2; + c1Name = JmxUtils.createObjectName(STATS_DOMAIN, + c1type + + "." + + ((DefaultStoreClient) c1).getClientId() + .toString()); + c2Name = JmxUtils.createObjectName(STATS_DOMAIN, + c2type + + "." + + ((DefaultStoreClient) c2).getClientId() + .toString()); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + } + + @Test + public void testSameContextOnJmx() throws Exception { + String clientContext = "clientContext"; + String jmxId1 = getAndIncrementJmxId(); + String jmxId2 = getAndIncrementJmxId(); + + StoreClient[] clients = new StoreClient[2]; + for(int i = 0; i < 2; i++) { + clients[i] = getFactoryWithClientContext(clientContext, getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + } + + // checking for aggregate stats + ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext + + ".aggregate-perf" + + jmxId1); + ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext + + ".aggregate-perf" + + jmxId2); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + + // checking for per store stats + String c1type = clientContext + ".test" + jmxId1; + String c2type = clientContext + ".test" + jmxId2; + c1Name = JmxUtils.createObjectName(STATS_DOMAIN, + c1type + + "." + + ((DefaultStoreClient) clients[0]).getClientId() + .toString()); + c2Name = JmxUtils.createObjectName(STATS_DOMAIN, + c2type + + "." + + ((DefaultStoreClient) clients[1]).getClientId() + .toString()); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + } + + @Test + public void testTwoClientNoContextOnJmx() throws Exception { + String clientContextCompare = "default"; + String jmxId1 = getAndIncrementJmxId(); + String jmxId2 = getAndIncrementJmxId(); + + StoreClient c1 = getFactory(getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + StoreClient c2 = getFactory(getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + + // checking for aggregate stats + ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare + + ".aggregate-perf" + + jmxId1); + ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare + + ".aggregate-perf" + + jmxId2); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + + // checking for per store stats + String c1type = clientContextCompare + ".test" + jmxId1; + String c2type = clientContextCompare + ".test" + jmxId2; + c1Name = JmxUtils.createObjectName(STATS_DOMAIN, + c1type + + "." + + ((DefaultStoreClient) c1).getClientId() + .toString()); + c2Name = JmxUtils.createObjectName(STATS_DOMAIN, + c2type + + "." + + ((DefaultStoreClient) c2).getClientId() + .toString()); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + } + + @Test + public void testTwoClientNullContextOnJmx() throws Exception { + String clientContextCompare = ""; + String jmxId1 = getAndIncrementJmxId(); + String jmxId2 = getAndIncrementJmxId(); + + StoreClient c1 = getFactoryWithClientContext(null, getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + StoreClient c2 = getFactoryWithClientContext(null, getValidBootstrapUrl()).getStoreClient(getValidStoreName()); + + // checking for aggregate stats + ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare + + ".aggregate-perf" + + jmxId1); + ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare + + ".aggregate-perf" + + jmxId2); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + + // checking for per store stats + String c1type = clientContextCompare + ".test" + jmxId1; + String c2type = clientContextCompare + ".test" + jmxId2; + c1Name = JmxUtils.createObjectName(STATS_DOMAIN, + c1type + + "." + + ((DefaultStoreClient) c1).getClientId() + .toString()); + c2Name = JmxUtils.createObjectName(STATS_DOMAIN, + c2type + + "." + + ((DefaultStoreClient) c2).getClientId() + .toString()); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + } + + @Test + public void testSameContextAndFactory() throws Exception { + String clientContext = "clientContext"; + String jmxId = getAndIncrementJmxId(); + StoreClientFactory factory = getFactoryWithClientContext(clientContext, + getValidBootstrapUrl()); + + StoreClient[] clients = new StoreClient[2]; + for(int i = 0; i < 2; i++) { + clients[i] = factory.getStoreClient(getValidStoreName()); + } + + ObjectName cName = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext + + ".aggregate-perf" + + jmxId); + checkForMbeanFound(cName); + mbServer.unregisterMBean(cName); + + // checking for per store stats + String ctype = clientContext + ".test" + jmxId; + ObjectName c1Name = JmxUtils.createObjectName(STATS_DOMAIN, + ctype + + "." + + ((DefaultStoreClient) clients[0]).getClientId() + .toString()); + ObjectName c2Name = JmxUtils.createObjectName(STATS_DOMAIN, + ctype + + "." + + ((DefaultStoreClient) clients[1]).getClientId() + .toString()); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + mbServer.unregisterMBean(c1Name); + } + + @Test + public void testDifferentId() throws Exception { + String clientContext = "clientContext"; + String jmxId = getAndIncrementJmxId(); + StoreClientFactory factory = getFactoryWithClientContext(clientContext, + getValidBootstrapUrl()); + + StoreClient[] clients = new StoreClient[2]; + clients[0] = factory.getStoreClient(getValidStoreName()); + clients[1] = factory.getStoreClient(getValidStoreName()); + + ObjectName cName = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext + + ".aggregate-perf" + + jmxId); + checkForMbeanFound(cName); + mbServer.unregisterMBean(cName); + + // checking for per store stats + String ctype = clientContext + ".test" + jmxId; + ObjectName c1Name = JmxUtils.createObjectName(STATS_DOMAIN, + ctype + + "." + + ((DefaultStoreClient) clients[0]).getClientId() + .toString()); + ObjectName c2Name = JmxUtils.createObjectName(STATS_DOMAIN, + ctype + + "." + + ((DefaultStoreClient) clients[1]).getClientId() + .toString()); + checkForMbeanFound(c1Name); + checkForMbeanFound(c2Name); + assertTrue(!c1Name.equals(c2Name)); + mbServer.unregisterMBean(c1Name); + mbServer.unregisterMBean(c2Name); + } + + private void checkForMbeanFound(ObjectName name) { + try { + mbServer.getMBeanInfo(name); + } catch(InstanceNotFoundException e) { + fail("MBean not found on the JMX Server: " + name.toString()); + } catch(Exception e) { + fail("Test failed: " + e.getMessage()); + } + } + + @Override + protected StoreClientFactory getFactoryWithSerializer(SerializerFactory factory, + String... bootstrapUrls) { + return new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrls) + .setEnableLazy(false) + .setSerializerFactory(factory)); + } + + @Override + protected String getValidBootstrapUrl() throws URISyntaxException { + return getLocalNode().getSocketUrl().toString(); + } + + @Override + protected String getValidScheme() { + return SocketStoreClientFactory.URL_SCHEME; + } +} From a3e9359c0667efd5b19863e9f5aac7dcb8bfb181 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Wed, 30 May 2012 18:56:26 -0700 Subject: [PATCH 072/209] Adding System store functionality --- .../client/AbstractStoreClientFactory.java | 12 ++- .../client/CachingStoreClientFactory.java | 7 ++ .../voldemort/client/DefaultStoreClient.java | 10 ++- .../client/MockStoreClientFactory.java | 7 ++ .../client/SocketStoreClientFactory.java | 4 + .../voldemort/client/StoreClientFactory.java | 14 +++ src/java/voldemort/client/SystemStore.java | 88 +++++++++++++++++++ .../server/SystemStoreConstants.java | 2 +- .../store/metadata/MetadataStore.java | 49 ++++++++++- .../voldemort/StaticStoreClientFactory.java | 7 ++ .../client/DefaultSocketStoreClientTest.java | 63 +++++++++++++ 11 files changed, 256 insertions(+), 7 deletions(-) create mode 100644 src/java/voldemort/client/SystemStore.java create mode 100644 test/unit/voldemort/client/DefaultSocketStoreClientTest.java diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index 9998846679..bfc36b2f2f 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -158,6 +158,14 @@ public StoreClient getStoreClient(String storeName, public Store getRawStore(String storeName, InconsistencyResolver> resolver, UUID clientId) { + return getRawStore(storeName, resolver, clientId, null); + } + + @SuppressWarnings("unchecked") + public Store getRawStore(String storeName, + InconsistencyResolver> resolver, + UUID clientId, + String customStoresXml) { logger.info("Client zone-id [" + clientZoneId + "] Attempting to obtain metadata for store [" + storeName + "] "); @@ -169,7 +177,9 @@ public Store getRawStore(String storeName, // Get cluster and store metadata String clusterXml = bootstrapMetadataWithRetries(MetadataStore.CLUSTER_KEY, bootstrapUrls); Cluster cluster = clusterMapper.readCluster(new StringReader(clusterXml), false); - String storesXml = bootstrapMetadataWithRetries(MetadataStore.STORES_KEY, bootstrapUrls); + String storesXml = customStoresXml; + if(storesXml == null) + storesXml = bootstrapMetadataWithRetries(MetadataStore.STORES_KEY, bootstrapUrls); if(logger.isDebugEnabled()) { logger.debug("Obtained cluster metadata xml" + clusterXml); diff --git a/src/java/voldemort/client/CachingStoreClientFactory.java b/src/java/voldemort/client/CachingStoreClientFactory.java index 5fd2953da3..dc8f94ef8f 100644 --- a/src/java/voldemort/client/CachingStoreClientFactory.java +++ b/src/java/voldemort/client/CachingStoreClientFactory.java @@ -123,4 +123,11 @@ else if(client instanceof LazyStoreClient) { logger.warn("Exception during bootstrapAllClients", e); } } + + public Store getRawStore(String storeName, + InconsistencyResolver> resolver, + UUID clientId, + String storesXml) { + return inner.getRawStore(storeName, resolver, clientId, storesXml); + } } diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index 0f90c04a31..c056f5ea10 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -18,8 +18,8 @@ import java.util.List; import java.util.Map; -import java.util.UUID; import java.util.Map.Entry; +import java.util.UUID; import org.apache.log4j.Logger; @@ -65,6 +65,7 @@ public class DefaultStoreClient implements StoreClient { private final InconsistencyResolver> resolver; private volatile Store store; private final UUID clientId; + private SystemStore sysStore; public DefaultStoreClient(String storeName, InconsistencyResolver> resolver, @@ -103,6 +104,13 @@ public DefaultStoreClient(String storeName, public void bootStrap() { logger.info("Bootstrapping metadata for store " + this.storeName); this.store = storeFactory.getRawStore(storeName, resolver, clientId); + + logger.info("Creating System store"); + String systemKey = storeName + "-client"; + this.sysStore = new SystemStore("voldsys$_client_registry", + this.storeFactory); + sysStore.putSysStore(systemKey, "Registered"); + logger.info("Getting value - " + sysStore.getSysStore(systemKey)); } public boolean delete(K key) { diff --git a/src/java/voldemort/client/MockStoreClientFactory.java b/src/java/voldemort/client/MockStoreClientFactory.java index 32a2997842..0cf6c4a8f8 100644 --- a/src/java/voldemort/client/MockStoreClientFactory.java +++ b/src/java/voldemort/client/MockStoreClientFactory.java @@ -213,4 +213,11 @@ public FailureDetector getFailureDetector() { return failureDetector; } + public Store getRawStore(String storeName, + InconsistencyResolver> resolver, + UUID clientId, + String storesXml) { + return null; + } + } diff --git a/src/java/voldemort/client/SocketStoreClientFactory.java b/src/java/voldemort/client/SocketStoreClientFactory.java index 18fab212d2..8843744da4 100644 --- a/src/java/voldemort/client/SocketStoreClientFactory.java +++ b/src/java/voldemort/client/SocketStoreClientFactory.java @@ -32,6 +32,7 @@ import voldemort.cluster.failuredetector.FailureDetectorConfig; import voldemort.cluster.failuredetector.FailureDetectorListener; import voldemort.server.RequestRoutingType; +import voldemort.server.SystemStoreConstants; import voldemort.store.Store; import voldemort.store.metadata.MetadataStore; import voldemort.store.socket.SocketDestination; @@ -182,4 +183,7 @@ public void close() { super.close(); } + public Store getSystemStore(String storeName) { + return getRawStore(storeName, null, null, SystemStoreConstants.SYSTEM_STORE_SCHEMA); + } } diff --git a/src/java/voldemort/client/StoreClientFactory.java b/src/java/voldemort/client/StoreClientFactory.java index f8fa8e710e..ee008d98af 100644 --- a/src/java/voldemort/client/StoreClientFactory.java +++ b/src/java/voldemort/client/StoreClientFactory.java @@ -86,6 +86,20 @@ Store getRawStore(String storeName, InconsistencyResolver> resolver, UUID clientId); + /** + * Get the underlying store, not the public StoreClient interface + * + * @param storeName The name of the store + * @param resolver The inconsistency resolver + * @param clientId The unique id of the client + * @param storesXml Custom set of stores containing storeName + * @return The appropriate store + */ + Store getRawStore(String storeName, + InconsistencyResolver> resolver, + UUID clientId, + String customStoresXml); + /** * Close the store client */ diff --git a/src/java/voldemort/client/SystemStore.java b/src/java/voldemort/client/SystemStore.java new file mode 100644 index 0000000000..7617fc4b24 --- /dev/null +++ b/src/java/voldemort/client/SystemStore.java @@ -0,0 +1,88 @@ +package voldemort.client; + +import java.util.List; +import java.util.concurrent.TimeUnit; + +import org.apache.log4j.Logger; + +import voldemort.VoldemortException; +import voldemort.server.SystemStoreConstants; +import voldemort.store.Store; +import voldemort.versioning.InconsistentDataException; +import voldemort.versioning.VectorClock; +import voldemort.versioning.Versioned; + +public class SystemStore { + + private final Logger logger = Logger.getLogger(DefaultStoreClient.class); + private final SocketStoreClientFactory systemStoreFactory; + private final String storeName; + private volatile Store sysStore; + + SystemStore(String storeName, StoreClientFactory factory) { + String prefix = storeName.substring(0, SystemStoreConstants.NAME_PREFIX.length()); + if(!SystemStoreConstants.NAME_PREFIX.equals(prefix)) + throw new VoldemortException("Illegal system store : " + storeName); + if(!(factory instanceof SocketStoreClientFactory)) + throw new VoldemortException("System store cannot be created without a Socket store client factory"); + + SocketStoreClientFactory clientFactory = (SocketStoreClientFactory) factory; + ClientConfig config = new ClientConfig(); + config.setSelectors(1) + .setBootstrapUrls(config.getBootstrapUrls()) + .setMaxConnectionsPerNode(2) + .setConnectionTimeout(1500, TimeUnit.MILLISECONDS) + .setSocketTimeout(5000, TimeUnit.MILLISECONDS) + .setRoutingTimeout(5000, TimeUnit.MILLISECONDS) + .setEnableJmx(false) + .setEnablePipelineRoutedStore(true) + .setClientZoneId(config.getClientZoneId()); + this.systemStoreFactory = new SocketStoreClientFactory(config); + this.storeName = storeName; + this.sysStore = this.systemStoreFactory.getSystemStore(this.storeName); + } + + public void putSysStore(K key, V value) throws VoldemortException { + logger.info("Invoking Put for key : " + key + " on store name : " + this.storeName); + Versioned versioned = getSysStore(key); + if(versioned == null) + versioned = Versioned.value(value, new VectorClock()); + else + versioned.setObject(value); + this.sysStore.put(key, versioned, null); + } + + public void putSysStore(K key, Versioned value) throws VoldemortException { + logger.info("Invoking Put for key : " + key + " on store name : " + this.storeName); + this.sysStore.put(key, value, null); + } + + public Versioned getSysStore(K key) throws VoldemortException { + logger.info("Invoking Get for key : " + key + " on store name : " + this.storeName); + Versioned versioned = null; + List> items = this.sysStore.get(key, null); + if(items.size() == 1) + versioned = items.get(0); + else if(items.size() > 1) + throw new InconsistentDataException("Unresolved versions returned from get(" + key + + ") = " + items, items); + if(versioned != null) + logger.info("Value for key : " + key + " = " + versioned.getValue() + + " on store name : " + this.storeName); + else + logger.info("Got null value"); + return versioned; + } + + public V getValueSysStore(K key) throws VoldemortException { + logger.info("Invoking Get for key : " + key + " on store name : " + this.storeName); + Versioned versioned = getSysStore(key); + if(versioned != null) { + logger.info("Value for key : " + key + " = " + versioned.getValue() + + " on store name : " + this.storeName); + return versioned.getValue(); + } + return null; + } + +} diff --git a/src/java/voldemort/server/SystemStoreConstants.java b/src/java/voldemort/server/SystemStoreConstants.java index 820037a920..cb048c66d1 100644 --- a/src/java/voldemort/server/SystemStoreConstants.java +++ b/src/java/voldemort/server/SystemStoreConstants.java @@ -5,7 +5,7 @@ */ public class SystemStoreConstants { - private static final String NAME_PREFIX = "voldsys$_"; + public static final String NAME_PREFIX = "voldsys$_"; public static enum SystemStoreName { voldsys$_client_registry, diff --git a/src/java/voldemort/store/metadata/MetadataStore.java b/src/java/voldemort/store/metadata/MetadataStore.java index 5a82acd99e..6d692ffe3f 100644 --- a/src/java/voldemort/store/metadata/MetadataStore.java +++ b/src/java/voldemort/store/metadata/MetadataStore.java @@ -41,6 +41,7 @@ import voldemort.routing.RouteToAllStrategy; import voldemort.routing.RoutingStrategy; import voldemort.routing.RoutingStrategyFactory; +import voldemort.server.SystemStoreConstants; import voldemort.server.rebalance.RebalancerState; import voldemort.store.StorageEngine; import voldemort.store.Store; @@ -73,6 +74,7 @@ public class MetadataStore implements StorageEngine { public static final String CLUSTER_KEY = "cluster.xml"; public static final String STORES_KEY = "stores.xml"; + public static final String SYSTEM_STORES_KEY = "systemStores"; public static final String SERVER_STATE_KEY = "server.state"; public static final String NODE_ID_KEY = "node.id"; public static final String REBALANCING_STEAL_INFO = "rebalancing.steal.info.key"; @@ -92,6 +94,7 @@ public class MetadataStore implements StorageEngine { // helper keys for metadataCacheOnly private static final String ROUTING_STRATEGY_KEY = "routing.strategy"; + private static final String SYSTEM_ROUTING_STRATEGY_KEY = "system.routing.strategy"; public static enum VoldemortState { NORMAL_SERVER, @@ -174,7 +177,8 @@ public synchronized void put(String key, Versioned value) { updateRoutingStrategies((Cluster) value.getValue(), getStoreDefList()); } else if(STORES_KEY.equals(key)) { updateRoutingStrategies(getCluster(), (List) value.getValue()); - } + } else if(SYSTEM_STORES_KEY.equals(key)) + throw new VoldemortException("Cannot overwrite system store definitions"); } else { throw new VoldemortException("Unhandled Key:" + key + " for MetadataStore put()"); @@ -191,8 +195,9 @@ public synchronized void put(String key, Versioned value) { public void put(String key, Object value) { if(METADATA_KEYS.contains(key)) { VectorClock version = (VectorClock) get(key, null).get(0).getVersion(); - put(key, new Versioned(value, version.incremented(getNodeId(), - System.currentTimeMillis()))); + put(key, + new Versioned(value, version.incremented(getNodeId(), + System.currentTimeMillis()))); } else { throw new VoldemortException("Unhandled Key:" + key + " for MetadataStore put()"); } @@ -297,6 +302,11 @@ public List getStoreDefList() { return (List) metadataCache.get(STORES_KEY).getValue(); } + @SuppressWarnings("unchecked") + public List getSystemStoreDefList() { + return (List) metadataCache.get(SYSTEM_STORES_KEY).getValue(); + } + public int getNodeId() { return (Integer) (metadataCache.get(NODE_ID_KEY).getValue()); } @@ -319,11 +329,21 @@ public RebalancerState getRebalancerState() { return (RebalancerState) metadataCache.get(REBALANCING_STEAL_INFO).getValue(); } + /* + * First check in the map of regular stores. If not present, check in the + * system stores map. + */ @SuppressWarnings("unchecked") public RoutingStrategy getRoutingStrategy(String storeName) { Map routingStrategyMap = (Map) metadataCache.get(ROUTING_STRATEGY_KEY) .getValue(); - return routingStrategyMap.get(storeName); + RoutingStrategy strategy = routingStrategyMap.get(storeName); + if(strategy == null) { + Map systemRoutingStrategyMap = (Map) metadataCache.get(SYSTEM_ROUTING_STRATEGY_KEY) + .getValue(); + strategy = systemRoutingStrategyMap.get(storeName); + } + return strategy; } /** @@ -362,6 +382,17 @@ private void updateRoutingStrategies(Cluster cluster, List stor } } + /* + * Initialize the routing strategy map for system stores. This is used + * during get / put on system stores. + */ + private void initSystemRoutingStrategies(Cluster cluster) { + HashMap routingStrategyMap = createRoutingStrategyMap(cluster, + getSystemStoreDefList()); + this.metadataCache.put(SYSTEM_ROUTING_STRATEGY_KEY, + new Versioned(routingStrategyMap)); + } + /** * Add the steal information to the rebalancer state * @@ -452,6 +483,10 @@ private void init(int nodeId) { initCache(CLUSTER_KEY); initCache(STORES_KEY); + // Initialize system store in the metadata cache + initSystemCache(); + initSystemRoutingStrategies(getCluster()); + initCache(NODE_ID_KEY, nodeId); if(getNodeId() != nodeId) throw new RuntimeException("Attempt to start previous node:" @@ -473,6 +508,12 @@ private synchronized void initCache(String key) { metadataCache.put(key, convertStringToObject(key, getInnerValue(key))); } + // Initialize the metadata cache with system store list + private synchronized void initSystemCache() { + List value = storeMapper.readStoreList(new StringReader(SystemStoreConstants.SYSTEM_STORE_SCHEMA)); + metadataCache.put(SYSTEM_STORES_KEY, new Versioned(value)); + } + private void initCache(String key, Object defaultValue) { try { initCache(key); diff --git a/test/common/voldemort/StaticStoreClientFactory.java b/test/common/voldemort/StaticStoreClientFactory.java index c1ec4c4513..913b75b188 100644 --- a/test/common/voldemort/StaticStoreClientFactory.java +++ b/test/common/voldemort/StaticStoreClientFactory.java @@ -74,4 +74,11 @@ public FailureDetector getFailureDetector() { return failureDetector; } + public Store getRawStore(String storeName, + InconsistencyResolver> resolver, + UUID clientId, + String storesXml) { + return null; + } + } diff --git a/test/unit/voldemort/client/DefaultSocketStoreClientTest.java b/test/unit/voldemort/client/DefaultSocketStoreClientTest.java new file mode 100644 index 0000000000..07b8db82f5 --- /dev/null +++ b/test/unit/voldemort/client/DefaultSocketStoreClientTest.java @@ -0,0 +1,63 @@ +package voldemort.client; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.fail; + +import org.junit.Before; +import org.junit.Test; + +import voldemort.utils.SystemTime; +import voldemort.utils.Time; +import voldemort.versioning.ObsoleteVersionException; +import voldemort.versioning.VectorClock; +import voldemort.versioning.Versioned; + +public class DefaultSocketStoreClientTest { + + protected StoreClient client; + protected int nodeId; + protected Time time; + + @Before + public void setUp() throws Exception { + String socketUrl = "tcp://localhost:6667"; + String storeName = "test"; + ClientConfig clientConfig = new ClientConfig().setMaxThreads(4) + .setMaxTotalConnections(4) + .setMaxConnectionsPerNode(4) + .setBootstrapUrls(socketUrl); + SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); + this.client = socketFactory.getStoreClient(storeName); + this.nodeId = 0; + this.time = SystemTime.INSTANCE; + } + + @Test + public void test() { + client.put("k", Versioned.value("v")); + Versioned v = client.get("k"); + assertEquals("GET should return the version set by PUT.", "v", v.getValue()); + VectorClock expected = new VectorClock(); + expected.incrementVersion(nodeId, time.getMilliseconds()); + assertEquals("The version should be incremented after a put.", expected, v.getVersion()); + try { + client.put("k", Versioned.value("v")); + fail("Put of obsolete version should throw exception."); + } catch(ObsoleteVersionException e) { + // this is good + } + // PUT of a concurrent version should succeed + client.put("k", + new Versioned("v2", + new VectorClock().incremented(nodeId + 1, + time.getMilliseconds()))); + assertEquals("GET should return the new value set by PUT.", "v2", client.getValue("k")); + assertEquals("GET should return the new version set by PUT.", + expected.incremented(nodeId + 1, time.getMilliseconds()), + client.get("k").getVersion()); + client.delete("k"); + assertNull("After a successful delete(k), get(k) should return null.", client.get("k")); + } + +} From 441a936d5cc120699146e953a6003e8be59b49af Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Tue, 5 Jun 2012 15:59:35 -0700 Subject: [PATCH 073/209] Added the Voldemort Client automated re-bootstrap mechanism --- src/java/voldemort/VoldemortAdminTool.java | 31 ++++ .../client/AbstractStoreClientFactory.java | 3 +- .../client/AsyncMetadataVersionManager.java | 125 +++++++++++++++ src/java/voldemort/client/ClientConfig.java | 14 ++ .../voldemort/client/DefaultStoreClient.java | 64 ++++++-- src/java/voldemort/client/SystemStore.java | 23 ++- .../routing/RouteToAllLocalPrefStrategy.java | 17 ++ .../routing/RoutingStrategyFactory.java | 2 + .../routing/RoutingStrategyType.java | 1 + .../server/SystemStoreConstants.java | 21 ++- .../store/routed/PipelineRoutedStore.java | 87 ++++++++++- .../routed/action/ConfigureNodesByZone.java | 147 ++++++++++++++++++ .../routed/action/ConfigureNodesDefault.java | 71 +++++++++ .../action/ConfigureNodesLocalHost.java | 71 +++++++++ .../action/ConfigureNodesLocalHostByZone.java | 72 +++++++++ 15 files changed, 716 insertions(+), 33 deletions(-) create mode 100644 src/java/voldemort/client/AsyncMetadataVersionManager.java create mode 100644 src/java/voldemort/routing/RouteToAllLocalPrefStrategy.java create mode 100644 src/java/voldemort/store/routed/action/ConfigureNodesByZone.java create mode 100644 src/java/voldemort/store/routed/action/ConfigureNodesDefault.java create mode 100644 src/java/voldemort/store/routed/action/ConfigureNodesLocalHost.java create mode 100644 src/java/voldemort/store/routed/action/ConfigureNodesLocalHostByZone.java diff --git a/src/java/voldemort/VoldemortAdminTool.java b/src/java/voldemort/VoldemortAdminTool.java index bc22f8ab56..5629691e97 100644 --- a/src/java/voldemort/VoldemortAdminTool.java +++ b/src/java/voldemort/VoldemortAdminTool.java @@ -45,6 +45,7 @@ import org.codehaus.jackson.JsonGenerator; import org.codehaus.jackson.map.ObjectMapper; +import voldemort.client.SystemStore; import voldemort.client.protocol.admin.AdminClient; import voldemort.client.protocol.admin.AdminClientConfig; import voldemort.cluster.Cluster; @@ -85,6 +86,7 @@ public class VoldemortAdminTool { private static final String ALL_METADATA = "all"; + private static SystemStore sysStoreVersion = null; @SuppressWarnings("unchecked") public static void main(String[] args) throws Exception { @@ -250,6 +252,13 @@ public static void main(String[] args) throws Exception { AdminClient adminClient = new AdminClient(url, new AdminClientConfig()); + // Initialize the system store for stores.xml version + String[] bootstrapUrls = new String[1]; + bootstrapUrls[0] = url; + sysStoreVersion = new SystemStore("voldsys$_metadata_version", + bootstrapUrls, + 0); + String ops = ""; if(options.has("delete-partitions")) { ops += "d"; @@ -433,6 +442,10 @@ public static void main(String[] args) throws Exception { adminClient, MetadataStore.STORES_KEY, mapper.writeStoreList(storeDefs)); + + // Update the store metadata version + updateStoreMetadataversion(); + } else if(metadataKey.compareTo(MetadataStore.REBALANCING_STEAL_INFO) == 0) { if(!Utils.isReadableFile(metadataValue)) throw new VoldemortException("Rebalancing steal info file path incorrect"); @@ -722,6 +735,24 @@ private static void executeCheckMetadata(AdminClient adminClient, String metadat } } + /* + * TODO: For now write one version for the entire stores.xml When we split + * the stores.xml, make this more granular + */ + private static void updateStoreMetadataversion() { + String versionKey = "stores.xml"; + Versioned storesVersion = sysStoreVersion.getSysStore(versionKey); + if(storesVersion == null) { + System.err.println("Current version is null. Assuming version 0."); + storesVersion = new Versioned((long) 1); + } else { + System.out.println("Version obtained = " + storesVersion.getValue()); + long newValue = storesVersion.getValue() + 1; + storesVersion.setObject(newValue); + } + sysStoreVersion.putSysStore(versionKey, storesVersion); + } + private static void executeSetMetadata(Integer nodeId, AdminClient adminClient, String key, diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index bfc36b2f2f..89ba43b3c2 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -151,7 +151,8 @@ public StoreClient getStoreClient(String storeName, this, 3, clientContextName, - sequencer.getAndIncrement()); + sequencer.getAndIncrement(), + config); } @SuppressWarnings("unchecked") diff --git a/src/java/voldemort/client/AsyncMetadataVersionManager.java b/src/java/voldemort/client/AsyncMetadataVersionManager.java new file mode 100644 index 0000000000..a0ce826e8b --- /dev/null +++ b/src/java/voldemort/client/AsyncMetadataVersionManager.java @@ -0,0 +1,125 @@ +package voldemort.client; + +import java.lang.Thread.UncaughtExceptionHandler; +import java.util.Random; +import java.util.concurrent.Callable; + +import org.apache.log4j.Level; +import org.apache.log4j.Logger; + +import voldemort.versioning.Versioned; + +/* + * The AsyncMetadataVersionManager is used to track the Metadata version on the + * cluster and if necessary Re-bootstrap the client. + * + * During initialization, it will retrieve the current version of the store (or + * the entire stores.xml depending upon granularity) and then periodically check + * whether this has been updated. During init if the initial version turns out + * to be null, it means that no change has been done to that store since it was + * created. In this case, we assume version '0'. + */ + +public class AsyncMetadataVersionManager implements Runnable { + + private final Logger logger = Logger.getLogger(this.getClass()); + private Versioned currentVersion; + private final SystemStore sysStore; + private final String systemKey = "stores.xml"; + private volatile boolean isRunning; + private final Callable storeClientThunk; + private long asyncMetadataCheckInterval; + + // Random delta generator + final int DELTA_MAX = 1000; + Random randomGenerator = new Random(System.currentTimeMillis()); + + public AsyncMetadataVersionManager(SystemStore systemStore, + long asyncMetadataCheckInterval, + Callable storeClientThunk) { + this(null, systemStore, asyncMetadataCheckInterval, storeClientThunk); + } + + public AsyncMetadataVersionManager(Versioned initialVersion, + SystemStore systemStore, + long asyncMetadataCheckInterval, + Callable storeClientThunk) { + this.sysStore = systemStore; + if(initialVersion == null) { + this.currentVersion = sysStore.getSysStore("stores.xml"); + + // If the received store version is null, assume version 0 + if(currentVersion == null) + currentVersion = new Versioned((long) 0); + } else { + currentVersion = initialVersion; + } + + // Initialize and start the background check thread + isRunning = true; + + Thread checkVersionThread = new Thread(this, "AsyncVersionCheckThread"); + checkVersionThread.setDaemon(true); + checkVersionThread.setUncaughtExceptionHandler(new UncaughtExceptionHandler() { + + public void uncaughtException(Thread t, Throwable e) { + if(logger.isEnabledFor(Level.ERROR)) + logger.error("Uncaught exception in Metadata Version check thread:", e); + } + }); + + this.storeClientThunk = storeClientThunk; + this.asyncMetadataCheckInterval = asyncMetadataCheckInterval; + checkVersionThread.start(); + + } + + public void destroy() { + isRunning = false; + } + + public void run() { + while(!Thread.currentThread().isInterrupted() && isRunning) { + try { + Thread.sleep(asyncMetadataCheckInterval); + } catch(InterruptedException e) { + break; + } + + Versioned newVersion = this.sysStore.getSysStore(systemKey); + + // If version obtained is null, the store is untouched. Continue + if(newVersion == null) { + logger.info("Metadata unchanged after creation ..."); + continue; + } + + logger.info("MetadataVersion check => Obtained " + systemKey + " version : " + + newVersion); + + if(!newVersion.equals(currentVersion)) { + logger.info("Metadata version mismatch detected."); + + // Determine a random delta delay between 0 to 1000 (ms) + int delta = randomGenerator.nextInt(DELTA_MAX); + + try { + logger.info("Sleeping for delta : " + delta + " (ms) before re-bootstrapping."); + Thread.sleep(delta); + } catch(InterruptedException e) { + break; + } + + // Invoke callback for bootstrap + try { + this.storeClientThunk.call(); + } catch(Exception e) { + e.printStackTrace(); + } + + // Update the current version + currentVersion = newVersion; + } + } + } +} diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index 4fb0011e2b..4751f74be8 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -79,6 +79,7 @@ public class ClientConfig { private volatile int maxBootstrapRetries = 2; private volatile String clientContextName = "default"; + private volatile long asyncCheckMetadataInterval = 5000; public ClientConfig() {} @@ -120,6 +121,7 @@ public ClientConfig() {} public static final String FAILUREDETECTOR_REQUEST_LENGTH_THRESHOLD_PROPERTY = "failuredetector_request_length_threshold"; public static final String MAX_BOOTSTRAP_RETRIES = "max_bootstrap_retries"; public static final String CLIENT_CONTEXT_NAME = "voldemort_client_context"; + public static final String ASYNC_CHECK_METADATA_INTERVAL = "check_metadata_interval"; /** * Instantiate the client config using a properties file @@ -282,6 +284,10 @@ private void setProperties(Properties properties) { if(props.containsKey(CLIENT_CONTEXT_NAME)) { this.setClientContextName(props.getString(CLIENT_CONTEXT_NAME, null)); } + + if(props.containsKey(ASYNC_CHECK_METADATA_INTERVAL)) { + this.setAsyncCheckMetadataInterval(props.getLong(ASYNC_CHECK_METADATA_INTERVAL, 5000)); + } } public int getMaxConnectionsPerNode() { @@ -707,4 +713,12 @@ public ClientConfig setClientContextName(String clientContextName) { return this; } + public long getAsyncCheckMetadataInterval() { + return asyncCheckMetadataInterval; + } + + public void setAsyncCheckMetadataInterval(long asyncCheckMetadataInterval) { + this.asyncCheckMetadataInterval = asyncCheckMetadataInterval; + } + } diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index c056f5ea10..436fcfd292 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -16,10 +16,12 @@ package voldemort.client; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.UUID; +import java.util.concurrent.Callable; import org.apache.log4j.Logger; @@ -60,26 +62,37 @@ public class DefaultStoreClient implements StoreClient { private final Logger logger = Logger.getLogger(DefaultStoreClient.class); private final StoreClientFactory storeFactory; + private final ClientConfig config; private final int metadataRefreshAttempts; private final String storeName; private final InconsistencyResolver> resolver; private volatile Store store; private final UUID clientId; - private SystemStore sysStore; + private final Map sysStoreMap; + private AsyncMetadataVersionManager asyncCheckMetadata; + + // Enumerate all the system stores + private final String METADATA_VERSION_STORE = "voldsys$_metadata_version"; + private final String CLIENT_REGISTRY_STORE = "voldsys$_client_registry"; + private final String STORE_DEFINITION_STORE = "voldsys$_client_store_definition"; + private final String[] systemStoreNames = { METADATA_VERSION_STORE, CLIENT_REGISTRY_STORE, + STORE_DEFINITION_STORE }; public DefaultStoreClient(String storeName, InconsistencyResolver> resolver, StoreClientFactory storeFactory, int maxMetadataRefreshAttempts) { - this(storeName, resolver, storeFactory, maxMetadataRefreshAttempts, null, 0); + this(storeName, resolver, storeFactory, maxMetadataRefreshAttempts, null, 0, null); } + @SuppressWarnings("unchecked") public DefaultStoreClient(String storeName, InconsistencyResolver> resolver, StoreClientFactory storeFactory, int maxMetadataRefreshAttempts, String clientContext, - int clientSequence) { + int clientSequence, + ClientConfig config) { this.storeName = Utils.notNull(storeName); this.resolver = resolver; @@ -88,6 +101,8 @@ public DefaultStoreClient(String storeName, this.clientId = AbstractStoreClientFactory.generateClientId(storeName, clientContext, clientSequence); + this.config = config; + // Registering self to be able to bootstrap client dynamically via JMX JmxUtils.registerMbean(this, JmxUtils.createObjectName(JmxUtils.getPackageName(this.getClass()), @@ -96,21 +111,50 @@ public DefaultStoreClient(String storeName, + storeName + "." + clientId.toString())); bootStrap(); + + // Initialize all the system stores + sysStoreMap = new HashMap(); + initializeSystemStores(); + + // Initialize the background thread for checking metadata version + if(config != null) { + SystemStore versionStore = this.sysStoreMap.get(METADATA_VERSION_STORE); + if(versionStore == null) + logger.info("Metadata version system store not found. Cannot run Metadata version check thread."); + else { + Callable bootstrapCallback = new Callable() { + + public Void call() throws Exception { + bootStrap(); + return null; + } + }; + + asyncCheckMetadata = new AsyncMetadataVersionManager(versionStore, + config.getAsyncCheckMetadataInterval(), + bootstrapCallback); + logger.info("Metadata version check thread started. Frequency = Every " + + config.getAsyncCheckMetadataInterval() + " ms"); + } + } + logger.info("Voldemort client created: clientContext=" + clientContext + " clientSequence=" + clientSequence + " clientId=" + clientId.toString()); } + public void initializeSystemStores() { + for(String storeName: systemStoreNames) { + SystemStore sysStore = new SystemStore(storeName, + config.getBootstrapUrls(), + config.getClientZoneId()); + this.sysStoreMap.put(storeName, sysStore); + } + } + @JmxOperation(description = "bootstrap metadata from the cluster.") public void bootStrap() { logger.info("Bootstrapping metadata for store " + this.storeName); this.store = storeFactory.getRawStore(storeName, resolver, clientId); - - logger.info("Creating System store"); - String systemKey = storeName + "-client"; - this.sysStore = new SystemStore("voldsys$_client_registry", - this.storeFactory); - sysStore.putSysStore(systemKey, "Registered"); - logger.info("Getting value - " + sysStore.getSysStore(systemKey)); } public boolean delete(K key) { diff --git a/src/java/voldemort/client/SystemStore.java b/src/java/voldemort/client/SystemStore.java index 7617fc4b24..d4c901ebbe 100644 --- a/src/java/voldemort/client/SystemStore.java +++ b/src/java/voldemort/client/SystemStore.java @@ -19,14 +19,11 @@ public class SystemStore { private final String storeName; private volatile Store sysStore; - SystemStore(String storeName, StoreClientFactory factory) { + public SystemStore(String storeName, String[] bootstrapUrls, int clientZoneID) { String prefix = storeName.substring(0, SystemStoreConstants.NAME_PREFIX.length()); if(!SystemStoreConstants.NAME_PREFIX.equals(prefix)) throw new VoldemortException("Illegal system store : " + storeName); - if(!(factory instanceof SocketStoreClientFactory)) - throw new VoldemortException("System store cannot be created without a Socket store client factory"); - SocketStoreClientFactory clientFactory = (SocketStoreClientFactory) factory; ClientConfig config = new ClientConfig(); config.setSelectors(1) .setBootstrapUrls(config.getBootstrapUrls()) @@ -43,7 +40,7 @@ public class SystemStore { } public void putSysStore(K key, V value) throws VoldemortException { - logger.info("Invoking Put for key : " + key + " on store name : " + this.storeName); + logger.debug("Invoking Put for key : " + key + " on store name : " + this.storeName); Versioned versioned = getSysStore(key); if(versioned == null) versioned = Versioned.value(value, new VectorClock()); @@ -53,12 +50,12 @@ public void putSysStore(K key, V value) throws VoldemortException { } public void putSysStore(K key, Versioned value) throws VoldemortException { - logger.info("Invoking Put for key : " + key + " on store name : " + this.storeName); + logger.debug("Invoking Put for key : " + key + " on store name : " + this.storeName); this.sysStore.put(key, value, null); } public Versioned getSysStore(K key) throws VoldemortException { - logger.info("Invoking Get for key : " + key + " on store name : " + this.storeName); + logger.debug("Invoking Get for key : " + key + " on store name : " + this.storeName); Versioned versioned = null; List> items = this.sysStore.get(key, null); if(items.size() == 1) @@ -67,19 +64,19 @@ else if(items.size() > 1) throw new InconsistentDataException("Unresolved versions returned from get(" + key + ") = " + items, items); if(versioned != null) - logger.info("Value for key : " + key + " = " + versioned.getValue() - + " on store name : " + this.storeName); + logger.debug("Value for key : " + key + " = " + versioned.getValue() + + " on store name : " + this.storeName); else - logger.info("Got null value"); + logger.debug("Got null value"); return versioned; } public V getValueSysStore(K key) throws VoldemortException { - logger.info("Invoking Get for key : " + key + " on store name : " + this.storeName); + logger.debug("Invoking Get for key : " + key + " on store name : " + this.storeName); Versioned versioned = getSysStore(key); if(versioned != null) { - logger.info("Value for key : " + key + " = " + versioned.getValue() - + " on store name : " + this.storeName); + logger.debug("Value for key : " + key + " = " + versioned.getValue() + + " on store name : " + this.storeName); return versioned.getValue(); } return null; diff --git a/src/java/voldemort/routing/RouteToAllLocalPrefStrategy.java b/src/java/voldemort/routing/RouteToAllLocalPrefStrategy.java new file mode 100644 index 0000000000..240aff1d49 --- /dev/null +++ b/src/java/voldemort/routing/RouteToAllLocalPrefStrategy.java @@ -0,0 +1,17 @@ +package voldemort.routing; + +import java.util.Collection; + +import voldemort.cluster.Node; + +public class RouteToAllLocalPrefStrategy extends RouteToAllStrategy { + + public RouteToAllLocalPrefStrategy(Collection nodes) { + super(nodes); + } + + @Override + public String getType() { + return RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY; + } +} diff --git a/src/java/voldemort/routing/RoutingStrategyFactory.java b/src/java/voldemort/routing/RoutingStrategyFactory.java index c8b8fe0158..b4901ba40c 100644 --- a/src/java/voldemort/routing/RoutingStrategyFactory.java +++ b/src/java/voldemort/routing/RoutingStrategyFactory.java @@ -24,6 +24,8 @@ public RoutingStrategy updateRoutingStrategy(StoreDefinition storeDef, Cluster c return new ZoneRoutingStrategy(cluster.getNodes(), storeDef.getZoneReplicationFactor(), storeDef.getReplicationFactor()); + } else if(RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY.equals(storeDef.getRoutingStrategyType())) { + return new RouteToAllLocalPrefStrategy(cluster.getNodes()); } else { throw new VoldemortException("RoutingStrategyType:" + storeDef.getRoutingStrategyType() + " not handled by " + this.getClass()); diff --git a/src/java/voldemort/routing/RoutingStrategyType.java b/src/java/voldemort/routing/RoutingStrategyType.java index 92ead1b1ae..60c1a49733 100644 --- a/src/java/voldemort/routing/RoutingStrategyType.java +++ b/src/java/voldemort/routing/RoutingStrategyType.java @@ -10,6 +10,7 @@ public class RoutingStrategyType { public final static String CONSISTENT_STRATEGY = "consistent-routing"; public final static String TO_ALL_STRATEGY = "all-routing"; public final static String ZONE_STRATEGY = "zone-routing"; + public final static String TO_ALL_LOCAL_PREF_STRATEGY = "local-pref-all-routing"; private final String name; diff --git a/src/java/voldemort/server/SystemStoreConstants.java b/src/java/voldemort/server/SystemStoreConstants.java index cb048c66d1..a7c546681c 100644 --- a/src/java/voldemort/server/SystemStoreConstants.java +++ b/src/java/voldemort/server/SystemStoreConstants.java @@ -34,6 +34,7 @@ public static enum SystemStoreName { + " " + " 7" + " " + + " " + " voldsys$_client_store_definition" + " zone-routing" @@ -50,7 +51,25 @@ public static enum SystemStoreName { + " string" + " " + " 7" - + " " + ""; + + " " + + + " " + + " voldsys$_metadata_version" + + " local-pref-all-routing" + + " proximity-handoff" + + " memory" + + " client" + + " 1" + + " 1" + + " 1" + + " " + + " string" + + " " + + " " + + " java-serialization" + + " " + " " + + + ""; public static boolean isSystemStore(String storeName) { return (null == storeName ? false : storeName.startsWith(NAME_PREFIX)); diff --git a/src/java/voldemort/store/routed/PipelineRoutedStore.java b/src/java/voldemort/store/routed/PipelineRoutedStore.java index 38ff90517d..3016a097a3 100644 --- a/src/java/voldemort/store/routed/PipelineRoutedStore.java +++ b/src/java/voldemort/store/routed/PipelineRoutedStore.java @@ -37,7 +37,12 @@ import voldemort.store.nonblockingstore.NonblockingStore; import voldemort.store.routed.Pipeline.Event; import voldemort.store.routed.Pipeline.Operation; +import voldemort.store.routed.action.AbstractConfigureNodes; import voldemort.store.routed.action.ConfigureNodes; +import voldemort.store.routed.action.ConfigureNodesByZone; +import voldemort.store.routed.action.ConfigureNodesDefault; +import voldemort.store.routed.action.ConfigureNodesLocalHost; +import voldemort.store.routed.action.ConfigureNodesLocalHostByZone; import voldemort.store.routed.action.GetAllConfigureNodes; import voldemort.store.routed.action.GetAllReadRepair; import voldemort.store.routed.action.IncrementClock; @@ -81,6 +86,13 @@ public class PipelineRoutedStore extends RoutedStore { private PipelineRoutedStats stats; private final int jmxId; + private enum ConfigureNodesType { + DEFAULT, + BYZONE, + DEFAULT_LOCAL, + BYZONE_LOCAL + } + /** * Create a PipelineRoutedStore * @@ -141,6 +153,66 @@ public PipelineRoutedStore(String name, } } + private ConfigureNodesType getNodeConfigurationType(BasicPipelineData>> pipelineData) { + // If Zone and local preference required + if(pipelineData.getZonesRequired() != null + && routingStrategy.getType().equals(RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY)) + return ConfigureNodesType.BYZONE_LOCAL; + + // If only local preference required + else if(pipelineData.getZonesRequired() == null + && routingStrategy.getType().equals(RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY)) + return ConfigureNodesType.DEFAULT_LOCAL; + + // If only Zone required + else if(pipelineData.getZonesRequired() != null + && !routingStrategy.getType() + .equals(RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY)) + return ConfigureNodesType.BYZONE; + + // Default case + return ConfigureNodesType.DEFAULT; + } + + private AbstractConfigureNodes>, BasicPipelineData>>> getNodeConfiguration(BasicPipelineData>> pipelineData, + ByteArray key) { + switch(getNodeConfigurationType(pipelineData)) { + case DEFAULT: + return new ConfigureNodesDefault>, BasicPipelineData>>>(pipelineData, + Event.CONFIGURED, + failureDetector, + storeDef.getRequiredReads(), + routingStrategy, + key); + case BYZONE: + return new ConfigureNodesByZone>, BasicPipelineData>>>(pipelineData, + Event.CONFIGURED, + failureDetector, + storeDef.getRequiredReads(), + routingStrategy, + key, + clientZone); + case DEFAULT_LOCAL: + return new ConfigureNodesLocalHost>, BasicPipelineData>>>(pipelineData, + Event.CONFIGURED, + failureDetector, + storeDef.getRequiredReads(), + routingStrategy, + key); + case BYZONE_LOCAL: + return new ConfigureNodesLocalHostByZone>, BasicPipelineData>>>(pipelineData, + Event.CONFIGURED, + failureDetector, + storeDef.getRequiredReads(), + routingStrategy, + key, + clientZone); + default: + return null; + } + + } + public List> get(final ByteArray key, final byte[] transforms) { StoreUtils.assertValidKey(key); @@ -164,14 +236,13 @@ public List> request(Store store) { }; - pipeline.addEventAction(Event.STARTED, - new ConfigureNodes>, BasicPipelineData>>>(pipelineData, - Event.CONFIGURED, - failureDetector, - storeDef.getRequiredReads(), - routingStrategy, - key, - clientZone)); + // Get the correct type of configure nodes action depending on the store + // requirements + AbstractConfigureNodes>, BasicPipelineData>>> configureNodes = getNodeConfiguration(pipelineData, + key); + + pipeline.addEventAction(Event.STARTED, configureNodes); + pipeline.addEventAction(Event.CONFIGURED, new PerformParallelRequests>, BasicPipelineData>>>(pipelineData, allowReadRepair ? Event.RESPONSES_RECEIVED diff --git a/src/java/voldemort/store/routed/action/ConfigureNodesByZone.java b/src/java/voldemort/store/routed/action/ConfigureNodesByZone.java new file mode 100644 index 0000000000..7a194f649b --- /dev/null +++ b/src/java/voldemort/store/routed/action/ConfigureNodesByZone.java @@ -0,0 +1,147 @@ +/* + * Copyright 2010 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package voldemort.store.routed.action; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +import voldemort.VoldemortException; +import voldemort.cluster.Node; +import voldemort.cluster.Zone; +import voldemort.cluster.failuredetector.FailureDetector; +import voldemort.routing.RoutingStrategy; +import voldemort.store.routed.BasicPipelineData; +import voldemort.store.routed.Pipeline; +import voldemort.store.routed.Pipeline.Event; +import voldemort.store.routed.Pipeline.Operation; +import voldemort.utils.ByteArray; +import voldemort.utils.ByteUtils; + +/* + * Configure the Nodes obtained via the routing strategy based on the zone + * information. Local zone nodes first, followed by the corresponding nodes from + * each of the other zones, ordered by proximity. + */ +public class ConfigureNodesByZone> extends + AbstractConfigureNodes { + + private final ByteArray key; + + private final Zone clientZone; + + public ConfigureNodesByZone(PD pipelineData, + Event completeEvent, + FailureDetector failureDetector, + int required, + RoutingStrategy routingStrategy, + ByteArray key, + Zone clientZone) { + super(pipelineData, completeEvent, failureDetector, required, routingStrategy); + this.key = key; + this.clientZone = clientZone; + } + + public List getNodes(ByteArray key, Operation op) { + List nodes = null; + try { + nodes = super.getNodes(key); + } catch(VoldemortException e) { + pipelineData.setFatalError(e); + return null; + } + + if(logger.isDebugEnabled()) + logger.debug("Adding " + nodes.size() + " node(s) to preference list"); + + if(pipelineData.getZonesRequired() > this.clientZone.getProximityList().size()) { + throw new VoldemortException("Number of zones required should be less than the total number of zones"); + } + + if(pipelineData.getZonesRequired() > required) { + throw new VoldemortException("Number of zones required should be less than the required number of " + + op.getSimpleName() + "s"); + } + + // Create zone id to node mapping + Map> zoneIdToNode = new HashMap>(); + for(Node node: nodes) { + List nodesList = null; + if(zoneIdToNode.containsKey(node.getZoneId())) { + nodesList = zoneIdToNode.get(node.getZoneId()); + } else { + nodesList = new ArrayList(); + zoneIdToNode.put(node.getZoneId(), nodesList); + } + nodesList.add(node); + } + + nodes = new ArrayList(); + LinkedList zoneProximityList = this.clientZone.getProximityList(); + if(op != Operation.PUT) { + // GET, GET_VERSIONS, DELETE + + // Add a node from every zone, upto a max of + // zoneCountReads/zoneCountWrites. + for(int index = 0; index < pipelineData.getZonesRequired(); index++) { + List zoneNodes = zoneIdToNode.get(zoneProximityList.get(index)); + if(zoneNodes != null && zoneNodes.size() > 0) { + nodes.add(zoneNodes.remove(0)); + } + } + + } + + // Add the rest, starting with client zone... + List clientZoneNodes = zoneIdToNode.get(clientZone.getId()); + if(clientZoneNodes != null && clientZoneNodes.size() > 0) + nodes.addAll(clientZoneNodes); + // ...followed by other zones sorted by proximity list + for(int index = 0; index < zoneProximityList.size(); index++) { + List zoneNodes = zoneIdToNode.get(zoneProximityList.get(index)); + if(zoneNodes != null && zoneNodes.size() > 0) { + nodes.addAll(zoneNodes); + } + } + + return nodes; + } + + public void execute(Pipeline pipeline) { + List nodes = null; + + nodes = getNodes(key, pipeline.getOperation()); + if(nodes == null) { + pipeline.abort(); + return; + } + + if(logger.isDebugEnabled()) { + StringBuilder nodeStr = new StringBuilder(); + for(Node node: nodes) { + nodeStr.append(node.getId() + ","); + } + logger.debug("Key " + ByteUtils.toHexString(key.get()) + + " final preference list to contact " + nodeStr); + } + pipelineData.setNodes(nodes); + pipeline.addEvent(completeEvent); + } + +} diff --git a/src/java/voldemort/store/routed/action/ConfigureNodesDefault.java b/src/java/voldemort/store/routed/action/ConfigureNodesDefault.java new file mode 100644 index 0000000000..b05379d74f --- /dev/null +++ b/src/java/voldemort/store/routed/action/ConfigureNodesDefault.java @@ -0,0 +1,71 @@ +package voldemort.store.routed.action; + +import java.util.List; + +import voldemort.VoldemortException; +import voldemort.cluster.Node; +import voldemort.cluster.failuredetector.FailureDetector; +import voldemort.routing.RoutingStrategy; +import voldemort.store.routed.BasicPipelineData; +import voldemort.store.routed.Pipeline; +import voldemort.store.routed.Pipeline.Event; +import voldemort.utils.ByteArray; +import voldemort.utils.ByteUtils; + +/* + * Default Configure Nodes that does not reorder the list of nodes obtained via + * the routing strategy + */ +public class ConfigureNodesDefault> extends + AbstractConfigureNodes { + + private final ByteArray key; + + public ConfigureNodesDefault(PD pipelineData, + Event completeEvent, + FailureDetector failureDetector, + int required, + RoutingStrategy routingStrategy, + ByteArray key) { + super(pipelineData, completeEvent, failureDetector, required, routingStrategy); + this.key = key; + } + + @Override + public List getNodes(ByteArray key) { + List nodes = null; + + try { + nodes = super.getNodes(key); + } catch(VoldemortException e) { + pipelineData.setFatalError(e); + return null; + } + return nodes; + } + + public void execute(Pipeline pipeline) { + List nodes = null; + + nodes = getNodes(key); + if(nodes == null) { + pipeline.abort(); + return; + } + + if(logger.isDebugEnabled()) + logger.debug("Adding " + nodes.size() + " node(s) to preference list"); + + if(logger.isDebugEnabled()) { + StringBuilder nodeStr = new StringBuilder(); + for(Node node: nodes) { + nodeStr.append(node.getId() + ","); + } + logger.debug("Key " + ByteUtils.toHexString(key.get()) + + " final preference list to contact " + nodeStr); + } + pipelineData.setNodes(nodes); + pipeline.addEvent(completeEvent); + } + +} diff --git a/src/java/voldemort/store/routed/action/ConfigureNodesLocalHost.java b/src/java/voldemort/store/routed/action/ConfigureNodesLocalHost.java new file mode 100644 index 0000000000..4eb2237380 --- /dev/null +++ b/src/java/voldemort/store/routed/action/ConfigureNodesLocalHost.java @@ -0,0 +1,71 @@ +package voldemort.store.routed.action; + +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.log4j.Logger; + +import voldemort.VoldemortException; +import voldemort.cluster.Node; +import voldemort.cluster.failuredetector.FailureDetector; +import voldemort.routing.RoutingStrategy; +import voldemort.store.routed.BasicPipelineData; +import voldemort.store.routed.Pipeline.Event; +import voldemort.utils.ByteArray; + +/* + * Use the default node list returned via the routing strategy. However give + * preference to the current node, if it is part of the preflist returned from + * the routing strategy. + */ + +public class ConfigureNodesLocalHost> extends + ConfigureNodesDefault { + + @SuppressWarnings("hiding") + private final Logger logger = Logger.getLogger(this.getClass()); + + public ConfigureNodesLocalHost(PD pipelineData, + Event completeEvent, + FailureDetector failureDetector, + int required, + RoutingStrategy routingStrategy, + ByteArray key) { + super(pipelineData, completeEvent, failureDetector, required, routingStrategy, key); + } + + /* + * If the current node exists in the nodes list, bring it to the front + */ + @Override + public List getNodes(ByteArray key) { + logger.debug("Giving pref to localhost ! "); + List nodes = null; + List reorderedNodes = new ArrayList(); + + try { + nodes = super.getNodes(key); + String currentHost = InetAddress.getLocalHost().getHostName(); + for(Node n: nodes) { + if(currentHost.contains(n.getHost()) || n.getHost().contains(currentHost)) { + logger.debug("Found localhost ! "); + reorderedNodes.add(n); + nodes.remove(n); + break; + } + } + reorderedNodes.addAll(nodes); + nodes = reorderedNodes; + } catch(VoldemortException e) { + pipelineData.setFatalError(e); + return null; + } catch(UnknownHostException e) { + e.printStackTrace(); + return null; + } + return nodes; + } + +} diff --git a/src/java/voldemort/store/routed/action/ConfigureNodesLocalHostByZone.java b/src/java/voldemort/store/routed/action/ConfigureNodesLocalHostByZone.java new file mode 100644 index 0000000000..ceb5e688eb --- /dev/null +++ b/src/java/voldemort/store/routed/action/ConfigureNodesLocalHostByZone.java @@ -0,0 +1,72 @@ +package voldemort.store.routed.action; + +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.List; + +import voldemort.VoldemortException; +import voldemort.cluster.Node; +import voldemort.cluster.Zone; +import voldemort.cluster.failuredetector.FailureDetector; +import voldemort.routing.RoutingStrategy; +import voldemort.store.routed.BasicPipelineData; +import voldemort.store.routed.Pipeline.Event; +import voldemort.store.routed.Pipeline.Operation; +import voldemort.utils.ByteArray; + +/* + * Use the zone aware node list returned via the routing strategy. However give + * preference to the current node, if it is part of the preflist returned from + * the routing strategy. + */ + +public class ConfigureNodesLocalHostByZone> extends + ConfigureNodesByZone { + + public ConfigureNodesLocalHostByZone(PD pipelineData, + Event completeEvent, + FailureDetector failureDetector, + int required, + RoutingStrategy routingStrategy, + ByteArray key, + Zone clientZone) { + super(pipelineData, + completeEvent, + failureDetector, + required, + routingStrategy, + key, + clientZone); + } + + /* + * If the current node exists in the nodes list, bring it to the front + */ + @Override + public List getNodes(ByteArray key, Operation op) { + List nodes = null; + List reorderedNodes = new ArrayList(); + + try { + nodes = super.getNodes(key, op); + String currentHost = InetAddress.getLocalHost().getHostName(); + for(Node n: nodes) { + if(currentHost.contains(n.getHost()) || n.getHost().contains(currentHost)) { + reorderedNodes.add(n); + nodes.remove(n); + break; + } + } + reorderedNodes.addAll(nodes); + nodes = reorderedNodes; + } catch(VoldemortException e) { + pipelineData.setFatalError(e); + return null; + } catch(UnknownHostException e) { + e.printStackTrace(); + return null; + } + return nodes; + } +} From a21bf212708ae7f6fc7cfce25dca91cf968ec966 Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Thu, 15 Mar 2012 12:40:44 -0700 Subject: [PATCH 074/209] add clientId for voldemort client --- .../voldemort/client/AbstractStoreClientFactory.java | 1 + src/java/voldemort/client/ClientConfig.java | 4 ++-- src/java/voldemort/client/DefaultStoreClient.java | 9 +-------- src/java/voldemort/client/StoreClientFactory.java | 3 +++ 4 files changed, 7 insertions(+), 10 deletions(-) diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index 89ba43b3c2..7fbbe80b33 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -124,6 +124,7 @@ public AbstractStoreClientFactory(ClientConfig config) { this.routedStoreFactory = new RoutedStoreFactory(config.isPipelineRoutedStoreEnabled(), threadPool, config.getTimeoutConfig()); + this.sequencer = new AtomicInteger(0); if(this.isJmxEnabled) { diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index 4751f74be8..20ca113355 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -717,8 +717,8 @@ public long getAsyncCheckMetadataInterval() { return asyncCheckMetadataInterval; } - public void setAsyncCheckMetadataInterval(long asyncCheckMetadataInterval) { + public ClientConfig setAsyncCheckMetadataInterval(long asyncCheckMetadataInterval) { this.asyncCheckMetadataInterval = asyncCheckMetadataInterval; + return this; } - } diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index 436fcfd292..bc313fbbc4 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -19,8 +19,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.UUID; +import java.util.Map.Entry; import java.util.concurrent.Callable; import org.apache.log4j.Logger; @@ -71,13 +71,6 @@ public class DefaultStoreClient implements StoreClient { private final Map sysStoreMap; private AsyncMetadataVersionManager asyncCheckMetadata; - // Enumerate all the system stores - private final String METADATA_VERSION_STORE = "voldsys$_metadata_version"; - private final String CLIENT_REGISTRY_STORE = "voldsys$_client_registry"; - private final String STORE_DEFINITION_STORE = "voldsys$_client_store_definition"; - private final String[] systemStoreNames = { METADATA_VERSION_STORE, CLIENT_REGISTRY_STORE, - STORE_DEFINITION_STORE }; - public DefaultStoreClient(String storeName, InconsistencyResolver> resolver, StoreClientFactory storeFactory, diff --git a/src/java/voldemort/client/StoreClientFactory.java b/src/java/voldemort/client/StoreClientFactory.java index ee008d98af..59070fbd96 100644 --- a/src/java/voldemort/client/StoreClientFactory.java +++ b/src/java/voldemort/client/StoreClientFactory.java @@ -87,6 +87,7 @@ Store getRawStore(String storeName, UUID clientId); /** +<<<<<<< HEAD * Get the underlying store, not the public StoreClient interface * * @param storeName The name of the store @@ -101,6 +102,8 @@ Store getRawStore(String storeName, String customStoresXml); /** +======= +>>>>>>> add clientId for voldemort client * Close the store client */ public void close(); From 234ac9b96b48d42b6c7bb76a17281ba3ef788921 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Wed, 30 May 2012 18:56:26 -0700 Subject: [PATCH 075/209] Adding System store functionality --- src/java/voldemort/client/DefaultStoreClient.java | 10 +++++++++- src/java/voldemort/client/StoreClientFactory.java | 6 ++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index bc313fbbc4..6fda072f01 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -32,6 +32,7 @@ import voldemort.cluster.Node; import voldemort.routing.RoutingStrategy; import voldemort.serialization.Serializer; +import voldemort.server.SystemStoreConstants; import voldemort.store.InvalidMetadataException; import voldemort.store.Store; import voldemort.store.StoreCapabilityType; @@ -136,7 +137,7 @@ public Void call() throws Exception { } public void initializeSystemStores() { - for(String storeName: systemStoreNames) { + for(String storeName: SystemStoreConstants.SystemStoreName.) { SystemStore sysStore = new SystemStore(storeName, config.getBootstrapUrls(), config.getClientZoneId()); @@ -148,6 +149,13 @@ public void initializeSystemStores() { public void bootStrap() { logger.info("Bootstrapping metadata for store " + this.storeName); this.store = storeFactory.getRawStore(storeName, resolver, clientId); + + logger.info("Creating System store"); + String systemKey = storeName + "-client"; + this.sysStore = new SystemStore("voldsys$_client_registry", + this.storeFactory); + sysStore.putSysStore(systemKey, "Registered"); + logger.info("Getting value - " + sysStore.getSysStore(systemKey)); } public boolean delete(K key) { diff --git a/src/java/voldemort/client/StoreClientFactory.java b/src/java/voldemort/client/StoreClientFactory.java index 59070fbd96..8663384e93 100644 --- a/src/java/voldemort/client/StoreClientFactory.java +++ b/src/java/voldemort/client/StoreClientFactory.java @@ -88,6 +88,9 @@ Store getRawStore(String storeName, /** <<<<<<< HEAD +<<<<<<< HEAD +======= +>>>>>>> Adding System store functionality * Get the underlying store, not the public StoreClient interface * * @param storeName The name of the store @@ -102,8 +105,11 @@ Store getRawStore(String storeName, String customStoresXml); /** +<<<<<<< HEAD ======= >>>>>>> add clientId for voldemort client +======= +>>>>>>> Adding System store functionality * Close the store client */ public void close(); From b2cbdf8d71281953f724ec73a9d264313ad60cc6 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Tue, 5 Jun 2012 15:59:35 -0700 Subject: [PATCH 076/209] Added the Voldemort Client automated re-bootstrap mechanism --- src/java/voldemort/client/DefaultStoreClient.java | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index 6fda072f01..5f5ab0dca4 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -138,7 +138,7 @@ public Void call() throws Exception { public void initializeSystemStores() { for(String storeName: SystemStoreConstants.SystemStoreName.) { - SystemStore sysStore = new SystemStore(storeName, + SystemStore sysStore = new SystemStore(storeName, config.getBootstrapUrls(), config.getClientZoneId()); this.sysStoreMap.put(storeName, sysStore); @@ -149,13 +149,6 @@ public void initializeSystemStores() { public void bootStrap() { logger.info("Bootstrapping metadata for store " + this.storeName); this.store = storeFactory.getRawStore(storeName, resolver, clientId); - - logger.info("Creating System store"); - String systemKey = storeName + "-client"; - this.sysStore = new SystemStore("voldsys$_client_registry", - this.storeFactory); - sysStore.putSysStore(systemKey, "Registered"); - logger.info("Getting value - " + sysStore.getSysStore(systemKey)); } public boolean delete(K key) { From ea649534c2ae824fd87a7a8efd5511ca14a785e4 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Wed, 30 May 2012 18:56:26 -0700 Subject: [PATCH 077/209] Adding System store functionality --- src/java/voldemort/client/DefaultStoreClient.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index 5f5ab0dca4..be1b94f161 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -69,6 +69,7 @@ public class DefaultStoreClient implements StoreClient { private final InconsistencyResolver> resolver; private volatile Store store; private final UUID clientId; + private final Map sysStoreMap; private AsyncMetadataVersionManager asyncCheckMetadata; @@ -149,6 +150,13 @@ public void initializeSystemStores() { public void bootStrap() { logger.info("Bootstrapping metadata for store " + this.storeName); this.store = storeFactory.getRawStore(storeName, resolver, clientId); + + logger.info("Creating System store"); + String systemKey = storeName + "-client"; + this.sysStore = new SystemStore("voldsys$_client_registry", + this.storeFactory); + sysStore.putSysStore(systemKey, "Registered"); + logger.info("Getting value - " + sysStore.getSysStore(systemKey)); } public boolean delete(K key) { From 7b3c8e2f0444f144fcfe2f7d0005e39a65529ef8 Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Tue, 12 Jun 2012 13:49:20 -0700 Subject: [PATCH 078/209] client registry impl --- src/java/voldemort/VoldemortAdminTool.java | 25 ++++- .../client/AbstractStoreClientFactory.java | 34 ++---- .../client/AsyncMetadataVersionManager.java | 2 +- src/java/voldemort/client/ClientInfo.java | 106 ++++++++++++++++++ .../voldemort/client/DefaultStoreClient.java | 87 ++++++++------ .../client/SocketStoreClientFactory.java | 2 +- src/java/voldemort/client/SystemStore.java | 86 ++++++++------ .../client/protocol/admin/AdminClient.java | 11 +- .../admin/FetchStreamRequestHandler.java | 13 ++- .../server/storage/StorageService.java | 6 +- src/java/voldemort/store/StoreDefinition.java | 2 +- .../store/metadata/MetadataStore.java | 2 +- .../store/system/SystemStoreConstants.java | 104 +++++++++++++++++ .../voldemort/xml/StoreDefinitionsMapper.java | 2 +- .../client/DefaultSocketStoreClientTest.java | 78 +++++++++++-- 15 files changed, 440 insertions(+), 120 deletions(-) create mode 100644 src/java/voldemort/client/ClientInfo.java create mode 100644 src/java/voldemort/store/system/SystemStoreConstants.java diff --git a/src/java/voldemort/VoldemortAdminTool.java b/src/java/voldemort/VoldemortAdminTool.java index 5629691e97..c3ce8353df 100644 --- a/src/java/voldemort/VoldemortAdminTool.java +++ b/src/java/voldemort/VoldemortAdminTool.java @@ -32,6 +32,7 @@ import java.io.PrintStream; import java.io.StringReader; import java.io.StringWriter; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -61,6 +62,7 @@ import voldemort.store.metadata.MetadataStore; import voldemort.store.metadata.MetadataStore.VoldemortState; import voldemort.store.readonly.ReadOnlyStorageConfiguration; +import voldemort.store.system.SystemStoreConstants; import voldemort.utils.ByteArray; import voldemort.utils.ByteUtils; import voldemort.utils.CmdUtils; @@ -792,9 +794,8 @@ private static void executeSetMetadata(Integer nodeId, + adminClient.getAdminClientCluster() .getNodeById(currentNodeId) .getId()); - adminClient.updateRemoteMetadata(currentNodeId, - key, - Versioned.value(value.toString(), updatedVersion)); + adminClient.updateRemoteMetadata(currentNodeId, key, Versioned.value(value.toString(), + updatedVersion)); } } @@ -972,7 +973,7 @@ private static void executeFetchEntries(Integer nodeId, List storeDefinitionList = adminClient.getRemoteStoreDefList(nodeId) .getValue(); - Map storeDefinitionMap = Maps.newHashMap(); + HashMap storeDefinitionMap = Maps.newHashMap(); for(StoreDefinition storeDefinition: storeDefinitionList) { storeDefinitionMap.put(storeDefinition.getName(), storeDefinition); } @@ -986,8 +987,14 @@ private static void executeFetchEntries(Integer nodeId, } List stores = storeNames; if(stores == null) { + // when no stores specified, all user defined store will be fetched, + // but not system stores. stores = Lists.newArrayList(); stores.addAll(storeDefinitionMap.keySet()); + } else { + // add system store to the map so they can be fetched when specified + // explicitly + storeDefinitionMap.putAll(getSystemStoreDef()); } // Pick up all the partitions @@ -1003,6 +1010,7 @@ private static void executeFetchEntries(Integer nodeId, storeDefinition = storeDefinitionMap.get(store); if(null == storeDefinition) { + System.out.println("No store found under the name \'" + store + "\'"); continue; } else { @@ -1031,6 +1039,15 @@ private static void executeFetchEntries(Integer nodeId, } } + private static Map getSystemStoreDef() { + Map sysStoreDef = Maps.newHashMap(); + List storesDefs = SystemStoreConstants.getAllSystemStoreDefs(); + for(StoreDefinition def: storesDefs) { + sysStoreDef.put(def.getName(), def); + } + return sysStoreDef; + } + private static void executeUpdateEntries(Integer nodeId, AdminClient adminClient, List storeNames, diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index 7fbbe80b33..4d9512813f 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -16,13 +16,9 @@ package voldemort.client; -import java.io.File; -import java.io.IOException; import java.io.StringReader; -import java.net.InetAddress; import java.net.URI; import java.net.URISyntaxException; -import java.net.UnknownHostException; import java.util.Collection; import java.util.List; import java.util.Map; @@ -194,8 +190,10 @@ public Store getRawStore(String storeName, for(StoreDefinition d: storeDefs) if(d.getName().equals(storeName)) storeDef = d; - if(storeDef == null) + if(storeDef == null) { + logger.error("Bootstrap - unknown store: " + storeName); throw new BootstrapFailureException("Unknown store '" + storeName + "'."); + } if(logger.isDebugEnabled()) { logger.debug(cluster.toString(true)); @@ -460,27 +458,16 @@ public String jmxId() { * @param clientSequence the client sequence number * @return unique client ID */ - public static UUID generateClientId(String storeName, String contextName, int clientSequence) { + public static UUID generateClientId(ClientInfo clientInfo) { + String contextName = clientInfo.getContext(); + int clientSequence = clientInfo.getClientSequence(); + String newLine = System.getProperty("line.separator"); StringBuilder context = new StringBuilder(contextName == null ? "" : contextName); context.append(0 == clientSequence ? "" : ("." + clientSequence)); - context.append(".").append(storeName); - - try { - InetAddress host = InetAddress.getLocalHost(); - context.append("@").append(host.getHostName()).append(":"); - } catch(UnknownHostException e) { - logger.info("Unable to obtain client hostname."); - logger.info(e.getMessage()); - } - - try { - String currentPath = new File(".").getCanonicalPath(); - context.append(currentPath).append(newLine); - } catch(IOException e) { - logger.info("Unable to obtain client run path."); - logger.info(e.getMessage()); - } + context.append(".").append(clientInfo.getStoreName()); + context.append("@").append(clientInfo.getLocalHostName()).append(":"); + context.append(clientInfo.getDeploymentPath()).append(newLine); if(logger.isDebugEnabled()) { logger.debug(context.toString()); @@ -488,4 +475,5 @@ public static UUID generateClientId(String storeName, String contextName, int cl return UUID.nameUUIDFromBytes(context.toString().getBytes()); } + } diff --git a/src/java/voldemort/client/AsyncMetadataVersionManager.java b/src/java/voldemort/client/AsyncMetadataVersionManager.java index a0ce826e8b..fa0df578c4 100644 --- a/src/java/voldemort/client/AsyncMetadataVersionManager.java +++ b/src/java/voldemort/client/AsyncMetadataVersionManager.java @@ -90,7 +90,7 @@ public void run() { // If version obtained is null, the store is untouched. Continue if(newVersion == null) { - logger.info("Metadata unchanged after creation ..."); + logger.debug("Metadata unchanged after creation ..."); continue; } diff --git a/src/java/voldemort/client/ClientInfo.java b/src/java/voldemort/client/ClientInfo.java new file mode 100644 index 0000000000..08886ad7d0 --- /dev/null +++ b/src/java/voldemort/client/ClientInfo.java @@ -0,0 +1,106 @@ +package voldemort.client; + +import java.io.File; +import java.io.IOException; +import java.io.Serializable; +import java.net.InetAddress; +import java.net.UnknownHostException; + +import org.apache.log4j.Logger; + +public class ClientInfo implements Serializable { + + /** + * + */ + private static final long serialVersionUID = 1L; + + protected static final Logger logger = Logger.getLogger(ClientInfo.class); + + private long bootstrapTime; + private String storeName; + private String context; + private int sequence; + private String localHostName; + private String deploymentPath; + + public ClientInfo(String storeName, String clientContext, int clientSequence, long bootstrapTime) { + this.bootstrapTime = bootstrapTime; + this.storeName = storeName; + this.context = clientContext; + this.sequence = clientSequence; + this.localHostName = createHostName(); + this.deploymentPath = createDeploymentPath(); + } + + private String createDeploymentPath() { + String currentPath = null; + try { + currentPath = new File(".").getCanonicalPath(); + } catch(IOException e) { + logger.warn("Unable to obtain client deployment path due to the following error:"); + logger.warn(e.getMessage()); + } + return currentPath; + } + + private String createHostName() { + String hostName = null; + try { + InetAddress host = InetAddress.getLocalHost(); + hostName = host.getHostName(); + } catch(UnknownHostException e) { + logger.warn("Unable to obtain client hostname due to the following error:"); + logger.warn(e.getMessage()); + } + return hostName; + } + + public void setStoreName(String storeName) { + this.storeName = storeName; + } + + public String getStoreName() { + return storeName; + } + + public void setBootstrapTime(long bootstrapTime) { + this.bootstrapTime = bootstrapTime; + } + + public long getBootstrapTime() { + return bootstrapTime; + } + + public void setContext(String clientContext) { + this.context = clientContext; + } + + public String getContext() { + return context; + } + + public void setClientSequence(int clientSequence) { + this.sequence = clientSequence; + } + + public int getClientSequence() { + return sequence; + } + + public void setDeploymentPath(String deploymentPath) { + this.deploymentPath = deploymentPath; + } + + public String getDeploymentPath() { + return deploymentPath; + } + + public void setLocalHostName(String localHostName) { + this.localHostName = localHostName; + } + + public String getLocalHostName() { + return localHostName; + } +} diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index be1b94f161..4aac50d8ee 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -73,6 +73,8 @@ public class DefaultStoreClient implements StoreClient { private final Map sysStoreMap; private AsyncMetadataVersionManager asyncCheckMetadata; + private ClientInfo clientInfo; + public DefaultStoreClient(String storeName, InconsistencyResolver> resolver, StoreClientFactory storeFactory, @@ -93,9 +95,11 @@ public DefaultStoreClient(String storeName, this.resolver = resolver; this.storeFactory = Utils.notNull(storeFactory); this.metadataRefreshAttempts = maxMetadataRefreshAttempts; - this.clientId = AbstractStoreClientFactory.generateClientId(storeName, - clientContext, - clientSequence); + this.clientInfo = new ClientInfo(storeName, + clientContext, + clientSequence, + System.currentTimeMillis()); + this.clientId = AbstractStoreClientFactory.generateClientId(clientInfo); this.config = config; // Registering self to be able to bootstrap client dynamically via JMX @@ -108,55 +112,70 @@ public DefaultStoreClient(String storeName, bootStrap(); // Initialize all the system stores - sysStoreMap = new HashMap(); - initializeSystemStores(); + sysStoreMap = createSystemStores(); // Initialize the background thread for checking metadata version if(config != null) { - SystemStore versionStore = this.sysStoreMap.get(METADATA_VERSION_STORE); - if(versionStore == null) - logger.info("Metadata version system store not found. Cannot run Metadata version check thread."); - else { - Callable bootstrapCallback = new Callable() { - - public Void call() throws Exception { - bootStrap(); - return null; - } - }; - - asyncCheckMetadata = new AsyncMetadataVersionManager(versionStore, - config.getAsyncCheckMetadataInterval(), - bootstrapCallback); - logger.info("Metadata version check thread started. Frequency = Every " - + config.getAsyncCheckMetadataInterval() + " ms"); - } + asyncCheckMetadata = createMetadataChecker(); } + registerClient(); logger.info("Voldemort client created: clientContext=" + clientContext + " clientSequence=" + clientSequence + " clientId=" + clientId.toString()); } - public void initializeSystemStores() { - for(String storeName: SystemStoreConstants.SystemStoreName.) { - SystemStore sysStore = new SystemStore(storeName, + private void registerClient() { + String name = SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(); + SystemStore clientRegistry = sysStoreMap.get(name); + if(null != clientRegistry) { + try { + clientRegistry.putSysStore(clientId.toString(), clientInfo); + } catch(Exception e) { + logger.warn("Unable to register with the cluster due to the following error:", e); + } + } else { + logger.warn(name + "not found. Unable to registry with voldemort cluster."); + } + } + + private Map createSystemStores() { + Map systemStores = new HashMap(); + for(SystemStoreConstants.SystemStoreName storeName: SystemStoreConstants.SystemStoreName.values()) { + SystemStore sysStore = new SystemStore(storeName.name(), config.getBootstrapUrls(), config.getClientZoneId()); - this.sysStoreMap.put(storeName, sysStore); + systemStores.put(storeName.name(), sysStore); } + return systemStores; + } + + private AsyncMetadataVersionManager createMetadataChecker() { + AsyncMetadataVersionManager asyncCheckMetadata = null; + SystemStore versionStore = this.sysStoreMap.get(SystemStoreConstants.SystemStoreName.voldsys$_metadata_version.name()); + if(versionStore == null) + logger.warn("Metadata version system store not found. Cannot run Metadata version check thread."); + else { + Callable bootstrapCallback = new Callable() { + + public Void call() throws Exception { + bootStrap(); + return null; + } + }; + + asyncCheckMetadata = new AsyncMetadataVersionManager(versionStore, + config.getAsyncCheckMetadataInterval(), + bootstrapCallback); + logger.info("Metadata version check thread started. Frequency = Every " + + config.getAsyncCheckMetadataInterval() + " ms"); + } + return asyncCheckMetadata; } @JmxOperation(description = "bootstrap metadata from the cluster.") public void bootStrap() { logger.info("Bootstrapping metadata for store " + this.storeName); this.store = storeFactory.getRawStore(storeName, resolver, clientId); - - logger.info("Creating System store"); - String systemKey = storeName + "-client"; - this.sysStore = new SystemStore("voldsys$_client_registry", - this.storeFactory); - sysStore.putSysStore(systemKey, "Registered"); - logger.info("Getting value - " + sysStore.getSysStore(systemKey)); } public boolean delete(K key) { diff --git a/src/java/voldemort/client/SocketStoreClientFactory.java b/src/java/voldemort/client/SocketStoreClientFactory.java index 8843744da4..fe10a1e64a 100644 --- a/src/java/voldemort/client/SocketStoreClientFactory.java +++ b/src/java/voldemort/client/SocketStoreClientFactory.java @@ -32,12 +32,12 @@ import voldemort.cluster.failuredetector.FailureDetectorConfig; import voldemort.cluster.failuredetector.FailureDetectorListener; import voldemort.server.RequestRoutingType; -import voldemort.server.SystemStoreConstants; import voldemort.store.Store; import voldemort.store.metadata.MetadataStore; import voldemort.store.socket.SocketDestination; import voldemort.store.socket.SocketStoreFactory; import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; +import voldemort.store.system.SystemStoreConstants; import voldemort.utils.ByteArray; import voldemort.utils.JmxUtils; import voldemort.versioning.InconsistencyResolver; diff --git a/src/java/voldemort/client/SystemStore.java b/src/java/voldemort/client/SystemStore.java index d4c901ebbe..8ed7184a1d 100644 --- a/src/java/voldemort/client/SystemStore.java +++ b/src/java/voldemort/client/SystemStore.java @@ -6,8 +6,8 @@ import org.apache.log4j.Logger; import voldemort.VoldemortException; -import voldemort.server.SystemStoreConstants; import voldemort.store.Store; +import voldemort.store.system.SystemStoreConstants; import voldemort.versioning.InconsistentDataException; import voldemort.versioning.VectorClock; import voldemort.versioning.Versioned; @@ -39,47 +39,67 @@ public SystemStore(String storeName, String[] bootstrapUrls, int clientZoneID) { this.sysStore = this.systemStoreFactory.getSystemStore(this.storeName); } - public void putSysStore(K key, V value) throws VoldemortException { - logger.debug("Invoking Put for key : " + key + " on store name : " + this.storeName); - Versioned versioned = getSysStore(key); - if(versioned == null) - versioned = Versioned.value(value, new VectorClock()); - else - versioned.setObject(value); - this.sysStore.put(key, versioned, null); + public void putSysStore(K key, V value) { + try { + logger.debug("Invoking Put for key : " + key + " on store name : " + this.storeName); + Versioned versioned = getSysStore(key); + if(versioned == null) + versioned = Versioned.value(value, new VectorClock()); + else + versioned.setObject(value); + this.sysStore.put(key, versioned, null); + } catch(Exception e) { + logger.info("Exception caught during putSysStore:"); + e.printStackTrace(); + } } - public void putSysStore(K key, Versioned value) throws VoldemortException { - logger.debug("Invoking Put for key : " + key + " on store name : " + this.storeName); - this.sysStore.put(key, value, null); + public void putSysStore(K key, Versioned value) { + try { + logger.debug("Invoking Put for key : " + key + " on store name : " + this.storeName); + this.sysStore.put(key, value, null); + } catch(Exception e) { + logger.info("Exception caught during putSysStore:"); + e.printStackTrace(); + } } - public Versioned getSysStore(K key) throws VoldemortException { + public Versioned getSysStore(K key) { logger.debug("Invoking Get for key : " + key + " on store name : " + this.storeName); Versioned versioned = null; - List> items = this.sysStore.get(key, null); - if(items.size() == 1) - versioned = items.get(0); - else if(items.size() > 1) - throw new InconsistentDataException("Unresolved versions returned from get(" + key - + ") = " + items, items); - if(versioned != null) - logger.debug("Value for key : " + key + " = " + versioned.getValue() - + " on store name : " + this.storeName); - else - logger.debug("Got null value"); + try { + List> items = this.sysStore.get(key, null); + if(items.size() == 1) + versioned = items.get(0); + else if(items.size() > 1) + throw new InconsistentDataException("Unresolved versions returned from get(" + key + + ") = " + items, items); + if(versioned != null) + logger.debug("Value for key : " + key + " = " + versioned.getValue() + + " on store name : " + this.storeName); + else + logger.debug("Got null value"); + } catch(Exception e) { + logger.info("Exception caught during getSysStore:"); + e.printStackTrace(); + } return versioned; } - public V getValueSysStore(K key) throws VoldemortException { - logger.debug("Invoking Get for key : " + key + " on store name : " + this.storeName); - Versioned versioned = getSysStore(key); - if(versioned != null) { - logger.debug("Value for key : " + key + " = " + versioned.getValue() - + " on store name : " + this.storeName); - return versioned.getValue(); + public V getValueSysStore(K key) { + V value = null; + try { + logger.debug("Invoking Get for key : " + key + " on store name : " + this.storeName); + Versioned versioned = getSysStore(key); + if(versioned != null) { + logger.debug("Value for key : " + key + " = " + versioned.getValue() + + " on store name : " + this.storeName); + value = versioned.getValue(); + } + } catch(Exception e) { + logger.info("Exception caught during getSysStore:"); + e.printStackTrace(); } - return null; + return value; } - } diff --git a/src/java/voldemort/client/protocol/admin/AdminClient.java b/src/java/voldemort/client/protocol/admin/AdminClient.java index 94203296a5..e4059ce061 100644 --- a/src/java/voldemort/client/protocol/admin/AdminClient.java +++ b/src/java/voldemort/client/protocol/admin/AdminClient.java @@ -31,8 +31,8 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Set; +import java.util.Map.Entry; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; @@ -49,8 +49,8 @@ import voldemort.client.protocol.VoldemortFilter; import voldemort.client.protocol.pb.ProtoUtils; import voldemort.client.protocol.pb.VAdminProto; -import voldemort.client.protocol.pb.VAdminProto.RebalancePartitionInfoMap; import voldemort.client.protocol.pb.VProto; +import voldemort.client.protocol.pb.VAdminProto.RebalancePartitionInfoMap; import voldemort.client.protocol.pb.VProto.RequestType; import voldemort.client.rebalance.RebalancePartitionsInfo; import voldemort.cluster.Cluster; @@ -72,6 +72,7 @@ import voldemort.store.slop.Slop; import voldemort.store.slop.Slop.Operation; import voldemort.store.socket.SocketDestination; +import voldemort.store.system.SystemStoreConstants; import voldemort.store.views.ViewStorageConfiguration; import voldemort.utils.ByteArray; import voldemort.utils.ByteUtils; @@ -962,9 +963,9 @@ public int rebalanceNode(RebalancePartitionsInfo stealInfo) { private HashMap> getReplicaToPartitionMap(int nodeId, String storeName, List partitions) { - - StoreDefinition def = RebalanceUtils.getStoreDefinitionWithName(getRemoteStoreDefList(nodeId).getValue(), - storeName); + List allStoreDefs = getRemoteStoreDefList(nodeId).getValue(); + allStoreDefs.addAll(SystemStoreConstants.getAllSystemStoreDefs()); + StoreDefinition def = RebalanceUtils.getStoreDefinitionWithName(allStoreDefs, storeName); HashMap> replicaToPartitionList = Maps.newHashMap(); for(int replicaNum = 0; replicaNum < def.getReplicationFactor(); replicaNum++) { replicaToPartitionList.put(replicaNum, partitions); diff --git a/src/java/voldemort/server/protocol/admin/FetchStreamRequestHandler.java b/src/java/voldemort/server/protocol/admin/FetchStreamRequestHandler.java index b73eda56ca..177fb5db95 100644 --- a/src/java/voldemort/server/protocol/admin/FetchStreamRequestHandler.java +++ b/src/java/voldemort/server/protocol/admin/FetchStreamRequestHandler.java @@ -23,6 +23,7 @@ import voldemort.store.metadata.MetadataStore; import voldemort.store.stats.StreamStats; import voldemort.store.stats.StreamStats.Handle; +import voldemort.store.system.SystemStoreConstants; import voldemort.utils.ByteArray; import voldemort.utils.ClosableIterator; import voldemort.utils.EventThrottler; @@ -81,7 +82,7 @@ protected FetchStreamRequestHandler(VAdminProto.FetchPartitionEntriesRequest req this.handle = stats.makeHandle(operation, replicaToPartitionList); this.storageEngine = AdminServiceRequestHandler.getStorageEngine(storeRepository, request.getStore()); - this.storeDef = metadataStore.getStoreDef(request.getStore()); + this.storeDef = getStoreDef(request.getStore(), metadataStore); if(request.hasInitialCluster()) { this.initialCluster = new ClusterMapper().readCluster(new StringReader(request.getInitialCluster())); } else { @@ -105,6 +106,16 @@ protected FetchStreamRequestHandler(VAdminProto.FetchPartitionEntriesRequest req } } + private StoreDefinition getStoreDef(String store, MetadataStore metadataStore) { + StoreDefinition def = null; + if(SystemStoreConstants.isSystemStore(store)) { + def = SystemStoreConstants.getSystemStoreDef(store); + } else { + def = metadataStore.getStoreDef(request.getStore()); + } + return def; + } + public final StreamRequestDirection getDirection() { return StreamRequestDirection.WRITING; } diff --git a/src/java/voldemort/server/storage/StorageService.java b/src/java/voldemort/server/storage/StorageService.java index 3027b6e237..a313702a6f 100644 --- a/src/java/voldemort/server/storage/StorageService.java +++ b/src/java/voldemort/server/storage/StorageService.java @@ -18,7 +18,6 @@ import static voldemort.cluster.failuredetector.FailureDetectorUtils.create; -import java.io.StringReader; import java.lang.management.ManagementFactory; import java.util.ArrayList; import java.util.Calendar; @@ -55,7 +54,6 @@ import voldemort.server.RequestRoutingType; import voldemort.server.ServiceType; import voldemort.server.StoreRepository; -import voldemort.server.SystemStoreConstants; import voldemort.server.VoldemortConfig; import voldemort.server.scheduler.DataCleanupJob; import voldemort.server.scheduler.SchedulerService; @@ -84,6 +82,7 @@ import voldemort.store.stats.StatTrackingStore; import voldemort.store.stats.StoreStats; import voldemort.store.stats.StoreStatsJmx; +import voldemort.store.system.SystemStoreConstants; import voldemort.store.versioned.InconsistencyResolvingStore; import voldemort.store.views.ViewStorageConfiguration; import voldemort.store.views.ViewStorageEngine; @@ -100,7 +99,6 @@ import voldemort.versioning.VectorClock; import voldemort.versioning.VectorClockInconsistencyResolver; import voldemort.versioning.Versioned; -import voldemort.xml.StoreDefinitionsMapper; /** * The service responsible for managing all storage types @@ -209,7 +207,7 @@ private void initSystemStorageConfig() { } private void initSystemStores() { - List storesDefs = (new StoreDefinitionsMapper()).readStoreList(new StringReader(SystemStoreConstants.SYSTEM_STORE_SCHEMA)); + List storesDefs = SystemStoreConstants.getAllSystemStoreDefs(); // TODO: replication factor can't now be determined unless the // cluster.xml is made available to the server at runtime. So we need to diff --git a/src/java/voldemort/store/StoreDefinition.java b/src/java/voldemort/store/StoreDefinition.java index 52d4dda7f0..15944f327a 100644 --- a/src/java/voldemort/store/StoreDefinition.java +++ b/src/java/voldemort/store/StoreDefinition.java @@ -22,8 +22,8 @@ import voldemort.client.RoutingTier; import voldemort.serialization.SerializerDefinition; -import voldemort.server.SystemStoreConstants; import voldemort.store.slop.strategy.HintedHandoffStrategyType; +import voldemort.store.system.SystemStoreConstants; import voldemort.utils.Utils; import com.google.common.base.Objects; diff --git a/src/java/voldemort/store/metadata/MetadataStore.java b/src/java/voldemort/store/metadata/MetadataStore.java index 6d692ffe3f..e38aa7a785 100644 --- a/src/java/voldemort/store/metadata/MetadataStore.java +++ b/src/java/voldemort/store/metadata/MetadataStore.java @@ -41,7 +41,6 @@ import voldemort.routing.RouteToAllStrategy; import voldemort.routing.RoutingStrategy; import voldemort.routing.RoutingStrategyFactory; -import voldemort.server.SystemStoreConstants; import voldemort.server.rebalance.RebalancerState; import voldemort.store.StorageEngine; import voldemort.store.Store; @@ -49,6 +48,7 @@ import voldemort.store.StoreDefinition; import voldemort.store.StoreUtils; import voldemort.store.configuration.ConfigurationStorageEngine; +import voldemort.store.system.SystemStoreConstants; import voldemort.utils.ByteArray; import voldemort.utils.ByteUtils; import voldemort.utils.ClosableIterator; diff --git a/src/java/voldemort/store/system/SystemStoreConstants.java b/src/java/voldemort/store/system/SystemStoreConstants.java new file mode 100644 index 0000000000..32a1d2e5be --- /dev/null +++ b/src/java/voldemort/store/system/SystemStoreConstants.java @@ -0,0 +1,104 @@ +package voldemort.store.system; + +import java.io.StringReader; +import java.util.List; + +import voldemort.store.StoreDefinition; +import voldemort.xml.StoreDefinitionsMapper; + +/** + * The various system stores + */ +public class SystemStoreConstants { + + public static final String NAME_PREFIX = "voldsys$_"; + + public static enum SystemStoreName { + voldsys$_client_registry, + voldsys$_client_store_definition, + voldsys$_metadata_version; + } + + public static final String SYSTEM_STORE_SCHEMA = "" + + " " + + " voldsys$_client_registry" + + " zone-routing" + + " proximity-handoff" + + " memory" + + " client" + + " 4" + + " " + + " 2" + + " 2" + + " " + + " 1" + + " 1" + + " " + + " string" + + " utf8" + + " " + + " " + // + + // " avro-specific" + // + + // " java=voldemort.client.ClientInfo" + + " java-serialization" + + " " + + " 7" + + " " + + + " " + + " voldsys$_client_store_definition" + + " zone-routing" + + " proximity-handoff" + + " memory" + + " client" + + " 1" + + " 1" + + " 1" + + " " + + " string" + + " " + + " " + + " string" + + " " + + " 7" + + " " + + + " " + + " voldsys$_metadata_version" + + " local-pref-all-routing" + + " proximity-handoff" + + " memory" + + " client" + + " 1" + + " 1" + + " 1" + + " " + + " string" + + " " + + " " + + " java-serialization" + + " " + " " + + + ""; + + public static boolean isSystemStore(String storeName) { + return (null == storeName ? false : storeName.startsWith(NAME_PREFIX)); + } + + public static List getAllSystemStoreDefs() { + return (new StoreDefinitionsMapper()).readStoreList(new StringReader(SystemStoreConstants.SYSTEM_STORE_SCHEMA)); + } + + public static StoreDefinition getSystemStoreDef(String name) { + StoreDefinition storeDef = null; + List allDefs = getAllSystemStoreDefs(); + for(StoreDefinition def: allDefs) { + if(name.equals(def.getName())) { + storeDef = def; + } + } + return storeDef; + } +} \ No newline at end of file diff --git a/src/java/voldemort/xml/StoreDefinitionsMapper.java b/src/java/voldemort/xml/StoreDefinitionsMapper.java index d1d4948b24..4fe6d9983d 100644 --- a/src/java/voldemort/xml/StoreDefinitionsMapper.java +++ b/src/java/voldemort/xml/StoreDefinitionsMapper.java @@ -46,11 +46,11 @@ import voldemort.routing.RoutingStrategyType; import voldemort.serialization.Compression; import voldemort.serialization.SerializerDefinition; -import voldemort.server.SystemStoreConstants; import voldemort.store.StoreDefinition; import voldemort.store.StoreDefinitionBuilder; import voldemort.store.StoreUtils; import voldemort.store.slop.strategy.HintedHandoffStrategyType; +import voldemort.store.system.SystemStoreConstants; import voldemort.store.views.ViewStorageConfiguration; import voldemort.utils.Utils; diff --git a/test/unit/voldemort/client/DefaultSocketStoreClientTest.java b/test/unit/voldemort/client/DefaultSocketStoreClientTest.java index 07b8db82f5..46945a76c2 100644 --- a/test/unit/voldemort/client/DefaultSocketStoreClientTest.java +++ b/test/unit/voldemort/client/DefaultSocketStoreClientTest.java @@ -4,33 +4,74 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.fail; +import java.io.File; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; + import org.junit.Before; import org.junit.Test; -import voldemort.utils.SystemTime; +import voldemort.ServerTestUtils; +import voldemort.TestUtils; +import voldemort.client.protocol.admin.AdminClient; +import voldemort.cluster.Cluster; +import voldemort.server.VoldemortServer; +import voldemort.store.StoreDefinition; +import voldemort.store.socket.SocketStoreFactory; +import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; import voldemort.utils.Time; import voldemort.versioning.ObsoleteVersionException; import voldemort.versioning.VectorClock; import voldemort.versioning.Versioned; +import voldemort.xml.StoreDefinitionsMapper; public class DefaultSocketStoreClientTest { + private static String testStoreName = "test-replication-memory"; + private static String storesXmlfile = "test/common/voldemort/config/stores.xml"; + private SocketStoreFactory socketStoreFactory = new ClientRequestExecutorPool(2, + 10000, + 100000, + 32 * 1024); + private static AtomicBoolean running = new AtomicBoolean(true); + private List storeDefs; + private VoldemortServer[] servers; + private Cluster cluster; + private AdminClient adminClient; + + public static String socketUrl = "tcp://localhost:6667"; protected StoreClient client; protected int nodeId; protected Time time; @Before public void setUp() throws Exception { - String socketUrl = "tcp://localhost:6667"; - String storeName = "test"; - ClientConfig clientConfig = new ClientConfig().setMaxThreads(4) - .setMaxTotalConnections(4) - .setMaxConnectionsPerNode(4) - .setBootstrapUrls(socketUrl); - SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); - this.client = socketFactory.getStoreClient(storeName); - this.nodeId = 0; - this.time = SystemTime.INSTANCE; + cluster = ServerTestUtils.getLocalCluster(2, new int[][] { { 0, 1, 2, 3 }, { 4, 5, 6, 7 } }); + + servers = new VoldemortServer[2]; + storeDefs = new StoreDefinitionsMapper().readStoreList(new File(storesXmlfile)); + + servers[0] = ServerTestUtils.startVoldemortServer(socketStoreFactory, + ServerTestUtils.createServerConfig(useNio, + 0, + TestUtils.createTempDir() + .getAbsolutePath(), + null, + storesXmlfile, + new Properties()), + cluster); + servers[1] = ServerTestUtils.startVoldemortServer(socketStoreFactory, + ServerTestUtils.createServerConfig(useNio, + 1, + TestUtils.createTempDir() + .getAbsolutePath(), + null, + storesXmlfile, + new Properties()), + cluster); + + adminClient = ServerTestUtils.getAdminClient(cluster); } @Test @@ -60,4 +101,19 @@ public void test() { assertNull("After a successful delete(k), get(k) should return null.", client.get("k")); } + @Test + public void testClientRegistryHappyPath() { + ClientConfig clientConfig = new ClientConfig().setMaxThreads(4) + .setMaxTotalConnections(4) + .setMaxConnectionsPerNode(4) + .setBootstrapUrls(socketUrl) + .setClientContextName("testClientRegistryHappyPath"); + SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); + this.client = socketFactory.getStoreClient(testStoreName); + client.put("k", "v"); + adminClient.fetchEntries(0, testStoreName, null, null, false); + adminClient.fetchEntries(1, testStoreName, null, null, false); + // TODO: verify that the values in registry are correct. + } + } From dbc2f54eac11d7171aeaaf8bde85f46e96f657b6 Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Tue, 26 Jun 2012 20:18:09 -0700 Subject: [PATCH 079/209] add version and update time to client registry --- META-INF/MANIFEST.MF | 7 + build.xml | 11 +- src/java/voldemort/client/ClientConfig.java | 11 + src/java/voldemort/client/ClientInfo.java | 66 +- .../voldemort/client/DefaultStoreClient.java | 7 +- .../server/SystemStoreConstants.java | 77 -- .../store/system/SystemStoreConstants.java | 11 +- .../voldemort/utils/ManifestFileReader.java | 27 + src/java/voldemort/utils/RebalanceUtils.java | 22 +- test/common/voldemort/config/stores.xml | 30 + .../voldemort/client/ClientRegistryTest.java | 699 ++++++++++++++++++ 11 files changed, 872 insertions(+), 96 deletions(-) create mode 100644 META-INF/MANIFEST.MF delete mode 100644 src/java/voldemort/server/SystemStoreConstants.java create mode 100644 src/java/voldemort/utils/ManifestFileReader.java create mode 100644 test/unit/voldemort/client/ClientRegistryTest.java diff --git a/META-INF/MANIFEST.MF b/META-INF/MANIFEST.MF new file mode 100644 index 0000000000..445b9656e5 --- /dev/null +++ b/META-INF/MANIFEST.MF @@ -0,0 +1,7 @@ +Manifest-Version: 1.0 +Ant-Version: Apache Ant 1.7.1 +Created-By: 20.8-b03-424 (Apple Inc.) +Implementation-Title: Voldemort +Implementation-Version: 0.90.1 +Implementation-Vendor: LinkedIn + diff --git a/build.xml b/build.xml index 149472494d..df20f6f28a 100644 --- a/build.xml +++ b/build.xml @@ -5,7 +5,7 @@ - + @@ -76,6 +76,12 @@ + + + + + + @@ -141,6 +147,9 @@ + + + diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index 20ca113355..9bda12e4cb 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -80,6 +80,8 @@ public class ClientConfig { private volatile int maxBootstrapRetries = 2; private volatile String clientContextName = "default"; private volatile long asyncCheckMetadataInterval = 5000; + /* 12 hr refresh internval */ + private volatile long clientRegistryRefreshInterval = 3600 * 1000 * 12; public ClientConfig() {} @@ -721,4 +723,13 @@ public ClientConfig setAsyncCheckMetadataInterval(long asyncCheckMetadataInterva this.asyncCheckMetadataInterval = asyncCheckMetadataInterval; return this; } + + public long getClientRegistryRefreshInterval() { + return this.clientRegistryRefreshInterval; + } + + public ClientConfig setClientRegistryRefreshInterval(long clientRegistryRefrshInterval) { + this.clientRegistryRefreshInterval = clientRegistryRefrshInterval; + return this; + } } diff --git a/src/java/voldemort/client/ClientInfo.java b/src/java/voldemort/client/ClientInfo.java index 08886ad7d0..19958bfa58 100644 --- a/src/java/voldemort/client/ClientInfo.java +++ b/src/java/voldemort/client/ClientInfo.java @@ -23,14 +23,26 @@ public class ClientInfo implements Serializable { private int sequence; private String localHostName; private String deploymentPath; - - public ClientInfo(String storeName, String clientContext, int clientSequence, long bootstrapTime) { + private long updateTime; + private String releaseVersion; + + public ClientInfo(String storeName, + String clientContext, + int clientSequence, + long bootstrapTime, + String version) { this.bootstrapTime = bootstrapTime; this.storeName = storeName; this.context = clientContext; this.sequence = clientSequence; this.localHostName = createHostName(); this.deploymentPath = createDeploymentPath(); + this.updateTime = bootstrapTime; + this.releaseVersion = version; + + if(logger.isDebugEnabled()) { + logger.debug(this.toString()); + } } private String createDeploymentPath() { @@ -103,4 +115,54 @@ public void setLocalHostName(String localHostName) { public String getLocalHostName() { return localHostName; } + + public void setUpdateTime(long updateTime) { + this.updateTime = updateTime; + } + + public long getUpdateTime() { + return this.updateTime; + } + + public void setReleaseVersion(String version) { + this.releaseVersion = version; + } + + public String getReleaseVersion() { + return this.releaseVersion; + } + + @Override + public boolean equals(Object object) { + if(this == object) + return true; + if(object == null) + return false; + if(!object.getClass().equals(ClientInfo.class)) + return false; + ClientInfo clientInfo = (ClientInfo) object; + return (this.bootstrapTime == clientInfo.bootstrapTime) + && (this.context.equals(clientInfo.context)) + && (this.deploymentPath.equals(clientInfo.deploymentPath)) + && (this.localHostName.equals(clientInfo.localHostName)) + && (this.sequence == clientInfo.sequence) + && (this.storeName.equals(clientInfo.storeName)) + && (this.updateTime == clientInfo.updateTime) + && (this.releaseVersion == clientInfo.releaseVersion); + } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + builder.append("boostrapTime[").append(bootstrapTime).append("], "); + builder.append("context[").append(context).append("], "); + builder.append("deploymentPath[").append(deploymentPath).append("], "); + builder.append("localHostName[").append(localHostName).append("], "); + builder.append("sequence[").append(sequence).append("], "); + builder.append("storeName[").append(storeName).append("], "); + builder.append("updateTime[").append(updateTime).append("], "); + builder.append("releaseVersion[").append(releaseVersion).append("]"); + return builder.toString(); + } + } diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index 4aac50d8ee..d82b614aff 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -37,6 +37,7 @@ import voldemort.store.Store; import voldemort.store.StoreCapabilityType; import voldemort.utils.JmxUtils; +import voldemort.utils.ManifestFileReader; import voldemort.utils.Utils; import voldemort.versioning.InconsistencyResolver; import voldemort.versioning.InconsistentDataException; @@ -98,7 +99,8 @@ public DefaultStoreClient(String storeName, this.clientInfo = new ClientInfo(storeName, clientContext, clientSequence, - System.currentTimeMillis()); + System.currentTimeMillis(), + ManifestFileReader.getReleaseVersion()); this.clientId = AbstractStoreClientFactory.generateClientId(clientInfo); this.config = config; @@ -120,8 +122,7 @@ public DefaultStoreClient(String storeName, } registerClient(); - logger.info("Voldemort client created: clientContext=" + clientContext + " clientSequence=" - + clientSequence + " clientId=" + clientId.toString()); + logger.info("Voldemort client created: " + clientId.toString() + "\n" + clientInfo); } private void registerClient() { diff --git a/src/java/voldemort/server/SystemStoreConstants.java b/src/java/voldemort/server/SystemStoreConstants.java deleted file mode 100644 index a7c546681c..0000000000 --- a/src/java/voldemort/server/SystemStoreConstants.java +++ /dev/null @@ -1,77 +0,0 @@ -package voldemort.server; - -/** - * The various system stores - */ -public class SystemStoreConstants { - - public static final String NAME_PREFIX = "voldsys$_"; - - public static enum SystemStoreName { - voldsys$_client_registry, - voldsys$_client_store_definition; - } - - public static final String SYSTEM_STORE_SCHEMA = "" - + " " - + " voldsys$_client_registry" - + " zone-routing" - + " proximity-handoff" - + " memory" - + " client" - + " 4" - + " " - + " 2" - + " 2" - + " " - + " 1" - + " 1" - + " " - + " string" - + " " - + " " - + " string" - + " " - + " 7" - + " " - - + " " - + " voldsys$_client_store_definition" - + " zone-routing" - + " proximity-handoff" - + " memory" - + " client" - + " 1" - + " 1" - + " 1" - + " " - + " string" - + " " - + " " - + " string" - + " " - + " 7" - + " " - - + " " - + " voldsys$_metadata_version" - + " local-pref-all-routing" - + " proximity-handoff" - + " memory" - + " client" - + " 1" - + " 1" - + " 1" - + " " - + " string" - + " " - + " " - + " java-serialization" - + " " + " " - - + ""; - - public static boolean isSystemStore(String storeName) { - return (null == storeName ? false : storeName.startsWith(NAME_PREFIX)); - } -} \ No newline at end of file diff --git a/src/java/voldemort/store/system/SystemStoreConstants.java b/src/java/voldemort/store/system/SystemStoreConstants.java index 32a1d2e5be..c818ed51fd 100644 --- a/src/java/voldemort/store/system/SystemStoreConstants.java +++ b/src/java/voldemort/store/system/SystemStoreConstants.java @@ -4,6 +4,7 @@ import java.util.List; import voldemort.store.StoreDefinition; +import voldemort.utils.RebalanceUtils; import voldemort.xml.StoreDefinitionsMapper; /** @@ -22,7 +23,7 @@ public static enum SystemStoreName { public static final String SYSTEM_STORE_SCHEMA = "" + " " + " voldsys$_client_registry" - + " zone-routing" + + " all-routing" + " proximity-handoff" + " memory" + " client" @@ -92,13 +93,7 @@ public static List getAllSystemStoreDefs() { } public static StoreDefinition getSystemStoreDef(String name) { - StoreDefinition storeDef = null; List allDefs = getAllSystemStoreDefs(); - for(StoreDefinition def: allDefs) { - if(name.equals(def.getName())) { - storeDef = def; - } - } - return storeDef; + return RebalanceUtils.getStoreDefinitionWithName(allDefs, name); } } \ No newline at end of file diff --git a/src/java/voldemort/utils/ManifestFileReader.java b/src/java/voldemort/utils/ManifestFileReader.java new file mode 100644 index 0000000000..a3287a3681 --- /dev/null +++ b/src/java/voldemort/utils/ManifestFileReader.java @@ -0,0 +1,27 @@ +package voldemort.utils; + +import java.io.FileInputStream; +import java.io.IOException; +import java.util.Properties; + +import org.apache.log4j.Logger; + +public class ManifestFileReader { + + protected static final Logger logger = Logger.getLogger(ManifestFileReader.class); + + private static String MANIFEST_FILE = "META-INF/MANIFEST.MF"; + private static String RELEASE_VERSION_KEY = "Implementation-Version"; + + public static String getReleaseVersion() { + String version = null; + Properties properties = new Properties(); + try { + properties.load(new FileInputStream(MANIFEST_FILE)); + version = properties.getProperty(RELEASE_VERSION_KEY); + } catch(IOException e) { + logger.warn("Unable to load voldemort release version due to the following error:", e); + } + return version; + } +} diff --git a/src/java/voldemort/utils/RebalanceUtils.java b/src/java/voldemort/utils/RebalanceUtils.java index dbfb2a8df8..26cd4d6b44 100644 --- a/src/java/voldemort/utils/RebalanceUtils.java +++ b/src/java/voldemort/utils/RebalanceUtils.java @@ -50,6 +50,7 @@ import voldemort.cluster.Node; import voldemort.routing.RoutingStrategy; import voldemort.routing.RoutingStrategyFactory; +import voldemort.routing.RoutingStrategyType; import voldemort.server.VoldemortConfig; import voldemort.server.rebalance.VoldemortRebalancingException; import voldemort.store.StoreDefinition; @@ -486,11 +487,21 @@ public static boolean checkKeyBelongsToPartition(int nodeId, HashMap> replicaToPartitionList, Cluster cluster, StoreDefinition storeDef) { - List keyPartitions = new RoutingStrategyFactory().updateRoutingStrategy(storeDef, - cluster) - .getPartitionList(key); - List nodePartitions = cluster.getNodeById(nodeId).getPartitionIds(); - return checkKeyBelongsToPartition(keyPartitions, nodePartitions, replicaToPartitionList); + boolean checkResult = false; + if(storeDef.getRoutingStrategyType().equals(RoutingStrategyType.TO_ALL_STRATEGY) + || storeDef.getRoutingStrategyType() + .equals(RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY)) { + checkResult = true; + } else { + List keyPartitions = new RoutingStrategyFactory().updateRoutingStrategy(storeDef, + cluster) + .getPartitionList(key); + List nodePartitions = cluster.getNodeById(nodeId).getPartitionIds(); + checkResult = checkKeyBelongsToPartition(keyPartitions, + nodePartitions, + replicaToPartitionList); + } + return checkResult; } /** @@ -1395,6 +1406,7 @@ public static StoreDefinition getStoreDefinitionWithName(List s for(StoreDefinition storeDef: storeDefs) { if(storeDef.getName().compareTo(storeName) == 0) { def = storeDef; + break; } } diff --git a/test/common/voldemort/config/stores.xml b/test/common/voldemort/config/stores.xml index f25f33efa1..116726bfd4 100644 --- a/test/common/voldemort/config/stores.xml +++ b/test/common/voldemort/config/stores.xml @@ -185,4 +185,34 @@ all-handoff 10 + + test-store-eventual-1 + bdb + client + 2 + 1 + 1 + + string + UTF-8 + + + java-serialization + + + + test-store-eventual-2 + bdb + client + 2 + 1 + 1 + + string + UTF-8 + + + java-serialization + + diff --git a/test/unit/voldemort/client/ClientRegistryTest.java b/test/unit/voldemort/client/ClientRegistryTest.java new file mode 100644 index 0000000000..ccb254a29c --- /dev/null +++ b/test/unit/voldemort/client/ClientRegistryTest.java @@ -0,0 +1,699 @@ +package voldemort.client; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Properties; + +import junit.framework.Assert; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import voldemort.ServerTestUtils; +import voldemort.TestUtils; +import voldemort.client.protocol.admin.AdminClient; +import voldemort.cluster.Cluster; +import voldemort.serialization.DefaultSerializerFactory; +import voldemort.serialization.Serializer; +import voldemort.serialization.SerializerFactory; +import voldemort.server.VoldemortServer; +import voldemort.store.socket.SocketStoreFactory; +import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; +import voldemort.store.system.SystemStoreConstants; +import voldemort.utils.ByteArray; +import voldemort.utils.Pair; +import voldemort.versioning.Versioned; + +import com.google.common.collect.Lists; + +@SuppressWarnings( { "unchecked" }) +public class ClientRegistryTest { + + public static final String SERVER_LOCAL_URL = "tcp://localhost:"; + public static final String TEST_STORE_NAME = "test-store-eventual-1"; + public static final String TEST_STORE_NAME2 = "test-store-eventual-2"; + public static final String STORES_XML_FILE = "test/common/voldemort/config/stores.xml"; + public static final String CLIENT_CONTEXT_NAME = "testClientRegistryHappyPath"; + public static final String CLIENT_CONTEXT_NAME2 = "testClientRegistryUnhappyPath"; + public static final long CLIENT_REGISTRY_REFRSH_INTERVAL = 1000; + public static final int TOTAL_SERVERS = 2; + + private SocketStoreFactory socketStoreFactory = new ClientRequestExecutorPool(TOTAL_SERVERS, + 10000, + 100000, + 32 * 1024); + private static VoldemortServer[] servers = null; + private static int[] serverPorts = null; + private Cluster cluster = ServerTestUtils.getLocalCluster(2, new int[][] { { 0, 1, 2, 3 }, + { 4, 5, 6, 7 } }); + private static AdminClient adminClient; + + private SerializerFactory serializerFactory = new DefaultSerializerFactory(); + private Serializer valueSerializer = (Serializer) serializerFactory.getSerializer(SystemStoreConstants.getSystemStoreDef(SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name()) + .getValueSerializer()); + private long startTime; + + @Before + public void setUp() throws Exception { + + if(null == servers) { + servers = new VoldemortServer[TOTAL_SERVERS]; + serverPorts = new int[TOTAL_SERVERS]; + + for(int i = 0; i < TOTAL_SERVERS; i++) { + servers[i] = ServerTestUtils.startVoldemortServer(socketStoreFactory, + ServerTestUtils.createServerConfig(true, + i, + TestUtils.createTempDir() + .getAbsolutePath(), + null, + STORES_XML_FILE, + new Properties()), + cluster); + serverPorts[i] = servers[i].getIdentityNode().getSocketPort(); + } + adminClient = ServerTestUtils.getAdminClient(cluster); + } + + startTime = System.currentTimeMillis(); + } + + @After + public void tearDown() throws Exception { + this.clearRegistryContent(); + } + + @Test + public void testHappyPath() { + List emptyPartitionList = Lists.newArrayList(); + ClientConfig clientConfig = new ClientConfig().setMaxThreads(4) + .setMaxTotalConnections(4) + .setMaxConnectionsPerNode(4) + .setBootstrapUrls(SERVER_LOCAL_URL + + serverPorts[0]) + .setClientContextName(CLIENT_CONTEXT_NAME) + .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL); + SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); + StoreClient client = socketFactory.getStoreClient(TEST_STORE_NAME); + client.put("k", "v"); + Iterator>> it = adminClient.fetchEntries(0, + SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(), + emptyPartitionList, + null, + false); + ArrayList infoList = getClientRegistryContent(it); + assertEquals(TEST_STORE_NAME, infoList.get(0).getStoreName()); + assertEquals(CLIENT_CONTEXT_NAME, infoList.get(0).getContext()); + assertEquals(0, infoList.get(0).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(0).getBootstrapTime()); + Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + assertEquals(1, infoList.size()); + + it = adminClient.fetchEntries(1, + SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(), + emptyPartitionList, + null, + false); + infoList = getClientRegistryContent(it); + assertEquals(TEST_STORE_NAME, infoList.get(0).getStoreName()); + assertEquals(CLIENT_CONTEXT_NAME, infoList.get(0).getContext()); + assertEquals(0, infoList.get(0).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(0).getBootstrapTime()); + Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + assertEquals(1, infoList.size()); + + try { + Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 2); + } catch(InterruptedException e) {} + // now the periodical update has gone through, it shall be higher than + // the bootstrap time + it = adminClient.fetchEntries(1, + SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(), + emptyPartitionList, + null, + false); + infoList = getClientRegistryContent(it); + assertTrue("Client registry not updated.", + infoList.get(0).getBootstrapTime() < infoList.get(0).getUpdateTime()); + } + + @Test + public void testTwoClients() { + List emptyPartitionList = Lists.newArrayList(); + ClientConfig clientConfig = new ClientConfig().setMaxThreads(4) + .setMaxTotalConnections(4) + .setMaxConnectionsPerNode(4) + .setBootstrapUrls(SERVER_LOCAL_URL + + serverPorts[0]) + .setClientContextName(CLIENT_CONTEXT_NAME) + .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL); + SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); + StoreClient client1 = socketFactory.getStoreClient(TEST_STORE_NAME); + StoreClient client2 = socketFactory.getStoreClient(TEST_STORE_NAME); + + client1.put("k1", "v1"); + client2.put("k2", "v2"); + + Iterator>> it = adminClient.fetchEntries(0, + SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(), + emptyPartitionList, + null, + false); + ArrayList infoList = getClientRegistryContent(it); + assertEquals(TEST_STORE_NAME, infoList.get(0).getStoreName()); + assertEquals(CLIENT_CONTEXT_NAME, infoList.get(0).getContext()); + assertTrue("Client registry sequence number incorrect", 1 >= infoList.get(0) + .getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(0).getBootstrapTime()); + Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + + assertEquals(TEST_STORE_NAME, infoList.get(1).getStoreName()); + assertEquals(CLIENT_CONTEXT_NAME, infoList.get(1).getContext()); + assertTrue("Client registry sequence number incorrect", 1 >= infoList.get(1) + .getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(1).getBootstrapTime()); + Assert.assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); + assertEquals(infoList.size(), 2); + + it = adminClient.fetchEntries(1, + SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(), + emptyPartitionList, + null, + false); + infoList = getClientRegistryContent(it); + assertEquals(TEST_STORE_NAME, infoList.get(0).getStoreName()); + assertEquals(CLIENT_CONTEXT_NAME, infoList.get(0).getContext()); + assertTrue("Client registry sequence number incorrect", 1 >= infoList.get(0) + .getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(0).getBootstrapTime()); + Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + + assertEquals(TEST_STORE_NAME, infoList.get(1).getStoreName()); + assertEquals(CLIENT_CONTEXT_NAME, infoList.get(1).getContext()); + assertTrue("Client registry sequence number incorrect", 1 >= infoList.get(1) + .getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(1).getBootstrapTime()); + Assert.assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); + + assertEquals(infoList.size(), 2); + + try { + Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 2); + } catch(InterruptedException e) {} + // now the periodical update has gone through, it shall be higher than + // the bootstrap time + it = adminClient.fetchEntries(1, + SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(), + emptyPartitionList, + null, + false); + infoList = getClientRegistryContent(it); + assertTrue("Client registry not updated.", + infoList.get(0).getBootstrapTime() < infoList.get(0).getUpdateTime()); + assertTrue("Client registry not updated.", + infoList.get(1).getBootstrapTime() < infoList.get(1).getUpdateTime()); + } + + @Test + public void testTwoStores() { + List emptyPartitionList = Lists.newArrayList(); + ClientConfig clientConfig = new ClientConfig().setMaxThreads(4) + .setMaxTotalConnections(4) + .setMaxConnectionsPerNode(4) + .setBootstrapUrls(SERVER_LOCAL_URL + + serverPorts[0]) + .setClientContextName(CLIENT_CONTEXT_NAME) + .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL); + SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); + StoreClient client1 = socketFactory.getStoreClient(TEST_STORE_NAME); + StoreClient client2 = socketFactory.getStoreClient(TEST_STORE_NAME2); + + client1.put("k1", "v1"); + client2.put("k2", "v2"); + + Iterator>> it = adminClient.fetchEntries(0, + SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(), + emptyPartitionList, + null, + false); + ArrayList infoList = getClientRegistryContent(it); + + assertEquals(CLIENT_CONTEXT_NAME, infoList.get(0).getContext()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(0).getBootstrapTime()); + Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + + assertEquals(CLIENT_CONTEXT_NAME, infoList.get(1).getContext()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(1).getBootstrapTime()); + Assert.assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); + + if(infoList.get(0).getStoreName().equals(TEST_STORE_NAME)) { + assertEquals(0, infoList.get(0).getClientSequence()); + assertEquals(TEST_STORE_NAME2, infoList.get(1).getStoreName()); + assertEquals(1, infoList.get(1).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + infoList.get(1).getBootstrapTime() >= infoList.get(0).getBootstrapTime()); + } else { + assertEquals(TEST_STORE_NAME2, infoList.get(0).getStoreName()); + assertEquals(1, infoList.get(0).getClientSequence()); + assertEquals(TEST_STORE_NAME, infoList.get(1).getStoreName()); + assertEquals(0, infoList.get(1).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + infoList.get(0).getBootstrapTime() >= infoList.get(1).getBootstrapTime()); + } + + it = adminClient.fetchEntries(1, + SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(), + emptyPartitionList, + null, + false); + infoList = getClientRegistryContent(it); + + assertEquals(CLIENT_CONTEXT_NAME, infoList.get(0).getContext()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(0).getBootstrapTime()); + Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + + assertEquals(CLIENT_CONTEXT_NAME, infoList.get(1).getContext()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(1).getBootstrapTime()); + Assert.assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); + + if(infoList.get(0).getStoreName().equals(TEST_STORE_NAME)) { + assertEquals(0, infoList.get(0).getClientSequence()); + assertEquals(TEST_STORE_NAME2, infoList.get(1).getStoreName()); + assertEquals(1, infoList.get(1).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + infoList.get(1).getBootstrapTime() >= infoList.get(0).getBootstrapTime()); + } else { + assertEquals(TEST_STORE_NAME2, infoList.get(0).getStoreName()); + assertEquals(1, infoList.get(0).getClientSequence()); + assertEquals(TEST_STORE_NAME, infoList.get(1).getStoreName()); + assertEquals(0, infoList.get(1).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + infoList.get(0).getBootstrapTime() >= infoList.get(1).getBootstrapTime()); + } + + try { + Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 2); + } catch(InterruptedException e) {} + // now the periodical update has gone through, it shall be higher than + // the bootstrap time + it = adminClient.fetchEntries(1, + SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(), + emptyPartitionList, + null, + false); + infoList = getClientRegistryContent(it); + assertTrue("Client registry not updated.", + infoList.get(0).getBootstrapTime() < infoList.get(0).getUpdateTime()); + assertTrue("Client registry not updated.", + infoList.get(1).getBootstrapTime() < infoList.get(1).getUpdateTime()); + } + + @Test + public void testTwoFactories() { + List emptyPartitionList = Lists.newArrayList(); + ClientConfig clientConfig = new ClientConfig().setMaxThreads(4) + .setMaxTotalConnections(4) + .setMaxConnectionsPerNode(4) + .setBootstrapUrls(SERVER_LOCAL_URL + + serverPorts[0]) + .setClientContextName(CLIENT_CONTEXT_NAME) + .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL); + SocketStoreClientFactory socketFactory1 = new SocketStoreClientFactory(clientConfig); + + ClientConfig clientConfig2 = new ClientConfig().setMaxThreads(4) + .setMaxTotalConnections(4) + .setMaxConnectionsPerNode(4) + .setBootstrapUrls(SERVER_LOCAL_URL + + serverPorts[0]) + .setClientContextName(CLIENT_CONTEXT_NAME2); + SocketStoreClientFactory socketFactory2 = new SocketStoreClientFactory(clientConfig2); + + StoreClient client1 = socketFactory1.getStoreClient(TEST_STORE_NAME); + StoreClient client2 = socketFactory2.getStoreClient(TEST_STORE_NAME2); + + client1.put("k1", "v1"); + client2.put("k2", "v2"); + + Iterator>> it = adminClient.fetchEntries(0, + SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(), + emptyPartitionList, + null, + false); + ArrayList infoList = getClientRegistryContent(it); + + Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + Assert.assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); + + if(infoList.get(0).getStoreName().equals(TEST_STORE_NAME)) { + assertEquals(CLIENT_CONTEXT_NAME, infoList.get(0).getContext()); + assertEquals(0, infoList.get(0).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(0).getBootstrapTime()); + + assertEquals(TEST_STORE_NAME2, infoList.get(1).getStoreName()); + assertEquals(CLIENT_CONTEXT_NAME2, infoList.get(1).getContext()); + assertEquals(0, infoList.get(1).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(1).getBootstrapTime()); + + assertTrue("Client registry bootstrap time incorrect", + infoList.get(1).getBootstrapTime() >= infoList.get(0).getBootstrapTime()); + + } else { + assertEquals(TEST_STORE_NAME2, infoList.get(0).getStoreName()); + assertEquals(CLIENT_CONTEXT_NAME2, infoList.get(0).getContext()); + assertEquals(0, infoList.get(0).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(0).getBootstrapTime()); + + assertEquals(TEST_STORE_NAME, infoList.get(1).getStoreName()); + assertEquals(CLIENT_CONTEXT_NAME, infoList.get(1).getContext()); + assertEquals(0, infoList.get(1).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(1).getBootstrapTime()); + + assertTrue("Client registry bootstrap time incorrect", + infoList.get(0).getBootstrapTime() >= infoList.get(1).getBootstrapTime()); + } + + it = adminClient.fetchEntries(1, + SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(), + emptyPartitionList, + null, + false); + infoList = getClientRegistryContent(it); + + Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + Assert.assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); + + if(infoList.get(0).getStoreName().equals(TEST_STORE_NAME)) { + assertEquals(CLIENT_CONTEXT_NAME, infoList.get(0).getContext()); + assertEquals(0, infoList.get(0).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(0).getBootstrapTime()); + + assertEquals(TEST_STORE_NAME2, infoList.get(1).getStoreName()); + assertEquals(CLIENT_CONTEXT_NAME2, infoList.get(1).getContext()); + assertEquals(0, infoList.get(1).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(1).getBootstrapTime()); + + assertTrue("Client registry bootstrap time incorrect", + infoList.get(1).getBootstrapTime() >= infoList.get(0).getBootstrapTime()); + + } else { + assertEquals(TEST_STORE_NAME2, infoList.get(0).getStoreName()); + assertEquals(CLIENT_CONTEXT_NAME2, infoList.get(0).getContext()); + assertEquals(0, infoList.get(0).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(0).getBootstrapTime()); + + assertEquals(TEST_STORE_NAME, infoList.get(1).getStoreName()); + assertEquals(CLIENT_CONTEXT_NAME, infoList.get(1).getContext()); + assertEquals(0, infoList.get(1).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(1).getBootstrapTime()); + + assertTrue("Client registry bootstrap time incorrect", + infoList.get(0).getBootstrapTime() >= infoList.get(1).getBootstrapTime()); + } + + try { + Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 2); + } catch(InterruptedException e) {} + // now the periodical update has gone through, it shall be higher than + // the bootstrap time + it = adminClient.fetchEntries(1, + SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(), + emptyPartitionList, + null, + false); + infoList = getClientRegistryContent(it); + assertTrue("Client registry not updated.", + infoList.get(0).getBootstrapTime() < infoList.get(0).getUpdateTime()); + assertTrue("Client registry not updated.", + infoList.get(1).getBootstrapTime() < infoList.get(1).getUpdateTime()); + } + + @Test + public void testOneServerFailre() { + // bring down one server before starting up the clients + servers[0].stop(); + + List emptyPartitionList = Lists.newArrayList(); + ClientConfig clientConfig = new ClientConfig().setMaxThreads(4) + .setMaxTotalConnections(4) + .setMaxConnectionsPerNode(4) + .setBootstrapUrls(SERVER_LOCAL_URL + + serverPorts[1]) + .setClientContextName(CLIENT_CONTEXT_NAME) + .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL); + SocketStoreClientFactory socketFactory1 = new SocketStoreClientFactory(clientConfig); + + ClientConfig clientConfig2 = new ClientConfig().setMaxThreads(4) + .setMaxTotalConnections(4) + .setMaxConnectionsPerNode(4) + .setBootstrapUrls(SERVER_LOCAL_URL + + serverPorts[1]) + .setClientContextName(CLIENT_CONTEXT_NAME2); + SocketStoreClientFactory socketFactory2 = new SocketStoreClientFactory(clientConfig2); + + StoreClient client1 = socketFactory1.getStoreClient(TEST_STORE_NAME); + StoreClient client2 = socketFactory2.getStoreClient(TEST_STORE_NAME2); + + client1.put("k1", "v1"); + client2.put("k2", "v2"); + + Iterator>> it = adminClient.fetchEntries(1, + SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(), + emptyPartitionList, + null, + false); + ArrayList infoList = getClientRegistryContent(it); + + Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + Assert.assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); + + if(infoList.get(0).getStoreName().equals(TEST_STORE_NAME)) { + assertEquals(CLIENT_CONTEXT_NAME, infoList.get(0).getContext()); + assertEquals(0, infoList.get(0).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(0).getBootstrapTime()); + + assertEquals(TEST_STORE_NAME2, infoList.get(1).getStoreName()); + assertEquals(CLIENT_CONTEXT_NAME2, infoList.get(1).getContext()); + assertEquals(0, infoList.get(1).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(1).getBootstrapTime()); + + assertTrue("Client registry bootstrap time incorrect", + infoList.get(1).getBootstrapTime() >= infoList.get(0).getBootstrapTime()); + + } else { + assertEquals(TEST_STORE_NAME2, infoList.get(0).getStoreName()); + assertEquals(CLIENT_CONTEXT_NAME2, infoList.get(0).getContext()); + assertEquals(0, infoList.get(0).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(0).getBootstrapTime()); + + assertEquals(TEST_STORE_NAME, infoList.get(1).getStoreName()); + assertEquals(CLIENT_CONTEXT_NAME, infoList.get(1).getContext()); + assertEquals(0, infoList.get(1).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(1).getBootstrapTime()); + + assertTrue("Client registry bootstrap time incorrect", + infoList.get(0).getBootstrapTime() >= infoList.get(1).getBootstrapTime()); + } + + try { + Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 2); + } catch(InterruptedException e) {} + // now the periodical update has gone through, it shall be higher than + // the bootstrap time + it = adminClient.fetchEntries(1, + SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(), + emptyPartitionList, + null, + false); + infoList = getClientRegistryContent(it); + assertTrue("Client registry not updated.", + infoList.get(0).getBootstrapTime() < infoList.get(0).getUpdateTime()); + assertTrue("Client registry not updated.", + infoList.get(1).getBootstrapTime() < infoList.get(1).getUpdateTime()); + } + + @Test + public void testRepeatRegistrationSameFactory() { + + List emptyPartitionList = Lists.newArrayList(); + ClientConfig clientConfig = new ClientConfig().setMaxThreads(4) + .setMaxTotalConnections(4) + .setMaxConnectionsPerNode(4) + .setBootstrapUrls(SERVER_LOCAL_URL + + serverPorts[1]) + .setClientContextName(CLIENT_CONTEXT_NAME) + .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL); + SocketStoreClientFactory socketFactory1 = new SocketStoreClientFactory(clientConfig); + + ClientConfig clientConfig2 = new ClientConfig().setMaxThreads(4) + .setMaxTotalConnections(4) + .setMaxConnectionsPerNode(4) + .setBootstrapUrls(SERVER_LOCAL_URL + + serverPorts[1]) + .setClientContextName(CLIENT_CONTEXT_NAME2); + SocketStoreClientFactory socketFactory2 = new SocketStoreClientFactory(clientConfig2); + + for(int i = 0; i < 3; i++) { + + StoreClient client1 = socketFactory1.getStoreClient(TEST_STORE_NAME); + StoreClient client2 = socketFactory2.getStoreClient(TEST_STORE_NAME2); + + client1.put("k1", "v1"); + client2.put("k2", "v2"); + } + + Iterator>> it = adminClient.fetchEntries(1, + SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(), + emptyPartitionList, + null, + false); + ArrayList infoList = getClientRegistryContent(it); + assertEquals("Incrrect # of entries created in client registry", 6, infoList.size()); + } + + @Test + public void testRepeatRegistrationDifferentFactories() { + long client1LastBootstrapTime = 0; + long client2LastBootstrapTime = 0; + for(int i = 0; i < 3; i++) { + + List emptyPartitionList = Lists.newArrayList(); + ClientConfig clientConfig = new ClientConfig().setMaxThreads(4) + .setMaxTotalConnections(4) + .setMaxConnectionsPerNode(4) + .setBootstrapUrls(SERVER_LOCAL_URL + + serverPorts[1]) + .setClientContextName(CLIENT_CONTEXT_NAME) + .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL); + SocketStoreClientFactory socketFactory1 = new SocketStoreClientFactory(clientConfig); + + ClientConfig clientConfig2 = new ClientConfig().setMaxThreads(4) + .setMaxTotalConnections(4) + .setMaxConnectionsPerNode(4) + .setBootstrapUrls(SERVER_LOCAL_URL + + serverPorts[1]) + .setClientContextName(CLIENT_CONTEXT_NAME2); + SocketStoreClientFactory socketFactory2 = new SocketStoreClientFactory(clientConfig2); + + StoreClient client1 = socketFactory1.getStoreClient(TEST_STORE_NAME); + StoreClient client2 = socketFactory2.getStoreClient(TEST_STORE_NAME2); + + client1.put("k1", "v1"); + client2.put("k2", "v2"); + + Iterator>> it = adminClient.fetchEntries(1, + SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(), + emptyPartitionList, + null, + false); + ArrayList infoList = getClientRegistryContent(it); + + assertEquals("Incrrect # of entries created in client registry", 2, infoList.size()); + + Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + Assert.assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); + + if(infoList.get(0).getStoreName().equals(TEST_STORE_NAME)) { + assertEquals(CLIENT_CONTEXT_NAME, infoList.get(0).getContext()); + assertEquals(0, infoList.get(0).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(0).getBootstrapTime()); + + assertEquals(TEST_STORE_NAME2, infoList.get(1).getStoreName()); + assertEquals(CLIENT_CONTEXT_NAME2, infoList.get(1).getContext()); + assertEquals(0, infoList.get(1).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(1).getBootstrapTime()); + + assertTrue("Client registry bootstrap time incorrect", + infoList.get(1).getBootstrapTime() >= infoList.get(0).getBootstrapTime()); + + } else { + assertEquals(TEST_STORE_NAME2, infoList.get(0).getStoreName()); + assertEquals(CLIENT_CONTEXT_NAME2, infoList.get(0).getContext()); + assertEquals(0, infoList.get(0).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(0).getBootstrapTime()); + + assertEquals(TEST_STORE_NAME, infoList.get(1).getStoreName()); + assertEquals(CLIENT_CONTEXT_NAME, infoList.get(1).getContext()); + assertEquals(0, infoList.get(1).getClientSequence()); + assertTrue("Client registry bootstrap time incorrect", + startTime <= infoList.get(1).getBootstrapTime()); + + assertTrue("Client registry bootstrap time incorrect", + infoList.get(0).getBootstrapTime() >= infoList.get(1).getBootstrapTime()); + } + + try { + Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 2); + } catch(InterruptedException e) {} + // now the periodical update has gone through, it shall be higher + // than + // the bootstrap time + it = adminClient.fetchEntries(1, + SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(), + emptyPartitionList, + null, + false); + infoList = getClientRegistryContent(it); + /* + * assertTrue("Client registry not updated.", + * infoList.get(0).getBootstrapTime() < + * infoList.get(0).getUpdateTime()); + * assertTrue("Client registry not updated.", + * infoList.get(1).getBootstrapTime() < + * infoList.get(1).getUpdateTime()); + */ + assertTrue("Bootstrap time does not increase client bounces", + infoList.get(0).getBootstrapTime() > client1LastBootstrapTime); + assertTrue("Bootstrap time does not increase client bounces", + infoList.get(1).getBootstrapTime() > client2LastBootstrapTime); + + client1LastBootstrapTime = infoList.get(0).getBootstrapTime(); + client2LastBootstrapTime = infoList.get(0).getBootstrapTime(); + } + } + + private ArrayList getClientRegistryContent(Iterator>> it) { + ArrayList infoList = Lists.newArrayList(); + while(it.hasNext()) { + infoList.add((ClientInfo) valueSerializer.toObject(it.next().getSecond().getValue())); + } + return infoList; + } + + private void clearRegistryContent() { + for(int i = 0; i < TOTAL_SERVERS; i++) { + servers[i].getStoreRepository() + .getStorageEngine(SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name()) + .truncate(); + } + } +} From ba4127c51755f50c3eef0b8b48f3374b40d5484b Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Wed, 27 Jun 2012 00:39:58 -0700 Subject: [PATCH 080/209] fix a merge error --- src/java/voldemort/client/DefaultStoreClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index d82b614aff..64977704af 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -32,10 +32,10 @@ import voldemort.cluster.Node; import voldemort.routing.RoutingStrategy; import voldemort.serialization.Serializer; -import voldemort.server.SystemStoreConstants; import voldemort.store.InvalidMetadataException; import voldemort.store.Store; import voldemort.store.StoreCapabilityType; +import voldemort.store.system.SystemStoreConstants; import voldemort.utils.JmxUtils; import voldemort.utils.ManifestFileReader; import voldemort.utils.Utils; From d4e28a810abc50efb72952e63929787998560d1a Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Wed, 27 Jun 2012 17:27:26 -0700 Subject: [PATCH 081/209] fixed addtional bugs during merge --- src/java/voldemort/client/SystemStore.java | 4 +- .../voldemort/client/ClientRegistryTest.java | 40 +++++++++---------- 2 files changed, 22 insertions(+), 22 deletions(-) diff --git a/src/java/voldemort/client/SystemStore.java b/src/java/voldemort/client/SystemStore.java index 8ed7184a1d..0e68529cf4 100644 --- a/src/java/voldemort/client/SystemStore.java +++ b/src/java/voldemort/client/SystemStore.java @@ -26,14 +26,14 @@ public SystemStore(String storeName, String[] bootstrapUrls, int clientZoneID) { ClientConfig config = new ClientConfig(); config.setSelectors(1) - .setBootstrapUrls(config.getBootstrapUrls()) + .setBootstrapUrls(bootstrapUrls) .setMaxConnectionsPerNode(2) .setConnectionTimeout(1500, TimeUnit.MILLISECONDS) .setSocketTimeout(5000, TimeUnit.MILLISECONDS) .setRoutingTimeout(5000, TimeUnit.MILLISECONDS) .setEnableJmx(false) .setEnablePipelineRoutedStore(true) - .setClientZoneId(config.getClientZoneId()); + .setClientZoneId(clientZoneID); this.systemStoreFactory = new SocketStoreClientFactory(config); this.storeName = storeName; this.sysStore = this.systemStoreFactory.getSystemStore(this.storeName); diff --git a/test/unit/voldemort/client/ClientRegistryTest.java b/test/unit/voldemort/client/ClientRegistryTest.java index ccb254a29c..5aaa80b2cd 100644 --- a/test/unit/voldemort/client/ClientRegistryTest.java +++ b/test/unit/voldemort/client/ClientRegistryTest.java @@ -1,6 +1,7 @@ package voldemort.client; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import java.util.ArrayList; @@ -8,8 +9,6 @@ import java.util.List; import java.util.Properties; -import junit.framework.Assert; - import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -112,7 +111,8 @@ public void testHappyPath() { assertEquals(0, infoList.get(0).getClientSequence()); assertTrue("Client registry bootstrap time incorrect", startTime <= infoList.get(0).getBootstrapTime()); - Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + + assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); assertEquals(1, infoList.size()); it = adminClient.fetchEntries(1, @@ -126,7 +126,7 @@ public void testHappyPath() { assertEquals(0, infoList.get(0).getClientSequence()); assertTrue("Client registry bootstrap time incorrect", startTime <= infoList.get(0).getBootstrapTime()); - Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); assertEquals(1, infoList.size()); try { @@ -173,7 +173,7 @@ public void testTwoClients() { .getClientSequence()); assertTrue("Client registry bootstrap time incorrect", startTime <= infoList.get(0).getBootstrapTime()); - Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); assertEquals(TEST_STORE_NAME, infoList.get(1).getStoreName()); assertEquals(CLIENT_CONTEXT_NAME, infoList.get(1).getContext()); @@ -181,7 +181,7 @@ public void testTwoClients() { .getClientSequence()); assertTrue("Client registry bootstrap time incorrect", startTime <= infoList.get(1).getBootstrapTime()); - Assert.assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); + assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); assertEquals(infoList.size(), 2); it = adminClient.fetchEntries(1, @@ -196,7 +196,7 @@ public void testTwoClients() { .getClientSequence()); assertTrue("Client registry bootstrap time incorrect", startTime <= infoList.get(0).getBootstrapTime()); - Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); assertEquals(TEST_STORE_NAME, infoList.get(1).getStoreName()); assertEquals(CLIENT_CONTEXT_NAME, infoList.get(1).getContext()); @@ -204,7 +204,7 @@ public void testTwoClients() { .getClientSequence()); assertTrue("Client registry bootstrap time incorrect", startTime <= infoList.get(1).getBootstrapTime()); - Assert.assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); + assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); assertEquals(infoList.size(), 2); @@ -252,12 +252,12 @@ public void testTwoStores() { assertEquals(CLIENT_CONTEXT_NAME, infoList.get(0).getContext()); assertTrue("Client registry bootstrap time incorrect", startTime <= infoList.get(0).getBootstrapTime()); - Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); assertEquals(CLIENT_CONTEXT_NAME, infoList.get(1).getContext()); assertTrue("Client registry bootstrap time incorrect", startTime <= infoList.get(1).getBootstrapTime()); - Assert.assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); + assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); if(infoList.get(0).getStoreName().equals(TEST_STORE_NAME)) { assertEquals(0, infoList.get(0).getClientSequence()); @@ -284,12 +284,12 @@ public void testTwoStores() { assertEquals(CLIENT_CONTEXT_NAME, infoList.get(0).getContext()); assertTrue("Client registry bootstrap time incorrect", startTime <= infoList.get(0).getBootstrapTime()); - Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); assertEquals(CLIENT_CONTEXT_NAME, infoList.get(1).getContext()); assertTrue("Client registry bootstrap time incorrect", startTime <= infoList.get(1).getBootstrapTime()); - Assert.assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); + assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); if(infoList.get(0).getStoreName().equals(TEST_STORE_NAME)) { assertEquals(0, infoList.get(0).getClientSequence()); @@ -356,8 +356,8 @@ public void testTwoFactories() { false); ArrayList infoList = getClientRegistryContent(it); - Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); - Assert.assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); + assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); if(infoList.get(0).getStoreName().equals(TEST_STORE_NAME)) { assertEquals(CLIENT_CONTEXT_NAME, infoList.get(0).getContext()); @@ -398,8 +398,8 @@ public void testTwoFactories() { false); infoList = getClientRegistryContent(it); - Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); - Assert.assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); + assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); if(infoList.get(0).getStoreName().equals(TEST_STORE_NAME)) { assertEquals(CLIENT_CONTEXT_NAME, infoList.get(0).getContext()); @@ -486,8 +486,8 @@ public void testOneServerFailre() { false); ArrayList infoList = getClientRegistryContent(it); - Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); - Assert.assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); + assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); if(infoList.get(0).getStoreName().equals(TEST_STORE_NAME)) { assertEquals(CLIENT_CONTEXT_NAME, infoList.get(0).getContext()); @@ -616,8 +616,8 @@ public void testRepeatRegistrationDifferentFactories() { assertEquals("Incrrect # of entries created in client registry", 2, infoList.size()); - Assert.assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); - Assert.assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); + assertNotNull("Client version is null", infoList.get(0).getReleaseVersion()); + assertNotNull("Client version is null", infoList.get(1).getReleaseVersion()); if(infoList.get(0).getStoreName().equals(TEST_STORE_NAME)) { assertEquals(CLIENT_CONTEXT_NAME, infoList.get(0).getContext()); From e695a64517e82091102c0d35a251c66033212d92 Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Thu, 28 Jun 2012 12:01:18 -0700 Subject: [PATCH 082/209] merge code for automatic reboostrap --- src/java/voldemort/VoldemortAdminTool.java | 41 +++-- .../client/AbstractStoreClientFactory.java | 15 +- .../client/AsyncMetadataVersionManager.java | 150 ++++++++++++------ .../client/CachingStoreClientFactory.java | 5 +- .../voldemort/client/DefaultStoreClient.java | 87 ++++++---- .../voldemort/client/LazyStoreClient.java | 16 ++ .../client/MockStoreClientFactory.java | 3 +- .../client/SocketStoreClientFactory.java | 8 +- .../voldemort/client/StoreClientFactory.java | 3 +- src/java/voldemort/client/SystemStore.java | 8 +- .../client/SystemStoreRepository.java | 42 +++++ .../voldemort/StaticStoreClientFactory.java | 3 +- test/common/voldemort/config/stores.xml | 10 ++ .../client/DefaultSocketStoreClientTest.java | 99 +++++++++--- 14 files changed, 363 insertions(+), 127 deletions(-) create mode 100644 src/java/voldemort/client/SystemStoreRepository.java diff --git a/src/java/voldemort/VoldemortAdminTool.java b/src/java/voldemort/VoldemortAdminTool.java index c3ce8353df..bc7384feaf 100644 --- a/src/java/voldemort/VoldemortAdminTool.java +++ b/src/java/voldemort/VoldemortAdminTool.java @@ -88,6 +88,8 @@ public class VoldemortAdminTool { private static final String ALL_METADATA = "all"; + private static final String STORES_VERSION_KEY = "stores.xml"; + private static final String CLUSTER_VERSION_KEY = "cluster.xml"; private static SystemStore sysStoreVersion = null; @SuppressWarnings("unchecked") @@ -429,6 +431,9 @@ public static void main(String[] args) throws Exception { adminClient, MetadataStore.CLUSTER_KEY, mapper.writeCluster(newCluster)); + + // Update the cluster.xml version info + updateMetadataversion(CLUSTER_VERSION_KEY); } else if(metadataKey.compareTo(MetadataStore.SERVER_STATE_KEY) == 0) { VoldemortState newState = VoldemortState.valueOf(metadataValue); executeSetMetadata(nodeId, @@ -445,8 +450,14 @@ public static void main(String[] args) throws Exception { MetadataStore.STORES_KEY, mapper.writeStoreList(storeDefs)); - // Update the store metadata version - updateStoreMetadataversion(); + /* + * Update the store metadata version + * + * TODO: For now write one version for the entire + * stores.xml When we split the stores.xml, make this + * more granular + */ + updateMetadataversion(STORES_VERSION_KEY); } else if(metadataKey.compareTo(MetadataStore.REBALANCING_STEAL_INFO) == 0) { if(!Utils.isReadableFile(metadataValue)) @@ -737,22 +748,17 @@ private static void executeCheckMetadata(AdminClient adminClient, String metadat } } - /* - * TODO: For now write one version for the entire stores.xml When we split - * the stores.xml, make this more granular - */ - private static void updateStoreMetadataversion() { - String versionKey = "stores.xml"; - Versioned storesVersion = sysStoreVersion.getSysStore(versionKey); - if(storesVersion == null) { + private static void updateMetadataversion(String versionKey) { + Versioned metadataVersion = sysStoreVersion.getSysStore(versionKey); + if(metadataVersion == null) { System.err.println("Current version is null. Assuming version 0."); - storesVersion = new Versioned((long) 1); + metadataVersion = new Versioned((long) 1); } else { - System.out.println("Version obtained = " + storesVersion.getValue()); - long newValue = storesVersion.getValue() + 1; - storesVersion.setObject(newValue); + System.out.println("Version obtained = " + metadataVersion.getValue()); + long newValue = metadataVersion.getValue() + 1; + metadataVersion.setObject(newValue); } - sysStoreVersion.putSysStore(versionKey, storesVersion); + sysStoreVersion.putSysStore(versionKey, metadataVersion); } private static void executeSetMetadata(Integer nodeId, @@ -794,8 +800,9 @@ private static void executeSetMetadata(Integer nodeId, + adminClient.getAdminClientCluster() .getNodeById(currentNodeId) .getId()); - adminClient.updateRemoteMetadata(currentNodeId, key, Versioned.value(value.toString(), - updatedVersion)); + adminClient.updateRemoteMetadata(currentNodeId, + key, + Versioned.value(value.toString(), updatedVersion)); } } diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index 4d9512813f..af455040a5 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -156,14 +156,15 @@ public StoreClient getStoreClient(String storeName, public Store getRawStore(String storeName, InconsistencyResolver> resolver, UUID clientId) { - return getRawStore(storeName, resolver, clientId, null); + return getRawStore(storeName, resolver, clientId, null, null); } @SuppressWarnings("unchecked") public Store getRawStore(String storeName, InconsistencyResolver> resolver, UUID clientId, - String customStoresXml) { + String customStoresXml, + String clusterXmlString) { logger.info("Client zone-id [" + clientZoneId + "] Attempting to obtain metadata for store [" + storeName + "] "); @@ -173,11 +174,17 @@ public Store getRawStore(String storeName, } } // Get cluster and store metadata - String clusterXml = bootstrapMetadataWithRetries(MetadataStore.CLUSTER_KEY, bootstrapUrls); + String clusterXml = clusterXmlString; + if(clusterXml == null) { + logger.debug("*************************** Fetching cluster.xml !!! ******************************************"); + clusterXml = bootstrapMetadataWithRetries(MetadataStore.CLUSTER_KEY, bootstrapUrls); + } Cluster cluster = clusterMapper.readCluster(new StringReader(clusterXml), false); String storesXml = customStoresXml; - if(storesXml == null) + if(storesXml == null) { + logger.debug("*************************** Fetching stores.xml !!! ******************************************"); storesXml = bootstrapMetadataWithRetries(MetadataStore.STORES_KEY, bootstrapUrls); + } if(logger.isDebugEnabled()) { logger.debug("Obtained cluster metadata xml" + clusterXml); diff --git a/src/java/voldemort/client/AsyncMetadataVersionManager.java b/src/java/voldemort/client/AsyncMetadataVersionManager.java index fa0df578c4..02302af01c 100644 --- a/src/java/voldemort/client/AsyncMetadataVersionManager.java +++ b/src/java/voldemort/client/AsyncMetadataVersionManager.java @@ -22,39 +22,48 @@ public class AsyncMetadataVersionManager implements Runnable { + private static final String STORES_VERSION_KEY = "stores.xml"; + private static final String CLUSTER_VERSION_KEY = "cluster.xml"; + private final Logger logger = Logger.getLogger(this.getClass()); - private Versioned currentVersion; - private final SystemStore sysStore; - private final String systemKey = "stores.xml"; + private Versioned currentStoreVersion; + private Versioned currentClusterVersion; private volatile boolean isRunning; private final Callable storeClientThunk; private long asyncMetadataCheckInterval; + private final SystemStoreRepository sysRepository; + private final ClientConfig clientConfig; // Random delta generator - final int DELTA_MAX = 1000; - Random randomGenerator = new Random(System.currentTimeMillis()); + private final int DELTA_MAX = 2000; + private final Random randomGenerator = new Random(System.currentTimeMillis()); - public AsyncMetadataVersionManager(SystemStore systemStore, - long asyncMetadataCheckInterval, + public AsyncMetadataVersionManager(SystemStoreRepository sysRepository, + ClientConfig config, Callable storeClientThunk) { - this(null, systemStore, asyncMetadataCheckInterval, storeClientThunk); - } + this.sysRepository = sysRepository; - public AsyncMetadataVersionManager(Versioned initialVersion, - SystemStore systemStore, - long asyncMetadataCheckInterval, - Callable storeClientThunk) { - this.sysStore = systemStore; - if(initialVersion == null) { - this.currentVersion = sysStore.getSysStore("stores.xml"); - - // If the received store version is null, assume version 0 - if(currentVersion == null) - currentVersion = new Versioned((long) 0); - } else { - currentVersion = initialVersion; + try { + this.currentStoreVersion = this.sysRepository.getVersionStore() + .getSysStore(STORES_VERSION_KEY); + + } catch(Exception e) { + logger.error("Exception while getting currentStoreVersion : " + e); } + try { + this.currentClusterVersion = this.sysRepository.getVersionStore() + .getSysStore(CLUSTER_VERSION_KEY); + } catch(Exception e) { + logger.error("Exception while getting currentClusterVersion : " + e); + } + + // If the received version is null, assume version 0 + if(currentStoreVersion == null) + currentStoreVersion = new Versioned((long) 0); + if(currentClusterVersion == null) + currentClusterVersion = new Versioned((long) 0); + // Initialize and start the background check thread isRunning = true; @@ -69,7 +78,8 @@ public void uncaughtException(Thread t, Throwable e) { }); this.storeClientThunk = storeClientThunk; - this.asyncMetadataCheckInterval = asyncMetadataCheckInterval; + this.asyncMetadataCheckInterval = config.getAsyncCheckMetadataInterval(); + this.clientConfig = config; checkVersionThread.start(); } @@ -78,48 +88,96 @@ public void destroy() { isRunning = false; } + /* + * This method checks for any update in the version for 'versionKey'. If + * there is any change, it returns the new version. Otherwise it will return + * a null. + */ + public Versioned fetchNewVersion(String versionKey, Versioned curVersion) { + try { + Versioned newVersion = this.sysRepository.getVersionStore() + .getSysStore(versionKey); + + // If version obtained is null, the store is untouched. Continue + if(newVersion != null) { + logger.debug("MetadataVersion check => Obtained " + versionKey + " version : " + + newVersion); + + if(!newVersion.equals(curVersion)) { + return newVersion; + } + } else { + logger.debug("Metadata unchanged after creation ..."); + } + } + + // Swallow all exceptions here (we dont want to fail the client). + catch(Exception e) { + logger.info("Could not retrieve Metadata Version. Exception : " + e); + } + + return null; + } + public void run() { + Versioned newStoresVersion, newClusterVersion; while(!Thread.currentThread().isInterrupted() && isRunning) { + newStoresVersion = newClusterVersion = null; + try { Thread.sleep(asyncMetadataCheckInterval); } catch(InterruptedException e) { break; } - Versioned newVersion = this.sysStore.getSysStore(systemKey); + newStoresVersion = fetchNewVersion(STORES_VERSION_KEY, currentStoreVersion); + newClusterVersion = fetchNewVersion(CLUSTER_VERSION_KEY, currentClusterVersion); - // If version obtained is null, the store is untouched. Continue - if(newVersion == null) { - logger.debug("Metadata unchanged after creation ..."); + // If nothing has been updated, continue + if(newStoresVersion == null && newClusterVersion == null) { continue; } - logger.info("MetadataVersion check => Obtained " + systemKey + " version : " - + newVersion); + logger.info("Metadata version mismatch detected."); - if(!newVersion.equals(currentVersion)) { - logger.info("Metadata version mismatch detected."); + // Determine a random delta delay between 0 to DELTA_MAX to sleep + int delta = randomGenerator.nextInt(DELTA_MAX); - // Determine a random delta delay between 0 to 1000 (ms) - int delta = randomGenerator.nextInt(DELTA_MAX); + try { + logger.info("Sleeping for delta : " + delta + " (ms) before re-bootstrapping."); + Thread.sleep(delta); + } catch(InterruptedException e) { + break; + } - try { - logger.info("Sleeping for delta : " + delta + " (ms) before re-bootstrapping."); - Thread.sleep(delta); - } catch(InterruptedException e) { - break; - } + try { + this.storeClientThunk.call(); - // Invoke callback for bootstrap - try { - this.storeClientThunk.call(); - } catch(Exception e) { - e.printStackTrace(); + if(newStoresVersion != null) { + currentStoreVersion = newStoresVersion; } - // Update the current version - currentVersion = newVersion; + if(newClusterVersion != null) { + currentClusterVersion = newClusterVersion; + } + } catch(Exception e) { + e.printStackTrace(); + logger.info(e.getMessage()); } + } } + + public Versioned getStoreMetadataVersion() { + return this.currentStoreVersion; + } + + public Versioned getClusterMetadataVersion() { + return this.currentClusterVersion; + } + + public void updateMetadataVersions() { + this.currentStoreVersion = fetchNewVersion(STORES_VERSION_KEY, null); + this.currentClusterVersion = fetchNewVersion(CLUSTER_VERSION_KEY, null); + } } diff --git a/src/java/voldemort/client/CachingStoreClientFactory.java b/src/java/voldemort/client/CachingStoreClientFactory.java index dc8f94ef8f..444441e75c 100644 --- a/src/java/voldemort/client/CachingStoreClientFactory.java +++ b/src/java/voldemort/client/CachingStoreClientFactory.java @@ -127,7 +127,8 @@ else if(client instanceof LazyStoreClient) { public Store getRawStore(String storeName, InconsistencyResolver> resolver, UUID clientId, - String storesXml) { - return inner.getRawStore(storeName, resolver, clientId, storesXml); + String storesXml, + String clusterXml) { + return inner.getRawStore(storeName, resolver, clientId, storesXml, clusterXml); } } diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index 870f29f863..89731723ec 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -16,7 +16,6 @@ package voldemort.client; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.UUID; @@ -27,6 +26,7 @@ import voldemort.VoldemortException; import voldemort.annotations.concurrency.Threadsafe; +import voldemort.annotations.jmx.JmxGetter; import voldemort.annotations.jmx.JmxManaged; import voldemort.annotations.jmx.JmxOperation; import voldemort.cluster.Node; @@ -35,6 +35,7 @@ import voldemort.store.InvalidMetadataException; import voldemort.store.Store; import voldemort.store.StoreCapabilityType; +import voldemort.store.metadata.MetadataStore; import voldemort.store.system.SystemStoreConstants; import voldemort.utils.JmxUtils; import voldemort.utils.ManifestFileReader; @@ -68,12 +69,12 @@ public class DefaultStoreClient implements StoreClient { private final int metadataRefreshAttempts; private final String storeName; private final InconsistencyResolver> resolver; - private volatile Store store; + private final SystemStoreRepository sysRepository; private final UUID clientId; - - private final Map sysStoreMap; - private AsyncMetadataVersionManager asyncCheckMetadata; + private volatile Store store; + private AsyncMetadataVersionManager asyncCheckMetadata = null; private ClientInfo clientInfo; + private String clusterXml; public DefaultStoreClient(String storeName, InconsistencyResolver> resolver, @@ -82,7 +83,6 @@ public DefaultStoreClient(String storeName, this(storeName, resolver, storeFactory, maxMetadataRefreshAttempts, null, 0, null); } - @SuppressWarnings("unchecked") public DefaultStoreClient(String storeName, InconsistencyResolver> resolver, StoreClientFactory storeFactory, @@ -102,6 +102,7 @@ public DefaultStoreClient(String storeName, ManifestFileReader.getReleaseVersion()); this.clientId = AbstractStoreClientFactory.generateClientId(clientInfo); this.config = config; + this.sysRepository = new SystemStoreRepository(); // Registering self to be able to bootstrap client dynamically via JMX JmxUtils.registerMbean(this, @@ -110,10 +111,9 @@ public DefaultStoreClient(String storeName, + "." + clientContext + "." + storeName + "." + clientId.toString())); - bootStrap(); - // Initialize all the system stores - sysStoreMap = createSystemStores(); + // Bootstrap this client + bootStrap(); // Initialize the background thread for checking metadata version if(config != null) { @@ -125,8 +125,7 @@ public DefaultStoreClient(String storeName, } private void registerClient() { - String name = SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(); - SystemStore clientRegistry = sysStoreMap.get(name); + SystemStore clientRegistry = this.sysRepository.getClientRegistryStore(); if(null != clientRegistry) { try { clientRegistry.putSysStore(clientId.toString(), clientInfo); @@ -134,28 +133,20 @@ private void registerClient() { logger.warn("Unable to register with the cluster due to the following error:", e); } } else { - logger.warn(name + "not found. Unable to registry with voldemort cluster."); + logger.warn(SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name() + + "not found. Unable to registry with voldemort cluster."); } } - private Map createSystemStores() { - Map systemStores = new HashMap(); - for(SystemStoreConstants.SystemStoreName storeName: SystemStoreConstants.SystemStoreName.values()) { - SystemStore sysStore = new SystemStore(storeName.name(), - config.getBootstrapUrls(), - config.getClientZoneId()); - systemStores.put(storeName.name(), sysStore); - } - return systemStores; - } - private AsyncMetadataVersionManager createMetadataChecker() { AsyncMetadataVersionManager asyncCheckMetadata = null; - SystemStore versionStore = this.sysStoreMap.get(SystemStoreConstants.SystemStoreName.voldsys$_metadata_version.name()); + SystemStore versionStore = this.sysRepository.getVersionStore(); if(versionStore == null) logger.warn("Metadata version system store not found. Cannot run Metadata version check thread."); else { - Callable bootstrapCallback = new Callable() { + + // Create a callback for re-bootstrapping the client + Callable rebootstrapCallback = new Callable() { public Void call() throws Exception { bootStrap(); @@ -163,9 +154,9 @@ public Void call() throws Exception { } }; - asyncCheckMetadata = new AsyncMetadataVersionManager(versionStore, - config.getAsyncCheckMetadataInterval(), - bootstrapCallback); + asyncCheckMetadata = new AsyncMetadataVersionManager(this.sysRepository, + config, + rebootstrapCallback); logger.info("Metadata version check thread started. Frequency = Every " + config.getAsyncCheckMetadataInterval() + " ms"); } @@ -175,7 +166,31 @@ public Void call() throws Exception { @JmxOperation(description = "bootstrap metadata from the cluster.") public void bootStrap() { logger.info("Bootstrapping metadata for store " + this.storeName); - this.store = storeFactory.getRawStore(storeName, resolver, clientId); + + /* + * Since we need cluster.xml for bootstrapping this client as well as + * all the System stores, just fetch it once and pass it around. + * + * Seems hackish since bootstrapMetadataWithRetries only exists for + * AbstractStoreClientFactory. TODO: Think about making this part of the + * generic interface ? + */ + clusterXml = ((AbstractStoreClientFactory) storeFactory).bootstrapMetadataWithRetries(MetadataStore.CLUSTER_KEY); + + this.store = storeFactory.getRawStore(storeName, resolver, clientId, null, clusterXml); + + // Create system stores + logger.info("Creating system stores for store " + this.storeName); + this.sysRepository.createSystemStores(this.config, this.clusterXml); + + /* + * Update to the new metadata versions (in case we got here from Invalid + * Metadata exception). This will prevent another bootstrap via the + * Async metadata checker + */ + if(asyncCheckMetadata != null) { + asyncCheckMetadata.updateMetadataVersions(); + } } public boolean delete(K key) { @@ -449,4 +464,18 @@ else if(versions.size() == 1) public UUID getClientId() { return clientId; } + + @JmxGetter(name = "getStoreMetadataVersion") + public String getStoreMetadataVersion() { + String result = "Current Store Metadata Version : " + + this.asyncCheckMetadata.getStoreMetadataVersion(); + return result; + } + + @JmxGetter(name = "getClusterMetadataVersion") + public String getClusterMetadataVersion() { + String result = "Current Cluster Metadata Version : " + + this.asyncCheckMetadata.getClusterMetadataVersion(); + return result; + } } diff --git a/src/java/voldemort/client/LazyStoreClient.java b/src/java/voldemort/client/LazyStoreClient.java index 3b12468f4d..d6f72eb722 100644 --- a/src/java/voldemort/client/LazyStoreClient.java +++ b/src/java/voldemort/client/LazyStoreClient.java @@ -20,6 +20,8 @@ import java.util.Map; import java.util.concurrent.Callable; +import org.apache.log4j.Logger; + import voldemort.VoldemortException; import voldemort.cluster.Node; import voldemort.versioning.ObsoleteVersionException; @@ -34,11 +36,25 @@ */ public class LazyStoreClient implements StoreClient { + private final Logger logger = Logger.getLogger(LazyStoreClient.class); private final Callable> storeClientThunk; private StoreClient storeClient; public LazyStoreClient(Callable> storeClientThunk) { this.storeClientThunk = storeClientThunk; + + /* + * Although it says Lazy Store Client, we try to bootstrap during + * initialization if we can. If the server isn't up at this time, it + * will be done lazily during the next client API call. + */ + try { + storeClient = initStoreClient(); + } catch(Exception e) { + storeClient = null; + e.printStackTrace(); + logger.info("Could not bootstrap right away. Trying on the next call ... "); + } } public synchronized StoreClient getStoreClient() { diff --git a/src/java/voldemort/client/MockStoreClientFactory.java b/src/java/voldemort/client/MockStoreClientFactory.java index 0cf6c4a8f8..b50f130d3b 100644 --- a/src/java/voldemort/client/MockStoreClientFactory.java +++ b/src/java/voldemort/client/MockStoreClientFactory.java @@ -216,7 +216,8 @@ public FailureDetector getFailureDetector() { public Store getRawStore(String storeName, InconsistencyResolver> resolver, UUID clientId, - String storesXml) { + String storesXml, + String clusterXml) { return null; } diff --git a/src/java/voldemort/client/SocketStoreClientFactory.java b/src/java/voldemort/client/SocketStoreClientFactory.java index fe10a1e64a..4447891e8e 100644 --- a/src/java/voldemort/client/SocketStoreClientFactory.java +++ b/src/java/voldemort/client/SocketStoreClientFactory.java @@ -183,7 +183,11 @@ public void close() { super.close(); } - public Store getSystemStore(String storeName) { - return getRawStore(storeName, null, null, SystemStoreConstants.SYSTEM_STORE_SCHEMA); + public Store getSystemStore(String storeName, String clusterXml) { + return getRawStore(storeName, + null, + null, + SystemStoreConstants.SYSTEM_STORE_SCHEMA, + clusterXml); } } diff --git a/src/java/voldemort/client/StoreClientFactory.java b/src/java/voldemort/client/StoreClientFactory.java index 51bdc3dcaa..5cff508ffa 100644 --- a/src/java/voldemort/client/StoreClientFactory.java +++ b/src/java/voldemort/client/StoreClientFactory.java @@ -105,7 +105,8 @@ Store getRawStore(String storeName, Store getRawStore(String storeName, InconsistencyResolver> resolver, UUID clientId, - String customStoresXml); + String customStoresXml, + String clusterXmlString); /** <<<<<<< HEAD diff --git a/src/java/voldemort/client/SystemStore.java b/src/java/voldemort/client/SystemStore.java index 0e68529cf4..f8ec3bd2e3 100644 --- a/src/java/voldemort/client/SystemStore.java +++ b/src/java/voldemort/client/SystemStore.java @@ -14,12 +14,16 @@ public class SystemStore { - private final Logger logger = Logger.getLogger(DefaultStoreClient.class); + private final Logger logger = Logger.getLogger(SystemStore.class); private final SocketStoreClientFactory systemStoreFactory; private final String storeName; private volatile Store sysStore; public SystemStore(String storeName, String[] bootstrapUrls, int clientZoneID) { + this(storeName, bootstrapUrls, clientZoneID, null); + } + + public SystemStore(String storeName, String[] bootstrapUrls, int clientZoneID, String clusterXml) { String prefix = storeName.substring(0, SystemStoreConstants.NAME_PREFIX.length()); if(!SystemStoreConstants.NAME_PREFIX.equals(prefix)) throw new VoldemortException("Illegal system store : " + storeName); @@ -36,7 +40,7 @@ public SystemStore(String storeName, String[] bootstrapUrls, int clientZoneID) { .setClientZoneId(clientZoneID); this.systemStoreFactory = new SocketStoreClientFactory(config); this.storeName = storeName; - this.sysStore = this.systemStoreFactory.getSystemStore(this.storeName); + this.sysStore = this.systemStoreFactory.getSystemStore(this.storeName, clusterXml); } public void putSysStore(K key, V value) { diff --git a/src/java/voldemort/client/SystemStoreRepository.java b/src/java/voldemort/client/SystemStoreRepository.java new file mode 100644 index 0000000000..9a6a020dd1 --- /dev/null +++ b/src/java/voldemort/client/SystemStoreRepository.java @@ -0,0 +1,42 @@ +package voldemort.client; + +import java.util.concurrent.ConcurrentHashMap; + +import voldemort.store.system.SystemStoreConstants; + +/* + * A repository that creates and maintains all the system stores in one place. + * The purpose is to act as a source of truth for all the system stores, since + * they can be recreated dynamically (in case cluster.xml changes). + */ + +public class SystemStoreRepository { + + private ConcurrentHashMap sysStoreMap; + + public SystemStoreRepository() { + sysStoreMap = new ConcurrentHashMap(); + } + + public void createSystemStores(ClientConfig config, String clusterXml) { + for(SystemStoreConstants.SystemStoreName storeName: SystemStoreConstants.SystemStoreName.values()) { + SystemStore sysVersionStore = new SystemStore(storeName.name(), + config.getBootstrapUrls(), + config.getClientZoneId(), + clusterXml); + this.sysStoreMap.put(storeName.name(), sysVersionStore); + } + } + + public SystemStore getVersionStore() { + String name = SystemStoreConstants.SystemStoreName.voldsys$_metadata_version.name(); + SystemStore sysVersionStore = sysStoreMap.get(name); + return sysVersionStore; + } + + public SystemStore getClientRegistryStore() { + String name = SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(); + SystemStore sysRegistryStore = sysStoreMap.get(name); + return sysRegistryStore; + } +} diff --git a/test/common/voldemort/StaticStoreClientFactory.java b/test/common/voldemort/StaticStoreClientFactory.java index 913b75b188..3d48f24b06 100644 --- a/test/common/voldemort/StaticStoreClientFactory.java +++ b/test/common/voldemort/StaticStoreClientFactory.java @@ -77,7 +77,8 @@ public FailureDetector getFailureDetector() { public Store getRawStore(String storeName, InconsistencyResolver> resolver, UUID clientId, - String storesXml) { + String storesXml, + String clusterXml) { return null; } diff --git a/test/common/voldemort/config/stores.xml b/test/common/voldemort/config/stores.xml index 116726bfd4..8613533a1a 100644 --- a/test/common/voldemort/config/stores.xml +++ b/test/common/voldemort/config/stores.xml @@ -186,8 +186,13 @@ 10 +<<<<<<< HEAD test-store-eventual-1 bdb +======= + test-basic-replication-memory + memory +>>>>>>> 1e726e7... Updated async metadata checker for both stores and cluster.xml. Updated DefaultSocketStoreClientTest. Moved system stores repository to a new class. client 2 1 @@ -197,6 +202,7 @@ UTF-8 +<<<<<<< HEAD java-serialization @@ -213,6 +219,10 @@ java-serialization +======= + string + UTF-8 +>>>>>>> 1e726e7... Updated async metadata checker for both stores and cluster.xml. Updated DefaultSocketStoreClientTest. Moved system stores repository to a new class. diff --git a/test/unit/voldemort/client/DefaultSocketStoreClientTest.java b/test/unit/voldemort/client/DefaultSocketStoreClientTest.java index 6d2bc94d2e..88ab067752 100644 --- a/test/unit/voldemort/client/DefaultSocketStoreClientTest.java +++ b/test/unit/voldemort/client/DefaultSocketStoreClientTest.java @@ -14,12 +14,14 @@ import voldemort.ServerTestUtils; import voldemort.TestUtils; +import voldemort.VoldemortException; import voldemort.client.protocol.admin.AdminClient; import voldemort.cluster.Cluster; import voldemort.server.VoldemortServer; import voldemort.store.StoreDefinition; import voldemort.store.socket.SocketStoreFactory; import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; +import voldemort.utils.SystemTime; import voldemort.utils.Time; import voldemort.versioning.ObsoleteVersionException; import voldemort.versioning.VectorClock; @@ -28,7 +30,7 @@ public class DefaultSocketStoreClientTest { - private static String testStoreName = "test-replication-memory"; + private static String testStoreName = "test-basic-replication-memory"; private static String storesXmlfile = "test/common/voldemort/config/stores.xml"; private SocketStoreFactory socketStoreFactory = new ClientRequestExecutorPool(2, 10000, @@ -40,10 +42,35 @@ public class DefaultSocketStoreClientTest { private Cluster cluster; private AdminClient adminClient; - public static String socketUrl = "tcp://localhost:6667"; + public static String socketUrl = ""; protected StoreClient client; protected int nodeId; protected Time time; + private boolean success = true; + + private class ParallelClient extends Thread { + + private final StoreClient parallelClient; + + public ParallelClient(StoreClient client) { + this.parallelClient = client; + } + + @Override + public void run() { + System.out.println("Doing a whole bunch of puts"); + try { + for(int i = 0; i < 10000; i++) + this.parallelClient.put("my-key" + i, "my-value" + i); + } catch(VoldemortException ve) { + if(!(ve instanceof ObsoleteVersionException)) { + success = false; + ve.printStackTrace(); + fail("Exception occured while doing a put."); + } + } + } + } @Before public void setUp() throws Exception { @@ -52,6 +79,7 @@ public void setUp() throws Exception { servers = new VoldemortServer[2]; storeDefs = new StoreDefinitionsMapper().readStoreList(new File(storesXmlfile)); + boolean useNio = true; servers[0] = ServerTestUtils.startVoldemortServer(socketStoreFactory, ServerTestUtils.createServerConfig(true, 0, @@ -72,16 +100,24 @@ public void setUp() throws Exception { cluster); adminClient = ServerTestUtils.getAdminClient(cluster); + socketUrl = servers[0].getIdentityNode().getSocketUrl().toString(); + + ClientConfig clientConfig = new ClientConfig().setMaxTotalConnections(4) + .setMaxConnectionsPerNode(4) + .setBootstrapUrls(socketUrl); + + SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); + this.client = socketFactory.getStoreClient(testStoreName); + this.nodeId = 0; + this.time = SystemTime.INSTANCE; } - @Test + // @Test public void test() { + client.delete("k"); client.put("k", Versioned.value("v")); Versioned v = client.get("k"); assertEquals("GET should return the version set by PUT.", "v", v.getValue()); - VectorClock expected = new VectorClock(); - expected.incrementVersion(nodeId, time.getMilliseconds()); - assertEquals("The version should be incremented after a put.", expected, v.getVersion()); try { client.put("k", Versioned.value("v")); fail("Put of obsolete version should throw exception."); @@ -94,26 +130,45 @@ public void test() { new VectorClock().incremented(nodeId + 1, time.getMilliseconds()))); assertEquals("GET should return the new value set by PUT.", "v2", client.getValue("k")); - assertEquals("GET should return the new version set by PUT.", - expected.incremented(nodeId + 1, time.getMilliseconds()), - client.get("k").getVersion()); client.delete("k"); assertNull("After a successful delete(k), get(k) should return null.", client.get("k")); } - @Test - public void testClientRegistryHappyPath() { - ClientConfig clientConfig = new ClientConfig().setMaxThreads(4) - .setMaxTotalConnections(4) - .setMaxConnectionsPerNode(4) - .setBootstrapUrls(socketUrl) - .setClientContextName("testClientRegistryHappyPath"); - SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); - this.client = socketFactory.getStoreClient(testStoreName); - client.put("k", "v"); - adminClient.fetchEntries(0, testStoreName, null, null, false); - adminClient.fetchEntries(1, testStoreName, null, null, false); - // TODO: verify that the values in registry are correct. + // @Test + public void testBatchOps() { + success = true; + + for(int j = 0; j < 10; j++) { + (new ParallelClient(client)).start(); + } + + try { + Thread.sleep(10000); + } catch(Exception e) {} + assertEquals(success, true); } + /* + * Test that a Node failure does not affect system stores. + */ + @Test + public void testSystemStoreFailures() { + success = true; + for(int j = 0; j < 100; j++) { + (new ParallelClient(client)).start(); + } + + servers[1].stop(); + try { + Thread.sleep(10000); + } catch(Exception e) {} + + assertEquals(success, true); + try { + servers[1].start(); + } catch(Exception e) { + e.printStackTrace(); + fail("Cannot restart Voldemort Server"); + } + } } From cb23d9fd32befddcbe4ddc0afced416007a78c19 Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Fri, 29 Jun 2012 12:09:55 -0700 Subject: [PATCH 083/209] resolve merge conflict in stores.xml --- test/common/voldemort/config/stores.xml | 29 +++++++++++++++---------- 1 file changed, 18 insertions(+), 11 deletions(-) diff --git a/test/common/voldemort/config/stores.xml b/test/common/voldemort/config/stores.xml index 8613533a1a..3dba316a34 100644 --- a/test/common/voldemort/config/stores.xml +++ b/test/common/voldemort/config/stores.xml @@ -185,14 +185,25 @@ all-handoff 10 + -<<<<<<< HEAD test-store-eventual-1 bdb -======= + client + 2 + 1 + 1 + + string + UTF-8 + + + java-serialization + + + test-basic-replication-memory memory ->>>>>>> 1e726e7... Updated async metadata checker for both stores and cluster.xml. Updated DefaultSocketStoreClientTest. Moved system stores repository to a new class. client 2 1 @@ -201,11 +212,11 @@ string UTF-8 - -<<<<<<< HEAD - java-serialization + + string + UTF-8 - + test-store-eventual-2 bdb @@ -219,10 +230,6 @@ java-serialization -======= - string - UTF-8 ->>>>>>> 1e726e7... Updated async metadata checker for both stores and cluster.xml. Updated DefaultSocketStoreClientTest. Moved system stores repository to a new class. From 9a582b493b6ce371ee45822339adc9bee867056f Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Fri, 29 Jun 2012 14:13:55 -0700 Subject: [PATCH 084/209] allows fetch-keys to fetch keys from system stores --- src/java/voldemort/VoldemortAdminTool.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/java/voldemort/VoldemortAdminTool.java b/src/java/voldemort/VoldemortAdminTool.java index bc7384feaf..7aec774c57 100644 --- a/src/java/voldemort/VoldemortAdminTool.java +++ b/src/java/voldemort/VoldemortAdminTool.java @@ -800,9 +800,8 @@ private static void executeSetMetadata(Integer nodeId, + adminClient.getAdminClientCluster() .getNodeById(currentNodeId) .getId()); - adminClient.updateRemoteMetadata(currentNodeId, - key, - Versioned.value(value.toString(), updatedVersion)); + adminClient.updateRemoteMetadata(currentNodeId, key, Versioned.value(value.toString(), + updatedVersion)); } } @@ -1251,6 +1250,10 @@ private static void executeFetchKeys(Integer nodeId, if(stores == null) { stores = Lists.newArrayList(); stores.addAll(storeDefinitionMap.keySet()); + } else { + // add system store to the map so they can be fetched when specified + // explicitly + storeDefinitionMap.putAll(getSystemStoreDef()); } // Pick up all the partitions From 20b47b795d5a0fadfb733d8decb3be769a10489a Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Mon, 2 Jul 2012 20:45:24 -0700 Subject: [PATCH 085/209] add SchedulerService in voldemort client --- src/java/voldemort/client/ClientConfig.java | 10 +- .../voldemort/client/DefaultStoreClient.java | 59 ++++++++--- src/java/voldemort/client/SystemStore.java | 11 ++- .../voldemort/client/VoldemortOperation.java | 14 --- .../AsyncMetadataVersionManager.java | 43 ++------ .../scheduler/ClientRegistryRefresher.java | 43 ++++++++ .../service}/AbstractService.java | 2 +- .../service}/SchedulerService.java | 5 +- .../service}/ServiceType.java | 2 +- .../service}/VoldemortService.java | 3 +- .../server/AbstractSocketService.java | 3 + .../server/SystemStoreConstants.java | 77 --------------- .../voldemort/server/VoldemortServer.java | 5 +- .../server/gossip/GossipService.java | 6 +- .../voldemort/server/http/HttpService.java | 4 +- .../voldemort/server/http/StoreServlet.java | 2 +- .../gui/ReadOnlyStoreManagementServlet.java | 2 +- .../server/http/gui/StatusServlet.java | 2 +- src/java/voldemort/server/jmx/JmxService.java | 6 +- .../server/niosocket/NioSocketService.java | 2 +- .../protocol/admin/AsyncOperationService.java | 6 +- .../server/rebalance/RebalancerService.java | 6 +- .../server/socket/SocketService.java | 2 +- .../server/storage/StorageService.java | 6 +- .../voldemort/client/ClientRegistryTest.java | 91 +++++++++++------ .../client/DefaultSocketStoreClientTest.java | 99 +++++-------------- .../ServerStoreVerifierTest.java | 2 +- test/unit/voldemort/server/ServiceTest.java | 2 + .../protocol/admin/AsyncOperationTest.java | 2 +- .../server/storage/StorageServiceTest.java | 2 +- 30 files changed, 238 insertions(+), 281 deletions(-) delete mode 100644 src/java/voldemort/client/VoldemortOperation.java rename src/java/voldemort/client/{ => scheduler}/AsyncMetadataVersionManager.java (81%) create mode 100644 src/java/voldemort/client/scheduler/ClientRegistryRefresher.java rename src/java/voldemort/{server => common/service}/AbstractService.java (98%) rename src/java/voldemort/{server/scheduler => common/service}/SchedulerService.java (97%) rename src/java/voldemort/{server => common/service}/ServiceType.java (94%) rename src/java/voldemort/{server => common/service}/VoldemortService.java (96%) delete mode 100644 src/java/voldemort/server/SystemStoreConstants.java diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index 9bda12e4cb..8d8e27d755 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -79,9 +79,11 @@ public class ClientConfig { private volatile int maxBootstrapRetries = 2; private volatile String clientContextName = "default"; + + /* 5 second check interval, in ms */ private volatile long asyncCheckMetadataInterval = 5000; - /* 12 hr refresh internval */ - private volatile long clientRegistryRefreshInterval = 3600 * 1000 * 12; + /* 12 hr refresh internval, in seconds */ + private volatile int clientRegistryRefreshInterval = 3600 * 12; public ClientConfig() {} @@ -724,11 +726,11 @@ public ClientConfig setAsyncCheckMetadataInterval(long asyncCheckMetadataInterva return this; } - public long getClientRegistryRefreshInterval() { + public int getClientRegistryRefreshInterval() { return this.clientRegistryRefreshInterval; } - public ClientConfig setClientRegistryRefreshInterval(long clientRegistryRefrshInterval) { + public ClientConfig setClientRegistryRefreshInterval(int clientRegistryRefrshInterval) { this.clientRegistryRefreshInterval = clientRegistryRefrshInterval; return this; } diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index 89731723ec..ccd8749890 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -16,10 +16,13 @@ package voldemort.client; +import java.util.Calendar; +import java.util.Date; +import java.util.GregorianCalendar; import java.util.List; import java.util.Map; -import java.util.UUID; import java.util.Map.Entry; +import java.util.UUID; import java.util.concurrent.Callable; import org.apache.log4j.Logger; @@ -29,7 +32,10 @@ import voldemort.annotations.jmx.JmxGetter; import voldemort.annotations.jmx.JmxManaged; import voldemort.annotations.jmx.JmxOperation; +import voldemort.client.scheduler.AsyncMetadataVersionManager; +import voldemort.client.scheduler.ClientRegistryRefresher; import voldemort.cluster.Node; +import voldemort.common.service.SchedulerService; import voldemort.routing.RoutingStrategy; import voldemort.serialization.Serializer; import voldemort.store.InvalidMetadataException; @@ -39,6 +45,7 @@ import voldemort.store.system.SystemStoreConstants; import voldemort.utils.JmxUtils; import voldemort.utils.ManifestFileReader; +import voldemort.utils.SystemTime; import voldemort.utils.Utils; import voldemort.versioning.InconsistencyResolver; import voldemort.versioning.InconsistentDataException; @@ -62,6 +69,9 @@ @JmxManaged(description = "A voldemort client") public class DefaultStoreClient implements StoreClient { + private static final int ASYNC_THREADS_COUNT = 2; + private static final boolean ALLOW_INTERRUPT_ASYNC = true; + private final Logger logger = Logger.getLogger(DefaultStoreClient.class); private final StoreClientFactory storeFactory; @@ -72,9 +82,10 @@ public class DefaultStoreClient implements StoreClient { private final SystemStoreRepository sysRepository; private final UUID clientId; private volatile Store store; - private AsyncMetadataVersionManager asyncCheckMetadata = null; + private final SchedulerService scheduler; private ClientInfo clientInfo; private String clusterXml; + private AsyncMetadataVersionManager asyncCheckMetadata = null; public DefaultStoreClient(String storeName, InconsistencyResolver> resolver, @@ -103,7 +114,9 @@ public DefaultStoreClient(String storeName, this.clientId = AbstractStoreClientFactory.generateClientId(clientInfo); this.config = config; this.sysRepository = new SystemStoreRepository(); - + this.scheduler = new SchedulerService(ASYNC_THREADS_COUNT, + SystemTime.INSTANCE, + ALLOW_INTERRUPT_ASYNC); // Registering self to be able to bootstrap client dynamically via JMX JmxUtils.registerMbean(this, JmxUtils.createObjectName(JmxUtils.getPackageName(this.getClass()), @@ -117,18 +130,31 @@ public DefaultStoreClient(String storeName, // Initialize the background thread for checking metadata version if(config != null) { - asyncCheckMetadata = createMetadataChecker(); + asyncCheckMetadata = scheduleMetadataChecker(clientId.toString(), + config.getAsyncCheckMetadataInterval()); } - registerClient(); + registerClient(clientId.toString(), config.getClientRegistryRefreshInterval()); logger.info("Voldemort client created: " + clientId.toString() + "\n" + clientInfo); } - private void registerClient() { + private void registerClient(String jobId, int interval) { SystemStore clientRegistry = this.sysRepository.getClientRegistryStore(); if(null != clientRegistry) { try { - clientRegistry.putSysStore(clientId.toString(), clientInfo); + Version version = clientRegistry.putSysStore(clientId.toString(), clientInfo); + ClientRegistryRefresher refresher = new ClientRegistryRefresher(clientRegistry, + clientId.toString(), + clientInfo, + version); + GregorianCalendar cal = new GregorianCalendar(); + cal.add(Calendar.SECOND, interval); + scheduler.schedule(jobId + refresher.getClass().getName(), + refresher, + cal.getTime(), + interval * 1000); + logger.info("Client registry refresher thread started, refresh frequency: " + + interval + " seconds"); } catch(Exception e) { logger.warn("Unable to register with the cluster due to the following error:", e); } @@ -138,7 +164,7 @@ private void registerClient() { } } - private AsyncMetadataVersionManager createMetadataChecker() { + private AsyncMetadataVersionManager scheduleMetadataChecker(String jobId, long interval) { AsyncMetadataVersionManager asyncCheckMetadata = null; SystemStore versionStore = this.sysRepository.getVersionStore(); if(versionStore == null) @@ -155,10 +181,17 @@ public Void call() throws Exception { }; asyncCheckMetadata = new AsyncMetadataVersionManager(this.sysRepository, - config, rebootstrapCallback); - logger.info("Metadata version check thread started. Frequency = Every " - + config.getAsyncCheckMetadataInterval() + " ms"); + + // schedule the job to run every 'checkInterval' period, starting + // now + scheduler.schedule(jobId + asyncCheckMetadata.getClass().getName(), + asyncCheckMetadata, + new Date(), + interval); + logger.info("Metadata version check thread started. Frequency = Every " + interval + + " ms"); + } return asyncCheckMetadata; } @@ -193,6 +226,10 @@ public void bootStrap() { } } + public void close() { + scheduler.stopInner(); + } + public boolean delete(K key) { Versioned versioned = get(key); if(versioned == null) diff --git a/src/java/voldemort/client/SystemStore.java b/src/java/voldemort/client/SystemStore.java index f8ec3bd2e3..66cf23d8f6 100644 --- a/src/java/voldemort/client/SystemStore.java +++ b/src/java/voldemort/client/SystemStore.java @@ -10,6 +10,7 @@ import voldemort.store.system.SystemStoreConstants; import voldemort.versioning.InconsistentDataException; import voldemort.versioning.VectorClock; +import voldemort.versioning.Version; import voldemort.versioning.Versioned; public class SystemStore { @@ -43,7 +44,8 @@ public SystemStore(String storeName, String[] bootstrapUrls, int clientZoneID, S this.sysStore = this.systemStoreFactory.getSystemStore(this.storeName, clusterXml); } - public void putSysStore(K key, V value) { + public Version putSysStore(K key, V value) { + Version version = null; try { logger.debug("Invoking Put for key : " + key + " on store name : " + this.storeName); Versioned versioned = getSysStore(key); @@ -52,20 +54,25 @@ public void putSysStore(K key, V value) { else versioned.setObject(value); this.sysStore.put(key, versioned, null); + version = versioned.getVersion(); } catch(Exception e) { logger.info("Exception caught during putSysStore:"); e.printStackTrace(); } + return version; } - public void putSysStore(K key, Versioned value) { + public Version putSysStore(K key, Versioned value) { + Version version = null; try { logger.debug("Invoking Put for key : " + key + " on store name : " + this.storeName); this.sysStore.put(key, value, null); + version = value.getVersion(); } catch(Exception e) { logger.info("Exception caught during putSysStore:"); e.printStackTrace(); } + return version; } public Versioned getSysStore(K key) { diff --git a/src/java/voldemort/client/VoldemortOperation.java b/src/java/voldemort/client/VoldemortOperation.java deleted file mode 100644 index 3c508ea351..0000000000 --- a/src/java/voldemort/client/VoldemortOperation.java +++ /dev/null @@ -1,14 +0,0 @@ -package voldemort.client; - -/** - * - * Set of operations supported by Voldemort. - * - */ -public enum VoldemortOperation { - GET, - PUT, - GETALL, - DELETE, - GETVERSIONS -} diff --git a/src/java/voldemort/client/AsyncMetadataVersionManager.java b/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java similarity index 81% rename from src/java/voldemort/client/AsyncMetadataVersionManager.java rename to src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java index 02302af01c..3b8d6f027b 100644 --- a/src/java/voldemort/client/AsyncMetadataVersionManager.java +++ b/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java @@ -1,4 +1,4 @@ -package voldemort.client; +package voldemort.client.scheduler; import java.lang.Thread.UncaughtExceptionHandler; import java.util.Random; @@ -7,6 +7,7 @@ import org.apache.log4j.Level; import org.apache.log4j.Logger; +import voldemort.client.SystemStoreRepository; import voldemort.versioning.Versioned; /* @@ -28,18 +29,14 @@ public class AsyncMetadataVersionManager implements Runnable { private final Logger logger = Logger.getLogger(this.getClass()); private Versioned currentStoreVersion; private Versioned currentClusterVersion; - private volatile boolean isRunning; private final Callable storeClientThunk; - private long asyncMetadataCheckInterval; private final SystemStoreRepository sysRepository; - private final ClientConfig clientConfig; // Random delta generator private final int DELTA_MAX = 2000; private final Random randomGenerator = new Random(System.currentTimeMillis()); public AsyncMetadataVersionManager(SystemStoreRepository sysRepository, - ClientConfig config, Callable storeClientThunk) { this.sysRepository = sysRepository; @@ -64,9 +61,6 @@ public AsyncMetadataVersionManager(SystemStoreRepository sysRepository, if(currentClusterVersion == null) currentClusterVersion = new Versioned((long) 0); - // Initialize and start the background check thread - isRunning = true; - Thread checkVersionThread = new Thread(this, "AsyncVersionCheckThread"); checkVersionThread.setDaemon(true); checkVersionThread.setUncaughtExceptionHandler(new UncaughtExceptionHandler() { @@ -78,14 +72,6 @@ public void uncaughtException(Thread t, Throwable e) { }); this.storeClientThunk = storeClientThunk; - this.asyncMetadataCheckInterval = config.getAsyncCheckMetadataInterval(); - this.clientConfig = config; - checkVersionThread.start(); - - } - - public void destroy() { - isRunning = false; } /* @@ -120,23 +106,12 @@ public Versioned fetchNewVersion(String versionKey, Versioned curVer } public void run() { - Versioned newStoresVersion, newClusterVersion; - while(!Thread.currentThread().isInterrupted() && isRunning) { - newStoresVersion = newClusterVersion = null; - - try { - Thread.sleep(asyncMetadataCheckInterval); - } catch(InterruptedException e) { - break; - } - - newStoresVersion = fetchNewVersion(STORES_VERSION_KEY, currentStoreVersion); - newClusterVersion = fetchNewVersion(CLUSTER_VERSION_KEY, currentClusterVersion); + Versioned newStoresVersion = fetchNewVersion(STORES_VERSION_KEY, currentStoreVersion); + Versioned newClusterVersion = fetchNewVersion(CLUSTER_VERSION_KEY, + currentClusterVersion); - // If nothing has been updated, continue - if(newStoresVersion == null && newClusterVersion == null) { - continue; - } + // If nothing has been updated, continue + if((newStoresVersion != null) || (newClusterVersion != null)) { logger.info("Metadata version mismatch detected."); @@ -147,7 +122,7 @@ public void run() { logger.info("Sleeping for delta : " + delta + " (ms) before re-bootstrapping."); Thread.sleep(delta); } catch(InterruptedException e) { - break; + // do nothing, continue. } try { @@ -164,8 +139,8 @@ public void run() { e.printStackTrace(); logger.info(e.getMessage()); } - } + } public Versioned getStoreMetadataVersion() { diff --git a/src/java/voldemort/client/scheduler/ClientRegistryRefresher.java b/src/java/voldemort/client/scheduler/ClientRegistryRefresher.java new file mode 100644 index 0000000000..2b57926b49 --- /dev/null +++ b/src/java/voldemort/client/scheduler/ClientRegistryRefresher.java @@ -0,0 +1,43 @@ +package voldemort.client.scheduler; + +import org.apache.log4j.Logger; + +import voldemort.client.ClientInfo; +import voldemort.client.SystemStore; +import voldemort.versioning.Version; +import voldemort.versioning.Versioned; + +public class ClientRegistryRefresher implements Runnable { + + private final Logger logger = Logger.getLogger(this.getClass()); + + private final SystemStore clientRegistry; + private ClientInfo clientInfo; + private final String clientId; + private Version lastVersion; + + public ClientRegistryRefresher(SystemStore clientRegistry, + String clientId, + ClientInfo clientInfo, + Version version) { + this.clientRegistry = clientRegistry; + this.clientInfo = clientInfo; + this.clientId = clientId; + this.lastVersion = version; + logger.info("Initial version obtained from client registry: " + version); + } + + public void run() { + clientInfo.setUpdateTime(System.currentTimeMillis()); + logger.info("updating client registry with the following info for client: " + clientId + + "\n" + clientInfo); + try { + lastVersion = clientRegistry.putSysStore(clientId, + new Versioned(clientInfo, + lastVersion)); + } catch(Exception e) { + logger.warn("encounted the following error while trying to update client registry: " + + e); + } + } +} diff --git a/src/java/voldemort/server/AbstractService.java b/src/java/voldemort/common/service/AbstractService.java similarity index 98% rename from src/java/voldemort/server/AbstractService.java rename to src/java/voldemort/common/service/AbstractService.java index 63bd46cbbf..222faed014 100644 --- a/src/java/voldemort/server/AbstractService.java +++ b/src/java/voldemort/common/service/AbstractService.java @@ -14,7 +14,7 @@ * the License. */ -package voldemort.server; +package voldemort.common.service; import java.util.concurrent.atomic.AtomicBoolean; diff --git a/src/java/voldemort/server/scheduler/SchedulerService.java b/src/java/voldemort/common/service/SchedulerService.java similarity index 97% rename from src/java/voldemort/server/scheduler/SchedulerService.java rename to src/java/voldemort/common/service/SchedulerService.java index e5c2941097..645be4015d 100644 --- a/src/java/voldemort/server/scheduler/SchedulerService.java +++ b/src/java/voldemort/common/service/SchedulerService.java @@ -14,7 +14,7 @@ * the License. */ -package voldemort.server.scheduler; +package voldemort.common.service; import java.util.Date; import java.util.List; @@ -31,9 +31,6 @@ import voldemort.annotations.jmx.JmxGetter; import voldemort.annotations.jmx.JmxManaged; import voldemort.annotations.jmx.JmxOperation; -import voldemort.server.AbstractService; -import voldemort.server.ServiceType; -import voldemort.server.VoldemortService; import voldemort.utils.Time; import com.google.common.collect.Lists; diff --git a/src/java/voldemort/server/ServiceType.java b/src/java/voldemort/common/service/ServiceType.java similarity index 94% rename from src/java/voldemort/server/ServiceType.java rename to src/java/voldemort/common/service/ServiceType.java index c2a6a47646..0fe7c7932b 100644 --- a/src/java/voldemort/server/ServiceType.java +++ b/src/java/voldemort/common/service/ServiceType.java @@ -1,4 +1,4 @@ -package voldemort.server; +package voldemort.common.service; /** * The various types of voldemort services diff --git a/src/java/voldemort/server/VoldemortService.java b/src/java/voldemort/common/service/VoldemortService.java similarity index 96% rename from src/java/voldemort/server/VoldemortService.java rename to src/java/voldemort/common/service/VoldemortService.java index c85726366b..9a4fffe92f 100644 --- a/src/java/voldemort/server/VoldemortService.java +++ b/src/java/voldemort/common/service/VoldemortService.java @@ -14,7 +14,8 @@ * the License. */ -package voldemort.server; +package voldemort.common.service; + /** * A service that runs in the voldemort server diff --git a/src/java/voldemort/server/AbstractSocketService.java b/src/java/voldemort/server/AbstractSocketService.java index 8369dad293..51a8a18a43 100644 --- a/src/java/voldemort/server/AbstractSocketService.java +++ b/src/java/voldemort/server/AbstractSocketService.java @@ -18,6 +18,9 @@ import voldemort.annotations.jmx.JmxGetter; import voldemort.annotations.jmx.JmxManaged; +import voldemort.common.service.AbstractService; +import voldemort.common.service.ServiceType; +import voldemort.common.service.VoldemortService; import voldemort.utils.JmxUtils; /** diff --git a/src/java/voldemort/server/SystemStoreConstants.java b/src/java/voldemort/server/SystemStoreConstants.java deleted file mode 100644 index a7c546681c..0000000000 --- a/src/java/voldemort/server/SystemStoreConstants.java +++ /dev/null @@ -1,77 +0,0 @@ -package voldemort.server; - -/** - * The various system stores - */ -public class SystemStoreConstants { - - public static final String NAME_PREFIX = "voldsys$_"; - - public static enum SystemStoreName { - voldsys$_client_registry, - voldsys$_client_store_definition; - } - - public static final String SYSTEM_STORE_SCHEMA = "" - + " " - + " voldsys$_client_registry" - + " zone-routing" - + " proximity-handoff" - + " memory" - + " client" - + " 4" - + " " - + " 2" - + " 2" - + " " - + " 1" - + " 1" - + " " - + " string" - + " " - + " " - + " string" - + " " - + " 7" - + " " - - + " " - + " voldsys$_client_store_definition" - + " zone-routing" - + " proximity-handoff" - + " memory" - + " client" - + " 1" - + " 1" - + " 1" - + " " - + " string" - + " " - + " " - + " string" - + " " - + " 7" - + " " - - + " " - + " voldsys$_metadata_version" - + " local-pref-all-routing" - + " proximity-handoff" - + " memory" - + " client" - + " 1" - + " 1" - + " 1" - + " " - + " string" - + " " - + " " - + " java-serialization" - + " " + " " - - + ""; - - public static boolean isSystemStore(String storeName) { - return (null == storeName ? false : storeName.startsWith(NAME_PREFIX)); - } -} \ No newline at end of file diff --git a/src/java/voldemort/server/VoldemortServer.java b/src/java/voldemort/server/VoldemortServer.java index af82eec8a9..f4134f6dd4 100644 --- a/src/java/voldemort/server/VoldemortServer.java +++ b/src/java/voldemort/server/VoldemortServer.java @@ -33,6 +33,10 @@ import voldemort.client.protocol.admin.AdminClient; import voldemort.cluster.Cluster; import voldemort.cluster.Node; +import voldemort.common.service.AbstractService; +import voldemort.common.service.SchedulerService; +import voldemort.common.service.ServiceType; +import voldemort.common.service.VoldemortService; import voldemort.server.gossip.GossipService; import voldemort.server.http.HttpService; import voldemort.server.jmx.JmxService; @@ -42,7 +46,6 @@ import voldemort.server.protocol.admin.AsyncOperationService; import voldemort.server.rebalance.Rebalancer; import voldemort.server.rebalance.RebalancerService; -import voldemort.server.scheduler.SchedulerService; import voldemort.server.socket.SocketService; import voldemort.server.storage.StorageService; import voldemort.store.configuration.ConfigurationStorageEngine; diff --git a/src/java/voldemort/server/gossip/GossipService.java b/src/java/voldemort/server/gossip/GossipService.java index bb7602dba5..170ecdc5ac 100644 --- a/src/java/voldemort/server/gossip/GossipService.java +++ b/src/java/voldemort/server/gossip/GossipService.java @@ -18,10 +18,10 @@ import voldemort.annotations.jmx.JmxManaged; import voldemort.client.protocol.admin.AdminClient; -import voldemort.server.AbstractService; -import voldemort.server.ServiceType; +import voldemort.common.service.AbstractService; +import voldemort.common.service.SchedulerService; +import voldemort.common.service.ServiceType; import voldemort.server.VoldemortConfig; -import voldemort.server.scheduler.SchedulerService; import voldemort.store.metadata.MetadataStore; import voldemort.utils.RebalanceUtils; diff --git a/src/java/voldemort/server/http/HttpService.java b/src/java/voldemort/server/http/HttpService.java index 75931d9a97..3c4229bd70 100644 --- a/src/java/voldemort/server/http/HttpService.java +++ b/src/java/voldemort/server/http/HttpService.java @@ -28,8 +28,8 @@ import voldemort.annotations.jmx.JmxGetter; import voldemort.annotations.jmx.JmxManaged; import voldemort.client.protocol.RequestFormatType; -import voldemort.server.AbstractService; -import voldemort.server.ServiceType; +import voldemort.common.service.AbstractService; +import voldemort.common.service.ServiceType; import voldemort.server.StoreRepository; import voldemort.server.VoldemortServer; import voldemort.server.http.gui.AdminServlet; diff --git a/src/java/voldemort/server/http/StoreServlet.java b/src/java/voldemort/server/http/StoreServlet.java index e9ab6eb18d..0e561ef6b3 100644 --- a/src/java/voldemort/server/http/StoreServlet.java +++ b/src/java/voldemort/server/http/StoreServlet.java @@ -29,7 +29,7 @@ import org.apache.log4j.Logger; -import voldemort.server.ServiceType; +import voldemort.common.service.ServiceType; import voldemort.server.VoldemortServer; import voldemort.server.protocol.RequestHandler; import voldemort.utils.Utils; diff --git a/src/java/voldemort/server/http/gui/ReadOnlyStoreManagementServlet.java b/src/java/voldemort/server/http/gui/ReadOnlyStoreManagementServlet.java index 088c80c367..d21094905f 100644 --- a/src/java/voldemort/server/http/gui/ReadOnlyStoreManagementServlet.java +++ b/src/java/voldemort/server/http/gui/ReadOnlyStoreManagementServlet.java @@ -30,7 +30,7 @@ import org.apache.log4j.Logger; import voldemort.VoldemortException; -import voldemort.server.ServiceType; +import voldemort.common.service.ServiceType; import voldemort.server.VoldemortConfig; import voldemort.server.VoldemortServer; import voldemort.server.http.VoldemortServletContextListener; diff --git a/src/java/voldemort/server/http/gui/StatusServlet.java b/src/java/voldemort/server/http/gui/StatusServlet.java index 3235f32f5e..86d9fa0801 100644 --- a/src/java/voldemort/server/http/gui/StatusServlet.java +++ b/src/java/voldemort/server/http/gui/StatusServlet.java @@ -15,8 +15,8 @@ import javax.servlet.http.HttpServletResponse; import voldemort.VoldemortException; +import voldemort.common.service.ServiceType; import voldemort.server.AbstractSocketService; -import voldemort.server.ServiceType; import voldemort.server.VoldemortServer; import voldemort.server.http.VoldemortServletContextListener; import voldemort.store.Store; diff --git a/src/java/voldemort/server/jmx/JmxService.java b/src/java/voldemort/server/jmx/JmxService.java index 9056027af5..c02484e9ec 100644 --- a/src/java/voldemort/server/jmx/JmxService.java +++ b/src/java/voldemort/server/jmx/JmxService.java @@ -30,11 +30,11 @@ import voldemort.annotations.jmx.JmxManaged; import voldemort.cluster.Cluster; -import voldemort.server.AbstractService; -import voldemort.server.ServiceType; +import voldemort.common.service.AbstractService; +import voldemort.common.service.ServiceType; +import voldemort.common.service.VoldemortService; import voldemort.server.StoreRepository; import voldemort.server.VoldemortServer; -import voldemort.server.VoldemortService; import voldemort.store.Store; import voldemort.store.bdb.BdbStorageEngine; import voldemort.store.bdb.stats.BdbEnvironmentStats; diff --git a/src/java/voldemort/server/niosocket/NioSocketService.java b/src/java/voldemort/server/niosocket/NioSocketService.java index b292fba739..3da380072f 100644 --- a/src/java/voldemort/server/niosocket/NioSocketService.java +++ b/src/java/voldemort/server/niosocket/NioSocketService.java @@ -32,8 +32,8 @@ import voldemort.VoldemortException; import voldemort.annotations.jmx.JmxGetter; +import voldemort.common.service.ServiceType; import voldemort.server.AbstractSocketService; -import voldemort.server.ServiceType; import voldemort.server.StatusManager; import voldemort.server.protocol.RequestHandlerFactory; import voldemort.utils.DaemonThreadFactory; diff --git a/src/java/voldemort/server/protocol/admin/AsyncOperationService.java b/src/java/voldemort/server/protocol/admin/AsyncOperationService.java index 521ad2ffc8..de0ad16603 100644 --- a/src/java/voldemort/server/protocol/admin/AsyncOperationService.java +++ b/src/java/voldemort/server/protocol/admin/AsyncOperationService.java @@ -25,9 +25,9 @@ import voldemort.VoldemortException; import voldemort.annotations.jmx.JmxManaged; import voldemort.annotations.jmx.JmxOperation; -import voldemort.server.AbstractService; -import voldemort.server.ServiceType; -import voldemort.server.scheduler.SchedulerService; +import voldemort.common.service.AbstractService; +import voldemort.common.service.SchedulerService; +import voldemort.common.service.ServiceType; /** * Asynchronous job scheduler for admin service operations. diff --git a/src/java/voldemort/server/rebalance/RebalancerService.java b/src/java/voldemort/server/rebalance/RebalancerService.java index 1c5b6791a8..caa590241a 100644 --- a/src/java/voldemort/server/rebalance/RebalancerService.java +++ b/src/java/voldemort/server/rebalance/RebalancerService.java @@ -1,12 +1,12 @@ package voldemort.server.rebalance; import voldemort.annotations.jmx.JmxManaged; -import voldemort.server.AbstractService; -import voldemort.server.ServiceType; +import voldemort.common.service.AbstractService; +import voldemort.common.service.SchedulerService; +import voldemort.common.service.ServiceType; import voldemort.server.StoreRepository; import voldemort.server.VoldemortConfig; import voldemort.server.protocol.admin.AsyncOperationService; -import voldemort.server.scheduler.SchedulerService; import voldemort.store.metadata.MetadataStore; /** diff --git a/src/java/voldemort/server/socket/SocketService.java b/src/java/voldemort/server/socket/SocketService.java index c8c7fd3085..68d7bc70dc 100644 --- a/src/java/voldemort/server/socket/SocketService.java +++ b/src/java/voldemort/server/socket/SocketService.java @@ -16,8 +16,8 @@ package voldemort.server.socket; +import voldemort.common.service.ServiceType; import voldemort.server.AbstractSocketService; -import voldemort.server.ServiceType; import voldemort.server.StatusManager; import voldemort.server.protocol.RequestHandlerFactory; diff --git a/src/java/voldemort/server/storage/StorageService.java b/src/java/voldemort/server/storage/StorageService.java index a313702a6f..0a26807baa 100644 --- a/src/java/voldemort/server/storage/StorageService.java +++ b/src/java/voldemort/server/storage/StorageService.java @@ -48,15 +48,15 @@ import voldemort.cluster.failuredetector.FailureDetector; import voldemort.cluster.failuredetector.FailureDetectorConfig; import voldemort.cluster.failuredetector.ServerStoreVerifier; +import voldemort.common.service.AbstractService; +import voldemort.common.service.SchedulerService; +import voldemort.common.service.ServiceType; import voldemort.routing.RoutingStrategy; import voldemort.routing.RoutingStrategyFactory; -import voldemort.server.AbstractService; import voldemort.server.RequestRoutingType; -import voldemort.server.ServiceType; import voldemort.server.StoreRepository; import voldemort.server.VoldemortConfig; import voldemort.server.scheduler.DataCleanupJob; -import voldemort.server.scheduler.SchedulerService; import voldemort.server.scheduler.slop.BlockingSlopPusherJob; import voldemort.server.scheduler.slop.StreamingSlopPusherJob; import voldemort.store.StorageConfiguration; diff --git a/test/unit/voldemort/client/ClientRegistryTest.java b/test/unit/voldemort/client/ClientRegistryTest.java index 5aaa80b2cd..7fe43d17ee 100644 --- a/test/unit/voldemort/client/ClientRegistryTest.java +++ b/test/unit/voldemort/client/ClientRegistryTest.java @@ -30,7 +30,7 @@ import com.google.common.collect.Lists; -@SuppressWarnings( { "unchecked" }) +@SuppressWarnings({ "unchecked" }) public class ClientRegistryTest { public static final String SERVER_LOCAL_URL = "tcp://localhost:"; @@ -39,7 +39,7 @@ public class ClientRegistryTest { public static final String STORES_XML_FILE = "test/common/voldemort/config/stores.xml"; public static final String CLIENT_CONTEXT_NAME = "testClientRegistryHappyPath"; public static final String CLIENT_CONTEXT_NAME2 = "testClientRegistryUnhappyPath"; - public static final long CLIENT_REGISTRY_REFRSH_INTERVAL = 1000; + public static final int CLIENT_REGISTRY_REFRSH_INTERVAL = 1; public static final int TOTAL_SERVERS = 2; private SocketStoreFactory socketStoreFactory = new ClientRequestExecutorPool(TOTAL_SERVERS, @@ -96,10 +96,11 @@ public void testHappyPath() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[0]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL); + .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setEnableLazy(false); SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); - StoreClient client = socketFactory.getStoreClient(TEST_STORE_NAME); - client.put("k", "v"); + StoreClient client1 = socketFactory.getStoreClient(TEST_STORE_NAME); + client1.put("k", "v"); Iterator>> it = adminClient.fetchEntries(0, SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(), emptyPartitionList, @@ -130,7 +131,7 @@ public void testHappyPath() { assertEquals(1, infoList.size()); try { - Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 2); + Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 1000 * 5); } catch(InterruptedException e) {} // now the periodical update has gone through, it shall be higher than // the bootstrap time @@ -142,6 +143,7 @@ public void testHappyPath() { infoList = getClientRegistryContent(it); assertTrue("Client registry not updated.", infoList.get(0).getBootstrapTime() < infoList.get(0).getUpdateTime()); + ((DefaultStoreClient) client1).close(); } @Test @@ -153,7 +155,8 @@ public void testTwoClients() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[0]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL); + .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setEnableLazy(false); SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); StoreClient client1 = socketFactory.getStoreClient(TEST_STORE_NAME); StoreClient client2 = socketFactory.getStoreClient(TEST_STORE_NAME); @@ -209,7 +212,7 @@ public void testTwoClients() { assertEquals(infoList.size(), 2); try { - Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 2); + Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 1000 * 5); } catch(InterruptedException e) {} // now the periodical update has gone through, it shall be higher than // the bootstrap time @@ -223,6 +226,9 @@ public void testTwoClients() { infoList.get(0).getBootstrapTime() < infoList.get(0).getUpdateTime()); assertTrue("Client registry not updated.", infoList.get(1).getBootstrapTime() < infoList.get(1).getUpdateTime()); + + ((DefaultStoreClient) client1).close(); + ((DefaultStoreClient) client2).close(); } @Test @@ -234,7 +240,8 @@ public void testTwoStores() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[0]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL); + .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setEnableLazy(false); SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); StoreClient client1 = socketFactory.getStoreClient(TEST_STORE_NAME); StoreClient client2 = socketFactory.getStoreClient(TEST_STORE_NAME2); @@ -307,7 +314,7 @@ public void testTwoStores() { } try { - Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 2); + Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 1000 * 5); } catch(InterruptedException e) {} // now the periodical update has gone through, it shall be higher than // the bootstrap time @@ -321,6 +328,9 @@ public void testTwoStores() { infoList.get(0).getBootstrapTime() < infoList.get(0).getUpdateTime()); assertTrue("Client registry not updated.", infoList.get(1).getBootstrapTime() < infoList.get(1).getUpdateTime()); + + ((DefaultStoreClient) client1).close(); + ((DefaultStoreClient) client2).close(); } @Test @@ -332,7 +342,8 @@ public void testTwoFactories() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[0]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL); + .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setEnableLazy(false); SocketStoreClientFactory socketFactory1 = new SocketStoreClientFactory(clientConfig); ClientConfig clientConfig2 = new ClientConfig().setMaxThreads(4) @@ -340,7 +351,9 @@ public void testTwoFactories() { .setMaxConnectionsPerNode(4) .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[0]) - .setClientContextName(CLIENT_CONTEXT_NAME2); + .setClientContextName(CLIENT_CONTEXT_NAME2) + .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setEnableLazy(false); SocketStoreClientFactory socketFactory2 = new SocketStoreClientFactory(clientConfig2); StoreClient client1 = socketFactory1.getStoreClient(TEST_STORE_NAME); @@ -434,7 +447,7 @@ public void testTwoFactories() { } try { - Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 2); + Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 1000 * 5); } catch(InterruptedException e) {} // now the periodical update has gone through, it shall be higher than // the bootstrap time @@ -448,6 +461,9 @@ public void testTwoFactories() { infoList.get(0).getBootstrapTime() < infoList.get(0).getUpdateTime()); assertTrue("Client registry not updated.", infoList.get(1).getBootstrapTime() < infoList.get(1).getUpdateTime()); + + ((DefaultStoreClient) client1).close(); + ((DefaultStoreClient) client2).close(); } @Test @@ -462,7 +478,8 @@ public void testOneServerFailre() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[1]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL); + .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setEnableLazy(false); SocketStoreClientFactory socketFactory1 = new SocketStoreClientFactory(clientConfig); ClientConfig clientConfig2 = new ClientConfig().setMaxThreads(4) @@ -470,7 +487,9 @@ public void testOneServerFailre() { .setMaxConnectionsPerNode(4) .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[1]) - .setClientContextName(CLIENT_CONTEXT_NAME2); + .setClientContextName(CLIENT_CONTEXT_NAME2) + .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setEnableLazy(false); SocketStoreClientFactory socketFactory2 = new SocketStoreClientFactory(clientConfig2); StoreClient client1 = socketFactory1.getStoreClient(TEST_STORE_NAME); @@ -522,7 +541,7 @@ public void testOneServerFailre() { } try { - Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 2); + Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 1000 * 5); } catch(InterruptedException e) {} // now the periodical update has gone through, it shall be higher than // the bootstrap time @@ -536,6 +555,9 @@ public void testOneServerFailre() { infoList.get(0).getBootstrapTime() < infoList.get(0).getUpdateTime()); assertTrue("Client registry not updated.", infoList.get(1).getBootstrapTime() < infoList.get(1).getUpdateTime()); + + ((DefaultStoreClient) client1).close(); + ((DefaultStoreClient) client2).close(); } @Test @@ -548,7 +570,8 @@ public void testRepeatRegistrationSameFactory() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[1]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL); + .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setEnableLazy(false); SocketStoreClientFactory socketFactory1 = new SocketStoreClientFactory(clientConfig); ClientConfig clientConfig2 = new ClientConfig().setMaxThreads(4) @@ -556,7 +579,9 @@ public void testRepeatRegistrationSameFactory() { .setMaxConnectionsPerNode(4) .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[1]) - .setClientContextName(CLIENT_CONTEXT_NAME2); + .setClientContextName(CLIENT_CONTEXT_NAME2) + .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setEnableLazy(false); SocketStoreClientFactory socketFactory2 = new SocketStoreClientFactory(clientConfig2); for(int i = 0; i < 3; i++) { @@ -566,6 +591,9 @@ public void testRepeatRegistrationSameFactory() { client1.put("k1", "v1"); client2.put("k2", "v2"); + + ((DefaultStoreClient) client1).close(); + ((DefaultStoreClient) client2).close(); } Iterator>> it = adminClient.fetchEntries(1, @@ -575,6 +603,7 @@ public void testRepeatRegistrationSameFactory() { false); ArrayList infoList = getClientRegistryContent(it); assertEquals("Incrrect # of entries created in client registry", 6, infoList.size()); + } @Test @@ -590,7 +619,8 @@ public void testRepeatRegistrationDifferentFactories() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[1]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL); + .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setEnableLazy(false); SocketStoreClientFactory socketFactory1 = new SocketStoreClientFactory(clientConfig); ClientConfig clientConfig2 = new ClientConfig().setMaxThreads(4) @@ -598,7 +628,9 @@ public void testRepeatRegistrationDifferentFactories() { .setMaxConnectionsPerNode(4) .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[1]) - .setClientContextName(CLIENT_CONTEXT_NAME2); + .setClientContextName(CLIENT_CONTEXT_NAME2) + .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setEnableLazy(false); SocketStoreClientFactory socketFactory2 = new SocketStoreClientFactory(clientConfig2); StoreClient client1 = socketFactory1.getStoreClient(TEST_STORE_NAME); @@ -652,7 +684,7 @@ public void testRepeatRegistrationDifferentFactories() { } try { - Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 2); + Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 1000 * 5); } catch(InterruptedException e) {} // now the periodical update has gone through, it shall be higher // than @@ -663,14 +695,12 @@ public void testRepeatRegistrationDifferentFactories() { null, false); infoList = getClientRegistryContent(it); - /* - * assertTrue("Client registry not updated.", - * infoList.get(0).getBootstrapTime() < - * infoList.get(0).getUpdateTime()); - * assertTrue("Client registry not updated.", - * infoList.get(1).getBootstrapTime() < - * infoList.get(1).getUpdateTime()); - */ + + assertTrue("Client registry not updated.", + infoList.get(0).getBootstrapTime() < infoList.get(0).getUpdateTime()); + assertTrue("Client registry not updated.", + infoList.get(1).getBootstrapTime() < infoList.get(1).getUpdateTime()); + assertTrue("Bootstrap time does not increase client bounces", infoList.get(0).getBootstrapTime() > client1LastBootstrapTime); assertTrue("Bootstrap time does not increase client bounces", @@ -678,6 +708,9 @@ public void testRepeatRegistrationDifferentFactories() { client1LastBootstrapTime = infoList.get(0).getBootstrapTime(); client2LastBootstrapTime = infoList.get(0).getBootstrapTime(); + + ((DefaultStoreClient) client1).close(); + ((DefaultStoreClient) client2).close(); } } diff --git a/test/unit/voldemort/client/DefaultSocketStoreClientTest.java b/test/unit/voldemort/client/DefaultSocketStoreClientTest.java index 88ab067752..6d2bc94d2e 100644 --- a/test/unit/voldemort/client/DefaultSocketStoreClientTest.java +++ b/test/unit/voldemort/client/DefaultSocketStoreClientTest.java @@ -14,14 +14,12 @@ import voldemort.ServerTestUtils; import voldemort.TestUtils; -import voldemort.VoldemortException; import voldemort.client.protocol.admin.AdminClient; import voldemort.cluster.Cluster; import voldemort.server.VoldemortServer; import voldemort.store.StoreDefinition; import voldemort.store.socket.SocketStoreFactory; import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; -import voldemort.utils.SystemTime; import voldemort.utils.Time; import voldemort.versioning.ObsoleteVersionException; import voldemort.versioning.VectorClock; @@ -30,7 +28,7 @@ public class DefaultSocketStoreClientTest { - private static String testStoreName = "test-basic-replication-memory"; + private static String testStoreName = "test-replication-memory"; private static String storesXmlfile = "test/common/voldemort/config/stores.xml"; private SocketStoreFactory socketStoreFactory = new ClientRequestExecutorPool(2, 10000, @@ -42,35 +40,10 @@ public class DefaultSocketStoreClientTest { private Cluster cluster; private AdminClient adminClient; - public static String socketUrl = ""; + public static String socketUrl = "tcp://localhost:6667"; protected StoreClient client; protected int nodeId; protected Time time; - private boolean success = true; - - private class ParallelClient extends Thread { - - private final StoreClient parallelClient; - - public ParallelClient(StoreClient client) { - this.parallelClient = client; - } - - @Override - public void run() { - System.out.println("Doing a whole bunch of puts"); - try { - for(int i = 0; i < 10000; i++) - this.parallelClient.put("my-key" + i, "my-value" + i); - } catch(VoldemortException ve) { - if(!(ve instanceof ObsoleteVersionException)) { - success = false; - ve.printStackTrace(); - fail("Exception occured while doing a put."); - } - } - } - } @Before public void setUp() throws Exception { @@ -79,7 +52,6 @@ public void setUp() throws Exception { servers = new VoldemortServer[2]; storeDefs = new StoreDefinitionsMapper().readStoreList(new File(storesXmlfile)); - boolean useNio = true; servers[0] = ServerTestUtils.startVoldemortServer(socketStoreFactory, ServerTestUtils.createServerConfig(true, 0, @@ -100,24 +72,16 @@ public void setUp() throws Exception { cluster); adminClient = ServerTestUtils.getAdminClient(cluster); - socketUrl = servers[0].getIdentityNode().getSocketUrl().toString(); - - ClientConfig clientConfig = new ClientConfig().setMaxTotalConnections(4) - .setMaxConnectionsPerNode(4) - .setBootstrapUrls(socketUrl); - - SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); - this.client = socketFactory.getStoreClient(testStoreName); - this.nodeId = 0; - this.time = SystemTime.INSTANCE; } - // @Test + @Test public void test() { - client.delete("k"); client.put("k", Versioned.value("v")); Versioned v = client.get("k"); assertEquals("GET should return the version set by PUT.", "v", v.getValue()); + VectorClock expected = new VectorClock(); + expected.incrementVersion(nodeId, time.getMilliseconds()); + assertEquals("The version should be incremented after a put.", expected, v.getVersion()); try { client.put("k", Versioned.value("v")); fail("Put of obsolete version should throw exception."); @@ -130,45 +94,26 @@ public void test() { new VectorClock().incremented(nodeId + 1, time.getMilliseconds()))); assertEquals("GET should return the new value set by PUT.", "v2", client.getValue("k")); + assertEquals("GET should return the new version set by PUT.", + expected.incremented(nodeId + 1, time.getMilliseconds()), + client.get("k").getVersion()); client.delete("k"); assertNull("After a successful delete(k), get(k) should return null.", client.get("k")); } - // @Test - public void testBatchOps() { - success = true; - - for(int j = 0; j < 10; j++) { - (new ParallelClient(client)).start(); - } - - try { - Thread.sleep(10000); - } catch(Exception e) {} - assertEquals(success, true); - } - - /* - * Test that a Node failure does not affect system stores. - */ @Test - public void testSystemStoreFailures() { - success = true; - for(int j = 0; j < 100; j++) { - (new ParallelClient(client)).start(); - } - - servers[1].stop(); - try { - Thread.sleep(10000); - } catch(Exception e) {} - - assertEquals(success, true); - try { - servers[1].start(); - } catch(Exception e) { - e.printStackTrace(); - fail("Cannot restart Voldemort Server"); - } + public void testClientRegistryHappyPath() { + ClientConfig clientConfig = new ClientConfig().setMaxThreads(4) + .setMaxTotalConnections(4) + .setMaxConnectionsPerNode(4) + .setBootstrapUrls(socketUrl) + .setClientContextName("testClientRegistryHappyPath"); + SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); + this.client = socketFactory.getStoreClient(testStoreName); + client.put("k", "v"); + adminClient.fetchEntries(0, testStoreName, null, null, false); + adminClient.fetchEntries(1, testStoreName, null, null, false); + // TODO: verify that the values in registry are correct. } + } diff --git a/test/unit/voldemort/cluster/failuredetector/ServerStoreVerifierTest.java b/test/unit/voldemort/cluster/failuredetector/ServerStoreVerifierTest.java index 62edde9cab..b6f4aa466e 100644 --- a/test/unit/voldemort/cluster/failuredetector/ServerStoreVerifierTest.java +++ b/test/unit/voldemort/cluster/failuredetector/ServerStoreVerifierTest.java @@ -35,7 +35,7 @@ import voldemort.VoldemortException; import voldemort.cluster.Cluster; import voldemort.cluster.Node; -import voldemort.server.ServiceType; +import voldemort.common.service.ServiceType; import voldemort.server.VoldemortConfig; import voldemort.server.VoldemortServer; import voldemort.server.storage.StorageService; diff --git a/test/unit/voldemort/server/ServiceTest.java b/test/unit/voldemort/server/ServiceTest.java index 8c04c298db..30fd4475ff 100644 --- a/test/unit/voldemort/server/ServiceTest.java +++ b/test/unit/voldemort/server/ServiceTest.java @@ -16,6 +16,8 @@ package voldemort.server; +import voldemort.common.service.AbstractService; +import voldemort.common.service.ServiceType; import junit.framework.TestCase; /** diff --git a/test/unit/voldemort/server/protocol/admin/AsyncOperationTest.java b/test/unit/voldemort/server/protocol/admin/AsyncOperationTest.java index 01b35886a3..16b7781347 100644 --- a/test/unit/voldemort/server/protocol/admin/AsyncOperationTest.java +++ b/test/unit/voldemort/server/protocol/admin/AsyncOperationTest.java @@ -26,7 +26,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import junit.framework.TestCase; -import voldemort.server.scheduler.SchedulerService; +import voldemort.common.service.SchedulerService; import voldemort.utils.SystemTime; /** diff --git a/test/unit/voldemort/server/storage/StorageServiceTest.java b/test/unit/voldemort/server/storage/StorageServiceTest.java index f51ceebc6f..f3fcf9d4c0 100644 --- a/test/unit/voldemort/server/storage/StorageServiceTest.java +++ b/test/unit/voldemort/server/storage/StorageServiceTest.java @@ -8,9 +8,9 @@ import voldemort.ServerTestUtils; import voldemort.TestUtils; import voldemort.cluster.Cluster; +import voldemort.common.service.SchedulerService; import voldemort.server.StoreRepository; import voldemort.server.VoldemortConfig; -import voldemort.server.scheduler.SchedulerService; import voldemort.store.StoreDefinition; import voldemort.store.metadata.MetadataStore; From e90fa82d25decdbee1f102302e7746e06a2e4ba2 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Tue, 3 Jul 2012 16:25:38 -0700 Subject: [PATCH 086/209] Adding tests for system stores, local pref routing and Version manager. Fixed some things in PipelineRoutedStore --- .../client/SystemStoreRepository.java | 14 +- .../AsyncMetadataVersionManager.java | 40 +++-- .../store/routed/PipelineRoutedStore.java | 65 ++++++-- .../action/ConfigureNodesLocalHostTest.java | 80 +++++++++ .../AsyncMetadataVersionManagerTest.java | 156 ++++++++++++++++++ .../store/system/SystemStoreTest.java | 129 +++++++++++++++ 6 files changed, 445 insertions(+), 39 deletions(-) create mode 100644 test/unit/voldemort/store/routed/action/ConfigureNodesLocalHostTest.java create mode 100644 test/unit/voldemort/store/system/AsyncMetadataVersionManagerTest.java create mode 100644 test/unit/voldemort/store/system/SystemStoreTest.java diff --git a/src/java/voldemort/client/SystemStoreRepository.java b/src/java/voldemort/client/SystemStoreRepository.java index 9a6a020dd1..e693351dbf 100644 --- a/src/java/voldemort/client/SystemStoreRepository.java +++ b/src/java/voldemort/client/SystemStoreRepository.java @@ -18,13 +18,17 @@ public SystemStoreRepository() { sysStoreMap = new ConcurrentHashMap(); } + public void addSystemStore(SystemStore newSysStore, String storeName) { + this.sysStoreMap.put(storeName, newSysStore); + } + public void createSystemStores(ClientConfig config, String clusterXml) { for(SystemStoreConstants.SystemStoreName storeName: SystemStoreConstants.SystemStoreName.values()) { - SystemStore sysVersionStore = new SystemStore(storeName.name(), - config.getBootstrapUrls(), - config.getClientZoneId(), - clusterXml); - this.sysStoreMap.put(storeName.name(), sysVersionStore); + SystemStore sysStore = new SystemStore(storeName.name(), + config.getBootstrapUrls(), + config.getClientZoneId(), + clusterXml); + this.sysStoreMap.put(storeName.name(), sysStore); } } diff --git a/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java b/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java index 3b8d6f027b..0db362f0e6 100644 --- a/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java +++ b/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java @@ -1,10 +1,8 @@ package voldemort.client.scheduler; -import java.lang.Thread.UncaughtExceptionHandler; import java.util.Random; import java.util.concurrent.Callable; -import org.apache.log4j.Level; import org.apache.log4j.Logger; import voldemort.client.SystemStoreRepository; @@ -15,16 +13,16 @@ * cluster and if necessary Re-bootstrap the client. * * During initialization, it will retrieve the current version of the store (or - * the entire stores.xml depending upon granularity) and then periodically check - * whether this has been updated. During init if the initial version turns out - * to be null, it means that no change has been done to that store since it was - * created. In this case, we assume version '0'. + * the entire stores.xml depending upon granularity) and cluster.xml and then + * periodically check whether this has been updated. During init if the initial + * version turns out to be null, it means that no change has been done to that + * store since it was created. In this case, we assume version '0'. */ public class AsyncMetadataVersionManager implements Runnable { - private static final String STORES_VERSION_KEY = "stores.xml"; - private static final String CLUSTER_VERSION_KEY = "cluster.xml"; + public static final String STORES_VERSION_KEY = "stores.xml"; + public static final String CLUSTER_VERSION_KEY = "cluster.xml"; private final Logger logger = Logger.getLogger(this.getClass()); private Versioned currentStoreVersion; @@ -36,6 +34,8 @@ public class AsyncMetadataVersionManager implements Runnable { private final int DELTA_MAX = 2000; private final Random randomGenerator = new Random(System.currentTimeMillis()); + public boolean isActive = false; + public AsyncMetadataVersionManager(SystemStoreRepository sysRepository, Callable storeClientThunk) { this.sysRepository = sysRepository; @@ -56,22 +56,18 @@ public AsyncMetadataVersionManager(SystemStoreRepository sysRepository, } // If the received version is null, assume version 0 - if(currentStoreVersion == null) + if(currentStoreVersion == null) { currentStoreVersion = new Versioned((long) 0); - if(currentClusterVersion == null) + } + if(currentClusterVersion == null) { currentClusterVersion = new Versioned((long) 0); + } - Thread checkVersionThread = new Thread(this, "AsyncVersionCheckThread"); - checkVersionThread.setDaemon(true); - checkVersionThread.setUncaughtExceptionHandler(new UncaughtExceptionHandler() { - - public void uncaughtException(Thread t, Throwable e) { - if(logger.isEnabledFor(Level.ERROR)) - logger.error("Uncaught exception in Metadata Version check thread:", e); - } - }); + logger.debug("Initial stores.xml version = " + this.currentStoreVersion); + logger.debug("Initial cluster.xml version = " + this.currentClusterVersion); this.storeClientThunk = storeClientThunk; + this.isActive = true; } /* @@ -126,15 +122,17 @@ public void run() { } try { - this.storeClientThunk.call(); - if(newStoresVersion != null) { + logger.info("Updating stores version"); currentStoreVersion = newStoresVersion; } if(newClusterVersion != null) { + logger.info("Updating cluster version"); currentClusterVersion = newClusterVersion; } + + this.storeClientThunk.call(); } catch(Exception e) { e.printStackTrace(); logger.info(e.getMessage()); diff --git a/src/java/voldemort/store/routed/PipelineRoutedStore.java b/src/java/voldemort/store/routed/PipelineRoutedStore.java index 3016a097a3..e9f45d2d64 100644 --- a/src/java/voldemort/store/routed/PipelineRoutedStore.java +++ b/src/java/voldemort/store/routed/PipelineRoutedStore.java @@ -153,30 +153,31 @@ public PipelineRoutedStore(String name, } } - private ConfigureNodesType getNodeConfigurationType(BasicPipelineData>> pipelineData) { + private ConfigureNodesType obtainNodeConfigurationType(Integer zonesRequired) { // If Zone and local preference required - if(pipelineData.getZonesRequired() != null + if(zonesRequired != null && routingStrategy.getType().equals(RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY)) return ConfigureNodesType.BYZONE_LOCAL; // If only local preference required - else if(pipelineData.getZonesRequired() == null + else if(zonesRequired == null && routingStrategy.getType().equals(RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY)) return ConfigureNodesType.DEFAULT_LOCAL; // If only Zone required - else if(pipelineData.getZonesRequired() != null + else if(zonesRequired != null && !routingStrategy.getType() .equals(RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY)) return ConfigureNodesType.BYZONE; // Default case return ConfigureNodesType.DEFAULT; + } private AbstractConfigureNodes>, BasicPipelineData>>> getNodeConfiguration(BasicPipelineData>> pipelineData, ByteArray key) { - switch(getNodeConfigurationType(pipelineData)) { + switch(obtainNodeConfigurationType(pipelineData.getZonesRequired())) { case DEFAULT: return new ConfigureNodesDefault>, BasicPipelineData>>>(pipelineData, Event.CONFIGURED, @@ -566,6 +567,45 @@ public boolean isHintedHandoffEnabled() { return slopStores != null; } + private AbstractConfigureNodes putNodeConfiguration(PutPipelineData pipelineData, + ByteArray key) { + switch(obtainNodeConfigurationType(pipelineData.getZonesRequired())) { + case DEFAULT: + return new ConfigureNodesDefault(pipelineData, + Event.CONFIGURED, + failureDetector, + storeDef.getRequiredWrites(), + routingStrategy, + key); + case BYZONE: + return new ConfigureNodesByZone(pipelineData, + Event.CONFIGURED, + failureDetector, + storeDef.getRequiredWrites(), + routingStrategy, + key, + clientZone); + case DEFAULT_LOCAL: + return new ConfigureNodesLocalHost(pipelineData, + Event.CONFIGURED, + failureDetector, + storeDef.getRequiredWrites(), + routingStrategy, + key); + case BYZONE_LOCAL: + return new ConfigureNodesLocalHostByZone(pipelineData, + Event.CONFIGURED, + failureDetector, + storeDef.getRequiredWrites(), + routingStrategy, + key, + clientZone); + default: + return null; + } + + } + public void put(ByteArray key, Versioned versioned, byte[] transforms) throws VoldemortException { StoreUtils.assertValidKey(key); @@ -585,6 +625,11 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) HintedHandoff hintedHandoff = null; + // Get the correct type of configure nodes action depending on the store + // requirements + AbstractConfigureNodes configureNodes = putNodeConfiguration(pipelineData, + key); + if(isHintedHandoffEnabled()) hintedHandoff = new HintedHandoff(failureDetector, slopStores, @@ -593,14 +638,8 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) pipelineData.getFailedNodes(), timeoutConfig.getOperationTimeout(VoldemortOpCode.PUT_OP_CODE)); - pipeline.addEventAction(Event.STARTED, - new ConfigureNodes(pipelineData, - Event.CONFIGURED, - failureDetector, - storeDef.getRequiredWrites(), - routingStrategy, - key, - clientZone)); + pipeline.addEventAction(Event.STARTED, configureNodes); + pipeline.addEventAction(Event.CONFIGURED, new PerformSerialPutRequests(pipelineData, isHintedHandoffEnabled() ? Event.RESPONSES_RECEIVED diff --git a/test/unit/voldemort/store/routed/action/ConfigureNodesLocalHostTest.java b/test/unit/voldemort/store/routed/action/ConfigureNodesLocalHostTest.java new file mode 100644 index 0000000000..e89d43e2b6 --- /dev/null +++ b/test/unit/voldemort/store/routed/action/ConfigureNodesLocalHostTest.java @@ -0,0 +1,80 @@ +package voldemort.store.routed.action; + +import static org.junit.Assert.assertEquals; + +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import org.junit.Test; + +import voldemort.TestUtils; +import voldemort.cluster.Cluster; +import voldemort.cluster.Node; +import voldemort.cluster.failuredetector.FailureDetector; +import voldemort.cluster.failuredetector.FailureDetectorConfig; +import voldemort.cluster.failuredetector.ThresholdFailureDetector; +import voldemort.routing.RouteToAllLocalPrefStrategy; +import voldemort.routing.RoutingStrategy; +import voldemort.store.routed.BasicPipelineData; +import voldemort.store.routed.Pipeline; +import voldemort.store.routed.Pipeline.Event; +import voldemort.store.routed.Pipeline.Operation; +import voldemort.utils.ByteArray; + +import com.google.common.collect.ImmutableList; + +public class ConfigureNodesLocalHostTest { + + protected final ByteArray aKey = TestUtils.toByteArray("vold"); + protected String currentHost = ""; + + private List getTestNodes() { + try { + currentHost = InetAddress.getLocalHost().getHostName(); + } catch(UnknownHostException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + return ImmutableList.of(node(0, "some-node-1", 2, 7, 14), + node(1, "some-node-2", 1, 10, 13), + node(2, currentHost, 3, 5, 17), + node(3, "some-node-3", 0, 11, 16), + node(4, "some-node-4", 6, 9, 15), + node(5, "some-node-5", 4, 8, 12)); + } + + private Node node(int id, String hostName, int... tags) { + List list = new ArrayList(tags.length); + for(int tag: tags) + list.add(tag); + return new Node(id, hostName, 8080, 6666, 6667, list); + } + + @Test + public void testConfigureNodesLocalHost() throws Exception { + List nodes = getTestNodes(); + Cluster cluster = new Cluster("test-route-all-local-pref-cluster", nodes); + FailureDetector failureDetector = new ThresholdFailureDetector(new FailureDetectorConfig().setNodes(nodes)); + RoutingStrategy routingStrategy = new RouteToAllLocalPrefStrategy(cluster.getNodes()); + BasicPipelineData pipelineData = new BasicPipelineData(); + ConfigureNodesLocalHost> action = new ConfigureNodesLocalHost>(pipelineData, + Event.COMPLETED, + failureDetector, + 1, + routingStrategy, + aKey); + Pipeline pipeline = new Pipeline(Operation.GET, 10000, TimeUnit.MILLISECONDS); + pipeline.addEventAction(Event.STARTED, action); + pipeline.addEvent(Event.STARTED); + pipeline.execute(); + + if(pipelineData.getFatalError() != null) + throw pipelineData.getFatalError(); + + assertEquals(cluster.getNodes().size(), pipelineData.getNodes().size()); + assertEquals(pipelineData.getNodes().get(0).getHost(), currentHost); + } +} diff --git a/test/unit/voldemort/store/system/AsyncMetadataVersionManagerTest.java b/test/unit/voldemort/store/system/AsyncMetadataVersionManagerTest.java new file mode 100644 index 0000000000..650d10b1e4 --- /dev/null +++ b/test/unit/voldemort/store/system/AsyncMetadataVersionManagerTest.java @@ -0,0 +1,156 @@ +package voldemort.store.system; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import java.util.Date; +import java.util.Properties; +import java.util.concurrent.Callable; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import voldemort.ServerTestUtils; +import voldemort.TestUtils; +import voldemort.client.SystemStore; +import voldemort.client.SystemStoreRepository; +import voldemort.client.scheduler.AsyncMetadataVersionManager; +import voldemort.cluster.Cluster; +import voldemort.common.service.SchedulerService; +import voldemort.server.VoldemortServer; +import voldemort.store.socket.SocketStoreFactory; +import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; +import voldemort.utils.SystemTime; +import voldemort.versioning.Versioned; + +public class AsyncMetadataVersionManagerTest { + + private static String storesXmlfile = "test/common/voldemort/config/stores.xml"; + String[] bootStrapUrls = null; + private String clusterXml; + private SocketStoreFactory socketStoreFactory = new ClientRequestExecutorPool(2, + 10000, + 100000, + 32 * 1024); + + private VoldemortServer[] servers; + private Cluster cluster; + public static String socketUrl = ""; + protected final int CLIENT_ZONE_ID = 0; + private long newVersion = 0; + + private SystemStore sysVersionStore; + private SystemStoreRepository repository; + private SchedulerService scheduler; + private AsyncMetadataVersionManager asyncCheckMetadata; + private boolean callbackDone = false; + private long updatedStoresVersion; + + @Before + public void setUp() throws Exception { + cluster = ServerTestUtils.getLocalCluster(2, new int[][] { { 0, 1, 2, 3 }, { 4, 5, 6, 7 } }); + servers = new VoldemortServer[2]; + + servers[0] = ServerTestUtils.startVoldemortServer(socketStoreFactory, + ServerTestUtils.createServerConfig(true, + 0, + TestUtils.createTempDir() + .getAbsolutePath(), + null, + storesXmlfile, + new Properties()), + cluster); + servers[1] = ServerTestUtils.startVoldemortServer(socketStoreFactory, + ServerTestUtils.createServerConfig(true, + 1, + TestUtils.createTempDir() + .getAbsolutePath(), + null, + storesXmlfile, + new Properties()), + cluster); + + socketUrl = servers[0].getIdentityNode().getSocketUrl().toString(); + + bootStrapUrls = new String[1]; + bootStrapUrls[0] = socketUrl; + sysVersionStore = new SystemStore(SystemStoreConstants.SystemStoreName.voldsys$_metadata_version.name(), + bootStrapUrls, + this.CLIENT_ZONE_ID); + repository = new SystemStoreRepository(); + repository.addSystemStore(sysVersionStore, + SystemStoreConstants.SystemStoreName.voldsys$_metadata_version.name()); + this.scheduler = new SchedulerService(2, SystemTime.INSTANCE, true); + } + + @After + public void tearDown() throws Exception { + servers[0].stop(); + servers[1].stop(); + } + + @Test + public void testBasicAsyncBehaviour() { + try { + Callable rebootstrapCallback = new Callable() { + + public Void call() throws Exception { + callback(); + return null; + } + }; + + // Write a base version of 100 + this.sysVersionStore.putSysStore(AsyncMetadataVersionManager.STORES_VERSION_KEY, 100l); + + // Giving enough time to complete the above put. + Thread.sleep(500); + + // Starting the Version Metadata Manager + this.asyncCheckMetadata = new AsyncMetadataVersionManager(this.repository, + rebootstrapCallback); + scheduler.schedule(asyncCheckMetadata.getClass().getName(), + asyncCheckMetadata, + new Date(), + 500); + + // Wait until the Version Manager is active + int maxRetries = 0; + while(maxRetries < 3 && !asyncCheckMetadata.isActive) { + Thread.sleep(500); + maxRetries++; + } + + // Updating the version metadata here for the Version Metadata + // Manager to detect + this.newVersion = 101; + this.sysVersionStore.putSysStore(AsyncMetadataVersionManager.STORES_VERSION_KEY, + this.newVersion); + + maxRetries = 0; + while(maxRetries < 3 && !callbackDone) { + Thread.sleep(2000); + maxRetries++; + } + + assertEquals(this.updatedStoresVersion, this.newVersion); + } catch(Exception e) { + e.printStackTrace(); + fail("Failed to start the Metadata Version Manager : " + e.getMessage()); + } + } + + private void callback() { + try { + Versioned storeVersion = this.asyncCheckMetadata.getStoreMetadataVersion(); + if(storeVersion != null) { + this.updatedStoresVersion = storeVersion.getValue(); + } + } catch(Exception e) { + fail("Error in updating stores.xml version: " + e.getMessage()); + } finally { + this.callbackDone = true; + } + } +} diff --git a/test/unit/voldemort/store/system/SystemStoreTest.java b/test/unit/voldemort/store/system/SystemStoreTest.java new file mode 100644 index 0000000000..73309cb8bc --- /dev/null +++ b/test/unit/voldemort/store/system/SystemStoreTest.java @@ -0,0 +1,129 @@ +package voldemort.store.system; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import java.util.Properties; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import voldemort.ServerTestUtils; +import voldemort.TestUtils; +import voldemort.client.AbstractStoreClientFactory; +import voldemort.client.ClientConfig; +import voldemort.client.SocketStoreClientFactory; +import voldemort.client.SystemStore; +import voldemort.cluster.Cluster; +import voldemort.server.VoldemortServer; +import voldemort.store.metadata.MetadataStore; +import voldemort.store.socket.SocketStoreFactory; +import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; + +public class SystemStoreTest { + + private static String storesXmlfile = "test/common/voldemort/config/stores.xml"; + String[] bootStrapUrls = null; + private String clusterXml; + private SocketStoreFactory socketStoreFactory = new ClientRequestExecutorPool(2, + 10000, + 100000, + 32 * 1024); + + private VoldemortServer[] servers; + private Cluster cluster; + public static String socketUrl = ""; + protected final int CLIENT_ZONE_ID = 0; + + @Before + public void setUp() throws Exception { + cluster = ServerTestUtils.getLocalCluster(2, new int[][] { { 0, 1, 2, 3 }, { 4, 5, 6, 7 } }); + servers = new VoldemortServer[2]; + + servers[0] = ServerTestUtils.startVoldemortServer(socketStoreFactory, + ServerTestUtils.createServerConfig(true, + 0, + TestUtils.createTempDir() + .getAbsolutePath(), + null, + storesXmlfile, + new Properties()), + cluster); + servers[1] = ServerTestUtils.startVoldemortServer(socketStoreFactory, + ServerTestUtils.createServerConfig(true, + 1, + TestUtils.createTempDir() + .getAbsolutePath(), + null, + storesXmlfile, + new Properties()), + cluster); + + socketUrl = servers[0].getIdentityNode().getSocketUrl().toString(); + + ClientConfig clientConfig = new ClientConfig().setMaxTotalConnections(4) + .setMaxConnectionsPerNode(4) + .setBootstrapUrls(socketUrl); + + SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); + bootStrapUrls = new String[1]; + bootStrapUrls[0] = socketUrl; + clusterXml = ((AbstractStoreClientFactory) socketFactory).bootstrapMetadataWithRetries(MetadataStore.CLUSTER_KEY); + + } + + @After + public void tearDown() throws Exception { + servers[0].stop(); + servers[1].stop(); + } + + @Test + public void testBasicStore() { + try { + SystemStore sysVersionStore = new SystemStore(SystemStoreConstants.SystemStoreName.voldsys$_metadata_version.name(), + bootStrapUrls, + this.CLIENT_ZONE_ID); + long storesVersion = 1; + sysVersionStore.putSysStore("stores.xml", storesVersion); + long version = sysVersionStore.getValueSysStore("stores.xml"); + assertEquals("Received incorrect version from the voldsys$_metadata_version system store", + storesVersion, + version); + } catch(Exception e) { + fail("Failed to create the default System Store : " + e.getMessage()); + } + } + + @Test + public void testCustomClusterXmlStore() { + try { + SystemStore sysVersionStore = new SystemStore(SystemStoreConstants.SystemStoreName.voldsys$_metadata_version.name(), + bootStrapUrls, + this.CLIENT_ZONE_ID, + this.clusterXml); + long storesVersion = 1; + sysVersionStore.putSysStore("stores.xml", storesVersion); + long version = sysVersionStore.getValueSysStore("stores.xml"); + assertEquals("Received incorrect version from the voldsys$_metadata_version system store", + storesVersion, + version); + } catch(Exception e) { + fail("Failed to create System Store with custom cluster Xml: " + e.getMessage()); + } + } + + @Test + public void testIllegalSystemStore() { + try { + SystemStore sysVersionStore = new SystemStore("test-store", + bootStrapUrls, + this.CLIENT_ZONE_ID, + this.clusterXml); + fail("Should not execute this. We can only connect to system store with a 'voldsys$' prefix."); + } catch(Exception e) { + // This is fine. + } + } +} From 88cb37cb8c56e36c4e923eabab2eb4ba08384729 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Fri, 6 Jul 2012 10:34:05 -0700 Subject: [PATCH 087/209] BDB+ Data cleanup Monitoring changes --- .../voldemort/server/VoldemortConfig.java | 15 ++ .../voldemort/server/VoldemortServer.java | 5 + .../server/scheduler/DataCleanupJob.java | 28 ++- .../scheduler/slop/BlockingSlopPusherJob.java | 2 +- .../slop/StreamingSlopPusherJob.java | 2 +- .../voldemort/server/storage/RepairJob.java | 17 +- .../server/storage/ScanPermitWrapper.java | 48 ++++- .../server/storage/StorageService.java | 13 ++ .../voldemort/store/bdb/BdbRuntimeConfig.java | 18 +- .../store/bdb/BdbStorageConfiguration.java | 13 +- .../voldemort/store/bdb/BdbStorageEngine.java | 4 +- .../store/bdb/stats/BdbEnvironmentStats.java | 166 ++++++++++++++---- .../bdb/stats/SpaceUtilizationStats.java | 70 ++++++++ src/java/voldemort/utils/JNAUtils.java | 84 +++++++++ test/integration/voldemort/CatBdbStore.java | 4 +- .../voldemort/performance/BdbGrowth.java | 3 +- .../store/bdb/BdbSplitStorageEngineTest.java | 16 +- .../store/bdb/BdbStorageEngineTest.java | 3 +- 18 files changed, 431 insertions(+), 80 deletions(-) create mode 100644 src/java/voldemort/store/bdb/stats/SpaceUtilizationStats.java create mode 100644 src/java/voldemort/utils/JNAUtils.java diff --git a/src/java/voldemort/server/VoldemortConfig.java b/src/java/voldemort/server/VoldemortConfig.java index 61e11c0df2..6f81db581d 100644 --- a/src/java/voldemort/server/VoldemortConfig.java +++ b/src/java/voldemort/server/VoldemortConfig.java @@ -85,6 +85,7 @@ public class VoldemortConfig implements Serializable { private int bdbLogIteratorReadSize; private boolean bdbFairLatches; private long bdbStatsCacheTtlMs; + private boolean bdbExposeSpaceUtilization; private String mysqlUsername; private String mysqlPassword; @@ -226,6 +227,7 @@ public VoldemortConfig(Props props) { this.bdbCleanerMaxBatchFiles = props.getInt("bdb.cleaner.max.batch.files", 0); this.bdbReadUncommitted = props.getBoolean("bdb.lock.read_uncommitted", true); this.bdbStatsCacheTtlMs = props.getLong("bdb.stats.cache.ttl.ms", 5 * Time.MS_PER_SECOND); + this.bdbExposeSpaceUtilization = props.getBoolean("bdb.expose.space.utilization", true); this.readOnlyBackups = props.getInt("readonly.backups", 1); this.readOnlySearchStrategy = props.getString("readonly.search.strategy", @@ -546,6 +548,19 @@ public void setBdbCacheSize(int bdbCacheSize) { this.bdbCacheSize = bdbCacheSize; } + /** + * This parameter controls whether we expose space utilization via MBean. If + * set to false, stat will always return 0; + * + */ + public boolean getBdbExposeSpaceUtilization() { + return bdbExposeSpaceUtilization; + } + + public void setBdbExposeSpaceUtilization(boolean bdbExposeSpaceUtilization) { + this.bdbExposeSpaceUtilization = bdbExposeSpaceUtilization; + } + /** * Given by "bdb.flush.transactions". If true then sync transactions to disk * immediately. default: false diff --git a/src/java/voldemort/server/VoldemortServer.java b/src/java/voldemort/server/VoldemortServer.java index af82eec8a9..9b20116c8a 100644 --- a/src/java/voldemort/server/VoldemortServer.java +++ b/src/java/voldemort/server/VoldemortServer.java @@ -47,6 +47,7 @@ import voldemort.server.storage.StorageService; import voldemort.store.configuration.ConfigurationStorageEngine; import voldemort.store.metadata.MetadataStore; +import voldemort.utils.JNAUtils; import voldemort.utils.RebalanceUtils; import voldemort.utils.SystemTime; import voldemort.utils.Utils; @@ -246,6 +247,8 @@ private List createServices() { @Override protected void startInner() throws VoldemortException { + // lock down jvm heap + JNAUtils.tryMlockall(); logger.info("Starting " + services.size() + " services."); long start = System.currentTimeMillis(); for(VoldemortService service: services) @@ -278,6 +281,8 @@ protected void stopInner() throws VoldemortException { if(exceptions.size() > 0) throw exceptions.get(0); + // release lock of jvm heap + JNAUtils.tryMunlockall(); } public static void main(String[] args) throws Exception { diff --git a/src/java/voldemort/server/scheduler/DataCleanupJob.java b/src/java/voldemort/server/scheduler/DataCleanupJob.java index ff3dd3adc1..5adfc4a4f5 100644 --- a/src/java/voldemort/server/scheduler/DataCleanupJob.java +++ b/src/java/voldemort/server/scheduler/DataCleanupJob.java @@ -16,8 +16,11 @@ package voldemort.server.scheduler; +import java.util.concurrent.atomic.AtomicLong; + import org.apache.log4j.Logger; +import voldemort.annotations.jmx.JmxGetter; import voldemort.server.storage.ScanPermitWrapper; import voldemort.store.StorageEngine; import voldemort.utils.ClosableIterator; @@ -42,6 +45,8 @@ public class DataCleanupJob implements Runnable { private final long maxAgeMs; private final Time time; private final EventThrottler throttler; + private long totalEntriesScanned = 0; + private AtomicLong progressThisRun; public DataCleanupJob(StorageEngine store, ScanPermitWrapper cleanupPermits, @@ -53,10 +58,12 @@ public DataCleanupJob(StorageEngine store, this.maxAgeMs = maxAgeMs; this.time = time; this.throttler = throttler; + this.progressThisRun = new AtomicLong(0); } public void run() { - acquireCleanupPermit(); + acquireCleanupPermit(progressThisRun); + ClosableIterator>> iterator = null; try { logger.info("Starting data cleanup on store \"" + store.getName() + "\"..."); @@ -70,7 +77,7 @@ public void run() { logger.info("Datacleanup job halted."); return; } - + progressThisRun.incrementAndGet(); Pair> keyAndVal = iterator.next(); VectorClock clock = (VectorClock) keyAndVal.getSecond().getVersion(); if(now - clock.getTimestamp() > maxAgeMs) { @@ -83,14 +90,21 @@ public void run() { // throttle on number of entries. throttler.maybeThrottle(1); } + // log the total items scanned, so we will get an idea of data + // growth in a cheap, periodic way logger.info("Data cleanup on store \"" + store.getName() + "\" is complete; " + deleted - + " items deleted."); + + " items deleted. " + progressThisRun.get() + " items scanned"); + } catch(Exception e) { logger.error("Error in data cleanup job for store " + store.getName() + ": ", e); } finally { closeIterator(iterator); logger.info("Releasing lock after data cleanup on \"" + store.getName() + "\"."); this.cleanupPermits.release(); + synchronized(this) { + totalEntriesScanned += progressThisRun.get(); + progressThisRun.set(0); + } } } @@ -103,14 +117,18 @@ private void closeIterator(ClosableIterator>> iterator) { } } - private void acquireCleanupPermit() { + private void acquireCleanupPermit(AtomicLong progress) { logger.info("Acquiring lock to perform data cleanup on \"" + store.getName() + "\"."); try { - this.cleanupPermits.acquire(); + this.cleanupPermits.acquire(progress); } catch(InterruptedException e) { throw new IllegalStateException("Datacleanup interrupted while waiting for cleanup permit.", e); } } + @JmxGetter(name = "numEntriesScanned", description = "Returns number of entries scanned") + public synchronized long getEntriesScanned() { + return totalEntriesScanned + progressThisRun.get(); + } } diff --git a/src/java/voldemort/server/scheduler/slop/BlockingSlopPusherJob.java b/src/java/voldemort/server/scheduler/slop/BlockingSlopPusherJob.java index 6e4b88dedf..774fb94b9d 100644 --- a/src/java/voldemort/server/scheduler/slop/BlockingSlopPusherJob.java +++ b/src/java/voldemort/server/scheduler/slop/BlockingSlopPusherJob.java @@ -219,7 +219,7 @@ public void run() { private void acquireRepairPermit() { logger.info("Acquiring lock to perform blocking slop pusher job "); try { - this.repairPermits.acquire(); + this.repairPermits.acquire(null); logger.info("Acquired lock to perform blocking slop pusher job "); } catch(InterruptedException e) { throw new IllegalStateException("Blocking slop pusher job interrupted while waiting for permit.", diff --git a/src/java/voldemort/server/scheduler/slop/StreamingSlopPusherJob.java b/src/java/voldemort/server/scheduler/slop/StreamingSlopPusherJob.java index bd6a5be339..6b5950b11d 100644 --- a/src/java/voldemort/server/scheduler/slop/StreamingSlopPusherJob.java +++ b/src/java/voldemort/server/scheduler/slop/StreamingSlopPusherJob.java @@ -372,7 +372,7 @@ protected Versioned computeNext() { private void acquireRepairPermit() { logger.info("Acquiring lock to perform streaming slop pusher job "); try { - this.repairPermits.acquire(); + this.repairPermits.acquire(null); logger.info("Acquired lock to perform streaming slop pusher job "); } catch(InterruptedException e) { stopAdminClient(); diff --git a/src/java/voldemort/server/storage/RepairJob.java b/src/java/voldemort/server/storage/RepairJob.java index 39e16895a0..aa95547210 100644 --- a/src/java/voldemort/server/storage/RepairJob.java +++ b/src/java/voldemort/server/storage/RepairJob.java @@ -4,6 +4,7 @@ import java.util.Date; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; import javax.management.MBeanOperationInfo; @@ -75,8 +76,8 @@ public void run() { for(StoreDefinition storeDef: metadataStore.getStoreDefList()) { localStats.put(storeDef.getName(), 0L); } - - if(!acquireRepairPermit()) + AtomicLong progress = new AtomicLong(0); + if(!acquireRepairPermit(progress)) return; try { // Get routing factory @@ -93,7 +94,6 @@ public void run() { metadataStore.getCluster()); long repairSlops = 0L; long numDeletedKeys = 0; - long numScannedKeys = 0; while(iterator.hasNext()) { Pair> keyAndVal; keyAndVal = iterator.next(); @@ -103,10 +103,9 @@ public void run() { engine.delete(keyAndVal.getFirst(), keyAndVal.getSecond().getVersion()); numDeletedKeys++; } - numScannedKeys++; - if(numScannedKeys % deleteBatchSize == 0) - logger.info("#Scanned:" + numScannedKeys + " #Deleted:" - + numDeletedKeys); + long itemsScanned = progress.incrementAndGet(); + if(itemsScanned % deleteBatchSize == 0) + logger.info("#Scanned:" + itemsScanned + " #Deleted:" + numDeletedKeys); } closeIterator(iterator); localStats.put(storeDef.getName(), repairSlops); @@ -149,9 +148,9 @@ private boolean isWritableStore(StoreDefinition storeDef) { } } - private boolean acquireRepairPermit() { + private boolean acquireRepairPermit(AtomicLong progress) { logger.info("Acquiring lock to perform repair job "); - if(this.repairPermits.tryAcquire()) { + if(this.repairPermits.tryAcquire(progress)) { logger.info("Acquired lock to perform repair job "); return true; } else { diff --git a/src/java/voldemort/server/storage/ScanPermitWrapper.java b/src/java/voldemort/server/storage/ScanPermitWrapper.java index 058973845a..5f4f84f979 100644 --- a/src/java/voldemort/server/storage/ScanPermitWrapper.java +++ b/src/java/voldemort/server/storage/ScanPermitWrapper.java @@ -2,49 +2,63 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicLong; public class ScanPermitWrapper { private final Semaphore scanPermits; - private List permitOwners; + private Map permitOwners; + private final int numPermits; + + private long totalEntriesScanned; public ScanPermitWrapper(final int numPermits) { + this.numPermits = numPermits; scanPermits = new Semaphore(numPermits); - permitOwners = Collections.synchronizedList(new ArrayList()); + permitOwners = Collections.synchronizedMap(new HashMap()); + } + + public static String getOwnerName() { + return Thread.currentThread().getStackTrace()[2].getClassName(); } - public void acquire() throws InterruptedException { + public void acquire(AtomicLong progress) throws InterruptedException { this.scanPermits.acquire(); synchronized(permitOwners) { - permitOwners.add(Thread.currentThread().getStackTrace()[2].getClassName()); + permitOwners.put(getOwnerName(), progress); } } public void release() { this.scanPermits.release(); synchronized(permitOwners) { - permitOwners.remove(Thread.currentThread().getStackTrace()[2].getClassName()); + AtomicLong scannedCount = permitOwners.get(getOwnerName()); + if(scannedCount != null) + totalEntriesScanned += scannedCount.get(); + permitOwners.remove(getOwnerName()); } } public List getPermitOwners() { List ownerList = new ArrayList(); synchronized(permitOwners) { - Iterator i = this.permitOwners.iterator(); + Iterator i = this.permitOwners.keySet().iterator(); while(i.hasNext()) ownerList.add(i.next()); } return ownerList; } - public boolean tryAcquire() { + public boolean tryAcquire(AtomicLong progress) { boolean gotPermit = this.scanPermits.tryAcquire(); if(gotPermit) { synchronized(permitOwners) { - permitOwners.add(Thread.currentThread().getStackTrace()[2].getClassName()); + permitOwners.put(getOwnerName(), progress); } } return gotPermit; @@ -53,4 +67,22 @@ public boolean tryAcquire() { public int availablePermits() { return this.scanPermits.availablePermits(); } + + public int getGrantedPermits() { + return numPermits - availablePermits(); + } + + public long getEntriesScanned() { + long itemsScanned = 0; + synchronized(permitOwners) { + for(Map.Entry progressEntry: permitOwners.entrySet()) { + AtomicLong progress = progressEntry.getValue(); + // slops are not included since they are tracked separately + if(progress != null) { + itemsScanned += progress.get(); + } + } + } + return totalEntriesScanned + itemsScanned; + } } diff --git a/src/java/voldemort/server/storage/StorageService.java b/src/java/voldemort/server/storage/StorageService.java index 8bb3919c6f..f941160d68 100644 --- a/src/java/voldemort/server/storage/StorageService.java +++ b/src/java/voldemort/server/storage/StorageService.java @@ -591,6 +591,9 @@ private void scheduleCleanupJob(StoreDefinition storeDef, * Time.MS_PER_DAY, SystemTime.INSTANCE, throttler); + if(voldemortConfig.isJmxEnabled()) { + JmxUtils.registerMbean("DataCleanupJob-" + engine.getName(), cleanupJob); + } this.scheduler.schedule("cleanup-" + storeDef.getName(), cleanupJob, @@ -819,4 +822,14 @@ public DynamicThrottleLimit getDynThrottleLimit() { public List getPermitOwners() { return this.scanPermitWrapper.getPermitOwners(); } + + @JmxGetter(name = "numGrantedScanPermits", description = "Returns number of scan permits granted at the moment") + public long getGrantedPermits() { + return this.scanPermitWrapper.getGrantedPermits(); + } + + @JmxGetter(name = "numEntriesScanned", description = "Returns number of entries scanned since last call") + public long getEntriesScanned() { + return this.scanPermitWrapper.getEntriesScanned(); + } } diff --git a/src/java/voldemort/store/bdb/BdbRuntimeConfig.java b/src/java/voldemort/store/bdb/BdbRuntimeConfig.java index f90040b78d..e568a02049 100644 --- a/src/java/voldemort/store/bdb/BdbRuntimeConfig.java +++ b/src/java/voldemort/store/bdb/BdbRuntimeConfig.java @@ -1,20 +1,23 @@ package voldemort.store.bdb; -import com.sleepycat.je.LockMode; import voldemort.server.VoldemortConfig; import voldemort.utils.Time; +import com.sleepycat.je.LockMode; + /** * Runtime (i.e., post Environment creation) configuration for BdbStorageEngine - * + * */ public class BdbRuntimeConfig { public static final long DEFAULT_STATS_CACHE_TTL_MS = 5 * Time.MS_PER_SECOND; public static final LockMode DEFAULT_LOCK_MODE = LockMode.READ_UNCOMMITTED; + public static final boolean DEFAULT_EXPOSE_SPACE_UTIL = true; private long statsCacheTtlMs = DEFAULT_STATS_CACHE_TTL_MS; private LockMode lockMode = DEFAULT_LOCK_MODE; + private boolean exposeSpaceUtil = DEFAULT_EXPOSE_SPACE_UTIL; public BdbRuntimeConfig() { @@ -22,9 +25,10 @@ public BdbRuntimeConfig() { public BdbRuntimeConfig(VoldemortConfig config) { LockMode lockMode = config.getBdbReadUncommitted() ? LockMode.READ_UNCOMMITTED - : LockMode.DEFAULT; + : LockMode.DEFAULT; setLockMode(lockMode); setStatsCacheTtlMs(config.getBdbStatsCacheTtlMs()); + setExposeSpaceUtil(config.getBdbExposeSpaceUtilization()); } public long getStatsCacheTtlMs() { @@ -44,4 +48,12 @@ public BdbRuntimeConfig setLockMode(LockMode lockMode) { this.lockMode = lockMode; return this; } + + public void setExposeSpaceUtil(boolean expose) { + this.exposeSpaceUtil = expose; + } + + public boolean getExposeSpaceUtil() { + return this.exposeSpaceUtil; + } } diff --git a/src/java/voldemort/store/bdb/BdbStorageConfiguration.java b/src/java/voldemort/store/bdb/BdbStorageConfiguration.java index b5b0b42c06..b59d029cfa 100644 --- a/src/java/voldemort/store/bdb/BdbStorageConfiguration.java +++ b/src/java/voldemort/store/bdb/BdbStorageConfiguration.java @@ -31,17 +31,16 @@ import voldemort.utils.ByteArray; import voldemort.utils.Time; +import com.google.common.collect.Maps; import com.sleepycat.je.Database; import com.sleepycat.je.DatabaseConfig; import com.sleepycat.je.DatabaseException; +import com.sleepycat.je.Durability; import com.sleepycat.je.Environment; import com.sleepycat.je.EnvironmentConfig; import com.sleepycat.je.EnvironmentStats; -import com.sleepycat.je.PreloadConfig; import com.sleepycat.je.StatsConfig; -import com.google.common.collect.Maps; - /** * The configuration that is shared between berkeley db instances. This includes * the db environment and the configuration @@ -70,13 +69,11 @@ public BdbStorageConfiguration(VoldemortConfig config) { environmentConfig.setTransactional(true); environmentConfig.setCacheSize(config.getBdbCacheSize()); if(config.isBdbWriteTransactionsEnabled() && config.isBdbFlushTransactionsEnabled()) { - environmentConfig.setTxnNoSync(false); - environmentConfig.setTxnWriteNoSync(false); + environmentConfig.setDurability(Durability.COMMIT_SYNC); } else if(config.isBdbWriteTransactionsEnabled() && !config.isBdbFlushTransactionsEnabled()) { - environmentConfig.setTxnNoSync(false); - environmentConfig.setTxnWriteNoSync(true); + environmentConfig.setDurability(Durability.COMMIT_WRITE_NO_SYNC); } else { - environmentConfig.setTxnNoSync(true); + environmentConfig.setDurability(Durability.COMMIT_NO_SYNC); } environmentConfig.setAllowCreate(true); environmentConfig.setConfigParam(EnvironmentConfig.LOG_FILE_MAX, diff --git a/src/java/voldemort/store/bdb/BdbStorageEngine.java b/src/java/voldemort/store/bdb/BdbStorageEngine.java index 61d809ee96..290aac61e7 100644 --- a/src/java/voldemort/store/bdb/BdbStorageEngine.java +++ b/src/java/voldemort/store/bdb/BdbStorageEngine.java @@ -102,7 +102,9 @@ public Version toObject(byte[] bytes) { }; this.isOpen = new AtomicBoolean(true); this.readLockMode = config.getLockMode(); - this.bdbEnvironmentStats = new BdbEnvironmentStats(environment, config.getStatsCacheTtlMs()); + this.bdbEnvironmentStats = new BdbEnvironmentStats(environment, + config.getStatsCacheTtlMs(), + config.getExposeSpaceUtil()); } public String getName() { diff --git a/src/java/voldemort/store/bdb/stats/BdbEnvironmentStats.java b/src/java/voldemort/store/bdb/stats/BdbEnvironmentStats.java index 487d961021..1b97dbd5d3 100644 --- a/src/java/voldemort/store/bdb/stats/BdbEnvironmentStats.java +++ b/src/java/voldemort/store/bdb/stats/BdbEnvironmentStats.java @@ -1,24 +1,27 @@ package voldemort.store.bdb.stats; +import java.util.concurrent.Callable; -import com.sleepycat.je.Environment; -import com.sleepycat.je.EnvironmentConfig; -import com.sleepycat.je.EnvironmentStats; -import com.sleepycat.je.StatsConfig; import voldemort.VoldemortException; import voldemort.annotations.Experimental; import voldemort.annotations.jmx.JmxGetter; import voldemort.utils.CachedCallable; -import java.util.concurrent.Callable; +import com.sleepycat.je.Environment; +import com.sleepycat.je.EnvironmentConfig; +import com.sleepycat.je.EnvironmentStats; +import com.sleepycat.je.StatsConfig; public class BdbEnvironmentStats { private final Environment environment; private final CachedCallable fastStats; + private final CachedCallable fastSpaceStats; + private final boolean exposeSpaceStats; - public BdbEnvironmentStats(Environment environment, long ttlMs) { + public BdbEnvironmentStats(Environment environment, long ttlMs, boolean exposeSpaceUtil) { this.environment = environment; + this.exposeSpaceStats = exposeSpaceUtil; Callable fastStatsCallable = new Callable() { public EnvironmentStats call() throws Exception { @@ -26,6 +29,14 @@ public EnvironmentStats call() throws Exception { } }; fastStats = new CachedCallable(fastStatsCallable, ttlMs); + + Callable fastDbStatsCallable = new Callable() { + + public SpaceUtilizationStats call() throws Exception { + return getSpaceUtilizationStats(); + } + }; + fastSpaceStats = new CachedCallable(fastDbStatsCallable, ttlMs); } private EnvironmentStats getEnvironmentStats(boolean fast) { @@ -34,6 +45,18 @@ private EnvironmentStats getEnvironmentStats(boolean fast) { return environment.getStats(config); } + private SpaceUtilizationStats getSpaceUtilizationStats() { + return new SpaceUtilizationStats(environment); + } + + private SpaceUtilizationStats getFastSpaceUtilizationStats() { + try { + return fastSpaceStats.call(); + } catch(Exception e) { + throw new VoldemortException(e); + } + } + private EnvironmentStats getFastStats() { try { return fastStats.call(); @@ -47,6 +70,8 @@ public String getFastStatsAsString() { return getFastStats().toString(); } + // 1. Caching + @JmxGetter(name = "NumCacheMiss") public long getNumCacheMiss() { return getFastStats().getNCacheMiss(); @@ -57,6 +82,22 @@ public long getNumNotResident() { return getFastStats().getNNotResident(); } + @JmxGetter(name = "TotalCacheSize") + public long getTotalCacheSize() { + return getFastStats().getSharedCacheTotalBytes(); + } + + @JmxGetter(name = "AllotedCacheSize") + public long getAllotedCacheSize() { + return getFastStats().getCacheTotalBytes(); + } + + @JmxGetter(name = "EvictionPasses") + public long getEvictedLNs() { + return getFastStats().getNEvictPasses(); + } + + // 2. IO @JmxGetter(name = "NumRandomWrites") public long getNumRandomWrites() { return getFastStats().getNRandomWrites(); @@ -97,6 +138,18 @@ public long getNumSequentialReadBytes() { return getFastStats().getNSequentialReadBytes(); } + @JmxGetter(name = "NumFSyncs") + public long getNumFSyncs() { + return getFastStats().getNFSyncs(); + } + + // 3. Cleaning & Checkpointing + + @JmxGetter(name = "NumCleanerEntriesRead") + public long getNumCleanerEntriesRead() { + return getFastStats().getNCleanerEntriesRead(); + } + @JmxGetter(name = "FileDeletionBacklog") public long getFileDeletionBacklog() { return getFastStats().getFileDeletionBacklog(); @@ -115,14 +168,14 @@ public long getCleanerBacklog() { return getFastStats().getCleanerBacklog(); } - @JmxGetter(name = "NumAcquiresWithContention") - public long getNumAcquiresWithContention() { - return getFastStats().getNAcquiresWithContention(); + @JmxGetter(name = "NumCleanerRuns") + public long getNumCleanerRuns() { + return getFastStats().getNCleanerRuns(); } - @JmxGetter(name = "NumAcquiresNoWaiters") - public long getNumAcquiresNoWaiters() { - return getFastStats().getNAcquiresNoWaiters(); + @JmxGetter(name = "NumCleanerDeletions") + public long getNumCleanerDeletions() { + return getFastStats().getNCleanerRuns(); } @JmxGetter(name = "NumCheckpoints") @@ -130,42 +183,65 @@ public long getNumCheckpoints() { return getFastStats().getNCheckpoints(); } - @JmxGetter(name = "NumCleanerEntriesRead") - public long getNumCleanerEntriesRead() { - return getFastStats().getNCleanerEntriesRead(); + @JmxGetter(name = "TotalSpace") + public long getTotalSpace() { + if(this.exposeSpaceStats) + return getFastSpaceUtilizationStats().getTotalSpaceUsed(); + else + return 0; } - @JmxGetter(name = "NumFSyncs") - public long getNumFSyncs() { - return getFastStats().getNFSyncs(); + @JmxGetter(name = "TotalSpaceUtilized") + public long getTotalSpaceUtilized() { + if(this.exposeSpaceStats) + return getFastSpaceUtilizationStats().getTotalSpaceUtilized(); + else + return 0; } - @JmxGetter(name = "NumCleanerRuns") - public long getNumCleanerRuns() { - return getFastStats().getNCleanerRuns(); + @JmxGetter(name = "UtilizationSummary", description = "Displays the disk space utilization for an environment.") + public String getUtilizationSummaryAsString() { + return getFastSpaceUtilizationStats().getSummariesAsString(); } - @JmxGetter(name = "NumCleanerDeletions") - public long getNumCleanerDeletions() { - return getFastStats().getNCleanerRuns(); + // 4. Latching/Locking + + @JmxGetter(name = "BtreeLatches") + public long getBtreeLatches() { + return getFastStats().getRelatchesRequired(); + } + + @JmxGetter(name = "NumAcquiresWithContention") + public long getNumAcquiresWithContention() { + return getFastStats().getNAcquiresWithContention(); + } + + @JmxGetter(name = "NumAcquiresNoWaiters") + public long getNumAcquiresNoWaiters() { + return getFastStats().getNAcquiresNoWaiters(); } - // Compound statistics + // Compound statistics derived from raw statistics @JmxGetter(name = "NumWritesTotal") public long getNumWritesTotal() { return getNumRandomWrites() + getNumSequentialWrites(); } + @JmxGetter(name = "NumWriteBytesTotal") + public long getNumWriteBytesTotal() { + return getNumSequentialWriteBytes() + getNumRandomWriteBytes(); + } + @JmxGetter(name = "PercentRandomWrites") public double getPercentRandomWrites() { - return safeGetPercentage(getNumRandomWrites(), getNumWritesTotal()); + return safeGetPercentage(getNumRandomWrites(), getNumWritesTotal()); } @JmxGetter(name = "PercentageRandomWriteBytes") public double getPercentageRandomWriteBytes() { - return safeGetPercentage(getNumRandomWriteBytes(), getNumRandomWriteBytes() + - getNumSequentialWriteBytes()); + return safeGetPercentage(getNumRandomWriteBytes(), getNumRandomWriteBytes() + + getNumSequentialWriteBytes()); } @JmxGetter(name = "NumReadsTotal") @@ -173,6 +249,11 @@ public long getNumReadsTotal() { return getNumRandomReads() + getNumSequentialReads(); } + @JmxGetter(name = "NumReadBytesTotal") + public long getNumReadBytesTotal() { + return getNumRandomReadBytes() + getNumSequentialReadBytes(); + } + @JmxGetter(name = "PercentageRandomReads") public double getPercentageRandomReads() { return safeGetPercentage(getNumRandomReads(), getNumReadsTotal()); @@ -180,8 +261,19 @@ public double getPercentageRandomReads() { @JmxGetter(name = "PercentageRandomReadBytes") public double getPercentageRandomReadBytes() { - return safeGetPercentage(getNumRandomWriteBytes(), getNumRandomReadBytes() + - getNumSequentialReadBytes()); + return safeGetPercentage(getNumRandomWriteBytes(), getNumRandomReadBytes() + + getNumSequentialReadBytes()); + } + + @JmxGetter(name = "PercentageReads") + public double getPercentageReads() { + return safeGetPercentage(getNumReadsTotal(), getNumReadsTotal() + getNumWritesTotal()); + } + + @JmxGetter(name = "PercentageReadBytes") + public double getPercentageReadBytes() { + return safeGetPercentage(getNumReadBytesTotal(), getNumWriteBytesTotal() + + getNumReadBytesTotal()); } @Experimental @@ -193,17 +285,21 @@ public double getPercentageCacheHits() { @Experimental @JmxGetter(name = "PercentageCacheMisses") public double getPercentageCacheMisses() { - return safeGetPercentage(getNumCacheMiss(), - getNumReadsTotal() + getNumWritesTotal()); + return safeGetPercentage(getNumCacheMiss(), getNumReadsTotal() + getNumWritesTotal()); } @JmxGetter(name = "PercentageContended") public double getPercentageContended() { - return safeGetPercentage(getNumAcquiresWithContention(), - getNumAcquiresWithContention() + getNumAcquiresNoWaiters()); + return safeGetPercentage(getNumAcquiresWithContention(), getNumAcquiresWithContention() + + getNumAcquiresNoWaiters()); + } + + @JmxGetter(name = "PercentageUtilization") + public double getPercentageUtilization() { + return safeGetPercentage(getTotalSpaceUtilized(), getTotalSpace()); } public static double safeGetPercentage(long rawNum, long total) { - return total == 0 ? 0.0d : rawNum / (float)total; + return total == 0 ? 0.0d : rawNum / (float) total; } } diff --git a/src/java/voldemort/store/bdb/stats/SpaceUtilizationStats.java b/src/java/voldemort/store/bdb/stats/SpaceUtilizationStats.java new file mode 100644 index 0000000000..f741f34731 --- /dev/null +++ b/src/java/voldemort/store/bdb/stats/SpaceUtilizationStats.java @@ -0,0 +1,70 @@ +package voldemort.store.bdb.stats; + +import java.util.Iterator; +import java.util.Map; +import java.util.SortedMap; + +import com.sleepycat.je.DbInternal; +import com.sleepycat.je.Environment; +import com.sleepycat.je.cleaner.FileSummary; +import com.sleepycat.je.cleaner.UtilizationProfile; +import com.sleepycat.je.dbi.EnvironmentImpl; + +/** + * Obtains the disk space utilization for the BDB environment + */ +final public class SpaceUtilizationStats { + + private final EnvironmentImpl envImpl; + + private SortedMap summaryMap; + private long totalSpaceUsed = 0; + private long totalSpaceUtilized = 0; + + public SpaceUtilizationStats(Environment env) { + this(DbInternal.getEnvironmentImpl(env)); + } + + private SpaceUtilizationStats(EnvironmentImpl envImpl) { + this.envImpl = envImpl; + UtilizationProfile profile = this.envImpl.getUtilizationProfile(); + summaryMap = profile.getFileSummaryMap(true); + + Iterator> fileItr = summaryMap.entrySet().iterator(); + while(fileItr.hasNext()) { + Map.Entry entry = fileItr.next(); + FileSummary fs = entry.getValue(); + totalSpaceUsed += fs.totalSize; + totalSpaceUtilized += fs.totalSize - fs.getObsoleteSize(); + } + } + + public long getTotalSpaceUsed() { + return totalSpaceUsed; + } + + public long getTotalSpaceUtilized() { + return totalSpaceUtilized; + } + + public String getSummariesAsString() { + StringBuffer summaryDetails = new StringBuffer(); + if(summaryMap != null) { + summaryDetails.append("file,util%\n"); + Iterator> fileItr = summaryMap.entrySet().iterator(); + while(fileItr.hasNext()) { + Map.Entry entry = fileItr.next(); + FileSummary fs = entry.getValue(); + long bytesUsed = fs.totalSize - fs.getObsoleteSize(); + summaryDetails.append(String.format("%s,%f\n", + Long.toHexString(entry.getKey().longValue()), + BdbEnvironmentStats.safeGetPercentage(bytesUsed, + fs.totalSize))); + } + summaryDetails.append(String.format("total,%f\n", + BdbEnvironmentStats.safeGetPercentage(totalSpaceUtilized, + totalSpaceUsed))); + } + return summaryDetails.toString(); + } +} \ No newline at end of file diff --git a/src/java/voldemort/utils/JNAUtils.java b/src/java/voldemort/utils/JNAUtils.java new file mode 100644 index 0000000000..3625c70fac --- /dev/null +++ b/src/java/voldemort/utils/JNAUtils.java @@ -0,0 +1,84 @@ +package voldemort.utils; + +import org.apache.log4j.Logger; + +import com.sun.jna.LastErrorException; +import com.sun.jna.Native; + +/** + * Native functions used through JNA + * + */ +public class JNAUtils { + + private static final Logger logger = Logger.getLogger(JNAUtils.class); + + /* Flags for mlock_all */ + private static final int MCL_CURRENT = 1; + private static final int MCL_FUTURE = 2; + + private static final int ENOMEM = 12; + + static { + try { + Native.register("c"); + } catch(NoClassDefFoundError e) { + logger.info("Could not locate JNA classes"); + } catch(UnsatisfiedLinkError e) { + logger.info("Failed to link to native library"); + } catch(NoSuchMethodError e) { + logger.warn("Older version of JNA. Please upgrade to 3.2.7+"); + } + } + + private static native int mlockall(int flags) throws LastErrorException; + + private static native int munlockall() throws LastErrorException; + + private static boolean isOperatingSystem(String os) { + if(System.getProperty("os.name").toLowerCase().contains(os)) + return true; + else + return false; + } + + public static void tryMlockall() { + try { + if(isOperatingSystem("windows")) + return; + // Since we demand-zero every page of the heap while bringing up the + // jvm, MCL_FUTURE is not needed + mlockall(MCL_CURRENT); + logger.info("mlockall() on JVM Heap successful"); + } catch(Exception e) { + if(!(e instanceof LastErrorException)) + logger.error("Unexpected error during mlock of server heap", e); + + LastErrorException le = (LastErrorException) e; + if(le.getErrorCode() == ENOMEM && isOperatingSystem("linux")) { + logger.warn("Unable to lock JVM memory (ENOMEM)." + + " This can result in part of the JVM being swapped out with higher Young gen stalls" + + " Increase RLIMIT_MEMLOCK or run Voldemort as root."); + } else if(!isOperatingSystem("mac")) { + // fixes a OS X oddity, where it still throws an error, even + // though mlockall succeeds + logger.warn("Unknown mlockall error " + le.getErrorCode()); + } + } + } + + public static void tryMunlockall() { + try { + if(isOperatingSystem("windows")) + return; + munlockall(); + logger.info("munlockall() on JVM Heap successful"); + } catch(Exception e) { + if(!(e instanceof LastErrorException)) + logger.error("Unexpected error during mlock of server heap", e); + LastErrorException le = (LastErrorException) e; + logger.warn("Error unlocking JVM heap " + le.getErrorCode()); + } + } + +} diff --git a/test/integration/voldemort/CatBdbStore.java b/test/integration/voldemort/CatBdbStore.java index 10655b5660..44b170fa8d 100644 --- a/test/integration/voldemort/CatBdbStore.java +++ b/test/integration/voldemort/CatBdbStore.java @@ -33,9 +33,9 @@ import com.sleepycat.je.Database; import com.sleepycat.je.DatabaseConfig; +import com.sleepycat.je.Durability; import com.sleepycat.je.Environment; import com.sleepycat.je.EnvironmentConfig; -import com.sleepycat.je.LockMode; public class CatBdbStore { @@ -51,7 +51,7 @@ public static void main(String[] args) throws Exception { VoldemortConfig config = new VoldemortConfig(new Props(new File(serverProperties))); EnvironmentConfig environmentConfig = new EnvironmentConfig(); - environmentConfig.setTxnNoSync(true); + environmentConfig.setDurability(Durability.COMMIT_NO_SYNC); environmentConfig.setAllowCreate(true); environmentConfig.setTransactional(config.isBdbWriteTransactionsEnabled()); Environment environment = new Environment(new File(bdbDir), environmentConfig); diff --git a/test/integration/voldemort/performance/BdbGrowth.java b/test/integration/voldemort/performance/BdbGrowth.java index 06dca53d1a..cf4574676f 100644 --- a/test/integration/voldemort/performance/BdbGrowth.java +++ b/test/integration/voldemort/performance/BdbGrowth.java @@ -29,6 +29,7 @@ import com.sleepycat.je.Database; import com.sleepycat.je.DatabaseConfig; import com.sleepycat.je.DatabaseEntry; +import com.sleepycat.je.Durability; import com.sleepycat.je.Environment; import com.sleepycat.je.EnvironmentConfig; @@ -56,7 +57,7 @@ public static void main(String[] args) throws Exception { environmentConfig = new EnvironmentConfig(); environmentConfig.setCacheSize(cacheSize); - environmentConfig.setTxnNoSync(true); + environmentConfig.setDurability(Durability.COMMIT_NO_SYNC); environmentConfig.setConfigParam(EnvironmentConfig.LOG_FILE_MAX, "1000000000"); environmentConfig.setConfigParam(EnvironmentConfig.CLEANER_MAX_BATCH_FILES, "100"); environmentConfig.setConfigParam(EnvironmentConfig.CLEANER_READ_SIZE, "52428800"); diff --git a/test/unit/voldemort/store/bdb/BdbSplitStorageEngineTest.java b/test/unit/voldemort/store/bdb/BdbSplitStorageEngineTest.java index ffce2d815b..32fbebcc28 100644 --- a/test/unit/voldemort/store/bdb/BdbSplitStorageEngineTest.java +++ b/test/unit/voldemort/store/bdb/BdbSplitStorageEngineTest.java @@ -30,10 +30,10 @@ import com.sleepycat.je.Database; import com.sleepycat.je.DatabaseConfig; import com.sleepycat.je.DatabaseException; +import com.sleepycat.je.Durability; import com.sleepycat.je.Environment; import com.sleepycat.je.EnvironmentConfig; import com.sleepycat.je.EnvironmentStats; -import com.sleepycat.je.LockMode; import com.sleepycat.je.StatsConfig; /** @@ -159,7 +159,7 @@ public void testMultipleEnvironment() { public void testUnsharedCache() throws DatabaseException { EnvironmentConfig environmentConfig = new EnvironmentConfig(); environmentConfig = new EnvironmentConfig(); - environmentConfig.setTxnNoSync(true); + environmentConfig.setDurability(Durability.COMMIT_NO_SYNC); environmentConfig.setAllowCreate(true); environmentConfig.setTransactional(true); environmentConfig.setSharedCache(false); @@ -178,7 +178,7 @@ public void testUnsharedCache() throws DatabaseException { public void testSharedCache() throws DatabaseException { EnvironmentConfig environmentConfig = new EnvironmentConfig(); - environmentConfig.setTxnNoSync(true); + environmentConfig.setDurability(Durability.COMMIT_NO_SYNC); environmentConfig.setAllowCreate(true); environmentConfig.setTransactional(true); environmentConfig.setSharedCache(true); @@ -201,7 +201,10 @@ private long getMaxCacheUsage(EnvironmentConfig environmentConfig, DatabaseConfi } Environment environmentA = new Environment(dirA, environmentConfig); Database databaseA = environmentA.openDatabase(null, "storeA", databaseConfig); - BdbStorageEngine storeA = new BdbStorageEngine("storeA", environmentA, databaseA, new BdbRuntimeConfig()); + BdbStorageEngine storeA = new BdbStorageEngine("storeA", + environmentA, + databaseA, + new BdbRuntimeConfig()); File dirB = new File(bdbMasterDir + "/" + "storeB"); if(!dirB.exists()) { @@ -209,7 +212,10 @@ private long getMaxCacheUsage(EnvironmentConfig environmentConfig, DatabaseConfi } Environment environmentB = new Environment(dirB, environmentConfig); Database databaseB = environmentB.openDatabase(null, "storeB", databaseConfig); - BdbStorageEngine storeB = new BdbStorageEngine("storeB", environmentB, databaseB, new BdbRuntimeConfig()); + BdbStorageEngine storeB = new BdbStorageEngine("storeB", + environmentB, + databaseB, + new BdbRuntimeConfig()); long maxCacheUsage = 0; for(int i = 0; i <= 4; i++) { diff --git a/test/unit/voldemort/store/bdb/BdbStorageEngineTest.java b/test/unit/voldemort/store/bdb/BdbStorageEngineTest.java index 1fbb7a3d55..d0289282fc 100644 --- a/test/unit/voldemort/store/bdb/BdbStorageEngineTest.java +++ b/test/unit/voldemort/store/bdb/BdbStorageEngineTest.java @@ -42,6 +42,7 @@ import com.sleepycat.je.Database; import com.sleepycat.je.DatabaseConfig; +import com.sleepycat.je.Durability; import com.sleepycat.je.Environment; import com.sleepycat.je.EnvironmentConfig; import com.sleepycat.je.LockMode; @@ -62,7 +63,7 @@ public class BdbStorageEngineTest extends AbstractStorageEngineTest { protected void setUp() throws Exception { super.setUp(); this.envConfig = new EnvironmentConfig(); - this.envConfig.setTxnNoSync(true); + this.envConfig.setDurability(Durability.COMMIT_NO_SYNC); this.envConfig.setAllowCreate(true); this.envConfig.setTransactional(true); this.tempDir = TestUtils.createTempDir(); From 05f23ec16e01f7a19c3cf8a9d0794a35136d1310 Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Mon, 9 Jul 2012 18:07:13 -0700 Subject: [PATCH 088/209] code clean up --- src/java/voldemort/VoldemortAdminTool.java | 20 ++++----- .../client/AbstractStoreClientFactory.java | 2 +- src/java/voldemort/client/ClientConfig.java | 45 ++++++++++++++++++- src/java/voldemort/client/ClientInfo.java | 21 +++++++++ .../voldemort/client/DefaultStoreClient.java | 16 +++---- .../voldemort/client/StoreClientFactory.java | 15 ------- .../scheduler/ClientRegistryRefresher.java | 42 ++++++++++++++++- .../store/system/SystemStoreConstants.java | 24 +++++++--- .../voldemort/utils/ManifestFileReader.java | 20 +++++++++ .../client/DefaultSocketStoreClientTest.java | 15 ------- 10 files changed, 162 insertions(+), 58 deletions(-) diff --git a/src/java/voldemort/VoldemortAdminTool.java b/src/java/voldemort/VoldemortAdminTool.java index 7aec774c57..baac31465a 100644 --- a/src/java/voldemort/VoldemortAdminTool.java +++ b/src/java/voldemort/VoldemortAdminTool.java @@ -998,9 +998,9 @@ private static void executeFetchEntries(Integer nodeId, stores = Lists.newArrayList(); stores.addAll(storeDefinitionMap.keySet()); } else { - // add system store to the map so they can be fetched when specified - // explicitly - storeDefinitionMap.putAll(getSystemStoreDef()); + // add system stores to the map so they can be fetched when + // specified explicitly + storeDefinitionMap.putAll(getSystemStoreDefs()); } // Pick up all the partitions @@ -1045,13 +1045,13 @@ private static void executeFetchEntries(Integer nodeId, } } - private static Map getSystemStoreDef() { - Map sysStoreDef = Maps.newHashMap(); + private static Map getSystemStoreDefs() { + Map sysStoreDefMap = Maps.newHashMap(); List storesDefs = SystemStoreConstants.getAllSystemStoreDefs(); for(StoreDefinition def: storesDefs) { - sysStoreDef.put(def.getName(), def); + sysStoreDefMap.put(def.getName(), def); } - return sysStoreDef; + return sysStoreDefMap; } private static void executeUpdateEntries(Integer nodeId, @@ -1251,9 +1251,9 @@ private static void executeFetchKeys(Integer nodeId, stores = Lists.newArrayList(); stores.addAll(storeDefinitionMap.keySet()); } else { - // add system store to the map so they can be fetched when specified - // explicitly - storeDefinitionMap.putAll(getSystemStoreDef()); + // add system stores to the map so they can be fetched when + // specified explicitly + storeDefinitionMap.putAll(getSystemStoreDefs()); } // Pick up all the partitions diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index af455040a5..cc14819937 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -458,7 +458,7 @@ public String jmxId() { /** * Generate a unique client ID based on: 0. clientContext, if specified; 1. - * storeName 2. run path 3. client sequence + * storeName; 2. deployment path; 3. client sequence * * @param storeName the name of the store the client is created for * @param contextName the name of the client context diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index 8d8e27d755..04b8b41e0b 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -84,6 +84,7 @@ public class ClientConfig { private volatile long asyncCheckMetadataInterval = 5000; /* 12 hr refresh internval, in seconds */ private volatile int clientRegistryRefreshInterval = 3600 * 12; + private volatile int asyncJobThreadPoolSize = 2; public ClientConfig() {} @@ -126,6 +127,8 @@ public ClientConfig() {} public static final String MAX_BOOTSTRAP_RETRIES = "max_bootstrap_retries"; public static final String CLIENT_CONTEXT_NAME = "voldemort_client_context"; public static final String ASYNC_CHECK_METADATA_INTERVAL = "check_metadata_interval"; + public static final String CLIENT_REGISTRY_REFRESH_INTERVAL = "client_registry_refresh_interval"; + public static final String ASYNC_JOB_THREAD_POOL_SIZE = "async_job_thread_pool_size"; /** * Instantiate the client config using a properties file @@ -286,11 +289,19 @@ private void setProperties(Properties properties) { this.setMaxBootstrapRetries(props.getInt(MAX_BOOTSTRAP_RETRIES)); if(props.containsKey(CLIENT_CONTEXT_NAME)) { - this.setClientContextName(props.getString(CLIENT_CONTEXT_NAME, null)); + this.setClientContextName(props.getString(CLIENT_CONTEXT_NAME)); } if(props.containsKey(ASYNC_CHECK_METADATA_INTERVAL)) { - this.setAsyncCheckMetadataInterval(props.getLong(ASYNC_CHECK_METADATA_INTERVAL, 5000)); + this.setAsyncCheckMetadataInterval(props.getLong(ASYNC_CHECK_METADATA_INTERVAL)); + } + + if(props.containsKey(CLIENT_REGISTRY_REFRESH_INTERVAL)) { + this.setClientRegistryRefreshInterval(props.getInt(CLIENT_REGISTRY_REFRESH_INTERVAL)); + } + + if(props.containsKey(ASYNC_JOB_THREAD_POOL_SIZE)) { + this.setClientRegistryRefreshInterval(props.getInt(ASYNC_JOB_THREAD_POOL_SIZE)); } } @@ -712,6 +723,11 @@ public String getClientContextName() { return clientContextName; } + /** + * Set the client context name + * + * @param clientContextName The name of client context + */ public ClientConfig setClientContextName(String clientContextName) { this.clientContextName = clientContextName; return this; @@ -721,6 +737,11 @@ public long getAsyncCheckMetadataInterval() { return asyncCheckMetadataInterval; } + /** + * Set the interval on which client checks for metadata change on servers + * + * @param asyncCheckMetadataInterval The metadata change interval + */ public ClientConfig setAsyncCheckMetadataInterval(long asyncCheckMetadataInterval) { this.asyncCheckMetadataInterval = asyncCheckMetadataInterval; return this; @@ -730,8 +751,28 @@ public int getClientRegistryRefreshInterval() { return this.clientRegistryRefreshInterval; } + /** + * Set the interval on which client refreshes its corresponding entry of the + * client registry on the servers + * + * @param clientRegistryRefreshInterval The refresh interval in seconds + */ public ClientConfig setClientRegistryRefreshInterval(int clientRegistryRefrshInterval) { this.clientRegistryRefreshInterval = clientRegistryRefrshInterval; return this; } + + public int getAsyncJobThreadPoolSize() { + return asyncJobThreadPoolSize; + } + + /** + * Set the # of threads for the async. job thread pool + * + * @param asyncJobThreadPoolSize The max # of threads in the async job + */ + public ClientConfig setAsyncJobThreadPoolSize(int asyncJobThreadPoolSize) { + this.asyncJobThreadPoolSize = asyncJobThreadPoolSize; + return this; + } } diff --git a/src/java/voldemort/client/ClientInfo.java b/src/java/voldemort/client/ClientInfo.java index b9e92ff225..d87f98b0a8 100644 --- a/src/java/voldemort/client/ClientInfo.java +++ b/src/java/voldemort/client/ClientInfo.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2012 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.client; import java.io.File; @@ -8,6 +24,11 @@ import org.apache.log4j.Logger; +/** + * A collection of voldemort client side information what will be populated into + * the voldemort cluster when a client is connected to a voldemort cluster + * + */ public class ClientInfo implements Serializable { /** diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index ccd8749890..3d507e71f7 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -21,9 +21,10 @@ import java.util.GregorianCalendar; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.UUID; +import java.util.Map.Entry; import java.util.concurrent.Callable; +import java.util.concurrent.TimeUnit; import org.apache.log4j.Logger; @@ -69,9 +70,6 @@ @JmxManaged(description = "A voldemort client") public class DefaultStoreClient implements StoreClient { - private static final int ASYNC_THREADS_COUNT = 2; - private static final boolean ALLOW_INTERRUPT_ASYNC = true; - private final Logger logger = Logger.getLogger(DefaultStoreClient.class); private final StoreClientFactory storeFactory; @@ -114,9 +112,9 @@ public DefaultStoreClient(String storeName, this.clientId = AbstractStoreClientFactory.generateClientId(clientInfo); this.config = config; this.sysRepository = new SystemStoreRepository(); - this.scheduler = new SchedulerService(ASYNC_THREADS_COUNT, + this.scheduler = new SchedulerService(config.getAsyncJobThreadPoolSize(), SystemTime.INSTANCE, - ALLOW_INTERRUPT_ASYNC); + true); // Registering self to be able to bootstrap client dynamically via JMX JmxUtils.registerMbean(this, JmxUtils.createObjectName(JmxUtils.getPackageName(this.getClass()), @@ -152,15 +150,15 @@ private void registerClient(String jobId, int interval) { scheduler.schedule(jobId + refresher.getClass().getName(), refresher, cal.getTime(), - interval * 1000); - logger.info("Client registry refresher thread started, refresh frequency: " + TimeUnit.MILLISECONDS.convert(interval, TimeUnit.SECONDS)); + logger.info("Client registry refresher thread started, refresh interval: " + interval + " seconds"); } catch(Exception e) { logger.warn("Unable to register with the cluster due to the following error:", e); } } else { logger.warn(SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name() - + "not found. Unable to registry with voldemort cluster."); + + " not found. Unable to registry with voldemort cluster."); } } diff --git a/src/java/voldemort/client/StoreClientFactory.java b/src/java/voldemort/client/StoreClientFactory.java index 5cff508ffa..71ebbef7dd 100644 --- a/src/java/voldemort/client/StoreClientFactory.java +++ b/src/java/voldemort/client/StoreClientFactory.java @@ -87,13 +87,6 @@ Store getRawStore(String storeName, UUID clientId); /** -<<<<<<< HEAD -<<<<<<< HEAD -<<<<<<< HEAD -======= ->>>>>>> Adding System store functionality -======= ->>>>>>> leigao/client-registry * Get the underlying store, not the public StoreClient interface * * @param storeName The name of the store @@ -109,14 +102,6 @@ Store getRawStore(String storeName, String clusterXmlString); /** -<<<<<<< HEAD -<<<<<<< HEAD -======= ->>>>>>> add clientId for voldemort client -======= ->>>>>>> Adding System store functionality -======= ->>>>>>> leigao/client-registry * Close the store client */ public void close(); diff --git a/src/java/voldemort/client/scheduler/ClientRegistryRefresher.java b/src/java/voldemort/client/scheduler/ClientRegistryRefresher.java index 2b57926b49..ed49a600a9 100644 --- a/src/java/voldemort/client/scheduler/ClientRegistryRefresher.java +++ b/src/java/voldemort/client/scheduler/ClientRegistryRefresher.java @@ -1,12 +1,34 @@ +/* + * Copyright 2008-2012 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.client.scheduler; import org.apache.log4j.Logger; import voldemort.client.ClientInfo; import voldemort.client.SystemStore; +import voldemort.versioning.ObsoleteVersionException; import voldemort.versioning.Version; import voldemort.versioning.Versioned; +/** + * An async. job that keeps client registry refreshed while the client is + * connected to the cluster + * + */ public class ClientRegistryRefresher implements Runnable { private final Logger logger = Logger.getLogger(this.getClass()); @@ -15,6 +37,7 @@ public class ClientRegistryRefresher implements Runnable { private ClientInfo clientInfo; private final String clientId; private Version lastVersion; + private boolean hadConflict; public ClientRegistryRefresher(SystemStore clientRegistry, String clientId, @@ -24,10 +47,20 @@ public ClientRegistryRefresher(SystemStore clientRegistry, this.clientInfo = clientInfo; this.clientId = clientId; this.lastVersion = version; + this.hadConflict = false; logger.info("Initial version obtained from client registry: " + version); } public void run() { + + if(hadConflict) { + // if we previously had a conflict during update, we will try to get + // a newer version before update this time. This case shall not + // happen under regular circumstances. But it is just avoid update + // keeping failing when strange situations occur. + lastVersion = clientRegistry.getSysStore(clientId).getVersion(); + hadConflict = false; + } clientInfo.setUpdateTime(System.currentTimeMillis()); logger.info("updating client registry with the following info for client: " + clientId + "\n" + clientInfo); @@ -35,8 +68,15 @@ public void run() { lastVersion = clientRegistry.putSysStore(clientId, new Versioned(clientInfo, lastVersion)); + } catch(ObsoleteVersionException e) { + Versioned existingValue = clientRegistry.getSysStore(clientId); + logger.warn("Multiple clients are updating the same client registry entry"); + logger.warn(" current value: " + clientInfo + " " + lastVersion); + logger.warn(" existing value: " + existingValue.getValue() + " " + + existingValue.getVersion()); + hadConflict = true; } catch(Exception e) { - logger.warn("encounted the following error while trying to update client registry: " + logger.warn("encountered the following error while trying to update client registry: " + e); } } diff --git a/src/java/voldemort/store/system/SystemStoreConstants.java b/src/java/voldemort/store/system/SystemStoreConstants.java index c818ed51fd..3bbb2562d3 100644 --- a/src/java/voldemort/store/system/SystemStoreConstants.java +++ b/src/java/voldemort/store/system/SystemStoreConstants.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2012 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.store.system; import java.io.StringReader; @@ -8,7 +24,9 @@ import voldemort.xml.StoreDefinitionsMapper; /** - * The various system stores + * A file that defines all constants for system stores, including the store + * definitions + * */ public class SystemStoreConstants { @@ -39,10 +57,6 @@ public static enum SystemStoreName { + " utf8" + " " + " " - // + - // " avro-specific" - // + - // " java=voldemort.client.ClientInfo" + " java-serialization" + " " + " 7" diff --git a/src/java/voldemort/utils/ManifestFileReader.java b/src/java/voldemort/utils/ManifestFileReader.java index a3287a3681..8ba26d7515 100644 --- a/src/java/voldemort/utils/ManifestFileReader.java +++ b/src/java/voldemort/utils/ManifestFileReader.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2012 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.utils; import java.io.FileInputStream; @@ -6,6 +22,10 @@ import org.apache.log4j.Logger; +/** + * A utility class that abstract out fields from manifest file + * + */ public class ManifestFileReader { protected static final Logger logger = Logger.getLogger(ManifestFileReader.class); diff --git a/test/unit/voldemort/client/DefaultSocketStoreClientTest.java b/test/unit/voldemort/client/DefaultSocketStoreClientTest.java index 6d2bc94d2e..d5cce3cac7 100644 --- a/test/unit/voldemort/client/DefaultSocketStoreClientTest.java +++ b/test/unit/voldemort/client/DefaultSocketStoreClientTest.java @@ -101,19 +101,4 @@ public void test() { assertNull("After a successful delete(k), get(k) should return null.", client.get("k")); } - @Test - public void testClientRegistryHappyPath() { - ClientConfig clientConfig = new ClientConfig().setMaxThreads(4) - .setMaxTotalConnections(4) - .setMaxConnectionsPerNode(4) - .setBootstrapUrls(socketUrl) - .setClientContextName("testClientRegistryHappyPath"); - SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); - this.client = socketFactory.getStoreClient(testStoreName); - client.put("k", "v"); - adminClient.fetchEntries(0, testStoreName, null, null, false); - adminClient.fetchEntries(1, testStoreName, null, null, false); - // TODO: verify that the values in registry are correct. - } - } From 70718b493e5d2df5cd15cd7077ea94e7dffeaf49 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Mon, 9 Jul 2012 18:34:18 -0700 Subject: [PATCH 089/209] Deleted unnescessary mbean registration --- src/java/voldemort/client/SocketStoreClientFactory.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/java/voldemort/client/SocketStoreClientFactory.java b/src/java/voldemort/client/SocketStoreClientFactory.java index 18fab212d2..c5c7336f5d 100644 --- a/src/java/voldemort/client/SocketStoreClientFactory.java +++ b/src/java/voldemort/client/SocketStoreClientFactory.java @@ -38,7 +38,6 @@ import voldemort.store.socket.SocketStoreFactory; import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; import voldemort.utils.ByteArray; -import voldemort.utils.JmxUtils; import voldemort.versioning.InconsistencyResolver; import voldemort.versioning.Versioned; @@ -69,11 +68,6 @@ public SocketStoreClientFactory(ClientConfig config) { config.getSocketBufferSize(), config.getSocketKeepAlive(), config.isJmxEnabled()); - if(config.isJmxEnabled()) - JmxUtils.registerMbean(storeFactory, - JmxUtils.createObjectName(JmxUtils.getPackageName(storeFactory.getClass()), - JmxUtils.getClassName(storeFactory.getClass()) - + jmxId())); } @Override From fe464194b2d34da476a12761869b44f104b942d8 Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Mon, 9 Jul 2012 20:49:07 -0700 Subject: [PATCH 090/209] avoid jmx id from being incremented when factory is created for system stores --- .../client/AbstractStoreClientFactory.java | 10 +++++++++- src/java/voldemort/client/SystemStore.java | 2 +- .../voldemort/client/SystemStoreClientFactory.java | 14 ++++++++++++++ test/unit/voldemort/client/ClientJmxTest.java | 2 +- 4 files changed, 25 insertions(+), 3 deletions(-) create mode 100644 src/java/voldemort/client/SystemStoreClientFactory.java diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index cc14819937..965cf45849 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -111,7 +111,7 @@ public AbstractStoreClientFactory(ClientConfig config) { this.bootstrapUrls = validateUrls(config.getBootstrapUrls()); this.isJmxEnabled = config.isJmxEnabled(); this.requestFormatType = config.getRequestFormatType(); - this.jmxId = jmxIdCounter.getAndIncrement(); + this.jmxId = getNextJmxId(); this.maxBootstrapRetries = config.getMaxBootstrapRetries(); this.stats = new StoreStats(); this.clientZoneId = config.getClientZoneId(); @@ -137,6 +137,14 @@ public AbstractStoreClientFactory(ClientConfig config) { } } + public int getNextJmxId() { + return jmxIdCounter.getAndIncrement(); + } + + public int getCurrentJmxId() { + return jmxIdCounter.get(); + } + public StoreClient getStoreClient(String storeName) { return getStoreClient(storeName, null); } diff --git a/src/java/voldemort/client/SystemStore.java b/src/java/voldemort/client/SystemStore.java index 66cf23d8f6..da37dfa9fe 100644 --- a/src/java/voldemort/client/SystemStore.java +++ b/src/java/voldemort/client/SystemStore.java @@ -39,7 +39,7 @@ public SystemStore(String storeName, String[] bootstrapUrls, int clientZoneID, S .setEnableJmx(false) .setEnablePipelineRoutedStore(true) .setClientZoneId(clientZoneID); - this.systemStoreFactory = new SocketStoreClientFactory(config); + this.systemStoreFactory = new SystemStoreClientFactory(config); this.storeName = storeName; this.sysStore = this.systemStoreFactory.getSystemStore(this.storeName, clusterXml); } diff --git a/src/java/voldemort/client/SystemStoreClientFactory.java b/src/java/voldemort/client/SystemStoreClientFactory.java new file mode 100644 index 0000000000..2cbd832464 --- /dev/null +++ b/src/java/voldemort/client/SystemStoreClientFactory.java @@ -0,0 +1,14 @@ +package voldemort.client; + +public class SystemStoreClientFactory extends SocketStoreClientFactory { + + public SystemStoreClientFactory(ClientConfig config) { + super(config); + } + + @Override + public int getNextJmxId() { + // for system store, we don't increment jmx id + return getCurrentJmxId(); + } +} diff --git a/test/unit/voldemort/client/ClientJmxTest.java b/test/unit/voldemort/client/ClientJmxTest.java index 1c215580f8..af178aebe8 100644 --- a/test/unit/voldemort/client/ClientJmxTest.java +++ b/test/unit/voldemort/client/ClientJmxTest.java @@ -36,7 +36,7 @@ public ClientJmxTest() { private static String getAndIncrementJmxId() { int current = factoryJmxId; factoryJmxId++; - return (0 == current ? "" : "." + current); + return (0 == current ? "" : "" + current); } @Override From fa3d6f6ff2189a0860ae98535a84cdc5372b168b Mon Sep 17 00:00:00 2001 From: Peter Bailis Date: Tue, 26 Jun 2012 13:37:50 -0700 Subject: [PATCH 091/209] Adding extra debug messages for tracing in Voldemort --- .../server/niosocket/AsyncRequestHandler.java | 28 +++++ .../vold/VoldemortNativeRequestHandler.java | 116 +++++++++++++++++ .../voldemort/store/bdb/BdbStorageEngine.java | 69 +++++++++++ .../store/routed/PipelineRoutedStore.java | 117 +++++++++++++++++- .../routed/action/AbstractReadRepair.java | 14 +++ .../action/PerformParallelPutRequests.java | 5 + .../action/PerformParallelRequests.java | 14 +++ .../action/PerformSerialGetAllRequests.java | 12 ++ .../action/PerformSerialPutRequests.java | 40 +++++- .../routed/action/PerformSerialRequests.java | 7 ++ .../voldemort/store/slop/HintedHandoff.java | 43 +++++-- .../voldemort/store/socket/SocketStore.java | 113 +++++++++++++++++ 12 files changed, 561 insertions(+), 17 deletions(-) diff --git a/src/java/voldemort/server/niosocket/AsyncRequestHandler.java b/src/java/voldemort/server/niosocket/AsyncRequestHandler.java index d4df1254d7..27772e7740 100644 --- a/src/java/voldemort/server/niosocket/AsyncRequestHandler.java +++ b/src/java/voldemort/server/niosocket/AsyncRequestHandler.java @@ -76,6 +76,11 @@ public AsyncRequestHandler(Selector selector, protected void read(SelectionKey selectionKey) throws IOException { int count = 0; + long startNs = -1; + + if(logger.isDebugEnabled()) + startNs = System.nanoTime(); + if((count = socketChannel.read(inputStream.getBuffer())) == -1) throw new EOFException("EOF for " + socketChannel.socket()); @@ -125,6 +130,14 @@ protected void read(SelectionKey selectionKey) throws IOException { streamRequestHandler = requestHandler.handleRequest(new DataInputStream(inputStream), new DataOutputStream(outputStream)); + if(logger.isDebugEnabled()) { + logger.debug("AsyncRequestHandler:read finished request from " + + socketChannel.socket().getRemoteSocketAddress() + " handlerRef: " + + System.identityHashCode(streamRequestHandler) + " at time: " + + System.currentTimeMillis() + " elapsed time: " + + (System.nanoTime() - startNs) + " ns"); + } + if(streamRequestHandler != null) { // In the case of a StreamRequestHandler, we handle that separately // (attempting to process multiple "segments"). @@ -282,8 +295,23 @@ private StreamRequestHandlerState handleStreamRequestInternal(SelectionKey selec if(logger.isTraceEnabled()) traceInputBufferState("Before streaming request handler"); + // this is the lowest level in the NioSocketServer stack at which we + // still have a reference to the client IP address and port + long startNs = -1; + + if(logger.isDebugEnabled()) + startNs = System.nanoTime(); + state = streamRequestHandler.handleRequest(dataInputStream, dataOutputStream); + if(logger.isDebugEnabled()) { + logger.debug("Handled request from " + + socketChannel.socket().getRemoteSocketAddress() + " handlerRef: " + + System.identityHashCode(streamRequestHandler) + " at time: " + + System.currentTimeMillis() + " elapsed time: " + + (System.nanoTime() - startNs) + " ns"); + } + if(logger.isTraceEnabled()) traceInputBufferState("After streaming request handler"); } catch(Exception e) { diff --git a/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java b/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java index b74b60a920..fa70a4b3eb 100644 --- a/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java +++ b/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java @@ -101,6 +101,14 @@ private RequestRoutingType getRoutingType(DataInputStream inputStream) throws IO private void handleGetVersion(DataInputStream inputStream, DataOutputStream outputStream, Store store) throws IOException { + long startTimeMs = -1; + long startTimeNs = -1; + + if(logger.isDebugEnabled()) { + startTimeMs = System.currentTimeMillis(); + startTimeNs = System.nanoTime(); + } + ByteArray key = readKey(inputStream); List results = null; try { @@ -112,11 +120,25 @@ private void handleGetVersion(DataInputStream inputStream, return; } outputStream.writeInt(results.size()); + + String clockStr = ""; + for(Version v: results) { byte[] clock = ((VectorClock) v).toBytes(); + + if(logger.isDebugEnabled()) + clockStr += clock + " "; + outputStream.writeInt(clock.length); outputStream.write(clock); } + + if(logger.isDebugEnabled()) { + logger.debug("GETVERSIONS started at: " + startTimeMs + " handlerRef: " + + System.identityHashCode(this) + " key: " + key + " " + + (System.nanoTime() - startTimeNs) + " ns, keySize: " + key.length() + + "clocks: " + clockStr); + } } /** @@ -269,6 +291,14 @@ private void writeResults(DataOutputStream outputStream, List> private void handleGet(DataInputStream inputStream, DataOutputStream outputStream, Store store) throws IOException { + long startTimeMs = -1; + long startTimeNs = -1; + + if(logger.isDebugEnabled()) { + startTimeMs = System.currentTimeMillis(); + startTimeNs = System.nanoTime(); + } + ByteArray key = readKey(inputStream); byte[] transforms = null; @@ -286,11 +316,22 @@ private void handleGet(DataInputStream inputStream, return; } writeResults(outputStream, results); + if(logger.isDebugEnabled()) { + debugLogReturnValue(key, results, startTimeMs, startTimeNs, "GET"); + } } private void handleGetAll(DataInputStream inputStream, DataOutputStream outputStream, Store store) throws IOException { + long startTimeMs = -1; + long startTimeNs = -1; + + if(logger.isDebugEnabled()) { + startTimeMs = System.currentTimeMillis(); + startTimeNs = System.nanoTime(); + } + // read keys int numKeys = inputStream.readInt(); List keys = new ArrayList(numKeys); @@ -321,18 +362,69 @@ private void handleGetAll(DataInputStream inputStream, // write back the results outputStream.writeInt(results.size()); + + if(logger.isDebugEnabled()) + logger.debug("GETALL start"); + for(Map.Entry>> entry: results.entrySet()) { // write the key outputStream.writeInt(entry.getKey().length()); outputStream.write(entry.getKey().get()); // write the values writeResults(outputStream, entry.getValue()); + + if(logger.isDebugEnabled()) { + debugLogReturnValue(entry.getKey(), + entry.getValue(), + startTimeMs, + startTimeNs, + "GETALL"); + } } + + if(logger.isDebugEnabled()) + logger.debug("GETALL end"); + } + + private void debugLogReturnValue(ByteArray key, + List> values, + long startTimeMs, + long startTimeNs, + String getType) { + long totalValueSize = 0; + String valueSizeStr = "["; + String valueHashStr = "["; + String versionsStr = "["; + for(Versioned b: values) { + int len = b.getValue().length; + totalValueSize += len; + valueSizeStr += len + ","; + valueHashStr += b.hashCode() + ","; + versionsStr += b.getVersion(); + } + valueSizeStr += "]"; + valueHashStr += "]"; + versionsStr += "]"; + + logger.debug(getType + " handlerRef: " + System.identityHashCode(this) + " start time: " + + startTimeMs + " key: " + key + " elapsed time: " + + (System.nanoTime() - startTimeNs) + " ns, keySize: " + key.length() + + " numResults: " + values.size() + " totalResultSize: " + totalValueSize + + " resultSizes: " + valueSizeStr + " resultHashes: " + valueHashStr + + " versions: " + versionsStr + " current time: " + System.currentTimeMillis()); } private void handlePut(DataInputStream inputStream, DataOutputStream outputStream, Store store) throws IOException { + long startTimeMs = -1; + long startTimeNs = -1; + + if(logger.isDebugEnabled()) { + startTimeMs = System.currentTimeMillis(); + startTimeNs = System.nanoTime(); + } + ByteArray key = readKey(inputStream); int valueSize = inputStream.readInt(); byte[] bytes = new byte[valueSize]; @@ -352,11 +444,28 @@ private void handlePut(DataInputStream inputStream, } catch(VoldemortException e) { writeException(outputStream, e); } + + if(logger.isDebugEnabled()) { + logger.debug("PUT started at: " + startTimeMs + " handlerRef: " + + System.identityHashCode(this) + " key: " + key + " " + + (System.nanoTime() - startTimeNs) + " ns, keySize: " + key.length() + + " valueHash: " + value.hashCode() + " valueSize: " + valueSize + + " clockSize: " + clock.sizeInBytes() + " time: " + + System.currentTimeMillis()); + } } private void handleDelete(DataInputStream inputStream, DataOutputStream outputStream, Store store) throws IOException { + long startTimeMs = -1; + long startTimeNs = -1; + + if(logger.isDebugEnabled()) { + startTimeMs = System.currentTimeMillis(); + startTimeNs = System.nanoTime(); + } + ByteArray key = readKey(inputStream); int versionSize = inputStream.readShort(); byte[] versionBytes = new byte[versionSize]; @@ -369,6 +478,13 @@ private void handleDelete(DataInputStream inputStream, } catch(VoldemortException e) { writeException(outputStream, e); } + + if(logger.isDebugEnabled()) { + logger.debug("DELETE started at: " + startTimeMs + " key: " + key + " handlerRef: " + + System.identityHashCode(this) + " time: " + + (System.nanoTime() - startTimeNs) + " ns, keySize: " + key.length() + + " clockSize: " + version.sizeInBytes()); + } } private void writeException(DataOutputStream stream, VoldemortException e) throws IOException { diff --git a/src/java/voldemort/store/bdb/BdbStorageEngine.java b/src/java/voldemort/store/bdb/BdbStorageEngine.java index 61d809ee96..89c0caf6e5 100644 --- a/src/java/voldemort/store/bdb/BdbStorageEngine.java +++ b/src/java/voldemort/store/bdb/BdbStorageEngine.java @@ -209,6 +209,11 @@ private List get(ByteArray key, Serializer serializer) throws PersistenceFailureException { StoreUtils.assertValidKey(key); + long startTimeNs = -1; + + if(logger.isTraceEnabled()) + startTimeNs = System.nanoTime(); + Cursor cursor = null; try { cursor = getBdbDatabase().openCursor(null, null); @@ -225,6 +230,13 @@ private List get(ByteArray key, logger.error(e); throw new PersistenceFailureException(e); } finally { + if(logger.isTraceEnabled()) { + logger.trace("Completed GET from key " + key + " (keyRef: " + + System.identityHashCode(key) + ") in " + + (System.nanoTime() - startTimeNs) + " ns at " + + System.currentTimeMillis()); + } + attemptClose(cursor); } } @@ -250,12 +262,25 @@ private Database getBdbDatabase() { public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { + + long startTimeNs = -1; + + if(logger.isTraceEnabled()) + startTimeNs = System.nanoTime(); + StoreUtils.assertValidKeys(keys); Map>> result = StoreUtils.newEmptyHashMap(keys); Cursor cursor = null; + + String keyStr = ""; + try { cursor = getBdbDatabase().openCursor(null, null); for(ByteArray key: keys) { + + if(logger.isTraceEnabled()) + keyStr += key + " "; + List> values = get(cursor, key, readLockMode, versionedSerializer); if(!values.isEmpty()) result.put(key, values); @@ -266,6 +291,12 @@ public Map>> getAll(Iterable keys, } finally { attemptClose(cursor); } + + if(logger.isTraceEnabled()) + logger.trace("Completed GETALL from keys " + keyStr + " in " + + (System.nanoTime() - startTimeNs) + " ns at " + + System.currentTimeMillis()); + return result; } @@ -275,6 +306,11 @@ private static List get(Cursor cursor, Serializer serializer) throws DatabaseException { StoreUtils.assertValidKey(key); + long startTimeNs = -1; + + if(logger.isTraceEnabled()) + startTimeNs = System.nanoTime(); + DatabaseEntry keyEntry = new DatabaseEntry(key.get()); DatabaseEntry valueEntry = new DatabaseEntry(); List results = Lists.newArrayList(); @@ -284,6 +320,13 @@ private static List get(Cursor cursor, lockMode)) { results.add(serializer.toObject(valueEntry.getData())); } + + if(logger.isTraceEnabled()) { + logger.trace("Completed GET from key " + key + " in " + + (System.nanoTime() - startTimeNs) + " ns at " + + System.currentTimeMillis()); + } + return results; } @@ -291,6 +334,11 @@ public void put(ByteArray key, Versioned value, byte[] transforms) throws PersistenceFailureException { StoreUtils.assertValidKey(key); + long startTimeNs = -1; + + if(logger.isTraceEnabled()) + startTimeNs = System.nanoTime(); + DatabaseEntry keyEntry = new DatabaseEntry(key.get()); boolean succeeded = false; Transaction transaction = null; @@ -338,10 +386,23 @@ else if(occurred == Occurred.AFTER) else attemptAbort(transaction); } + + if(logger.isTraceEnabled()) { + logger.trace("Completed PUT to key " + key + " (keyRef: " + + System.identityHashCode(key) + " value " + value + " in " + + (System.nanoTime() - startTimeNs) + " ns at " + + System.currentTimeMillis()); + } } public boolean delete(ByteArray key, Version version) throws PersistenceFailureException { StoreUtils.assertValidKey(key); + + long startTimeNs = -1; + + if(logger.isTraceEnabled()) + startTimeNs = System.nanoTime(); + boolean deletedSomething = false; Cursor cursor = null; Transaction transaction = null; @@ -366,6 +427,14 @@ public boolean delete(ByteArray key, Version version) throws PersistenceFailureE logger.error(e); throw new PersistenceFailureException(e); } finally { + + if(logger.isTraceEnabled()) { + logger.trace("Completed DELETE of key " + key + " (keyRef: " + + System.identityHashCode(key) + ") in " + + (System.nanoTime() - startTimeNs) + " ns at " + + System.currentTimeMillis()); + } + try { attemptClose(cursor); } finally { diff --git a/src/java/voldemort/store/routed/PipelineRoutedStore.java b/src/java/voldemort/store/routed/PipelineRoutedStore.java index 38ff90517d..dfde9b350c 100644 --- a/src/java/voldemort/store/routed/PipelineRoutedStore.java +++ b/src/java/voldemort/store/routed/PipelineRoutedStore.java @@ -144,6 +144,14 @@ public PipelineRoutedStore(String name, public List> get(final ByteArray key, final byte[] transforms) { StoreUtils.assertValidKey(key); + long startTimeMs = -1; + long startTimeNs = -1; + + if(logger.isDebugEnabled()) { + startTimeMs = System.currentTimeMillis(); + startTimeNs = System.nanoTime(); + } + BasicPipelineData>> pipelineData = new BasicPipelineData>>(); if(zoneRoutingEnabled) pipelineData.setZonesRequired(storeDef.getZoneCountReads()); @@ -242,14 +250,42 @@ public List> request(Store store) { results.addAll(value); } + if(logger.isDebugEnabled()) { + logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + key + + " keyRef: " + System.identityHashCode(key) + "; started at " + + startTimeMs + " took " + (System.nanoTime() - startTimeNs) + " values: " + + formatNodeValuesFromGet(pipelineData.getResponses())); + } + return results; } + private String formatNodeValuesFromGet(List>>> results) { + // log all retrieved values + StringBuilder builder = new StringBuilder(); + builder.append("{"); + for(Response>> r: results) { + builder.append("(nodeId=" + r.getNode().getId() + ", key=" + r.getKey() + + ", retrieved= " + r.getValue() + "), "); + } + builder.append("}"); + + return builder.toString(); + } + public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { StoreUtils.assertValidKeys(keys); + long startTimeMs = -1; + long startTimeNs = -1; + + if(logger.isDebugEnabled()) { + startTimeMs = System.currentTimeMillis(); + startTimeNs = System.nanoTime(); + } + boolean allowReadRepair = repairReads && (transforms == null || transforms.size() == 0); GetAllPipelineData pipelineData = new GetAllPipelineData(); @@ -318,12 +354,41 @@ public Map>> getAll(Iterable keys, if(pipelineData.getFatalError() != null) throw pipelineData.getFatalError(); + if(logger.isDebugEnabled()) { + logger.debug("Finished " + pipeline.getOperation().getSimpleName() + "for keys " + keys + + " keyRef: " + System.identityHashCode(keys) + "; started at " + + startTimeMs + " took " + (System.nanoTime() - startTimeNs) + " values: " + + formatNodeValuesFromGetAll(pipelineData.getResponses())); + } + return pipelineData.getResult(); } + private String formatNodeValuesFromGetAll(List, Map>>>> list) { + // log all retrieved values + StringBuilder builder = new StringBuilder(); + builder.append("{"); + for(Response, Map>>> r: list) { + builder.append("(nodeId=" + r.getNode().getId() + ", key=" + r.getKey() + + ", retrieved= " + r.getValue() + ")"); + builder.append(", "); + } + builder.append("}"); + + return builder.toString(); + } + public List getVersions(final ByteArray key) { StoreUtils.assertValidKey(key); + long startTimeMs = -1; + long startTimeNs = -1; + + if(logger.isDebugEnabled()) { + startTimeMs = System.currentTimeMillis(); + startTimeNs = System.nanoTime(); + } + BasicPipelineData> pipelineData = new BasicPipelineData>(); if(zoneRoutingEnabled) pipelineData.setZonesRequired(storeDef.getZoneCountReads()); @@ -385,7 +450,7 @@ public List request(Store store) { pipeline.addEvent(Event.STARTED); if(logger.isDebugEnabled()) { - logger.debug("Operation " + pipeline.getOperation().getSimpleName() + "Key " + logger.debug("Operation " + pipeline.getOperation().getSimpleName() + " Key " + ByteUtils.toHexString(key.get())); } try { @@ -403,12 +468,40 @@ public List request(Store store) { for(Response> response: pipelineData.getResponses()) results.addAll(response.getValue()); + if(logger.isDebugEnabled()) { + logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + key + + " keyRef: " + System.identityHashCode(key) + "; started at " + + startTimeMs + " took " + (System.nanoTime() - startTimeNs) + " values: " + + formatNodeValuesFromGetVersions(pipelineData.getResponses())); + } + return results; } + private String formatNodeValuesFromGetVersions(List>> results) { + // log all retrieved values + StringBuilder builder = new StringBuilder(); + builder.append("{"); + for(Response> r: results) { + builder.append("(nodeId=" + r.getNode().getId() + ", key=" + r.getKey() + + ", retrieved= " + r.getValue() + "), "); + } + builder.append("}"); + + return builder.toString(); + } + public boolean delete(final ByteArray key, final Version version) throws VoldemortException { StoreUtils.assertValidKey(key); + long startTimeMs = -1; + long startTimeNs = -1; + + if(logger.isDebugEnabled()) { + startTimeMs = System.currentTimeMillis(); + startTimeNs = System.nanoTime(); + } + BasicPipelineData pipelineData = new BasicPipelineData(); if(zoneRoutingEnabled) pipelineData.setZonesRequired(storeDef.getZoneCountWrites()); @@ -480,6 +573,12 @@ public boolean delete(final ByteArray key, final Version version) throws Voldemo throw e; } + if(logger.isDebugEnabled()) { + logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + + key.get() + " keyRef: " + System.identityHashCode(key) + "; started at " + + startTimeMs + " took " + (System.nanoTime() - startTimeNs)); + } + if(pipelineData.getFatalError() != null) throw pipelineData.getFatalError(); @@ -497,6 +596,15 @@ public boolean isHintedHandoffEnabled() { public void put(ByteArray key, Versioned versioned, byte[] transforms) throws VoldemortException { + + long startTimeMs = -1; + long startTimeNs = -1; + + if(logger.isDebugEnabled()) { + startTimeMs = System.currentTimeMillis(); + startTimeNs = System.nanoTime(); + } + StoreUtils.assertValidKey(key); PutPipelineData pipelineData = new PutPipelineData(); if(zoneRoutingEnabled) @@ -591,6 +699,13 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) throw e; } + if(logger.isDebugEnabled()) { + logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + key + + " keyRef: " + System.identityHashCode(key) + "; started at " + + startTimeMs + " took " + (System.nanoTime() - startTimeNs) + " value: " + + versioned.getValue() + " (size: " + versioned.getValue().length + ")"); + } + if(pipelineData.getFatalError() != null) throw pipelineData.getFatalError(); } diff --git a/src/java/voldemort/store/routed/action/AbstractReadRepair.java b/src/java/voldemort/store/routed/action/AbstractReadRepair.java index 2588c36df0..6e2ea19d6f 100644 --- a/src/java/voldemort/store/routed/action/AbstractReadRepair.java +++ b/src/java/voldemort/store/routed/action/AbstractReadRepair.java @@ -75,6 +75,11 @@ protected void insertNodeValue(Node node, ByteArray key, List> public void execute(Pipeline pipeline) { insertNodeValues(); + long startTimeNs = -1; + + if(logger.isTraceEnabled()) + startTimeNs = System.nanoTime(); + if(nodeValues.size() > 1 && preferred > 1) { List> toReadRepair = Lists.newArrayList(); @@ -111,6 +116,15 @@ public void execute(Pipeline pipeline) { logger.debug("Read repair failed: ", e); } } + + if(logger.isDebugEnabled()) { + String logStr = "Repaired (node, key, version): ("; + for(NodeValue v: toReadRepair) { + logStr += "(" + v.getNodeId() + ", " + v.getKey() + "," + v.getVersion() + ") "; + } + logStr += "in " + (System.nanoTime() - startTimeNs) + " ns"; + logger.debug(logStr); + } } pipeline.addEvent(completeEvent); diff --git a/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java b/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java index c459057589..a8373418b5 100644 --- a/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java +++ b/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java @@ -129,6 +129,11 @@ public void requestComplete(Object result, long requestTime) { requestTime); responses.put(node.getId(), response); + if(logger.isDebugEnabled()) + logger.debug("Finished secondary PUT for key " + key + " (keyRef: " + + System.identityHashCode(key) + "); took " + requestTime + + " ms on node " + node.getId() + "(" + node.getHost() + ")"); + if(isHintedHandoffEnabled() && pipeline.isFinished()) { if(response.getValue() instanceof UnreachableStoreException) { Slop slop = new Slop(pipelineData.getStoreName(), diff --git a/src/java/voldemort/store/routed/action/PerformParallelRequests.java b/src/java/voldemort/store/routed/action/PerformParallelRequests.java index a5a0ef9aa6..73399d75b0 100644 --- a/src/java/voldemort/store/routed/action/PerformParallelRequests.java +++ b/src/java/voldemort/store/routed/action/PerformParallelRequests.java @@ -95,6 +95,8 @@ public void execute(final Pipeline pipeline) { final Node node = nodes.get(i); pipelineData.incrementNodeIndex(); + final long startMs = logger.isDebugEnabled() ? System.currentTimeMillis() : -1; + NonblockingStoreCallback callback = new NonblockingStoreCallback() { public void requestComplete(Object result, long requestTime) { @@ -107,6 +109,13 @@ public void requestComplete(Object result, long requestTime) { key, result, requestTime); + if(logger.isDebugEnabled()) + logger.debug("Finished " + pipeline.getOperation().getSimpleName() + + " for key " + key + " (keyRef: " + + System.identityHashCode(key) + "); started at " + startMs + + " took " + requestTime + " ms on node " + node.getId() + "(" + + node.getHost() + ")"); + responses.put(node.getId(), response); latch.countDown(); @@ -164,6 +173,11 @@ else if(pipeline.getOperation() == Operation.GET_VERSIONS) } } + if(logger.isDebugEnabled()) + logger.debug("GET for key " + key + " (keyRef: " + System.identityHashCode(key) + + "); successes: " + pipelineData.getSuccesses() + " preferred: " + + preferred + " required: " + required); + if(pipelineData.getSuccesses() < required) { if(insufficientSuccessesEvent != null) { pipeline.addEvent(insufficientSuccessesEvent); diff --git a/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java b/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java index 2f5d07e7a2..c8135781d4 100644 --- a/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java +++ b/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java @@ -79,6 +79,11 @@ public void execute(Pipeline pipeline) { boolean zoneRequirement = false; MutableInt successCount = pipelineData.getSuccessCount(key); + if(logger.isDebugEnabled()) + logger.debug("GETALL for key " + key + " (keyRef: " + System.identityHashCode(key) + + ") successes: " + successCount.intValue() + " preferred: " + preferred + + " required: " + required); + if(successCount.intValue() >= preferred) { if(pipelineData.getZonesRequired() != null) { @@ -132,6 +137,13 @@ public void execute(Pipeline pipeline) { pipelineData.getResponses().add(response); failureDetector.recordSuccess(response.getNode(), response.getRequestTime()); + if(logger.isDebugEnabled()) + logger.debug("GET for key " + key + " (keyRef: " + + System.identityHashCode(key) + ") successes: " + + successCount.intValue() + " preferred: " + preferred + + " required: " + required + " new GET success on node " + + node.getId()); + HashSet zoneResponses = null; if(pipelineData.getKeyToZoneResponse().containsKey(key)) { zoneResponses = pipelineData.getKeyToZoneResponse().get(key); diff --git a/src/java/voldemort/store/routed/action/PerformSerialPutRequests.java b/src/java/voldemort/store/routed/action/PerformSerialPutRequests.java index 5d8debe17a..7b7b3d65fd 100644 --- a/src/java/voldemort/store/routed/action/PerformSerialPutRequests.java +++ b/src/java/voldemort/store/routed/action/PerformSerialPutRequests.java @@ -74,11 +74,20 @@ public void execute(Pipeline pipeline) { int currentNode = 0; List nodes = pipelineData.getNodes(); + long startMasterMs = -1; + long startMasterNs = -1; + + if(logger.isDebugEnabled()) { + startMasterMs = System.currentTimeMillis(); + startMasterNs = System.nanoTime(); + } + if(logger.isDebugEnabled()) logger.debug("Performing serial put requests to determine master"); + Node node = null; for(; currentNode < nodes.size(); currentNode++) { - Node node = nodes.get(currentNode); + node = nodes.get(currentNode); pipelineData.incrementNodeIndex(); VectorClock versionedClock = (VectorClock) versioned.getVersion(); @@ -86,8 +95,8 @@ public void execute(Pipeline pipeline) { versionedClock.incremented(node.getId(), time.getMilliseconds())); - if(logger.isTraceEnabled()) - logger.trace("Attempt #" + (currentNode + 1) + " to perform put (node " + if(logger.isDebugEnabled()) + logger.debug("Attempt #" + (currentNode + 1) + " to perform put (node " + node.getId() + ")"); long start = System.nanoTime(); @@ -98,8 +107,8 @@ public void execute(Pipeline pipeline) { pipelineData.incrementSuccesses(); failureDetector.recordSuccess(node, requestTime); - if(logger.isTraceEnabled()) - logger.trace("Put on node " + node.getId() + " succeeded, using as master"); + if(logger.isDebugEnabled()) + logger.debug("Put on node " + node.getId() + " succeeded, using as master"); pipelineData.setMaster(node); pipelineData.setVersionedCopy(versionedCopy); @@ -108,6 +117,12 @@ public void execute(Pipeline pipeline) { } catch(Exception e) { long requestTime = (System.nanoTime() - start) / Time.NS_PER_MS; + if(logger.isDebugEnabled()) + logger.debug("Master PUT at node " + currentNode + "(" + node.getHost() + ")" + + " failed (" + e.getMessage() + ") in " + + (System.nanoTime() - start) + " ns" + " (keyRef: " + + System.identityHashCode(key) + ")"); + if(handleResponseError(e, node, requestTime, pipeline, failureDetector)) return; } @@ -157,10 +172,25 @@ public void execute(Pipeline pipeline) { } } else { + if(logger.isDebugEnabled()) + logger.debug("Finished master PUT for key " + key + " (keyRef: " + + System.identityHashCode(key) + "); started at " + + startMasterMs + " took " + + (System.nanoTime() - startMasterNs) + " ns on node " + + (node == null ? "NULL" : node.getId()) + "(" + + (node == null ? "NULL" : node.getHost()) + "); now complete"); + pipeline.addEvent(completeEvent); } } } else { + if(logger.isDebugEnabled()) + logger.debug("Finished master PUT for key " + key + " (keyRef: " + + System.identityHashCode(key) + "); started at " + startMasterMs + + " took " + (System.nanoTime() - startMasterNs) + " ns on node " + + (node == null ? "NULL" : node.getId()) + "(" + + (node == null ? "NULL" : node.getHost()) + ")"); + pipeline.addEvent(masterDeterminedEvent); } } diff --git a/src/java/voldemort/store/routed/action/PerformSerialRequests.java b/src/java/voldemort/store/routed/action/PerformSerialRequests.java index 70c8fa563a..4c447174d8 100644 --- a/src/java/voldemort/store/routed/action/PerformSerialRequests.java +++ b/src/java/voldemort/store/routed/action/PerformSerialRequests.java @@ -98,6 +98,13 @@ public void execute(Pipeline pipeline) { result, ((System.nanoTime() - start) / Time.NS_PER_MS)); + if(logger.isDebugEnabled()) + logger.debug(pipeline.getOperation().getSimpleName() + " for key " + key + + " successes: " + pipelineData.getSuccesses() + " preferred: " + + preferred + " required: " + required + " new " + + pipeline.getOperation().getSimpleName() + " success on node " + + node.getId()); + pipelineData.incrementSuccesses(); pipelineData.getResponses().add(response); failureDetector.recordSuccess(response.getNode(), response.getRequestTime()); diff --git a/src/java/voldemort/store/slop/HintedHandoff.java b/src/java/voldemort/store/slop/HintedHandoff.java index a4a66217b3..5e506a85e2 100644 --- a/src/java/voldemort/store/slop/HintedHandoff.java +++ b/src/java/voldemort/store/slop/HintedHandoff.java @@ -101,15 +101,17 @@ public void sendHintParallel(final Node failedNode, final Version version, final for(final Node node: handoffStrategy.routeHint(failedNode)) { int nodeId = node.getId(); - if(logger.isTraceEnabled()) - logger.trace("Sending an async hint to " + nodeId); + + if(logger.isDebugEnabled()) + logger.debug("Sending an async hint to " + nodeId); if(!failedNodes.contains(node) && failureDetector.isAvailable(node)) { NonblockingStore nonblockingStore = nonblockingSlopStores.get(nodeId); Utils.notNull(nonblockingStore); final long startNs = System.nanoTime(); - if(logger.isTraceEnabled()) - logger.trace("Attempt to write " + slop.getKey() + " for " + failedNode + + if(logger.isDebugEnabled()) + logger.debug("Slop attempt to write " + slop.getKey() + " for " + failedNode + " to node " + node); NonblockingStoreCallback callback = new NonblockingStoreCallback() { @@ -127,6 +129,13 @@ public void requestComplete(Object result, long requestTime) { failedNodes.add(node); if(response.getValue() instanceof UnreachableStoreException) { UnreachableStoreException use = (UnreachableStoreException) response.getValue(); + + if(logger.isDebugEnabled()) + logger.debug("Write of key " + slop.getKey() + " for " + + failedNode + " to node " + node + + " failed due to unreachable: " + + use.getMessage()); + failureDetector.recordException(node, (System.nanoTime() - startNs) / Time.NS_PER_MS, @@ -136,6 +145,12 @@ public void requestComplete(Object result, long requestTime) { } return; } + + if(logger.isDebugEnabled()) + logger.debug("Slop write of key " + slop.getKey() + " for " + + failedNode + " to node " + node + " succeeded in " + + (System.nanoTime() - startNs) + " ns"); + failureDetector.recordSuccess(node, (System.nanoTime() - startNs) / Time.NS_PER_MS); @@ -151,7 +166,7 @@ public void requestComplete(Object result, long requestTime) { } } } - + /** * Send a hint of a request originally meant for the failed node to another * node in the ring, as selected by the {@link HintedHandoffStrategy} @@ -166,8 +181,8 @@ public boolean sendHintSerial(Node failedNode, Version version, Slop slop) { boolean persisted = false; for(Node node: handoffStrategy.routeHint(failedNode)) { int nodeId = node.getId(); - if(logger.isTraceEnabled()) - logger.trace("Trying to send hint to " + nodeId); + if(logger.isDebugEnabled()) + logger.debug("Trying to send hint to " + nodeId); if(!failedNodes.contains(node) && failureDetector.isAvailable(node)) { Store slopStore = slopStores.get(nodeId); @@ -175,10 +190,10 @@ public boolean sendHintSerial(Node failedNode, Version version, Slop slop) { long startNs = System.nanoTime(); try { - if(logger.isTraceEnabled()) - logger.trace("Attempt to handoff " + slop.getOperation() + " on " - + slop.getKey() + " for " + failedNode - + " to node " + node); + if(logger.isDebugEnabled()) + logger.debug("Slop attempt to write " + slop.getKey() + " (keyRef: " + + System.identityHashCode(slop.getKey()) + ") for " + + failedNode + " to node " + node); // No transform needs to applied to the slop slopStore.put(slop.makeKey(), new Versioned(slop, version), null); @@ -197,6 +212,12 @@ public boolean sendHintSerial(Node failedNode, Version version, Slop slop) { } catch(ObsoleteVersionException e) { logger.debug(e, e); } + + if(logger.isDebugEnabled()) + logger.debug("Slop write of key " + slop.getKey() + " (keyRef: " + + System.identityHashCode(slop.getKey()) + " for " + failedNode + + " to node " + node + " succeeded in " + + (System.nanoTime() - startNs) + " ns"); } } diff --git a/src/java/voldemort/store/socket/SocketStore.java b/src/java/voldemort/store/socket/SocketStore.java index 0886341521..3781a77cbe 100644 --- a/src/java/voldemort/store/socket/SocketStore.java +++ b/src/java/voldemort/store/socket/SocketStore.java @@ -100,6 +100,9 @@ public void submitDeleteRequest(ByteArray key, requestRoutingType, key, version); + if(logger.isDebugEnabled()) + logger.debug("DELETE keyRef: " + System.identityHashCode(key) + " requestRef: " + + System.identityHashCode(clientRequest)); requestAsync(clientRequest, callback, timeoutMs, "delete"); } @@ -113,6 +116,9 @@ public void submitGetRequest(ByteArray key, requestRoutingType, key, transforms); + if(logger.isDebugEnabled()) + logger.debug("GET keyRef: " + System.identityHashCode(key) + " requestRef: " + + System.identityHashCode(clientRequest)); requestAsync(clientRequest, callback, timeoutMs, "get"); } @@ -126,6 +132,9 @@ public void submitGetAllRequest(Iterable keys, requestRoutingType, keys, transforms); + if(logger.isDebugEnabled()) + logger.debug("GETALL keyRef: " + System.identityHashCode(keys) + " requestRef: " + + System.identityHashCode(clientRequest)); requestAsync(clientRequest, callback, timeoutMs, "get all"); } @@ -137,6 +146,9 @@ public void submitGetVersionsRequest(ByteArray key, requestFormat, requestRoutingType, key); + if(logger.isDebugEnabled()) + logger.debug("GETVERSIONS keyRef: " + System.identityHashCode(key) + " requestRef: " + + System.identityHashCode(clientRequest)); requestAsync(clientRequest, callback, timeoutMs, "get versions"); } @@ -152,6 +164,9 @@ public void submitPutRequest(ByteArray key, key, value, transforms); + if(logger.isDebugEnabled()) + logger.debug("PUT keyRef: " + System.identityHashCode(key) + " requestRef: " + + System.identityHashCode(clientRequest)); requestAsync(clientRequest, callback, timeoutMs, "put"); } @@ -162,6 +177,9 @@ public boolean delete(ByteArray key, Version version) throws VoldemortException requestRoutingType, key, version); + if(logger.isDebugEnabled()) + logger.debug("DELETE keyRef: " + System.identityHashCode(key) + " requestRef: " + + System.identityHashCode(clientRequest)); return request(clientRequest, "delete"); } @@ -172,6 +190,9 @@ public List> get(ByteArray key, byte[] transforms) throws Vold requestRoutingType, key, transforms); + if(logger.isDebugEnabled()) + logger.debug("GET keyRef: " + System.identityHashCode(key) + " requestRef: " + + System.identityHashCode(clientRequest)); return request(clientRequest, "get"); } @@ -184,6 +205,9 @@ public Map>> getAll(Iterable keys, requestRoutingType, keys, transforms); + if(logger.isDebugEnabled()) + logger.debug("GETALL keyRef: " + System.identityHashCode(keys) + " requestRef: " + + System.identityHashCode(clientRequest)); return request(clientRequest, "getAll"); } @@ -193,6 +217,9 @@ public List getVersions(ByteArray key) { requestFormat, requestRoutingType, key); + if(logger.isDebugEnabled()) + logger.debug("GETVERSIONS keyRef: " + System.identityHashCode(key) + " requestRef: " + + System.identityHashCode(clientRequest)); return request(clientRequest, "getVersions"); } @@ -205,6 +232,9 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) key, versioned, transforms); + if(logger.isDebugEnabled()) + logger.debug("PUT keyRef: " + System.identityHashCode(key) + " requestRef: " + + System.identityHashCode(clientRequest)); request(clientRequest, "put"); } @@ -240,17 +270,40 @@ public void close() throws VoldemortException { private T request(ClientRequest delegate, String operationName) { ClientRequestExecutor clientRequestExecutor = pool.checkout(destination); + + long startTimeMs = -1; + long startTimeNs = -1; + + if(logger.isDebugEnabled()) { + startTimeMs = System.currentTimeMillis(); + startTimeNs = System.nanoTime(); + } + + String debugMsgStr = ""; + BlockingClientRequest blockingClientRequest = null; try { blockingClientRequest = new BlockingClientRequest(delegate, timeoutMs); clientRequestExecutor.addClientRequest(blockingClientRequest, timeoutMs); blockingClientRequest.await(); + + if(logger.isDebugEnabled()) + debugMsgStr += "success"; + return blockingClientRequest.getResult(); } catch(InterruptedException e) { + + if(logger.isDebugEnabled()) + debugMsgStr += "unreachable: " + e.getMessage(); + throw new UnreachableStoreException("Failure in " + operationName + " on " + destination + ": " + e.getMessage(), e); } catch(IOException e) { clientRequestExecutor.close(); + + if(logger.isDebugEnabled()) + debugMsgStr += "failure: " + e.getMessage(); + throw new UnreachableStoreException("Failure in " + operationName + " on " + destination + ": " + e.getMessage(), e); } finally { @@ -258,6 +311,29 @@ private T request(ClientRequest delegate, String operationName) { // close the executor if we timed out clientRequestExecutor.close(); } + + if(logger.isDebugEnabled()) { + logger.debug("Sync request end, type: " + + operationName + + " requestRef: " + + System.identityHashCode(delegate) + + " totalTimeNs: " + + (System.nanoTime() - startTimeNs) + + " start time: " + + startTimeMs + + " end time: " + + System.currentTimeMillis() + + " client:" + + clientRequestExecutor.getSocketChannel().socket().getLocalAddress() + + ":" + + clientRequestExecutor.getSocketChannel().socket().getLocalPort() + + " server: " + + clientRequestExecutor.getSocketChannel() + .socket() + .getRemoteSocketAddress() + " outcome: " + + debugMsgStr); + } + pool.checkin(destination, clientRequestExecutor); } } @@ -285,6 +361,23 @@ private void requestAsync(ClientRequest delegate, try { clientRequestExecutor = pool.checkout(destination); + + if(logger.isDebugEnabled()) { + logger.debug("Async request start; type: " + + operationName + + " requestRef: " + + System.identityHashCode(delegate) + + " time: " + + System.currentTimeMillis() + + " server: " + + clientRequestExecutor.getSocketChannel() + .socket() + .getRemoteSocketAddress() + " local socket: " + + clientRequestExecutor.getSocketChannel().socket().getLocalAddress() + + ":" + + clientRequestExecutor.getSocketChannel().socket().getLocalPort()); + } + } catch(Exception e) { // If we can't check out a socket from the pool, we'll usually get // either an IOException (subclass) or an UnreachableStoreException @@ -335,6 +428,26 @@ public NonblockingStoreCallbackClientRequest(ClientRequest clientRequest, private void invokeCallback(Object o, long requestTime) { if(callback != null) { try { + if(logger.isDebugEnabled()) { + logger.debug("Async request end; requestRef: " + + System.identityHashCode(clientRequest) + + " time: " + + System.currentTimeMillis() + + " server: " + + clientRequestExecutor.getSocketChannel() + .socket() + .getRemoteSocketAddress() + + " local socket: " + + clientRequestExecutor.getSocketChannel() + .socket() + .getLocalAddress() + + ":" + + clientRequestExecutor.getSocketChannel() + .socket() + .getLocalPort() + " result: " + + o.toString()); + } + callback.requestComplete(o, requestTime); } catch(Exception e) { if(logger.isEnabledFor(Level.WARN)) From 962298eeb006566c11ce685b354b0082fc44067f Mon Sep 17 00:00:00 2001 From: Brendan Harris Date: Tue, 10 Jul 2012 19:41:03 -0700 Subject: [PATCH 092/209] Added default behavior, handling of long and short options for modifying defaults, zone support, random seed generation, removed text width formatting as it is not necessary, cleaned up format, refactored code, sys module no longer necessary (now that argparse is used) so removed it, added default interpreter path --- bin/generate_cluster_xml.py | 126 +++++++++++++++++++++++++----------- 1 file changed, 87 insertions(+), 39 deletions(-) diff --git a/bin/generate_cluster_xml.py b/bin/generate_cluster_xml.py index 3cd56644d7..57e766583f 100644 --- a/bin/generate_cluster_xml.py +++ b/bin/generate_cluster_xml.py @@ -1,42 +1,90 @@ +#!/usr/bin/python + import sys import random +import argparse + +# Get a random seed +seed = int(random.randint(00000000001,99999999999)) + +# Setup and argument parser +parser = argparse.ArgumentParser(description='Build a voldemort cluster.xml.') +# Add supported arguments +parser.add_argument('-N', '--name', type=str, default='voldemort', dest='name', + help='the name you want to give the clusteer') +parser.add_argument('-n', '--nodes', type=int, default=2, dest='nodes', + help='the number of nodes in the cluster') +parser.add_argument('-p', '--partitions', type=int, default=300, + dest='partitions', help='number of partitions per node') +parser.add_argument('-s', '--socket-port', type=int, default=6666, + dest='sock_port', help='socket port number') +parser.add_argument('-a', '--admin-port', type=int, default=6667, + dest='admin_port', help='admin port number') +parser.add_argument('-H', '--http-port', type=int, default=6665, + dest='http_port', help='http port number') +genType = parser.add_mutually_exclusive_group() +genType.add_argument('-S', '--seed', type=int, default=seed, dest='seed', + help='seed for randomizing partition distribution') +genType.add_argument('-l', '--loops', type=int, default=1000, dest='loops', + help='loop n times, using a different random seed every \ + time (Note: not currently supported)') +parser.add_argument('-z', '--zones', type=int, dest='zones', + help='if using zones, the number of zones you will have\ + (Note: you must add your own field \ + manually)') + +# Parse arguments +args = parser.parse_args() + +# Check args +if args.zones: + zones = args.zones + if (args.nodes % zones) != 0: + print "Number of nodes must be evenly divisible by number of zones" + sys.exit(1) + +# Store arguments +nodes = args.nodes +partitions = args.partitions +name = args.name +http_port = args.http_port +sock_port = args.sock_port +admin_port = args.admin_port + +# Generate the full list of partition IDs +part_ids = range(nodes * partitions) +# Generate full list of zone IDs +if args.zones: + zone_ids = range(zones) + zone_id = 0 +## Use known seed so this is repeatable +#random.seed(3119578866) +random.seed(seed) +random.shuffle(part_ids) + +# Assining partitions to nodes and printing cluster.xml +part_map = dict() +print "" % seed +print "" +print " %s" % name + +for i in xrange(nodes): + part_map[i] = ", ".join(str(p) for p in sorted(part_ids[i*partitions:(i+1)*partitions])) + + print " " + print " %d" % i + print " host%d" % i + print " %d" % http_port + print " %d" % sock_port + print " %d" % admin_port + print " %s" % part_map[i] + # If zones are being used, assign a zone-id + if args.zones: + print " %d" % zone_id + if zone_id == (zones - 1): + zone_id = 0 + else: + zone_id += 1 + print " " -if len(sys.argv) != 3: - print >> sys.stderr, "USAGE: python generate_cluster_xml.py " - sys.exit() - -FORMAT_WIDTH = 10 - -nodes = 0 -for line in open(sys.argv[1],'r'): - nodes+=1 - -partitions = int(sys.argv[2]) - -ids = range(nodes * partitions) - -# use known seed so this is repeatable -random.seed(92873498274) -random.shuffle(ids) - -print '' -print 'prodcluster' -id = 0 -for host in open(sys.argv[1],'r'): - print '' - print " %d" % id - print " %s" % host.strip() - print ' 8081' - print ' 6666' - print ' ', - node_ids = sorted(ids[id*partitions:(id+1)*partitions]) - for j in xrange(len(node_ids)): - print str(node_ids[j]) + ',', - if j % FORMAT_WIDTH == FORMAT_WIDTH - 1: - print ' ', - print ' ' - print '' - id += 1 -print '' - - +print "" From a5606293772bd599f4941d1fbf04f612e2186686 Mon Sep 17 00:00:00 2001 From: Brendan Harris Date: Tue, 10 Jul 2012 19:44:59 -0700 Subject: [PATCH 093/209] Fixing typo --- bin/generate_cluster_xml.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/generate_cluster_xml.py b/bin/generate_cluster_xml.py index 57e766583f..5d7c6065b2 100644 --- a/bin/generate_cluster_xml.py +++ b/bin/generate_cluster_xml.py @@ -30,7 +30,7 @@ time (Note: not currently supported)') parser.add_argument('-z', '--zones', type=int, dest='zones', help='if using zones, the number of zones you will have\ - (Note: you must add your own field \ + (Note: you must add your own fields \ manually)') # Parse arguments From ffff0f7f169fc91d96c161dcc970902594ffa282 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Wed, 11 Jul 2012 16:52:35 -0700 Subject: [PATCH 094/209] Added log4j properties folder for junit test --- build.properties | 3 +++ build.xml | 1 + 2 files changed, 4 insertions(+) diff --git a/build.properties b/build.properties index f363ea1bff..f64eb57de0 100644 --- a/build.properties +++ b/build.properties @@ -36,5 +36,8 @@ tomcat.manager.username=tomcat tomcat.manager.password=tomcat tomcat.context=/voldemort +## Log4j +log4j.properties.dir=src/java + ## Release curr.release=0.90.1 diff --git a/build.xml b/build.xml index 149472494d..d664ea97fb 100644 --- a/build.xml +++ b/build.xml @@ -372,6 +372,7 @@ + From 3aa9d12206242181988e3739805ebfad45c37f88 Mon Sep 17 00:00:00 2001 From: Brendan Harris Date: Thu, 12 Jul 2012 13:14:16 -0700 Subject: [PATCH 095/209] Fixed --seed argument handling so that it actually works, now --- bin/generate_cluster_xml.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/bin/generate_cluster_xml.py b/bin/generate_cluster_xml.py index 5d7c6065b2..08ef4c0f9a 100644 --- a/bin/generate_cluster_xml.py +++ b/bin/generate_cluster_xml.py @@ -5,7 +5,7 @@ import argparse # Get a random seed -seed = int(random.randint(00000000001,99999999999)) +rseed = int(random.randint(00000000001,99999999999)) # Setup and argument parser parser = argparse.ArgumentParser(description='Build a voldemort cluster.xml.') @@ -23,7 +23,7 @@ parser.add_argument('-H', '--http-port', type=int, default=6665, dest='http_port', help='http port number') genType = parser.add_mutually_exclusive_group() -genType.add_argument('-S', '--seed', type=int, default=seed, dest='seed', +genType.add_argument('-S', '--seed', type=int, default=rseed, dest='seed', help='seed for randomizing partition distribution') genType.add_argument('-l', '--loops', type=int, default=1000, dest='loops', help='loop n times, using a different random seed every \ @@ -50,6 +50,7 @@ http_port = args.http_port sock_port = args.sock_port admin_port = args.admin_port +seed = args.seed # Generate the full list of partition IDs part_ids = range(nodes * partitions) From 986c8f23bcd7f3d7bfd77d410afc0277acae5459 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Wed, 4 Jul 2012 13:12:25 -0700 Subject: [PATCH 096/209] BDB Cache partitioning --- .../voldemort/protocol/voldemort_admin_pb2.py | 99 +- .../performance/BdbBuildPerformanceTest.java | 3 +- .../MysqlBuildPerformanceTest.java | 3 +- .../krati/KratiStorageConfiguration.java | 13 +- .../store/krati/KratiStorageEngine.java | 2 +- src/java/voldemort/VoldemortAdminTool.java | 18 +- .../client/protocol/admin/AdminClient.java | 39 + .../client/protocol/pb/VAdminProto.java | 878 ++++++++++++++++-- src/java/voldemort/server/jmx/JmxService.java | 9 - .../admin/AdminServiceRequestHandler.java | 77 +- .../server/storage/StorageService.java | 40 +- .../voldemort/store/StorageConfiguration.java | 11 +- src/java/voldemort/store/StoreDefinition.java | 31 +- .../store/StoreDefinitionBuilder.java | 61 +- .../store/bdb/BdbStorageConfiguration.java | 114 ++- .../memory/CacheStorageConfiguration.java | 10 +- .../memory/InMemoryStorageConfiguration.java | 10 +- .../mysql/MysqlStorageConfiguration.java | 9 +- .../ReadOnlyStorageConfiguration.java | 15 +- .../store/views/ViewStorageConfiguration.java | 7 +- .../voldemort/xml/StoreDefinitionsMapper.java | 11 + src/java/voldemort/xml/stores.xsd | 1 + src/proto/voldemort-admin.proto | 11 + test/common/voldemort/TestUtils.java | 64 ++ .../CacheStorageEnginePerformanceTest.java | 3 +- .../StorageEnginePerformanceTest.java | 2 +- .../performance/benchmark/Benchmark.java | 3 +- .../store/noop/NoopStorageConfiguration.java | 11 +- .../PausableStorageConfiguration.java | 10 +- .../store/bdb/BdbCachePartitioningTest.java | 142 +++ .../store/bdb/BdbSplitStorageEngineTest.java | 8 +- .../store/memory/CacheStorageEngineTest.java | 2 +- 32 files changed, 1553 insertions(+), 164 deletions(-) create mode 100644 test/unit/voldemort/store/bdb/BdbCachePartitioningTest.java diff --git a/clients/python/voldemort/protocol/voldemort_admin_pb2.py b/clients/python/voldemort/protocol/voldemort_admin_pb2.py index e2c3451c07..008ac0cda0 100644 --- a/clients/python/voldemort/protocol/voldemort_admin_pb2.py +++ b/clients/python/voldemort/protocol/voldemort_admin_pb2.py @@ -10,7 +10,7 @@ DESCRIPTOR = descriptor.FileDescriptor( name='voldemort-admin.proto', package='voldemort', - serialized_pb='\n\x15voldemort-admin.proto\x12\tvoldemort\x1a\x16voldemort-client.proto\"!\n\x12GetMetadataRequest\x12\x0b\n\x03key\x18\x01 \x02(\x0c\"]\n\x13GetMetadataResponse\x12%\n\x07version\x18\x01 \x01(\x0b\x32\x14.voldemort.Versioned\x12\x1f\n\x05\x65rror\x18\x02 \x01(\x0b\x32\x10.voldemort.Error\"M\n\x15UpdateMetadataRequest\x12\x0b\n\x03key\x18\x01 \x02(\x0c\x12\'\n\tversioned\x18\x02 \x02(\x0b\x32\x14.voldemort.Versioned\"9\n\x16UpdateMetadataResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"7\n\tFileEntry\x12\x11\n\tfile_name\x18\x01 \x02(\t\x12\x17\n\x0f\x66ile_size_bytes\x18\x02 \x02(\x03\"F\n\x0ePartitionEntry\x12\x0b\n\x03key\x18\x01 \x02(\x0c\x12\'\n\tversioned\x18\x02 \x02(\x0b\x32\x14.voldemort.Versioned\"\x8e\x01\n\x1dUpdatePartitionEntriesRequest\x12\r\n\x05store\x18\x01 \x02(\t\x12\x32\n\x0fpartition_entry\x18\x02 \x02(\x0b\x32\x19.voldemort.PartitionEntry\x12*\n\x06\x66ilter\x18\x03 \x01(\x0b\x32\x1a.voldemort.VoldemortFilter\"A\n\x1eUpdatePartitionEntriesResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"-\n\x0fVoldemortFilter\x12\x0c\n\x04name\x18\x01 \x02(\t\x12\x0c\n\x04\x64\x61ta\x18\x02 \x02(\x0c\"\xaf\x01\n\x18UpdateSlopEntriesRequest\x12\r\n\x05store\x18\x01 \x02(\t\x12\x0b\n\x03key\x18\x02 \x02(\x0c\x12\'\n\x07version\x18\x03 \x02(\x0b\x32\x16.voldemort.VectorClock\x12,\n\x0crequest_type\x18\x04 \x02(\x0e\x32\x16.voldemort.RequestType\x12\r\n\x05value\x18\x05 \x01(\x0c\x12\x11\n\ttransform\x18\x06 \x01(\x0c\"<\n\x19UpdateSlopEntriesResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"d\n\x1a\x46\x65tchPartitionFilesRequest\x12\r\n\x05store\x18\x01 \x02(\t\x12\x37\n\x14replica_to_partition\x18\x02 \x03(\x0b\x32\x19.voldemort.PartitionTuple\"\xd7\x01\n\x1c\x46\x65tchPartitionEntriesRequest\x12\x37\n\x14replica_to_partition\x18\x01 \x03(\x0b\x32\x19.voldemort.PartitionTuple\x12\r\n\x05store\x18\x02 \x02(\t\x12*\n\x06\x66ilter\x18\x03 \x01(\x0b\x32\x1a.voldemort.VoldemortFilter\x12\x14\n\x0c\x66\x65tch_values\x18\x04 \x01(\x08\x12\x14\n\x0cskip_records\x18\x05 \x01(\x03\x12\x17\n\x0finitial_cluster\x18\x06 \x01(\t\"\x81\x01\n\x1d\x46\x65tchPartitionEntriesResponse\x12\x32\n\x0fpartition_entry\x18\x01 \x01(\x0b\x32\x19.voldemort.PartitionEntry\x12\x0b\n\x03key\x18\x02 \x01(\x0c\x12\x1f\n\x05\x65rror\x18\x03 \x01(\x0b\x32\x10.voldemort.Error\"\xac\x01\n\x1d\x44\x65letePartitionEntriesRequest\x12\r\n\x05store\x18\x01 \x02(\t\x12\x37\n\x14replica_to_partition\x18\x02 \x03(\x0b\x32\x19.voldemort.PartitionTuple\x12*\n\x06\x66ilter\x18\x03 \x01(\x0b\x32\x1a.voldemort.VoldemortFilter\x12\x17\n\x0finitial_cluster\x18\x04 \x01(\t\"P\n\x1e\x44\x65letePartitionEntriesResponse\x12\r\n\x05\x63ount\x18\x01 \x01(\x03\x12\x1f\n\x05\x65rror\x18\x02 \x01(\x0b\x32\x10.voldemort.Error\"\xcf\x01\n\x1dInitiateFetchAndUpdateRequest\x12\x0f\n\x07node_id\x18\x01 \x02(\x05\x12\r\n\x05store\x18\x02 \x02(\t\x12*\n\x06\x66ilter\x18\x03 \x01(\x0b\x32\x1a.voldemort.VoldemortFilter\x12\x37\n\x14replica_to_partition\x18\x04 \x03(\x0b\x32\x19.voldemort.PartitionTuple\x12\x17\n\x0finitial_cluster\x18\x05 \x01(\t\x12\x10\n\x08optimize\x18\x06 \x01(\x08\"1\n\x1b\x41syncOperationStatusRequest\x12\x12\n\nrequest_id\x18\x01 \x02(\x05\"/\n\x19\x41syncOperationStopRequest\x12\x12\n\nrequest_id\x18\x01 \x02(\x05\"=\n\x1a\x41syncOperationStopResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"2\n\x19\x41syncOperationListRequest\x12\x15\n\rshow_complete\x18\x02 \x02(\x08\"R\n\x1a\x41syncOperationListResponse\x12\x13\n\x0brequest_ids\x18\x01 \x03(\x05\x12\x1f\n\x05\x65rror\x18\x02 \x01(\x0b\x32\x10.voldemort.Error\":\n\x0ePartitionTuple\x12\x14\n\x0creplica_type\x18\x01 \x02(\x05\x12\x12\n\npartitions\x18\x02 \x03(\x05\"e\n\x16PerStorePartitionTuple\x12\x12\n\nstore_name\x18\x01 \x02(\t\x12\x37\n\x14replica_to_partition\x18\x02 \x03(\x0b\x32\x19.voldemort.PartitionTuple\"\xf8\x01\n\x19RebalancePartitionInfoMap\x12\x12\n\nstealer_id\x18\x01 \x02(\x05\x12\x10\n\x08\x64onor_id\x18\x02 \x02(\x05\x12\x0f\n\x07\x61ttempt\x18\x03 \x02(\x05\x12\x43\n\x18replica_to_add_partition\x18\x04 \x03(\x0b\x32!.voldemort.PerStorePartitionTuple\x12\x46\n\x1breplica_to_delete_partition\x18\x05 \x03(\x0b\x32!.voldemort.PerStorePartitionTuple\x12\x17\n\x0finitial_cluster\x18\x06 \x02(\t\"f\n\x1cInitiateRebalanceNodeRequest\x12\x46\n\x18rebalance_partition_info\x18\x01 \x02(\x0b\x32$.voldemort.RebalancePartitionInfoMap\"m\n#InitiateRebalanceNodeOnDonorRequest\x12\x46\n\x18rebalance_partition_info\x18\x01 \x03(\x0b\x32$.voldemort.RebalancePartitionInfoMap\"\x8a\x01\n\x1c\x41syncOperationStatusResponse\x12\x12\n\nrequest_id\x18\x01 \x01(\x05\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12\x0e\n\x06status\x18\x03 \x01(\t\x12\x10\n\x08\x63omplete\x18\x04 \x01(\x08\x12\x1f\n\x05\x65rror\x18\x05 \x01(\x0b\x32\x10.voldemort.Error\"\'\n\x16TruncateEntriesRequest\x12\r\n\x05store\x18\x01 \x02(\t\":\n\x17TruncateEntriesResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"*\n\x0f\x41\x64\x64StoreRequest\x12\x17\n\x0fstoreDefinition\x18\x01 \x02(\t\"3\n\x10\x41\x64\x64StoreResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"\'\n\x12\x44\x65leteStoreRequest\x12\x11\n\tstoreName\x18\x01 \x02(\t\"6\n\x13\x44\x65leteStoreResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"P\n\x11\x46\x65tchStoreRequest\x12\x12\n\nstore_name\x18\x01 \x02(\t\x12\x11\n\tstore_dir\x18\x02 \x02(\t\x12\x14\n\x0cpush_version\x18\x03 \x01(\x03\"9\n\x10SwapStoreRequest\x12\x12\n\nstore_name\x18\x01 \x02(\t\x12\x11\n\tstore_dir\x18\x02 \x02(\t\"P\n\x11SwapStoreResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\x12\x1a\n\x12previous_store_dir\x18\x02 \x01(\t\"@\n\x14RollbackStoreRequest\x12\x12\n\nstore_name\x18\x01 \x02(\t\x12\x14\n\x0cpush_version\x18\x02 \x02(\x03\"8\n\x15RollbackStoreResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"&\n\x10RepairJobRequest\x12\x12\n\nstore_name\x18\x01 \x01(\t\"4\n\x11RepairJobResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"=\n\x14ROStoreVersionDirMap\x12\x12\n\nstore_name\x18\x01 \x02(\t\x12\x11\n\tstore_dir\x18\x02 \x02(\t\"/\n\x19GetROMaxVersionDirRequest\x12\x12\n\nstore_name\x18\x01 \x03(\t\"y\n\x1aGetROMaxVersionDirResponse\x12:\n\x11ro_store_versions\x18\x01 \x03(\x0b\x32\x1f.voldemort.ROStoreVersionDirMap\x12\x1f\n\x05\x65rror\x18\x02 \x01(\x0b\x32\x10.voldemort.Error\"3\n\x1dGetROCurrentVersionDirRequest\x12\x12\n\nstore_name\x18\x01 \x03(\t\"}\n\x1eGetROCurrentVersionDirResponse\x12:\n\x11ro_store_versions\x18\x01 \x03(\x0b\x32\x1f.voldemort.ROStoreVersionDirMap\x12\x1f\n\x05\x65rror\x18\x02 \x01(\x0b\x32\x10.voldemort.Error\"/\n\x19GetROStorageFormatRequest\x12\x12\n\nstore_name\x18\x01 \x03(\t\"y\n\x1aGetROStorageFormatResponse\x12:\n\x11ro_store_versions\x18\x01 \x03(\x0b\x32\x1f.voldemort.ROStoreVersionDirMap\x12\x1f\n\x05\x65rror\x18\x02 \x01(\x0b\x32\x10.voldemort.Error\"@\n\x17\x46\x61iledFetchStoreRequest\x12\x12\n\nstore_name\x18\x01 \x02(\t\x12\x11\n\tstore_dir\x18\x02 \x02(\t\";\n\x18\x46\x61iledFetchStoreResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"\xe6\x01\n\x1bRebalanceStateChangeRequest\x12K\n\x1drebalance_partition_info_list\x18\x01 \x03(\x0b\x32$.voldemort.RebalancePartitionInfoMap\x12\x16\n\x0e\x63luster_string\x18\x02 \x02(\t\x12\x0f\n\x07swap_ro\x18\x03 \x02(\x08\x12\x1f\n\x17\x63hange_cluster_metadata\x18\x04 \x02(\x08\x12\x1e\n\x16\x63hange_rebalance_state\x18\x05 \x02(\x08\x12\x10\n\x08rollback\x18\x06 \x02(\x08\"?\n\x1cRebalanceStateChangeResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"G\n DeleteStoreRebalanceStateRequest\x12\x12\n\nstore_name\x18\x01 \x02(\t\x12\x0f\n\x07node_id\x18\x02 \x02(\x05\"D\n!DeleteStoreRebalanceStateResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"h\n\x13NativeBackupRequest\x12\x12\n\nstore_name\x18\x01 \x02(\t\x12\x12\n\nbackup_dir\x18\x02 \x02(\t\x12\x14\n\x0cverify_files\x18\x03 \x02(\x08\x12\x13\n\x0bincremental\x18\x04 \x02(\x08\"\xb7\x0e\n\x15VoldemortAdminRequest\x12)\n\x04type\x18\x01 \x02(\x0e\x32\x1b.voldemort.AdminRequestType\x12\x33\n\x0cget_metadata\x18\x02 \x01(\x0b\x32\x1d.voldemort.GetMetadataRequest\x12\x39\n\x0fupdate_metadata\x18\x03 \x01(\x0b\x32 .voldemort.UpdateMetadataRequest\x12J\n\x18update_partition_entries\x18\x04 \x01(\x0b\x32(.voldemort.UpdatePartitionEntriesRequest\x12H\n\x17\x66\x65tch_partition_entries\x18\x05 \x01(\x0b\x32\'.voldemort.FetchPartitionEntriesRequest\x12J\n\x18\x64\x65lete_partition_entries\x18\x06 \x01(\x0b\x32(.voldemort.DeletePartitionEntriesRequest\x12K\n\x19initiate_fetch_and_update\x18\x07 \x01(\x0b\x32(.voldemort.InitiateFetchAndUpdateRequest\x12\x46\n\x16\x61sync_operation_status\x18\x08 \x01(\x0b\x32&.voldemort.AsyncOperationStatusRequest\x12H\n\x17initiate_rebalance_node\x18\t \x01(\x0b\x32\'.voldemort.InitiateRebalanceNodeRequest\x12\x42\n\x14\x61sync_operation_stop\x18\n \x01(\x0b\x32$.voldemort.AsyncOperationStopRequest\x12\x42\n\x14\x61sync_operation_list\x18\x0b \x01(\x0b\x32$.voldemort.AsyncOperationListRequest\x12;\n\x10truncate_entries\x18\x0c \x01(\x0b\x32!.voldemort.TruncateEntriesRequest\x12-\n\tadd_store\x18\r \x01(\x0b\x32\x1a.voldemort.AddStoreRequest\x12\x33\n\x0c\x64\x65lete_store\x18\x0e \x01(\x0b\x32\x1d.voldemort.DeleteStoreRequest\x12\x31\n\x0b\x66\x65tch_store\x18\x0f \x01(\x0b\x32\x1c.voldemort.FetchStoreRequest\x12/\n\nswap_store\x18\x10 \x01(\x0b\x32\x1b.voldemort.SwapStoreRequest\x12\x37\n\x0erollback_store\x18\x11 \x01(\x0b\x32\x1f.voldemort.RollbackStoreRequest\x12\x44\n\x16get_ro_max_version_dir\x18\x12 \x01(\x0b\x32$.voldemort.GetROMaxVersionDirRequest\x12L\n\x1aget_ro_current_version_dir\x18\x13 \x01(\x0b\x32(.voldemort.GetROCurrentVersionDirRequest\x12\x44\n\x15\x66\x65tch_partition_files\x18\x14 \x01(\x0b\x32%.voldemort.FetchPartitionFilesRequest\x12@\n\x13update_slop_entries\x18\x16 \x01(\x0b\x32#.voldemort.UpdateSlopEntriesRequest\x12>\n\x12\x66\x61iled_fetch_store\x18\x18 \x01(\x0b\x32\".voldemort.FailedFetchStoreRequest\x12\x43\n\x15get_ro_storage_format\x18\x19 \x01(\x0b\x32$.voldemort.GetROStorageFormatRequest\x12\x46\n\x16rebalance_state_change\x18\x1a \x01(\x0b\x32&.voldemort.RebalanceStateChangeRequest\x12/\n\nrepair_job\x18\x1b \x01(\x0b\x32\x1b.voldemort.RepairJobRequest\x12X\n initiate_rebalance_node_on_donor\x18\x1c \x01(\x0b\x32..voldemort.InitiateRebalanceNodeOnDonorRequest\x12Q\n\x1c\x64\x65lete_store_rebalance_state\x18\x1d \x01(\x0b\x32+.voldemort.DeleteStoreRebalanceStateRequest\x12\x35\n\rnative_backup\x18\x1e \x01(\x0b\x32\x1e.voldemort.NativeBackupRequest*\xb4\x05\n\x10\x41\x64minRequestType\x12\x10\n\x0cGET_METADATA\x10\x00\x12\x13\n\x0fUPDATE_METADATA\x10\x01\x12\x1c\n\x18UPDATE_PARTITION_ENTRIES\x10\x02\x12\x1b\n\x17\x46\x45TCH_PARTITION_ENTRIES\x10\x03\x12\x1c\n\x18\x44\x45LETE_PARTITION_ENTRIES\x10\x04\x12\x1d\n\x19INITIATE_FETCH_AND_UPDATE\x10\x05\x12\x1a\n\x16\x41SYNC_OPERATION_STATUS\x10\x06\x12\x1b\n\x17INITIATE_REBALANCE_NODE\x10\x07\x12\x18\n\x14\x41SYNC_OPERATION_STOP\x10\x08\x12\x18\n\x14\x41SYNC_OPERATION_LIST\x10\t\x12\x14\n\x10TRUNCATE_ENTRIES\x10\n\x12\r\n\tADD_STORE\x10\x0b\x12\x10\n\x0c\x44\x45LETE_STORE\x10\x0c\x12\x0f\n\x0b\x46\x45TCH_STORE\x10\r\x12\x0e\n\nSWAP_STORE\x10\x0e\x12\x12\n\x0eROLLBACK_STORE\x10\x0f\x12\x1a\n\x16GET_RO_MAX_VERSION_DIR\x10\x10\x12\x1e\n\x1aGET_RO_CURRENT_VERSION_DIR\x10\x11\x12\x19\n\x15\x46\x45TCH_PARTITION_FILES\x10\x12\x12\x17\n\x13UPDATE_SLOP_ENTRIES\x10\x14\x12\x16\n\x12\x46\x41ILED_FETCH_STORE\x10\x16\x12\x19\n\x15GET_RO_STORAGE_FORMAT\x10\x17\x12\x1a\n\x16REBALANCE_STATE_CHANGE\x10\x18\x12\x0e\n\nREPAIR_JOB\x10\x19\x12$\n INITIATE_REBALANCE_NODE_ON_DONOR\x10\x1a\x12 \n\x1c\x44\x45LETE_STORE_REBALANCE_STATE\x10\x1b\x12\x11\n\rNATIVE_BACKUP\x10\x1c\x42-\n\x1cvoldemort.client.protocol.pbB\x0bVAdminProtoH\x01') + serialized_pb='\n\x15voldemort-admin.proto\x12\tvoldemort\x1a\x16voldemort-client.proto\"!\n\x12GetMetadataRequest\x12\x0b\n\x03key\x18\x01 \x02(\x0c\"]\n\x13GetMetadataResponse\x12%\n\x07version\x18\x01 \x01(\x0b\x32\x14.voldemort.Versioned\x12\x1f\n\x05\x65rror\x18\x02 \x01(\x0b\x32\x10.voldemort.Error\"M\n\x15UpdateMetadataRequest\x12\x0b\n\x03key\x18\x01 \x02(\x0c\x12\'\n\tversioned\x18\x02 \x02(\x0b\x32\x14.voldemort.Versioned\"9\n\x16UpdateMetadataResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"7\n\tFileEntry\x12\x11\n\tfile_name\x18\x01 \x02(\t\x12\x17\n\x0f\x66ile_size_bytes\x18\x02 \x02(\x03\"F\n\x0ePartitionEntry\x12\x0b\n\x03key\x18\x01 \x02(\x0c\x12\'\n\tversioned\x18\x02 \x02(\x0b\x32\x14.voldemort.Versioned\"\x8e\x01\n\x1dUpdatePartitionEntriesRequest\x12\r\n\x05store\x18\x01 \x02(\t\x12\x32\n\x0fpartition_entry\x18\x02 \x02(\x0b\x32\x19.voldemort.PartitionEntry\x12*\n\x06\x66ilter\x18\x03 \x01(\x0b\x32\x1a.voldemort.VoldemortFilter\"A\n\x1eUpdatePartitionEntriesResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"-\n\x0fVoldemortFilter\x12\x0c\n\x04name\x18\x01 \x02(\t\x12\x0c\n\x04\x64\x61ta\x18\x02 \x02(\x0c\"\xaf\x01\n\x18UpdateSlopEntriesRequest\x12\r\n\x05store\x18\x01 \x02(\t\x12\x0b\n\x03key\x18\x02 \x02(\x0c\x12\'\n\x07version\x18\x03 \x02(\x0b\x32\x16.voldemort.VectorClock\x12,\n\x0crequest_type\x18\x04 \x02(\x0e\x32\x16.voldemort.RequestType\x12\r\n\x05value\x18\x05 \x01(\x0c\x12\x11\n\ttransform\x18\x06 \x01(\x0c\"<\n\x19UpdateSlopEntriesResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"d\n\x1a\x46\x65tchPartitionFilesRequest\x12\r\n\x05store\x18\x01 \x02(\t\x12\x37\n\x14replica_to_partition\x18\x02 \x03(\x0b\x32\x19.voldemort.PartitionTuple\"\xd7\x01\n\x1c\x46\x65tchPartitionEntriesRequest\x12\x37\n\x14replica_to_partition\x18\x01 \x03(\x0b\x32\x19.voldemort.PartitionTuple\x12\r\n\x05store\x18\x02 \x02(\t\x12*\n\x06\x66ilter\x18\x03 \x01(\x0b\x32\x1a.voldemort.VoldemortFilter\x12\x14\n\x0c\x66\x65tch_values\x18\x04 \x01(\x08\x12\x14\n\x0cskip_records\x18\x05 \x01(\x03\x12\x17\n\x0finitial_cluster\x18\x06 \x01(\t\"\x81\x01\n\x1d\x46\x65tchPartitionEntriesResponse\x12\x32\n\x0fpartition_entry\x18\x01 \x01(\x0b\x32\x19.voldemort.PartitionEntry\x12\x0b\n\x03key\x18\x02 \x01(\x0c\x12\x1f\n\x05\x65rror\x18\x03 \x01(\x0b\x32\x10.voldemort.Error\"\xac\x01\n\x1d\x44\x65letePartitionEntriesRequest\x12\r\n\x05store\x18\x01 \x02(\t\x12\x37\n\x14replica_to_partition\x18\x02 \x03(\x0b\x32\x19.voldemort.PartitionTuple\x12*\n\x06\x66ilter\x18\x03 \x01(\x0b\x32\x1a.voldemort.VoldemortFilter\x12\x17\n\x0finitial_cluster\x18\x04 \x01(\t\"P\n\x1e\x44\x65letePartitionEntriesResponse\x12\r\n\x05\x63ount\x18\x01 \x01(\x03\x12\x1f\n\x05\x65rror\x18\x02 \x01(\x0b\x32\x10.voldemort.Error\"\xcf\x01\n\x1dInitiateFetchAndUpdateRequest\x12\x0f\n\x07node_id\x18\x01 \x02(\x05\x12\r\n\x05store\x18\x02 \x02(\t\x12*\n\x06\x66ilter\x18\x03 \x01(\x0b\x32\x1a.voldemort.VoldemortFilter\x12\x37\n\x14replica_to_partition\x18\x04 \x03(\x0b\x32\x19.voldemort.PartitionTuple\x12\x17\n\x0finitial_cluster\x18\x05 \x01(\t\x12\x10\n\x08optimize\x18\x06 \x01(\x08\"1\n\x1b\x41syncOperationStatusRequest\x12\x12\n\nrequest_id\x18\x01 \x02(\x05\"/\n\x19\x41syncOperationStopRequest\x12\x12\n\nrequest_id\x18\x01 \x02(\x05\"=\n\x1a\x41syncOperationStopResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"2\n\x19\x41syncOperationListRequest\x12\x15\n\rshow_complete\x18\x02 \x02(\x08\"R\n\x1a\x41syncOperationListResponse\x12\x13\n\x0brequest_ids\x18\x01 \x03(\x05\x12\x1f\n\x05\x65rror\x18\x02 \x01(\x0b\x32\x10.voldemort.Error\":\n\x0ePartitionTuple\x12\x14\n\x0creplica_type\x18\x01 \x02(\x05\x12\x12\n\npartitions\x18\x02 \x03(\x05\"e\n\x16PerStorePartitionTuple\x12\x12\n\nstore_name\x18\x01 \x02(\t\x12\x37\n\x14replica_to_partition\x18\x02 \x03(\x0b\x32\x19.voldemort.PartitionTuple\"\xf8\x01\n\x19RebalancePartitionInfoMap\x12\x12\n\nstealer_id\x18\x01 \x02(\x05\x12\x10\n\x08\x64onor_id\x18\x02 \x02(\x05\x12\x0f\n\x07\x61ttempt\x18\x03 \x02(\x05\x12\x43\n\x18replica_to_add_partition\x18\x04 \x03(\x0b\x32!.voldemort.PerStorePartitionTuple\x12\x46\n\x1breplica_to_delete_partition\x18\x05 \x03(\x0b\x32!.voldemort.PerStorePartitionTuple\x12\x17\n\x0finitial_cluster\x18\x06 \x02(\t\"f\n\x1cInitiateRebalanceNodeRequest\x12\x46\n\x18rebalance_partition_info\x18\x01 \x02(\x0b\x32$.voldemort.RebalancePartitionInfoMap\"m\n#InitiateRebalanceNodeOnDonorRequest\x12\x46\n\x18rebalance_partition_info\x18\x01 \x03(\x0b\x32$.voldemort.RebalancePartitionInfoMap\"\x8a\x01\n\x1c\x41syncOperationStatusResponse\x12\x12\n\nrequest_id\x18\x01 \x01(\x05\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12\x0e\n\x06status\x18\x03 \x01(\t\x12\x10\n\x08\x63omplete\x18\x04 \x01(\x08\x12\x1f\n\x05\x65rror\x18\x05 \x01(\x0b\x32\x10.voldemort.Error\"\'\n\x16TruncateEntriesRequest\x12\r\n\x05store\x18\x01 \x02(\t\":\n\x17TruncateEntriesResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"*\n\x0f\x41\x64\x64StoreRequest\x12\x17\n\x0fstoreDefinition\x18\x01 \x02(\t\"3\n\x10\x41\x64\x64StoreResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"\'\n\x12\x44\x65leteStoreRequest\x12\x11\n\tstoreName\x18\x01 \x02(\t\"6\n\x13\x44\x65leteStoreResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"P\n\x11\x46\x65tchStoreRequest\x12\x12\n\nstore_name\x18\x01 \x02(\t\x12\x11\n\tstore_dir\x18\x02 \x02(\t\x12\x14\n\x0cpush_version\x18\x03 \x01(\x03\"9\n\x10SwapStoreRequest\x12\x12\n\nstore_name\x18\x01 \x02(\t\x12\x11\n\tstore_dir\x18\x02 \x02(\t\"P\n\x11SwapStoreResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\x12\x1a\n\x12previous_store_dir\x18\x02 \x01(\t\"@\n\x14RollbackStoreRequest\x12\x12\n\nstore_name\x18\x01 \x02(\t\x12\x14\n\x0cpush_version\x18\x02 \x02(\x03\"8\n\x15RollbackStoreResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"&\n\x10RepairJobRequest\x12\x12\n\nstore_name\x18\x01 \x01(\t\"4\n\x11RepairJobResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"=\n\x14ROStoreVersionDirMap\x12\x12\n\nstore_name\x18\x01 \x02(\t\x12\x11\n\tstore_dir\x18\x02 \x02(\t\"/\n\x19GetROMaxVersionDirRequest\x12\x12\n\nstore_name\x18\x01 \x03(\t\"y\n\x1aGetROMaxVersionDirResponse\x12:\n\x11ro_store_versions\x18\x01 \x03(\x0b\x32\x1f.voldemort.ROStoreVersionDirMap\x12\x1f\n\x05\x65rror\x18\x02 \x01(\x0b\x32\x10.voldemort.Error\"3\n\x1dGetROCurrentVersionDirRequest\x12\x12\n\nstore_name\x18\x01 \x03(\t\"}\n\x1eGetROCurrentVersionDirResponse\x12:\n\x11ro_store_versions\x18\x01 \x03(\x0b\x32\x1f.voldemort.ROStoreVersionDirMap\x12\x1f\n\x05\x65rror\x18\x02 \x01(\x0b\x32\x10.voldemort.Error\"/\n\x19GetROStorageFormatRequest\x12\x12\n\nstore_name\x18\x01 \x03(\t\"y\n\x1aGetROStorageFormatResponse\x12:\n\x11ro_store_versions\x18\x01 \x03(\x0b\x32\x1f.voldemort.ROStoreVersionDirMap\x12\x1f\n\x05\x65rror\x18\x02 \x01(\x0b\x32\x10.voldemort.Error\"@\n\x17\x46\x61iledFetchStoreRequest\x12\x12\n\nstore_name\x18\x01 \x02(\t\x12\x11\n\tstore_dir\x18\x02 \x02(\t\";\n\x18\x46\x61iledFetchStoreResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"\xe6\x01\n\x1bRebalanceStateChangeRequest\x12K\n\x1drebalance_partition_info_list\x18\x01 \x03(\x0b\x32$.voldemort.RebalancePartitionInfoMap\x12\x16\n\x0e\x63luster_string\x18\x02 \x02(\t\x12\x0f\n\x07swap_ro\x18\x03 \x02(\x08\x12\x1f\n\x17\x63hange_cluster_metadata\x18\x04 \x02(\x08\x12\x1e\n\x16\x63hange_rebalance_state\x18\x05 \x02(\x08\x12\x10\n\x08rollback\x18\x06 \x02(\x08\"?\n\x1cRebalanceStateChangeResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"G\n DeleteStoreRebalanceStateRequest\x12\x12\n\nstore_name\x18\x01 \x02(\t\x12\x0f\n\x07node_id\x18\x02 \x02(\x05\"D\n!DeleteStoreRebalanceStateResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"h\n\x13NativeBackupRequest\x12\x12\n\nstore_name\x18\x01 \x02(\t\x12\x12\n\nbackup_dir\x18\x02 \x02(\t\x12\x14\n\x0cverify_files\x18\x03 \x02(\x08\x12\x13\n\x0bincremental\x18\x04 \x02(\x08\">\n\x14ReserveMemoryRequest\x12\x12\n\nstore_name\x18\x01 \x02(\t\x12\x12\n\nsize_in_mb\x18\x02 \x02(\x03\"8\n\x15ReserveMemoryResponse\x12\x1f\n\x05\x65rror\x18\x01 \x01(\x0b\x32\x10.voldemort.Error\"\xf0\x0e\n\x15VoldemortAdminRequest\x12)\n\x04type\x18\x01 \x02(\x0e\x32\x1b.voldemort.AdminRequestType\x12\x33\n\x0cget_metadata\x18\x02 \x01(\x0b\x32\x1d.voldemort.GetMetadataRequest\x12\x39\n\x0fupdate_metadata\x18\x03 \x01(\x0b\x32 .voldemort.UpdateMetadataRequest\x12J\n\x18update_partition_entries\x18\x04 \x01(\x0b\x32(.voldemort.UpdatePartitionEntriesRequest\x12H\n\x17\x66\x65tch_partition_entries\x18\x05 \x01(\x0b\x32\'.voldemort.FetchPartitionEntriesRequest\x12J\n\x18\x64\x65lete_partition_entries\x18\x06 \x01(\x0b\x32(.voldemort.DeletePartitionEntriesRequest\x12K\n\x19initiate_fetch_and_update\x18\x07 \x01(\x0b\x32(.voldemort.InitiateFetchAndUpdateRequest\x12\x46\n\x16\x61sync_operation_status\x18\x08 \x01(\x0b\x32&.voldemort.AsyncOperationStatusRequest\x12H\n\x17initiate_rebalance_node\x18\t \x01(\x0b\x32\'.voldemort.InitiateRebalanceNodeRequest\x12\x42\n\x14\x61sync_operation_stop\x18\n \x01(\x0b\x32$.voldemort.AsyncOperationStopRequest\x12\x42\n\x14\x61sync_operation_list\x18\x0b \x01(\x0b\x32$.voldemort.AsyncOperationListRequest\x12;\n\x10truncate_entries\x18\x0c \x01(\x0b\x32!.voldemort.TruncateEntriesRequest\x12-\n\tadd_store\x18\r \x01(\x0b\x32\x1a.voldemort.AddStoreRequest\x12\x33\n\x0c\x64\x65lete_store\x18\x0e \x01(\x0b\x32\x1d.voldemort.DeleteStoreRequest\x12\x31\n\x0b\x66\x65tch_store\x18\x0f \x01(\x0b\x32\x1c.voldemort.FetchStoreRequest\x12/\n\nswap_store\x18\x10 \x01(\x0b\x32\x1b.voldemort.SwapStoreRequest\x12\x37\n\x0erollback_store\x18\x11 \x01(\x0b\x32\x1f.voldemort.RollbackStoreRequest\x12\x44\n\x16get_ro_max_version_dir\x18\x12 \x01(\x0b\x32$.voldemort.GetROMaxVersionDirRequest\x12L\n\x1aget_ro_current_version_dir\x18\x13 \x01(\x0b\x32(.voldemort.GetROCurrentVersionDirRequest\x12\x44\n\x15\x66\x65tch_partition_files\x18\x14 \x01(\x0b\x32%.voldemort.FetchPartitionFilesRequest\x12@\n\x13update_slop_entries\x18\x16 \x01(\x0b\x32#.voldemort.UpdateSlopEntriesRequest\x12>\n\x12\x66\x61iled_fetch_store\x18\x18 \x01(\x0b\x32\".voldemort.FailedFetchStoreRequest\x12\x43\n\x15get_ro_storage_format\x18\x19 \x01(\x0b\x32$.voldemort.GetROStorageFormatRequest\x12\x46\n\x16rebalance_state_change\x18\x1a \x01(\x0b\x32&.voldemort.RebalanceStateChangeRequest\x12/\n\nrepair_job\x18\x1b \x01(\x0b\x32\x1b.voldemort.RepairJobRequest\x12X\n initiate_rebalance_node_on_donor\x18\x1c \x01(\x0b\x32..voldemort.InitiateRebalanceNodeOnDonorRequest\x12Q\n\x1c\x64\x65lete_store_rebalance_state\x18\x1d \x01(\x0b\x32+.voldemort.DeleteStoreRebalanceStateRequest\x12\x35\n\rnative_backup\x18\x1e \x01(\x0b\x32\x1e.voldemort.NativeBackupRequest\x12\x37\n\x0ereserve_memory\x18\x1f \x01(\x0b\x32\x1f.voldemort.ReserveMemoryRequest*\xc8\x05\n\x10\x41\x64minRequestType\x12\x10\n\x0cGET_METADATA\x10\x00\x12\x13\n\x0fUPDATE_METADATA\x10\x01\x12\x1c\n\x18UPDATE_PARTITION_ENTRIES\x10\x02\x12\x1b\n\x17\x46\x45TCH_PARTITION_ENTRIES\x10\x03\x12\x1c\n\x18\x44\x45LETE_PARTITION_ENTRIES\x10\x04\x12\x1d\n\x19INITIATE_FETCH_AND_UPDATE\x10\x05\x12\x1a\n\x16\x41SYNC_OPERATION_STATUS\x10\x06\x12\x1b\n\x17INITIATE_REBALANCE_NODE\x10\x07\x12\x18\n\x14\x41SYNC_OPERATION_STOP\x10\x08\x12\x18\n\x14\x41SYNC_OPERATION_LIST\x10\t\x12\x14\n\x10TRUNCATE_ENTRIES\x10\n\x12\r\n\tADD_STORE\x10\x0b\x12\x10\n\x0c\x44\x45LETE_STORE\x10\x0c\x12\x0f\n\x0b\x46\x45TCH_STORE\x10\r\x12\x0e\n\nSWAP_STORE\x10\x0e\x12\x12\n\x0eROLLBACK_STORE\x10\x0f\x12\x1a\n\x16GET_RO_MAX_VERSION_DIR\x10\x10\x12\x1e\n\x1aGET_RO_CURRENT_VERSION_DIR\x10\x11\x12\x19\n\x15\x46\x45TCH_PARTITION_FILES\x10\x12\x12\x17\n\x13UPDATE_SLOP_ENTRIES\x10\x14\x12\x16\n\x12\x46\x41ILED_FETCH_STORE\x10\x16\x12\x19\n\x15GET_RO_STORAGE_FORMAT\x10\x17\x12\x1a\n\x16REBALANCE_STATE_CHANGE\x10\x18\x12\x0e\n\nREPAIR_JOB\x10\x19\x12$\n INITIATE_REBALANCE_NODE_ON_DONOR\x10\x1a\x12 \n\x1c\x44\x45LETE_STORE_REBALANCE_STATE\x10\x1b\x12\x11\n\rNATIVE_BACKUP\x10\x1c\x12\x12\n\x0eRESERVE_MEMORY\x10\x1d\x42-\n\x1cvoldemort.client.protocol.pbB\x0bVAdminProtoH\x01') _ADMINREQUESTTYPE = descriptor.EnumDescriptor( name='AdminRequestType', @@ -126,11 +126,15 @@ name='NATIVE_BACKUP', index=26, number=28, options=None, type=None), + descriptor.EnumValueDescriptor( + name='RESERVE_MEMORY', index=27, number=29, + options=None, + type=None), ], containing_type=None, options=None, - serialized_start=6792, - serialized_end=7484, + serialized_start=6971, + serialized_end=7683, ) @@ -161,6 +165,7 @@ INITIATE_REBALANCE_NODE_ON_DONOR = 26 DELETE_STORE_REBALANCE_STATE = 27 NATIVE_BACKUP = 28 +RESERVE_MEMORY = 29 @@ -2124,6 +2129,69 @@ ) +_RESERVEMEMORYREQUEST = descriptor.Descriptor( + name='ReserveMemoryRequest', + full_name='voldemort.ReserveMemoryRequest', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + descriptor.FieldDescriptor( + name='store_name', full_name='voldemort.ReserveMemoryRequest.store_name', index=0, + number=1, type=9, cpp_type=9, label=2, + has_default_value=False, default_value=unicode("", "utf-8"), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + descriptor.FieldDescriptor( + name='size_in_mb', full_name='voldemort.ReserveMemoryRequest.size_in_mb', index=1, + number=2, type=3, cpp_type=2, label=2, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + extension_ranges=[], + serialized_start=4941, + serialized_end=5003, +) + + +_RESERVEMEMORYRESPONSE = descriptor.Descriptor( + name='ReserveMemoryResponse', + full_name='voldemort.ReserveMemoryResponse', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + descriptor.FieldDescriptor( + name='error', full_name='voldemort.ReserveMemoryResponse.error', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + options=None, + is_extendable=False, + extension_ranges=[], + serialized_start=5005, + serialized_end=5061, +) + + _VOLDEMORTADMINREQUEST = descriptor.Descriptor( name='VoldemortAdminRequest', full_name='voldemort.VoldemortAdminRequest', @@ -2327,6 +2395,13 @@ message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None), + descriptor.FieldDescriptor( + name='reserve_memory', full_name='voldemort.VoldemortAdminRequest.reserve_memory', index=28, + number=31, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None), ], extensions=[ ], @@ -2336,8 +2411,8 @@ options=None, is_extendable=False, extension_ranges=[], - serialized_start=4942, - serialized_end=6789, + serialized_start=5064, + serialized_end=6968, ) import voldemort_client_pb2 @@ -2387,6 +2462,7 @@ _REBALANCESTATECHANGEREQUEST.fields_by_name['rebalance_partition_info_list'].message_type = _REBALANCEPARTITIONINFOMAP _REBALANCESTATECHANGERESPONSE.fields_by_name['error'].message_type = voldemort_client_pb2._ERROR _DELETESTOREREBALANCESTATERESPONSE.fields_by_name['error'].message_type = voldemort_client_pb2._ERROR +_RESERVEMEMORYRESPONSE.fields_by_name['error'].message_type = voldemort_client_pb2._ERROR _VOLDEMORTADMINREQUEST.fields_by_name['type'].enum_type = _ADMINREQUESTTYPE _VOLDEMORTADMINREQUEST.fields_by_name['get_metadata'].message_type = _GETMETADATAREQUEST _VOLDEMORTADMINREQUEST.fields_by_name['update_metadata'].message_type = _UPDATEMETADATAREQUEST @@ -2415,6 +2491,7 @@ _VOLDEMORTADMINREQUEST.fields_by_name['initiate_rebalance_node_on_donor'].message_type = _INITIATEREBALANCENODEONDONORREQUEST _VOLDEMORTADMINREQUEST.fields_by_name['delete_store_rebalance_state'].message_type = _DELETESTOREREBALANCESTATEREQUEST _VOLDEMORTADMINREQUEST.fields_by_name['native_backup'].message_type = _NATIVEBACKUPREQUEST +_VOLDEMORTADMINREQUEST.fields_by_name['reserve_memory'].message_type = _RESERVEMEMORYREQUEST class GetMetadataRequest(message.Message): __metaclass__ = reflection.GeneratedProtocolMessageType @@ -2746,6 +2823,18 @@ class NativeBackupRequest(message.Message): # @@protoc_insertion_point(class_scope:voldemort.NativeBackupRequest) +class ReserveMemoryRequest(message.Message): + __metaclass__ = reflection.GeneratedProtocolMessageType + DESCRIPTOR = _RESERVEMEMORYREQUEST + + # @@protoc_insertion_point(class_scope:voldemort.ReserveMemoryRequest) + +class ReserveMemoryResponse(message.Message): + __metaclass__ = reflection.GeneratedProtocolMessageType + DESCRIPTOR = _RESERVEMEMORYRESPONSE + + # @@protoc_insertion_point(class_scope:voldemort.ReserveMemoryResponse) + class VoldemortAdminRequest(message.Message): __metaclass__ = reflection.GeneratedProtocolMessageType DESCRIPTOR = _VOLDEMORTADMINREQUEST diff --git a/contrib/hadoop-store-builder/perf/voldemort/contrib/batchindexer/performance/BdbBuildPerformanceTest.java b/contrib/hadoop-store-builder/perf/voldemort/contrib/batchindexer/performance/BdbBuildPerformanceTest.java index 9df60c11fe..b2c67df2ee 100644 --- a/contrib/hadoop-store-builder/perf/voldemort/contrib/batchindexer/performance/BdbBuildPerformanceTest.java +++ b/contrib/hadoop-store-builder/perf/voldemort/contrib/batchindexer/performance/BdbBuildPerformanceTest.java @@ -28,6 +28,7 @@ import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.SequenceFileRecordReader; +import voldemort.TestUtils; import voldemort.performance.PerformanceTest; import voldemort.server.VoldemortConfig; import voldemort.store.Store; @@ -51,7 +52,7 @@ public static void main(String[] args) throws FileNotFoundException, IOException String storeName = args[1]; String jsonDataFile = args[2]; - final Store store = new BdbStorageConfiguration(new VoldemortConfig(new Props(new File(serverPropsFile)))).getStore(storeName); + final Store store = new BdbStorageConfiguration(new VoldemortConfig(new Props(new File(serverPropsFile)))).getStore(TestUtils.makeStoreDefinition(storeName)); final AtomicInteger obsoletes = new AtomicInteger(0); diff --git a/contrib/hadoop-store-builder/perf/voldemort/contrib/batchindexer/performance/MysqlBuildPerformanceTest.java b/contrib/hadoop-store-builder/perf/voldemort/contrib/batchindexer/performance/MysqlBuildPerformanceTest.java index ccf09e6c1a..d7fe084ea1 100644 --- a/contrib/hadoop-store-builder/perf/voldemort/contrib/batchindexer/performance/MysqlBuildPerformanceTest.java +++ b/contrib/hadoop-store-builder/perf/voldemort/contrib/batchindexer/performance/MysqlBuildPerformanceTest.java @@ -28,6 +28,7 @@ import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.SequenceFileRecordReader; +import voldemort.TestUtils; import voldemort.performance.PerformanceTest; import voldemort.server.VoldemortConfig; import voldemort.store.Store; @@ -51,7 +52,7 @@ public static void main(String[] args) throws FileNotFoundException, IOException String storeName = args[1]; String jsonDataFile = args[2]; - final Store store = new MysqlStorageConfiguration(new VoldemortConfig(new Props(new File(serverPropsFile)))).getStore(storeName); + final Store store = new MysqlStorageConfiguration(new VoldemortConfig(new Props(new File(serverPropsFile)))).getStore(TestUtils.makeStoreDefinition(storeName)); final AtomicInteger obsoletes = new AtomicInteger(0); diff --git a/contrib/krati/src/java/voldemort/store/krati/KratiStorageConfiguration.java b/contrib/krati/src/java/voldemort/store/krati/KratiStorageConfiguration.java index 236ffef6cb..5f3a41574c 100644 --- a/contrib/krati/src/java/voldemort/store/krati/KratiStorageConfiguration.java +++ b/contrib/krati/src/java/voldemort/store/krati/KratiStorageConfiguration.java @@ -4,11 +4,14 @@ import krati.core.segment.MappedSegmentFactory; import krati.core.segment.SegmentFactory; + import org.apache.log4j.Logger; +import voldemort.VoldemortException; import voldemort.server.VoldemortConfig; import voldemort.store.StorageConfiguration; import voldemort.store.StorageEngine; +import voldemort.store.StoreDefinition; import voldemort.utils.ByteArray; import voldemort.utils.Props; import voldemort.utils.ReflectUtils; @@ -42,16 +45,16 @@ public KratiStorageConfiguration(VoldemortConfig config) { public void close() {} - public StorageEngine getStore(String storeName) { + public StorageEngine getStore(StoreDefinition storeDef) { synchronized(lock) { - File storeDir = new File(dataDirectory, storeName); + File storeDir = new File(dataDirectory, storeDef.getName()); if(!storeDir.exists()) { logger.info("Creating krati data directory '" + storeDir.getAbsolutePath() + "'."); storeDir.mkdirs(); } SegmentFactory segmentFactory = (SegmentFactory) ReflectUtils.callConstructor(factoryClass); - return new KratiStorageEngine(storeName, + return new KratiStorageEngine(storeDef.getName(), segmentFactory, segmentFileSizeMb, lockStripes, @@ -65,4 +68,8 @@ public String getType() { return TYPE_NAME; } + public void update(StoreDefinition storeDef) { + throw new VoldemortException("Storage config updates not permitted for " + + this.getClass().getCanonicalName()); + } } diff --git a/contrib/krati/src/java/voldemort/store/krati/KratiStorageEngine.java b/contrib/krati/src/java/voldemort/store/krati/KratiStorageEngine.java index 6f5ccac8e2..098c083df3 100644 --- a/contrib/krati/src/java/voldemort/store/krati/KratiStorageEngine.java +++ b/contrib/krati/src/java/voldemort/store/krati/KratiStorageEngine.java @@ -291,7 +291,7 @@ public KratiClosableIterator(List>> list) { } public void close() { - // Nothing to close here + // Nothing to close here } public boolean hasNext() { diff --git a/src/java/voldemort/VoldemortAdminTool.java b/src/java/voldemort/VoldemortAdminTool.java index bc22f8ab56..4603230ec3 100644 --- a/src/java/voldemort/VoldemortAdminTool.java +++ b/src/java/voldemort/VoldemortAdminTool.java @@ -220,6 +220,10 @@ public static void main(String[] args) throws Exception { .withRequiredArg() .describedAs("version") .ofType(Long.class); + parser.accepts("reserve-memory", "Memory in MB to reserve for the store") + .withRequiredArg() + .describedAs("size-in-mb") + .ofType(Long.class); OptionSet options = parser.parse(args); @@ -236,7 +240,7 @@ public static void main(String[] args) throws Exception { || options.has("ro-metadata") || options.has("set-metadata") || options.has("get-metadata") || options.has("check-metadata") || options.has("key-distribution")) || options.has("truncate") || options.has("clear-rebalancing-metadata") - || options.has("async") || options.has("native-backup") || options.has("rollback"))) { + || options.has("async") || options.has("native-backup") || options.has("rollback") || options.has("reserve-memory"))) { System.err.println("Missing required arguments: " + Joiner.on(", ").join(missing)); printHelp(System.err, parser); System.exit(1); @@ -311,11 +315,17 @@ public static void main(String[] args) throws Exception { } ops += "o"; } + if(options.has("reserve-memory")) { + if(!options.has("stores")) { + Utils.croak("Specify the list of stores to reserve memory"); + } + ops += "f"; + } if(ops.length() < 1) { Utils.croak("At least one of (delete-partitions, restore, add-node, fetch-entries, " + "fetch-keys, add-stores, delete-store, update-entries, get-metadata, ro-metadata, " + "set-metadata, check-metadata, key-distribution, clear-rebalancing-metadata, async, " - + "repair-job, native-backup) must be specified"); + + "repair-job, native-backup, rollback, reserve-memory) must be specified"); } List storeNames = null; @@ -480,6 +490,10 @@ public static void main(String[] args) throws Exception { long pushVersion = (Long) options.valueOf("version"); executeRollback(nodeId, storeName, pushVersion, adminClient); } + if(ops.contains("f")) { + long reserveMB = (Long) options.valueOf("reserve-memory"); + adminClient.reserveMemory(nodeId, storeNames, reserveMB); + } } catch(Exception e) { e.printStackTrace(); Utils.croak(e.getMessage()); diff --git a/src/java/voldemort/client/protocol/admin/AdminClient.java b/src/java/voldemort/client/protocol/admin/AdminClient.java index 94203296a5..21c0116904 100644 --- a/src/java/voldemort/client/protocol/admin/AdminClient.java +++ b/src/java/voldemort/client/protocol/admin/AdminClient.java @@ -26,6 +26,7 @@ import java.nio.channels.Channels; import java.nio.channels.FileChannel; import java.nio.channels.ReadableByteChannel; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.Iterator; @@ -2448,4 +2449,42 @@ public void nativeBackup(int nodeId, int asyncId = response.getRequestId(); waitForCompletion(nodeId, asyncId, timeOut, TimeUnit.MINUTES); } + + /** + * Reserve memory for the stores + * + * @param nodeId The node id to reserve, -1 for entire cluster + * @param stores list of stores for which to reserve + * @param sizeInMB size of reservation + */ + public void reserveMemory(int nodeId, List stores, long sizeInMB) { + + List reserveNodes = new ArrayList(); + if(nodeId == -1) { + // if no node is specified send it to the entire cluster + for(Node node: currentCluster.getNodes()) + reserveNodes.add(node.getId()); + } else { + reserveNodes.add(nodeId); + } + for(String storeName: stores) { + for(Integer reserveNodeId: reserveNodes) { + + VAdminProto.ReserveMemoryRequest reserveRequest = VAdminProto.ReserveMemoryRequest.newBuilder() + .setStoreName(storeName) + .setSizeInMb(sizeInMB) + .build(); + VAdminProto.VoldemortAdminRequest adminRequest = VAdminProto.VoldemortAdminRequest.newBuilder() + .setReserveMemory(reserveRequest) + .setType(VAdminProto.AdminRequestType.RESERVE_MEMORY) + .build(); + VAdminProto.ReserveMemoryResponse.Builder response = sendAndReceive(reserveNodeId, + adminRequest, + VAdminProto.ReserveMemoryResponse.newBuilder()); + if(response.hasError()) + throwException(response.getError()); + } + logger.info("Finished reserving memory for store : " + storeName); + } + } } diff --git a/src/java/voldemort/client/protocol/pb/VAdminProto.java b/src/java/voldemort/client/protocol/pb/VAdminProto.java index 419cbe9e9a..4fed345bc4 100644 --- a/src/java/voldemort/client/protocol/pb/VAdminProto.java +++ b/src/java/voldemort/client/protocol/pb/VAdminProto.java @@ -37,6 +37,7 @@ public enum AdminRequestType INITIATE_REBALANCE_NODE_ON_DONOR(24, 26), DELETE_STORE_REBALANCE_STATE(25, 27), NATIVE_BACKUP(26, 28), + RESERVE_MEMORY(27, 29), ; @@ -71,6 +72,7 @@ public static AdminRequestType valueOf(int value) { case 26: return INITIATE_REBALANCE_NODE_ON_DONOR; case 27: return DELETE_STORE_REBALANCE_STATE; case 28: return NATIVE_BACKUP; + case 29: return RESERVE_MEMORY; default: return null; } } @@ -101,7 +103,7 @@ public AdminRequestType findValueByNumber(int number) { } private static final AdminRequestType[] VALUES = { - GET_METADATA, UPDATE_METADATA, UPDATE_PARTITION_ENTRIES, FETCH_PARTITION_ENTRIES, DELETE_PARTITION_ENTRIES, INITIATE_FETCH_AND_UPDATE, ASYNC_OPERATION_STATUS, INITIATE_REBALANCE_NODE, ASYNC_OPERATION_STOP, ASYNC_OPERATION_LIST, TRUNCATE_ENTRIES, ADD_STORE, DELETE_STORE, FETCH_STORE, SWAP_STORE, ROLLBACK_STORE, GET_RO_MAX_VERSION_DIR, GET_RO_CURRENT_VERSION_DIR, FETCH_PARTITION_FILES, UPDATE_SLOP_ENTRIES, FAILED_FETCH_STORE, GET_RO_STORAGE_FORMAT, REBALANCE_STATE_CHANGE, REPAIR_JOB, INITIATE_REBALANCE_NODE_ON_DONOR, DELETE_STORE_REBALANCE_STATE, NATIVE_BACKUP, + GET_METADATA, UPDATE_METADATA, UPDATE_PARTITION_ENTRIES, FETCH_PARTITION_ENTRIES, DELETE_PARTITION_ENTRIES, INITIATE_FETCH_AND_UPDATE, ASYNC_OPERATION_STATUS, INITIATE_REBALANCE_NODE, ASYNC_OPERATION_STOP, ASYNC_OPERATION_LIST, TRUNCATE_ENTRIES, ADD_STORE, DELETE_STORE, FETCH_STORE, SWAP_STORE, ROLLBACK_STORE, GET_RO_MAX_VERSION_DIR, GET_RO_CURRENT_VERSION_DIR, FETCH_PARTITION_FILES, UPDATE_SLOP_ENTRIES, FAILED_FETCH_STORE, GET_RO_STORAGE_FORMAT, REBALANCE_STATE_CHANGE, REPAIR_JOB, INITIATE_REBALANCE_NODE_ON_DONOR, DELETE_STORE_REBALANCE_STATE, NATIVE_BACKUP, RESERVE_MEMORY, }; public static AdminRequestType valueOf( com.google.protobuf.Descriptors.EnumValueDescriptor desc) { @@ -19959,6 +19961,646 @@ public Builder clearIncremental() { // @@protoc_insertion_point(class_scope:voldemort.NativeBackupRequest) } + public static final class ReserveMemoryRequest extends + com.google.protobuf.GeneratedMessage { + // Use ReserveMemoryRequest.newBuilder() to construct. + private ReserveMemoryRequest() { + initFields(); + } + private ReserveMemoryRequest(boolean noInit) {} + + private static final ReserveMemoryRequest defaultInstance; + public static ReserveMemoryRequest getDefaultInstance() { + return defaultInstance; + } + + public ReserveMemoryRequest getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return voldemort.client.protocol.pb.VAdminProto.internal_static_voldemort_ReserveMemoryRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return voldemort.client.protocol.pb.VAdminProto.internal_static_voldemort_ReserveMemoryRequest_fieldAccessorTable; + } + + // required string store_name = 1; + public static final int STORE_NAME_FIELD_NUMBER = 1; + private boolean hasStoreName; + private java.lang.String storeName_ = ""; + public boolean hasStoreName() { return hasStoreName; } + public java.lang.String getStoreName() { return storeName_; } + + // required int64 size_in_mb = 2; + public static final int SIZE_IN_MB_FIELD_NUMBER = 2; + private boolean hasSizeInMb; + private long sizeInMb_ = 0L; + public boolean hasSizeInMb() { return hasSizeInMb; } + public long getSizeInMb() { return sizeInMb_; } + + private void initFields() { + } + public final boolean isInitialized() { + if (!hasStoreName) return false; + if (!hasSizeInMb) return false; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (hasStoreName()) { + output.writeString(1, getStoreName()); + } + if (hasSizeInMb()) { + output.writeInt64(2, getSizeInMb()); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (hasStoreName()) { + size += com.google.protobuf.CodedOutputStream + .computeStringSize(1, getStoreName()); + } + if (hasSizeInMb()) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(2, getSizeInMb()); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder { + private voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest result; + + // Construct using voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest.newBuilder() + private Builder() {} + + private static Builder create() { + Builder builder = new Builder(); + builder.result = new voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest(); + return builder; + } + + protected voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest internalGetResult() { + return result; + } + + public Builder clear() { + if (result == null) { + throw new IllegalStateException( + "Cannot call clear() after build()."); + } + result = new voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest(); + return this; + } + + public Builder clone() { + return create().mergeFrom(result); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest.getDescriptor(); + } + + public voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest getDefaultInstanceForType() { + return voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest.getDefaultInstance(); + } + + public boolean isInitialized() { + return result.isInitialized(); + } + public voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest build() { + if (result != null && !isInitialized()) { + throw newUninitializedMessageException(result); + } + return buildPartial(); + } + + private voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + if (!isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return buildPartial(); + } + + public voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest buildPartial() { + if (result == null) { + throw new IllegalStateException( + "build() has already been called on this Builder."); + } + voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest returnMe = result; + result = null; + return returnMe; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest) { + return mergeFrom((voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest other) { + if (other == voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest.getDefaultInstance()) return this; + if (other.hasStoreName()) { + setStoreName(other.getStoreName()); + } + if (other.hasSizeInMb()) { + setSizeInMb(other.getSizeInMb()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + return this; + } + break; + } + case 10: { + setStoreName(input.readString()); + break; + } + case 16: { + setSizeInMb(input.readInt64()); + break; + } + } + } + } + + + // required string store_name = 1; + public boolean hasStoreName() { + return result.hasStoreName(); + } + public java.lang.String getStoreName() { + return result.getStoreName(); + } + public Builder setStoreName(java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + result.hasStoreName = true; + result.storeName_ = value; + return this; + } + public Builder clearStoreName() { + result.hasStoreName = false; + result.storeName_ = getDefaultInstance().getStoreName(); + return this; + } + + // required int64 size_in_mb = 2; + public boolean hasSizeInMb() { + return result.hasSizeInMb(); + } + public long getSizeInMb() { + return result.getSizeInMb(); + } + public Builder setSizeInMb(long value) { + result.hasSizeInMb = true; + result.sizeInMb_ = value; + return this; + } + public Builder clearSizeInMb() { + result.hasSizeInMb = false; + result.sizeInMb_ = 0L; + return this; + } + + // @@protoc_insertion_point(builder_scope:voldemort.ReserveMemoryRequest) + } + + static { + defaultInstance = new ReserveMemoryRequest(true); + voldemort.client.protocol.pb.VAdminProto.internalForceInit(); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:voldemort.ReserveMemoryRequest) + } + + public static final class ReserveMemoryResponse extends + com.google.protobuf.GeneratedMessage { + // Use ReserveMemoryResponse.newBuilder() to construct. + private ReserveMemoryResponse() { + initFields(); + } + private ReserveMemoryResponse(boolean noInit) {} + + private static final ReserveMemoryResponse defaultInstance; + public static ReserveMemoryResponse getDefaultInstance() { + return defaultInstance; + } + + public ReserveMemoryResponse getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return voldemort.client.protocol.pb.VAdminProto.internal_static_voldemort_ReserveMemoryResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return voldemort.client.protocol.pb.VAdminProto.internal_static_voldemort_ReserveMemoryResponse_fieldAccessorTable; + } + + // optional .voldemort.Error error = 1; + public static final int ERROR_FIELD_NUMBER = 1; + private boolean hasError; + private voldemort.client.protocol.pb.VProto.Error error_; + public boolean hasError() { return hasError; } + public voldemort.client.protocol.pb.VProto.Error getError() { return error_; } + + private void initFields() { + error_ = voldemort.client.protocol.pb.VProto.Error.getDefaultInstance(); + } + public final boolean isInitialized() { + if (hasError()) { + if (!getError().isInitialized()) return false; + } + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (hasError()) { + output.writeMessage(1, getError()); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (hasError()) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, getError()); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder { + private voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse result; + + // Construct using voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse.newBuilder() + private Builder() {} + + private static Builder create() { + Builder builder = new Builder(); + builder.result = new voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse(); + return builder; + } + + protected voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse internalGetResult() { + return result; + } + + public Builder clear() { + if (result == null) { + throw new IllegalStateException( + "Cannot call clear() after build()."); + } + result = new voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse(); + return this; + } + + public Builder clone() { + return create().mergeFrom(result); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse.getDescriptor(); + } + + public voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse getDefaultInstanceForType() { + return voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse.getDefaultInstance(); + } + + public boolean isInitialized() { + return result.isInitialized(); + } + public voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse build() { + if (result != null && !isInitialized()) { + throw newUninitializedMessageException(result); + } + return buildPartial(); + } + + private voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + if (!isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return buildPartial(); + } + + public voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse buildPartial() { + if (result == null) { + throw new IllegalStateException( + "build() has already been called on this Builder."); + } + voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse returnMe = result; + result = null; + return returnMe; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse) { + return mergeFrom((voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse other) { + if (other == voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse.getDefaultInstance()) return this; + if (other.hasError()) { + mergeError(other.getError()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + return this; + } + break; + } + case 10: { + voldemort.client.protocol.pb.VProto.Error.Builder subBuilder = voldemort.client.protocol.pb.VProto.Error.newBuilder(); + if (hasError()) { + subBuilder.mergeFrom(getError()); + } + input.readMessage(subBuilder, extensionRegistry); + setError(subBuilder.buildPartial()); + break; + } + } + } + } + + + // optional .voldemort.Error error = 1; + public boolean hasError() { + return result.hasError(); + } + public voldemort.client.protocol.pb.VProto.Error getError() { + return result.getError(); + } + public Builder setError(voldemort.client.protocol.pb.VProto.Error value) { + if (value == null) { + throw new NullPointerException(); + } + result.hasError = true; + result.error_ = value; + return this; + } + public Builder setError(voldemort.client.protocol.pb.VProto.Error.Builder builderForValue) { + result.hasError = true; + result.error_ = builderForValue.build(); + return this; + } + public Builder mergeError(voldemort.client.protocol.pb.VProto.Error value) { + if (result.hasError() && + result.error_ != voldemort.client.protocol.pb.VProto.Error.getDefaultInstance()) { + result.error_ = + voldemort.client.protocol.pb.VProto.Error.newBuilder(result.error_).mergeFrom(value).buildPartial(); + } else { + result.error_ = value; + } + result.hasError = true; + return this; + } + public Builder clearError() { + result.hasError = false; + result.error_ = voldemort.client.protocol.pb.VProto.Error.getDefaultInstance(); + return this; + } + + // @@protoc_insertion_point(builder_scope:voldemort.ReserveMemoryResponse) + } + + static { + defaultInstance = new ReserveMemoryResponse(true); + voldemort.client.protocol.pb.VAdminProto.internalForceInit(); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:voldemort.ReserveMemoryResponse) + } + public static final class VoldemortAdminRequest extends com.google.protobuf.GeneratedMessage { // Use VoldemortAdminRequest.newBuilder() to construct. @@ -20182,6 +20824,13 @@ public VoldemortAdminRequest getDefaultInstanceForType() { public boolean hasNativeBackup() { return hasNativeBackup; } public voldemort.client.protocol.pb.VAdminProto.NativeBackupRequest getNativeBackup() { return nativeBackup_; } + // optional .voldemort.ReserveMemoryRequest reserve_memory = 31; + public static final int RESERVE_MEMORY_FIELD_NUMBER = 31; + private boolean hasReserveMemory; + private voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest reserveMemory_; + public boolean hasReserveMemory() { return hasReserveMemory; } + public voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest getReserveMemory() { return reserveMemory_; } + private void initFields() { type_ = voldemort.client.protocol.pb.VAdminProto.AdminRequestType.GET_METADATA; getMetadata_ = voldemort.client.protocol.pb.VAdminProto.GetMetadataRequest.getDefaultInstance(); @@ -20211,6 +20860,7 @@ private void initFields() { initiateRebalanceNodeOnDonor_ = voldemort.client.protocol.pb.VAdminProto.InitiateRebalanceNodeOnDonorRequest.getDefaultInstance(); deleteStoreRebalanceState_ = voldemort.client.protocol.pb.VAdminProto.DeleteStoreRebalanceStateRequest.getDefaultInstance(); nativeBackup_ = voldemort.client.protocol.pb.VAdminProto.NativeBackupRequest.getDefaultInstance(); + reserveMemory_ = voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest.getDefaultInstance(); } public final boolean isInitialized() { if (!hasType) return false; @@ -20283,6 +20933,9 @@ public final boolean isInitialized() { if (hasNativeBackup()) { if (!getNativeBackup().isInitialized()) return false; } + if (hasReserveMemory()) { + if (!getReserveMemory().isInitialized()) return false; + } return true; } @@ -20373,6 +21026,9 @@ public void writeTo(com.google.protobuf.CodedOutputStream output) if (hasNativeBackup()) { output.writeMessage(30, getNativeBackup()); } + if (hasReserveMemory()) { + output.writeMessage(31, getReserveMemory()); + } getUnknownFields().writeTo(output); } @@ -20494,6 +21150,10 @@ public int getSerializedSize() { size += com.google.protobuf.CodedOutputStream .computeMessageSize(30, getNativeBackup()); } + if (hasReserveMemory()) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(31, getReserveMemory()); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -20736,6 +21396,9 @@ public Builder mergeFrom(voldemort.client.protocol.pb.VAdminProto.VoldemortAdmin if (other.hasNativeBackup()) { mergeNativeBackup(other.getNativeBackup()); } + if (other.hasReserveMemory()) { + mergeReserveMemory(other.getReserveMemory()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -21014,6 +21677,15 @@ public Builder mergeFrom( setNativeBackup(subBuilder.buildPartial()); break; } + case 250: { + voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest.Builder subBuilder = voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest.newBuilder(); + if (hasReserveMemory()) { + subBuilder.mergeFrom(getReserveMemory()); + } + input.readMessage(subBuilder, extensionRegistry); + setReserveMemory(subBuilder.buildPartial()); + break; + } } } } @@ -22039,6 +22711,43 @@ public Builder clearNativeBackup() { return this; } + // optional .voldemort.ReserveMemoryRequest reserve_memory = 31; + public boolean hasReserveMemory() { + return result.hasReserveMemory(); + } + public voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest getReserveMemory() { + return result.getReserveMemory(); + } + public Builder setReserveMemory(voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest value) { + if (value == null) { + throw new NullPointerException(); + } + result.hasReserveMemory = true; + result.reserveMemory_ = value; + return this; + } + public Builder setReserveMemory(voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest.Builder builderForValue) { + result.hasReserveMemory = true; + result.reserveMemory_ = builderForValue.build(); + return this; + } + public Builder mergeReserveMemory(voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest value) { + if (result.hasReserveMemory() && + result.reserveMemory_ != voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest.getDefaultInstance()) { + result.reserveMemory_ = + voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest.newBuilder(result.reserveMemory_).mergeFrom(value).buildPartial(); + } else { + result.reserveMemory_ = value; + } + result.hasReserveMemory = true; + return this; + } + public Builder clearReserveMemory() { + result.hasReserveMemory = false; + result.reserveMemory_ = voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest.getDefaultInstance(); + return this; + } + // @@protoc_insertion_point(builder_scope:voldemort.VoldemortAdminRequest) } @@ -22326,6 +23035,16 @@ public Builder clearNativeBackup() { private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_voldemort_NativeBackupRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_voldemort_ReserveMemoryRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_voldemort_ReserveMemoryRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_voldemort_ReserveMemoryResponse_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_voldemort_ReserveMemoryResponse_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor internal_static_voldemort_VoldemortAdminRequest_descriptor; private static @@ -22463,72 +23182,77 @@ public Builder clearNativeBackup() { "\022\037\n\005error\030\001 \001(\0132\020.voldemort.Error\"h\n\023Nat" + "iveBackupRequest\022\022\n\nstore_name\030\001 \002(\t\022\022\n\n" + "backup_dir\030\002 \002(\t\022\024\n\014verify_files\030\003 \002(\010\022\023" + - "\n\013incremental\030\004 \002(\010\"\267\016\n\025VoldemortAdminRe" + - "quest\022)\n\004type\030\001 \002(\0162\033.voldemort.AdminReq" + - "uestType\0223\n\014get_metadata\030\002 \001(\0132\035.voldemo" + - "rt.GetMetadataRequest\0229\n\017update_metadata" + - "\030\003 \001(\0132 .voldemort.UpdateMetadataRequest" + - "\022J\n\030update_partition_entries\030\004 \001(\0132(.vol" + - "demort.UpdatePartitionEntriesRequest\022H\n\027", - "fetch_partition_entries\030\005 \001(\0132\'.voldemor" + - "t.FetchPartitionEntriesRequest\022J\n\030delete" + - "_partition_entries\030\006 \001(\0132(.voldemort.Del" + - "etePartitionEntriesRequest\022K\n\031initiate_f" + - "etch_and_update\030\007 \001(\0132(.voldemort.Initia" + - "teFetchAndUpdateRequest\022F\n\026async_operati" + - "on_status\030\010 \001(\0132&.voldemort.AsyncOperati" + - "onStatusRequest\022H\n\027initiate_rebalance_no" + - "de\030\t \001(\0132\'.voldemort.InitiateRebalanceNo" + - "deRequest\022B\n\024async_operation_stop\030\n \001(\0132", - "$.voldemort.AsyncOperationStopRequest\022B\n" + - "\024async_operation_list\030\013 \001(\0132$.voldemort." + - "AsyncOperationListRequest\022;\n\020truncate_en" + - "tries\030\014 \001(\0132!.voldemort.TruncateEntriesR" + - "equest\022-\n\tadd_store\030\r \001(\0132\032.voldemort.Ad" + - "dStoreRequest\0223\n\014delete_store\030\016 \001(\0132\035.vo" + - "ldemort.DeleteStoreRequest\0221\n\013fetch_stor" + - "e\030\017 \001(\0132\034.voldemort.FetchStoreRequest\022/\n" + - "\nswap_store\030\020 \001(\0132\033.voldemort.SwapStoreR" + - "equest\0227\n\016rollback_store\030\021 \001(\0132\037.voldemo", - "rt.RollbackStoreRequest\022D\n\026get_ro_max_ve" + - "rsion_dir\030\022 \001(\0132$.voldemort.GetROMaxVers" + - "ionDirRequest\022L\n\032get_ro_current_version_" + - "dir\030\023 \001(\0132(.voldemort.GetROCurrentVersio" + - "nDirRequest\022D\n\025fetch_partition_files\030\024 \001" + - "(\0132%.voldemort.FetchPartitionFilesReques" + - "t\022@\n\023update_slop_entries\030\026 \001(\0132#.voldemo" + - "rt.UpdateSlopEntriesRequest\022>\n\022failed_fe" + - "tch_store\030\030 \001(\0132\".voldemort.FailedFetchS" + - "toreRequest\022C\n\025get_ro_storage_format\030\031 \001", - "(\0132$.voldemort.GetROStorageFormatRequest" + - "\022F\n\026rebalance_state_change\030\032 \001(\0132&.volde" + - "mort.RebalanceStateChangeRequest\022/\n\nrepa" + - "ir_job\030\033 \001(\0132\033.voldemort.RepairJobReques" + - "t\022X\n initiate_rebalance_node_on_donor\030\034 " + - "\001(\0132..voldemort.InitiateRebalanceNodeOnD" + - "onorRequest\022Q\n\034delete_store_rebalance_st" + - "ate\030\035 \001(\0132+.voldemort.DeleteStoreRebalan" + - "ceStateRequest\0225\n\rnative_backup\030\036 \001(\0132\036." + - "voldemort.NativeBackupRequest*\264\005\n\020AdminR", - "equestType\022\020\n\014GET_METADATA\020\000\022\023\n\017UPDATE_M" + - "ETADATA\020\001\022\034\n\030UPDATE_PARTITION_ENTRIES\020\002\022" + - "\033\n\027FETCH_PARTITION_ENTRIES\020\003\022\034\n\030DELETE_P" + - "ARTITION_ENTRIES\020\004\022\035\n\031INITIATE_FETCH_AND" + - "_UPDATE\020\005\022\032\n\026ASYNC_OPERATION_STATUS\020\006\022\033\n" + - "\027INITIATE_REBALANCE_NODE\020\007\022\030\n\024ASYNC_OPER" + - "ATION_STOP\020\010\022\030\n\024ASYNC_OPERATION_LIST\020\t\022\024" + - "\n\020TRUNCATE_ENTRIES\020\n\022\r\n\tADD_STORE\020\013\022\020\n\014D" + - "ELETE_STORE\020\014\022\017\n\013FETCH_STORE\020\r\022\016\n\nSWAP_S" + - "TORE\020\016\022\022\n\016ROLLBACK_STORE\020\017\022\032\n\026GET_RO_MAX", - "_VERSION_DIR\020\020\022\036\n\032GET_RO_CURRENT_VERSION" + - "_DIR\020\021\022\031\n\025FETCH_PARTITION_FILES\020\022\022\027\n\023UPD" + - "ATE_SLOP_ENTRIES\020\024\022\026\n\022FAILED_FETCH_STORE" + - "\020\026\022\031\n\025GET_RO_STORAGE_FORMAT\020\027\022\032\n\026REBALAN" + - "CE_STATE_CHANGE\020\030\022\016\n\nREPAIR_JOB\020\031\022$\n INI" + - "TIATE_REBALANCE_NODE_ON_DONOR\020\032\022 \n\034DELET" + - "E_STORE_REBALANCE_STATE\020\033\022\021\n\rNATIVE_BACK" + - "UP\020\034B-\n\034voldemort.client.protocol.pbB\013VA" + - "dminProtoH\001" + "\n\013incremental\030\004 \002(\010\">\n\024ReserveMemoryRequ" + + "est\022\022\n\nstore_name\030\001 \002(\t\022\022\n\nsize_in_mb\030\002 " + + "\002(\003\"8\n\025ReserveMemoryResponse\022\037\n\005error\030\001 " + + "\001(\0132\020.voldemort.Error\"\360\016\n\025VoldemortAdmin" + + "Request\022)\n\004type\030\001 \002(\0162\033.voldemort.AdminR" + + "equestType\0223\n\014get_metadata\030\002 \001(\0132\035.volde" + + "mort.GetMetadataRequest\0229\n\017update_metada", + "ta\030\003 \001(\0132 .voldemort.UpdateMetadataReque" + + "st\022J\n\030update_partition_entries\030\004 \001(\0132(.v" + + "oldemort.UpdatePartitionEntriesRequest\022H" + + "\n\027fetch_partition_entries\030\005 \001(\0132\'.voldem" + + "ort.FetchPartitionEntriesRequest\022J\n\030dele" + + "te_partition_entries\030\006 \001(\0132(.voldemort.D" + + "eletePartitionEntriesRequest\022K\n\031initiate" + + "_fetch_and_update\030\007 \001(\0132(.voldemort.Init" + + "iateFetchAndUpdateRequest\022F\n\026async_opera" + + "tion_status\030\010 \001(\0132&.voldemort.AsyncOpera", + "tionStatusRequest\022H\n\027initiate_rebalance_" + + "node\030\t \001(\0132\'.voldemort.InitiateRebalance" + + "NodeRequest\022B\n\024async_operation_stop\030\n \001(" + + "\0132$.voldemort.AsyncOperationStopRequest\022" + + "B\n\024async_operation_list\030\013 \001(\0132$.voldemor" + + "t.AsyncOperationListRequest\022;\n\020truncate_" + + "entries\030\014 \001(\0132!.voldemort.TruncateEntrie" + + "sRequest\022-\n\tadd_store\030\r \001(\0132\032.voldemort." + + "AddStoreRequest\0223\n\014delete_store\030\016 \001(\0132\035." + + "voldemort.DeleteStoreRequest\0221\n\013fetch_st", + "ore\030\017 \001(\0132\034.voldemort.FetchStoreRequest\022" + + "/\n\nswap_store\030\020 \001(\0132\033.voldemort.SwapStor" + + "eRequest\0227\n\016rollback_store\030\021 \001(\0132\037.volde" + + "mort.RollbackStoreRequest\022D\n\026get_ro_max_" + + "version_dir\030\022 \001(\0132$.voldemort.GetROMaxVe" + + "rsionDirRequest\022L\n\032get_ro_current_versio" + + "n_dir\030\023 \001(\0132(.voldemort.GetROCurrentVers" + + "ionDirRequest\022D\n\025fetch_partition_files\030\024" + + " \001(\0132%.voldemort.FetchPartitionFilesRequ" + + "est\022@\n\023update_slop_entries\030\026 \001(\0132#.volde", + "mort.UpdateSlopEntriesRequest\022>\n\022failed_" + + "fetch_store\030\030 \001(\0132\".voldemort.FailedFetc" + + "hStoreRequest\022C\n\025get_ro_storage_format\030\031" + + " \001(\0132$.voldemort.GetROStorageFormatReque" + + "st\022F\n\026rebalance_state_change\030\032 \001(\0132&.vol" + + "demort.RebalanceStateChangeRequest\022/\n\nre" + + "pair_job\030\033 \001(\0132\033.voldemort.RepairJobRequ" + + "est\022X\n initiate_rebalance_node_on_donor\030" + + "\034 \001(\0132..voldemort.InitiateRebalanceNodeO" + + "nDonorRequest\022Q\n\034delete_store_rebalance_", + "state\030\035 \001(\0132+.voldemort.DeleteStoreRebal" + + "anceStateRequest\0225\n\rnative_backup\030\036 \001(\0132" + + "\036.voldemort.NativeBackupRequest\0227\n\016reser" + + "ve_memory\030\037 \001(\0132\037.voldemort.ReserveMemor" + + "yRequest*\310\005\n\020AdminRequestType\022\020\n\014GET_MET" + + "ADATA\020\000\022\023\n\017UPDATE_METADATA\020\001\022\034\n\030UPDATE_P" + + "ARTITION_ENTRIES\020\002\022\033\n\027FETCH_PARTITION_EN" + + "TRIES\020\003\022\034\n\030DELETE_PARTITION_ENTRIES\020\004\022\035\n" + + "\031INITIATE_FETCH_AND_UPDATE\020\005\022\032\n\026ASYNC_OP" + + "ERATION_STATUS\020\006\022\033\n\027INITIATE_REBALANCE_N", + "ODE\020\007\022\030\n\024ASYNC_OPERATION_STOP\020\010\022\030\n\024ASYNC" + + "_OPERATION_LIST\020\t\022\024\n\020TRUNCATE_ENTRIES\020\n\022" + + "\r\n\tADD_STORE\020\013\022\020\n\014DELETE_STORE\020\014\022\017\n\013FETC" + + "H_STORE\020\r\022\016\n\nSWAP_STORE\020\016\022\022\n\016ROLLBACK_ST" + + "ORE\020\017\022\032\n\026GET_RO_MAX_VERSION_DIR\020\020\022\036\n\032GET" + + "_RO_CURRENT_VERSION_DIR\020\021\022\031\n\025FETCH_PARTI" + + "TION_FILES\020\022\022\027\n\023UPDATE_SLOP_ENTRIES\020\024\022\026\n" + + "\022FAILED_FETCH_STORE\020\026\022\031\n\025GET_RO_STORAGE_" + + "FORMAT\020\027\022\032\n\026REBALANCE_STATE_CHANGE\020\030\022\016\n\n" + + "REPAIR_JOB\020\031\022$\n INITIATE_REBALANCE_NODE_", + "ON_DONOR\020\032\022 \n\034DELETE_STORE_REBALANCE_STA" + + "TE\020\033\022\021\n\rNATIVE_BACKUP\020\034\022\022\n\016RESERVE_MEMOR" + + "Y\020\035B-\n\034voldemort.client.protocol.pbB\013VAd" + + "minProtoH\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -22975,12 +23699,28 @@ public com.google.protobuf.ExtensionRegistry assignDescriptors( new java.lang.String[] { "StoreName", "BackupDir", "VerifyFiles", "Incremental", }, voldemort.client.protocol.pb.VAdminProto.NativeBackupRequest.class, voldemort.client.protocol.pb.VAdminProto.NativeBackupRequest.Builder.class); - internal_static_voldemort_VoldemortAdminRequest_descriptor = + internal_static_voldemort_ReserveMemoryRequest_descriptor = getDescriptor().getMessageTypes().get(55); + internal_static_voldemort_ReserveMemoryRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_voldemort_ReserveMemoryRequest_descriptor, + new java.lang.String[] { "StoreName", "SizeInMb", }, + voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest.class, + voldemort.client.protocol.pb.VAdminProto.ReserveMemoryRequest.Builder.class); + internal_static_voldemort_ReserveMemoryResponse_descriptor = + getDescriptor().getMessageTypes().get(56); + internal_static_voldemort_ReserveMemoryResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_voldemort_ReserveMemoryResponse_descriptor, + new java.lang.String[] { "Error", }, + voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse.class, + voldemort.client.protocol.pb.VAdminProto.ReserveMemoryResponse.Builder.class); + internal_static_voldemort_VoldemortAdminRequest_descriptor = + getDescriptor().getMessageTypes().get(57); internal_static_voldemort_VoldemortAdminRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_voldemort_VoldemortAdminRequest_descriptor, - new java.lang.String[] { "Type", "GetMetadata", "UpdateMetadata", "UpdatePartitionEntries", "FetchPartitionEntries", "DeletePartitionEntries", "InitiateFetchAndUpdate", "AsyncOperationStatus", "InitiateRebalanceNode", "AsyncOperationStop", "AsyncOperationList", "TruncateEntries", "AddStore", "DeleteStore", "FetchStore", "SwapStore", "RollbackStore", "GetRoMaxVersionDir", "GetRoCurrentVersionDir", "FetchPartitionFiles", "UpdateSlopEntries", "FailedFetchStore", "GetRoStorageFormat", "RebalanceStateChange", "RepairJob", "InitiateRebalanceNodeOnDonor", "DeleteStoreRebalanceState", "NativeBackup", }, + new java.lang.String[] { "Type", "GetMetadata", "UpdateMetadata", "UpdatePartitionEntries", "FetchPartitionEntries", "DeletePartitionEntries", "InitiateFetchAndUpdate", "AsyncOperationStatus", "InitiateRebalanceNode", "AsyncOperationStop", "AsyncOperationList", "TruncateEntries", "AddStore", "DeleteStore", "FetchStore", "SwapStore", "RollbackStore", "GetRoMaxVersionDir", "GetRoCurrentVersionDir", "FetchPartitionFiles", "UpdateSlopEntries", "FailedFetchStore", "GetRoStorageFormat", "RebalanceStateChange", "RepairJob", "InitiateRebalanceNodeOnDonor", "DeleteStoreRebalanceState", "NativeBackup", "ReserveMemory", }, voldemort.client.protocol.pb.VAdminProto.VoldemortAdminRequest.class, voldemort.client.protocol.pb.VAdminProto.VoldemortAdminRequest.Builder.class); return null; diff --git a/src/java/voldemort/server/jmx/JmxService.java b/src/java/voldemort/server/jmx/JmxService.java index 9056027af5..278fac79d0 100644 --- a/src/java/voldemort/server/jmx/JmxService.java +++ b/src/java/voldemort/server/jmx/JmxService.java @@ -36,8 +36,6 @@ import voldemort.server.VoldemortServer; import voldemort.server.VoldemortService; import voldemort.store.Store; -import voldemort.store.bdb.BdbStorageEngine; -import voldemort.store.bdb.stats.BdbEnvironmentStats; import voldemort.utils.ByteArray; import voldemort.utils.JmxUtils; @@ -88,13 +86,6 @@ protected void startInner() { registerBean(store, JmxUtils.createObjectName(JmxUtils.getPackageName(store.getClass()), store.getName())); - if(store instanceof BdbStorageEngine) { - // Temp hack for now - BdbStorageEngine bdbStore = (BdbStorageEngine) store; - registerBean(bdbStore.getBdbEnvironmentStats(), - JmxUtils.createObjectName(JmxUtils.getPackageName(BdbEnvironmentStats.class), - store.getName())); - } } } diff --git a/src/java/voldemort/server/protocol/admin/AdminServiceRequestHandler.java b/src/java/voldemort/server/protocol/admin/AdminServiceRequestHandler.java index 76263f9fb8..28f10c074c 100644 --- a/src/java/voldemort/server/protocol/admin/AdminServiceRequestHandler.java +++ b/src/java/voldemort/server/protocol/admin/AdminServiceRequestHandler.java @@ -50,6 +50,7 @@ import voldemort.store.ErrorCodeMapper; import voldemort.store.StorageEngine; import voldemort.store.StoreDefinition; +import voldemort.store.StoreDefinitionBuilder; import voldemort.store.StoreOperationFailureException; import voldemort.store.backup.NativeBackupable; import voldemort.store.metadata.MetadataStore; @@ -259,6 +260,10 @@ public StreamRequestHandler handleRequest(final DataInputStream inputStream, case NATIVE_BACKUP: ProtoUtils.writeMessage(outputStream, handleNativeBackup(request.getNativeBackup())); break; + case RESERVE_MEMORY: + ProtoUtils.writeMessage(outputStream, + handleReserveMemory(request.getReserveMemory())); + break; default: throw new VoldemortException("Unkown operation " + request.getType()); } @@ -299,7 +304,8 @@ private VAdminProto.DeleteStoreRebalanceStateResponse handleDeleteStoreRebalance } catch(VoldemortException e) { response.setError(ProtoUtils.encodeError(errorCodeMapper, e)); logger.error("handleDeleteStoreRebalanceState failed for request(" - + request.toString() + ")", e); + + request.toString() + ")", + e); } } return response.build(); @@ -1458,4 +1464,73 @@ public void stop() { return response.build(); } + + public VAdminProto.ReserveMemoryResponse handleReserveMemory(VAdminProto.ReserveMemoryRequest request) { + VAdminProto.ReserveMemoryResponse.Builder response = VAdminProto.ReserveMemoryResponse.newBuilder(); + + try { + String storeName = request.getStoreName(); + long reserveMB = request.getSizeInMb(); + + synchronized(lock) { + if(storeRepository.hasLocalStore(storeName)) { + + logger.info("Setting memory foot print of store '" + storeName + "' to " + + reserveMB + " MB"); + + // update store's metadata (this also has the effect of + // updating the stores.xml file) + List storeDefList = metadataStore.getStoreDefList(); + + for(int i = 0; i < storeDefList.size(); i++) { + StoreDefinition storeDef = storeDefList.get(i); + if(!storeDef.isView() && storeDef.getName().equals(storeName)) { + StoreDefinition newStoreDef = new StoreDefinitionBuilder().setName(storeDef.getName()) + .setType(storeDef.getType()) + .setDescription(storeDef.getDescription()) + .setOwners(storeDef.getOwners()) + .setKeySerializer(storeDef.getKeySerializer()) + .setValueSerializer(storeDef.getValueSerializer()) + .setRoutingPolicy(storeDef.getRoutingPolicy()) + .setRoutingStrategyType(storeDef.getRoutingStrategyType()) + .setReplicationFactor(storeDef.getReplicationFactor()) + .setPreferredReads(storeDef.getPreferredReads()) + .setRequiredReads(storeDef.getRequiredReads()) + .setPreferredWrites(storeDef.getPreferredWrites()) + .setRequiredWrites(storeDef.getRequiredWrites()) + .setRetentionPeriodDays(storeDef.getRetentionDays()) + .setRetentionScanThrottleRate(storeDef.getRetentionScanThrottleRate()) + .setZoneReplicationFactor(storeDef.getZoneReplicationFactor()) + .setZoneCountReads(storeDef.getZoneCountReads()) + .setZoneCountWrites(storeDef.getZoneCountWrites()) + .setHintedHandoffStrategy(storeDef.getHintedHandoffStrategyType()) + .setHintPrefListSize(storeDef.getHintPrefListSize()) + .setMemoryFootprintMB(reserveMB) + .build(); + + storeDefList.set(i, newStoreDef); + storageService.updateStore(newStoreDef); + } + } + + // save the changes + try { + metadataStore.put(MetadataStore.STORES_KEY, storeDefList); + } catch(Exception e) { + throw new VoldemortException(e); + } + + } else { + logger.error("Failure to reserve memory. Store '" + storeName + + "' does not exist"); + throw new StoreOperationFailureException(String.format("Store '%s' does not exist on this server", + storeName)); + } + } + } catch(VoldemortException e) { + response.setError(ProtoUtils.encodeError(errorCodeMapper, e)); + logger.error("handleReserveMemory failed for request(" + request.toString() + ")", e); + } + return response.build(); + } } diff --git a/src/java/voldemort/server/storage/StorageService.java b/src/java/voldemort/server/storage/StorageService.java index f941160d68..1ef2c46442 100644 --- a/src/java/voldemort/server/storage/StorageService.java +++ b/src/java/voldemort/server/storage/StorageService.java @@ -214,7 +214,33 @@ protected void startInner() { + voldemortConfig.getSlopStoreType() + " storage engine has not been enabled."); - SlopStorageEngine slopEngine = new SlopStorageEngine(config.getStore(SlopStorageEngine.SLOP_STORE_NAME), + // make a dummy store definition object + StoreDefinition slopStoreDefinition = new StoreDefinition(SlopStorageEngine.SLOP_STORE_NAME, + null, + null, + null, + null, + null, + null, + null, + 0, + null, + 0, + null, + 0, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + 0); + SlopStorageEngine slopEngine = new SlopStorageEngine(config.getStore(slopStoreDefinition), metadata.getCluster()); registerEngine(slopEngine, false, "slop"); storeRepository.setSlopStore(slopEngine); @@ -282,6 +308,16 @@ protected void startInner() { logger.info("All stores initialized."); } + public void updateStore(StoreDefinition storeDef) { + logger.info("Updating store '" + storeDef.getName() + "' (" + storeDef.getType() + ")."); + StorageConfiguration config = storageConfigs.get(storeDef.getType()); + if(config == null) + throw new ConfigurationException("Attempt to open store " + storeDef.getName() + + " but " + storeDef.getType() + + " storage engine has not been enabled."); + config.update(storeDef); + } + public void openStore(StoreDefinition storeDef) { logger.info("Opening store '" + storeDef.getName() + "' (" + storeDef.getType() + ")."); @@ -299,7 +335,7 @@ public void openStore(StoreDefinition storeDef) { ((ReadOnlyStorageConfiguration) config).setRoutingStrategy(routingStrategy); } - final StorageEngine engine = config.getStore(storeDef.getName()); + final StorageEngine engine = config.getStore(storeDef); // Update the routing strategy + add listener to metadata if(storeDef.getType().compareTo(ReadOnlyStorageConfiguration.TYPE_NAME) == 0) { metadata.addMetadataStoreListener(storeDef.getName(), new MetadataStoreListener() { diff --git a/src/java/voldemort/store/StorageConfiguration.java b/src/java/voldemort/store/StorageConfiguration.java index c795e215b2..a17f583730 100644 --- a/src/java/voldemort/store/StorageConfiguration.java +++ b/src/java/voldemort/store/StorageConfiguration.java @@ -36,16 +36,23 @@ public interface StorageConfiguration { /** * Get an initialized storage implementation * - * @param name The name of the storage + * @param storeDef store definition * @return The storage engine */ - public StorageEngine getStore(String name); + public StorageEngine getStore(StoreDefinition storeDef); /** * Get the type of stores returned by this configuration */ public String getType(); + /** + * Update the storage configuration at runtime + * + * @param storeDef new store definition object + */ + public void update(StoreDefinition storeDef); + /** * Close the storage configuration */ diff --git a/src/java/voldemort/store/StoreDefinition.java b/src/java/voldemort/store/StoreDefinition.java index 0df48fcf84..2c93d64eb4 100644 --- a/src/java/voldemort/store/StoreDefinition.java +++ b/src/java/voldemort/store/StoreDefinition.java @@ -60,6 +60,7 @@ public class StoreDefinition implements Serializable { private final HintedHandoffStrategyType hintedHandoffStrategyType; private final Integer hintPrefListSize; private final List owners; + private final long memoryFootprintMB; public StoreDefinition(String name, String type, @@ -84,21 +85,23 @@ public StoreDefinition(String name, String factory, HintedHandoffStrategyType hintedHandoffStrategyType, Integer hintPrefListSize, - List owners) { + List owners, + long memoryFootprintMB) { this.name = Utils.notNull(name); - this.type = Utils.notNull(type); + this.type = type; this.description = description; this.replicationFactor = replicationFactor; this.preferredReads = preferredReads; this.requiredReads = requiredReads; this.preferredWrites = preferredWrites; this.requiredWrites = requiredWrites; - this.routingPolicy = Utils.notNull(routingPolicy); - this.keySerializer = Utils.notNull(keySerializer); - this.valueSerializer = Utils.notNull(valueSerializer); + this.routingPolicy = routingPolicy; + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; this.transformsSerializer = transformsSerializer; this.retentionPeriodDays = retentionDays; this.retentionScanThrottleRate = retentionThrottleRate; + this.memoryFootprintMB = memoryFootprintMB; this.routingStrategyType = routingStrategyType; this.viewOf = viewOfStore; this.valueTransformation = valTrans; @@ -109,10 +112,16 @@ public StoreDefinition(String name, this.hintedHandoffStrategyType = hintedHandoffStrategyType; this.hintPrefListSize = hintPrefListSize; this.owners = owners; - checkParameterLegality(); } - private void checkParameterLegality() { + protected void checkParameterLegality() { + + // null checks + Utils.notNull(this.type); + Utils.notNull(routingPolicy); + Utils.notNull(keySerializer); + Utils.notNull(valueSerializer); + if(requiredReads < 1) throw new IllegalArgumentException("Cannot have a requiredReads number less than 1."); else if(requiredReads > replicationFactor) @@ -329,6 +338,14 @@ public List getOwners() { return this.owners; } + public long getMemoryFootprintMB() { + return this.memoryFootprintMB; + } + + public boolean hasMemoryFootprint() { + return memoryFootprintMB != 0; + } + @Override public boolean equals(Object o) { if(this == o) diff --git a/src/java/voldemort/store/StoreDefinitionBuilder.java b/src/java/voldemort/store/StoreDefinitionBuilder.java index 89648a48c5..95900b4935 100644 --- a/src/java/voldemort/store/StoreDefinitionBuilder.java +++ b/src/java/voldemort/store/StoreDefinitionBuilder.java @@ -40,6 +40,7 @@ public class StoreDefinitionBuilder { private HintedHandoffStrategyType hintedHandoffStrategy = null; private Integer hintPrefListSize = null; private List owners = null; + private long memoryFootprintMB = 0; public String getName() { return Utils.notNull(name); @@ -273,30 +274,42 @@ public StoreDefinitionBuilder setOwners(List owners) { return this; } + public long getMemoryFootprintMB() { + return memoryFootprintMB; + } + + public StoreDefinitionBuilder setMemoryFootprintMB(long memoryFootprintMB) { + this.memoryFootprintMB = memoryFootprintMB; + return this; + } + public StoreDefinition build() { - return new StoreDefinition(this.getName(), - this.getType(), - this.getDescription(), - this.getKeySerializer(), - this.getValueSerializer(), - this.getTransformsSerializer(), - this.getRoutingPolicy(), - this.getRoutingStrategyType(), - this.getReplicationFactor(), - this.getPreferredReads(), - this.getRequiredReads(), - this.getPreferredWrites(), - this.getRequiredWrites(), - this.getViewOf(), - this.getView(), - this.getZoneReplicationFactor(), - this.getZoneCountReads(), - this.getZoneCountWrites(), - this.getRetentionPeriodDays(), - this.getRetentionScanThrottleRate(), - this.getSerializerFactory(), - this.getHintedHandoffStrategy(), - this.getHintPrefListSize(), - this.getOwners()); + StoreDefinition storeDef = new StoreDefinition(this.getName(), + this.getType(), + this.getDescription(), + this.getKeySerializer(), + this.getValueSerializer(), + this.getTransformsSerializer(), + this.getRoutingPolicy(), + this.getRoutingStrategyType(), + this.getReplicationFactor(), + this.getPreferredReads(), + this.getRequiredReads(), + this.getPreferredWrites(), + this.getRequiredWrites(), + this.getViewOf(), + this.getView(), + this.getZoneReplicationFactor(), + this.getZoneCountReads(), + this.getZoneCountWrites(), + this.getRetentionPeriodDays(), + this.getRetentionScanThrottleRate(), + this.getSerializerFactory(), + this.getHintedHandoffStrategy(), + this.getHintPrefListSize(), + this.getOwners(), + this.getMemoryFootprintMB()); + storeDef.checkParameterLegality(); + return storeDef; } } diff --git a/src/java/voldemort/store/bdb/BdbStorageConfiguration.java b/src/java/voldemort/store/bdb/BdbStorageConfiguration.java index b59d029cfa..1bd6f3caf9 100644 --- a/src/java/voldemort/store/bdb/BdbStorageConfiguration.java +++ b/src/java/voldemort/store/bdb/BdbStorageConfiguration.java @@ -17,7 +17,9 @@ package voldemort.store.bdb; import java.io.File; +import java.util.HashSet; import java.util.Map; +import java.util.Set; import java.util.concurrent.TimeUnit; import org.apache.log4j.Logger; @@ -28,7 +30,9 @@ import voldemort.store.StorageConfiguration; import voldemort.store.StorageEngine; import voldemort.store.StorageInitializationException; +import voldemort.store.StoreDefinition; import voldemort.utils.ByteArray; +import voldemort.utils.JmxUtils; import voldemort.utils.Time; import com.google.common.collect.Maps; @@ -38,7 +42,7 @@ import com.sleepycat.je.Durability; import com.sleepycat.je.Environment; import com.sleepycat.je.EnvironmentConfig; -import com.sleepycat.je.EnvironmentStats; +import com.sleepycat.je.EnvironmentMutableConfig; import com.sleepycat.je.StatsConfig; /** @@ -54,6 +58,7 @@ public class BdbStorageConfiguration implements StorageConfiguration { private static final String SHARED_ENV_KEY = "shared"; private static Logger logger = Logger.getLogger(BdbStorageConfiguration.class); + private static final long BYTES_PER_MB = 1048576; private final Object lock = new Object(); private final Map environments = Maps.newHashMap(); @@ -62,12 +67,15 @@ public class BdbStorageConfiguration implements StorageConfiguration { private final String bdbMasterDir; private final boolean useOneEnvPerStore; private final VoldemortConfig voldemortConfig; + private final long totalCacheSize; + private long reservedCacheSize = 0; + private Set unreservedStores; public BdbStorageConfiguration(VoldemortConfig config) { this.voldemortConfig = config; environmentConfig = new EnvironmentConfig(); environmentConfig.setTransactional(true); - environmentConfig.setCacheSize(config.getBdbCacheSize()); + totalCacheSize = config.getBdbCacheSize(); if(config.isBdbWriteTransactionsEnabled() && config.isBdbFlushTransactionsEnabled()) { environmentConfig.setDurability(Durability.COMMIT_SYNC); } else if(config.isBdbWriteTransactionsEnabled() && !config.isBdbFlushTransactionsEnabled()) { @@ -111,20 +119,24 @@ public BdbStorageConfiguration(VoldemortConfig config) { databaseConfig.setTransactional(true); bdbMasterDir = config.getBdbDataDirectory(); useOneEnvPerStore = config.isBdbOneEnvPerStore(); - if(useOneEnvPerStore) - environmentConfig.setSharedCache(true); + unreservedStores = new HashSet(); } - public StorageEngine getStore(String storeName) { + public StorageEngine getStore(StoreDefinition storeDef) { synchronized(lock) { try { - Environment environment = getEnvironment(storeName); + String storeName = storeDef.getName(); + Environment environment = getEnvironment(storeDef); Database db = environment.openDatabase(null, storeName, databaseConfig); BdbRuntimeConfig runtimeConfig = new BdbRuntimeConfig(voldemortConfig); BdbStorageEngine engine = new BdbStorageEngine(storeName, environment, db, runtimeConfig); + if(voldemortConfig.isJmxEnabled()) { + // register the environment stats mbean + JmxUtils.registerMbean(storeName, engine.getBdbEnvironmentStats()); + } return engine; } catch(DatabaseException d) { throw new StorageInitializationException(d); @@ -132,10 +144,30 @@ public StorageEngine getStore(String storeName) { } } - private Environment getEnvironment(String storeName) throws DatabaseException { + /** + * When a reservation is made, we need to shrink the shared cache + * accordingly to guarantee memory foot print of the new store. NOTE: This + * is not an instantaeneous operation. Changes will take effect only when + * traffic is thrown and eviction happens.( Won't happen until Network ports + * are opened anyway which is rightfully done after storage service).When + * changing this dynamically, we might want to block until the shared cache + * shrinks enough + * + */ + private void adjustCacheSizes() { + long newSharedCacheSize = this.totalCacheSize - this.reservedCacheSize; + logger.info("Setting the shared cache size to " + newSharedCacheSize); + for(Environment environment: unreservedStores) { + EnvironmentMutableConfig mConfig = environment.getMutableConfig(); + mConfig.setCacheSize(newSharedCacheSize); + environment.setMutableConfig(mConfig); + } + } + + public Environment getEnvironment(StoreDefinition storeDef) throws DatabaseException { + String storeName = storeDef.getName(); synchronized(lock) { if(useOneEnvPerStore) { - // if we have already created this environment return a // reference if(environments.containsKey(storeName)) @@ -145,10 +177,28 @@ private Environment getEnvironment(String storeName) throws DatabaseException { File bdbDir = new File(bdbMasterDir, storeName); createBdbDirIfNecessary(bdbDir); + // configure the BDB cache + if(storeDef.hasMemoryFootprint()) { + // make room for the reservation, by adjusting other stores + long reservedBytes = storeDef.getMemoryFootprintMB() * BYTES_PER_MB; + this.reservedCacheSize += reservedBytes; + adjustCacheSizes(); + environmentConfig.setSharedCache(false); + environmentConfig.setCacheSize(reservedBytes); + } else { + environmentConfig.setSharedCache(true); + environmentConfig.setCacheSize(this.totalCacheSize - this.reservedCacheSize); + } + Environment environment = new Environment(bdbDir, environmentConfig); logger.info("Creating environment for " + storeName + ": "); logEnvironmentConfig(environment.getConfig()); environments.put(storeName, environment); + + // save this up so we can adjust later if needed + if(!storeDef.hasMemoryFootprint()) + this.unreservedStores.add(environment); + return environment; } else { if(!environments.isEmpty()) @@ -183,18 +233,24 @@ private void logEnvironmentConfig(EnvironmentConfig config) { + config.getConfigParam(EnvironmentConfig.CLEANER_MIN_FILE_UTILIZATION)); logger.info(" BDB " + EnvironmentConfig.LOG_FILE_MAX + " = " + config.getConfigParam(EnvironmentConfig.LOG_FILE_MAX)); + logger.info(" BDB " + config.toString().replace('\n', ',')); } public String getType() { return TYPE_NAME; } - public EnvironmentStats getStats(String storeName, boolean fast) { - StatsConfig config = new StatsConfig(); - config.setFast(fast); + public String getStats(String storeName, boolean fast) { try { - Environment env = getEnvironment(storeName); - return env.getStats(config); + if(environments.containsKey(storeName)) { + StatsConfig config = new StatsConfig(); + config.setFast(fast); + Environment env = environments.get(storeName); + return env.getStats(config).toString(); + } else { + // return empty string if environment not created yet + return ""; + } } catch(DatabaseException e) { throw new VoldemortException(e); } @@ -207,7 +263,7 @@ public String getEnvStatsAsString(String storeName) throws Exception { @JmxOperation(description = "A variety of stats about one BDB environment.") public String getEnvStatsAsString(String storeName, boolean fast) throws Exception { - String envStats = getStats(storeName, fast).toString(); + String envStats = getStats(storeName, fast); logger.debug("Bdb Environment stats:\n" + envStats); return envStats; } @@ -241,4 +297,34 @@ public void close() { } } + /** + * Detect what has changed in the store definition and rewire BDB + * environments accordingly. + * + * @param storeDef updated store definition + */ + public void update(StoreDefinition storeDef) { + if(!useOneEnvPerStore) + throw new VoldemortException("Memory foot print can be set only when using different environments per store"); + + Environment environment = environments.get(storeDef.getName()); + // change reservation amount of reserved store + if(!unreservedStores.contains(environment) && storeDef.hasMemoryFootprint()) { + EnvironmentMutableConfig mConfig = environment.getMutableConfig(); + long currentCacheSize = mConfig.getCacheSize(); + long newCacheSize = storeDef.getMemoryFootprintMB() * BYTES_PER_MB; + if(currentCacheSize != newCacheSize) { + this.reservedCacheSize = this.reservedCacheSize - currentCacheSize + newCacheSize; + adjustCacheSizes(); + mConfig.setCacheSize(newCacheSize); + environment.setMutableConfig(mConfig); + logger.info("Setting private cache for store " + storeDef.getName() + " to " + + newCacheSize); + } + } else { + // we cannot support changing a reserved store to unreserved or vice + // versa since the sharedCache param is not mutable + throw new VoldemortException("Cannot switch between shared and private cache dynamically"); + } + } } diff --git a/src/java/voldemort/store/memory/CacheStorageConfiguration.java b/src/java/voldemort/store/memory/CacheStorageConfiguration.java index 52e610764a..431c0a849a 100644 --- a/src/java/voldemort/store/memory/CacheStorageConfiguration.java +++ b/src/java/voldemort/store/memory/CacheStorageConfiguration.java @@ -19,9 +19,11 @@ import java.util.List; import java.util.concurrent.ConcurrentMap; +import voldemort.VoldemortException; import voldemort.server.VoldemortConfig; import voldemort.store.StorageConfiguration; import voldemort.store.StorageEngine; +import voldemort.store.StoreDefinition; import voldemort.utils.ByteArray; import voldemort.versioning.Versioned; @@ -45,14 +47,18 @@ public CacheStorageConfiguration(VoldemortConfig config) {} public void close() {} - public StorageEngine getStore(String name) { + public StorageEngine getStore(StoreDefinition storeDef) { ConcurrentMap>> backingMap = new MapMaker().softValues() .makeMap(); - return new InMemoryStorageEngine(name, backingMap); + return new InMemoryStorageEngine(storeDef.getName(), backingMap); } public String getType() { return TYPE_NAME; } + public void update(StoreDefinition storeDef) { + throw new VoldemortException("Storage config updates not permitted for " + this.getType() + + " storage engine"); + } } diff --git a/src/java/voldemort/store/memory/InMemoryStorageConfiguration.java b/src/java/voldemort/store/memory/InMemoryStorageConfiguration.java index 7193c89cdf..df40f6ae37 100644 --- a/src/java/voldemort/store/memory/InMemoryStorageConfiguration.java +++ b/src/java/voldemort/store/memory/InMemoryStorageConfiguration.java @@ -19,9 +19,11 @@ import java.util.List; import java.util.concurrent.ConcurrentHashMap; +import voldemort.VoldemortException; import voldemort.server.VoldemortConfig; import voldemort.store.StorageConfiguration; import voldemort.store.StorageEngine; +import voldemort.store.StoreDefinition; import voldemort.utils.ByteArray; import voldemort.versioning.Versioned; @@ -39,8 +41,8 @@ public InMemoryStorageConfiguration() {} @SuppressWarnings("unused") public InMemoryStorageConfiguration(VoldemortConfig config) {} - public StorageEngine getStore(String name) { - return new InMemoryStorageEngine(name, + public StorageEngine getStore(StoreDefinition storeDef) { + return new InMemoryStorageEngine(storeDef.getName(), new ConcurrentHashMap>>()); } @@ -50,4 +52,8 @@ public String getType() { public void close() {} + public void update(StoreDefinition storeDef) { + throw new VoldemortException("Storage config updates not permitted for " + + this.getClass().getCanonicalName()); + } } diff --git a/src/java/voldemort/store/mysql/MysqlStorageConfiguration.java b/src/java/voldemort/store/mysql/MysqlStorageConfiguration.java index c2d8e12b8a..57e5bce400 100644 --- a/src/java/voldemort/store/mysql/MysqlStorageConfiguration.java +++ b/src/java/voldemort/store/mysql/MysqlStorageConfiguration.java @@ -24,6 +24,7 @@ import voldemort.server.VoldemortConfig; import voldemort.store.StorageConfiguration; import voldemort.store.StorageEngine; +import voldemort.store.StoreDefinition; import voldemort.utils.ByteArray; public class MysqlStorageConfiguration implements StorageConfiguration { @@ -42,8 +43,8 @@ public MysqlStorageConfiguration(VoldemortConfig config) { this.dataSource = ds; } - public StorageEngine getStore(String name) { - return new MysqlStorageEngine(name, dataSource); + public StorageEngine getStore(StoreDefinition storeDef) { + return new MysqlStorageEngine(storeDef.getName(), dataSource); } public String getType() { @@ -58,4 +59,8 @@ public void close() { } } + public void update(StoreDefinition storeDef) { + throw new VoldemortException("Storage config updates not permitted for " + + this.getClass().getCanonicalName()); + } } diff --git a/src/java/voldemort/store/readonly/ReadOnlyStorageConfiguration.java b/src/java/voldemort/store/readonly/ReadOnlyStorageConfiguration.java index 78d8e16bef..ce6759e3c0 100644 --- a/src/java/voldemort/store/readonly/ReadOnlyStorageConfiguration.java +++ b/src/java/voldemort/store/readonly/ReadOnlyStorageConfiguration.java @@ -25,10 +25,12 @@ import javax.management.MBeanServer; import javax.management.ObjectName; +import voldemort.VoldemortException; import voldemort.routing.RoutingStrategy; import voldemort.server.VoldemortConfig; import voldemort.store.StorageConfiguration; import voldemort.store.StorageEngine; +import voldemort.store.StoreDefinition; import voldemort.utils.ByteArray; import voldemort.utils.JmxUtils; import voldemort.utils.ReflectUtils; @@ -65,16 +67,17 @@ public void setRoutingStrategy(RoutingStrategy routingStrategy) { this.routingStrategy = routingStrategy; } - public StorageEngine getStore(String name) { - ReadOnlyStorageEngine store = new ReadOnlyStorageEngine(name, + public StorageEngine getStore(StoreDefinition storeDef) { + ReadOnlyStorageEngine store = new ReadOnlyStorageEngine(storeDef.getName(), this.searcher, this.routingStrategy, this.nodeId, - new File(storageDir, name), + new File(storageDir, + storeDef.getName()), numBackups, deleteBackupMs); ObjectName objName = JmxUtils.createObjectName(JmxUtils.getPackageName(store.getClass()), - name + nodeId); + storeDef.getName() + nodeId); JmxUtils.registerMbean(ManagementFactory.getPlatformMBeanServer(), JmxUtils.createModelMBean(store), objName); @@ -87,4 +90,8 @@ public String getType() { return TYPE_NAME; } + public void update(StoreDefinition storeDef) { + throw new VoldemortException("Storage config updates not permitted for " + + this.getClass().getCanonicalName()); + } } diff --git a/src/java/voldemort/store/views/ViewStorageConfiguration.java b/src/java/voldemort/store/views/ViewStorageConfiguration.java index be634ad6eb..35480ca9fa 100644 --- a/src/java/voldemort/store/views/ViewStorageConfiguration.java +++ b/src/java/voldemort/store/views/ViewStorageConfiguration.java @@ -34,7 +34,8 @@ public ViewStorageConfiguration(VoldemortConfig config, public void close() {} - public StorageEngine getStore(String name) { + public StorageEngine getStore(StoreDefinition storeDef) { + String name = storeDef.getName(); StoreDefinition def = StoreUtils.getStoreDef(storeDefs, name); String targetName = def.getViewTargetStoreName(); StoreDefinition targetDef = StoreUtils.getStoreDef(storeDefs, targetName); @@ -86,4 +87,8 @@ public static SerializerFactory loadSerializerFactory(String className) { return (View) ReflectUtils.callConstructor(viewClass, new Object[] {}); } + public void update(StoreDefinition storeDef) { + throw new UnsupportedViewOperationException("Storage config updates not permitted for " + + this.getClass().getCanonicalName()); + } } diff --git a/src/java/voldemort/xml/StoreDefinitionsMapper.java b/src/java/voldemort/xml/StoreDefinitionsMapper.java index c51913d1a5..0926129834 100644 --- a/src/java/voldemort/xml/StoreDefinitionsMapper.java +++ b/src/java/voldemort/xml/StoreDefinitionsMapper.java @@ -96,6 +96,7 @@ public class StoreDefinitionsMapper { public final static String VIEW_TRANS_ELMT = "view-class"; public final static String VIEW_SERIALIZER_FACTORY_ELMT = "view-serializer-factory"; private final static String STORE_VERSION_ATTR = "version"; + private final static String STORE_MEMORY_FOOTPRINT = "memory-footprint"; private final Schema schema; @@ -248,6 +249,11 @@ private StoreDefinition readStore(Element store) { Integer hintPrefListSize = (null != hintPrefListSizeStr) ? Integer.parseInt(hintPrefListSizeStr) : null; + String memoryFootprintStr = store.getChildText(STORE_MEMORY_FOOTPRINT); + long memoryFootprintMB = 0; + if(memoryFootprintStr != null) + memoryFootprintMB = Long.parseLong(memoryFootprintStr); + return new StoreDefinitionBuilder().setName(name) .setType(storeType) .setDescription(description) @@ -268,6 +274,7 @@ private StoreDefinition readStore(Element store) { .setZoneCountWrites(zoneCountWrites) .setHintedHandoffStrategy(hintedHandoffStrategy) .setHintPrefListSize(hintPrefListSize) + .setMemoryFootprintMB(memoryFootprintMB) .build(); } @@ -460,6 +467,10 @@ private Element storeToElement(StoreDefinition storeDefinition) { if(storeDefinition.hasRetentionScanThrottleRate()) store.addContent(new Element(STORE_RETENTION_SCAN_THROTTLE_RATE_ELMT).setText(Integer.toString(storeDefinition.getRetentionScanThrottleRate()))); + if(storeDefinition.hasMemoryFootprint()) { + store.addContent(new Element(STORE_MEMORY_FOOTPRINT).setText(Long.toString(storeDefinition.getMemoryFootprintMB()))); + } + return store; } diff --git a/src/java/voldemort/xml/stores.xsd b/src/java/voldemort/xml/stores.xsd index b55c00f7ab..8dbd9e930d 100644 --- a/src/java/voldemort/xml/stores.xsd +++ b/src/java/voldemort/xml/stores.xsd @@ -44,6 +44,7 @@ minOccurs="0" maxOccurs="1" /> + diff --git a/src/proto/voldemort-admin.proto b/src/proto/voldemort-admin.proto index 6e9590c34c..581ee2e016 100644 --- a/src/proto/voldemort-admin.proto +++ b/src/proto/voldemort-admin.proto @@ -285,6 +285,15 @@ message NativeBackupRequest { required bool incremental = 4; } +message ReserveMemoryRequest { + required string store_name = 1; + required int64 size_in_mb = 2; +} + +message ReserveMemoryResponse { + optional Error error = 1; +} + enum AdminRequestType { GET_METADATA = 0; UPDATE_METADATA = 1; @@ -313,6 +322,7 @@ enum AdminRequestType { INITIATE_REBALANCE_NODE_ON_DONOR = 26; DELETE_STORE_REBALANCE_STATE = 27; NATIVE_BACKUP = 28; + RESERVE_MEMORY = 29; } message VoldemortAdminRequest { @@ -344,4 +354,5 @@ message VoldemortAdminRequest { optional InitiateRebalanceNodeOnDonorRequest initiate_rebalance_node_on_donor = 28; optional DeleteStoreRebalanceStateRequest delete_store_rebalance_state = 29; optional NativeBackupRequest native_backup = 30; + optional ReserveMemoryRequest reserve_memory = 31; } diff --git a/test/common/voldemort/TestUtils.java b/test/common/voldemort/TestUtils.java index 8d305ee075..fadbb64dd1 100644 --- a/test/common/voldemort/TestUtils.java +++ b/test/common/voldemort/TestUtils.java @@ -31,6 +31,7 @@ import voldemort.cluster.Cluster; import voldemort.cluster.Node; import voldemort.store.Store; +import voldemort.store.StoreDefinition; import voldemort.utils.ByteArray; import voldemort.utils.Utils; import voldemort.versioning.VectorClock; @@ -340,4 +341,67 @@ public static T getPrivateValue(Object instance, String fieldName) throws Ex return (T) eventDataQueueField.get(instance); } + /** + * Wrapper to get a StoreDefinition object constructed, given a store name + */ + public static StoreDefinition makeStoreDefinition(String storeName) { + return new StoreDefinition(storeName, + null, + null, + null, + null, + null, + null, + null, + 0, + null, + 0, + null, + 0, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + 0); + } + + /** + * Wrapper to get a StoreDefinition object constructed, given a store name, + * memory foot print + */ + public static StoreDefinition makeStoreDefinition(String storeName, long memFootprintMB) { + return new StoreDefinition(storeName, + null, + null, + null, + null, + null, + null, + null, + 0, + null, + 0, + null, + 0, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + memFootprintMB); + } + } diff --git a/test/integration/voldemort/performance/CacheStorageEnginePerformanceTest.java b/test/integration/voldemort/performance/CacheStorageEnginePerformanceTest.java index edfebcc048..49d8584018 100644 --- a/test/integration/voldemort/performance/CacheStorageEnginePerformanceTest.java +++ b/test/integration/voldemort/performance/CacheStorageEnginePerformanceTest.java @@ -18,6 +18,7 @@ import java.util.concurrent.atomic.AtomicInteger; +import voldemort.TestUtils; import voldemort.store.Store; import voldemort.store.memory.CacheStorageConfiguration; import voldemort.utils.ByteArray; @@ -38,7 +39,7 @@ public static void main(String[] args) { final int mod = 100; final int readMax = (int) readPercent * mod; - final Store store = new CacheStorageConfiguration(null).getStore("test"); + final Store store = new CacheStorageConfiguration(null).getStore(TestUtils.makeStoreDefinition("test")); final AtomicInteger obsoletes = new AtomicInteger(0); PerformanceTest readWriteTest = new PerformanceTest() { diff --git a/test/integration/voldemort/performance/StorageEnginePerformanceTest.java b/test/integration/voldemort/performance/StorageEnginePerformanceTest.java index 987f1d2fb1..320de8c44f 100644 --- a/test/integration/voldemort/performance/StorageEnginePerformanceTest.java +++ b/test/integration/voldemort/performance/StorageEnginePerformanceTest.java @@ -108,7 +108,7 @@ public static void main(String[] args) throws Exception { VoldemortConfig config = new VoldemortConfig(props); StorageConfiguration storageConfig = (StorageConfiguration) ReflectUtils.callConstructor(ReflectUtils.loadClass(storageEngineClass), new Object[] { config }); - StorageEngine engine = storageConfig.getStore("test"); + StorageEngine engine = storageConfig.getStore(TestUtils.makeStoreDefinition("test")); @SuppressWarnings("unchecked") final Store store = new SerializingStore(engine, new StringSerializer(), diff --git a/test/integration/voldemort/performance/benchmark/Benchmark.java b/test/integration/voldemort/performance/benchmark/Benchmark.java index 6f62737c7b..557d3ec6d7 100644 --- a/test/integration/voldemort/performance/benchmark/Benchmark.java +++ b/test/integration/voldemort/performance/benchmark/Benchmark.java @@ -27,6 +27,7 @@ import joptsimple.OptionSet; import voldemort.ServerTestUtils; import voldemort.StaticStoreClientFactory; +import voldemort.TestUtils; import voldemort.VoldemortException; import voldemort.client.AbstractStoreClientFactory; import voldemort.client.ClientConfig; @@ -357,7 +358,7 @@ public void initializeStore(Props benchmarkProps) throws Exception { StorageConfiguration conf = (StorageConfiguration) ReflectUtils.callConstructor(ReflectUtils.loadClass(storageEngineClass), new Object[] { ServerTestUtils.getVoldemortConfig() }); - StorageEngine engine = conf.getStore(DUMMY_DB); + StorageEngine engine = conf.getStore(TestUtils.makeStoreDefinition(DUMMY_DB)); if(conf.getType().compareTo(ViewStorageConfiguration.TYPE_NAME) == 0) { engine = new ViewStorageEngine(STORE_NAME, engine, diff --git a/test/integration/voldemort/store/noop/NoopStorageConfiguration.java b/test/integration/voldemort/store/noop/NoopStorageConfiguration.java index ebc2c38cdb..741009ad1d 100644 --- a/test/integration/voldemort/store/noop/NoopStorageConfiguration.java +++ b/test/integration/voldemort/store/noop/NoopStorageConfiguration.java @@ -16,9 +16,11 @@ package voldemort.store.noop; +import voldemort.VoldemortException; import voldemort.server.VoldemortConfig; import voldemort.store.StorageConfiguration; import voldemort.store.StorageEngine; +import voldemort.store.StoreDefinition; import voldemort.utils.ByteArray; /** @@ -49,8 +51,8 @@ public NoopStorageConfiguration(VoldemortConfig config) { reflect = config.getAllProps().getBoolean(REFLECT_PROPERTY, false); } - public StorageEngine getStore(String name) { - return new NoopStorageEngine(name, reflect); + public StorageEngine getStore(StoreDefinition storeDef) { + return new NoopStorageEngine(storeDef.getName(), reflect); } public String getType() { @@ -58,4 +60,9 @@ public String getType() { } public void close() {} + + public void update(StoreDefinition storeDef) { + throw new VoldemortException("Storage config updates not permitted for " + + this.getClass().getCanonicalName()); + } } diff --git a/test/integration/voldemort/store/pausable/PausableStorageConfiguration.java b/test/integration/voldemort/store/pausable/PausableStorageConfiguration.java index a1079ed933..225f68b514 100644 --- a/test/integration/voldemort/store/pausable/PausableStorageConfiguration.java +++ b/test/integration/voldemort/store/pausable/PausableStorageConfiguration.java @@ -1,8 +1,10 @@ package voldemort.store.pausable; +import voldemort.VoldemortException; import voldemort.server.VoldemortConfig; import voldemort.store.StorageConfiguration; import voldemort.store.StorageEngine; +import voldemort.store.StoreDefinition; import voldemort.store.memory.InMemoryStorageEngine; import voldemort.utils.ByteArray; @@ -19,12 +21,16 @@ public PausableStorageConfiguration(@SuppressWarnings("unused") VoldemortConfig public void close() {} - public StorageEngine getStore(String name) { - return new PausableStorageEngine(new InMemoryStorageEngine(name)); + public StorageEngine getStore(StoreDefinition storeDef) { + return new PausableStorageEngine(new InMemoryStorageEngine(storeDef.getName())); } public String getType() { return TYPE_NAME; } + public void update(StoreDefinition storeDef) { + throw new VoldemortException("Storage config updates not permitted for " + + this.getClass().getCanonicalName()); + } } diff --git a/test/unit/voldemort/store/bdb/BdbCachePartitioningTest.java b/test/unit/voldemort/store/bdb/BdbCachePartitioningTest.java new file mode 100644 index 0000000000..7fc931c86d --- /dev/null +++ b/test/unit/voldemort/store/bdb/BdbCachePartitioningTest.java @@ -0,0 +1,142 @@ +/* + * Copyright 2008-2012 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package voldemort.store.bdb; + +import java.io.File; + +import junit.framework.TestCase; + +import org.apache.commons.io.FileDeleteStrategy; + +import voldemort.TestUtils; +import voldemort.server.VoldemortConfig; +import voldemort.store.StoreDefinition; +import voldemort.utils.Props; +import voldemort.versioning.Versioned; + +/** + * checks that BDB cache partitioning works and caches stay within limits + * + */ +public class BdbCachePartitioningTest extends TestCase { + + private File bdbMasterDir; + private BdbStorageConfiguration bdbStorage; + + @Override + protected void setUp() throws Exception { + super.setUp(); + bdbMasterDir = TestUtils.createTempDir(); + FileDeleteStrategy.FORCE.delete(bdbMasterDir); + } + + @Override + protected void tearDown() throws Exception { + super.tearDown(); + try { + if(bdbStorage != null) + bdbStorage.close(); + } finally { + FileDeleteStrategy.FORCE.delete(bdbMasterDir); + } + } + + /** + * Tests that, given no data completely fits in memory (realistic prod + * conditions), stores will stay within their limits, no matter how much + * disproportinate traffic you throw at it + */ + public void testStaticPrivateCaches() { + + int totalCache = 20 * 1024 * 1024; // total cache size + int shareA = 10 * 1024 * 1024;// A reserves 10MB + int shareB = 5 * 1024 * 1024;// B reserves 5MB + int shareC = totalCache - shareA - shareB; // the rest, 5 MB + int numRecords = 40; + + // lets use all the default values. + Props props = new Props(); + props.put("node.id", 1); + props.put("voldemort.home", "test/common/voldemort/config"); + VoldemortConfig voldemortConfig = new VoldemortConfig(props); + voldemortConfig.setBdbCacheSize(totalCache); + voldemortConfig.setBdbOneEnvPerStore(true); + voldemortConfig.setBdbDataDirectory(bdbMasterDir.toURI().getPath()); + + bdbStorage = new BdbStorageConfiguration(voldemortConfig); + StoreDefinition defA = TestUtils.makeStoreDefinition("storeA", shareA / (1024 * 1024)); + BdbStorageEngine storeA = (BdbStorageEngine) bdbStorage.getStore(defA); + + StoreDefinition defB = TestUtils.makeStoreDefinition("storeB", shareB / (1024 * 1024)); + BdbStorageEngine storeB = (BdbStorageEngine) bdbStorage.getStore(defB); + + StoreDefinition defC = TestUtils.makeStoreDefinition("storeC"); + BdbStorageEngine storeC = (BdbStorageEngine) bdbStorage.getStore(defC); + + // load data into the stores; each store is guaranteed to be ~ 40MB. + // Data won't fit in memory + byte[] value = new byte[1024 * 1024]; + for(int i = 0; i < numRecords; i++) { + storeA.put(TestUtils.toByteArray("testKey" + i), new Versioned(value), null); + storeB.put(TestUtils.toByteArray("testKey" + i), new Versioned(value), null); + storeC.put(TestUtils.toByteArray("testKey" + i), new Versioned(value), null); + } + + // we will bring all of that data into the cache, by doing a keywalk. + // This should expand the cache as much as possible + for(int i = 0; i < numRecords; i++) { + storeA.get(TestUtils.toByteArray("testKey" + i), null); + storeB.get(TestUtils.toByteArray("testKey" + i), null); + storeC.get(TestUtils.toByteArray("testKey" + i), null); + } + + long cacheSizeA = bdbStorage.getEnvironment(defA).getConfig().getCacheSize(); + long cacheSizeB = bdbStorage.getEnvironment(defB).getConfig().getCacheSize(); + long cacheSizeC = bdbStorage.getEnvironment(defC).getConfig().getCacheSize(); + + // check that they are certainly equal to expected limits. This should + // be true since the cache would definitely expand enough + assertTrue(cacheSizeA >= shareA); + assertTrue(cacheSizeB >= shareB); + assertTrue(cacheSizeC >= shareC); + + // check that they are not exceedingly high than their limits. Small + // overflows are expected. But should not be more than a 1MB + assertTrue((cacheSizeA - (shareA)) <= (1024 * 1024)); + assertTrue((cacheSizeB - (shareB)) <= (1024 * 1024)); + assertTrue((cacheSizeC - (shareC)) <= (1024 * 1024)); + + // try doing reads on store C alone, for which we have no reservations. + // The other stores should not shrink. This simulates a spike on one + // store + for(int cycle = 0; cycle < 2; cycle++) { + for(int i = 0; i < numRecords; i++) { + storeC.get(TestUtils.toByteArray("testKey" + i), null); + } + } + + long cacheSizeANow = bdbStorage.getEnvironment(defA).getConfig().getCacheSize(); + long cacheSizeBNow = bdbStorage.getEnvironment(defB).getConfig().getCacheSize(); + + assertTrue(cacheSizeA == cacheSizeANow); + assertTrue(cacheSizeB == cacheSizeBNow); + + storeA.close(); + storeB.close(); + storeC.close(); + } +} diff --git a/test/unit/voldemort/store/bdb/BdbSplitStorageEngineTest.java b/test/unit/voldemort/store/bdb/BdbSplitStorageEngineTest.java index 32fbebcc28..3a68734397 100644 --- a/test/unit/voldemort/store/bdb/BdbSplitStorageEngineTest.java +++ b/test/unit/voldemort/store/bdb/BdbSplitStorageEngineTest.java @@ -79,8 +79,8 @@ public void testNoMultipleEnvironment() { voldemortConfig.setBdbOneEnvPerStore(false); bdbStorage = new BdbStorageConfiguration(voldemortConfig); - BdbStorageEngine storeA = (BdbStorageEngine) bdbStorage.getStore("storeA"); - BdbStorageEngine storeB = (BdbStorageEngine) bdbStorage.getStore("storeB"); + BdbStorageEngine storeA = (BdbStorageEngine) bdbStorage.getStore(TestUtils.makeStoreDefinition("storeA")); + BdbStorageEngine storeB = (BdbStorageEngine) bdbStorage.getStore(TestUtils.makeStoreDefinition("storeB")); storeA.put(TestUtils.toByteArray("testKey1"), new Versioned("value".getBytes()), @@ -124,8 +124,8 @@ public void testMultipleEnvironment() { voldemortConfig.setBdbDataDirectory(bdbMasterDir.toURI().getPath()); bdbStorage = new BdbStorageConfiguration(voldemortConfig); - BdbStorageEngine storeA = (BdbStorageEngine) bdbStorage.getStore("storeA"); - BdbStorageEngine storeB = (BdbStorageEngine) bdbStorage.getStore("storeB"); + BdbStorageEngine storeA = (BdbStorageEngine) bdbStorage.getStore(TestUtils.makeStoreDefinition("storeA")); + BdbStorageEngine storeB = (BdbStorageEngine) bdbStorage.getStore(TestUtils.makeStoreDefinition("storeB")); storeA.put(TestUtils.toByteArray("testKey1"), new Versioned("value".getBytes()), diff --git a/test/unit/voldemort/store/memory/CacheStorageEngineTest.java b/test/unit/voldemort/store/memory/CacheStorageEngineTest.java index ae27c99a64..d8bf271ede 100644 --- a/test/unit/voldemort/store/memory/CacheStorageEngineTest.java +++ b/test/unit/voldemort/store/memory/CacheStorageEngineTest.java @@ -41,7 +41,7 @@ public void setUp() throws Exception { @Override public StorageEngine getStorageEngine() { - return new CacheStorageConfiguration().getStore("test"); + return new CacheStorageConfiguration().getStore(TestUtils.makeStoreDefinition("test")); } public void testNoPressureBehavior() { From 4e628fe96f88803059dcd39f5014af8f83b351bf Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Tue, 10 Jul 2012 11:12:52 -0700 Subject: [PATCH 097/209] add minimumSharedCache param + more tests. --- .../voldemort/server/VoldemortConfig.java | 10 + .../admin/AdminServiceRequestHandler.java | 1 + src/java/voldemort/store/StoreDefinition.java | 8 +- .../store/bdb/BdbStorageConfiguration.java | 50 +++- src/java/voldemort/utils/ByteUtils.java | 3 + .../store/bdb/BdbCachePartitioningTest.java | 259 +++++++++++++++--- 6 files changed, 285 insertions(+), 46 deletions(-) diff --git a/src/java/voldemort/server/VoldemortConfig.java b/src/java/voldemort/server/VoldemortConfig.java index 6f81db581d..3fc9ffc154 100644 --- a/src/java/voldemort/server/VoldemortConfig.java +++ b/src/java/voldemort/server/VoldemortConfig.java @@ -86,6 +86,7 @@ public class VoldemortConfig implements Serializable { private boolean bdbFairLatches; private long bdbStatsCacheTtlMs; private boolean bdbExposeSpaceUtilization; + private long bdbMinimumSharedCache; private String mysqlUsername; private String mysqlPassword; @@ -228,6 +229,7 @@ public VoldemortConfig(Props props) { this.bdbReadUncommitted = props.getBoolean("bdb.lock.read_uncommitted", true); this.bdbStatsCacheTtlMs = props.getLong("bdb.stats.cache.ttl.ms", 5 * Time.MS_PER_SECOND); this.bdbExposeSpaceUtilization = props.getBoolean("bdb.expose.space.utilization", true); + this.bdbMinimumSharedCache = props.getLong("bdb.minimum.shared.cache", 0); this.readOnlyBackups = props.getInt("readonly.backups", 1); this.readOnlySearchStrategy = props.getString("readonly.search.strategy", @@ -1167,6 +1169,14 @@ public void setBdbStatsCacheTtlMs(long statsCacheTtlMs) { this.bdbStatsCacheTtlMs = statsCacheTtlMs; } + public long getBdbMinimumSharedCache() { + return this.bdbMinimumSharedCache; + } + + public void setBdbMinimumSharedCache(long minimumSharedCache) { + this.bdbMinimumSharedCache = minimumSharedCache; + } + public int getSchedulerThreads() { return schedulerThreads; } diff --git a/src/java/voldemort/server/protocol/admin/AdminServiceRequestHandler.java b/src/java/voldemort/server/protocol/admin/AdminServiceRequestHandler.java index 28f10c074c..51f91980c1 100644 --- a/src/java/voldemort/server/protocol/admin/AdminServiceRequestHandler.java +++ b/src/java/voldemort/server/protocol/admin/AdminServiceRequestHandler.java @@ -1510,6 +1510,7 @@ public VAdminProto.ReserveMemoryResponse handleReserveMemory(VAdminProto.Reserve storeDefList.set(i, newStoreDef); storageService.updateStore(newStoreDef); + break; } } diff --git a/src/java/voldemort/store/StoreDefinition.java b/src/java/voldemort/store/StoreDefinition.java index 2c93d64eb4..ab9593328b 100644 --- a/src/java/voldemort/store/StoreDefinition.java +++ b/src/java/voldemort/store/StoreDefinition.java @@ -388,7 +388,8 @@ && getRoutingPolicy() == def.getRoutingPolicy() def.getSerializerFactory() != null ? def.getSerializerFactory() : null) && Objects.equal(getHintedHandoffStrategyType(), def.getHintedHandoffStrategyType()) - && Objects.equal(getHintPrefListSize(), def.getHintPrefListSize()); + && Objects.equal(getHintPrefListSize(), def.getHintPrefListSize()) + && Objects.equal(getMemoryFootprintMB(), def.getMemoryFootprintMB()); } @Override @@ -419,7 +420,8 @@ public int hashCode() { hasHintedHandoffStrategyType() ? getHintedHandoffStrategyType() : null, hasHintPreflistSize() ? getHintPrefListSize() : null, - getOwners()); + getOwners(), + getMemoryFootprintMB()); } @Override @@ -439,6 +441,6 @@ public String toString() { + getZoneCountWrites() + ", serializer factory = " + getSerializerFactory() + ")" + ", hinted-handoff-strategy = " + getHintedHandoffStrategyType() + ", hint-preflist-size = " + getHintPrefListSize() + ", owners = " + getOwners() - + ")"; + + ", memory-footprint(MB)" + getMemoryFootprintMB() + ")"; } } diff --git a/src/java/voldemort/store/bdb/BdbStorageConfiguration.java b/src/java/voldemort/store/bdb/BdbStorageConfiguration.java index 1bd6f3caf9..bf956e16c6 100644 --- a/src/java/voldemort/store/bdb/BdbStorageConfiguration.java +++ b/src/java/voldemort/store/bdb/BdbStorageConfiguration.java @@ -32,6 +32,7 @@ import voldemort.store.StorageInitializationException; import voldemort.store.StoreDefinition; import voldemort.utils.ByteArray; +import voldemort.utils.ByteUtils; import voldemort.utils.JmxUtils; import voldemort.utils.Time; @@ -58,8 +59,6 @@ public class BdbStorageConfiguration implements StorageConfiguration { private static final String SHARED_ENV_KEY = "shared"; private static Logger logger = Logger.getLogger(BdbStorageConfiguration.class); - private static final long BYTES_PER_MB = 1048576; - private final Object lock = new Object(); private final Map environments = Maps.newHashMap(); private final EnvironmentConfig environmentConfig; @@ -67,7 +66,6 @@ public class BdbStorageConfiguration implements StorageConfiguration { private final String bdbMasterDir; private final boolean useOneEnvPerStore; private final VoldemortConfig voldemortConfig; - private final long totalCacheSize; private long reservedCacheSize = 0; private Set unreservedStores; @@ -75,7 +73,6 @@ public BdbStorageConfiguration(VoldemortConfig config) { this.voldemortConfig = config; environmentConfig = new EnvironmentConfig(); environmentConfig.setTransactional(true); - totalCacheSize = config.getBdbCacheSize(); if(config.isBdbWriteTransactionsEnabled() && config.isBdbFlushTransactionsEnabled()) { environmentConfig.setDurability(Durability.COMMIT_SYNC); } else if(config.isBdbWriteTransactionsEnabled() && !config.isBdbFlushTransactionsEnabled()) { @@ -155,7 +152,7 @@ public StorageEngine getStore(StoreDefinition storeDe * */ private void adjustCacheSizes() { - long newSharedCacheSize = this.totalCacheSize - this.reservedCacheSize; + long newSharedCacheSize = voldemortConfig.getBdbCacheSize() - this.reservedCacheSize; logger.info("Setting the shared cache size to " + newSharedCacheSize); for(Environment environment: unreservedStores) { EnvironmentMutableConfig mConfig = environment.getMutableConfig(); @@ -180,14 +177,27 @@ public Environment getEnvironment(StoreDefinition storeDef) throws DatabaseExcep // configure the BDB cache if(storeDef.hasMemoryFootprint()) { // make room for the reservation, by adjusting other stores - long reservedBytes = storeDef.getMemoryFootprintMB() * BYTES_PER_MB; - this.reservedCacheSize += reservedBytes; + long reservedBytes = storeDef.getMemoryFootprintMB() * ByteUtils.BYTES_PER_MB; + long newReservedCacheSize = this.reservedCacheSize + reservedBytes; + + // check that we leave a 'minimum' shared cache + if((voldemortConfig.getBdbCacheSize() - newReservedCacheSize) < voldemortConfig.getBdbMinimumSharedCache()) { + throw new StorageInitializationException("Reservation of " + + storeDef.getMemoryFootprintMB() + + " MB for store " + + storeName + + " violates minimum shared cache size of " + + voldemortConfig.getBdbMinimumSharedCache()); + } + + this.reservedCacheSize = newReservedCacheSize; adjustCacheSizes(); environmentConfig.setSharedCache(false); environmentConfig.setCacheSize(reservedBytes); } else { environmentConfig.setSharedCache(true); - environmentConfig.setCacheSize(this.totalCacheSize - this.reservedCacheSize); + environmentConfig.setCacheSize(voldemortConfig.getBdbCacheSize() + - this.reservedCacheSize); } Environment environment = new Environment(bdbDir, environmentConfig); @@ -307,14 +317,28 @@ public void update(StoreDefinition storeDef) { if(!useOneEnvPerStore) throw new VoldemortException("Memory foot print can be set only when using different environments per store"); - Environment environment = environments.get(storeDef.getName()); + String storeName = storeDef.getName(); + Environment environment = environments.get(storeName); // change reservation amount of reserved store if(!unreservedStores.contains(environment) && storeDef.hasMemoryFootprint()) { EnvironmentMutableConfig mConfig = environment.getMutableConfig(); long currentCacheSize = mConfig.getCacheSize(); - long newCacheSize = storeDef.getMemoryFootprintMB() * BYTES_PER_MB; + long newCacheSize = storeDef.getMemoryFootprintMB() * ByteUtils.BYTES_PER_MB; if(currentCacheSize != newCacheSize) { - this.reservedCacheSize = this.reservedCacheSize - currentCacheSize + newCacheSize; + long newReservedCacheSize = this.reservedCacheSize - currentCacheSize + + newCacheSize; + + // check that we leave a 'minimum' shared cache + if((voldemortConfig.getBdbCacheSize() - newReservedCacheSize) < voldemortConfig.getBdbMinimumSharedCache()) { + throw new StorageInitializationException("Reservation of " + + storeDef.getMemoryFootprintMB() + + " MB for store " + + storeName + + " violates minimum shared cache size of " + + voldemortConfig.getBdbMinimumSharedCache()); + } + + this.reservedCacheSize = newReservedCacheSize; adjustCacheSizes(); mConfig.setCacheSize(newCacheSize); environment.setMutableConfig(mConfig); @@ -327,4 +351,8 @@ public void update(StoreDefinition storeDef) { throw new VoldemortException("Cannot switch between shared and private cache dynamically"); } } + + public long getReservedCacheSize() { + return this.reservedCacheSize; + } } diff --git a/src/java/voldemort/utils/ByteUtils.java b/src/java/voldemort/utils/ByteUtils.java index c58fc47680..4fb375458a 100644 --- a/src/java/voldemort/utils/ByteUtils.java +++ b/src/java/voldemort/utils/ByteUtils.java @@ -53,6 +53,9 @@ public class ByteUtils { public static final int MASK_00111111 = Integer.parseInt("00111111", 2); public static final int MASK_00011111 = Integer.parseInt("00011111", 2); + public static final int BYTES_PER_MB = 1048576; + public static final long BYTES_PER_GB = 1073741824; + public static MessageDigest getDigest(String algorithm) { try { return MessageDigest.getInstance(algorithm); diff --git a/test/unit/voldemort/store/bdb/BdbCachePartitioningTest.java b/test/unit/voldemort/store/bdb/BdbCachePartitioningTest.java index 7fc931c86d..5f4fa0d110 100644 --- a/test/unit/voldemort/store/bdb/BdbCachePartitioningTest.java +++ b/test/unit/voldemort/store/bdb/BdbCachePartitioningTest.java @@ -24,10 +24,16 @@ import voldemort.TestUtils; import voldemort.server.VoldemortConfig; +import voldemort.store.StorageInitializationException; import voldemort.store.StoreDefinition; +import voldemort.utils.ByteUtils; import voldemort.utils.Props; import voldemort.versioning.Versioned; +import com.sleepycat.je.Environment; +import com.sleepycat.je.EnvironmentStats; +import com.sleepycat.je.StatsConfig; + /** * checks that BDB cache partitioning works and caches stay within limits * @@ -55,6 +61,21 @@ protected void tearDown() throws Exception { } } + private EnvironmentStats getStats(Environment environment) { + StatsConfig config = new StatsConfig(); + config.setFast(true); + return environment.getStats(config); + } + + private long getAndCheckCacheSize(BdbStorageEngine engine, StoreDefinition storeDef, String key) { + engine.get(TestUtils.toByteArray(key), null); + return getStats(bdbStorage.getEnvironment(storeDef)).getCacheTotalBytes(); + } + + private long getCacheSize(StoreDefinition storeDef) { + return getStats(bdbStorage.getEnvironment(storeDef)).getCacheTotalBytes(); + } + /** * Tests that, given no data completely fits in memory (realistic prod * conditions), stores will stay within their limits, no matter how much @@ -62,9 +83,9 @@ protected void tearDown() throws Exception { */ public void testStaticPrivateCaches() { - int totalCache = 20 * 1024 * 1024; // total cache size - int shareA = 10 * 1024 * 1024;// A reserves 10MB - int shareB = 5 * 1024 * 1024;// B reserves 5MB + int totalCache = 20 * ByteUtils.BYTES_PER_MB; // total cache size + int shareA = 10 * ByteUtils.BYTES_PER_MB;// A reserves 10MB + int shareB = 5 * ByteUtils.BYTES_PER_MB;// B reserves 5MB int shareC = totalCache - shareA - shareB; // the rest, 5 MB int numRecords = 40; @@ -78,18 +99,28 @@ public void testStaticPrivateCaches() { voldemortConfig.setBdbDataDirectory(bdbMasterDir.toURI().getPath()); bdbStorage = new BdbStorageConfiguration(voldemortConfig); - StoreDefinition defA = TestUtils.makeStoreDefinition("storeA", shareA / (1024 * 1024)); + StoreDefinition defA = TestUtils.makeStoreDefinition("storeA", shareA + / (ByteUtils.BYTES_PER_MB)); BdbStorageEngine storeA = (BdbStorageEngine) bdbStorage.getStore(defA); - StoreDefinition defB = TestUtils.makeStoreDefinition("storeB", shareB / (1024 * 1024)); + StoreDefinition defB = TestUtils.makeStoreDefinition("storeB", shareB + / (ByteUtils.BYTES_PER_MB)); BdbStorageEngine storeB = (BdbStorageEngine) bdbStorage.getStore(defB); StoreDefinition defC = TestUtils.makeStoreDefinition("storeC"); BdbStorageEngine storeC = (BdbStorageEngine) bdbStorage.getStore(defC); + // before any traffic, the cache will not have grown + assertTrue(Math.abs(shareA - getCacheSize(defA)) > ByteUtils.BYTES_PER_MB); + assertTrue(Math.abs(shareB - getCacheSize(defB)) > ByteUtils.BYTES_PER_MB); + + // sharedCacheSize reading 0 confirms that the store has a private cache + assertEquals(0, getStats(bdbStorage.getEnvironment(defA)).getSharedCacheTotalBytes()); + assertEquals(0, getStats(bdbStorage.getEnvironment(defB)).getSharedCacheTotalBytes()); + // load data into the stores; each store is guaranteed to be ~ 40MB. // Data won't fit in memory - byte[] value = new byte[1024 * 1024]; + byte[] value = new byte[ByteUtils.BYTES_PER_MB]; for(int i = 0; i < numRecords; i++) { storeA.put(TestUtils.toByteArray("testKey" + i), new Versioned(value), null); storeB.put(TestUtils.toByteArray("testKey" + i), new Versioned(value), null); @@ -98,45 +129,209 @@ public void testStaticPrivateCaches() { // we will bring all of that data into the cache, by doing a keywalk. // This should expand the cache as much as possible - for(int i = 0; i < numRecords; i++) { - storeA.get(TestUtils.toByteArray("testKey" + i), null); - storeB.get(TestUtils.toByteArray("testKey" + i), null); - storeC.get(TestUtils.toByteArray("testKey" + i), null); - } + long cacheSizeA = Long.MIN_VALUE; + long cacheSizeB = Long.MIN_VALUE; + long cacheSizeC = Long.MIN_VALUE; - long cacheSizeA = bdbStorage.getEnvironment(defA).getConfig().getCacheSize(); - long cacheSizeB = bdbStorage.getEnvironment(defB).getConfig().getCacheSize(); - long cacheSizeC = bdbStorage.getEnvironment(defC).getConfig().getCacheSize(); + for(int cycle = 0; cycle < 10; cycle++) { + for(int i = 0; i < numRecords; i++) { + long cycleCacheSizeA = getAndCheckCacheSize(storeA, defA, "testKey" + i); + long cycleCacheSizeB = getAndCheckCacheSize(storeB, defB, "testKey" + i); + long cycleCacheSizeC = getAndCheckCacheSize(storeC, defC, "testKey" + i); + // record the maximum cache size, each store every grew to + cacheSizeA = (cycleCacheSizeA > cacheSizeA) ? cycleCacheSizeA : cacheSizeA; + cacheSizeB = (cycleCacheSizeB > cacheSizeB) ? cycleCacheSizeB : cacheSizeB; + cacheSizeC = (cycleCacheSizeC > cacheSizeC) ? cycleCacheSizeC : cacheSizeC; + } + } - // check that they are certainly equal to expected limits. This should - // be true since the cache would definitely expand enough - assertTrue(cacheSizeA >= shareA); - assertTrue(cacheSizeB >= shareB); - assertTrue(cacheSizeC >= shareC); + // check that they are certainly less than expected limits. + assertTrue(cacheSizeA <= shareA); + assertTrue(cacheSizeB <= shareB); + assertTrue(cacheSizeC <= shareC); // check that they are not exceedingly high than their limits. Small - // overflows are expected. But should not be more than a 1MB - assertTrue((cacheSizeA - (shareA)) <= (1024 * 1024)); - assertTrue((cacheSizeB - (shareB)) <= (1024 * 1024)); - assertTrue((cacheSizeC - (shareC)) <= (1024 * 1024)); + // overflows are okay. But should not be more than a 1MB + assertTrue(Math.abs(cacheSizeA - shareA) <= ByteUtils.BYTES_PER_MB); + assertTrue(Math.abs(cacheSizeB - shareB) <= ByteUtils.BYTES_PER_MB); + assertTrue(Math.abs(cacheSizeC - shareC) <= ByteUtils.BYTES_PER_MB); // try doing reads on store C alone, for which we have no reservations. - // The other stores should not shrink. This simulates a spike on one - // store - for(int cycle = 0; cycle < 2; cycle++) { + // This simulates a spike on one store + long cacheSizeCNow = Long.MIN_VALUE; + for(int cycle = 0; cycle < 10; cycle++) { for(int i = 0; i < numRecords; i++) { - storeC.get(TestUtils.toByteArray("testKey" + i), null); + long cycleCacheSizeCNow = getAndCheckCacheSize(storeC, defC, "testkey" + i); + // record the maximum cache size, each store grew to + cacheSizeCNow = (cycleCacheSizeCNow > cacheSizeCNow) ? cycleCacheSizeCNow + : cacheSizeCNow; } } - long cacheSizeANow = bdbStorage.getEnvironment(defA).getConfig().getCacheSize(); - long cacheSizeBNow = bdbStorage.getEnvironment(defB).getConfig().getCacheSize(); - - assertTrue(cacheSizeA == cacheSizeANow); - assertTrue(cacheSizeB == cacheSizeBNow); + assertTrue(cacheSizeCNow <= shareC); storeA.close(); storeB.close(); storeC.close(); } + + /** + * Tests that any reservation that will not violate minimum shared cache + * will fail, during server startup and dynamic updation + */ + public void testMinimumSharedCache() { + int totalCache = 20 * ByteUtils.BYTES_PER_MB; // total cache size + int shareA = 10 * ByteUtils.BYTES_PER_MB;// A reserves 10MB + + // lets use all the default values. + Props props = new Props(); + props.put("node.id", 1); + props.put("voldemort.home", "test/common/voldemort/config"); + VoldemortConfig voldemortConfig = new VoldemortConfig(props); + voldemortConfig.setBdbCacheSize(totalCache); + voldemortConfig.setBdbOneEnvPerStore(true); + voldemortConfig.setBdbDataDirectory(bdbMasterDir.toURI().getPath()); + voldemortConfig.setBdbMinimumSharedCache(15 * ByteUtils.BYTES_PER_MB); + + BdbStorageEngine storeA = null; + bdbStorage = new BdbStorageConfiguration(voldemortConfig); + assertEquals(0, bdbStorage.getReservedCacheSize()); + + try { + StoreDefinition defA = TestUtils.makeStoreDefinition("storeA", shareA + / ByteUtils.BYTES_PER_MB); + storeA = (BdbStorageEngine) bdbStorage.getStore(defA); + fail("Should have thrown exception since minSharedCache will be violated"); + } catch(StorageInitializationException sie) { + // should come here. + } + // failing operations should not alter reserved cache size + assertEquals(0, bdbStorage.getReservedCacheSize()); + + voldemortConfig.setBdbMinimumSharedCache(10 * ByteUtils.BYTES_PER_MB); + bdbStorage = new BdbStorageConfiguration(voldemortConfig); + try { + StoreDefinition defA = TestUtils.makeStoreDefinition("storeA", shareA + / ByteUtils.BYTES_PER_MB); + storeA = (BdbStorageEngine) bdbStorage.getStore(defA); + } catch(StorageInitializationException sie) { + // should not come here. + fail("minSharedCache should n't have been violated"); + } + assertEquals(shareA, bdbStorage.getReservedCacheSize()); + + long reserveCacheSize = bdbStorage.getReservedCacheSize(); + // now, try increasing the reservation dynamically and it should fail + try { + StoreDefinition defA = TestUtils.makeStoreDefinition("storeA", 15); + bdbStorage.update(defA); + fail("Should have thrown exception since minSharedCache will be violated"); + } catch(StorageInitializationException sie) { + // should come here. + } + // this failure cannot alter the reservedCacheSize + assertEquals(reserveCacheSize, bdbStorage.getReservedCacheSize()); + + if(storeA != null) + storeA.close(); + } + + public void testDynamicReservations() { + int totalCache = 20 * ByteUtils.BYTES_PER_MB; // total cache size + int shareA = 10 * ByteUtils.BYTES_PER_MB;// A reserves 10MB + int shareB = totalCache - shareA; + int numRecords = 40; + + // lets use all the default values. + Props props = new Props(); + props.put("node.id", 1); + props.put("voldemort.home", "test/common/voldemort/config"); + VoldemortConfig voldemortConfig = new VoldemortConfig(props); + voldemortConfig.setBdbCacheSize(totalCache); + voldemortConfig.setBdbOneEnvPerStore(true); + voldemortConfig.setBdbDataDirectory(bdbMasterDir.toURI().getPath()); + voldemortConfig.setBdbMinimumSharedCache(5 * ByteUtils.BYTES_PER_MB); + + bdbStorage = new BdbStorageConfiguration(voldemortConfig); + StoreDefinition defA = TestUtils.makeStoreDefinition("storeA", shareA / (1024 * 1024)); + BdbStorageEngine storeA = (BdbStorageEngine) bdbStorage.getStore(defA); + + StoreDefinition defB = TestUtils.makeStoreDefinition("storeB"); + BdbStorageEngine storeB = (BdbStorageEngine) bdbStorage.getStore(defB); + + // load data into the stores; each store is guaranteed to be ~ 40MB. + // Data won't fit in memory + byte[] value = new byte[ByteUtils.BYTES_PER_MB]; + for(int i = 0; i < numRecords; i++) { + storeA.put(TestUtils.toByteArray("testKey" + i), new Versioned(value), null); + storeB.put(TestUtils.toByteArray("testKey" + i), new Versioned(value), null); + } + + // 1. start with 10MB reserved cache for A and the rest 10MB for B + long cacheSizeA = Long.MIN_VALUE; + long cacheSizeB = Long.MIN_VALUE; + + for(int cycle = 0; cycle < 10; cycle++) { + for(int i = 0; i < numRecords; i++) { + long cycleCacheSizeA = getAndCheckCacheSize(storeA, defA, "testKey" + i); + long cycleCacheSizeB = getAndCheckCacheSize(storeB, defB, "testKey" + i); + // record the maximum cache size, each store every grew to + cacheSizeA = (cycleCacheSizeA > cacheSizeA) ? cycleCacheSizeA : cacheSizeA; + cacheSizeB = (cycleCacheSizeB > cacheSizeB) ? cycleCacheSizeB : cacheSizeB; + } + } + + assertTrue(Math.abs(cacheSizeA - shareA) <= ByteUtils.BYTES_PER_MB); + assertTrue(Math.abs(cacheSizeB - shareB) <= ByteUtils.BYTES_PER_MB); + + // 2. dynamically grow the cache to 15MB and watch B shrink. + shareA = 15 * ByteUtils.BYTES_PER_MB; + shareB = totalCache - shareA; + defA = TestUtils.makeStoreDefinition("storeA", shareA / (1024 * 1024)); + bdbStorage.update(defA); + + cacheSizeA = Long.MIN_VALUE; + cacheSizeB = Long.MIN_VALUE; + + for(int cycle = 0; cycle < 10; cycle++) { + for(int i = 0; i < numRecords; i++) { + long cycleCacheSizeA = getAndCheckCacheSize(storeA, defA, "testKey" + i); + long cycleCacheSizeB = getAndCheckCacheSize(storeB, defB, "testKey" + i); + // record the maximum cache size, each store every grew to + cacheSizeA = (cycleCacheSizeA > cacheSizeA) ? cycleCacheSizeA : cacheSizeA; + cacheSizeB = (cycleCacheSizeB > cacheSizeB) ? cycleCacheSizeB : cacheSizeB; + } + } + + assertTrue(Math.abs(cacheSizeA - shareA) <= ByteUtils.BYTES_PER_MB); + assertTrue(Math.abs(cacheSizeB - shareB) <= ByteUtils.BYTES_PER_MB); + + // 3. dynamically shrink it back to 10MB and watch B expand again. + shareA = 10 * ByteUtils.BYTES_PER_MB; + shareB = totalCache - shareA; + defA = TestUtils.makeStoreDefinition("storeA", shareA / (1024 * 1024)); + bdbStorage.update(defA); + + cacheSizeA = Long.MIN_VALUE; + cacheSizeB = Long.MIN_VALUE; + + for(int cycle = 0; cycle < 10; cycle++) { + for(int i = 0; i < numRecords; i++) { + long cycleCacheSizeA = getAndCheckCacheSize(storeA, defA, "testKey" + i); + long cycleCacheSizeB = getAndCheckCacheSize(storeB, defB, "testKey" + i); + // record the maximum cache size, each store every grew to + cacheSizeA = (cycleCacheSizeA > cacheSizeA) ? cycleCacheSizeA : cacheSizeA; + cacheSizeB = (cycleCacheSizeB > cacheSizeB) ? cycleCacheSizeB : cacheSizeB; + } + } + + // check that they are not exceedingly high than their limits. Small + // overflows are expected. But should not be more than a 1MB + assertTrue(Math.abs(cacheSizeA - shareA) <= ByteUtils.BYTES_PER_MB); + assertTrue(Math.abs(cacheSizeB - shareB) <= ByteUtils.BYTES_PER_MB); + + storeA.close(); + storeB.close(); + } + } From 415d982c6da4f1c5c717b9281570469681c6a464 Mon Sep 17 00:00:00 2001 From: Brendan Harris Date: Fri, 20 Jul 2012 14:53:52 -0700 Subject: [PATCH 098/209] Getting rid of some unused and deprecated code, added in a comment to help explain and separate the partition shuffling --- bin/generate_cluster_xml.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/bin/generate_cluster_xml.py b/bin/generate_cluster_xml.py index 08ef4c0f9a..dd1187d595 100644 --- a/bin/generate_cluster_xml.py +++ b/bin/generate_cluster_xml.py @@ -58,8 +58,8 @@ if args.zones: zone_ids = range(zones) zone_id = 0 -## Use known seed so this is repeatable -#random.seed(3119578866) + +# Shuffle up the partitions random.seed(seed) random.shuffle(part_ids) From 3886bb1a20944cf4c7501f07ed2474db218bdd8c Mon Sep 17 00:00:00 2001 From: Peter Bailis Date: Fri, 20 Jul 2012 13:56:40 -0700 Subject: [PATCH 099/209] Include log4j resources.dir in all tests via build.xml --- build.properties | 3 --- build.xml | 3 ++- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/build.properties b/build.properties index f64eb57de0..f363ea1bff 100644 --- a/build.properties +++ b/build.properties @@ -36,8 +36,5 @@ tomcat.manager.username=tomcat tomcat.manager.password=tomcat tomcat.context=/voldemort -## Log4j -log4j.properties.dir=src/java - ## Release curr.release=0.90.1 diff --git a/build.xml b/build.xml index d664ea97fb..b92a53d3a0 100644 --- a/build.xml +++ b/build.xml @@ -27,6 +27,7 @@ + @@ -37,6 +38,7 @@ + @@ -372,7 +374,6 @@ - From 2c6b61f198de355b0274b49b52fece1531e25b63 Mon Sep 17 00:00:00 2001 From: Peter Bailis Date: Fri, 20 Jul 2012 14:28:37 -0700 Subject: [PATCH 100/209] PUT returns null, leading to a NullPointerException when debug logging is enabled in SocketStore --- src/java/voldemort/store/socket/SocketStore.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/voldemort/store/socket/SocketStore.java b/src/java/voldemort/store/socket/SocketStore.java index 3781a77cbe..91b8dd5600 100644 --- a/src/java/voldemort/store/socket/SocketStore.java +++ b/src/java/voldemort/store/socket/SocketStore.java @@ -445,7 +445,7 @@ private void invokeCallback(Object o, long requestTime) { + clientRequestExecutor.getSocketChannel() .socket() .getLocalPort() + " result: " - + o.toString()); + + o); } callback.requestComplete(o, requestTime); From 21ff9346e3b21f73267282305183af6478f90e4f Mon Sep 17 00:00:00 2001 From: Zoran Simic Date: Mon, 23 Jul 2012 23:10:01 -0700 Subject: [PATCH 101/209] Removed 'test-view' from config/single_node_cluster as bin/voldemort-server.sh fails to start with it --- .gitignore | 1 + config/single_node_cluster/config/stores.xml | 14 -------------- 2 files changed, 1 insertion(+), 14 deletions(-) diff --git a/.gitignore b/.gitignore index 18cd22df29..cd6301beb2 100644 --- a/.gitignore +++ b/.gitignore @@ -13,3 +13,4 @@ server.state .version .temp .idea +data/ diff --git a/config/single_node_cluster/config/stores.xml b/config/single_node_cluster/config/stores.xml index ef4654295a..ca52d3b771 100644 --- a/config/single_node_cluster/config/stores.xml +++ b/config/single_node_cluster/config/stores.xml @@ -15,18 +15,4 @@ string - - test-view - test - ron@hogwarts.edu - - voldemort.store.views.UpperCaseView - - - string - - - string - - From fd1f9fb8cb062091f49f761ee513557d4f57f22a Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Wed, 25 Jul 2012 15:26:02 -0700 Subject: [PATCH 102/209] add close method for shutting down client's SchedulerService in AbstractStoreClientFactory --- .../client/AbstractStoreClientFactory.java | 24 +++++++++- .../voldemort/client/DefaultStoreClient.java | 48 ++++++++++--------- .../voldemort/client/ClientRegistryTest.java | 26 +++++----- 3 files changed, 60 insertions(+), 38 deletions(-) diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index 965cf45849..ae124f43a4 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -20,6 +20,8 @@ import java.net.URI; import java.net.URISyntaxException; import java.util.Collection; +import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.UUID; @@ -33,6 +35,7 @@ import voldemort.cluster.Cluster; import voldemort.cluster.Node; import voldemort.cluster.failuredetector.FailureDetector; +import voldemort.common.service.SchedulerService; import voldemort.serialization.ByteArraySerializer; import voldemort.serialization.IdentitySerializer; import voldemort.serialization.SerializationException; @@ -58,6 +61,7 @@ import voldemort.store.versioned.InconsistencyResolvingStore; import voldemort.utils.ByteArray; import voldemort.utils.JmxUtils; +import voldemort.utils.SystemTime; import voldemort.versioning.ChainedResolver; import voldemort.versioning.InconsistencyResolver; import voldemort.versioning.TimeBasedInconsistencyResolver; @@ -101,6 +105,7 @@ public abstract class AbstractStoreClientFactory implements StoreClientFactory { private final int clientZoneId; private final String clientContextName; private final AtomicInteger sequencer; + private final HashSet clientAsyncServiceRepo; public AbstractStoreClientFactory(ClientConfig config) { this.config = config; @@ -122,6 +127,7 @@ public AbstractStoreClientFactory(ClientConfig config) { config.getTimeoutConfig()); this.sequencer = new AtomicInteger(0); + this.clientAsyncServiceRepo = new HashSet(); if(this.isJmxEnabled) { JmxUtils.registerMbean(threadPool, @@ -151,13 +157,19 @@ public StoreClient getStoreClient(String storeName) { public StoreClient getStoreClient(String storeName, InconsistencyResolver> resolver) { + SchedulerService service = new SchedulerService(config.getAsyncJobThreadPoolSize(), + SystemTime.INSTANCE, + true); + clientAsyncServiceRepo.add(service); + return new DefaultStoreClient(storeName, resolver, this, 3, clientContextName, sequencer.getAndIncrement(), - config); + config, + service); } @SuppressWarnings("unchecked") @@ -457,6 +469,16 @@ public void close() { if(failureDetector != null) failureDetector.destroy(); + + stopClientAsyncSchedulers(); + } + + private void stopClientAsyncSchedulers() { + Iterator it = clientAsyncServiceRepo.iterator(); + while(it.hasNext()) { + it.next().stop(); + } + clientAsyncServiceRepo.clear(); } /* Give a unique id to avoid jmx clashes */ diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index 3d507e71f7..df26617e68 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -46,7 +46,6 @@ import voldemort.store.system.SystemStoreConstants; import voldemort.utils.JmxUtils; import voldemort.utils.ManifestFileReader; -import voldemort.utils.SystemTime; import voldemort.utils.Utils; import voldemort.versioning.InconsistencyResolver; import voldemort.versioning.InconsistentDataException; @@ -89,7 +88,7 @@ public DefaultStoreClient(String storeName, InconsistencyResolver> resolver, StoreClientFactory storeFactory, int maxMetadataRefreshAttempts) { - this(storeName, resolver, storeFactory, maxMetadataRefreshAttempts, null, 0, null); + this(storeName, resolver, storeFactory, maxMetadataRefreshAttempts, null, 0, null, null); } public DefaultStoreClient(String storeName, @@ -98,7 +97,8 @@ public DefaultStoreClient(String storeName, int maxMetadataRefreshAttempts, String clientContext, int clientSequence, - ClientConfig config) { + ClientConfig config, + SchedulerService scheduler) { this.storeName = Utils.notNull(storeName); this.resolver = resolver; @@ -112,9 +112,7 @@ public DefaultStoreClient(String storeName, this.clientId = AbstractStoreClientFactory.generateClientId(clientInfo); this.config = config; this.sysRepository = new SystemStoreRepository(); - this.scheduler = new SchedulerService(config.getAsyncJobThreadPoolSize(), - SystemTime.INSTANCE, - true); + this.scheduler = scheduler; // Registering self to be able to bootstrap client dynamically via JMX JmxUtils.registerMbean(this, JmxUtils.createObjectName(JmxUtils.getPackageName(this.getClass()), @@ -147,12 +145,17 @@ private void registerClient(String jobId, int interval) { version); GregorianCalendar cal = new GregorianCalendar(); cal.add(Calendar.SECOND, interval); - scheduler.schedule(jobId + refresher.getClass().getName(), - refresher, - cal.getTime(), - TimeUnit.MILLISECONDS.convert(interval, TimeUnit.SECONDS)); - logger.info("Client registry refresher thread started, refresh interval: " - + interval + " seconds"); + + if(scheduler != null) { + scheduler.schedule(jobId + refresher.getClass().getName(), + refresher, + cal.getTime(), + TimeUnit.MILLISECONDS.convert(interval, TimeUnit.SECONDS)); + logger.info("Client registry refresher thread started, refresh interval: " + + interval + " seconds"); + } else { + logger.warn("Client registry won't run because scheduler service is not configured"); + } } catch(Exception e) { logger.warn("Unable to register with the cluster due to the following error:", e); } @@ -183,13 +186,16 @@ public Void call() throws Exception { // schedule the job to run every 'checkInterval' period, starting // now - scheduler.schedule(jobId + asyncCheckMetadata.getClass().getName(), - asyncCheckMetadata, - new Date(), - interval); - logger.info("Metadata version check thread started. Frequency = Every " + interval - + " ms"); - + if(scheduler != null) { + scheduler.schedule(jobId + asyncCheckMetadata.getClass().getName(), + asyncCheckMetadata, + new Date(), + interval); + logger.info("Metadata version check thread started. Frequency = Every " + interval + + " ms"); + } else { + logger.warn("Metadata version check thread won't start because the scheduler service is not configured."); + } } return asyncCheckMetadata; } @@ -224,10 +230,6 @@ public void bootStrap() { } } - public void close() { - scheduler.stopInner(); - } - public boolean delete(K key) { Versioned versioned = get(key); if(versioned == null) diff --git a/test/unit/voldemort/client/ClientRegistryTest.java b/test/unit/voldemort/client/ClientRegistryTest.java index 7fe43d17ee..1cca91df59 100644 --- a/test/unit/voldemort/client/ClientRegistryTest.java +++ b/test/unit/voldemort/client/ClientRegistryTest.java @@ -30,7 +30,7 @@ import com.google.common.collect.Lists; -@SuppressWarnings({ "unchecked" }) +@SuppressWarnings( { "unchecked" }) public class ClientRegistryTest { public static final String SERVER_LOCAL_URL = "tcp://localhost:"; @@ -143,7 +143,7 @@ public void testHappyPath() { infoList = getClientRegistryContent(it); assertTrue("Client registry not updated.", infoList.get(0).getBootstrapTime() < infoList.get(0).getUpdateTime()); - ((DefaultStoreClient) client1).close(); + socketFactory.close(); } @Test @@ -227,8 +227,7 @@ public void testTwoClients() { assertTrue("Client registry not updated.", infoList.get(1).getBootstrapTime() < infoList.get(1).getUpdateTime()); - ((DefaultStoreClient) client1).close(); - ((DefaultStoreClient) client2).close(); + socketFactory.close(); } @Test @@ -329,8 +328,7 @@ public void testTwoStores() { assertTrue("Client registry not updated.", infoList.get(1).getBootstrapTime() < infoList.get(1).getUpdateTime()); - ((DefaultStoreClient) client1).close(); - ((DefaultStoreClient) client2).close(); + socketFactory.close(); } @Test @@ -462,8 +460,8 @@ public void testTwoFactories() { assertTrue("Client registry not updated.", infoList.get(1).getBootstrapTime() < infoList.get(1).getUpdateTime()); - ((DefaultStoreClient) client1).close(); - ((DefaultStoreClient) client2).close(); + socketFactory1.close(); + socketFactory2.close(); } @Test @@ -556,8 +554,8 @@ public void testOneServerFailre() { assertTrue("Client registry not updated.", infoList.get(1).getBootstrapTime() < infoList.get(1).getUpdateTime()); - ((DefaultStoreClient) client1).close(); - ((DefaultStoreClient) client2).close(); + socketFactory1.close(); + socketFactory2.close(); } @Test @@ -592,8 +590,6 @@ public void testRepeatRegistrationSameFactory() { client1.put("k1", "v1"); client2.put("k2", "v2"); - ((DefaultStoreClient) client1).close(); - ((DefaultStoreClient) client2).close(); } Iterator>> it = adminClient.fetchEntries(1, @@ -604,6 +600,8 @@ public void testRepeatRegistrationSameFactory() { ArrayList infoList = getClientRegistryContent(it); assertEquals("Incrrect # of entries created in client registry", 6, infoList.size()); + socketFactory1.close(); + socketFactory2.close(); } @Test @@ -709,8 +707,8 @@ public void testRepeatRegistrationDifferentFactories() { client1LastBootstrapTime = infoList.get(0).getBootstrapTime(); client2LastBootstrapTime = infoList.get(0).getBootstrapTime(); - ((DefaultStoreClient) client1).close(); - ((DefaultStoreClient) client2).close(); + socketFactory1.close(); + socketFactory2.close(); } } From ab97905843bd360ac0928320a20bb39b42f6a083 Mon Sep 17 00:00:00 2001 From: Brendan Harris Date: Thu, 26 Jul 2012 16:40:38 -0700 Subject: [PATCH 103/209] Per Jay Wylie's review, no longer using a dict to store the partition sets and fixed two typos --- bin/generate_cluster_xml.py | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/bin/generate_cluster_xml.py b/bin/generate_cluster_xml.py index dd1187d595..1811fdd2fc 100644 --- a/bin/generate_cluster_xml.py +++ b/bin/generate_cluster_xml.py @@ -11,7 +11,7 @@ parser = argparse.ArgumentParser(description='Build a voldemort cluster.xml.') # Add supported arguments parser.add_argument('-N', '--name', type=str, default='voldemort', dest='name', - help='the name you want to give the clusteer') + help='the name you want to give the cluster') parser.add_argument('-n', '--nodes', type=int, default=2, dest='nodes', help='the number of nodes in the cluster') parser.add_argument('-p', '--partitions', type=int, default=300, @@ -63,14 +63,13 @@ random.seed(seed) random.shuffle(part_ids) -# Assining partitions to nodes and printing cluster.xml -part_map = dict() +# Printing cluster.xml print "" % seed print "" print " %s" % name for i in xrange(nodes): - part_map[i] = ", ".join(str(p) for p in sorted(part_ids[i*partitions:(i+1)*partitions])) + node_partitions = ", ".join(str(p) for p in sorted(part_ids[i*partitions:(i+1)*partitions])) print " " print " %d" % i @@ -78,7 +77,7 @@ print " %d" % http_port print " %d" % sock_port print " %d" % admin_port - print " %s" % part_map[i] + print " %s" % node_partitions # If zones are being used, assign a zone-id if args.zones: print " %d" % zone_id From 40a3d99a2323f65d3ba95050ca329047705b812b Mon Sep 17 00:00:00 2001 From: Jakob Homan Date: Fri, 27 Jul 2012 19:16:37 -0700 Subject: [PATCH 104/209] Don't make a copy of the buffer, as it may shrink down to one elemnent. Also, use a bufferedoutputstream. --- .../voldemort/store/readonly/fetcher/HdfsFetcher.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/fetcher/HdfsFetcher.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/fetcher/HdfsFetcher.java index cf9cbb7603..0b58cd5dd4 100644 --- a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/fetcher/HdfsFetcher.java +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/fetcher/HdfsFetcher.java @@ -16,6 +16,7 @@ package voldemort.store.readonly.fetcher; +import java.io.BufferedOutputStream; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -200,16 +201,16 @@ private void copyFileWithCheckSum(FileSystem fs, OutputStream output = null; try { input = fs.open(source); - output = new FileOutputStream(dest); + output = new BufferedOutputStream(new FileOutputStream(dest)); byte[] buffer = new byte[bufferSize]; while(true) { int read = input.read(buffer); if(read < 0) { break; - } else if(read < bufferSize) { - buffer = ByteUtils.copy(buffer, 0, read); + } else { + output.write(buffer, 0, read); } - output.write(buffer); + if(fileCheckSumGenerator != null) fileCheckSumGenerator.update(buffer); if(throttler != null) From e77ee0f003449ad232e1a54dcf3fb30645bab0ad Mon Sep 17 00:00:00 2001 From: Lei Gao Date: Mon, 30 Jul 2012 16:21:25 -0700 Subject: [PATCH 105/209] undo changes to mbean registration names - don't use client context as part of the name --- .../client/AbstractStoreClientFactory.java | 65 ++------ .../client/CachingStoreClientFactory.java | 12 +- src/java/voldemort/client/ClientConfig.java | 2 +- .../voldemort/client/DefaultStoreClient.java | 49 ++++-- .../client/MockStoreClientFactory.java | 10 +- .../client/SocketStoreClientFactory.java | 12 +- .../voldemort/client/StoreClientFactory.java | 15 -- src/java/voldemort/client/SystemStore.java | 24 ++- src/java/voldemort/utils/JmxUtils.java | 23 +++ .../voldemort/StaticStoreClientFactory.java | 11 +- test/unit/voldemort/client/ClientJmxTest.java | 140 ++++++------------ 11 files changed, 141 insertions(+), 222 deletions(-) diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index ae124f43a4..90ae483ca5 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -24,7 +24,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; @@ -104,7 +103,7 @@ public abstract class AbstractStoreClientFactory implements StoreClientFactory { private final RoutedStoreFactory routedStoreFactory; private final int clientZoneId; private final String clientContextName; - private final AtomicInteger sequencer; + private final AtomicInteger clientSequencer; private final HashSet clientAsyncServiceRepo; public AbstractStoreClientFactory(ClientConfig config) { @@ -120,26 +119,22 @@ public AbstractStoreClientFactory(ClientConfig config) { this.maxBootstrapRetries = config.getMaxBootstrapRetries(); this.stats = new StoreStats(); this.clientZoneId = config.getClientZoneId(); - this.clientContextName = (null == config.getClientContextName() ? "" - : config.getClientContextName()); + this.clientContextName = config.getClientContextName(); this.routedStoreFactory = new RoutedStoreFactory(config.isPipelineRoutedStoreEnabled(), threadPool, config.getTimeoutConfig()); - this.sequencer = new AtomicInteger(0); + this.clientSequencer = new AtomicInteger(0); this.clientAsyncServiceRepo = new HashSet(); if(this.isJmxEnabled) { JmxUtils.registerMbean(threadPool, JmxUtils.createObjectName(JmxUtils.getPackageName(threadPool.getClass()), JmxUtils.getClassName(threadPool.getClass()) - + "." - + clientContextName + jmxId())); JmxUtils.registerMbean(new StoreStatsJmx(stats), JmxUtils.createObjectName("voldemort.store.stats.aggregate", - clientContextName + ".aggregate-perf" - + jmxId())); + "aggregate-perf" + jmxId())); } } @@ -167,22 +162,20 @@ public StoreClient getStoreClient(String storeName, this, 3, clientContextName, - sequencer.getAndIncrement(), + clientSequencer.getAndIncrement(), config, service); } @SuppressWarnings("unchecked") public Store getRawStore(String storeName, - InconsistencyResolver> resolver, - UUID clientId) { - return getRawStore(storeName, resolver, clientId, null, null); + InconsistencyResolver> resolver) { + return getRawStore(storeName, resolver, null, null); } @SuppressWarnings("unchecked") public Store getRawStore(String storeName, InconsistencyResolver> resolver, - UUID clientId, String customStoresXml, String clusterXmlString) { @@ -279,13 +272,7 @@ public Store getRawStore(String storeName, store = statStore; JmxUtils.registerMbean(new StoreStatsJmx(statStore.getStats()), JmxUtils.createObjectName(JmxUtils.getPackageName(store.getClass()), - clientContextName - + "." - + store.getName() - + jmxId() - + (null == clientId ? "" - : "." - + clientId.toString()))); + store.getName() + jmxId())); } if(storeDef.getKeySerializer().hasCompression() @@ -319,11 +306,6 @@ public Store getRawStore(String storeName, return serializedStore; } - public Store getRawStore(String storeName, - InconsistencyResolver> resolver) { - return getRawStore(storeName, resolver, null); - } - protected ClientConfig getConfig() { return config; } @@ -481,36 +463,13 @@ private void stopClientAsyncSchedulers() { clientAsyncServiceRepo.clear(); } + protected String getClientContext() { + return clientContextName; + } + /* Give a unique id to avoid jmx clashes */ public String jmxId() { return jmxId == 0 ? "" : Integer.toString(jmxId); } - /** - * Generate a unique client ID based on: 0. clientContext, if specified; 1. - * storeName; 2. deployment path; 3. client sequence - * - * @param storeName the name of the store the client is created for - * @param contextName the name of the client context - * @param clientSequence the client sequence number - * @return unique client ID - */ - public static UUID generateClientId(ClientInfo clientInfo) { - String contextName = clientInfo.getContext(); - int clientSequence = clientInfo.getClientSequence(); - - String newLine = System.getProperty("line.separator"); - StringBuilder context = new StringBuilder(contextName == null ? "" : contextName); - context.append(0 == clientSequence ? "" : ("." + clientSequence)); - context.append(".").append(clientInfo.getStoreName()); - context.append("@").append(clientInfo.getLocalHostName()).append(":"); - context.append(clientInfo.getDeploymentPath()).append(newLine); - - if(logger.isDebugEnabled()) { - logger.debug(context.toString()); - } - - return UUID.nameUUIDFromBytes(context.toString().getBytes()); - } - } diff --git a/src/java/voldemort/client/CachingStoreClientFactory.java b/src/java/voldemort/client/CachingStoreClientFactory.java index 444441e75c..f773130f3c 100644 --- a/src/java/voldemort/client/CachingStoreClientFactory.java +++ b/src/java/voldemort/client/CachingStoreClientFactory.java @@ -17,7 +17,6 @@ package voldemort.client; import java.util.List; -import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -76,13 +75,7 @@ public StoreClient getStoreClient(String storeName, public Store getRawStore(String storeName, InconsistencyResolver> resolver) { - return getRawStore(storeName, resolver, null); - } - - public Store getRawStore(String storeName, - InconsistencyResolver> resolver, - UUID clientId) { - return inner.getRawStore(storeName, resolver, clientId); + return getRawStore(storeName, resolver); } public void close() { @@ -126,9 +119,8 @@ else if(client instanceof LazyStoreClient) { public Store getRawStore(String storeName, InconsistencyResolver> resolver, - UUID clientId, String storesXml, String clusterXml) { - return inner.getRawStore(storeName, resolver, clientId, storesXml, clusterXml); + return inner.getRawStore(storeName, resolver, storesXml, clusterXml); } } diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index 04b8b41e0b..07e7d6f89e 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -78,7 +78,7 @@ public class ClientConfig { private long failureDetectorRequestLengthThreshold = socketTimeoutMs; private volatile int maxBootstrapRetries = 2; - private volatile String clientContextName = "default"; + private volatile String clientContextName = ""; /* 5 second check interval, in ms */ private volatile long asyncCheckMetadataInterval = 5000; diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index df26617e68..3a224d7425 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -21,7 +21,6 @@ import java.util.GregorianCalendar; import java.util.List; import java.util.Map; -import java.util.UUID; import java.util.Map.Entry; import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; @@ -77,7 +76,7 @@ public class DefaultStoreClient implements StoreClient { private final String storeName; private final InconsistencyResolver> resolver; private final SystemStoreRepository sysRepository; - private final UUID clientId; + private final String clientId; private volatile Store store; private final SchedulerService scheduler; private ClientInfo clientInfo; @@ -109,7 +108,7 @@ public DefaultStoreClient(String storeName, clientSequence, System.currentTimeMillis(), ManifestFileReader.getReleaseVersion()); - this.clientId = AbstractStoreClientFactory.generateClientId(clientInfo); + this.clientId = generateClientId(clientInfo); this.config = config; this.sysRepository = new SystemStoreRepository(); this.scheduler = scheduler; @@ -117,9 +116,7 @@ public DefaultStoreClient(String storeName, JmxUtils.registerMbean(this, JmxUtils.createObjectName(JmxUtils.getPackageName(this.getClass()), JmxUtils.getClassName(this.getClass()) - + "." + clientContext + "." - + storeName + "." - + clientId.toString())); + + "." + storeName)); // Bootstrap this client bootStrap(); @@ -130,17 +127,17 @@ public DefaultStoreClient(String storeName, config.getAsyncCheckMetadataInterval()); } - registerClient(clientId.toString(), config.getClientRegistryRefreshInterval()); - logger.info("Voldemort client created: " + clientId.toString() + "\n" + clientInfo); + registerClient(clientId, config.getClientRegistryRefreshInterval()); + logger.info("Voldemort client created: " + clientId + "\n" + clientInfo); } private void registerClient(String jobId, int interval) { SystemStore clientRegistry = this.sysRepository.getClientRegistryStore(); if(null != clientRegistry) { try { - Version version = clientRegistry.putSysStore(clientId.toString(), clientInfo); + Version version = clientRegistry.putSysStore(clientId, clientInfo); ClientRegistryRefresher refresher = new ClientRegistryRefresher(clientRegistry, - clientId.toString(), + clientId, clientInfo, version); GregorianCalendar cal = new GregorianCalendar(); @@ -214,7 +211,7 @@ public void bootStrap() { */ clusterXml = ((AbstractStoreClientFactory) storeFactory).bootstrapMetadataWithRetries(MetadataStore.CLUSTER_KEY); - this.store = storeFactory.getRawStore(storeName, resolver, clientId, null, clusterXml); + this.store = storeFactory.getRawStore(storeName, resolver, null, clusterXml); // Create system stores logger.info("Creating system stores for store " + this.storeName); @@ -498,7 +495,7 @@ else if(versions.size() == 1) } - public UUID getClientId() { + public String getClientId() { return clientId; } @@ -515,4 +512,32 @@ public String getClusterMetadataVersion() { + this.asyncCheckMetadata.getClusterMetadataVersion(); return result; } + + /** + * Generate a unique client ID based on: 0. clientContext, if specified; 1. + * storeName; 2. deployment path; 3. client sequence + * + * @param storeName the name of the store the client is created for + * @param contextName the name of the client context + * @param clientSequence the client sequence number + * @return unique client ID + */ + public String generateClientId(ClientInfo clientInfo) { + String contextName = clientInfo.getContext(); + int clientSequence = clientInfo.getClientSequence(); + + String newLine = System.getProperty("line.separator"); + StringBuilder context = new StringBuilder(contextName == null ? "" : contextName); + context.append(0 == clientSequence ? "" : ("." + clientSequence)); + context.append(".").append(clientInfo.getStoreName()); + context.append("@").append(clientInfo.getLocalHostName()).append(":"); + context.append(clientInfo.getDeploymentPath()).append(newLine); + + if(logger.isDebugEnabled()) { + logger.debug(context.toString()); + } + + return context.toString(); + } + } diff --git a/src/java/voldemort/client/MockStoreClientFactory.java b/src/java/voldemort/client/MockStoreClientFactory.java index b50f130d3b..f552232e78 100644 --- a/src/java/voldemort/client/MockStoreClientFactory.java +++ b/src/java/voldemort/client/MockStoreClientFactory.java @@ -18,7 +18,6 @@ import java.io.StringReader; import java.util.List; -import java.util.UUID; import voldemort.cluster.failuredetector.FailureDetector; import voldemort.cluster.failuredetector.NoopFailureDetector; @@ -107,8 +106,7 @@ public StoreClient getStoreClient(String storeName, } public Store getRawStore(String storeName, - InconsistencyResolver> resolver, - UUID clientId) { + InconsistencyResolver> resolver) { if(this.storesXml != null) return getRawStore(storeName); @@ -133,11 +131,6 @@ public Store getRawStore(String storeName, return consistentStore; } - public Store getRawStore(String storeName, - InconsistencyResolver> resolver) { - return getRawStore(storeName, resolver, null); - } - private Store getRawStore(String storeName) { List storeDefs = storeMapper.readStoreList(new StringReader(storesXml)); StoreDefinition storeDef = null; @@ -215,7 +208,6 @@ public FailureDetector getFailureDetector() { public Store getRawStore(String storeName, InconsistencyResolver> resolver, - UUID clientId, String storesXml, String clusterXml) { return null; diff --git a/src/java/voldemort/client/SocketStoreClientFactory.java b/src/java/voldemort/client/SocketStoreClientFactory.java index 4447891e8e..1b26d5c3a9 100644 --- a/src/java/voldemort/client/SocketStoreClientFactory.java +++ b/src/java/voldemort/client/SocketStoreClientFactory.java @@ -39,7 +39,6 @@ import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; import voldemort.store.system.SystemStoreConstants; import voldemort.utils.ByteArray; -import voldemort.utils.JmxUtils; import voldemort.versioning.InconsistencyResolver; import voldemort.versioning.Versioned; @@ -70,11 +69,6 @@ public SocketStoreClientFactory(ClientConfig config) { config.getSocketBufferSize(), config.getSocketKeepAlive(), config.isJmxEnabled()); - if(config.isJmxEnabled()) - JmxUtils.registerMbean(storeFactory, - JmxUtils.createObjectName(JmxUtils.getPackageName(storeFactory.getClass()), - JmxUtils.getClassName(storeFactory.getClass()) - + jmxId())); } @Override @@ -184,10 +178,6 @@ public void close() { } public Store getSystemStore(String storeName, String clusterXml) { - return getRawStore(storeName, - null, - null, - SystemStoreConstants.SYSTEM_STORE_SCHEMA, - clusterXml); + return getRawStore(storeName, null, SystemStoreConstants.SYSTEM_STORE_SCHEMA, clusterXml); } } diff --git a/src/java/voldemort/client/StoreClientFactory.java b/src/java/voldemort/client/StoreClientFactory.java index 71ebbef7dd..2ae6abc6d8 100644 --- a/src/java/voldemort/client/StoreClientFactory.java +++ b/src/java/voldemort/client/StoreClientFactory.java @@ -16,8 +16,6 @@ package voldemort.client; -import java.util.UUID; - import voldemort.cluster.failuredetector.FailureDetector; import voldemort.store.Store; import voldemort.versioning.InconsistencyResolver; @@ -74,18 +72,6 @@ public StoreClient getStoreClient(String storeName, Store getRawStore(String storeName, InconsistencyResolver> resolver); - /** - * Get the underlying store, not the public StoreClient interface - * - * @param storeName The name of the store - * @param resolver The inconsistency resolver - * @param clientId The unique id of the client - * @return The appropriate store - */ - Store getRawStore(String storeName, - InconsistencyResolver> resolver, - UUID clientId); - /** * Get the underlying store, not the public StoreClient interface * @@ -97,7 +83,6 @@ Store getRawStore(String storeName, */ Store getRawStore(String storeName, InconsistencyResolver> resolver, - UUID clientId, String customStoresXml, String clusterXmlString); diff --git a/src/java/voldemort/client/SystemStore.java b/src/java/voldemort/client/SystemStore.java index da37dfa9fe..42f0b117fd 100644 --- a/src/java/voldemort/client/SystemStore.java +++ b/src/java/voldemort/client/SystemStore.java @@ -56,8 +56,10 @@ public Version putSysStore(K key, V value) { this.sysStore.put(key, versioned, null); version = versioned.getVersion(); } catch(Exception e) { - logger.info("Exception caught during putSysStore:"); - e.printStackTrace(); + logger.info("Exception caught during putSysStore: " + e); + if(logger.isDebugEnabled()) { + e.printStackTrace(); + } } return version; } @@ -69,8 +71,10 @@ public Version putSysStore(K key, Versioned value) { this.sysStore.put(key, value, null); version = value.getVersion(); } catch(Exception e) { - logger.info("Exception caught during putSysStore:"); - e.printStackTrace(); + logger.info("Exception caught during putSysStore: " + e); + if(logger.isDebugEnabled()) { + e.printStackTrace(); + } } return version; } @@ -91,8 +95,10 @@ else if(items.size() > 1) else logger.debug("Got null value"); } catch(Exception e) { - logger.info("Exception caught during getSysStore:"); - e.printStackTrace(); + logger.info("Exception caught during getSysStore: " + e); + if(logger.isDebugEnabled()) { + e.printStackTrace(); + } } return versioned; } @@ -108,8 +114,10 @@ public V getValueSysStore(K key) { value = versioned.getValue(); } } catch(Exception e) { - logger.info("Exception caught during getSysStore:"); - e.printStackTrace(); + logger.info("Exception caught during getSysStore: " + e); + if(logger.isDebugEnabled()) { + e.printStackTrace(); + } } return value; } diff --git a/src/java/voldemort/utils/JmxUtils.java b/src/java/voldemort/utils/JmxUtils.java index 87cb62423c..c4d49a6ac8 100644 --- a/src/java/voldemort/utils/JmxUtils.java +++ b/src/java/voldemort/utils/JmxUtils.java @@ -68,6 +68,8 @@ public class JmxUtils { private static final Object LOCK = new Object(); private static final Logger logger = Logger.getLogger(JmxUtils.class); + public static final String MBEAN_NAME_SEPARATOR = "-"; + /** * Create a model mbean from an object using the description given in the * Jmx annotation if present. Only operations are supported so far, no @@ -350,4 +352,25 @@ public static void unregisterMbean(ObjectName name) { } } + /** + * Generate a mbean display name. If originalName is an empty string or + * null, return an empty string. Otherwise, return SEPARATOR + originalName. + * + * @param originalName The original name to be displayed + */ + public static String getJmxDisplayNameString(String originalName) { + return ((null == originalName) || "".equals(originalName)) ? "" + : (MBEAN_NAME_SEPARATOR + originalName); + } + + /** + * Generate a mbean display name. If originalName is 0, return an empty + * string. Otherwise, return SEPARATOR + originalName. + * + * @param originalName The original name to be displayed + */ + public static String getJmxDisplayNameString(int originalName) { + return (0 == originalName) ? "" : (MBEAN_NAME_SEPARATOR + originalName); + } + } diff --git a/test/common/voldemort/StaticStoreClientFactory.java b/test/common/voldemort/StaticStoreClientFactory.java index 3d48f24b06..4899125919 100644 --- a/test/common/voldemort/StaticStoreClientFactory.java +++ b/test/common/voldemort/StaticStoreClientFactory.java @@ -2,7 +2,6 @@ import java.util.Arrays; import java.util.List; -import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import voldemort.client.DefaultStoreClient; @@ -41,17 +40,10 @@ public StaticStoreClientFactory(Store... stores) { failureDetector = new NoopFailureDetector(); } - @SuppressWarnings("unchecked") - public Store getRawStore(String storeName, - InconsistencyResolver> resolver, - UUID clientId) { - return (Store) stores.get(Math.max(current.getAndIncrement(), stores.size() - 1)); - } - @SuppressWarnings("unchecked") public Store getRawStore(String storeName, InconsistencyResolver> resolver) { - return getRawStore(storeName, resolver, null); + return (Store) stores.get(Math.max(current.getAndIncrement(), stores.size() - 1)); } @SuppressWarnings("unchecked") @@ -76,7 +68,6 @@ public FailureDetector getFailureDetector() { public Store getRawStore(String storeName, InconsistencyResolver> resolver, - UUID clientId, String storesXml, String clusterXml) { return null; diff --git a/test/unit/voldemort/client/ClientJmxTest.java b/test/unit/voldemort/client/ClientJmxTest.java index af178aebe8..cb67bb1ff7 100644 --- a/test/unit/voldemort/client/ClientJmxTest.java +++ b/test/unit/voldemort/client/ClientJmxTest.java @@ -16,6 +16,16 @@ import voldemort.server.AbstractSocketService; import voldemort.utils.JmxUtils; +/** + * + * @author lgao Note: this test suite was originally created for testing mbean + * registration with client context. Because changing mbean names can be + * difficult for customers who builds monitoring systems based on the + * mbean names. We need to give some more thoughts on using client + * context as part of mbean names. This test suit is just a place holder + * for now. + */ + public class ClientJmxTest extends AbstractStoreClientFactoryTest { private static String STATS_DOMAIN = "voldemort.store.stats"; @@ -88,11 +98,9 @@ public void testTwoClientContextOnJmx() throws Exception { getValidBootstrapUrl()).getStoreClient(getValidStoreName()); // checking for aggregate stats - ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext1 - + ".aggregate-perf" + ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, "aggregate-perf" + jmxId1); - ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext2 - + ".aggregate-perf" + ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, "aggregate-perf" + jmxId2); checkForMbeanFound(c1Name); checkForMbeanFound(c2Name); @@ -100,18 +108,12 @@ public void testTwoClientContextOnJmx() throws Exception { mbServer.unregisterMBean(c2Name); // checking for per store stats - String c1type = clientContext1 + ".test" + jmxId1; - String c2type = clientContext2 + ".test" + jmxId2; - c1Name = JmxUtils.createObjectName(STATS_DOMAIN, - c1type - + "." - + ((DefaultStoreClient) c1).getClientId() - .toString()); - c2Name = JmxUtils.createObjectName(STATS_DOMAIN, - c2type - + "." - + ((DefaultStoreClient) c2).getClientId() - .toString()); + String c1type = "test" + jmxId1; + String c2type = "test" + jmxId2; + c1Name = JmxUtils.createObjectName(STATS_DOMAIN, c1type); + + c2Name = JmxUtils.createObjectName(STATS_DOMAIN, c2type); + checkForMbeanFound(c1Name); checkForMbeanFound(c2Name); mbServer.unregisterMBean(c1Name); @@ -130,11 +132,9 @@ public void testSameContextOnJmx() throws Exception { } // checking for aggregate stats - ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext - + ".aggregate-perf" + ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, "aggregate-perf" + jmxId1); - ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext - + ".aggregate-perf" + ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, "aggregate-perf" + jmxId2); checkForMbeanFound(c1Name); checkForMbeanFound(c2Name); @@ -142,18 +142,10 @@ public void testSameContextOnJmx() throws Exception { mbServer.unregisterMBean(c2Name); // checking for per store stats - String c1type = clientContext + ".test" + jmxId1; - String c2type = clientContext + ".test" + jmxId2; - c1Name = JmxUtils.createObjectName(STATS_DOMAIN, - c1type - + "." - + ((DefaultStoreClient) clients[0]).getClientId() - .toString()); - c2Name = JmxUtils.createObjectName(STATS_DOMAIN, - c2type - + "." - + ((DefaultStoreClient) clients[1]).getClientId() - .toString()); + String c1type = "test" + jmxId1; + String c2type = "test" + jmxId2; + c1Name = JmxUtils.createObjectName(STATS_DOMAIN, c1type); + c2Name = JmxUtils.createObjectName(STATS_DOMAIN, c2type); checkForMbeanFound(c1Name); checkForMbeanFound(c2Name); mbServer.unregisterMBean(c1Name); @@ -162,7 +154,7 @@ public void testSameContextOnJmx() throws Exception { @Test public void testTwoClientNoContextOnJmx() throws Exception { - String clientContextCompare = "default"; + String clientContextCompare = ""; String jmxId1 = getAndIncrementJmxId(); String jmxId2 = getAndIncrementJmxId(); @@ -170,11 +162,9 @@ public void testTwoClientNoContextOnJmx() throws Exception { StoreClient c2 = getFactory(getValidBootstrapUrl()).getStoreClient(getValidStoreName()); // checking for aggregate stats - ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare - + ".aggregate-perf" + ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, "aggregate-perf" + jmxId1); - ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare - + ".aggregate-perf" + ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, "aggregate-perf" + jmxId2); checkForMbeanFound(c1Name); checkForMbeanFound(c2Name); @@ -182,18 +172,10 @@ public void testTwoClientNoContextOnJmx() throws Exception { mbServer.unregisterMBean(c2Name); // checking for per store stats - String c1type = clientContextCompare + ".test" + jmxId1; - String c2type = clientContextCompare + ".test" + jmxId2; - c1Name = JmxUtils.createObjectName(STATS_DOMAIN, - c1type - + "." - + ((DefaultStoreClient) c1).getClientId() - .toString()); - c2Name = JmxUtils.createObjectName(STATS_DOMAIN, - c2type - + "." - + ((DefaultStoreClient) c2).getClientId() - .toString()); + String c1type = clientContextCompare + "test" + jmxId1; + String c2type = clientContextCompare + "test" + jmxId2; + c1Name = JmxUtils.createObjectName(STATS_DOMAIN, c1type); + c2Name = JmxUtils.createObjectName(STATS_DOMAIN, c2type); checkForMbeanFound(c1Name); checkForMbeanFound(c2Name); mbServer.unregisterMBean(c1Name); @@ -210,11 +192,9 @@ public void testTwoClientNullContextOnJmx() throws Exception { StoreClient c2 = getFactoryWithClientContext(null, getValidBootstrapUrl()).getStoreClient(getValidStoreName()); // checking for aggregate stats - ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare - + ".aggregate-perf" + ObjectName c1Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, "aggregate-perf" + jmxId1); - ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContextCompare - + ".aggregate-perf" + ObjectName c2Name = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, "aggregate-perf" + jmxId2); checkForMbeanFound(c1Name); checkForMbeanFound(c2Name); @@ -222,18 +202,10 @@ public void testTwoClientNullContextOnJmx() throws Exception { mbServer.unregisterMBean(c2Name); // checking for per store stats - String c1type = clientContextCompare + ".test" + jmxId1; - String c2type = clientContextCompare + ".test" + jmxId2; - c1Name = JmxUtils.createObjectName(STATS_DOMAIN, - c1type - + "." - + ((DefaultStoreClient) c1).getClientId() - .toString()); - c2Name = JmxUtils.createObjectName(STATS_DOMAIN, - c2type - + "." - + ((DefaultStoreClient) c2).getClientId() - .toString()); + String c1type = clientContextCompare + "test" + jmxId1; + String c2type = clientContextCompare + "test" + jmxId2; + c1Name = JmxUtils.createObjectName(STATS_DOMAIN, c1type); + c2Name = JmxUtils.createObjectName(STATS_DOMAIN, c2type); checkForMbeanFound(c1Name); checkForMbeanFound(c2Name); mbServer.unregisterMBean(c1Name); @@ -252,24 +224,15 @@ public void testSameContextAndFactory() throws Exception { clients[i] = factory.getStoreClient(getValidStoreName()); } - ObjectName cName = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext - + ".aggregate-perf" + ObjectName cName = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, "aggregate-perf" + jmxId); checkForMbeanFound(cName); mbServer.unregisterMBean(cName); // checking for per store stats - String ctype = clientContext + ".test" + jmxId; - ObjectName c1Name = JmxUtils.createObjectName(STATS_DOMAIN, - ctype - + "." - + ((DefaultStoreClient) clients[0]).getClientId() - .toString()); - ObjectName c2Name = JmxUtils.createObjectName(STATS_DOMAIN, - ctype - + "." - + ((DefaultStoreClient) clients[1]).getClientId() - .toString()); + String ctype = "test" + jmxId; + ObjectName c1Name = JmxUtils.createObjectName(STATS_DOMAIN, ctype); + ObjectName c2Name = JmxUtils.createObjectName(STATS_DOMAIN, ctype); checkForMbeanFound(c1Name); checkForMbeanFound(c2Name); mbServer.unregisterMBean(c1Name); @@ -286,29 +249,20 @@ public void testDifferentId() throws Exception { clients[0] = factory.getStoreClient(getValidStoreName()); clients[1] = factory.getStoreClient(getValidStoreName()); - ObjectName cName = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, clientContext - + ".aggregate-perf" + ObjectName cName = JmxUtils.createObjectName(AGGREGATE_STATS_DOMAIN, "aggregate-perf" + jmxId); checkForMbeanFound(cName); mbServer.unregisterMBean(cName); // checking for per store stats - String ctype = clientContext + ".test" + jmxId; - ObjectName c1Name = JmxUtils.createObjectName(STATS_DOMAIN, - ctype - + "." - + ((DefaultStoreClient) clients[0]).getClientId() - .toString()); - ObjectName c2Name = JmxUtils.createObjectName(STATS_DOMAIN, - ctype - + "." - + ((DefaultStoreClient) clients[1]).getClientId() - .toString()); + String ctype = "test" + jmxId; + ObjectName c1Name = JmxUtils.createObjectName(STATS_DOMAIN, ctype); + ObjectName c2Name = JmxUtils.createObjectName(STATS_DOMAIN, ctype); checkForMbeanFound(c1Name); checkForMbeanFound(c2Name); - assertTrue(!c1Name.equals(c2Name)); + // assertTrue(!c1Name.equals(c2Name)); mbServer.unregisterMBean(c1Name); - mbServer.unregisterMBean(c2Name); + // mbServer.unregisterMBean(c2Name); } private void checkForMbeanFound(ObjectName name) { From d15b6059ee2e9e624d3ccc281a6cf622b2407f91 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Tue, 31 Jul 2012 18:05:10 -0700 Subject: [PATCH 106/209] Updating the checksum code to handle computation for a buffer range --- .../java/voldemort/store/readonly/fetcher/HdfsFetcher.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/fetcher/HdfsFetcher.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/fetcher/HdfsFetcher.java index e949c12d48..74fcc8edfc 100644 --- a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/fetcher/HdfsFetcher.java +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/fetcher/HdfsFetcher.java @@ -291,11 +291,11 @@ private void copyFileWithCheckSum(FileSystem fs, if(read < 0) { break; } else { - output.write(buffer, 0, read); + output.write(buffer, 0, read); } if(fileCheckSumGenerator != null) - fileCheckSumGenerator.update(buffer); + fileCheckSumGenerator.update(buffer, 0, read); if(throttler != null) throttler.maybeThrottle(read); stats.recordBytes(read); From fcffeb5f06684dd65bc7612506ab589d422703fa Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Thu, 2 Aug 2012 14:27:37 -0700 Subject: [PATCH 107/209] Added file backed storage engine. Factored out ZenStoreClient from DefaultStoreClient (with a configurable switch). Added unit tests. --- src/java/voldemort/VoldemortAdminTool.java | 46 +-- .../client/AbstractStoreClientFactory.java | 18 +- src/java/voldemort/client/ClientConfig.java | 25 +- src/java/voldemort/client/ClientInfo.java | 38 +-- .../voldemort/client/DefaultStoreClient.java | 184 +---------- .../client/SystemStoreRepository.java | 10 +- src/java/voldemort/client/ZenStoreClient.java | 240 ++++++++++++++ .../AsyncMetadataVersionManager.java | 68 ++-- .../scheduler/ClientRegistryRefresher.java | 8 +- .../server/storage/StorageService.java | 68 +++- ...FileBackedCachingStorageConfiguration.java | 28 ++ .../FileBackedCachingStorageEngine.java | 297 ++++++++++++++++++ .../store/system/SystemStoreConstants.java | 21 +- .../utils/MetadataVersionStoreUtils.java | 45 +++ test/unit/voldemort/client/ClientJmxTest.java | 48 +-- .../voldemort/client/ClientRegistryTest.java | 48 +-- .../client/EndToEndRebootstrapTest.java | 224 +++++++++++++ .../server/storage/StorageServiceTest.java | 55 ++++ .../FileBackedCachingStorageEngineTest.java | 147 +++++++++ .../AsyncMetadataVersionManagerTest.java | 35 ++- 20 files changed, 1327 insertions(+), 326 deletions(-) create mode 100644 src/java/voldemort/client/ZenStoreClient.java create mode 100644 src/java/voldemort/store/configuration/FileBackedCachingStorageConfiguration.java create mode 100644 src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java create mode 100644 src/java/voldemort/utils/MetadataVersionStoreUtils.java create mode 100644 test/unit/voldemort/client/EndToEndRebootstrapTest.java create mode 100644 test/unit/voldemort/store/configuration/FileBackedCachingStorageEngineTest.java diff --git a/src/java/voldemort/VoldemortAdminTool.java b/src/java/voldemort/VoldemortAdminTool.java index 7aec774c57..b7f70be2c4 100644 --- a/src/java/voldemort/VoldemortAdminTool.java +++ b/src/java/voldemort/VoldemortAdminTool.java @@ -36,6 +36,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.Set; import joptsimple.OptionParser; @@ -67,6 +68,7 @@ import voldemort.utils.ByteUtils; import voldemort.utils.CmdUtils; import voldemort.utils.KeyDistributionGenerator; +import voldemort.utils.MetadataVersionStoreUtils; import voldemort.utils.Pair; import voldemort.utils.Utils; import voldemort.versioning.VectorClock; @@ -90,7 +92,6 @@ public class VoldemortAdminTool { private static final String ALL_METADATA = "all"; private static final String STORES_VERSION_KEY = "stores.xml"; private static final String CLUSTER_VERSION_KEY = "cluster.xml"; - private static SystemStore sysStoreVersion = null; @SuppressWarnings("unchecked") public static void main(String[] args) throws Exception { @@ -259,9 +260,9 @@ public static void main(String[] args) throws Exception { // Initialize the system store for stores.xml version String[] bootstrapUrls = new String[1]; bootstrapUrls[0] = url; - sysStoreVersion = new SystemStore("voldsys$_metadata_version", - bootstrapUrls, - 0); + SystemStore sysStoreVersion = new SystemStore(SystemStoreConstants.SystemStoreName.voldsys$_metadata_version_persistence.name(), + bootstrapUrls, + 0); String ops = ""; if(options.has("delete-partitions")) { @@ -433,7 +434,7 @@ public static void main(String[] args) throws Exception { mapper.writeCluster(newCluster)); // Update the cluster.xml version info - updateMetadataversion(CLUSTER_VERSION_KEY); + updateMetadataversion(CLUSTER_VERSION_KEY, sysStoreVersion); } else if(metadataKey.compareTo(MetadataStore.SERVER_STATE_KEY) == 0) { VoldemortState newState = VoldemortState.valueOf(metadataValue); executeSetMetadata(nodeId, @@ -457,7 +458,7 @@ public static void main(String[] args) throws Exception { * stores.xml When we split the stores.xml, make this * more granular */ - updateMetadataversion(STORES_VERSION_KEY); + updateMetadataversion(STORES_VERSION_KEY, sysStoreVersion); } else if(metadataKey.compareTo(MetadataStore.REBALANCING_STEAL_INFO) == 0) { if(!Utils.isReadableFile(metadataValue)) @@ -748,23 +749,25 @@ private static void executeCheckMetadata(AdminClient adminClient, String metadat } } - private static void updateMetadataversion(String versionKey) { - Versioned metadataVersion = sysStoreVersion.getSysStore(versionKey); - if(metadataVersion == null) { - System.err.println("Current version is null. Assuming version 0."); - metadataVersion = new Versioned((long) 1); + // Get the metadata version for the given key (cluster or store) + public static void updateMetadataversion(String versionKey, + SystemStore sysStoreVersion) { + Properties props = MetadataVersionStoreUtils.getProperties(sysStoreVersion); + if(props.getProperty(versionKey) != null) { + System.out.println("Version obtained = " + props.getProperty(versionKey)); + long newValue = Long.parseLong(props.getProperty(versionKey)) + 1; + props.setProperty(versionKey, Long.toString(newValue)); } else { - System.out.println("Version obtained = " + metadataVersion.getValue()); - long newValue = metadataVersion.getValue() + 1; - metadataVersion.setObject(newValue); + System.err.println("Current version is null. Assuming version 0."); + props.setProperty(versionKey, "0"); } - sysStoreVersion.putSysStore(versionKey, metadataVersion); + MetadataVersionStoreUtils.setProperties(sysStoreVersion, props); } - private static void executeSetMetadata(Integer nodeId, - AdminClient adminClient, - String key, - Object value) { + public static void executeSetMetadata(Integer nodeId, + AdminClient adminClient, + String key, + Object value) { List nodeIds = Lists.newArrayList(); VectorClock updatedVersion = null; @@ -800,8 +803,9 @@ private static void executeSetMetadata(Integer nodeId, + adminClient.getAdminClientCluster() .getNodeById(currentNodeId) .getId()); - adminClient.updateRemoteMetadata(currentNodeId, key, Versioned.value(value.toString(), - updatedVersion)); + adminClient.updateRemoteMetadata(currentNodeId, + key, + Versioned.value(value.toString(), updatedVersion)); } } diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index af455040a5..5db7dd998a 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -143,13 +143,17 @@ public StoreClient getStoreClient(String storeName) { public StoreClient getStoreClient(String storeName, InconsistencyResolver> resolver) { - return new DefaultStoreClient(storeName, - resolver, - this, - 3, - clientContextName, - sequencer.getAndIncrement(), - config); + if(this.config.isDefaultClientEnabled()) { + return new DefaultStoreClient(storeName, resolver, this, 3); + } + + return new ZenStoreClient(storeName, + resolver, + this, + 3, + clientContextName, + sequencer.getAndIncrement(), + config); } @SuppressWarnings("unchecked") diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index 8d8e27d755..dd86bc7c83 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -68,6 +68,9 @@ public class ClientConfig { private volatile boolean enablePipelineRoutedStore = true; private volatile int clientZoneId = Zone.DEFAULT_ZONE_ID; + // Flag to control which store client to use. Default = Enhanced + private volatile boolean useDefaultClient = false; + private volatile String failureDetectorImplementation = FailureDetectorConfig.DEFAULT_IMPLEMENTATION_CLASS_NAME; private volatile long failureDetectorBannagePeriod = FailureDetectorConfig.DEFAULT_BANNAGE_PERIOD; private volatile int failureDetectorThreshold = FailureDetectorConfig.DEFAULT_THRESHOLD; @@ -126,6 +129,7 @@ public ClientConfig() {} public static final String MAX_BOOTSTRAP_RETRIES = "max_bootstrap_retries"; public static final String CLIENT_CONTEXT_NAME = "voldemort_client_context"; public static final String ASYNC_CHECK_METADATA_INTERVAL = "check_metadata_interval"; + private static final String USE_DEFAULT_CLIENT = "use_default_client"; /** * Instantiate the client config using a properties file @@ -248,6 +252,9 @@ private void setProperties(Properties properties) { if(props.containsKey(CLIENT_ZONE_ID)) this.setClientZoneId(props.getInt(CLIENT_ZONE_ID)); + if(props.containsKey(USE_DEFAULT_CLIENT)) + this.enableDefaultClient(props.getBoolean(USE_DEFAULT_CLIENT)); + if(props.containsKey(FAILUREDETECTOR_IMPLEMENTATION_PROPERTY)) this.setFailureDetectorImplementation(props.getString(FAILUREDETECTOR_IMPLEMENTATION_PROPERTY)); @@ -290,7 +297,7 @@ private void setProperties(Properties properties) { } if(props.containsKey(ASYNC_CHECK_METADATA_INTERVAL)) { - this.setAsyncCheckMetadataInterval(props.getLong(ASYNC_CHECK_METADATA_INTERVAL, 5000)); + this.setAsyncMetadataRefreshInMs(props.getLong(ASYNC_CHECK_METADATA_INTERVAL, 5000)); } } @@ -608,6 +615,14 @@ public int getClientZoneId() { return this.clientZoneId; } + public void enableDefaultClient(boolean enableDefault) { + this.useDefaultClient = enableDefault; + } + + public boolean isDefaultClientEnabled() { + return this.useDefaultClient; + } + public boolean isPipelineRoutedStoreEnabled() { return enablePipelineRoutedStore; } @@ -717,20 +732,20 @@ public ClientConfig setClientContextName(String clientContextName) { return this; } - public long getAsyncCheckMetadataInterval() { + public long getAsyncMetadataRefreshInMs() { return asyncCheckMetadataInterval; } - public ClientConfig setAsyncCheckMetadataInterval(long asyncCheckMetadataInterval) { + public ClientConfig setAsyncMetadataRefreshInMs(long asyncCheckMetadataInterval) { this.asyncCheckMetadataInterval = asyncCheckMetadataInterval; return this; } - public int getClientRegistryRefreshInterval() { + public int getClientRegistryUpdateInSecs() { return this.clientRegistryRefreshInterval; } - public ClientConfig setClientRegistryRefreshInterval(int clientRegistryRefrshInterval) { + public ClientConfig setClientRegistryUpdateInSecs(int clientRegistryRefrshInterval) { this.clientRegistryRefreshInterval = clientRegistryRefrshInterval; return this; } diff --git a/src/java/voldemort/client/ClientInfo.java b/src/java/voldemort/client/ClientInfo.java index b9e92ff225..9f21e31b37 100644 --- a/src/java/voldemort/client/ClientInfo.java +++ b/src/java/voldemort/client/ClientInfo.java @@ -45,7 +45,7 @@ public ClientInfo(String storeName, } } - private String createDeploymentPath() { + private synchronized String createDeploymentPath() { String currentPath = null; try { currentPath = new File(".").getCanonicalPath(); @@ -56,7 +56,7 @@ private String createDeploymentPath() { return currentPath; } - private String createHostName() { + private synchronized String createHostName() { String hostName = null; try { InetAddress host = InetAddress.getLocalHost(); @@ -68,67 +68,67 @@ private String createHostName() { return hostName; } - public void setStoreName(String storeName) { + public synchronized void setStoreName(String storeName) { this.storeName = storeName; } - public String getStoreName() { + public synchronized String getStoreName() { return storeName; } - public void setBootstrapTime(long bootstrapTime) { + public synchronized void setBootstrapTime(long bootstrapTime) { this.bootstrapTime = bootstrapTime; } - public long getBootstrapTime() { + public synchronized long getBootstrapTime() { return bootstrapTime; } - public void setContext(String clientContext) { + public synchronized void setContext(String clientContext) { this.context = clientContext; } - public String getContext() { + public synchronized String getContext() { return context; } - public void setClientSequence(int clientSequence) { + public synchronized void setClientSequence(int clientSequence) { this.sequence = clientSequence; } - public int getClientSequence() { + public synchronized int getClientSequence() { return sequence; } - public void setDeploymentPath(String deploymentPath) { + public synchronized void setDeploymentPath(String deploymentPath) { this.deploymentPath = deploymentPath; } - public String getDeploymentPath() { + public synchronized String getDeploymentPath() { return deploymentPath; } - public void setLocalHostName(String localHostName) { + public synchronized void setLocalHostName(String localHostName) { this.localHostName = localHostName; } - public String getLocalHostName() { + public synchronized String getLocalHostName() { return localHostName; } - public void setUpdateTime(long updateTime) { + public synchronized void setUpdateTime(long updateTime) { this.updateTime = updateTime; } - public long getUpdateTime() { + public synchronized long getUpdateTime() { return this.updateTime; } - public void setReleaseVersion(String version) { + public synchronized void setReleaseVersion(String version) { this.releaseVersion = version; } - public String getReleaseVersion() { + public synchronized String getReleaseVersion() { return this.releaseVersion; } @@ -152,7 +152,7 @@ public boolean equals(Object object) { } @Override - public String toString() { + public synchronized String toString() { StringBuilder builder = new StringBuilder(); builder.append("boostrapTime[").append(bootstrapTime).append("], "); builder.append("context[").append(context).append("], "); diff --git a/src/java/voldemort/client/DefaultStoreClient.java b/src/java/voldemort/client/DefaultStoreClient.java index ccd8749890..efd71ba888 100644 --- a/src/java/voldemort/client/DefaultStoreClient.java +++ b/src/java/voldemort/client/DefaultStoreClient.java @@ -16,36 +16,23 @@ package voldemort.client; -import java.util.Calendar; -import java.util.Date; -import java.util.GregorianCalendar; import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.UUID; -import java.util.concurrent.Callable; import org.apache.log4j.Logger; import voldemort.VoldemortException; import voldemort.annotations.concurrency.Threadsafe; -import voldemort.annotations.jmx.JmxGetter; import voldemort.annotations.jmx.JmxManaged; import voldemort.annotations.jmx.JmxOperation; -import voldemort.client.scheduler.AsyncMetadataVersionManager; -import voldemort.client.scheduler.ClientRegistryRefresher; import voldemort.cluster.Node; -import voldemort.common.service.SchedulerService; import voldemort.routing.RoutingStrategy; import voldemort.serialization.Serializer; import voldemort.store.InvalidMetadataException; import voldemort.store.Store; import voldemort.store.StoreCapabilityType; -import voldemort.store.metadata.MetadataStore; -import voldemort.store.system.SystemStoreConstants; import voldemort.utils.JmxUtils; -import voldemort.utils.ManifestFileReader; -import voldemort.utils.SystemTime; import voldemort.utils.Utils; import voldemort.versioning.InconsistencyResolver; import voldemort.versioning.InconsistentDataException; @@ -69,165 +56,38 @@ @JmxManaged(description = "A voldemort client") public class DefaultStoreClient implements StoreClient { - private static final int ASYNC_THREADS_COUNT = 2; - private static final boolean ALLOW_INTERRUPT_ASYNC = true; - private final Logger logger = Logger.getLogger(DefaultStoreClient.class); - private final StoreClientFactory storeFactory; - - private final ClientConfig config; - private final int metadataRefreshAttempts; - private final String storeName; - private final InconsistencyResolver> resolver; - private final SystemStoreRepository sysRepository; - private final UUID clientId; - private volatile Store store; - private final SchedulerService scheduler; - private ClientInfo clientInfo; - private String clusterXml; - private AsyncMetadataVersionManager asyncCheckMetadata = null; + protected StoreClientFactory storeFactory; + protected int metadataRefreshAttempts; + protected String storeName; + protected InconsistencyResolver> resolver; + protected volatile Store store; public DefaultStoreClient(String storeName, InconsistencyResolver> resolver, StoreClientFactory storeFactory, int maxMetadataRefreshAttempts) { - this(storeName, resolver, storeFactory, maxMetadataRefreshAttempts, null, 0, null); - } - - public DefaultStoreClient(String storeName, - InconsistencyResolver> resolver, - StoreClientFactory storeFactory, - int maxMetadataRefreshAttempts, - String clientContext, - int clientSequence, - ClientConfig config) { - this.storeName = Utils.notNull(storeName); this.resolver = resolver; this.storeFactory = Utils.notNull(storeFactory); this.metadataRefreshAttempts = maxMetadataRefreshAttempts; - this.clientInfo = new ClientInfo(storeName, - clientContext, - clientSequence, - System.currentTimeMillis(), - ManifestFileReader.getReleaseVersion()); - this.clientId = AbstractStoreClientFactory.generateClientId(clientInfo); - this.config = config; - this.sysRepository = new SystemStoreRepository(); - this.scheduler = new SchedulerService(ASYNC_THREADS_COUNT, - SystemTime.INSTANCE, - ALLOW_INTERRUPT_ASYNC); + // Registering self to be able to bootstrap client dynamically via JMX JmxUtils.registerMbean(this, JmxUtils.createObjectName(JmxUtils.getPackageName(this.getClass()), JmxUtils.getClassName(this.getClass()) - + "." + clientContext + "." - + storeName + "." - + clientId.toString())); + + "." + storeName)); - // Bootstrap this client bootStrap(); - - // Initialize the background thread for checking metadata version - if(config != null) { - asyncCheckMetadata = scheduleMetadataChecker(clientId.toString(), - config.getAsyncCheckMetadataInterval()); - } - - registerClient(clientId.toString(), config.getClientRegistryRefreshInterval()); - logger.info("Voldemort client created: " + clientId.toString() + "\n" + clientInfo); } - private void registerClient(String jobId, int interval) { - SystemStore clientRegistry = this.sysRepository.getClientRegistryStore(); - if(null != clientRegistry) { - try { - Version version = clientRegistry.putSysStore(clientId.toString(), clientInfo); - ClientRegistryRefresher refresher = new ClientRegistryRefresher(clientRegistry, - clientId.toString(), - clientInfo, - version); - GregorianCalendar cal = new GregorianCalendar(); - cal.add(Calendar.SECOND, interval); - scheduler.schedule(jobId + refresher.getClass().getName(), - refresher, - cal.getTime(), - interval * 1000); - logger.info("Client registry refresher thread started, refresh frequency: " - + interval + " seconds"); - } catch(Exception e) { - logger.warn("Unable to register with the cluster due to the following error:", e); - } - } else { - logger.warn(SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name() - + "not found. Unable to registry with voldemort cluster."); - } - } - - private AsyncMetadataVersionManager scheduleMetadataChecker(String jobId, long interval) { - AsyncMetadataVersionManager asyncCheckMetadata = null; - SystemStore versionStore = this.sysRepository.getVersionStore(); - if(versionStore == null) - logger.warn("Metadata version system store not found. Cannot run Metadata version check thread."); - else { - - // Create a callback for re-bootstrapping the client - Callable rebootstrapCallback = new Callable() { - - public Void call() throws Exception { - bootStrap(); - return null; - } - }; - - asyncCheckMetadata = new AsyncMetadataVersionManager(this.sysRepository, - rebootstrapCallback); - - // schedule the job to run every 'checkInterval' period, starting - // now - scheduler.schedule(jobId + asyncCheckMetadata.getClass().getName(), - asyncCheckMetadata, - new Date(), - interval); - logger.info("Metadata version check thread started. Frequency = Every " + interval - + " ms"); - - } - return asyncCheckMetadata; - } + // Default constructor invoked from child class + public DefaultStoreClient() {} @JmxOperation(description = "bootstrap metadata from the cluster.") public void bootStrap() { logger.info("Bootstrapping metadata for store " + this.storeName); - - /* - * Since we need cluster.xml for bootstrapping this client as well as - * all the System stores, just fetch it once and pass it around. - * - * Seems hackish since bootstrapMetadataWithRetries only exists for - * AbstractStoreClientFactory. TODO: Think about making this part of the - * generic interface ? - */ - clusterXml = ((AbstractStoreClientFactory) storeFactory).bootstrapMetadataWithRetries(MetadataStore.CLUSTER_KEY); - - this.store = storeFactory.getRawStore(storeName, resolver, clientId, null, clusterXml); - - // Create system stores - logger.info("Creating system stores for store " + this.storeName); - this.sysRepository.createSystemStores(this.config, this.clusterXml); - - /* - * Update to the new metadata versions (in case we got here from Invalid - * Metadata exception). This will prevent another bootstrap via the - * Async metadata checker - */ - if(asyncCheckMetadata != null) { - asyncCheckMetadata.updateMetadataVersions(); - } - } - - public void close() { - scheduler.stopInner(); + this.store = storeFactory.getRawStore(storeName, resolver); } public boolean delete(K key) { @@ -297,7 +157,7 @@ public Versioned get(K key, Versioned defaultValue, Object transform) { + " metadata refresh attempts failed."); } - private List getVersions(K key) { + protected List getVersions(K key) { for(int attempts = 0; attempts < this.metadataRefreshAttempts; attempts++) { try { return store.getVersions(key); @@ -311,7 +171,7 @@ private List getVersions(K key) { + " metadata refresh attempts failed."); } - private Versioned getItemOrThrow(K key, Versioned defaultValue, List> items) { + protected Versioned getItemOrThrow(K key, Versioned defaultValue, List> items) { if(items.size() == 0) return defaultValue; else if(items.size() == 1) @@ -441,7 +301,7 @@ public List getResponsibleNodes(K key) { } @SuppressWarnings("unused") - private Version getVersion(K key) { + protected Version getVersion(K key) { List versions = getVersions(key); if(versions.size() == 0) return null; @@ -497,22 +357,4 @@ else if(versions.size() == 1) return put(key, versioned, transforms); } - - public UUID getClientId() { - return clientId; - } - - @JmxGetter(name = "getStoreMetadataVersion") - public String getStoreMetadataVersion() { - String result = "Current Store Metadata Version : " - + this.asyncCheckMetadata.getStoreMetadataVersion(); - return result; - } - - @JmxGetter(name = "getClusterMetadataVersion") - public String getClusterMetadataVersion() { - String result = "Current Cluster Metadata Version : " - + this.asyncCheckMetadata.getClusterMetadataVersion(); - return result; - } } diff --git a/src/java/voldemort/client/SystemStoreRepository.java b/src/java/voldemort/client/SystemStoreRepository.java index e693351dbf..bdc31e89f9 100644 --- a/src/java/voldemort/client/SystemStoreRepository.java +++ b/src/java/voldemort/client/SystemStoreRepository.java @@ -38,9 +38,15 @@ public SystemStore getVersionStore() { return sysVersionStore; } - public SystemStore getClientRegistryStore() { + public SystemStore getClientRegistryStore() { String name = SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(); - SystemStore sysRegistryStore = sysStoreMap.get(name); + SystemStore sysRegistryStore = sysStoreMap.get(name); return sysRegistryStore; } + + public SystemStore getMetadataVersionStore() { + String name = SystemStoreConstants.SystemStoreName.voldsys$_metadata_version_persistence.name(); + SystemStore sysVersionStore = sysStoreMap.get(name); + return sysVersionStore; + } } diff --git a/src/java/voldemort/client/ZenStoreClient.java b/src/java/voldemort/client/ZenStoreClient.java new file mode 100644 index 0000000000..45467e99a9 --- /dev/null +++ b/src/java/voldemort/client/ZenStoreClient.java @@ -0,0 +1,240 @@ +/* + * Copyright 2008-2009 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package voldemort.client; + +import java.util.Calendar; +import java.util.Date; +import java.util.GregorianCalendar; +import java.util.UUID; +import java.util.concurrent.Callable; + +import org.apache.log4j.Logger; + +import voldemort.annotations.concurrency.Threadsafe; +import voldemort.annotations.jmx.JmxGetter; +import voldemort.annotations.jmx.JmxManaged; +import voldemort.annotations.jmx.JmxOperation; +import voldemort.client.scheduler.AsyncMetadataVersionManager; +import voldemort.client.scheduler.ClientRegistryRefresher; +import voldemort.common.service.SchedulerService; +import voldemort.store.metadata.MetadataStore; +import voldemort.store.system.SystemStoreConstants; +import voldemort.utils.JmxUtils; +import voldemort.utils.ManifestFileReader; +import voldemort.utils.SystemTime; +import voldemort.utils.Utils; +import voldemort.versioning.InconsistencyResolver; +import voldemort.versioning.Version; +import voldemort.versioning.Versioned; + +/** + * The enhanced {@link voldemort.client.StoreClient StoreClient} implementation + * you get back from a {@link voldemort.client.StoreClientFactory + * StoreClientFactory} + * + * + * @param The key type + * @param The value type + */ +@Threadsafe +@JmxManaged(description = "A voldemort client") +public class ZenStoreClient extends DefaultStoreClient { + + private static final int ASYNC_THREADS_COUNT = 2; + private static final boolean ALLOW_INTERRUPT_ASYNC = true; + + private final Logger logger = Logger.getLogger(ZenStoreClient.class); + + private final AbstractStoreClientFactory abstractStoreFactory; + private final ClientConfig config; + private final SystemStoreRepository sysRepository; + private final UUID clientId; + private final SchedulerService scheduler; + private ClientInfo clientInfo; + private String clusterXml; + private AsyncMetadataVersionManager asyncCheckMetadata = null; + + public ZenStoreClient(String storeName, + InconsistencyResolver> resolver, + AbstractStoreClientFactory storeFactory, + int maxMetadataRefreshAttempts) { + this(storeName, resolver, storeFactory, maxMetadataRefreshAttempts, null, 0, null); + } + + public ZenStoreClient(String storeName, + InconsistencyResolver> resolver, + AbstractStoreClientFactory storeFactory, + int maxMetadataRefreshAttempts, + String clientContext, + int clientSequence, + ClientConfig config) { + + super(); + this.storeName = Utils.notNull(storeName); + this.resolver = resolver; + this.abstractStoreFactory = Utils.notNull(storeFactory); + this.storeFactory = this.abstractStoreFactory; + this.metadataRefreshAttempts = maxMetadataRefreshAttempts; + this.clientInfo = new ClientInfo(storeName, + clientContext, + clientSequence, + System.currentTimeMillis(), + ManifestFileReader.getReleaseVersion()); + this.clientId = AbstractStoreClientFactory.generateClientId(clientInfo); + this.config = config; + this.sysRepository = new SystemStoreRepository(); + this.scheduler = new SchedulerService(ASYNC_THREADS_COUNT, + SystemTime.INSTANCE, + ALLOW_INTERRUPT_ASYNC); + // Registering self to be able to bootstrap client dynamically via JMX + JmxUtils.registerMbean(this, + JmxUtils.createObjectName(JmxUtils.getPackageName(this.getClass()), + JmxUtils.getClassName(this.getClass()) + + "." + clientContext + "." + + storeName + "." + + clientId.toString())); + + // Bootstrap this client + bootStrap(); + + // Initialize the background thread for checking metadata version + if(config != null) { + asyncCheckMetadata = scheduleMetadataChecker(clientId.toString(), + config.getAsyncMetadataRefreshInMs()); + } + + registerClient(clientId.toString(), config.getClientRegistryUpdateInSecs()); + logger.info("Voldemort client created: " + clientId.toString() + "\n" + clientInfo); + } + + private void registerClient(String jobId, int interval) { + SystemStore clientRegistry = this.sysRepository.getClientRegistryStore(); + if(null != clientRegistry) { + try { + Version version = clientRegistry.putSysStore(clientId.toString(), + clientInfo.toString()); + ClientRegistryRefresher refresher = new ClientRegistryRefresher(clientRegistry, + clientId.toString(), + clientInfo, + version); + GregorianCalendar cal = new GregorianCalendar(); + cal.add(Calendar.SECOND, interval); + scheduler.schedule(jobId + refresher.getClass().getName(), + refresher, + cal.getTime(), + interval * 1000); + logger.info("Client registry refresher thread started, refresh frequency: " + + interval + " seconds"); + } catch(Exception e) { + logger.warn("Unable to register with the cluster due to the following error:", e); + } + } else { + logger.warn(SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name() + + "not found. Unable to registry with voldemort cluster."); + } + } + + private AsyncMetadataVersionManager scheduleMetadataChecker(String jobId, long interval) { + AsyncMetadataVersionManager asyncCheckMetadata = null; + SystemStore versionStore = this.sysRepository.getVersionStore(); + if(versionStore == null) + logger.warn("Metadata version system store not found. Cannot run Metadata version check thread."); + else { + + // Create a callback for re-bootstrapping the client + Callable rebootstrapCallback = new Callable() { + + public Void call() throws Exception { + bootStrap(); + return null; + } + }; + + asyncCheckMetadata = new AsyncMetadataVersionManager(this.sysRepository, + rebootstrapCallback, + null); + + // schedule the job to run every 'checkInterval' period, starting + // now + scheduler.schedule(jobId + asyncCheckMetadata.getClass().getName(), + asyncCheckMetadata, + new Date(), + interval); + logger.info("Metadata version check thread started. Frequency = Every " + interval + + " ms"); + + } + return asyncCheckMetadata; + } + + @Override + @JmxOperation(description = "bootstrap metadata from the cluster.") + public void bootStrap() { + logger.info("Bootstrapping metadata for store " + this.storeName); + + /* + * Since we need cluster.xml for bootstrapping this client as well as + * all the System stores, just fetch it once and pass it around. + */ + clusterXml = abstractStoreFactory.bootstrapMetadataWithRetries(MetadataStore.CLUSTER_KEY); + + this.store = abstractStoreFactory.getRawStore(storeName, + resolver, + clientId, + null, + clusterXml); + + // Create system stores + logger.info("Creating system stores for store " + this.storeName); + this.sysRepository.createSystemStores(this.config, this.clusterXml); + + /* + * Update to the new metadata versions (in case we got here from Invalid + * Metadata exception). This will prevent another bootstrap via the + * Async metadata checker + */ + if(asyncCheckMetadata != null) { + asyncCheckMetadata.updateMetadataVersions(); + } + + if(this.clientInfo != null) { + this.clientInfo.setBootstrapTime(System.currentTimeMillis()); + } + } + + public void close() { + scheduler.stopInner(); + } + + public UUID getClientId() { + return clientId; + } + + @JmxGetter(name = "getStoreMetadataVersion") + public String getStoreMetadataVersion() { + String result = "Current Store Metadata Version : " + + this.asyncCheckMetadata.getStoreMetadataVersion(); + return result; + } + + @JmxGetter(name = "getClusterMetadataVersion") + public String getClusterMetadataVersion() { + String result = "Current Cluster Metadata Version : " + + this.asyncCheckMetadata.getClusterMetadataVersion(); + return result; + } +} diff --git a/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java b/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java index 0db362f0e6..651186d999 100644 --- a/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java +++ b/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java @@ -1,12 +1,13 @@ package voldemort.client.scheduler; +import java.util.Properties; import java.util.Random; import java.util.concurrent.Callable; import org.apache.log4j.Logger; import voldemort.client.SystemStoreRepository; -import voldemort.versioning.Versioned; +import voldemort.utils.MetadataVersionStoreUtils; /* * The AsyncMetadataVersionManager is used to track the Metadata version on the @@ -21,12 +22,13 @@ public class AsyncMetadataVersionManager implements Runnable { - public static final String STORES_VERSION_KEY = "stores.xml"; + public static String STORES_VERSION_KEY = "stores.xml"; public static final String CLUSTER_VERSION_KEY = "cluster.xml"; + public static final String VERSIONS_METADATA_STORE = "metadata-versions"; private final Logger logger = Logger.getLogger(this.getClass()); - private Versioned currentStoreVersion; - private Versioned currentClusterVersion; + private Long currentStoreVersion; + private Long currentClusterVersion; private final Callable storeClientThunk; private final SystemStoreRepository sysRepository; @@ -37,30 +39,35 @@ public class AsyncMetadataVersionManager implements Runnable { public boolean isActive = false; public AsyncMetadataVersionManager(SystemStoreRepository sysRepository, - Callable storeClientThunk) { + Callable storeClientThunk, + String storeName) { this.sysRepository = sysRepository; + if(storeName != null) { + STORES_VERSION_KEY = storeName; + } + + // Get the properties object from the system store (containing versions) + Properties versionProps = MetadataVersionStoreUtils.getProperties(this.sysRepository.getMetadataVersionStore()); try { - this.currentStoreVersion = this.sysRepository.getVersionStore() - .getSysStore(STORES_VERSION_KEY); + this.currentStoreVersion = getCurrentVersion(STORES_VERSION_KEY, versionProps); } catch(Exception e) { logger.error("Exception while getting currentStoreVersion : " + e); } try { - this.currentClusterVersion = this.sysRepository.getVersionStore() - .getSysStore(CLUSTER_VERSION_KEY); + this.currentClusterVersion = getCurrentVersion(CLUSTER_VERSION_KEY, versionProps); } catch(Exception e) { logger.error("Exception while getting currentClusterVersion : " + e); } // If the received version is null, assume version 0 if(currentStoreVersion == null) { - currentStoreVersion = new Versioned((long) 0); + currentStoreVersion = new Long(0); } if(currentClusterVersion == null) { - currentClusterVersion = new Versioned((long) 0); + currentClusterVersion = new Long(0); } logger.debug("Initial stores.xml version = " + this.currentStoreVersion); @@ -70,15 +77,25 @@ public AsyncMetadataVersionManager(SystemStoreRepository sysRepository, this.isActive = true; } + public Long getCurrentVersion(String versionKey, Properties versionProps) { + Long versionValue = null; + + if(versionProps.getProperty(versionKey) != null) { + versionValue = Long.parseLong(versionProps.getProperty(versionKey)); + } + + logger.debug("*********** For key : " + versionKey + " received value = " + versionValue); + return versionValue; + } + /* * This method checks for any update in the version for 'versionKey'. If * there is any change, it returns the new version. Otherwise it will return * a null. */ - public Versioned fetchNewVersion(String versionKey, Versioned curVersion) { + public Long fetchNewVersion(String versionKey, Long curVersion, Properties versionProps) { try { - Versioned newVersion = this.sysRepository.getVersionStore() - .getSysStore(versionKey); + Long newVersion = getCurrentVersion(versionKey, versionProps); // If version obtained is null, the store is untouched. Continue if(newVersion != null) { @@ -102,9 +119,16 @@ public Versioned fetchNewVersion(String versionKey, Versioned curVer } public void run() { - Versioned newStoresVersion = fetchNewVersion(STORES_VERSION_KEY, currentStoreVersion); - Versioned newClusterVersion = fetchNewVersion(CLUSTER_VERSION_KEY, - currentClusterVersion); + + // Get the properties object from the system store (containing versions) + Properties versionProps = MetadataVersionStoreUtils.getProperties(this.sysRepository.getMetadataVersionStore()); + + Long newStoresVersion = fetchNewVersion(STORES_VERSION_KEY, + currentStoreVersion, + versionProps); + Long newClusterVersion = fetchNewVersion(CLUSTER_VERSION_KEY, + currentClusterVersion, + versionProps); // If nothing has been updated, continue if((newStoresVersion != null) || (newClusterVersion != null)) { @@ -141,16 +165,18 @@ public void run() { } - public Versioned getStoreMetadataVersion() { + public Long getStoreMetadataVersion() { return this.currentStoreVersion; } - public Versioned getClusterMetadataVersion() { + public Long getClusterMetadataVersion() { return this.currentClusterVersion; } + // Fetch the latest versions for cluster and store public void updateMetadataVersions() { - this.currentStoreVersion = fetchNewVersion(STORES_VERSION_KEY, null); - this.currentClusterVersion = fetchNewVersion(CLUSTER_VERSION_KEY, null); + Properties versionProps = MetadataVersionStoreUtils.getProperties(this.sysRepository.getMetadataVersionStore()); + this.currentStoreVersion = fetchNewVersion(STORES_VERSION_KEY, null, versionProps); + this.currentClusterVersion = fetchNewVersion(CLUSTER_VERSION_KEY, null, versionProps); } } diff --git a/src/java/voldemort/client/scheduler/ClientRegistryRefresher.java b/src/java/voldemort/client/scheduler/ClientRegistryRefresher.java index 2b57926b49..03c94d0f48 100644 --- a/src/java/voldemort/client/scheduler/ClientRegistryRefresher.java +++ b/src/java/voldemort/client/scheduler/ClientRegistryRefresher.java @@ -11,12 +11,12 @@ public class ClientRegistryRefresher implements Runnable { private final Logger logger = Logger.getLogger(this.getClass()); - private final SystemStore clientRegistry; + private final SystemStore clientRegistry; private ClientInfo clientInfo; private final String clientId; private Version lastVersion; - public ClientRegistryRefresher(SystemStore clientRegistry, + public ClientRegistryRefresher(SystemStore clientRegistry, String clientId, ClientInfo clientInfo, Version version) { @@ -33,8 +33,8 @@ public void run() { + "\n" + clientInfo); try { lastVersion = clientRegistry.putSysStore(clientId, - new Versioned(clientInfo, - lastVersion)); + new Versioned(clientInfo.toString(), + lastVersion)); } catch(Exception e) { logger.warn("encounted the following error while trying to update client registry: " + e); diff --git a/src/java/voldemort/server/storage/StorageService.java b/src/java/voldemort/server/storage/StorageService.java index 0a26807baa..72cc5d0015 100644 --- a/src/java/voldemort/server/storage/StorageService.java +++ b/src/java/voldemort/server/storage/StorageService.java @@ -18,6 +18,7 @@ import static voldemort.cluster.failuredetector.FailureDetectorUtils.create; +import java.io.ByteArrayInputStream; import java.lang.management.ManagementFactory; import java.util.ArrayList; import java.util.Calendar; @@ -26,6 +27,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; @@ -63,6 +65,7 @@ import voldemort.store.StorageEngine; import voldemort.store.Store; import voldemort.store.StoreDefinition; +import voldemort.store.configuration.FileBackedCachingStorageConfiguration; import voldemort.store.invalidmetadata.InvalidMetadataCheckingStore; import voldemort.store.logging.LoggingStore; import voldemort.store.memory.InMemoryStorageConfiguration; @@ -109,6 +112,9 @@ public class StorageService extends AbstractService { private static final Logger logger = Logger.getLogger(StorageService.class.getName()); + public static final String VERSIONS_METADATA_STORE = "metadata-versions"; + public static final String CLUSTER_VERSION_KEY = "cluster.xml"; + public static final String STORES_VERSION_KEY = "stores.xml"; private final VoldemortConfig voldemortConfig; private final StoreRepository storeRepository; @@ -204,6 +210,10 @@ private void initSystemStorageConfig() { } // add FileStorage config here + if(!storageConfigs.containsKey(FileBackedCachingStorageConfiguration.TYPE_NAME)) { + storageConfigs.put(FileBackedCachingStorageConfiguration.TYPE_NAME, + new FileBackedCachingStorageConfiguration(voldemortConfig)); + } } private void initSystemStores() { @@ -220,7 +230,7 @@ private void initSystemStores() { } private void updateRepFactor(List storesDefs) { - // need impl + // need impl } @Override @@ -300,9 +310,12 @@ protected void startInner() { // now that we have all our stores, we can initialize views pointing at // those stores - for(StoreDefinition def: storeDefs) + for(StoreDefinition def: storeDefs) { if(def.isView()) openStore(def); + } + + initializeMetadataVersions(storeDefs); // enable aggregate jmx statistics if(voldemortConfig.isStatTrackingEnabled()) @@ -319,6 +332,57 @@ protected void startInner() { logger.info("All stores initialized."); } + protected void initializeMetadataVersions(List storeDefs) { + Store versionStore = storeRepository.getLocalStore(SystemStoreConstants.SystemStoreName.voldsys$_metadata_version_persistence.name()); + Properties props = new Properties(); + + try { + ByteArray metadataVersionsKey = new ByteArray(VERSIONS_METADATA_STORE.getBytes()); + List> versionList = versionStore.get(metadataVersionsKey, null); + VectorClock newClock = null; + + if(versionList != null && versionList.size() > 0) { + byte[] versionsByteArray = versionList.get(0).getValue(); + if(versionsByteArray != null) { + props.load(new ByteArrayInputStream(versionsByteArray)); + } + newClock = (VectorClock) versionList.get(0).getVersion(); + newClock = newClock.incremented(0, System.currentTimeMillis()); + } else { + newClock = new VectorClock(); + } + + // Check if version exists for cluster.xml + if(!props.containsKey(CLUSTER_VERSION_KEY)) { + props.setProperty(CLUSTER_VERSION_KEY, "0"); + } + + // Check if version exists for stores.xml + if(!props.containsKey(STORES_VERSION_KEY)) { + props.setProperty(STORES_VERSION_KEY, "0"); + } + + // Check if version exists for each store + for(StoreDefinition def: storeDefs) { + if(!props.containsKey(def.getName())) { + props.setProperty(def.getName(), "0"); + } + } + + StringBuilder finalVersionList = new StringBuilder(); + for(String propName: props.stringPropertyNames()) { + finalVersionList.append(propName + "=" + props.getProperty(propName) + "\n"); + } + System.err.println(finalVersionList); + versionStore.put(metadataVersionsKey, + new Versioned(finalVersionList.toString().getBytes(), newClock), + null); + + } catch(Exception e) { + e.printStackTrace(); + } + } + public void openSystemStore(StoreDefinition storeDef) { logger.info("Opening system store '" + storeDef.getName() + "' (" + storeDef.getType() diff --git a/src/java/voldemort/store/configuration/FileBackedCachingStorageConfiguration.java b/src/java/voldemort/store/configuration/FileBackedCachingStorageConfiguration.java new file mode 100644 index 0000000000..c43c781e3a --- /dev/null +++ b/src/java/voldemort/store/configuration/FileBackedCachingStorageConfiguration.java @@ -0,0 +1,28 @@ +package voldemort.store.configuration; + +import voldemort.server.VoldemortConfig; +import voldemort.store.StorageConfiguration; +import voldemort.store.StorageEngine; +import voldemort.utils.ByteArray; + +public class FileBackedCachingStorageConfiguration implements StorageConfiguration { + + public static final String TYPE_NAME = "file"; + private final String inputPath; + + public FileBackedCachingStorageConfiguration(VoldemortConfig config) { + this.inputPath = config.getMetadataDirectory(); + System.err.println("Created a new File backed caching engine config ..."); + } + + public StorageEngine getStore(String name) { + return new FileBackedCachingStorageEngine(name, inputPath); + } + + public String getType() { + return TYPE_NAME; + } + + public void close() {} + +} diff --git a/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java b/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java new file mode 100644 index 0000000000..bd441e80e9 --- /dev/null +++ b/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java @@ -0,0 +1,297 @@ +package voldemort.store.configuration; + +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileReader; +import java.io.FileWriter; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.commons.codec.binary.Hex; +import org.apache.commons.io.FileUtils; +import org.apache.log4j.Logger; + +import voldemort.VoldemortException; +import voldemort.store.StorageEngine; +import voldemort.store.StoreCapabilityType; +import voldemort.store.StoreUtils; +import voldemort.utils.ByteArray; +import voldemort.utils.ClosableIterator; +import voldemort.utils.Pair; +import voldemort.versioning.ObsoleteVersionException; +import voldemort.versioning.Occurred; +import voldemort.versioning.VectorClock; +import voldemort.versioning.Version; +import voldemort.versioning.Versioned; + +public class FileBackedCachingStorageEngine implements StorageEngine { + + private final static Logger logger = Logger.getLogger(FileBackedCachingStorageEngine.class); + private static final CharSequence NEW_PROPERTY_SEPARATOR = "[name="; + + private final String inputPath; + private final String inputDirectory; + private final String name; + private Map metadataMap; + private VectorClock cachedVersion = null; + + public FileBackedCachingStorageEngine(String name, String inputDirectory) { + this.name = name; + this.inputDirectory = inputDirectory; + File directory = new File(this.inputDirectory); + if(!directory.exists() && directory.canRead()) { + throw new IllegalArgumentException("Directory " + directory.getAbsolutePath() + + " does not exist or can not be read."); + } + + this.inputPath = inputDirectory + "/" + name; + this.metadataMap = new HashMap(); + this.loadData(); + logger.debug("Created a new File backed caching engine. File location = " + inputPath); + } + + private File getVersionFile() { + return new File(this.inputDirectory, this.name + ".version"); + } + + // Read the Vector clock stored in '${name}.version' file + private VectorClock readVersion() { + try { + + if(this.cachedVersion == null) { + File versionFile = getVersionFile(); + if(versionFile.exists()) { + // read the version file and return version. + String hexCode = FileUtils.readFileToString(versionFile, "UTF-8"); + this.cachedVersion = new VectorClock(Hex.decodeHex(hexCode.toCharArray())); + } + } + return this.cachedVersion; + } catch(Exception e) { + throw new VoldemortException("Failed to read Version for file :" + this.name, e); + } + } + + // Write a new Vector clock stored in '${name}.version' file + private void writeVersion(VectorClock newClock) { + File versionFile = getVersionFile(); + try { + if(!versionFile.exists() || versionFile.delete()) { + String hexCode = new String(Hex.encodeHex(newClock.toBytes())); + FileUtils.writeStringToFile(versionFile, hexCode, "UTF-8"); + this.cachedVersion = newClock; + } + } catch(Exception e) { + throw new VoldemortException("Failed to write Version for the current file :" + + this.name, e); + } + } + + private void loadData() { + + try { + BufferedReader reader = new BufferedReader(new FileReader(new File(this.inputPath))); + String line = reader.readLine(); + + while(line != null) { + if(line.contains(NEW_PROPERTY_SEPARATOR)) { + String key = null; + StringBuilder value = new StringBuilder(); + String parts[] = line.split("="); + + // Found a new property block. + // First read the key + if(parts.length == 2) { + key = parts[1].substring(0, parts[1].length() - 1); + + // Now read the value block ! + while((line = reader.readLine()) != null && line.length() != 0 + && !line.contains(NEW_PROPERTY_SEPARATOR)) { + if(value.length() == 0) { + value.append(line); + } else { + value.append("\n" + line); + } + } + + // Now add the key and value to the hashmap + this.metadataMap.put(key, value.toString()); + } + } else { + line = reader.readLine(); + } + } + } catch(FileNotFoundException e) { + logger.debug("File used for persistence does not exist !!"); + } catch(IOException e) { + logger.debug("Error in flushing data to file : " + e); + } + } + + // Flush the in-memory data to the file + private synchronized void flushData() { + BufferedWriter writer = null; + try { + writer = new BufferedWriter(new FileWriter(new File(this.inputPath))); + for(String key: this.metadataMap.keySet()) { + writer.write(NEW_PROPERTY_SEPARATOR + key.toString() + "]\n"); + writer.write(this.metadataMap.get(key).toString()); + writer.write("\n\n"); + } + writer.flush(); + } catch(IOException e) { + logger.error("IO exception while flushing data to file backed storage: " + + e.getMessage()); + } + + try { + if(writer != null) + writer.close(); + } catch(Exception e) { + logger.error("Error while flushing data to file backed storage: " + e.getMessage()); + } + } + + public String getName() { + return this.name; + } + + public void close() throws VoldemortException {} + + public Object getCapability(StoreCapabilityType capability) { + throw new VoldemortException("No extra capability."); + } + + public ClosableIterator>> entries() { + throw new VoldemortException("Iteration not supported in FileBackedCachingStorageEngine"); + } + + public ClosableIterator keys() { + throw new VoldemortException("Keys iteration not supported in FileBackedCachingStorageEngine"); + } + + public void truncate() { + throw new VoldemortException("Truncate not supported in FileBackedCachingStorageEngine"); + } + + public boolean isPartitionAware() { + return false; + } + + // Assigning new Vector clock here: TODO: Decide what vector clock to use ? + public List> get(ByteArray key, byte[] transforms) throws VoldemortException { + StoreUtils.assertValidKey(key); + String keyString = new String(key.get()); + List> found = new ArrayList>(); + byte[] resultBytes = null; + String value = this.metadataMap.get(keyString); + if(value != null) { + resultBytes = value.getBytes(); + found.add(new Versioned(resultBytes, readVersion())); + } + return found; + } + + public Map>> getAll(Iterable keys, + Map transforms) + throws VoldemortException { + StoreUtils.assertValidKeys(keys); + Map>> result = StoreUtils.newEmptyHashMap(keys); + for(ByteArray key: keys) { + List> values = get(key, null); + if(!values.isEmpty()) + result.put(key, values); + } + return result; + } + + public List getVersions(ByteArray key) { + List> values = get(key, null); + List versions = new ArrayList(values.size()); + for(Versioned value: values) { + versions.add(value.getVersion()); + } + return versions; + } + + public void put(ByteArray key, Versioned value, byte[] transforms) + throws VoldemortException { + StoreUtils.assertValidKey(key); + + // Validate the Vector clock + VectorClock clock = readVersion(); + if(clock != null) { + if(value.getVersion().compare(clock) == Occurred.BEFORE) { + throw new ObsoleteVersionException("A successor version " + clock + " to this " + + value.getVersion() + + " exists for the current file : " + this.name); + } else if(value.getVersion().compare(clock) == Occurred.CONCURRENTLY) { + throw new ObsoleteVersionException("Concurrent Operation not allowed on Metadata."); + } + } + + // Update the cache copy + this.metadataMap.put(new String(key.get()), new String(value.getValue())); + + // Flush the data to the file + this.flushData(); + + // Persist the new Vector clock + writeVersion((VectorClock) value.getVersion()); + } + + public boolean delete(ByteArray key, Version version) throws VoldemortException { + boolean deleteSuccessful = false; + StoreUtils.assertValidKey(key); + String keyString = new String(key.get()); + String initialValue = this.metadataMap.get(keyString); + if(initialValue != null) { + String removedValue = this.metadataMap.remove(keyString); + if(removedValue != null) { + deleteSuccessful = (initialValue.equals(removedValue)); + } + } + if(deleteSuccessful) { + this.flushData(); + } + return deleteSuccessful; + } + + // public static void main(String[] args) { + // FileBackedCachingStorageEngine engine = new + // FileBackedCachingStorageEngine("metadata-versions-stores", + // "."); + // ByteArray key = new ByteArray("property_name".getBytes()); + // String result = new String(engine.get(key, null).get(0).getValue()); + // System.out.println("Received: \n" + result); + // + // key = new ByteArray("versions".getBytes()); + // result = new String(engine.get(key, null).get(0).getValue()); + // System.out.println("Received: \n" + result); + // + // key = new ByteArray("ame".getBytes()); + // result = ""; + // byte[] res = engine.get(key, null).get(0).getValue(); + // if(res != null) { + // result = new String(res); + // } + // System.out.println("Received: \n" + result); + // + // key = new ByteArray("client-config".getBytes()); + // byte[] value = "selectors=10".getBytes(); + // engine.put(key, new Versioned(value), null); + // + // key = new ByteArray("client-config".getBytes()); + // result = new String(engine.get(key, null).get(0).getValue()); + // System.out.println("Received: \n" + result); + // + // key = new ByteArray("my_awesome_store".getBytes()); + // value = "cluster.xml=22\nmy_awesome_store=4".getBytes(); + // engine.put(key, new Versioned(value), null); + // } +} diff --git a/src/java/voldemort/store/system/SystemStoreConstants.java b/src/java/voldemort/store/system/SystemStoreConstants.java index c818ed51fd..0c9799e60e 100644 --- a/src/java/voldemort/store/system/SystemStoreConstants.java +++ b/src/java/voldemort/store/system/SystemStoreConstants.java @@ -16,8 +16,8 @@ public class SystemStoreConstants { public static enum SystemStoreName { voldsys$_client_registry, - voldsys$_client_store_definition, - voldsys$_metadata_version; + voldsys$_metadata_version, + voldsys$_metadata_version_persistence; } public static final String SYSTEM_STORE_SCHEMA = "" @@ -39,18 +39,14 @@ public static enum SystemStoreName { + " utf8" + " " + " " - // + - // " avro-specific" - // + - // " java=voldemort.client.ClientInfo" - + " java-serialization" + + " string" + " " + " 7" + " " + " " - + " voldsys$_client_store_definition" - + " zone-routing" + + " voldsys$_metadata_version" + + " local-pref-all-routing" + " proximity-handoff" + " memory" + " client" @@ -63,14 +59,13 @@ public static enum SystemStoreName { + " " + " string" + " " - + " 7" + " " + " " - + " voldsys$_metadata_version" + + " voldsys$_metadata_version_persistence" + " local-pref-all-routing" + " proximity-handoff" - + " memory" + + " file" + " client" + " 1" + " 1" @@ -79,7 +74,7 @@ public static enum SystemStoreName { + " string" + " " + " " - + " java-serialization" + + " string" + " " + " " + ""; diff --git a/src/java/voldemort/utils/MetadataVersionStoreUtils.java b/src/java/voldemort/utils/MetadataVersionStoreUtils.java new file mode 100644 index 0000000000..1322bb0558 --- /dev/null +++ b/src/java/voldemort/utils/MetadataVersionStoreUtils.java @@ -0,0 +1,45 @@ +package voldemort.utils; + +import java.io.ByteArrayInputStream; +import java.util.Properties; + +import org.apache.log4j.Logger; + +import voldemort.client.SystemStore; + +public class MetadataVersionStoreUtils { + + public static final String VERSIONS_METADATA_STORE = "metadata-versions"; + private final static Logger logger = Logger.getLogger(MetadataVersionStoreUtils.class); + + public static Properties getProperties(SystemStore versionStore) { + Properties props = null; + try { + String versionList = versionStore.getSysStore(VERSIONS_METADATA_STORE).getValue(); + + if(versionList != null) { + props = new Properties(); + props.load(new ByteArrayInputStream(versionList.getBytes())); + } + } catch(Exception e) { + logger.debug("Got exception in getting properties : " + e.getMessage()); + } + + return props; + } + + public static void setProperties(SystemStore versionStore, Properties props) { + StringBuilder finalVersionList = new StringBuilder(); + for(String propName: props.stringPropertyNames()) { + if(finalVersionList.length() == 0) { + finalVersionList.append(propName + "=" + props.getProperty(propName)); + } else { + finalVersionList.append("\n" + propName + "=" + props.getProperty(propName)); + } + } + + System.err.println(finalVersionList); + + versionStore.putSysStore(VERSIONS_METADATA_STORE, finalVersionList.toString()); + } +} diff --git a/test/unit/voldemort/client/ClientJmxTest.java b/test/unit/voldemort/client/ClientJmxTest.java index 1c215580f8..2c0da2ddb1 100644 --- a/test/unit/voldemort/client/ClientJmxTest.java +++ b/test/unit/voldemort/client/ClientJmxTest.java @@ -105,13 +105,13 @@ public void testTwoClientContextOnJmx() throws Exception { c1Name = JmxUtils.createObjectName(STATS_DOMAIN, c1type + "." - + ((DefaultStoreClient) c1).getClientId() - .toString()); + + ((ZenStoreClient) c1).getClientId() + .toString()); c2Name = JmxUtils.createObjectName(STATS_DOMAIN, c2type + "." - + ((DefaultStoreClient) c2).getClientId() - .toString()); + + ((ZenStoreClient) c2).getClientId() + .toString()); checkForMbeanFound(c1Name); checkForMbeanFound(c2Name); mbServer.unregisterMBean(c1Name); @@ -147,13 +147,13 @@ public void testSameContextOnJmx() throws Exception { c1Name = JmxUtils.createObjectName(STATS_DOMAIN, c1type + "." - + ((DefaultStoreClient) clients[0]).getClientId() - .toString()); + + ((ZenStoreClient) clients[0]).getClientId() + .toString()); c2Name = JmxUtils.createObjectName(STATS_DOMAIN, c2type + "." - + ((DefaultStoreClient) clients[1]).getClientId() - .toString()); + + ((ZenStoreClient) clients[1]).getClientId() + .toString()); checkForMbeanFound(c1Name); checkForMbeanFound(c2Name); mbServer.unregisterMBean(c1Name); @@ -187,13 +187,13 @@ public void testTwoClientNoContextOnJmx() throws Exception { c1Name = JmxUtils.createObjectName(STATS_DOMAIN, c1type + "." - + ((DefaultStoreClient) c1).getClientId() - .toString()); + + ((ZenStoreClient) c1).getClientId() + .toString()); c2Name = JmxUtils.createObjectName(STATS_DOMAIN, c2type + "." - + ((DefaultStoreClient) c2).getClientId() - .toString()); + + ((ZenStoreClient) c2).getClientId() + .toString()); checkForMbeanFound(c1Name); checkForMbeanFound(c2Name); mbServer.unregisterMBean(c1Name); @@ -227,13 +227,13 @@ public void testTwoClientNullContextOnJmx() throws Exception { c1Name = JmxUtils.createObjectName(STATS_DOMAIN, c1type + "." - + ((DefaultStoreClient) c1).getClientId() - .toString()); + + ((ZenStoreClient) c1).getClientId() + .toString()); c2Name = JmxUtils.createObjectName(STATS_DOMAIN, c2type + "." - + ((DefaultStoreClient) c2).getClientId() - .toString()); + + ((ZenStoreClient) c2).getClientId() + .toString()); checkForMbeanFound(c1Name); checkForMbeanFound(c2Name); mbServer.unregisterMBean(c1Name); @@ -263,13 +263,13 @@ public void testSameContextAndFactory() throws Exception { ObjectName c1Name = JmxUtils.createObjectName(STATS_DOMAIN, ctype + "." - + ((DefaultStoreClient) clients[0]).getClientId() - .toString()); + + ((ZenStoreClient) clients[0]).getClientId() + .toString()); ObjectName c2Name = JmxUtils.createObjectName(STATS_DOMAIN, ctype + "." - + ((DefaultStoreClient) clients[1]).getClientId() - .toString()); + + ((ZenStoreClient) clients[1]).getClientId() + .toString()); checkForMbeanFound(c1Name); checkForMbeanFound(c2Name); mbServer.unregisterMBean(c1Name); @@ -297,13 +297,13 @@ public void testDifferentId() throws Exception { ObjectName c1Name = JmxUtils.createObjectName(STATS_DOMAIN, ctype + "." - + ((DefaultStoreClient) clients[0]).getClientId() - .toString()); + + ((ZenStoreClient) clients[0]).getClientId() + .toString()); ObjectName c2Name = JmxUtils.createObjectName(STATS_DOMAIN, ctype + "." - + ((DefaultStoreClient) clients[1]).getClientId() - .toString()); + + ((ZenStoreClient) clients[1]).getClientId() + .toString()); checkForMbeanFound(c1Name); checkForMbeanFound(c2Name); assertTrue(!c1Name.equals(c2Name)); diff --git a/test/unit/voldemort/client/ClientRegistryTest.java b/test/unit/voldemort/client/ClientRegistryTest.java index 7fe43d17ee..1dc55f6636 100644 --- a/test/unit/voldemort/client/ClientRegistryTest.java +++ b/test/unit/voldemort/client/ClientRegistryTest.java @@ -96,7 +96,7 @@ public void testHappyPath() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[0]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); StoreClient client1 = socketFactory.getStoreClient(TEST_STORE_NAME); @@ -143,7 +143,7 @@ public void testHappyPath() { infoList = getClientRegistryContent(it); assertTrue("Client registry not updated.", infoList.get(0).getBootstrapTime() < infoList.get(0).getUpdateTime()); - ((DefaultStoreClient) client1).close(); + ((ZenStoreClient) client1).close(); } @Test @@ -155,7 +155,7 @@ public void testTwoClients() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[0]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); StoreClient client1 = socketFactory.getStoreClient(TEST_STORE_NAME); @@ -227,8 +227,8 @@ public void testTwoClients() { assertTrue("Client registry not updated.", infoList.get(1).getBootstrapTime() < infoList.get(1).getUpdateTime()); - ((DefaultStoreClient) client1).close(); - ((DefaultStoreClient) client2).close(); + ((ZenStoreClient) client1).close(); + ((ZenStoreClient) client2).close(); } @Test @@ -240,7 +240,7 @@ public void testTwoStores() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[0]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); StoreClient client1 = socketFactory.getStoreClient(TEST_STORE_NAME); @@ -329,8 +329,8 @@ public void testTwoStores() { assertTrue("Client registry not updated.", infoList.get(1).getBootstrapTime() < infoList.get(1).getUpdateTime()); - ((DefaultStoreClient) client1).close(); - ((DefaultStoreClient) client2).close(); + ((ZenStoreClient) client1).close(); + ((ZenStoreClient) client2).close(); } @Test @@ -342,7 +342,7 @@ public void testTwoFactories() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[0]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory1 = new SocketStoreClientFactory(clientConfig); @@ -352,7 +352,7 @@ public void testTwoFactories() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[0]) .setClientContextName(CLIENT_CONTEXT_NAME2) - .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory2 = new SocketStoreClientFactory(clientConfig2); @@ -462,8 +462,8 @@ public void testTwoFactories() { assertTrue("Client registry not updated.", infoList.get(1).getBootstrapTime() < infoList.get(1).getUpdateTime()); - ((DefaultStoreClient) client1).close(); - ((DefaultStoreClient) client2).close(); + ((ZenStoreClient) client1).close(); + ((ZenStoreClient) client2).close(); } @Test @@ -478,7 +478,7 @@ public void testOneServerFailre() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[1]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory1 = new SocketStoreClientFactory(clientConfig); @@ -488,7 +488,7 @@ public void testOneServerFailre() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[1]) .setClientContextName(CLIENT_CONTEXT_NAME2) - .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory2 = new SocketStoreClientFactory(clientConfig2); @@ -556,8 +556,8 @@ public void testOneServerFailre() { assertTrue("Client registry not updated.", infoList.get(1).getBootstrapTime() < infoList.get(1).getUpdateTime()); - ((DefaultStoreClient) client1).close(); - ((DefaultStoreClient) client2).close(); + ((ZenStoreClient) client1).close(); + ((ZenStoreClient) client2).close(); } @Test @@ -570,7 +570,7 @@ public void testRepeatRegistrationSameFactory() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[1]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory1 = new SocketStoreClientFactory(clientConfig); @@ -580,7 +580,7 @@ public void testRepeatRegistrationSameFactory() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[1]) .setClientContextName(CLIENT_CONTEXT_NAME2) - .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory2 = new SocketStoreClientFactory(clientConfig2); @@ -592,8 +592,8 @@ public void testRepeatRegistrationSameFactory() { client1.put("k1", "v1"); client2.put("k2", "v2"); - ((DefaultStoreClient) client1).close(); - ((DefaultStoreClient) client2).close(); + ((ZenStoreClient) client1).close(); + ((ZenStoreClient) client2).close(); } Iterator>> it = adminClient.fetchEntries(1, @@ -619,7 +619,7 @@ public void testRepeatRegistrationDifferentFactories() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[1]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory1 = new SocketStoreClientFactory(clientConfig); @@ -629,7 +629,7 @@ public void testRepeatRegistrationDifferentFactories() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[1]) .setClientContextName(CLIENT_CONTEXT_NAME2) - .setClientRegistryRefreshInterval(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory2 = new SocketStoreClientFactory(clientConfig2); @@ -709,8 +709,8 @@ public void testRepeatRegistrationDifferentFactories() { client1LastBootstrapTime = infoList.get(0).getBootstrapTime(); client2LastBootstrapTime = infoList.get(0).getBootstrapTime(); - ((DefaultStoreClient) client1).close(); - ((DefaultStoreClient) client2).close(); + ((ZenStoreClient) client1).close(); + ((ZenStoreClient) client2).close(); } } diff --git a/test/unit/voldemort/client/EndToEndRebootstrapTest.java b/test/unit/voldemort/client/EndToEndRebootstrapTest.java new file mode 100644 index 0000000000..dc3d52a407 --- /dev/null +++ b/test/unit/voldemort/client/EndToEndRebootstrapTest.java @@ -0,0 +1,224 @@ +package voldemort.client; + +import java.io.IOException; +import java.io.StringWriter; +import java.util.Iterator; +import java.util.Properties; + +import junit.framework.TestCase; + +import org.codehaus.jackson.JsonFactory; +import org.codehaus.jackson.JsonGenerator; +import org.codehaus.jackson.map.ObjectMapper; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import voldemort.ServerTestUtils; +import voldemort.TestUtils; +import voldemort.VoldemortAdminTool; +import voldemort.VoldemortException; +import voldemort.client.protocol.admin.AdminClient; +import voldemort.client.protocol.admin.AdminClientConfig; +import voldemort.cluster.Cluster; +import voldemort.cluster.Node; +import voldemort.serialization.DefaultSerializerFactory; +import voldemort.serialization.Serializer; +import voldemort.serialization.SerializerDefinition; +import voldemort.serialization.SerializerFactory; +import voldemort.server.VoldemortServer; +import voldemort.store.StoreDefinition; +import voldemort.store.compress.CompressionStrategy; +import voldemort.store.compress.CompressionStrategyFactory; +import voldemort.store.socket.SocketStoreFactory; +import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; +import voldemort.store.system.SystemStoreConstants; +import voldemort.utils.ByteArray; +import voldemort.utils.Pair; +import voldemort.versioning.VectorClock; +import voldemort.versioning.Versioned; +import voldemort.xml.ClusterMapper; + +public class EndToEndRebootstrapTest extends TestCase { + + private static final String STORE_NAME = "test-replication-persistent"; + private static final String CLUSTER_KEY = "cluster.xml"; + private static String storesXmlfile = "test/common/voldemort/config/stores.xml"; + String[] bootStrapUrls = null; + private SocketStoreFactory socketStoreFactory = new ClientRequestExecutorPool(2, + 10000, + 100000, + 32 * 1024); + + private VoldemortServer[] servers; + private StoreClient storeClient; + SystemStore sysStoreVersion; + private Cluster cluster; + public static String socketUrl = ""; + protected final int CLIENT_ZONE_ID = 0; + + @Override + @Before + public void setUp() throws Exception { + cluster = ServerTestUtils.getLocalCluster(2, new int[][] { { 0, 1, 2, 3 }, { 4, 5, 6, 7 } }); + servers = new VoldemortServer[2]; + servers[0] = ServerTestUtils.startVoldemortServer(socketStoreFactory, + ServerTestUtils.createServerConfig(true, + 0, + TestUtils.createTempDir() + .getAbsolutePath(), + null, + storesXmlfile, + new Properties()), + cluster); + servers[1] = ServerTestUtils.startVoldemortServer(socketStoreFactory, + ServerTestUtils.createServerConfig(true, + 1, + TestUtils.createTempDir() + .getAbsolutePath(), + null, + storesXmlfile, + new Properties()), + cluster); + + socketUrl = servers[0].getIdentityNode().getSocketUrl().toString(); + bootStrapUrls = new String[1]; + bootStrapUrls[0] = socketUrl; + + Node node = cluster.getNodeById(0); + String bootstrapUrl = "tcp://" + node.getHost() + ":" + node.getSocketPort(); + ClientConfig clientConfig = new ClientConfig(); + clientConfig.setClientRegistryUpdateInSecs(5); + clientConfig.setAsyncMetadataRefreshInMs(5000); + clientConfig.setBootstrapUrls(bootstrapUrl); + StoreClientFactory storeClientFactory = new SocketStoreClientFactory(clientConfig); + storeClient = storeClientFactory.getStoreClient(STORE_NAME); + sysStoreVersion = new SystemStore(SystemStoreConstants.SystemStoreName.voldsys$_metadata_version_persistence.name(), + bootStrapUrls, + 0); + + } + + @Override + @After + public void tearDown() throws Exception { + servers[0].stop(); + servers[1].stop(); + } + + private String getClientInfo(AdminClient adminClient, + String storeName, + StoreDefinition storeDefinition) throws IOException { + Iterator>> iterator = adminClient.fetchEntries(0, + storeName, + cluster.getNodeById(0) + .getPartitionIds(), + null, + true); + + boolean hasData = iterator.hasNext(); + assertTrue(hasData); + + String clientInfo = null; + CompressionStrategy valueCompressionStrategy = null; + + SerializerFactory serializerFactory = new DefaultSerializerFactory(); + StringWriter stringWriter = new StringWriter(); + JsonGenerator generator = new JsonFactory(new ObjectMapper()).createJsonGenerator(stringWriter); + + SerializerDefinition valueSerializerDef = storeDefinition.getValueSerializer(); + if(null != valueSerializerDef && valueSerializerDef.hasCompression()) { + valueCompressionStrategy = new CompressionStrategyFactory().get(valueSerializerDef.getCompression()); + } + + @SuppressWarnings("unchecked") + Serializer valueSerializer = (Serializer) serializerFactory.getSerializer(storeDefinition.getValueSerializer()); + + try { + + Pair> kvPair = iterator.next(); + VectorClock version = (VectorClock) kvPair.getSecond().getVersion(); + byte[] valueBytes = kvPair.getSecond().getValue(); + + Object valueObject = valueSerializer.toObject((null == valueCompressionStrategy) ? valueBytes + : valueCompressionStrategy.inflate(valueBytes)); + + generator.writeObject(valueObject); + + StringBuffer buf = stringWriter.getBuffer(); + clientInfo = buf.toString(); + buf.setLength(0); + } finally { + + } + return clientInfo; + } + + @Test + public void testEndToEndRebootstrap() { + try { + // Do a sample get, put to check client is correctly initialized. + String key = "city"; + String value = "SF"; + String storeName = SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(); + String bootstrapTime = ""; + String newBootstrapTime = ""; + AdminClient adminClient = new AdminClient(bootStrapUrls[0], new AdminClientConfig()); + + try { + storeClient.put(key, value); + String received = storeClient.getValue(key); + assertEquals(value, received); + } catch(VoldemortException ve) { + fail("Error in doing basic get, put"); + } + + // Track the original bootstrap timestamp published by client + String originalClientInfo = getClientInfo(adminClient, + storeName, + SystemStoreConstants.getSystemStoreDef(storeName)); + try { + bootstrapTime = originalClientInfo.split("]")[0].split("\\[")[1]; + } catch(Exception e) { + fail("Error in retrieving bootstrap time: " + e); + } + + // Update cluster.xml metadata + VoldemortAdminTool adminTool = new VoldemortAdminTool(); + ClusterMapper mapper = new ClusterMapper(); + for(Node node: cluster.getNodes()) { + VoldemortAdminTool.executeSetMetadata(node.getId(), + adminClient, + CLUSTER_KEY, + mapper.writeCluster(cluster)); + VoldemortAdminTool.updateMetadataversion(CLUSTER_KEY, sysStoreVersion); + + } + + // Wait for about 15 seconds to be sure + try { + Thread.sleep(15000); + } catch(Exception e) { + fail("Interrupted ."); + } + + // Retrieve the new client bootstrap timestamp + String newClientInfo = getClientInfo(adminClient, + storeName, + SystemStoreConstants.getSystemStoreDef(storeName)); + try { + newBootstrapTime = newClientInfo.split("]")[0].split("\\[")[1]; + } catch(Exception e) { + fail("Error in retrieving bootstrap time: " + e); + } + + assertFalse(bootstrapTime.equals(newBootstrapTime)); + long origTime = Long.parseLong(bootstrapTime); + long newTime = Long.parseLong(newBootstrapTime); + assertTrue(newTime > origTime); + + } catch(Exception e) { + fail("Error in validating end to end client rebootstrap : " + e); + } + } +} diff --git a/test/unit/voldemort/server/storage/StorageServiceTest.java b/test/unit/voldemort/server/storage/StorageServiceTest.java index f3fcf9d4c0..9cd5ace29f 100644 --- a/test/unit/voldemort/server/storage/StorageServiceTest.java +++ b/test/unit/voldemort/server/storage/StorageServiceTest.java @@ -1,7 +1,9 @@ package voldemort.server.storage; +import java.io.ByteArrayInputStream; import java.io.File; import java.util.List; +import java.util.Properties; import junit.framework.TestCase; import voldemort.MockTime; @@ -11,8 +13,12 @@ import voldemort.common.service.SchedulerService; import voldemort.server.StoreRepository; import voldemort.server.VoldemortConfig; +import voldemort.store.Store; import voldemort.store.StoreDefinition; import voldemort.store.metadata.MetadataStore; +import voldemort.store.system.SystemStoreConstants; +import voldemort.utils.ByteArray; +import voldemort.versioning.Versioned; /** * Test that the storage service is able to load all stores. @@ -61,4 +67,53 @@ public void testStores() { } } } + + public void testMetadataVersionsInit() { + Store versionStore = storeRepository.getLocalStore(SystemStoreConstants.SystemStoreName.voldsys$_metadata_version_persistence.name()); + Properties props = new Properties(); + + try { + ByteArray metadataVersionsKey = new ByteArray(StorageService.VERSIONS_METADATA_STORE.getBytes()); + List> versionList = versionStore.get(metadataVersionsKey, null); + + if(versionList != null && versionList.size() > 0) { + byte[] versionsByteArray = versionList.get(0).getValue(); + if(versionsByteArray != null) { + props.load(new ByteArrayInputStream(versionsByteArray)); + } else { + fail("Illegal value returned for metadata key: " + + StorageService.VERSIONS_METADATA_STORE); + } + } else { + fail("Illegal value returned for metadata key: " + + StorageService.VERSIONS_METADATA_STORE); + } + + // Check if version exists for cluster.xml + if(!props.containsKey(StorageService.CLUSTER_VERSION_KEY)) { + fail(StorageService.CLUSTER_VERSION_KEY + " not present in " + + StorageService.VERSIONS_METADATA_STORE); + } + + // Check if version exists for stores.xml + if(!props.containsKey(StorageService.STORES_VERSION_KEY)) { + fail(StorageService.STORES_VERSION_KEY + " not present in " + + StorageService.VERSIONS_METADATA_STORE); + } + + // Check if version exists for each store + for(StoreDefinition def: storeDefs) { + if(!props.containsKey(def.getName())) { + fail(def.getName() + " store not present in " + + StorageService.VERSIONS_METADATA_STORE); + } + } + } catch(Exception e) { + fail("Error in retrieving : " + + StorageService.VERSIONS_METADATA_STORE + + " key from " + + SystemStoreConstants.SystemStoreName.voldsys$_metadata_version_persistence.name() + + " store. "); + } + } } diff --git a/test/unit/voldemort/store/configuration/FileBackedCachingStorageEngineTest.java b/test/unit/voldemort/store/configuration/FileBackedCachingStorageEngineTest.java new file mode 100644 index 0000000000..46186d28fa --- /dev/null +++ b/test/unit/voldemort/store/configuration/FileBackedCachingStorageEngineTest.java @@ -0,0 +1,147 @@ +package voldemort.store.configuration; + +import static voldemort.TestUtils.getClock; + +import java.io.File; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import org.apache.commons.io.FileDeleteStrategy; +import org.junit.Test; + +import voldemort.TestUtils; +import voldemort.VoldemortException; +import voldemort.store.AbstractStoreTest; +import voldemort.store.Store; +import voldemort.utils.ByteArray; +import voldemort.versioning.VectorClock; +import voldemort.versioning.Versioned; + +public class FileBackedCachingStorageEngineTest extends + AbstractStoreTest { + + private File tempDir; + + @Override + public void setUp() throws Exception { + super.setUp(); + if(null != tempDir && tempDir.exists()) + FileDeleteStrategy.FORCE.delete(tempDir); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + if(null != tempDir && tempDir.exists()) + FileDeleteStrategy.FORCE.delete(tempDir); + } + + /* + * Calling getStrings to make it readable (easier debugging) + */ + @Override + public List getKeys(int numKeys) { + List keyList = getStrings(numKeys, 10); + List byteArrayKeyList = new ArrayList(); + for(String s: keyList) { + byteArrayKeyList.add(new ByteArray(s.getBytes())); + } + return byteArrayKeyList; + } + + @Override + public Store getStore() { + if(null == tempDir || !tempDir.exists()) { + tempDir = TestUtils.createTempDir(); + } + return new FileBackedCachingStorageEngine("file-backed-test", tempDir.getAbsolutePath()); + } + + @Override + protected boolean allowConcurrentOperations() { + return false; + } + + @Override + protected boolean valuesEqual(byte[] t1, byte[] t2) { + return Arrays.equals(t1, t2); + } + + @Override + public List getValues(int numValues) { + List keyList = getStrings(numValues, 10); + List byteArrayKeyList = new ArrayList(); + for(String s: keyList) { + byteArrayKeyList.add(s.getBytes()); + } + return byteArrayKeyList; + } + + @Override + public void testDelete() { + ByteArray key = getKey(); + Store store = getStore(); + VectorClock c1 = getClock(1, 1); + byte[] value = getValue(); + + // can't delete something that isn't there + assertTrue(!store.delete(key, c1)); + + store.put(key, new Versioned(value, c1), null); + assertEquals(1, store.get(key, null).size()); + + // now delete that version too + assertTrue("Delete failed!", store.delete(key, c1)); + assertEquals(0, store.get(key, null).size()); + } + + @Override + @Test + public void testGetAll() throws Exception { + Store store = getStore(); + int putCount = 10; + List keys = getKeys(putCount); + List values = getValues(putCount); + assertEquals(putCount, values.size()); + VectorClock clock = new VectorClock(); + for(int i = 0; i < putCount; i++) { + store.put(keys.get(i), new Versioned(values.get(i), clock), null); + clock = clock.incremented(0, System.currentTimeMillis()); + } + + int countForGet = putCount / 2; + List keysForGet = keys.subList(0, countForGet); + List valuesForGet = values.subList(0, countForGet); + Map>> result = store.getAll(keysForGet, null); + assertEquals(countForGet, result.size()); + for(int i = 0; i < keysForGet.size(); ++i) { + ByteArray key = keysForGet.get(i); + byte[] expectedValue = valuesForGet.get(i); + List> versioneds = result.get(key); + assertGetAllValues(expectedValue, versioneds); + } + } + + @Test + public void testConcurrentWriteFailure() { + ByteArray key = getKey(); + Store store = getStore(); + VectorClock c1 = getClock(1, 1); + VectorClock c2 = getClock(1, 2); + byte[] value = getValue(); + + // put two conflicting versions, then delete one + Versioned v1 = new Versioned(value, c1); + Versioned v2 = new Versioned(value, c2); + store.put(key, v1, null); + try { + store.put(key, v2, null); + fail("Concurrent write succeeded in FileBackedCachingStorageEngine. Should not be allowed."); + } catch(VoldemortException ve) { + // This is OK + } + } + +} diff --git a/test/unit/voldemort/store/system/AsyncMetadataVersionManagerTest.java b/test/unit/voldemort/store/system/AsyncMetadataVersionManagerTest.java index 650d10b1e4..564c613af1 100644 --- a/test/unit/voldemort/store/system/AsyncMetadataVersionManagerTest.java +++ b/test/unit/voldemort/store/system/AsyncMetadataVersionManagerTest.java @@ -22,13 +22,11 @@ import voldemort.store.socket.SocketStoreFactory; import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; import voldemort.utils.SystemTime; -import voldemort.versioning.Versioned; public class AsyncMetadataVersionManagerTest { private static String storesXmlfile = "test/common/voldemort/config/stores.xml"; String[] bootStrapUrls = null; - private String clusterXml; private SocketStoreFactory socketStoreFactory = new ClientRequestExecutorPool(2, 10000, 100000, @@ -40,7 +38,7 @@ public class AsyncMetadataVersionManagerTest { protected final int CLIENT_ZONE_ID = 0; private long newVersion = 0; - private SystemStore sysVersionStore; + private SystemStore sysVersionStore; private SystemStoreRepository repository; private SchedulerService scheduler; private AsyncMetadataVersionManager asyncCheckMetadata; @@ -75,12 +73,12 @@ public void setUp() throws Exception { bootStrapUrls = new String[1]; bootStrapUrls[0] = socketUrl; - sysVersionStore = new SystemStore(SystemStoreConstants.SystemStoreName.voldsys$_metadata_version.name(), - bootStrapUrls, - this.CLIENT_ZONE_ID); + sysVersionStore = new SystemStore(SystemStoreConstants.SystemStoreName.voldsys$_metadata_version_persistence.name(), + bootStrapUrls, + this.CLIENT_ZONE_ID); repository = new SystemStoreRepository(); repository.addSystemStore(sysVersionStore, - SystemStoreConstants.SystemStoreName.voldsys$_metadata_version.name()); + SystemStoreConstants.SystemStoreName.voldsys$_metadata_version_persistence.name()); this.scheduler = new SchedulerService(2, SystemTime.INSTANCE, true); } @@ -92,6 +90,7 @@ public void tearDown() throws Exception { @Test public void testBasicAsyncBehaviour() { + String storeVersionKey = "stores.xml"; try { Callable rebootstrapCallback = new Callable() { @@ -102,14 +101,19 @@ public Void call() throws Exception { }; // Write a base version of 100 - this.sysVersionStore.putSysStore(AsyncMetadataVersionManager.STORES_VERSION_KEY, 100l); + String existingVersions = this.sysVersionStore.getSysStore(AsyncMetadataVersionManager.VERSIONS_METADATA_STORE) + .getValue(); + existingVersions += storeVersionKey + "=100"; + this.sysVersionStore.putSysStore(AsyncMetadataVersionManager.VERSIONS_METADATA_STORE, + existingVersions); // Giving enough time to complete the above put. Thread.sleep(500); // Starting the Version Metadata Manager this.asyncCheckMetadata = new AsyncMetadataVersionManager(this.repository, - rebootstrapCallback); + rebootstrapCallback, + null); scheduler.schedule(asyncCheckMetadata.getClass().getName(), asyncCheckMetadata, new Date(), @@ -125,8 +129,13 @@ public Void call() throws Exception { // Updating the version metadata here for the Version Metadata // Manager to detect this.newVersion = 101; - this.sysVersionStore.putSysStore(AsyncMetadataVersionManager.STORES_VERSION_KEY, - this.newVersion); + System.err.println("Incrementing the version for : " + storeVersionKey); + existingVersions = this.sysVersionStore.getSysStore(AsyncMetadataVersionManager.VERSIONS_METADATA_STORE) + .getValue(); + existingVersions = existingVersions.replaceAll(storeVersionKey + "=100", + storeVersionKey + "=101"); + this.sysVersionStore.putSysStore(AsyncMetadataVersionManager.VERSIONS_METADATA_STORE, + existingVersions); maxRetries = 0; while(maxRetries < 3 && !callbackDone) { @@ -143,9 +152,9 @@ public Void call() throws Exception { private void callback() { try { - Versioned storeVersion = this.asyncCheckMetadata.getStoreMetadataVersion(); + Long storeVersion = this.asyncCheckMetadata.getStoreMetadataVersion(); if(storeVersion != null) { - this.updatedStoresVersion = storeVersion.getValue(); + this.updatedStoresVersion = storeVersion; } } catch(Exception e) { fail("Error in updating stores.xml version: " + e.getMessage()); From ec190a098e865e45c79469cbe903644466f2e2a9 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Tue, 7 Aug 2012 15:14:20 -0700 Subject: [PATCH 108/209] Added CNAME for github docs to accept project-voldemort.com --- CNAME | 1 + 1 file changed, 1 insertion(+) create mode 100644 CNAME diff --git a/CNAME b/CNAME new file mode 100644 index 0000000000..7c45ce77e1 --- /dev/null +++ b/CNAME @@ -0,0 +1 @@ +project-voldemort.com From 3feca7263ecbf57db3f254a7620a84c59e43a964 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Tue, 7 Aug 2012 16:28:24 -0700 Subject: [PATCH 109/209] Changed CNAME file for test --- CNAME | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CNAME b/CNAME index 7c45ce77e1..31eeccbbb3 100644 --- a/CNAME +++ b/CNAME @@ -1 +1 @@ -project-voldemort.com +beta.project-voldemort.com From 98643ea18481ebfbb0fe1a7575ae6157f84f939e Mon Sep 17 00:00:00 2001 From: Peter Bailis Date: Wed, 8 Aug 2012 18:25:06 -0700 Subject: [PATCH 110/209] Improving debug messages in request tracing --- .../server/niosocket/AsyncRequestHandler.java | 11 ++-- .../vold/VoldemortNativeRequestHandler.java | 34 ++++++----- .../voldemort/store/bdb/BdbStorageEngine.java | 10 ++-- .../store/routed/PipelineRoutedStore.java | 45 ++++++++------ .../routed/action/AbstractReadRepair.java | 11 ++-- .../action/PerformParallelDeleteRequests.java | 6 +- .../action/PerformParallelPutRequests.java | 4 +- .../action/PerformParallelRequests.java | 16 ++--- .../action/PerformSerialGetAllRequests.java | 19 +++--- .../action/PerformSerialPutRequests.java | 12 ++-- .../routed/action/PerformSerialRequests.java | 8 ++- .../voldemort/store/slop/HintedHandoff.java | 60 ++++++++++--------- src/java/voldemort/utils/ByteArray.java | 14 +++++ 13 files changed, 149 insertions(+), 101 deletions(-) diff --git a/src/java/voldemort/server/niosocket/AsyncRequestHandler.java b/src/java/voldemort/server/niosocket/AsyncRequestHandler.java index 27772e7740..d6af05b642 100644 --- a/src/java/voldemort/server/niosocket/AsyncRequestHandler.java +++ b/src/java/voldemort/server/niosocket/AsyncRequestHandler.java @@ -127,13 +127,16 @@ protected void read(SelectionKey selectionKey) throws IOException { if(logger.isTraceEnabled()) logger.trace("Starting execution for " + socketChannel.socket()); - streamRequestHandler = requestHandler.handleRequest(new DataInputStream(inputStream), - new DataOutputStream(outputStream)); + DataInputStream dataInputStream = new DataInputStream(inputStream); + DataOutputStream dataOutputStream = new DataOutputStream(outputStream); + + streamRequestHandler = requestHandler.handleRequest(dataInputStream, + dataOutputStream); if(logger.isDebugEnabled()) { logger.debug("AsyncRequestHandler:read finished request from " + socketChannel.socket().getRemoteSocketAddress() + " handlerRef: " - + System.identityHashCode(streamRequestHandler) + " at time: " + + System.identityHashCode(dataInputStream) + " at time: " + System.currentTimeMillis() + " elapsed time: " + (System.nanoTime() - startNs) + " ns"); } @@ -307,7 +310,7 @@ private StreamRequestHandlerState handleStreamRequestInternal(SelectionKey selec if(logger.isDebugEnabled()) { logger.debug("Handled request from " + socketChannel.socket().getRemoteSocketAddress() + " handlerRef: " - + System.identityHashCode(streamRequestHandler) + " at time: " + + System.identityHashCode(dataInputStream) + " at time: " + System.currentTimeMillis() + " elapsed time: " + (System.nanoTime() - startNs) + " ns"); } diff --git a/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java b/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java index fa70a4b3eb..cadce8b8bb 100644 --- a/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java +++ b/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java @@ -135,7 +135,8 @@ private void handleGetVersion(DataInputStream inputStream, if(logger.isDebugEnabled()) { logger.debug("GETVERSIONS started at: " + startTimeMs + " handlerRef: " - + System.identityHashCode(this) + " key: " + key + " " + + System.identityHashCode(inputStream) + " key: " + + ByteUtils.toHexString(key.get()) + " " + (System.nanoTime() - startTimeNs) + " ns, keySize: " + key.length() + "clocks: " + clockStr); } @@ -317,7 +318,7 @@ private void handleGet(DataInputStream inputStream, } writeResults(outputStream, results); if(logger.isDebugEnabled()) { - debugLogReturnValue(key, results, startTimeMs, startTimeNs, "GET"); + debugLogReturnValue(inputStream, key, results, startTimeMs, startTimeNs, "GET"); } } @@ -374,7 +375,8 @@ private void handleGetAll(DataInputStream inputStream, writeResults(outputStream, entry.getValue()); if(logger.isDebugEnabled()) { - debugLogReturnValue(entry.getKey(), + debugLogReturnValue(inputStream, + entry.getKey(), entry.getValue(), startTimeMs, startTimeNs, @@ -386,7 +388,8 @@ private void handleGetAll(DataInputStream inputStream, logger.debug("GETALL end"); } - private void debugLogReturnValue(ByteArray key, + private void debugLogReturnValue(DataInputStream input, + ByteArray key, List> values, long startTimeMs, long startTimeNs, @@ -406,12 +409,13 @@ private void debugLogReturnValue(ByteArray key, valueHashStr += "]"; versionsStr += "]"; - logger.debug(getType + " handlerRef: " + System.identityHashCode(this) + " start time: " - + startTimeMs + " key: " + key + " elapsed time: " - + (System.nanoTime() - startTimeNs) + " ns, keySize: " + key.length() - + " numResults: " + values.size() + " totalResultSize: " + totalValueSize - + " resultSizes: " + valueSizeStr + " resultHashes: " + valueHashStr - + " versions: " + versionsStr + " current time: " + System.currentTimeMillis()); + logger.debug(getType + " handlerRef: " + System.identityHashCode(input) + " start time: " + + startTimeMs + " key: " + ByteUtils.toHexString(key.get()) + + " elapsed time: " + (System.nanoTime() - startTimeNs) + " ns, keySize: " + + key.length() + " numResults: " + values.size() + " totalResultSize: " + + totalValueSize + " resultSizes: " + valueSizeStr + " resultHashes: " + + valueHashStr + " versions: " + versionsStr + " current time: " + + System.currentTimeMillis()); } private void handlePut(DataInputStream inputStream, @@ -447,9 +451,10 @@ private void handlePut(DataInputStream inputStream, if(logger.isDebugEnabled()) { logger.debug("PUT started at: " + startTimeMs + " handlerRef: " - + System.identityHashCode(this) + " key: " + key + " " + + System.identityHashCode(inputStream) + " key: " + + ByteUtils.toHexString(key.get()) + " " + (System.nanoTime() - startTimeNs) + " ns, keySize: " + key.length() - + " valueHash: " + value.hashCode() + " valueSize: " + valueSize + + " valueHash: " + value.hashCode() + " valueSize: " + value.length + " clockSize: " + clock.sizeInBytes() + " time: " + System.currentTimeMillis()); } @@ -480,8 +485,9 @@ private void handleDelete(DataInputStream inputStream, } if(logger.isDebugEnabled()) { - logger.debug("DELETE started at: " + startTimeMs + " key: " + key + " handlerRef: " - + System.identityHashCode(this) + " time: " + logger.debug("DELETE started at: " + startTimeMs + " key: " + + ByteUtils.toHexString(key.get()) + " handlerRef: " + + System.identityHashCode(inputStream) + " time: " + (System.nanoTime() - startTimeNs) + " ns, keySize: " + key.length() + " clockSize: " + version.sizeInBytes()); } diff --git a/src/java/voldemort/store/bdb/BdbStorageEngine.java b/src/java/voldemort/store/bdb/BdbStorageEngine.java index 1b2013ca46..b74e31cfe6 100644 --- a/src/java/voldemort/store/bdb/BdbStorageEngine.java +++ b/src/java/voldemort/store/bdb/BdbStorageEngine.java @@ -281,7 +281,7 @@ public Map>> getAll(Iterable keys, for(ByteArray key: keys) { if(logger.isTraceEnabled()) - keyStr += key + " "; + keyStr += ByteUtils.toHexString(key.get()) + " "; List> values = get(cursor, key, readLockMode, versionedSerializer); if(!values.isEmpty()) @@ -324,7 +324,7 @@ private static List get(Cursor cursor, } if(logger.isTraceEnabled()) { - logger.trace("Completed GET from key " + key + " in " + logger.trace("Completed GET from key " + ByteUtils.toHexString(key.get()) + " in " + (System.nanoTime() - startTimeNs) + " ns at " + System.currentTimeMillis()); } @@ -390,7 +390,7 @@ else if(occurred == Occurred.AFTER) } if(logger.isTraceEnabled()) { - logger.trace("Completed PUT to key " + key + " (keyRef: " + logger.trace("Completed PUT to key " + ByteUtils.toHexString(key.get()) + " (keyRef: " + System.identityHashCode(key) + " value " + value + " in " + (System.nanoTime() - startTimeNs) + " ns at " + System.currentTimeMillis()); @@ -431,8 +431,8 @@ public boolean delete(ByteArray key, Version version) throws PersistenceFailureE } finally { if(logger.isTraceEnabled()) { - logger.trace("Completed DELETE of key " + key + " (keyRef: " - + System.identityHashCode(key) + ") in " + logger.trace("Completed DELETE of key " + ByteUtils.toHexString(key.get()) + + " (keyRef: " + System.identityHashCode(key) + ") in " + (System.nanoTime() - startTimeNs) + " ns at " + System.currentTimeMillis()); } diff --git a/src/java/voldemort/store/routed/PipelineRoutedStore.java b/src/java/voldemort/store/routed/PipelineRoutedStore.java index dfde9b350c..dc8e20265e 100644 --- a/src/java/voldemort/store/routed/PipelineRoutedStore.java +++ b/src/java/voldemort/store/routed/PipelineRoutedStore.java @@ -251,9 +251,10 @@ public List> request(Store store) { } if(logger.isDebugEnabled()) { - logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + key - + " keyRef: " + System.identityHashCode(key) + "; started at " - + startTimeMs + " took " + (System.nanoTime() - startTimeNs) + " values: " + logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + + ByteUtils.toHexString(key.get()) + " keyRef: " + + System.identityHashCode(key) + "; started at " + startTimeMs + " took " + + (System.nanoTime() - startTimeNs) + " values: " + formatNodeValuesFromGet(pipelineData.getResponses())); } @@ -355,9 +356,10 @@ public Map>> getAll(Iterable keys, throw pipelineData.getFatalError(); if(logger.isDebugEnabled()) { - logger.debug("Finished " + pipeline.getOperation().getSimpleName() + "for keys " + keys - + " keyRef: " + System.identityHashCode(keys) + "; started at " - + startTimeMs + " took " + (System.nanoTime() - startTimeNs) + " values: " + logger.debug("Finished " + pipeline.getOperation().getSimpleName() + "for keys " + + ByteArray.toHexStrings(keys) + " keyRef: " + + System.identityHashCode(keys) + "; started at " + startTimeMs + " took " + + (System.nanoTime() - startTimeNs) + " values: " + formatNodeValuesFromGetAll(pipelineData.getResponses())); } @@ -369,8 +371,9 @@ private String formatNodeValuesFromGetAll(List, Map StringBuilder builder = new StringBuilder(); builder.append("{"); for(Response, Map>>> r: list) { - builder.append("(nodeId=" + r.getNode().getId() + ", key=" + r.getKey() - + ", retrieved= " + r.getValue() + ")"); + builder.append("(nodeId=" + r.getNode().getId() + ", keys=" + + ByteArray.toHexStrings(r.getKey()) + ", retrieved= " + r.getValue() + + ")"); builder.append(", "); } builder.append("}"); @@ -469,9 +472,10 @@ public List request(Store store) { results.addAll(response.getValue()); if(logger.isDebugEnabled()) { - logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + key - + " keyRef: " + System.identityHashCode(key) + "; started at " - + startTimeMs + " took " + (System.nanoTime() - startTimeNs) + " values: " + logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + + ByteUtils.toHexString(key.get()) + " keyRef: " + + System.identityHashCode(key) + "; started at " + startTimeMs + " took " + + (System.nanoTime() - startTimeNs) + " values: " + formatNodeValuesFromGetVersions(pipelineData.getResponses())); } @@ -483,8 +487,9 @@ private String formatNodeValuesFromGetVersions(List> r: results) { - builder.append("(nodeId=" + r.getNode().getId() + ", key=" + r.getKey() - + ", retrieved= " + r.getValue() + "), "); + builder.append("(nodeId=" + r.getNode().getId() + ", key=" + + ByteUtils.toHexString(r.getKey().get()) + ", retrieved= " + + r.getValue() + "), "); } builder.append("}"); @@ -575,8 +580,9 @@ public boolean delete(final ByteArray key, final Version version) throws Voldemo if(logger.isDebugEnabled()) { logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " - + key.get() + " keyRef: " + System.identityHashCode(key) + "; started at " - + startTimeMs + " took " + (System.nanoTime() - startTimeNs)); + + ByteUtils.toHexString(key.get()) + " keyRef: " + + System.identityHashCode(key) + "; started at " + startTimeMs + " took " + + (System.nanoTime() - startTimeNs)); } if(pipelineData.getFatalError() != null) @@ -700,10 +706,11 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) } if(logger.isDebugEnabled()) { - logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + key - + " keyRef: " + System.identityHashCode(key) + "; started at " - + startTimeMs + " took " + (System.nanoTime() - startTimeNs) + " value: " - + versioned.getValue() + " (size: " + versioned.getValue().length + ")"); + logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + + ByteUtils.toHexString(key.get()) + " keyRef: " + + System.identityHashCode(key) + "; started at " + startTimeMs + " took " + + (System.nanoTime() - startTimeNs) + " value: " + versioned.getValue() + + " (size: " + versioned.getValue().length + ")"); } if(pipelineData.getFatalError() != null) diff --git a/src/java/voldemort/store/routed/action/AbstractReadRepair.java b/src/java/voldemort/store/routed/action/AbstractReadRepair.java index 6e2ea19d6f..ac2dac7829 100644 --- a/src/java/voldemort/store/routed/action/AbstractReadRepair.java +++ b/src/java/voldemort/store/routed/action/AbstractReadRepair.java @@ -24,10 +24,11 @@ import voldemort.store.nonblockingstore.NonblockingStore; import voldemort.store.routed.NodeValue; import voldemort.store.routed.Pipeline; +import voldemort.store.routed.Pipeline.Event; import voldemort.store.routed.PipelineData; import voldemort.store.routed.ReadRepairer; -import voldemort.store.routed.Pipeline.Event; import voldemort.utils.ByteArray; +import voldemort.utils.ByteUtils; import voldemort.versioning.VectorClock; import voldemort.versioning.Versioned; @@ -101,7 +102,8 @@ public void execute(Pipeline pipeline) { try { if(logger.isDebugEnabled()) logger.debug("Doing read repair on node " + v.getNodeId() + " for key '" - + v.getKey() + "' with version " + v.getVersion() + "."); + + ByteUtils.toHexString(v.getKey().get()) + "' with version " + + v.getVersion() + "."); NonblockingStore store = nonblockingStores.get(v.getNodeId()); store.submitPutRequest(v.getKey(), v.getVersioned(), null, null, timeoutMs); @@ -110,8 +112,9 @@ public void execute(Pipeline pipeline) { logger.debug("Read repair cancelled due to application level exception on node " + v.getNodeId() + " for key '" - + v.getKey() - + "' with version " + v.getVersion() + ": " + e.getMessage()); + + ByteUtils.toHexString(v.getKey().get()) + + "' with version " + + v.getVersion() + ": " + e.getMessage()); } catch(Exception e) { logger.debug("Read repair failed: ", e); } diff --git a/src/java/voldemort/store/routed/action/PerformParallelDeleteRequests.java b/src/java/voldemort/store/routed/action/PerformParallelDeleteRequests.java index 7f30c90a9b..634b470805 100644 --- a/src/java/voldemort/store/routed/action/PerformParallelDeleteRequests.java +++ b/src/java/voldemort/store/routed/action/PerformParallelDeleteRequests.java @@ -93,9 +93,9 @@ public void execute(final Pipeline pipeline) { public void requestComplete(Object result, long requestTime) { if(logger.isTraceEnabled()) - logger.info(pipeline.getOperation().getSimpleName() - + " response received (" + requestTime + " ms.) from node " - + node.getId()); + logger.trace(pipeline.getOperation().getSimpleName() + + " response received (" + requestTime + " ms.) from node " + + node.getId()); Response response = new Response(node, key, diff --git a/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java b/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java index a8373418b5..b2e3bc09be 100644 --- a/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java +++ b/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java @@ -42,6 +42,7 @@ import voldemort.store.slop.HintedHandoff; import voldemort.store.slop.Slop; import voldemort.utils.ByteArray; +import voldemort.utils.ByteUtils; import voldemort.utils.Time; import voldemort.versioning.ObsoleteVersionException; import voldemort.versioning.Versioned; @@ -130,7 +131,8 @@ public void requestComplete(Object result, long requestTime) { responses.put(node.getId(), response); if(logger.isDebugEnabled()) - logger.debug("Finished secondary PUT for key " + key + " (keyRef: " + logger.debug("Finished secondary PUT for key " + + ByteUtils.toHexString(key.get()) + " (keyRef: " + System.identityHashCode(key) + "); took " + requestTime + " ms on node " + node.getId() + "(" + node.getHost() + ")"); diff --git a/src/java/voldemort/store/routed/action/PerformParallelRequests.java b/src/java/voldemort/store/routed/action/PerformParallelRequests.java index 73399d75b0..18385a5f7f 100644 --- a/src/java/voldemort/store/routed/action/PerformParallelRequests.java +++ b/src/java/voldemort/store/routed/action/PerformParallelRequests.java @@ -38,6 +38,7 @@ import voldemort.store.routed.Pipeline.Operation; import voldemort.store.routed.Response; import voldemort.utils.ByteArray; +import voldemort.utils.ByteUtils; import voldemort.utils.Utils; public class PerformParallelRequests> extends @@ -111,10 +112,10 @@ public void requestComplete(Object result, long requestTime) { requestTime); if(logger.isDebugEnabled()) logger.debug("Finished " + pipeline.getOperation().getSimpleName() - + " for key " + key + " (keyRef: " - + System.identityHashCode(key) + "); started at " + startMs - + " took " + requestTime + " ms on node " + node.getId() + "(" - + node.getHost() + ")"); + + " for key " + ByteUtils.toHexString(key.get()) + + " (keyRef: " + System.identityHashCode(key) + + "); started at " + startMs + " took " + requestTime + + " ms on node " + node.getId() + "(" + node.getHost() + ")"); responses.put(node.getId(), response); latch.countDown(); @@ -174,9 +175,10 @@ else if(pipeline.getOperation() == Operation.GET_VERSIONS) } if(logger.isDebugEnabled()) - logger.debug("GET for key " + key + " (keyRef: " + System.identityHashCode(key) - + "); successes: " + pipelineData.getSuccesses() + " preferred: " - + preferred + " required: " + required); + logger.debug("GET for key " + ByteUtils.toHexString(key.get()) + " (keyRef: " + + System.identityHashCode(key) + "); successes: " + + pipelineData.getSuccesses() + " preferred: " + preferred + " required: " + + required); if(pipelineData.getSuccesses() < required) { if(insufficientSuccessesEvent != null) { diff --git a/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java b/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java index c8135781d4..bcfcc820ad 100644 --- a/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java +++ b/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java @@ -79,10 +79,11 @@ public void execute(Pipeline pipeline) { boolean zoneRequirement = false; MutableInt successCount = pipelineData.getSuccessCount(key); - if(logger.isDebugEnabled()) - logger.debug("GETALL for key " + key + " (keyRef: " + System.identityHashCode(key) - + ") successes: " + successCount.intValue() + " preferred: " + preferred - + " required: " + required); + if(logger.isDebugEnabled()) + logger.debug("GETALL for key " + ByteUtils.toHexString(key.get()) + " (keyRef: " + + System.identityHashCode(key) + ") successes: " + + successCount.intValue() + " preferred: " + preferred + " required: " + + required); if(successCount.intValue() >= preferred) { if(pipelineData.getZonesRequired() != null) { @@ -138,11 +139,11 @@ public void execute(Pipeline pipeline) { failureDetector.recordSuccess(response.getNode(), response.getRequestTime()); if(logger.isDebugEnabled()) - logger.debug("GET for key " + key + " (keyRef: " - + System.identityHashCode(key) + ") successes: " - + successCount.intValue() + " preferred: " + preferred - + " required: " + required + " new GET success on node " - + node.getId()); + logger.debug("GET for key " + ByteUtils.toHexString(key.get()) + + " (keyRef: " + System.identityHashCode(key) + + ") successes: " + successCount.intValue() + " preferred: " + + preferred + " required: " + required + + " new GET success on node " + node.getId()); HashSet zoneResponses = null; if(pipelineData.getKeyToZoneResponse().containsKey(key)) { diff --git a/src/java/voldemort/store/routed/action/PerformSerialPutRequests.java b/src/java/voldemort/store/routed/action/PerformSerialPutRequests.java index 7b7b3d65fd..08a25c7bc7 100644 --- a/src/java/voldemort/store/routed/action/PerformSerialPutRequests.java +++ b/src/java/voldemort/store/routed/action/PerformSerialPutRequests.java @@ -29,6 +29,7 @@ import voldemort.store.routed.Pipeline.Event; import voldemort.store.routed.PutPipelineData; import voldemort.utils.ByteArray; +import voldemort.utils.ByteUtils; import voldemort.utils.Time; import voldemort.versioning.VectorClock; import voldemort.versioning.Versioned; @@ -173,7 +174,8 @@ public void execute(Pipeline pipeline) { } else { if(logger.isDebugEnabled()) - logger.debug("Finished master PUT for key " + key + " (keyRef: " + logger.debug("Finished master PUT for key " + + ByteUtils.toHexString(key.get()) + " (keyRef: " + System.identityHashCode(key) + "); started at " + startMasterMs + " took " + (System.nanoTime() - startMasterNs) + " ns on node " @@ -185,10 +187,10 @@ public void execute(Pipeline pipeline) { } } else { if(logger.isDebugEnabled()) - logger.debug("Finished master PUT for key " + key + " (keyRef: " - + System.identityHashCode(key) + "); started at " + startMasterMs - + " took " + (System.nanoTime() - startMasterNs) + " ns on node " - + (node == null ? "NULL" : node.getId()) + "(" + logger.debug("Finished master PUT for key " + ByteUtils.toHexString(key.get()) + + " (keyRef: " + System.identityHashCode(key) + "); started at " + + startMasterMs + " took " + (System.nanoTime() - startMasterNs) + + " ns on node " + (node == null ? "NULL" : node.getId()) + "(" + (node == null ? "NULL" : node.getHost()) + ")"); pipeline.addEvent(masterDeterminedEvent); diff --git a/src/java/voldemort/store/routed/action/PerformSerialRequests.java b/src/java/voldemort/store/routed/action/PerformSerialRequests.java index 4c447174d8..6feafd140c 100644 --- a/src/java/voldemort/store/routed/action/PerformSerialRequests.java +++ b/src/java/voldemort/store/routed/action/PerformSerialRequests.java @@ -31,6 +31,7 @@ import voldemort.store.routed.Pipeline.Event; import voldemort.store.routed.Response; import voldemort.utils.ByteArray; +import voldemort.utils.ByteUtils; import voldemort.utils.Time; public class PerformSerialRequests> extends @@ -99,9 +100,10 @@ public void execute(Pipeline pipeline) { ((System.nanoTime() - start) / Time.NS_PER_MS)); if(logger.isDebugEnabled()) - logger.debug(pipeline.getOperation().getSimpleName() + " for key " + key - + " successes: " + pipelineData.getSuccesses() + " preferred: " - + preferred + " required: " + required + " new " + logger.debug(pipeline.getOperation().getSimpleName() + " for key " + + ByteUtils.toHexString(key.get()) + " successes: " + + pipelineData.getSuccesses() + " preferred: " + preferred + + " required: " + required + " new " + pipeline.getOperation().getSimpleName() + " success on node " + node.getId()); diff --git a/src/java/voldemort/store/slop/HintedHandoff.java b/src/java/voldemort/store/slop/HintedHandoff.java index 5e506a85e2..ff2d81b823 100644 --- a/src/java/voldemort/store/slop/HintedHandoff.java +++ b/src/java/voldemort/store/slop/HintedHandoff.java @@ -27,11 +27,12 @@ import voldemort.serialization.SlopSerializer; import voldemort.store.Store; import voldemort.store.UnreachableStoreException; -import voldemort.store.slop.strategy.HintedHandoffStrategy; import voldemort.store.nonblockingstore.NonblockingStore; import voldemort.store.nonblockingstore.NonblockingStoreCallback; import voldemort.store.routed.Response; +import voldemort.store.slop.strategy.HintedHandoffStrategy; import voldemort.utils.ByteArray; +import voldemort.utils.ByteUtils; import voldemort.utils.Time; import voldemort.utils.Utils; import voldemort.versioning.ObsoleteVersionException; @@ -67,7 +68,8 @@ public class HintedHandoff { * Create a Hinted Handoff object * * @param failureDetector The failure detector - * @param nonblockingSlopStores A map of node ids to nonb-locking slop stores + * @param nonblockingSlopStores A map of node ids to nonb-locking slop + * stores * @param slopStores A map of node ids to blocking slop stores * @param handoffStrategy The {@link HintedHandoffStrategy} implementation * @param failedNodes A list of nodes in the original preflist for the @@ -89,15 +91,18 @@ public HintedHandoff(FailureDetector failureDetector, } /** - * Like {@link #sendHintSerial(voldemort.cluster.Node, voldemort.versioning.Version, Slop)}, - * but doesn't block the pipeline. Intended for handling prolonged failures without - * incurring a performance cost. - * - * @see #sendHintSerial(voldemort.cluster.Node, voldemort.versioning.Version, Slop) + * Like + * {@link #sendHintSerial(voldemort.cluster.Node, voldemort.versioning.Version, Slop)} + * , but doesn't block the pipeline. Intended for handling prolonged + * failures without incurring a performance cost. + * + * @see #sendHintSerial(voldemort.cluster.Node, + * voldemort.versioning.Version, Slop) */ public void sendHintParallel(final Node failedNode, final Version version, final Slop slop) { final ByteArray slopKey = slop.makeKey(); - Versioned slopVersioned = new Versioned(slopSerializer.toBytes(slop), version); + Versioned slopVersioned = new Versioned(slopSerializer.toBytes(slop), + version); for(final Node node: handoffStrategy.routeHint(failedNode)) { int nodeId = node.getId(); @@ -111,10 +116,12 @@ public void sendHintParallel(final Node failedNode, final Version version, final final long startNs = System.nanoTime(); if(logger.isDebugEnabled()) - logger.debug("Slop attempt to write " + slop.getKey() + " for " + failedNode - + " to node " + node); + logger.debug("Slop attempt to write " + + ByteUtils.toHexString(slop.getKey().get()) + " for " + + failedNode + " to node " + node); NonblockingStoreCallback callback = new NonblockingStoreCallback() { + public void requestComplete(Object result, long requestTime) { Response response = new Response(node, slopKey, @@ -130,15 +137,16 @@ public void requestComplete(Object result, long requestTime) { if(response.getValue() instanceof UnreachableStoreException) { UnreachableStoreException use = (UnreachableStoreException) response.getValue(); - if(logger.isDebugEnabled()) - logger.debug("Write of key " + slop.getKey() + " for " - + failedNode + " to node " + node - + " failed due to unreachable: " - + use.getMessage()); + if(logger.isDebugEnabled()) + logger.debug("Write of key " + + ByteUtils.toHexString(slop.getKey().get()) + + " for " + failedNode + " to node " + node + + " failed due to unreachable: " + + use.getMessage()); failureDetector.recordException(node, (System.nanoTime() - startNs) - / Time.NS_PER_MS, + / Time.NS_PER_MS, use); } sendHintSerial(failedNode, version, slop); @@ -147,7 +155,8 @@ public void requestComplete(Object result, long requestTime) { } if(logger.isDebugEnabled()) - logger.debug("Slop write of key " + slop.getKey() + " for " + logger.debug("Slop write of key " + + ByteUtils.toHexString(slop.getKey().get()) + " for " + failedNode + " to node " + node + " succeeded in " + (System.nanoTime() - startNs) + " ns"); @@ -157,16 +166,12 @@ public void requestComplete(Object result, long requestTime) { } }; - nonblockingStore.submitPutRequest(slopKey, - slopVersioned, - null, - callback, - timeoutMs); + nonblockingStore.submitPutRequest(slopKey, slopVersioned, null, callback, timeoutMs); break; } } } - + /** * Send a hint of a request originally meant for the failed node to another * node in the ring, as selected by the {@link HintedHandoffStrategy} @@ -191,7 +196,8 @@ public boolean sendHintSerial(Node failedNode, Version version, Slop slop) { try { if(logger.isDebugEnabled()) - logger.debug("Slop attempt to write " + slop.getKey() + " (keyRef: " + logger.debug("Slop attempt to write " + + ByteUtils.toHexString(slop.getKey().get()) + " (keyRef: " + System.identityHashCode(slop.getKey()) + ") for " + failedNode + " to node " + node); @@ -214,9 +220,9 @@ public boolean sendHintSerial(Node failedNode, Version version, Slop slop) { } if(logger.isDebugEnabled()) - logger.debug("Slop write of key " + slop.getKey() + " (keyRef: " - + System.identityHashCode(slop.getKey()) + " for " + failedNode - + " to node " + node + " succeeded in " + logger.debug("Slop write of key " + ByteUtils.toHexString(slop.getKey().get()) + + " (keyRef: " + System.identityHashCode(slop.getKey()) + " for " + + failedNode + " to node " + node + " succeeded in " + (System.nanoTime() - startNs) + " ns"); } } diff --git a/src/java/voldemort/utils/ByteArray.java b/src/java/voldemort/utils/ByteArray.java index 71dc609b2d..c6ccf86ada 100644 --- a/src/java/voldemort/utils/ByteArray.java +++ b/src/java/voldemort/utils/ByteArray.java @@ -1,6 +1,7 @@ package voldemort.utils; import java.io.Serializable; +import java.util.ArrayList; import java.util.Arrays; /** @@ -43,6 +44,19 @@ public String toString() { return Arrays.toString(underlying); } + /** + * Translate the each ByteArray in an iterable into a hexidecimal string + * + * @param arrays The array of bytes to translate + * @return An iterable of converted strings + */ + public static Iterable toHexStrings(Iterable arrays) { + ArrayList ret = new ArrayList(); + for(ByteArray array: arrays) + ret.add(ByteUtils.toHexString(array.get())); + return ret; + } + public int length() { return underlying.length; } From 4f8c9789505aa33ba1c2018567248cbf8b63007f Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Thu, 9 Aug 2012 17:47:54 -0700 Subject: [PATCH 111/209] Deleted CNAME in master --- CNAME | 1 - 1 file changed, 1 deletion(-) delete mode 100644 CNAME diff --git a/CNAME b/CNAME deleted file mode 100644 index 31eeccbbb3..0000000000 --- a/CNAME +++ /dev/null @@ -1 +0,0 @@ -beta.project-voldemort.com From 89ded28e158ad2502f91fada78e72922e0337b41 Mon Sep 17 00:00:00 2001 From: Jay J Wylie Date: Fri, 10 Aug 2012 09:55:05 -0700 Subject: [PATCH 112/209] Cleaned up help/usage messages within the client shell. In particular, print out possible values of meta_key for getmetadata and clarified how to use fetchkeys and fetch. --- src/java/voldemort/VoldemortClientShell.java | 37 ++++++++++++-------- 1 file changed, 23 insertions(+), 14 deletions(-) diff --git a/src/java/voldemort/VoldemortClientShell.java b/src/java/voldemort/VoldemortClientShell.java index 81de718b75..6c364cac84 100644 --- a/src/java/voldemort/VoldemortClientShell.java +++ b/src/java/voldemort/VoldemortClientShell.java @@ -268,26 +268,35 @@ private static void processCommands(StoreClientFactory factory, writer.flush(); } } else if(line.startsWith("help")) { + System.out.println(); System.out.println("Commands:"); - System.out.println("put key value -- Associate the given value with the key."); - System.out.println("get key -- Retrieve the value associated with the key."); - System.out.println("getall key -- Retrieve the value(s) associated with the key."); - System.out.println("delete key -- Remove all values associated with the key."); - System.out.println("preflist key -- Get node preference list for given key."); - System.out.println("getmetadata node_id key -- Get metadata associated with key from node_id."); - System.out.println("fetchkeys node_id store_name partitions -- Fetch all keys from given partitions" - + " (a comma separated list) of store_name on node_id. Optionally, write to file_name."); - System.out.println("fetch node_id store_name partitions -- Fetch all entries from given partitions" - + " (a comma separated list) of store_name on node_id. Optionally, write to file_name."); - System.out.println("help -- Print this message."); - System.out.println("exit -- Exit from this shell."); + System.out.println(PROMPT + "put key value --- Associate the given value with the key."); + System.out.println(PROMPT + "get key --- Retrieve the value associated with the key."); + System.out.println(PROMPT + "getall key1 [key2...] --- Retrieve the value(s) associated with the key(s)."); + System.out.println(PROMPT + "delete key --- Remove all values associated with the key."); + System.out.println(PROMPT + "preflist key --- Get node preference list for given key."); + String metaKeyValues = voldemort.store.metadata.MetadataStore.METADATA_KEYS.toString(); + System.out.println(PROMPT + "getmetadata node_id meta_key --- Get store metadata associated " + + "with meta_key from node_id. meta_key may be one of " + + metaKeyValues.substring(1, metaKeyValues.length() - 1) + + "."); + System.out.println(PROMPT + "fetchkeys node_id store_name partitions --- Fetch all keys " + + "from given partitions (a comma separated list) of store_name on " + + "node_id. Optionally, write to file_name. " + + "Use getmetadata to determine appropriate values for store_name and partitions"); + System.out.println(PROMPT + "fetch node_id store_name partitions --- Fetch all entries " + + "from given partitions (a comma separated list) of store_name on " + + "node_id. Optionally, write to file_name. " + + "Use getmetadata to determine appropriate values for store_name and partitions"); + System.out.println(PROMPT + "help --- Print this message."); + System.out.println(PROMPT + "exit --- Exit from this shell."); System.out.println(); } else if(line.startsWith("quit") || line.startsWith("exit")) { System.out.println("k k thx bye."); System.exit(0); } else { - System.err.println("Invalid command."); + System.err.println("Invalid command. (Try 'help' for usage.)"); } } catch(EndOfFileException e) { System.err.println("Expected additional token."); @@ -298,7 +307,7 @@ private static void processCommands(StoreClientFactory factory, System.err.println("Exception thrown during operation."); e.printStackTrace(System.err); } catch(ArrayIndexOutOfBoundsException e) { - System.err.println("Invalid command."); + System.err.println("Invalid command. (Try 'help' for usage.)"); } catch(Exception e) { System.err.println("Unexpected error:"); e.printStackTrace(System.err); From 210a2bdb8b43588327608fe779133bc65447e866 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Fri, 17 Aug 2012 12:48:18 -0700 Subject: [PATCH 113/209] Revert "Improving debug messages in request tracing" This reverts commit 98643ea18481ebfbb0fe1a7575ae6157f84f939e. --- .../server/niosocket/AsyncRequestHandler.java | 11 ++-- .../vold/VoldemortNativeRequestHandler.java | 34 +++++------ .../voldemort/store/bdb/BdbStorageEngine.java | 10 ++-- .../store/routed/PipelineRoutedStore.java | 45 ++++++-------- .../routed/action/AbstractReadRepair.java | 11 ++-- .../action/PerformParallelDeleteRequests.java | 6 +- .../action/PerformParallelPutRequests.java | 4 +- .../action/PerformParallelRequests.java | 16 +++-- .../action/PerformSerialGetAllRequests.java | 19 +++--- .../action/PerformSerialPutRequests.java | 12 ++-- .../routed/action/PerformSerialRequests.java | 8 +-- .../voldemort/store/slop/HintedHandoff.java | 60 +++++++++---------- src/java/voldemort/utils/ByteArray.java | 14 ----- 13 files changed, 101 insertions(+), 149 deletions(-) diff --git a/src/java/voldemort/server/niosocket/AsyncRequestHandler.java b/src/java/voldemort/server/niosocket/AsyncRequestHandler.java index d6af05b642..27772e7740 100644 --- a/src/java/voldemort/server/niosocket/AsyncRequestHandler.java +++ b/src/java/voldemort/server/niosocket/AsyncRequestHandler.java @@ -127,16 +127,13 @@ protected void read(SelectionKey selectionKey) throws IOException { if(logger.isTraceEnabled()) logger.trace("Starting execution for " + socketChannel.socket()); - DataInputStream dataInputStream = new DataInputStream(inputStream); - DataOutputStream dataOutputStream = new DataOutputStream(outputStream); - - streamRequestHandler = requestHandler.handleRequest(dataInputStream, - dataOutputStream); + streamRequestHandler = requestHandler.handleRequest(new DataInputStream(inputStream), + new DataOutputStream(outputStream)); if(logger.isDebugEnabled()) { logger.debug("AsyncRequestHandler:read finished request from " + socketChannel.socket().getRemoteSocketAddress() + " handlerRef: " - + System.identityHashCode(dataInputStream) + " at time: " + + System.identityHashCode(streamRequestHandler) + " at time: " + System.currentTimeMillis() + " elapsed time: " + (System.nanoTime() - startNs) + " ns"); } @@ -310,7 +307,7 @@ private StreamRequestHandlerState handleStreamRequestInternal(SelectionKey selec if(logger.isDebugEnabled()) { logger.debug("Handled request from " + socketChannel.socket().getRemoteSocketAddress() + " handlerRef: " - + System.identityHashCode(dataInputStream) + " at time: " + + System.identityHashCode(streamRequestHandler) + " at time: " + System.currentTimeMillis() + " elapsed time: " + (System.nanoTime() - startNs) + " ns"); } diff --git a/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java b/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java index cadce8b8bb..fa70a4b3eb 100644 --- a/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java +++ b/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java @@ -135,8 +135,7 @@ private void handleGetVersion(DataInputStream inputStream, if(logger.isDebugEnabled()) { logger.debug("GETVERSIONS started at: " + startTimeMs + " handlerRef: " - + System.identityHashCode(inputStream) + " key: " - + ByteUtils.toHexString(key.get()) + " " + + System.identityHashCode(this) + " key: " + key + " " + (System.nanoTime() - startTimeNs) + " ns, keySize: " + key.length() + "clocks: " + clockStr); } @@ -318,7 +317,7 @@ private void handleGet(DataInputStream inputStream, } writeResults(outputStream, results); if(logger.isDebugEnabled()) { - debugLogReturnValue(inputStream, key, results, startTimeMs, startTimeNs, "GET"); + debugLogReturnValue(key, results, startTimeMs, startTimeNs, "GET"); } } @@ -375,8 +374,7 @@ private void handleGetAll(DataInputStream inputStream, writeResults(outputStream, entry.getValue()); if(logger.isDebugEnabled()) { - debugLogReturnValue(inputStream, - entry.getKey(), + debugLogReturnValue(entry.getKey(), entry.getValue(), startTimeMs, startTimeNs, @@ -388,8 +386,7 @@ private void handleGetAll(DataInputStream inputStream, logger.debug("GETALL end"); } - private void debugLogReturnValue(DataInputStream input, - ByteArray key, + private void debugLogReturnValue(ByteArray key, List> values, long startTimeMs, long startTimeNs, @@ -409,13 +406,12 @@ private void debugLogReturnValue(DataInputStream input, valueHashStr += "]"; versionsStr += "]"; - logger.debug(getType + " handlerRef: " + System.identityHashCode(input) + " start time: " - + startTimeMs + " key: " + ByteUtils.toHexString(key.get()) - + " elapsed time: " + (System.nanoTime() - startTimeNs) + " ns, keySize: " - + key.length() + " numResults: " + values.size() + " totalResultSize: " - + totalValueSize + " resultSizes: " + valueSizeStr + " resultHashes: " - + valueHashStr + " versions: " + versionsStr + " current time: " - + System.currentTimeMillis()); + logger.debug(getType + " handlerRef: " + System.identityHashCode(this) + " start time: " + + startTimeMs + " key: " + key + " elapsed time: " + + (System.nanoTime() - startTimeNs) + " ns, keySize: " + key.length() + + " numResults: " + values.size() + " totalResultSize: " + totalValueSize + + " resultSizes: " + valueSizeStr + " resultHashes: " + valueHashStr + + " versions: " + versionsStr + " current time: " + System.currentTimeMillis()); } private void handlePut(DataInputStream inputStream, @@ -451,10 +447,9 @@ private void handlePut(DataInputStream inputStream, if(logger.isDebugEnabled()) { logger.debug("PUT started at: " + startTimeMs + " handlerRef: " - + System.identityHashCode(inputStream) + " key: " - + ByteUtils.toHexString(key.get()) + " " + + System.identityHashCode(this) + " key: " + key + " " + (System.nanoTime() - startTimeNs) + " ns, keySize: " + key.length() - + " valueHash: " + value.hashCode() + " valueSize: " + value.length + + " valueHash: " + value.hashCode() + " valueSize: " + valueSize + " clockSize: " + clock.sizeInBytes() + " time: " + System.currentTimeMillis()); } @@ -485,9 +480,8 @@ private void handleDelete(DataInputStream inputStream, } if(logger.isDebugEnabled()) { - logger.debug("DELETE started at: " + startTimeMs + " key: " - + ByteUtils.toHexString(key.get()) + " handlerRef: " - + System.identityHashCode(inputStream) + " time: " + logger.debug("DELETE started at: " + startTimeMs + " key: " + key + " handlerRef: " + + System.identityHashCode(this) + " time: " + (System.nanoTime() - startTimeNs) + " ns, keySize: " + key.length() + " clockSize: " + version.sizeInBytes()); } diff --git a/src/java/voldemort/store/bdb/BdbStorageEngine.java b/src/java/voldemort/store/bdb/BdbStorageEngine.java index b74e31cfe6..1b2013ca46 100644 --- a/src/java/voldemort/store/bdb/BdbStorageEngine.java +++ b/src/java/voldemort/store/bdb/BdbStorageEngine.java @@ -281,7 +281,7 @@ public Map>> getAll(Iterable keys, for(ByteArray key: keys) { if(logger.isTraceEnabled()) - keyStr += ByteUtils.toHexString(key.get()) + " "; + keyStr += key + " "; List> values = get(cursor, key, readLockMode, versionedSerializer); if(!values.isEmpty()) @@ -324,7 +324,7 @@ private static List get(Cursor cursor, } if(logger.isTraceEnabled()) { - logger.trace("Completed GET from key " + ByteUtils.toHexString(key.get()) + " in " + logger.trace("Completed GET from key " + key + " in " + (System.nanoTime() - startTimeNs) + " ns at " + System.currentTimeMillis()); } @@ -390,7 +390,7 @@ else if(occurred == Occurred.AFTER) } if(logger.isTraceEnabled()) { - logger.trace("Completed PUT to key " + ByteUtils.toHexString(key.get()) + " (keyRef: " + logger.trace("Completed PUT to key " + key + " (keyRef: " + System.identityHashCode(key) + " value " + value + " in " + (System.nanoTime() - startTimeNs) + " ns at " + System.currentTimeMillis()); @@ -431,8 +431,8 @@ public boolean delete(ByteArray key, Version version) throws PersistenceFailureE } finally { if(logger.isTraceEnabled()) { - logger.trace("Completed DELETE of key " + ByteUtils.toHexString(key.get()) - + " (keyRef: " + System.identityHashCode(key) + ") in " + logger.trace("Completed DELETE of key " + key + " (keyRef: " + + System.identityHashCode(key) + ") in " + (System.nanoTime() - startTimeNs) + " ns at " + System.currentTimeMillis()); } diff --git a/src/java/voldemort/store/routed/PipelineRoutedStore.java b/src/java/voldemort/store/routed/PipelineRoutedStore.java index dc8e20265e..dfde9b350c 100644 --- a/src/java/voldemort/store/routed/PipelineRoutedStore.java +++ b/src/java/voldemort/store/routed/PipelineRoutedStore.java @@ -251,10 +251,9 @@ public List> request(Store store) { } if(logger.isDebugEnabled()) { - logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " - + ByteUtils.toHexString(key.get()) + " keyRef: " - + System.identityHashCode(key) + "; started at " + startTimeMs + " took " - + (System.nanoTime() - startTimeNs) + " values: " + logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + key + + " keyRef: " + System.identityHashCode(key) + "; started at " + + startTimeMs + " took " + (System.nanoTime() - startTimeNs) + " values: " + formatNodeValuesFromGet(pipelineData.getResponses())); } @@ -356,10 +355,9 @@ public Map>> getAll(Iterable keys, throw pipelineData.getFatalError(); if(logger.isDebugEnabled()) { - logger.debug("Finished " + pipeline.getOperation().getSimpleName() + "for keys " - + ByteArray.toHexStrings(keys) + " keyRef: " - + System.identityHashCode(keys) + "; started at " + startTimeMs + " took " - + (System.nanoTime() - startTimeNs) + " values: " + logger.debug("Finished " + pipeline.getOperation().getSimpleName() + "for keys " + keys + + " keyRef: " + System.identityHashCode(keys) + "; started at " + + startTimeMs + " took " + (System.nanoTime() - startTimeNs) + " values: " + formatNodeValuesFromGetAll(pipelineData.getResponses())); } @@ -371,9 +369,8 @@ private String formatNodeValuesFromGetAll(List, Map StringBuilder builder = new StringBuilder(); builder.append("{"); for(Response, Map>>> r: list) { - builder.append("(nodeId=" + r.getNode().getId() + ", keys=" - + ByteArray.toHexStrings(r.getKey()) + ", retrieved= " + r.getValue() - + ")"); + builder.append("(nodeId=" + r.getNode().getId() + ", key=" + r.getKey() + + ", retrieved= " + r.getValue() + ")"); builder.append(", "); } builder.append("}"); @@ -472,10 +469,9 @@ public List request(Store store) { results.addAll(response.getValue()); if(logger.isDebugEnabled()) { - logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " - + ByteUtils.toHexString(key.get()) + " keyRef: " - + System.identityHashCode(key) + "; started at " + startTimeMs + " took " - + (System.nanoTime() - startTimeNs) + " values: " + logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + key + + " keyRef: " + System.identityHashCode(key) + "; started at " + + startTimeMs + " took " + (System.nanoTime() - startTimeNs) + " values: " + formatNodeValuesFromGetVersions(pipelineData.getResponses())); } @@ -487,9 +483,8 @@ private String formatNodeValuesFromGetVersions(List> r: results) { - builder.append("(nodeId=" + r.getNode().getId() + ", key=" - + ByteUtils.toHexString(r.getKey().get()) + ", retrieved= " - + r.getValue() + "), "); + builder.append("(nodeId=" + r.getNode().getId() + ", key=" + r.getKey() + + ", retrieved= " + r.getValue() + "), "); } builder.append("}"); @@ -580,9 +575,8 @@ public boolean delete(final ByteArray key, final Version version) throws Voldemo if(logger.isDebugEnabled()) { logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " - + ByteUtils.toHexString(key.get()) + " keyRef: " - + System.identityHashCode(key) + "; started at " + startTimeMs + " took " - + (System.nanoTime() - startTimeNs)); + + key.get() + " keyRef: " + System.identityHashCode(key) + "; started at " + + startTimeMs + " took " + (System.nanoTime() - startTimeNs)); } if(pipelineData.getFatalError() != null) @@ -706,11 +700,10 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) } if(logger.isDebugEnabled()) { - logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " - + ByteUtils.toHexString(key.get()) + " keyRef: " - + System.identityHashCode(key) + "; started at " + startTimeMs + " took " - + (System.nanoTime() - startTimeNs) + " value: " + versioned.getValue() - + " (size: " + versioned.getValue().length + ")"); + logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + key + + " keyRef: " + System.identityHashCode(key) + "; started at " + + startTimeMs + " took " + (System.nanoTime() - startTimeNs) + " value: " + + versioned.getValue() + " (size: " + versioned.getValue().length + ")"); } if(pipelineData.getFatalError() != null) diff --git a/src/java/voldemort/store/routed/action/AbstractReadRepair.java b/src/java/voldemort/store/routed/action/AbstractReadRepair.java index ac2dac7829..6e2ea19d6f 100644 --- a/src/java/voldemort/store/routed/action/AbstractReadRepair.java +++ b/src/java/voldemort/store/routed/action/AbstractReadRepair.java @@ -24,11 +24,10 @@ import voldemort.store.nonblockingstore.NonblockingStore; import voldemort.store.routed.NodeValue; import voldemort.store.routed.Pipeline; -import voldemort.store.routed.Pipeline.Event; import voldemort.store.routed.PipelineData; import voldemort.store.routed.ReadRepairer; +import voldemort.store.routed.Pipeline.Event; import voldemort.utils.ByteArray; -import voldemort.utils.ByteUtils; import voldemort.versioning.VectorClock; import voldemort.versioning.Versioned; @@ -102,8 +101,7 @@ public void execute(Pipeline pipeline) { try { if(logger.isDebugEnabled()) logger.debug("Doing read repair on node " + v.getNodeId() + " for key '" - + ByteUtils.toHexString(v.getKey().get()) + "' with version " - + v.getVersion() + "."); + + v.getKey() + "' with version " + v.getVersion() + "."); NonblockingStore store = nonblockingStores.get(v.getNodeId()); store.submitPutRequest(v.getKey(), v.getVersioned(), null, null, timeoutMs); @@ -112,9 +110,8 @@ public void execute(Pipeline pipeline) { logger.debug("Read repair cancelled due to application level exception on node " + v.getNodeId() + " for key '" - + ByteUtils.toHexString(v.getKey().get()) - + "' with version " - + v.getVersion() + ": " + e.getMessage()); + + v.getKey() + + "' with version " + v.getVersion() + ": " + e.getMessage()); } catch(Exception e) { logger.debug("Read repair failed: ", e); } diff --git a/src/java/voldemort/store/routed/action/PerformParallelDeleteRequests.java b/src/java/voldemort/store/routed/action/PerformParallelDeleteRequests.java index 634b470805..7f30c90a9b 100644 --- a/src/java/voldemort/store/routed/action/PerformParallelDeleteRequests.java +++ b/src/java/voldemort/store/routed/action/PerformParallelDeleteRequests.java @@ -93,9 +93,9 @@ public void execute(final Pipeline pipeline) { public void requestComplete(Object result, long requestTime) { if(logger.isTraceEnabled()) - logger.trace(pipeline.getOperation().getSimpleName() - + " response received (" + requestTime + " ms.) from node " - + node.getId()); + logger.info(pipeline.getOperation().getSimpleName() + + " response received (" + requestTime + " ms.) from node " + + node.getId()); Response response = new Response(node, key, diff --git a/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java b/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java index b2e3bc09be..a8373418b5 100644 --- a/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java +++ b/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java @@ -42,7 +42,6 @@ import voldemort.store.slop.HintedHandoff; import voldemort.store.slop.Slop; import voldemort.utils.ByteArray; -import voldemort.utils.ByteUtils; import voldemort.utils.Time; import voldemort.versioning.ObsoleteVersionException; import voldemort.versioning.Versioned; @@ -131,8 +130,7 @@ public void requestComplete(Object result, long requestTime) { responses.put(node.getId(), response); if(logger.isDebugEnabled()) - logger.debug("Finished secondary PUT for key " - + ByteUtils.toHexString(key.get()) + " (keyRef: " + logger.debug("Finished secondary PUT for key " + key + " (keyRef: " + System.identityHashCode(key) + "); took " + requestTime + " ms on node " + node.getId() + "(" + node.getHost() + ")"); diff --git a/src/java/voldemort/store/routed/action/PerformParallelRequests.java b/src/java/voldemort/store/routed/action/PerformParallelRequests.java index 18385a5f7f..73399d75b0 100644 --- a/src/java/voldemort/store/routed/action/PerformParallelRequests.java +++ b/src/java/voldemort/store/routed/action/PerformParallelRequests.java @@ -38,7 +38,6 @@ import voldemort.store.routed.Pipeline.Operation; import voldemort.store.routed.Response; import voldemort.utils.ByteArray; -import voldemort.utils.ByteUtils; import voldemort.utils.Utils; public class PerformParallelRequests> extends @@ -112,10 +111,10 @@ public void requestComplete(Object result, long requestTime) { requestTime); if(logger.isDebugEnabled()) logger.debug("Finished " + pipeline.getOperation().getSimpleName() - + " for key " + ByteUtils.toHexString(key.get()) - + " (keyRef: " + System.identityHashCode(key) - + "); started at " + startMs + " took " + requestTime - + " ms on node " + node.getId() + "(" + node.getHost() + ")"); + + " for key " + key + " (keyRef: " + + System.identityHashCode(key) + "); started at " + startMs + + " took " + requestTime + " ms on node " + node.getId() + "(" + + node.getHost() + ")"); responses.put(node.getId(), response); latch.countDown(); @@ -175,10 +174,9 @@ else if(pipeline.getOperation() == Operation.GET_VERSIONS) } if(logger.isDebugEnabled()) - logger.debug("GET for key " + ByteUtils.toHexString(key.get()) + " (keyRef: " - + System.identityHashCode(key) + "); successes: " - + pipelineData.getSuccesses() + " preferred: " + preferred + " required: " - + required); + logger.debug("GET for key " + key + " (keyRef: " + System.identityHashCode(key) + + "); successes: " + pipelineData.getSuccesses() + " preferred: " + + preferred + " required: " + required); if(pipelineData.getSuccesses() < required) { if(insufficientSuccessesEvent != null) { diff --git a/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java b/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java index bcfcc820ad..c8135781d4 100644 --- a/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java +++ b/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java @@ -79,11 +79,10 @@ public void execute(Pipeline pipeline) { boolean zoneRequirement = false; MutableInt successCount = pipelineData.getSuccessCount(key); - if(logger.isDebugEnabled()) - logger.debug("GETALL for key " + ByteUtils.toHexString(key.get()) + " (keyRef: " - + System.identityHashCode(key) + ") successes: " - + successCount.intValue() + " preferred: " + preferred + " required: " - + required); + if(logger.isDebugEnabled()) + logger.debug("GETALL for key " + key + " (keyRef: " + System.identityHashCode(key) + + ") successes: " + successCount.intValue() + " preferred: " + preferred + + " required: " + required); if(successCount.intValue() >= preferred) { if(pipelineData.getZonesRequired() != null) { @@ -139,11 +138,11 @@ public void execute(Pipeline pipeline) { failureDetector.recordSuccess(response.getNode(), response.getRequestTime()); if(logger.isDebugEnabled()) - logger.debug("GET for key " + ByteUtils.toHexString(key.get()) - + " (keyRef: " + System.identityHashCode(key) - + ") successes: " + successCount.intValue() + " preferred: " - + preferred + " required: " + required - + " new GET success on node " + node.getId()); + logger.debug("GET for key " + key + " (keyRef: " + + System.identityHashCode(key) + ") successes: " + + successCount.intValue() + " preferred: " + preferred + + " required: " + required + " new GET success on node " + + node.getId()); HashSet zoneResponses = null; if(pipelineData.getKeyToZoneResponse().containsKey(key)) { diff --git a/src/java/voldemort/store/routed/action/PerformSerialPutRequests.java b/src/java/voldemort/store/routed/action/PerformSerialPutRequests.java index 08a25c7bc7..7b7b3d65fd 100644 --- a/src/java/voldemort/store/routed/action/PerformSerialPutRequests.java +++ b/src/java/voldemort/store/routed/action/PerformSerialPutRequests.java @@ -29,7 +29,6 @@ import voldemort.store.routed.Pipeline.Event; import voldemort.store.routed.PutPipelineData; import voldemort.utils.ByteArray; -import voldemort.utils.ByteUtils; import voldemort.utils.Time; import voldemort.versioning.VectorClock; import voldemort.versioning.Versioned; @@ -174,8 +173,7 @@ public void execute(Pipeline pipeline) { } else { if(logger.isDebugEnabled()) - logger.debug("Finished master PUT for key " - + ByteUtils.toHexString(key.get()) + " (keyRef: " + logger.debug("Finished master PUT for key " + key + " (keyRef: " + System.identityHashCode(key) + "); started at " + startMasterMs + " took " + (System.nanoTime() - startMasterNs) + " ns on node " @@ -187,10 +185,10 @@ public void execute(Pipeline pipeline) { } } else { if(logger.isDebugEnabled()) - logger.debug("Finished master PUT for key " + ByteUtils.toHexString(key.get()) - + " (keyRef: " + System.identityHashCode(key) + "); started at " - + startMasterMs + " took " + (System.nanoTime() - startMasterNs) - + " ns on node " + (node == null ? "NULL" : node.getId()) + "(" + logger.debug("Finished master PUT for key " + key + " (keyRef: " + + System.identityHashCode(key) + "); started at " + startMasterMs + + " took " + (System.nanoTime() - startMasterNs) + " ns on node " + + (node == null ? "NULL" : node.getId()) + "(" + (node == null ? "NULL" : node.getHost()) + ")"); pipeline.addEvent(masterDeterminedEvent); diff --git a/src/java/voldemort/store/routed/action/PerformSerialRequests.java b/src/java/voldemort/store/routed/action/PerformSerialRequests.java index 6feafd140c..4c447174d8 100644 --- a/src/java/voldemort/store/routed/action/PerformSerialRequests.java +++ b/src/java/voldemort/store/routed/action/PerformSerialRequests.java @@ -31,7 +31,6 @@ import voldemort.store.routed.Pipeline.Event; import voldemort.store.routed.Response; import voldemort.utils.ByteArray; -import voldemort.utils.ByteUtils; import voldemort.utils.Time; public class PerformSerialRequests> extends @@ -100,10 +99,9 @@ public void execute(Pipeline pipeline) { ((System.nanoTime() - start) / Time.NS_PER_MS)); if(logger.isDebugEnabled()) - logger.debug(pipeline.getOperation().getSimpleName() + " for key " - + ByteUtils.toHexString(key.get()) + " successes: " - + pipelineData.getSuccesses() + " preferred: " + preferred - + " required: " + required + " new " + logger.debug(pipeline.getOperation().getSimpleName() + " for key " + key + + " successes: " + pipelineData.getSuccesses() + " preferred: " + + preferred + " required: " + required + " new " + pipeline.getOperation().getSimpleName() + " success on node " + node.getId()); diff --git a/src/java/voldemort/store/slop/HintedHandoff.java b/src/java/voldemort/store/slop/HintedHandoff.java index ff2d81b823..5e506a85e2 100644 --- a/src/java/voldemort/store/slop/HintedHandoff.java +++ b/src/java/voldemort/store/slop/HintedHandoff.java @@ -27,12 +27,11 @@ import voldemort.serialization.SlopSerializer; import voldemort.store.Store; import voldemort.store.UnreachableStoreException; +import voldemort.store.slop.strategy.HintedHandoffStrategy; import voldemort.store.nonblockingstore.NonblockingStore; import voldemort.store.nonblockingstore.NonblockingStoreCallback; import voldemort.store.routed.Response; -import voldemort.store.slop.strategy.HintedHandoffStrategy; import voldemort.utils.ByteArray; -import voldemort.utils.ByteUtils; import voldemort.utils.Time; import voldemort.utils.Utils; import voldemort.versioning.ObsoleteVersionException; @@ -68,8 +67,7 @@ public class HintedHandoff { * Create a Hinted Handoff object * * @param failureDetector The failure detector - * @param nonblockingSlopStores A map of node ids to nonb-locking slop - * stores + * @param nonblockingSlopStores A map of node ids to nonb-locking slop stores * @param slopStores A map of node ids to blocking slop stores * @param handoffStrategy The {@link HintedHandoffStrategy} implementation * @param failedNodes A list of nodes in the original preflist for the @@ -91,18 +89,15 @@ public HintedHandoff(FailureDetector failureDetector, } /** - * Like - * {@link #sendHintSerial(voldemort.cluster.Node, voldemort.versioning.Version, Slop)} - * , but doesn't block the pipeline. Intended for handling prolonged - * failures without incurring a performance cost. - * - * @see #sendHintSerial(voldemort.cluster.Node, - * voldemort.versioning.Version, Slop) + * Like {@link #sendHintSerial(voldemort.cluster.Node, voldemort.versioning.Version, Slop)}, + * but doesn't block the pipeline. Intended for handling prolonged failures without + * incurring a performance cost. + * + * @see #sendHintSerial(voldemort.cluster.Node, voldemort.versioning.Version, Slop) */ public void sendHintParallel(final Node failedNode, final Version version, final Slop slop) { final ByteArray slopKey = slop.makeKey(); - Versioned slopVersioned = new Versioned(slopSerializer.toBytes(slop), - version); + Versioned slopVersioned = new Versioned(slopSerializer.toBytes(slop), version); for(final Node node: handoffStrategy.routeHint(failedNode)) { int nodeId = node.getId(); @@ -116,12 +111,10 @@ public void sendHintParallel(final Node failedNode, final Version version, final final long startNs = System.nanoTime(); if(logger.isDebugEnabled()) - logger.debug("Slop attempt to write " - + ByteUtils.toHexString(slop.getKey().get()) + " for " - + failedNode + " to node " + node); + logger.debug("Slop attempt to write " + slop.getKey() + " for " + failedNode + + " to node " + node); NonblockingStoreCallback callback = new NonblockingStoreCallback() { - public void requestComplete(Object result, long requestTime) { Response response = new Response(node, slopKey, @@ -137,16 +130,15 @@ public void requestComplete(Object result, long requestTime) { if(response.getValue() instanceof UnreachableStoreException) { UnreachableStoreException use = (UnreachableStoreException) response.getValue(); - if(logger.isDebugEnabled()) - logger.debug("Write of key " - + ByteUtils.toHexString(slop.getKey().get()) - + " for " + failedNode + " to node " + node - + " failed due to unreachable: " - + use.getMessage()); + if(logger.isDebugEnabled()) + logger.debug("Write of key " + slop.getKey() + " for " + + failedNode + " to node " + node + + " failed due to unreachable: " + + use.getMessage()); failureDetector.recordException(node, (System.nanoTime() - startNs) - / Time.NS_PER_MS, + / Time.NS_PER_MS, use); } sendHintSerial(failedNode, version, slop); @@ -155,8 +147,7 @@ public void requestComplete(Object result, long requestTime) { } if(logger.isDebugEnabled()) - logger.debug("Slop write of key " - + ByteUtils.toHexString(slop.getKey().get()) + " for " + logger.debug("Slop write of key " + slop.getKey() + " for " + failedNode + " to node " + node + " succeeded in " + (System.nanoTime() - startNs) + " ns"); @@ -166,12 +157,16 @@ public void requestComplete(Object result, long requestTime) { } }; - nonblockingStore.submitPutRequest(slopKey, slopVersioned, null, callback, timeoutMs); + nonblockingStore.submitPutRequest(slopKey, + slopVersioned, + null, + callback, + timeoutMs); break; } } } - + /** * Send a hint of a request originally meant for the failed node to another * node in the ring, as selected by the {@link HintedHandoffStrategy} @@ -196,8 +191,7 @@ public boolean sendHintSerial(Node failedNode, Version version, Slop slop) { try { if(logger.isDebugEnabled()) - logger.debug("Slop attempt to write " - + ByteUtils.toHexString(slop.getKey().get()) + " (keyRef: " + logger.debug("Slop attempt to write " + slop.getKey() + " (keyRef: " + System.identityHashCode(slop.getKey()) + ") for " + failedNode + " to node " + node); @@ -220,9 +214,9 @@ public boolean sendHintSerial(Node failedNode, Version version, Slop slop) { } if(logger.isDebugEnabled()) - logger.debug("Slop write of key " + ByteUtils.toHexString(slop.getKey().get()) - + " (keyRef: " + System.identityHashCode(slop.getKey()) + " for " - + failedNode + " to node " + node + " succeeded in " + logger.debug("Slop write of key " + slop.getKey() + " (keyRef: " + + System.identityHashCode(slop.getKey()) + " for " + failedNode + + " to node " + node + " succeeded in " + (System.nanoTime() - startNs) + " ns"); } } diff --git a/src/java/voldemort/utils/ByteArray.java b/src/java/voldemort/utils/ByteArray.java index c6ccf86ada..71dc609b2d 100644 --- a/src/java/voldemort/utils/ByteArray.java +++ b/src/java/voldemort/utils/ByteArray.java @@ -1,7 +1,6 @@ package voldemort.utils; import java.io.Serializable; -import java.util.ArrayList; import java.util.Arrays; /** @@ -44,19 +43,6 @@ public String toString() { return Arrays.toString(underlying); } - /** - * Translate the each ByteArray in an iterable into a hexidecimal string - * - * @param arrays The array of bytes to translate - * @return An iterable of converted strings - */ - public static Iterable toHexStrings(Iterable arrays) { - ArrayList ret = new ArrayList(); - for(ByteArray array: arrays) - ret.add(ByteUtils.toHexString(array.get())); - return ret; - } - public int length() { return underlying.length; } From 9bafcc84547b9c18ec0d1ae88d91c9e0e8ec34b6 Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Mon, 27 Aug 2012 14:17:34 -0700 Subject: [PATCH 114/209] - 'retention-frequency' to control frequency for running DataCleanupJob - Use same jmxid as the factory across the board - add server config to control socket backlog - add count() jmx call to obtain number of k-v pairs from store --- .../client/AbstractStoreClientFactory.java | 21 +- .../client/SocketStoreClientFactory.java | 5 +- .../voldemort/server/VoldemortConfig.java | 11 + .../voldemort/server/VoldemortServer.java | 6 +- .../server/niosocket/NioSocketService.java | 8 +- .../server/storage/StorageService.java | 8 +- src/java/voldemort/store/StoreDefinition.java | 11 + .../store/StoreDefinitionBuilder.java | 11 + .../store/bdb/BdbStorageConfiguration.java | 16 ++ .../store/routed/PipelineRoutedStore.java | 17 +- .../store/routed/RoutedStoreFactory.java | 9 +- .../ClientRequestExecutorPool.java | 11 +- .../store/stats/ClientSocketStats.java | 14 +- src/java/voldemort/utils/JmxUtils.java | 13 +- src/java/voldemort/utils/Time.java | 3 +- .../voldemort/xml/StoreDefinitionsMapper.java | 6 + src/java/voldemort/xml/stores.xsd | 2 + test/common/voldemort/ServerTestUtils.java | 3 +- test/common/voldemort/TestUtils.java | 3 +- .../voldemort/VoldemortTestConstants.java | 5 + test/common/voldemort/config/two-stores.xml | 39 ++++ .../AbstractStoreClientFactoryTest.java | 2 +- .../SocketStoreClientFactoryMbeanTest.java | 213 ++++++++++++++++++ .../scheduled/DataCleanupJobTest.java | 131 ++++++++++- .../socket/ClientRequestExecutorPoolTest.java | 3 +- .../store/stats/ClientSocketStatsTest.java | 4 +- 26 files changed, 510 insertions(+), 65 deletions(-) create mode 100644 test/common/voldemort/config/two-stores.xml create mode 100644 test/unit/voldemort/client/SocketStoreClientFactoryMbeanTest.java diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index 55d41f5e6f..830a5d20cf 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -91,7 +91,7 @@ public abstract class AbstractStoreClientFactory implements StoreClientFactory { private final SerializerFactory serializerFactory; private final boolean isJmxEnabled; private final RequestFormatType requestFormatType; - private final int jmxId; + protected final int jmxId; protected volatile FailureDetector failureDetector; private final int maxBootstrapRetries; private final StoreStats stats; @@ -120,10 +120,11 @@ public AbstractStoreClientFactory(ClientConfig config) { JmxUtils.registerMbean(threadPool, JmxUtils.createObjectName(JmxUtils.getPackageName(threadPool.getClass()), JmxUtils.getClassName(threadPool.getClass()) - + jmxId())); + + JmxUtils.getJmxId(jmxId))); JmxUtils.registerMbean(new StoreStatsJmx(stats), JmxUtils.createObjectName("voldemort.store.stats.aggregate", - "aggregate-perf" + jmxId())); + "aggregate-perf" + + JmxUtils.getJmxId(jmxId))); } } @@ -215,7 +216,8 @@ public Store getRawStore(String storeName, repairReads, clientZoneId, getFailureDetector(), - isJmxEnabled); + isJmxEnabled, + jmxId); store = new LoggingStore(store); if(isJmxEnabled) { @@ -223,7 +225,8 @@ public Store getRawStore(String storeName, store = statStore; JmxUtils.registerMbean(new StoreStatsJmx(statStore.getStats()), JmxUtils.createObjectName(JmxUtils.getPackageName(store.getClass()), - store.getName() + jmxId())); + store.getName() + + JmxUtils.getJmxId(jmxId))); } if(storeDef.getKeySerializer().hasCompression() @@ -279,7 +282,7 @@ public FailureDetector getFailureDetector() { JmxUtils.registerMbean(failureDetector, JmxUtils.createObjectName(JmxUtils.getPackageName(failureDetector.getClass()), JmxUtils.getClassName(failureDetector.getClass()) - + jmxId())); + + JmxUtils.getJmxId(jmxId))); } } } @@ -403,10 +406,4 @@ public void close() { if(failureDetector != null) failureDetector.destroy(); } - - /* Give a unique id to avoid jmx clashes */ - public String jmxId() { - return jmxId == 0 ? "" : Integer.toString(jmxId); - } - } diff --git a/src/java/voldemort/client/SocketStoreClientFactory.java b/src/java/voldemort/client/SocketStoreClientFactory.java index c5c7336f5d..3dce3e9471 100644 --- a/src/java/voldemort/client/SocketStoreClientFactory.java +++ b/src/java/voldemort/client/SocketStoreClientFactory.java @@ -67,7 +67,8 @@ public SocketStoreClientFactory(ClientConfig config) { config.getSocketTimeout(TimeUnit.MILLISECONDS), config.getSocketBufferSize(), config.getSocketKeepAlive(), - config.isJmxEnabled()); + config.isJmxEnabled(), + jmxId); } @Override @@ -149,7 +150,7 @@ protected Store getStoreInternal(Node node) { FailureDetectorConfig failureDetectorConfig = new FailureDetectorConfig(config).setNodes(nodes) .setStoreVerifier(storeVerifier); - return create(failureDetectorConfig, true, failureDetectorListener); + return create(failureDetectorConfig, false, failureDetectorListener); } @Override diff --git a/src/java/voldemort/server/VoldemortConfig.java b/src/java/voldemort/server/VoldemortConfig.java index 3fc9ffc154..e1626e0a81 100644 --- a/src/java/voldemort/server/VoldemortConfig.java +++ b/src/java/voldemort/server/VoldemortConfig.java @@ -113,6 +113,7 @@ public class VoldemortConfig implements Serializable { private boolean useNioConnector; private int nioConnectorSelectors; private int nioAdminConnectorSelectors; + private int nioAcceptorBacklog; private int clientSelectors; private int clientRoutingTimeoutMs; @@ -277,6 +278,8 @@ public VoldemortConfig(Props props) { this.nioAdminConnectorSelectors = props.getInt("nio.admin.connector.selectors", Math.max(8, Runtime.getRuntime() .availableProcessors())); + // a value <= 0 forces the default to be used + this.nioAcceptorBacklog = props.getInt("nio.acceptor.backlog", -1); this.clientSelectors = props.getInt("client.selectors", 4); this.clientMaxConnectionsPerNode = props.getInt("client.max.connections.per.node", 50); @@ -1288,6 +1291,14 @@ public void setNioAdminConnectorSelectors(int nioAdminConnectorSelectors) { this.nioAdminConnectorSelectors = nioAdminConnectorSelectors; } + public int getNioAcceptorBacklog() { + return nioAcceptorBacklog; + } + + public void setNioAcceptorBacklog(int nioAcceptorBacklog) { + this.nioAcceptorBacklog = nioAcceptorBacklog; + } + public int getAdminSocketBufferSize() { return adminStreamBufferSize; } diff --git a/src/java/voldemort/server/VoldemortServer.java b/src/java/voldemort/server/VoldemortServer.java index 9b20116c8a..0ec64999a9 100644 --- a/src/java/voldemort/server/VoldemortServer.java +++ b/src/java/voldemort/server/VoldemortServer.java @@ -183,7 +183,8 @@ private List createServices() { voldemortConfig.getSocketBufferSize(), voldemortConfig.getNioConnectorSelectors(), "nio-socket-server", - voldemortConfig.isJmxEnabled())); + voldemortConfig.isJmxEnabled(), + voldemortConfig.getNioAcceptorBacklog())); } else { logger.info("Using BIO Connector."); services.add(new SocketService(socketRequestHandlerFactory, @@ -222,7 +223,8 @@ private List createServices() { voldemortConfig.getAdminSocketBufferSize(), voldemortConfig.getNioAdminConnectorSelectors(), "admin-server", - voldemortConfig.isJmxEnabled())); + voldemortConfig.isJmxEnabled(), + voldemortConfig.getNioAcceptorBacklog())); } else { logger.info("Using BIO Connector for Admin Service."); services.add(new SocketService(adminRequestHandlerFactory, diff --git a/src/java/voldemort/server/niosocket/NioSocketService.java b/src/java/voldemort/server/niosocket/NioSocketService.java index b292fba739..a0a18fa5b1 100644 --- a/src/java/voldemort/server/niosocket/NioSocketService.java +++ b/src/java/voldemort/server/niosocket/NioSocketService.java @@ -72,6 +72,8 @@ public class NioSocketService extends AbstractSocketService { private final int socketBufferSize; + private final int acceptorBacklog; + private final StatusManager statusManager; private final Thread acceptorThread; @@ -83,10 +85,12 @@ public NioSocketService(RequestHandlerFactory requestHandlerFactory, int socketBufferSize, int selectors, String serviceName, - boolean enableJmx) { + boolean enableJmx, + int acceptorBacklog) { super(ServiceType.SOCKET, port, serviceName, enableJmx); this.requestHandlerFactory = requestHandlerFactory; this.socketBufferSize = socketBufferSize; + this.acceptorBacklog = acceptorBacklog; try { this.serverSocketChannel = ServerSocketChannel.open(); @@ -122,7 +126,7 @@ protected void startInner() { selectorManagerThreadPool.execute(selectorManagers[i]); } - serverSocketChannel.socket().bind(endpoint); + serverSocketChannel.socket().bind(endpoint, acceptorBacklog); serverSocketChannel.socket().setReceiveBufferSize(socketBufferSize); serverSocketChannel.socket().setReuseAddress(true); diff --git a/src/java/voldemort/server/storage/StorageService.java b/src/java/voldemort/server/storage/StorageService.java index 1ef2c46442..53dd14731e 100644 --- a/src/java/voldemort/server/storage/StorageService.java +++ b/src/java/voldemort/server/storage/StorageService.java @@ -239,6 +239,7 @@ protected void startInner() { null, null, null, + null, 0); SlopStorageEngine slopEngine = new SlopStorageEngine(config.getStore(slopStoreDefinition), metadata.getCluster()); @@ -631,12 +632,13 @@ private void scheduleCleanupJob(StoreDefinition storeDef, JmxUtils.registerMbean("DataCleanupJob-" + engine.getName(), cleanupJob); } + long retentionFreqHours = storeDef.hasRetentionFrequencyDays() ? (storeDef.getRetentionFrequencyDays() * Time.HOURS_PER_DAY) + : voldemortConfig.getRetentionCleanupScheduledPeriodInHour(); + this.scheduler.schedule("cleanup-" + storeDef.getName(), cleanupJob, startTime, - voldemortConfig.getRetentionCleanupScheduledPeriodInHour() - * Time.MS_PER_HOUR); - + retentionFreqHours * Time.MS_PER_HOUR); } @Override diff --git a/src/java/voldemort/store/StoreDefinition.java b/src/java/voldemort/store/StoreDefinition.java index ab9593328b..d28a8edddc 100644 --- a/src/java/voldemort/store/StoreDefinition.java +++ b/src/java/voldemort/store/StoreDefinition.java @@ -50,6 +50,7 @@ public class StoreDefinition implements Serializable { private final int requiredReads; private final Integer retentionPeriodDays; private final Integer retentionScanThrottleRate; + private final Integer retentionFrequencyDays; private final String routingStrategyType; private final String viewOf; private final HashMap zoneReplicationFactor; @@ -82,6 +83,7 @@ public StoreDefinition(String name, Integer zoneCountWrites, Integer retentionDays, Integer retentionThrottleRate, + Integer retentionFrequencyDays, String factory, HintedHandoffStrategyType hintedHandoffStrategyType, Integer hintPrefListSize, @@ -101,6 +103,7 @@ public StoreDefinition(String name, this.transformsSerializer = transformsSerializer; this.retentionPeriodDays = retentionDays; this.retentionScanThrottleRate = retentionThrottleRate; + this.retentionFrequencyDays = retentionFrequencyDays; this.memoryFootprintMB = memoryFootprintMB; this.routingStrategyType = routingStrategyType; this.viewOf = viewOfStore; @@ -282,6 +285,14 @@ public Integer getRetentionScanThrottleRate() { return this.retentionScanThrottleRate; } + public boolean hasRetentionFrequencyDays() { + return this.retentionFrequencyDays != null; + } + + public Integer getRetentionFrequencyDays() { + return this.retentionFrequencyDays; + } + public boolean isView() { return this.viewOf != null; } diff --git a/src/java/voldemort/store/StoreDefinitionBuilder.java b/src/java/voldemort/store/StoreDefinitionBuilder.java index 95900b4935..ba426e1aba 100644 --- a/src/java/voldemort/store/StoreDefinitionBuilder.java +++ b/src/java/voldemort/store/StoreDefinitionBuilder.java @@ -30,6 +30,7 @@ public class StoreDefinitionBuilder { private int requiredReads = -1; private Integer retentionPeriodDays = null; private Integer retentionScanThrottleRate = null; + private Integer retentionFrequencyDays = null; private String routingStrategyType = null; private String viewOf = null; private HashMap zoneReplicationFactor = null; @@ -180,6 +181,15 @@ public StoreDefinitionBuilder setRetentionScanThrottleRate(Integer retentionScan return this; } + public Integer getRetentionFrequencyDays() { + return this.retentionFrequencyDays; + } + + public StoreDefinitionBuilder setRetentionFrequencyDays(Integer retentionFreqDays) { + this.retentionFrequencyDays = retentionFreqDays; + return this; + } + public String getRoutingStrategyType() { return routingStrategyType; } @@ -304,6 +314,7 @@ public StoreDefinition build() { this.getZoneCountWrites(), this.getRetentionPeriodDays(), this.getRetentionScanThrottleRate(), + this.getRetentionFrequencyDays(), this.getSerializerFactory(), this.getHintedHandoffStrategy(), this.getHintPrefListSize(), diff --git a/src/java/voldemort/store/bdb/BdbStorageConfiguration.java b/src/java/voldemort/store/bdb/BdbStorageConfiguration.java index bf956e16c6..512daf4065 100644 --- a/src/java/voldemort/store/bdb/BdbStorageConfiguration.java +++ b/src/java/voldemort/store/bdb/BdbStorageConfiguration.java @@ -294,6 +294,22 @@ public void cleanLogs() { } } + @JmxOperation(description = "Obtain the number of k-v entries in the store") + public long getEntryCount(String storeName) throws Exception { + Environment storeEnv = environments.get(storeName); + if(storeEnv != null) { + Database storeDb = null; + try { + storeDb = storeEnv.openDatabase(null, storeName, databaseConfig); + return storeDb.count(); + } finally { + if(storeDb != null) + storeDb.close(); + } + } + return 0; + } + public void close() { synchronized(lock) { try { diff --git a/src/java/voldemort/store/routed/PipelineRoutedStore.java b/src/java/voldemort/store/routed/PipelineRoutedStore.java index dfde9b350c..45c6011329 100644 --- a/src/java/voldemort/store/routed/PipelineRoutedStore.java +++ b/src/java/voldemort/store/routed/PipelineRoutedStore.java @@ -21,7 +21,6 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import voldemort.VoldemortException; import voldemort.client.TimeoutConfig; @@ -70,8 +69,6 @@ */ public class PipelineRoutedStore extends RoutedStore { - private static AtomicInteger jmxIdCounter = new AtomicInteger(0); - private final Map nonblockingStores; private final Map> slopStores; private final Map nonblockingSlopStores; @@ -79,7 +76,6 @@ public class PipelineRoutedStore extends RoutedStore { private Zone clientZone; private boolean zoneRoutingEnabled; private PipelineRoutedStats stats; - private final int jmxId; /** * Create a PipelineRoutedStore @@ -94,6 +90,8 @@ public class PipelineRoutedStore extends RoutedStore { * @param clientZoneId Zone the client is in * @param timeoutMs Routing timeout * @param failureDetector Failure detector object + * @param jmxEnabled is monitoring enabled + * @param jmxId unique ID for the factory instance */ public PipelineRoutedStore(String name, Map> innerStores, @@ -106,7 +104,8 @@ public PipelineRoutedStore(String name, int clientZoneId, TimeoutConfig timeoutConfig, FailureDetector failureDetector, - boolean jmxEnabled) { + boolean jmxEnabled, + int jmxId) { super(name, innerStores, cluster, @@ -122,7 +121,6 @@ public PipelineRoutedStore(String name, } else { zoneRoutingEnabled = false; } - this.jmxId = jmxIdCounter.getAndIncrement(); this.nonblockingStores = new ConcurrentHashMap(nonblockingStores); this.slopStores = slopStores; if(storeDef.hasHintedHandoffStrategyType()) { @@ -137,7 +135,7 @@ public PipelineRoutedStore(String name, stats = new PipelineRoutedStats(); JmxUtils.registerMbean(stats, JmxUtils.createObjectName(JmxUtils.getPackageName(stats.getClass()), - getName() + jmxId())); + getName() + JmxUtils.getJmxId(jmxId))); } } @@ -727,9 +725,4 @@ public void close() { super.close(); } - - /* Give a unique id to avoid jmx clashes */ - private String jmxId() { - return jmxId == 0 ? "" : Integer.toString(jmxId); - } } diff --git a/src/java/voldemort/store/routed/RoutedStoreFactory.java b/src/java/voldemort/store/routed/RoutedStoreFactory.java index 24b9210c82..66486cc21d 100644 --- a/src/java/voldemort/store/routed/RoutedStoreFactory.java +++ b/src/java/voldemort/store/routed/RoutedStoreFactory.java @@ -68,7 +68,8 @@ public RoutedStore create(Cluster cluster, repairReads, clientZoneId, failureDetector, - false); + false, + 0); } public RoutedStore create(Cluster cluster, @@ -80,7 +81,8 @@ public RoutedStore create(Cluster cluster, boolean repairReads, int clientZoneId, FailureDetector failureDetector, - boolean jmxEnabled) { + boolean jmxEnabled, + int jmxId) { if(isPipelineRoutedStoreEnabled) { return new PipelineRoutedStore(storeDefinition.getName(), nodeStores, @@ -93,7 +95,8 @@ public RoutedStore create(Cluster cluster, clientZoneId, timeoutConfig, failureDetector, - jmxEnabled); + jmxEnabled, + jmxId); } else { if(storeDefinition.getRoutingStrategyType() .compareTo(RoutingStrategyType.ZONE_STRATEGY) == 0) { diff --git a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java index 1c7ee64bc3..76011759cf 100644 --- a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java +++ b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java @@ -56,17 +56,19 @@ public ClientRequestExecutorPool(int selectors, int soTimeoutMs, int socketBufferSize, boolean socketKeepAlive, - boolean enableJmx) { + boolean enableJmx, + int jmxId) { ResourcePoolConfig config = new ResourcePoolConfig().setIsFair(true) .setMaxPoolSize(maxConnectionsPerNode) .setMaxInvalidAttempts(maxConnectionsPerNode) .setTimeout(connectionTimeoutMs, TimeUnit.MILLISECONDS); if(enableJmx) { - stats = new ClientSocketStats(); + stats = new ClientSocketStats(jmxId); JmxUtils.registerMbean(new ClientSocketStatsJmx(stats), JmxUtils.createObjectName(JmxUtils.getPackageName(this.getClass()), - "aggregated")); + "aggregated" + + JmxUtils.getJmxId(jmxId))); } else { stats = null; } @@ -95,7 +97,8 @@ public ClientRequestExecutorPool(int selectors, soTimeoutMs, socketBufferSize, socketKeepAlive, - false); + false, + 0); } public ClientRequestExecutorPool(int maxConnectionsPerNode, diff --git a/src/java/voldemort/store/stats/ClientSocketStats.java b/src/java/voldemort/store/stats/ClientSocketStats.java index 721f67a802..335ca8302c 100644 --- a/src/java/voldemort/store/stats/ClientSocketStats.java +++ b/src/java/voldemort/store/stats/ClientSocketStats.java @@ -47,6 +47,8 @@ public class ClientSocketStats { private final AtomicInteger connectionsDestroyed = new AtomicInteger(0); private final AtomicInteger connectionsCheckedout = new AtomicInteger(0); + private final int jmxId; + /** * To construct a per node stats object * @@ -57,11 +59,13 @@ public class ClientSocketStats { */ public ClientSocketStats(ClientSocketStats parent, SocketDestination destination, - KeyedResourcePool pool) { + KeyedResourcePool pool, + int jmxId) { this.parent = parent; this.statsMap = null; this.destination = destination; this.pool = pool; + this.jmxId = jmxId; } /** @@ -69,11 +73,12 @@ public ClientSocketStats(ClientSocketStats parent, * * @param pool The socket pool that will give out connection information */ - public ClientSocketStats() { + public ClientSocketStats(int jmxId) { this.parent = null; this.statsMap = new ConcurrentHashMap(); this.destination = null; this.pool = null; + this.jmxId = jmxId; } /* get per node stats, create one if not exist */ @@ -83,14 +88,15 @@ private ClientSocketStats getOrCreateNodeStats(SocketDestination destination) { } ClientSocketStats stats = statsMap.get(destination); if(stats == null) { - stats = new ClientSocketStats(this, destination, pool); + stats = new ClientSocketStats(this, destination, pool, jmxId); statsMap.putIfAbsent(destination, stats); stats = statsMap.get(destination); JmxUtils.registerMbean(new ClientSocketStatsJmx(stats), JmxUtils.createObjectName(JmxUtils.getPackageName(ClientRequestExecutor.class), "stats_" + destination.toString() - .replace(':', '_'))); + .replace(':', '_') + + JmxUtils.getJmxId(jmxId))); } return stats; } diff --git a/src/java/voldemort/utils/JmxUtils.java b/src/java/voldemort/utils/JmxUtils.java index 87cb62423c..844743ec22 100644 --- a/src/java/voldemort/utils/JmxUtils.java +++ b/src/java/voldemort/utils/JmxUtils.java @@ -134,8 +134,7 @@ public static ModelMBeanOperationInfo[] extractOperationInfo(Object object) { description, extractParameterInfo(m), m.getReturnType() - .getName(), - impact); + .getName(), impact); info.getDescriptor().setField("visibility", Integer.toString(visibility)); infos.add(info); } @@ -350,4 +349,14 @@ public static void unregisterMbean(ObjectName name) { } } + /** + * Return the string representation of jmxId + * + * @param jmxId + * @return + */ + public static String getJmxId(int jmxId) { + return jmxId == 0 ? "" : Integer.toString(jmxId); + } + } diff --git a/src/java/voldemort/utils/Time.java b/src/java/voldemort/utils/Time.java index 8b5e407f86..fd27426b1c 100644 --- a/src/java/voldemort/utils/Time.java +++ b/src/java/voldemort/utils/Time.java @@ -25,6 +25,7 @@ */ public interface Time { + public final static long HOURS_PER_DAY = 24; public final static long US_PER_MS = 1000; public final static long NS_PER_US = 1000; public final static long NS_PER_MS = US_PER_MS * NS_PER_US; @@ -32,7 +33,7 @@ public interface Time { public final static long US_PER_SECOND = US_PER_MS * MS_PER_SECOND; public final static long NS_PER_SECOND = NS_PER_US * US_PER_SECOND; public final static long SECONDS_PER_HOUR = 60 * 60; - public final static long SECONDS_PER_DAY = 24 * SECONDS_PER_HOUR; + public final static long SECONDS_PER_DAY = HOURS_PER_DAY * SECONDS_PER_HOUR; public final static long MS_PER_HOUR = SECONDS_PER_HOUR * MS_PER_SECOND; public final static long MS_PER_DAY = SECONDS_PER_DAY * MS_PER_SECOND; diff --git a/src/java/voldemort/xml/StoreDefinitionsMapper.java b/src/java/voldemort/xml/StoreDefinitionsMapper.java index 0926129834..8c15ff728e 100644 --- a/src/java/voldemort/xml/StoreDefinitionsMapper.java +++ b/src/java/voldemort/xml/StoreDefinitionsMapper.java @@ -83,6 +83,7 @@ public class StoreDefinitionsMapper { public final static String STORE_REQUIRED_READS_ELMT = "required-reads"; public final static String STORE_PREFERRED_READS_ELMT = "preferred-reads"; public final static String STORE_RETENTION_POLICY_ELMT = "retention-days"; + public final static String STORE_RETENTION_FREQ_ELMT = "retention-frequency"; public final static String STORE_RETENTION_SCAN_THROTTLE_RATE_ELMT = "retention-scan-throttle-rate"; public final static String STORE_ROUTING_STRATEGY = "routing-strategy"; public final static String STORE_ZONE_ID_ELMT = "zone-id"; @@ -221,11 +222,15 @@ private StoreDefinition readStore(Element store) { Element retention = store.getChild(STORE_RETENTION_POLICY_ELMT); Integer retentionPolicyDays = null; Integer retentionThrottleRate = null; + Integer retentionFreqDays = null; if(retention != null) { retentionPolicyDays = Integer.parseInt(retention.getText()); Element throttleRate = store.getChild(STORE_RETENTION_SCAN_THROTTLE_RATE_ELMT); if(throttleRate != null) retentionThrottleRate = Integer.parseInt(throttleRate.getText()); + Element retentionFreqElement = store.getChild(STORE_RETENTION_FREQ_ELMT); + if(retentionFreqElement != null) + retentionFreqDays = Integer.parseInt(retentionFreqElement.getText()); } if(routingStrategyType.compareTo(RoutingStrategyType.ZONE_STRATEGY) == 0) { @@ -269,6 +274,7 @@ private StoreDefinition readStore(Element store) { .setRequiredWrites(requiredWrites) .setRetentionPeriodDays(retentionPolicyDays) .setRetentionScanThrottleRate(retentionThrottleRate) + .setRetentionFrequencyDays(retentionFreqDays) .setZoneReplicationFactor(zoneReplicationFactor) .setZoneCountReads(zoneCountReads) .setZoneCountWrites(zoneCountWrites) diff --git a/src/java/voldemort/xml/stores.xsd b/src/java/voldemort/xml/stores.xsd index 8dbd9e930d..f9cd79a3d5 100644 --- a/src/java/voldemort/xml/stores.xsd +++ b/src/java/voldemort/xml/stores.xsd @@ -42,6 +42,8 @@ + diff --git a/test/common/voldemort/ServerTestUtils.java b/test/common/voldemort/ServerTestUtils.java index 9dfe1418a8..78537a8a52 100644 --- a/test/common/voldemort/ServerTestUtils.java +++ b/test/common/voldemort/ServerTestUtils.java @@ -146,7 +146,8 @@ public static AbstractSocketService getSocketService(boolean useNio, bufferSize, coreConnections, "client-request-service", - false); + false, + -1); } else { socketService = new SocketService(requestHandlerFactory, port, diff --git a/test/common/voldemort/TestUtils.java b/test/common/voldemort/TestUtils.java index fadbb64dd1..9d0a6e2612 100644 --- a/test/common/voldemort/TestUtils.java +++ b/test/common/voldemort/TestUtils.java @@ -369,6 +369,7 @@ public static StoreDefinition makeStoreDefinition(String storeName) { null, null, null, + null, 0); } @@ -401,7 +402,7 @@ public static StoreDefinition makeStoreDefinition(String storeName, long memFoot null, null, null, + null, memFootprintMB); } - } diff --git a/test/common/voldemort/VoldemortTestConstants.java b/test/common/voldemort/VoldemortTestConstants.java index 34ac46cdaa..723e95fa6e 100644 --- a/test/common/voldemort/VoldemortTestConstants.java +++ b/test/common/voldemort/VoldemortTestConstants.java @@ -42,6 +42,10 @@ public static String getSingleStoreDefinitionsXml() { return readString("config/single-store.xml"); } + public static String getTwoStoreDefinitionsXml() { + return readString("config/two-stores.xml"); + } + public static String getNoVersionStoreDefinitionsXml() { return readString("config/no-version-store.xml"); } @@ -65,6 +69,7 @@ public static Cluster getTwoNodeCluster() { public static String getTenNodeClusterXml() { return readString("config/ten-node-cluster.xml"); } + public static String getNineNodeClusterXml() { return readString("config/nine-node-cluster.xml"); } diff --git a/test/common/voldemort/config/two-stores.xml b/test/common/voldemort/config/two-stores.xml new file mode 100644 index 0000000000..360e48b422 --- /dev/null +++ b/test/common/voldemort/config/two-stores.xml @@ -0,0 +1,39 @@ + + + + test + bdb + client + 1 + 1 + 1 + 1 + 1 + + string + UTF-8 + + + string + UTF-8 + + + + best + bdb + client + 1 + 1 + 1 + 1 + 1 + + string + UTF-8 + + + string + UTF-8 + + + diff --git a/test/unit/voldemort/client/AbstractStoreClientFactoryTest.java b/test/unit/voldemort/client/AbstractStoreClientFactoryTest.java index 3ed171e520..0906a1b2e8 100644 --- a/test/unit/voldemort/client/AbstractStoreClientFactoryTest.java +++ b/test/unit/voldemort/client/AbstractStoreClientFactoryTest.java @@ -46,7 +46,7 @@ public abstract class AbstractStoreClientFactoryTest extends TestCase { @Override @Before public void setUp() throws Exception { - this.storeDefinitionXml = VoldemortTestConstants.getSingleStoreDefinitionsXml(); + this.storeDefinitionXml = VoldemortTestConstants.getTwoStoreDefinitionsXml(); this.cluster = ServerTestUtils.getLocalCluster(1); this.node = cluster.getNodes().iterator().next(); } diff --git a/test/unit/voldemort/client/SocketStoreClientFactoryMbeanTest.java b/test/unit/voldemort/client/SocketStoreClientFactoryMbeanTest.java new file mode 100644 index 0000000000..dc77b6ddf4 --- /dev/null +++ b/test/unit/voldemort/client/SocketStoreClientFactoryMbeanTest.java @@ -0,0 +1,213 @@ +package voldemort.client; + +import java.lang.management.ManagementFactory; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Set; + +import javax.management.MBeanServer; +import javax.management.ObjectName; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; + +import voldemort.utils.JmxUtils; + +/** + * + * Smoke test to see how many Mbeans we create in each monitoring domain + * + */ +public class SocketStoreClientFactoryMbeanTest extends SocketStoreClientFactoryTest { + + // there should one of these per store (that has a store client), per + // factory + private static String STATS_DOMAIN = "voldemort.store.stats"; + private static String AGGREGATE_STATS_DOMAIN = "voldemort.store.stats.aggregate"; + private static String PIPELINE_ROUTED_STATS_DOMAIN = "voldemort.store.routed"; + + // there should one of these per factory + private static String CLIENT_DOMAIN = "voldemort.client"; + private static String CLUSTER_FAILUREDETECTOR_DOMAIN = "voldemort.cluster.failuredetector"; + + // there should be one of these per factory per host in the cluster the + // factory talks to (plus one aggregate) + private static String CLIENT_REQUEST_DOMAIN = "voldemort.store.socket.clientrequest"; + + private MBeanServer mbServer = null; + // list of factory objects to be closed at the end. + private List factories; + + public SocketStoreClientFactoryMbeanTest(boolean useNio, boolean useLazy) { + super(useNio, useLazy); + } + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + mbServer = ManagementFactory.getPlatformMBeanServer(); + factories = new ArrayList(); + } + + @Override + @After + public void tearDown() throws Exception { + mbServer = null; + for(StoreClientFactory factory: factories) + factory.close(); + + super.tearDown(); + } + + @Parameters + public static Collection configs() { + return Arrays.asList(new Object[][] { { true, false } }); + } + + private void checkMbeanIdCount(String domain, String type, int maxMbeans, boolean unregister) { + ObjectName oName = JmxUtils.createObjectName(domain, type); + Set objects = mbServer.queryNames(oName, null); + assertFalse("Extra mbeans found", objects.size() > maxMbeans); + assertFalse("Fewer than expected mbeans found", objects.size() < maxMbeans); + + if(unregister) { + try { + for(ObjectName objName: objects) + mbServer.unregisterMBean(objName); + } catch(Exception e) { + fail("Problem unregistering mbeans " + e.getMessage()); + } + } + } + + private void bootStrap(List> clients, int n) { + for(int i = 0; i < n; i++) { + for(DefaultStoreClient client: clients) + client.bootStrap(); + } + } + + @Test + public void testMultipleDistinctClientsOnSingleFactory() { + try { + StoreClientFactory factory = getFactory(getValidBootstrapUrl()); + List> clients = new ArrayList>(); + + clients.add((DefaultStoreClient) factory.getStoreClient("test")); + clients.add((DefaultStoreClient) factory.getStoreClient("best")); + factories.add(factory); + + // bootstrap a number of times + bootStrap(clients, 10); + + checkMbeanIdCount(CLIENT_DOMAIN, "ClientThreadPool*", 1, true); + checkMbeanIdCount(CLIENT_DOMAIN, "DefaultStoreClient*", 2, true); + checkMbeanIdCount(CLUSTER_FAILUREDETECTOR_DOMAIN, "ThresholdFailureDetector*", 1, true); + checkMbeanIdCount(PIPELINE_ROUTED_STATS_DOMAIN, "*", 2, true); + checkMbeanIdCount(CLIENT_REQUEST_DOMAIN, "aggregated*", 1, true); + checkMbeanIdCount(CLIENT_REQUEST_DOMAIN, "stats_localhost*", 1, true); + checkMbeanIdCount(AGGREGATE_STATS_DOMAIN, "aggregate-perf*", 1, true); + checkMbeanIdCount(STATS_DOMAIN, "*", 2, true); + + } catch(Exception e) { + fail("Unexpected error " + e.getMessage()); + } + } + + @Test + public void testMultipleIndistinctClientsOnSingleFactory() { + try { + StoreClientFactory factory = getFactory(getValidBootstrapUrl()); + List> clients = new ArrayList>(); + + clients.add((DefaultStoreClient) factory.getStoreClient("test")); + clients.add((DefaultStoreClient) factory.getStoreClient("best")); + clients.add((DefaultStoreClient) factory.getStoreClient("test")); + clients.add((DefaultStoreClient) factory.getStoreClient("best")); + factories.add(factory); + + // bootstrap a number of times + bootStrap(clients, 10); + + checkMbeanIdCount(CLIENT_DOMAIN, "ClientThreadPool*", 1, true); + checkMbeanIdCount(CLIENT_DOMAIN, "DefaultStoreClient*", 2, true); + checkMbeanIdCount(CLUSTER_FAILUREDETECTOR_DOMAIN, "ThresholdFailureDetector*", 1, true); + checkMbeanIdCount(PIPELINE_ROUTED_STATS_DOMAIN, "*", 2, true); + checkMbeanIdCount(CLIENT_REQUEST_DOMAIN, "aggregated*", 1, true); + checkMbeanIdCount(CLIENT_REQUEST_DOMAIN, "stats_localhost*", 1, true); + checkMbeanIdCount(AGGREGATE_STATS_DOMAIN, "aggregate-perf*", 1, true); + checkMbeanIdCount(STATS_DOMAIN, "*", 2, true); + + } catch(Exception e) { + fail("Unexpected error " + e.getMessage()); + } + } + + @Test + public void testMultipleDistinctClientsOnMultipleFactories() { + try { + StoreClientFactory testfactory = getFactory(getValidBootstrapUrl()); + List> clients = new ArrayList>(); + clients.add((DefaultStoreClient) testfactory.getStoreClient("test")); + StoreClientFactory bestfactory = getFactory(getValidBootstrapUrl()); + clients.add((DefaultStoreClient) bestfactory.getStoreClient("best")); + factories.add(testfactory); + factories.add(bestfactory); + + // bootstrap a number of times + bootStrap(clients, 10); + + checkMbeanIdCount(CLIENT_DOMAIN, "ClientThreadPool*", 2, true); + checkMbeanIdCount(CLIENT_DOMAIN, "DefaultStoreClient*", 2, true); + checkMbeanIdCount(CLUSTER_FAILUREDETECTOR_DOMAIN, "ThresholdFailureDetector*", 2, true); + checkMbeanIdCount(PIPELINE_ROUTED_STATS_DOMAIN, "*", 2, true); + checkMbeanIdCount(CLIENT_REQUEST_DOMAIN, "aggregated*", 2, true); + checkMbeanIdCount(CLIENT_REQUEST_DOMAIN, "stats_localhost*", 2, true); + checkMbeanIdCount(AGGREGATE_STATS_DOMAIN, "aggregate-perf*", 2, true); + checkMbeanIdCount(STATS_DOMAIN, "*", 2, true); + + } catch(Exception e) { + fail("Unexpected error " + e.getMessage()); + } + } + + @Test + public void testMultipleInDistinctClientsOnMultipleFactories() { + try { + StoreClientFactory factory1 = getFactory(getValidBootstrapUrl()); + List> clients = new ArrayList>(); + clients.add((DefaultStoreClient) factory1.getStoreClient("test")); + clients.add((DefaultStoreClient) factory1.getStoreClient("test")); + clients.add((DefaultStoreClient) factory1.getStoreClient("best")); + clients.add((DefaultStoreClient) factory1.getStoreClient("best")); + factories.add(factory1); + + StoreClientFactory factory2 = getFactory(getValidBootstrapUrl()); + clients.add((DefaultStoreClient) factory2.getStoreClient("test")); + clients.add((DefaultStoreClient) factory2.getStoreClient("test")); + clients.add((DefaultStoreClient) factory2.getStoreClient("best")); + clients.add((DefaultStoreClient) factory2.getStoreClient("best")); + factories.add(factory2); + + // bootstrap a number of times + bootStrap(clients, 10); + + checkMbeanIdCount(CLIENT_DOMAIN, "ClientThreadPool*", 2, true); + checkMbeanIdCount(CLIENT_DOMAIN, "DefaultStoreClient*", 2, true); + checkMbeanIdCount(CLUSTER_FAILUREDETECTOR_DOMAIN, "ThresholdFailureDetector*", 2, true); + checkMbeanIdCount(PIPELINE_ROUTED_STATS_DOMAIN, "*", 4, true); + checkMbeanIdCount(CLIENT_REQUEST_DOMAIN, "aggregated*", 2, true); + checkMbeanIdCount(CLIENT_REQUEST_DOMAIN, "stats_localhost*", 2, true); + checkMbeanIdCount(AGGREGATE_STATS_DOMAIN, "aggregate-perf*", 2, true); + checkMbeanIdCount(STATS_DOMAIN, "*", 4, true); + + } catch(Exception e) { + fail("Unexpected error " + e.getMessage()); + } + } +} diff --git a/test/unit/voldemort/scheduled/DataCleanupJobTest.java b/test/unit/voldemort/scheduled/DataCleanupJobTest.java index 89dfb98bce..9580e2b3f7 100644 --- a/test/unit/voldemort/scheduled/DataCleanupJobTest.java +++ b/test/unit/voldemort/scheduled/DataCleanupJobTest.java @@ -16,15 +16,27 @@ package voldemort.scheduled; +import java.io.File; +import java.util.Date; import java.util.List; import junit.framework.TestCase; + +import org.apache.commons.io.FileDeleteStrategy; + import voldemort.MockTime; +import voldemort.TestUtils; +import voldemort.server.VoldemortConfig; import voldemort.server.scheduler.DataCleanupJob; +import voldemort.server.scheduler.SchedulerService; import voldemort.server.storage.ScanPermitWrapper; import voldemort.store.StorageEngine; -import voldemort.store.memory.InMemoryStorageEngine; +import voldemort.store.StoreDefinition; +import voldemort.store.bdb.BdbStorageConfiguration; +import voldemort.utils.ByteArray; import voldemort.utils.EventThrottler; +import voldemort.utils.Props; +import voldemort.utils.SystemTime; import voldemort.utils.Time; import voldemort.versioning.VectorClock; import voldemort.versioning.Versioned; @@ -32,12 +44,105 @@ public class DataCleanupJobTest extends TestCase { private MockTime time; - private StorageEngine engine; + private StorageEngine engine; + private File storeDir; + private BdbStorageConfiguration bdbStorage; @Override - public void setUp() { + public void setUp() throws Exception { time = new MockTime(); - engine = new InMemoryStorageEngine("test"); + storeDir = TestUtils.createTempDir(); + FileDeleteStrategy.FORCE.delete(storeDir); + + // lets use all the default values. + Props props = new Props(); + props.put("node.id", 1); + props.put("voldemort.home", "test/common/voldemort/config"); + VoldemortConfig voldemortConfig = new VoldemortConfig(props); + voldemortConfig.setBdbCacheSize(1024 * 1024); + voldemortConfig.setBdbOneEnvPerStore(true); + voldemortConfig.setBdbDataDirectory(storeDir.toURI().getPath()); + + bdbStorage = new BdbStorageConfiguration(voldemortConfig); + StoreDefinition defA = TestUtils.makeStoreDefinition("cleanupTestStore"); + engine = bdbStorage.getStore(defA); + } + + @Override + protected void tearDown() throws Exception { + super.tearDown(); + try { + if(engine != null) + engine.close(); + if(bdbStorage != null) + bdbStorage.close(); + } finally { + FileDeleteStrategy.FORCE.delete(storeDir); + } + } + + public void testCleanupFrequency() { + + SchedulerService scheduler = new SchedulerService(1, time); + + try { + Date now = new Date(); + + // clean up will purge everything older than last 2 seconds + Runnable cleanupJob = new DataCleanupJob(engine, + new ScanPermitWrapper(1), + 2 * Time.MS_PER_SECOND, + SystemTime.INSTANCE, + new EventThrottler(1)); + + // and will run every 5 seconds starting now + scheduler.schedule("cleanup-freq-test", cleanupJob, now, 5 * Time.MS_PER_SECOND); + + // load some data + for(int i = 0; i < 10; i++) { + ByteArray b = new ByteArray(Integer.toString(i).getBytes()); + engine.put(b, new Versioned(b.get()), null); + } + // sleep for 2 seconds + Thread.sleep(2 * Time.MS_PER_SECOND); + + // None of the keys should have been deleted, i.e data cleanup + // should n't have run. + for(int i = 0; i < 10; i++) { + ByteArray b = new ByteArray(Integer.toString(i).getBytes()); + List> found = engine.get(b, null); + assertTrue("Did not find key '" + i + "' in store!", found.size() > 0); + } + + // wait till 4 seconds from start + Thread.sleep(System.currentTimeMillis() - (now.getTime() + 4 * Time.MS_PER_SECOND)); + // load some more data + for(int i = 10; i < 20; i++) { + ByteArray b = new ByteArray(Integer.toString(i).getBytes()); + engine.put(b, new Versioned(b.get()), null); + } + + // give time for data cleanup to finally run + Thread.sleep(System.currentTimeMillis() - (now.getTime() + 6 * Time.MS_PER_SECOND)); + + // first batch of writes should have been deleted + for(int i = 0; i < 10; i++) { + ByteArray b = new ByteArray(Integer.toString(i).getBytes()); + List> found = engine.get(b, null); + assertTrue("Expected key '" + i + "' to be deleted!", found.size() == 0); + } + // and later ones retained. + for(int i = 10; i < 20; i++) { + ByteArray b = new ByteArray(Integer.toString(i).getBytes()); + List> found = engine.get(b, null); + assertTrue("Expected key '" + i + "' to be retained!", found.size() > 0); + } + + } catch(Exception e) { + + } finally { + scheduler.stop(); + } } public void testCleanupCleansUp() { @@ -51,11 +156,11 @@ public void testCleanupCleansUp() { put("a"); // now run cleanup - new DataCleanupJob(engine, - new ScanPermitWrapper(1), - Time.MS_PER_DAY, - time, - new EventThrottler(1)).run(); + new DataCleanupJob(engine, + new ScanPermitWrapper(1), + Time.MS_PER_DAY, + time, + new EventThrottler(1)).run(); // Check that all the later keys are there AND the key updated later assertContains("a", "d", "e", "f"); @@ -64,7 +169,7 @@ public void testCleanupCleansUp() { private void put(String... items) { for(String item: items) { VectorClock clock = null; - List> found = engine.get(item, null); + List> found = engine.get(new ByteArray(item.getBytes()), null); if(found.size() == 0) { clock = new VectorClock(time.getMilliseconds()); } else if(found.size() == 1) { @@ -73,13 +178,15 @@ private void put(String... items) { } else { fail("Found multiple versions."); } - engine.put(item, new Versioned(item, clock), null); + engine.put(new ByteArray(item.getBytes()), + new Versioned(item.getBytes(), clock), + null); } } private void assertContains(String... keys) { for(String key: keys) { - List> found = engine.get(key, null); + List> found = engine.get(new ByteArray(key.getBytes()), null); assertTrue("Did not find key '" + key + "' in store!", found.size() > 0); } } diff --git a/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java b/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java index 091a6073f8..70931f5e91 100644 --- a/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java +++ b/test/unit/voldemort/server/socket/ClientRequestExecutorPoolTest.java @@ -76,7 +76,8 @@ public void setUp() { 1000, 32 * 1024, false, - true); + true, + 0); this.dest1 = new SocketDestination("localhost", port, RequestFormatType.VOLDEMORT_V1); RequestHandlerFactory handlerFactory = ServerTestUtils.getSocketRequestHandlerFactory(new StoreRepository()); this.server = ServerTestUtils.getSocketService(useNio, diff --git a/test/unit/voldemort/store/stats/ClientSocketStatsTest.java b/test/unit/voldemort/store/stats/ClientSocketStatsTest.java index 9f0a5ae3e8..785dbad9f4 100644 --- a/test/unit/voldemort/store/stats/ClientSocketStatsTest.java +++ b/test/unit/voldemort/store/stats/ClientSocketStatsTest.java @@ -41,13 +41,13 @@ public void setUp() throws Exception { this.port = ServerTestUtils.findFreePort(); this.dest1 = new SocketDestination("localhost", port, RequestFormatType.VOLDEMORT_V1); this.dest2 = new SocketDestination("localhost", port + 1, RequestFormatType.VOLDEMORT_V1); - this.masterStats = new ClientSocketStats(); + this.masterStats = new ClientSocketStats(0); pool = null; } @Test public void testNewNodeStatsObject() { - ClientSocketStats stats = new ClientSocketStats(masterStats, dest1, pool); + ClientSocketStats stats = new ClientSocketStats(masterStats, dest1, pool, 0); assertNotNull(stats); } From 33c809ed9447856adf0e21677268bd6a11a0634f Mon Sep 17 00:00:00 2001 From: Vinoth Chandar Date: Tue, 28 Aug 2012 15:23:58 -0700 Subject: [PATCH 115/209] Adding unregisterMBean calls --- .../client/AbstractStoreClientFactory.java | 24 +++++++++++++++---- .../store/routed/PipelineRoutedStore.java | 14 +++++++++-- .../ClientRequestExecutorPool.java | 16 +++++++++---- .../store/stats/ClientSocketStats.java | 3 ++- .../voldemort/xml/StoreDefinitionsMapper.java | 6 ++--- 5 files changed, 47 insertions(+), 16 deletions(-) diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index 830a5d20cf..0c905de313 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -279,10 +279,12 @@ public FailureDetector getFailureDetector() { result = failureDetector; if(result == null) { failureDetector = result = initFailureDetector(config, cluster.getNodes()); - JmxUtils.registerMbean(failureDetector, - JmxUtils.createObjectName(JmxUtils.getPackageName(failureDetector.getClass()), - JmxUtils.getClassName(failureDetector.getClass()) - + JmxUtils.getJmxId(jmxId))); + if(isJmxEnabled) { + JmxUtils.registerMbean(failureDetector, + JmxUtils.createObjectName(JmxUtils.getPackageName(failureDetector.getClass()), + JmxUtils.getClassName(failureDetector.getClass()) + + JmxUtils.getJmxId(jmxId))); + } } } } @@ -403,7 +405,19 @@ public void close() { this.threadPool.shutdownNow(); } - if(failureDetector != null) + if(failureDetector != null) { failureDetector.destroy(); + if(isJmxEnabled) { + JmxUtils.unregisterMbean(JmxUtils.createObjectName(JmxUtils.getPackageName(failureDetector.getClass()), + JmxUtils.getClassName(failureDetector.getClass()) + + JmxUtils.getJmxId(jmxId))); + JmxUtils.unregisterMbean(JmxUtils.createObjectName(JmxUtils.getPackageName(threadPool.getClass()), + JmxUtils.getClassName(threadPool.getClass()) + + JmxUtils.getJmxId(jmxId))); + JmxUtils.unregisterMbean(JmxUtils.createObjectName("voldemort.store.stats.aggregate", + "aggregate-perf" + + JmxUtils.getJmxId(jmxId))); + } + } } } diff --git a/src/java/voldemort/store/routed/PipelineRoutedStore.java b/src/java/voldemort/store/routed/PipelineRoutedStore.java index 45c6011329..1dffd8111f 100644 --- a/src/java/voldemort/store/routed/PipelineRoutedStore.java +++ b/src/java/voldemort/store/routed/PipelineRoutedStore.java @@ -76,6 +76,8 @@ public class PipelineRoutedStore extends RoutedStore { private Zone clientZone; private boolean zoneRoutingEnabled; private PipelineRoutedStats stats; + private boolean jmxEnabled; + private int jmxId; /** * Create a PipelineRoutedStore @@ -131,11 +133,14 @@ public PipelineRoutedStore(String name, this.handoffStrategy = null; } - if(jmxEnabled) { + this.jmxEnabled = jmxEnabled; + this.jmxId = jmxId; + if(this.jmxEnabled) { stats = new PipelineRoutedStats(); JmxUtils.registerMbean(stats, JmxUtils.createObjectName(JmxUtils.getPackageName(stats.getClass()), - getName() + JmxUtils.getJmxId(jmxId))); + getName() + + JmxUtils.getJmxId(this.jmxId))); } } @@ -720,6 +725,11 @@ public void close() { } } + if(this.jmxEnabled) { + JmxUtils.unregisterMbean(JmxUtils.createObjectName(JmxUtils.getPackageName(stats.getClass()), + getName() + JmxUtils.getJmxId(jmxId))); + } + if(exception != null) throw exception; diff --git a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java index 76011759cf..c9e5509950 100644 --- a/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java +++ b/src/java/voldemort/store/socket/clientrequest/ClientRequestExecutorPool.java @@ -49,6 +49,8 @@ public class ClientRequestExecutorPool implements SocketStoreFactory { private final KeyedResourcePool pool; private final ClientRequestExecutorFactory factory; private final ClientSocketStats stats; + private final boolean jmxEnabled; + private final int jmxId; public ClientRequestExecutorPool(int selectors, int maxConnectionsPerNode, @@ -56,19 +58,21 @@ public ClientRequestExecutorPool(int selectors, int soTimeoutMs, int socketBufferSize, boolean socketKeepAlive, - boolean enableJmx, + boolean jmxEnabled, int jmxId) { ResourcePoolConfig config = new ResourcePoolConfig().setIsFair(true) .setMaxPoolSize(maxConnectionsPerNode) .setMaxInvalidAttempts(maxConnectionsPerNode) .setTimeout(connectionTimeoutMs, TimeUnit.MILLISECONDS); - if(enableJmx) { + this.jmxEnabled = jmxEnabled; + this.jmxId = jmxId; + if(this.jmxEnabled) { stats = new ClientSocketStats(jmxId); JmxUtils.registerMbean(new ClientSocketStatsJmx(stats), JmxUtils.createObjectName(JmxUtils.getPackageName(this.getClass()), "aggregated" - + JmxUtils.getJmxId(jmxId))); + + JmxUtils.getJmxId(this.jmxId))); } else { stats = null; } @@ -180,8 +184,10 @@ public void close() { // unregister MBeans if(stats != null) { try { - JmxUtils.unregisterMbean(JmxUtils.createObjectName(JmxUtils.getPackageName(ClientRequestExecutor.class), - "aggregated")); + if(this.jmxEnabled) + JmxUtils.unregisterMbean(JmxUtils.createObjectName(JmxUtils.getPackageName(this.getClass()), + "aggregated" + + JmxUtils.getJmxId(this.jmxId))); } catch(Exception e) {} stats.close(); } diff --git a/src/java/voldemort/store/stats/ClientSocketStats.java b/src/java/voldemort/store/stats/ClientSocketStats.java index 335ca8302c..67eb845672 100644 --- a/src/java/voldemort/store/stats/ClientSocketStats.java +++ b/src/java/voldemort/store/stats/ClientSocketStats.java @@ -236,7 +236,8 @@ public void close() { "stats_" + destination.toString() .replace(':', - '_'))); + '_') + + JmxUtils.getJmxId(jmxId))); } catch(Exception e) {} } } diff --git a/src/java/voldemort/xml/StoreDefinitionsMapper.java b/src/java/voldemort/xml/StoreDefinitionsMapper.java index 8c15ff728e..26f33f62b0 100644 --- a/src/java/voldemort/xml/StoreDefinitionsMapper.java +++ b/src/java/voldemort/xml/StoreDefinitionsMapper.java @@ -228,9 +228,9 @@ private StoreDefinition readStore(Element store) { Element throttleRate = store.getChild(STORE_RETENTION_SCAN_THROTTLE_RATE_ELMT); if(throttleRate != null) retentionThrottleRate = Integer.parseInt(throttleRate.getText()); - Element retentionFreqElement = store.getChild(STORE_RETENTION_FREQ_ELMT); - if(retentionFreqElement != null) - retentionFreqDays = Integer.parseInt(retentionFreqElement.getText()); + Element retentionFreqDaysElement = store.getChild(STORE_RETENTION_FREQ_ELMT); + if(retentionFreqDaysElement != null) + retentionFreqDays = Integer.parseInt(retentionFreqDaysElement.getText()); } if(routingStrategyType.compareTo(RoutingStrategyType.ZONE_STRATEGY) == 0) { From 2b048f3f2924d96fe84bbb9792226b8fbd8381a0 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Thu, 23 Aug 2012 21:16:52 -0700 Subject: [PATCH 116/209] Fixed a bug that will make GetAll to go to one more node than preferred --- .../action/PerformParallelGetAllRequests.java | 17 ++-- .../action/PerformSerialGetAllRequests.java | 11 ++- .../voldemort/store/AbstractStoreTest.java | 41 ++++++--- .../store/routed/GetallNodeReachTest.java | 91 +++++++++++++++++++ 4 files changed, 133 insertions(+), 27 deletions(-) create mode 100644 test/unit/voldemort/store/routed/GetallNodeReachTest.java diff --git a/src/java/voldemort/store/routed/action/PerformParallelGetAllRequests.java b/src/java/voldemort/store/routed/action/PerformParallelGetAllRequests.java index 444fdd930e..357f30c97f 100644 --- a/src/java/voldemort/store/routed/action/PerformParallelGetAllRequests.java +++ b/src/java/voldemort/store/routed/action/PerformParallelGetAllRequests.java @@ -16,6 +16,7 @@ package voldemort.store.routed.action; +import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; import java.util.List; @@ -137,24 +138,26 @@ public void requestComplete(Object result, long requestTime) { successCount.increment(); List> retrieved = values.get(key); + if(retrieved == null) { + retrieved = new ArrayList>(); + } /* * retrieved can be null if there are no values for the key * provided */ - if(retrieved != null) { - List> existing = pipelineData.getResult().get(key); + List> existing = pipelineData.getResult().get(key); - if(existing == null) - pipelineData.getResult().put(key, Lists.newArrayList(retrieved)); - else - existing.addAll(retrieved); - } + if(existing == null) + pipelineData.getResult().put(key, Lists.newArrayList(retrieved)); + else + existing.addAll(retrieved); HashSet zoneResponses = null; if(pipelineData.getKeyToZoneResponse().containsKey(key)) { zoneResponses = pipelineData.getKeyToZoneResponse().get(key); } else { zoneResponses = new HashSet(); + pipelineData.getKeyToZoneResponse().put(key, zoneResponses); } zoneResponses.add(response.getNode().getZoneId()); } diff --git a/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java b/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java index c8135781d4..f1719a8f55 100644 --- a/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java +++ b/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java @@ -79,13 +79,13 @@ public void execute(Pipeline pipeline) { boolean zoneRequirement = false; MutableInt successCount = pipelineData.getSuccessCount(key); - if(logger.isDebugEnabled()) - logger.debug("GETALL for key " + key + " (keyRef: " + System.identityHashCode(key) - + ") successes: " + successCount.intValue() + " preferred: " + preferred - + " required: " + required); + if(logger.isDebugEnabled()) + logger.debug("GETALL for key " + key + " (keyRef: " + System.identityHashCode(key) + + ") successes: " + successCount.intValue() + " preferred: " + + preferred + " required: " + required); if(successCount.intValue() >= preferred) { - if(pipelineData.getZonesRequired() != null) { + if(pipelineData.getZonesRequired() != null && pipelineData.getZonesRequired() > 0) { if(pipelineData.getKeyToZoneResponse().containsKey(key)) { int zonesSatisfied = pipelineData.getKeyToZoneResponse().get(key).size(); @@ -149,6 +149,7 @@ public void execute(Pipeline pipeline) { zoneResponses = pipelineData.getKeyToZoneResponse().get(key); } else { zoneResponses = new HashSet(); + pipelineData.getKeyToZoneResponse().put(key, zoneResponses); } zoneResponses.add(response.getNode().getZoneId()); diff --git a/test/unit/voldemort/store/AbstractStoreTest.java b/test/unit/voldemort/store/AbstractStoreTest.java index 93c5efb719..e2adf72f01 100644 --- a/test/unit/voldemort/store/AbstractStoreTest.java +++ b/test/unit/voldemort/store/AbstractStoreTest.java @@ -125,8 +125,8 @@ public void testNullKeys() throws Exception { // this is good } try { - store.getAll(Collections. singleton(null), Collections. singletonMap(null, - null)); + store.getAll(Collections. singleton(null), + Collections. singletonMap(null, null)); fail("Store should not getAll null keys!"); } catch(IllegalArgumentException e) { // this is good @@ -141,12 +141,13 @@ public void testNullKeys() throws Exception { @Test public void testPutNullValue() { - // Store store = getStore(); - // K key = getKey(); - // store.put(key, new Versioned(null)); - // List> found = store.get(key); - // assertEquals("Wrong number of values.", 1, found.size()); - // assertEquals("Returned non-null value.", null, found.get(0).getValue()); + // Store store = getStore(); + // K key = getKey(); + // store.put(key, new Versioned(null)); + // List> found = store.get(key); + // assertEquals("Wrong number of values.", 1, found.size()); + // assertEquals("Returned non-null value.", null, + // found.get(0).getValue()); } @Test @@ -155,12 +156,8 @@ public void testGetAndDeleteNonExistentKey() throws Exception { Store store = getStore(); List> found = store.get(key, null); assertEquals("Found non-existent key: " + found, 0, found.size()); - assertTrue("Delete of non-existent key succeeded.", !store.delete(key, getClock(1, - 1, - 2, - 2, - 3, - 3))); + assertTrue("Delete of non-existent key succeeded.", + !store.delete(key, getClock(1, 1, 2, 2, 3, 3))); } private void testObsoletePutFails(String message, @@ -315,7 +312,21 @@ public void testGetAll() throws Exception { public void testGetAllWithAbsentKeys() throws Exception { Store store = getStore(); Map>> result = store.getAll(getKeys(3), null); - assertEquals(0, result.size()); + boolean resultZero = (result.size() == 0); + boolean resultEmpty = true; + if(!resultZero) { + if(result.get(result.keySet().toArray()[0]).size() != 0) { + resultEmpty = false; + } + if(result.get(result.keySet().toArray()[1]).size() != 0) { + resultEmpty = false; + } + if(result.get(result.keySet().toArray()[2]).size() != 0) { + resultEmpty = false; + } + } + assertTrue(resultZero || resultEmpty); + } @Test diff --git a/test/unit/voldemort/store/routed/GetallNodeReachTest.java b/test/unit/voldemort/store/routed/GetallNodeReachTest.java new file mode 100644 index 0000000000..5a31b0b93c --- /dev/null +++ b/test/unit/voldemort/store/routed/GetallNodeReachTest.java @@ -0,0 +1,91 @@ +package voldemort.store.routed; + +import static org.junit.Assert.assertEquals; +import static voldemort.VoldemortTestConstants.getFourNodeClusterWithZones; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Executors; + +import org.junit.Before; +import org.junit.Test; + +import voldemort.TestUtils; +import voldemort.client.RoutingTier; +import voldemort.client.TimeoutConfig; +import voldemort.cluster.Cluster; +import voldemort.cluster.Node; +import voldemort.cluster.failuredetector.NoopFailureDetector; +import voldemort.routing.RoutingStrategyType; +import voldemort.serialization.SerializerDefinition; +import voldemort.store.Store; +import voldemort.store.StoreDefinition; +import voldemort.store.StoreDefinitionBuilder; +import voldemort.store.memory.InMemoryStorageConfiguration; +import voldemort.store.memory.InMemoryStorageEngine; +import voldemort.utils.ByteArray; +import voldemort.versioning.Versioned; + +import com.google.common.collect.Maps; + +public class GetallNodeReachTest { + + private Cluster cluster; + + @Before + public void setUp() throws Exception { + cluster = getFourNodeClusterWithZones(); + } + + @Test + public void testGetallTouchOne() throws Exception { + RoutedStore store = null; + HashMap zoneReplicationFactor = new HashMap(); + zoneReplicationFactor.put(0, 2); + zoneReplicationFactor.put(1, 1); + zoneReplicationFactor.put(2, 1); + StoreDefinition storeDef = new StoreDefinitionBuilder().setName("test") + .setType(InMemoryStorageConfiguration.TYPE_NAME) + .setRoutingPolicy(RoutingTier.CLIENT) + .setRoutingStrategyType(RoutingStrategyType.ZONE_STRATEGY) + .setReplicationFactor(4) + .setZoneReplicationFactor(zoneReplicationFactor) + .setKeySerializer(new SerializerDefinition("string")) + .setValueSerializer(new SerializerDefinition("string")) + .setPreferredReads(2) + .setRequiredReads(1) + .setPreferredWrites(1) + .setRequiredWrites(1) + .setZoneCountReads(0) + .setZoneCountWrites(0) + .build(); + Map> subStores = Maps.newHashMap(); + for(Node n: cluster.getNodes()) { + Store subStore = new InMemoryStorageEngine("test"); + subStores.put(n.getId(), subStore); + + } + RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(true, + Executors.newFixedThreadPool(2), + new TimeoutConfig(1000L, + false)); + + store = routedStoreFactory.create(cluster, + storeDef, + subStores, + true, + new NoopFailureDetector()); + Versioned v = Versioned.value("v".getBytes()); + subStores.get(0).put(TestUtils.toByteArray("k011"), v, null); + subStores.get(1).put(TestUtils.toByteArray("k011"), v, null); + subStores.get(2).put(TestUtils.toByteArray("k100"), v, null); + List keys011 = new ArrayList(); + keys011.add(TestUtils.toByteArray("k011")); + List keys100 = new ArrayList(); + keys100.add(TestUtils.toByteArray("k100")); + assertEquals(store.getAll(keys011, null).get(TestUtils.toByteArray("k011")).size(), 2); + assertEquals(store.getAll(keys100, null).get(TestUtils.toByteArray("k100")).size(), 0); + } +} From da04c859617ebe593ff159b3b5b90aff0a94ecc3 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Mon, 27 Aug 2012 20:10:54 -0700 Subject: [PATCH 117/209] More tests for getall fix --- test/unit/voldemort/server/EndToEndTest.java | 6 +- .../store/routed/GetallNodeReachTest.java | 286 ++++++++++++++++-- 2 files changed, 256 insertions(+), 36 deletions(-) diff --git a/test/unit/voldemort/server/EndToEndTest.java b/test/unit/voldemort/server/EndToEndTest.java index e5278f8907..45dbe3bf24 100644 --- a/test/unit/voldemort/server/EndToEndTest.java +++ b/test/unit/voldemort/server/EndToEndTest.java @@ -1,8 +1,8 @@ package voldemort.server; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; @@ -120,8 +120,8 @@ public void testSanity() { assertEquals("getAll works as expected", "Moscow", capitals.get("Russia").getValue()); assertEquals("getAll works as expected", "Kiev", capitals.get("Ukraine").getValue()); - - assertFalse("getAll works as expected", capitals.containsKey("Japan")); + assertTrue("getAll works as expected", + !capitals.containsKey("Japan") || capitals.get("Japan") == null); storeClient.delete("Ukraine"); assertNull("delete works as expected", storeClient.get("Ukraine")); diff --git a/test/unit/voldemort/store/routed/GetallNodeReachTest.java b/test/unit/voldemort/store/routed/GetallNodeReachTest.java index 5a31b0b93c..8f03c495df 100644 --- a/test/unit/voldemort/store/routed/GetallNodeReachTest.java +++ b/test/unit/voldemort/store/routed/GetallNodeReachTest.java @@ -1,6 +1,9 @@ package voldemort.store.routed; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static voldemort.VoldemortTestConstants.getEightNodeClusterWithZones; import static voldemort.VoldemortTestConstants.getFourNodeClusterWithZones; import java.util.ArrayList; @@ -33,39 +36,18 @@ public class GetallNodeReachTest { private Cluster cluster; + private StoreDefinition storeDef; + private RoutedStore store; + Map> subStores; @Before - public void setUp() throws Exception { - cluster = getFourNodeClusterWithZones(); - } + public void setUp() throws Exception {} - @Test - public void testGetallTouchOne() throws Exception { - RoutedStore store = null; - HashMap zoneReplicationFactor = new HashMap(); - zoneReplicationFactor.put(0, 2); - zoneReplicationFactor.put(1, 1); - zoneReplicationFactor.put(2, 1); - StoreDefinition storeDef = new StoreDefinitionBuilder().setName("test") - .setType(InMemoryStorageConfiguration.TYPE_NAME) - .setRoutingPolicy(RoutingTier.CLIENT) - .setRoutingStrategyType(RoutingStrategyType.ZONE_STRATEGY) - .setReplicationFactor(4) - .setZoneReplicationFactor(zoneReplicationFactor) - .setKeySerializer(new SerializerDefinition("string")) - .setValueSerializer(new SerializerDefinition("string")) - .setPreferredReads(2) - .setRequiredReads(1) - .setPreferredWrites(1) - .setRequiredWrites(1) - .setZoneCountReads(0) - .setZoneCountWrites(0) - .build(); - Map> subStores = Maps.newHashMap(); + private void makeStore() { + subStores = Maps.newHashMap(); for(Node n: cluster.getNodes()) { Store subStore = new InMemoryStorageEngine("test"); subStores.put(n.getId(), subStore); - } RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(true, Executors.newFixedThreadPool(2), @@ -77,15 +59,253 @@ public void testGetallTouchOne() throws Exception { subStores, true, new NoopFailureDetector()); + } + + @Test + public void testGetallTouchOneZone() throws Exception { + cluster = getFourNodeClusterWithZones(); + HashMap zoneReplicationFactor = new HashMap(); + zoneReplicationFactor.put(0, 2); + zoneReplicationFactor.put(1, 1); + zoneReplicationFactor.put(2, 1); + storeDef = new StoreDefinitionBuilder().setName("test") + .setType(InMemoryStorageConfiguration.TYPE_NAME) + .setRoutingPolicy(RoutingTier.CLIENT) + .setRoutingStrategyType(RoutingStrategyType.ZONE_STRATEGY) + .setReplicationFactor(4) + .setZoneReplicationFactor(zoneReplicationFactor) + .setKeySerializer(new SerializerDefinition("string")) + .setValueSerializer(new SerializerDefinition("string")) + .setPreferredReads(2) + .setRequiredReads(1) + .setPreferredWrites(1) + .setRequiredWrites(1) + .setZoneCountReads(0) + .setZoneCountWrites(0) + .build(); + makeStore(); + Versioned v = Versioned.value("v".getBytes()); + subStores.get(0).put(TestUtils.toByteArray("k011_zone0_only"), v, null); + subStores.get(1).put(TestUtils.toByteArray("k011_zone0_only"), v, null); + subStores.get(2).put(TestUtils.toByteArray("k100_zone1_only"), v, null); + /* test single key getall */ + List keys011 = new ArrayList(); + keys011.add(TestUtils.toByteArray("k011_zone0_only")); + List keys100 = new ArrayList(); + keys100.add(TestUtils.toByteArray("k100_zone1_only")); + assertEquals(2, store.getAll(keys011, null) + .get(TestUtils.toByteArray("k011_zone0_only")) + .size()); + assertEquals(0, store.getAll(keys100, null) + .get(TestUtils.toByteArray("k100_zone1_only")) + .size()); + /* test multiple keys getall */ + List keys = new ArrayList(); + keys.add(TestUtils.toByteArray("k011_zone0_only")); + keys.add(TestUtils.toByteArray("k100_zone1_only")); + Map>> result = store.getAll(keys, null); + assertEquals(2, result.get(TestUtils.toByteArray("k011_zone0_only")).size()); + assertEquals(0, result.get(TestUtils.toByteArray("k100_zone1_only")).size()); + } + + @Test + public void testGetall_211() throws Exception { + cluster = getFourNodeClusterWithZones(); + HashMap zoneReplicationFactor = new HashMap(); + zoneReplicationFactor.put(0, 2); + zoneReplicationFactor.put(1, 1); + zoneReplicationFactor.put(2, 1); + storeDef = new StoreDefinitionBuilder().setName("test") + .setType(InMemoryStorageConfiguration.TYPE_NAME) + .setRoutingPolicy(RoutingTier.CLIENT) + .setRoutingStrategyType(RoutingStrategyType.ZONE_STRATEGY) + .setReplicationFactor(4) + .setZoneReplicationFactor(zoneReplicationFactor) + .setKeySerializer(new SerializerDefinition("string")) + .setValueSerializer(new SerializerDefinition("string")) + .setPreferredReads(1) + .setRequiredReads(1) + .setPreferredWrites(1) + .setRequiredWrites(1) + .setZoneCountReads(0) + .setZoneCountWrites(0) + .build(); + makeStore(); Versioned v = Versioned.value("v".getBytes()); + // k### indicates existence of itself in different nodes + // k**1 means this key exists at least on node 0 + // k*1* means this key exists at least on node 1 + // k0** means this key does not exist on node 2 + subStores.get(0).put(TestUtils.toByteArray("k001"), v, null); subStores.get(0).put(TestUtils.toByteArray("k011"), v, null); + subStores.get(0).put(TestUtils.toByteArray("k101"), v, null); + subStores.get(0).put(TestUtils.toByteArray("k111"), v, null); + subStores.get(1).put(TestUtils.toByteArray("k010"), v, null); subStores.get(1).put(TestUtils.toByteArray("k011"), v, null); + subStores.get(1).put(TestUtils.toByteArray("k110"), v, null); + subStores.get(1).put(TestUtils.toByteArray("k111"), v, null); subStores.get(2).put(TestUtils.toByteArray("k100"), v, null); - List keys011 = new ArrayList(); - keys011.add(TestUtils.toByteArray("k011")); - List keys100 = new ArrayList(); - keys100.add(TestUtils.toByteArray("k100")); - assertEquals(store.getAll(keys011, null).get(TestUtils.toByteArray("k011")).size(), 2); - assertEquals(store.getAll(keys100, null).get(TestUtils.toByteArray("k100")).size(), 0); + subStores.get(2).put(TestUtils.toByteArray("k101"), v, null); + subStores.get(2).put(TestUtils.toByteArray("k110"), v, null); + subStores.get(2).put(TestUtils.toByteArray("k111"), v, null); + + /* test multiple keys getall */ + List keys = new ArrayList(); + keys.add(TestUtils.toByteArray("k000")); + keys.add(TestUtils.toByteArray("k001")); + keys.add(TestUtils.toByteArray("k010")); + keys.add(TestUtils.toByteArray("k011")); + keys.add(TestUtils.toByteArray("k100")); + keys.add(TestUtils.toByteArray("k101")); + keys.add(TestUtils.toByteArray("k110")); + keys.add(TestUtils.toByteArray("k111")); + Map>> result = store.getAll(keys, null); + assertFalse(result.containsKey(TestUtils.toByteArray("not_included"))); + assertEquals(0, result.get(TestUtils.toByteArray("k000")).size()); + assertEquals(1, result.get(TestUtils.toByteArray("k001")).size()); + assertEquals(0, result.get(TestUtils.toByteArray("k010")).size()); + assertEquals(1, result.get(TestUtils.toByteArray("k011")).size()); + assertEquals(0, result.get(TestUtils.toByteArray("k100")).size()); + assertEquals(1, result.get(TestUtils.toByteArray("k101")).size()); + assertEquals(0, result.get(TestUtils.toByteArray("k110")).size()); + assertEquals(1, result.get(TestUtils.toByteArray("k111")).size()); + } + + @Test + public void testGetall_211_zoneCountRead_1() throws Exception { + cluster = getFourNodeClusterWithZones(); + HashMap zoneReplicationFactor = new HashMap(); + zoneReplicationFactor.put(0, 2); + zoneReplicationFactor.put(1, 1); + zoneReplicationFactor.put(2, 1); + /* + * First n nodes on the preference list will be one node from each + * remote n zones, where n=zoneCountReads, therefore preferred read + * should be set > n if want to include local zone node results in + * parallel request + */ + storeDef = new StoreDefinitionBuilder().setName("test") + .setType(InMemoryStorageConfiguration.TYPE_NAME) + .setRoutingPolicy(RoutingTier.CLIENT) + .setRoutingStrategyType(RoutingStrategyType.ZONE_STRATEGY) + .setReplicationFactor(4) + .setZoneReplicationFactor(zoneReplicationFactor) + .setKeySerializer(new SerializerDefinition("string")) + .setValueSerializer(new SerializerDefinition("string")) + .setPreferredReads(2) + .setRequiredReads(1) + .setPreferredWrites(1) + .setRequiredWrites(1) + .setZoneCountReads(1) + .setZoneCountWrites(0) + .build(); + makeStore(); + Versioned v = Versioned.value("v".getBytes()); + subStores.get(0).put(TestUtils.toByteArray("k001"), v, null); + subStores.get(0).put(TestUtils.toByteArray("k011"), v, null); + subStores.get(0).put(TestUtils.toByteArray("k101"), v, null); + subStores.get(0).put(TestUtils.toByteArray("k111"), v, null); + subStores.get(1).put(TestUtils.toByteArray("k010"), v, null); + subStores.get(1).put(TestUtils.toByteArray("k011"), v, null); + subStores.get(1).put(TestUtils.toByteArray("k110"), v, null); + subStores.get(1).put(TestUtils.toByteArray("k111"), v, null); + subStores.get(2).put(TestUtils.toByteArray("k100"), v, null); + subStores.get(2).put(TestUtils.toByteArray("k101"), v, null); + subStores.get(2).put(TestUtils.toByteArray("k110"), v, null); + subStores.get(2).put(TestUtils.toByteArray("k111"), v, null); + + /* test multiple keys getall */ + List keys = new ArrayList(); + keys.add(TestUtils.toByteArray("k000")); + keys.add(TestUtils.toByteArray("k001")); + keys.add(TestUtils.toByteArray("k010")); + keys.add(TestUtils.toByteArray("k011")); + keys.add(TestUtils.toByteArray("k100")); + keys.add(TestUtils.toByteArray("k101")); + keys.add(TestUtils.toByteArray("k110")); + keys.add(TestUtils.toByteArray("k111")); + Map>> result = store.getAll(keys, null); + assertFalse(result.containsKey(TestUtils.toByteArray("not_included"))); + /* client will first try all the nodes in local zone */ + assertEquals(0, result.get(TestUtils.toByteArray("k000")).size()); + assertEquals(1, result.get(TestUtils.toByteArray("k001")).size()); + // don't know which one of node 0 or 1 comes second on preferece list + // for key + // k*10 or k*01 + // therefore it can be 1 or 0 beside existence on node 2 + int size = -1; + size = result.get(TestUtils.toByteArray("k010")).size(); + assertTrue(size == 1 || size == 0); + assertEquals(1, result.get(TestUtils.toByteArray("k011")).size()); + assertEquals(1, result.get(TestUtils.toByteArray("k100")).size()); + size = result.get(TestUtils.toByteArray("k101")).size(); + assertTrue(size == 1 || size == 2); + size = result.get(TestUtils.toByteArray("k110")).size(); + assertTrue(size == 1 || size == 2); + assertEquals(2, result.get(TestUtils.toByteArray("k111")).size()); + } + + @Test + public void testGetall_322() throws Exception { + cluster = getEightNodeClusterWithZones(); + HashMap zoneReplicationFactor = new HashMap(); + zoneReplicationFactor.put(0, 4); + zoneReplicationFactor.put(1, 4); + /* + * First n nodes on the preference list will be one node from each + * remote n zones, where n=zoneCountReads, therefore preferred read + * should be set > n if want to include local zone node results in + * parallel request + */ + storeDef = new StoreDefinitionBuilder().setName("test") + .setType(InMemoryStorageConfiguration.TYPE_NAME) + .setRoutingPolicy(RoutingTier.CLIENT) + .setRoutingStrategyType(RoutingStrategyType.ZONE_STRATEGY) + .setReplicationFactor(8) + .setZoneReplicationFactor(zoneReplicationFactor) + .setKeySerializer(new SerializerDefinition("string")) + .setValueSerializer(new SerializerDefinition("string")) + .setPreferredReads(2) + .setRequiredReads(2) + .setPreferredWrites(2) + .setRequiredWrites(2) + .setZoneCountReads(0) + .setZoneCountWrites(0) + .build(); + makeStore(); + Versioned v = Versioned.value("v".getBytes()); + subStores.get(0).put(TestUtils.toByteArray("k1111_1111"), v, null); + subStores.get(0).put(TestUtils.toByteArray("k0000_1111"), v, null); + + subStores.get(1).put(TestUtils.toByteArray("k1111_1111"), v, null); + subStores.get(1).put(TestUtils.toByteArray("k0000_1111"), v, null); + + subStores.get(2).put(TestUtils.toByteArray("k1111_1111"), v, null); + subStores.get(2).put(TestUtils.toByteArray("k0000_1111"), v, null); + + subStores.get(3).put(TestUtils.toByteArray("k0000_1111"), v, null); + subStores.get(3).put(TestUtils.toByteArray("k1111_1111"), v, null); + + subStores.get(4).put(TestUtils.toByteArray("k1111_1111"), v, null); + subStores.get(4).put(TestUtils.toByteArray("k1111_0000"), v, null); + subStores.get(5).put(TestUtils.toByteArray("k1111_1111"), v, null); + subStores.get(5).put(TestUtils.toByteArray("k1111_0000"), v, null); + subStores.get(6).put(TestUtils.toByteArray("k1111_1111"), v, null); + subStores.get(6).put(TestUtils.toByteArray("k1111_0000"), v, null); + subStores.get(7).put(TestUtils.toByteArray("k1111_1111"), v, null); + subStores.get(7).put(TestUtils.toByteArray("k1111_0000"), v, null); + + /* test multiple keys getall */ + List keys = new ArrayList(); + keys.add(TestUtils.toByteArray("k0000_0000")); + keys.add(TestUtils.toByteArray("k0000_1111")); + keys.add(TestUtils.toByteArray("k1111_0000")); + keys.add(TestUtils.toByteArray("k1111_1111")); + Map>> result = store.getAll(keys, null); + assertFalse(result.containsKey(TestUtils.toByteArray("not_included"))); + assertEquals(0, result.get(TestUtils.toByteArray("k0000_0000")).size()); + assertEquals(2, result.get(TestUtils.toByteArray("k0000_1111")).size()); + assertEquals(0, result.get(TestUtils.toByteArray("k1111_0000")).size()); + assertEquals(2, result.get(TestUtils.toByteArray("k1111_1111")).size()); } } From 604324ddb82c355a7f2fc60cc1511819961174e7 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Wed, 5 Sep 2012 17:39:53 -0700 Subject: [PATCH 118/209] Minor changes to tests --- test/unit/voldemort/server/EndToEndTest.java | 3 +-- .../voldemort/store/routed/GetallNodeReachTest.java | 12 +++--------- 2 files changed, 4 insertions(+), 11 deletions(-) diff --git a/test/unit/voldemort/server/EndToEndTest.java b/test/unit/voldemort/server/EndToEndTest.java index 45dbe3bf24..f8e9c798c8 100644 --- a/test/unit/voldemort/server/EndToEndTest.java +++ b/test/unit/voldemort/server/EndToEndTest.java @@ -120,8 +120,7 @@ public void testSanity() { assertEquals("getAll works as expected", "Moscow", capitals.get("Russia").getValue()); assertEquals("getAll works as expected", "Kiev", capitals.get("Ukraine").getValue()); - assertTrue("getAll works as expected", - !capitals.containsKey("Japan") || capitals.get("Japan") == null); + assertTrue("getAll works as expected", capitals.get("Japan") == null); storeClient.delete("Ukraine"); assertNull("delete works as expected", storeClient.get("Ukraine")); diff --git a/test/unit/voldemort/store/routed/GetallNodeReachTest.java b/test/unit/voldemort/store/routed/GetallNodeReachTest.java index 8f03c495df..869c4a0ed1 100644 --- a/test/unit/voldemort/store/routed/GetallNodeReachTest.java +++ b/test/unit/voldemort/store/routed/GetallNodeReachTest.java @@ -249,19 +249,13 @@ public void testGetall_211_zoneCountRead_1() throws Exception { public void testGetall_322() throws Exception { cluster = getEightNodeClusterWithZones(); HashMap zoneReplicationFactor = new HashMap(); - zoneReplicationFactor.put(0, 4); - zoneReplicationFactor.put(1, 4); - /* - * First n nodes on the preference list will be one node from each - * remote n zones, where n=zoneCountReads, therefore preferred read - * should be set > n if want to include local zone node results in - * parallel request - */ + zoneReplicationFactor.put(0, 3); + zoneReplicationFactor.put(1, 3); storeDef = new StoreDefinitionBuilder().setName("test") .setType(InMemoryStorageConfiguration.TYPE_NAME) .setRoutingPolicy(RoutingTier.CLIENT) .setRoutingStrategyType(RoutingStrategyType.ZONE_STRATEGY) - .setReplicationFactor(8) + .setReplicationFactor(6) .setZoneReplicationFactor(zoneReplicationFactor) .setKeySerializer(new SerializerDefinition("string")) .setValueSerializer(new SerializerDefinition("string")) From 614eab9a41157db1bb3a5b6b8f145f3f9e223981 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Tue, 3 Jul 2012 14:11:42 -0700 Subject: [PATCH 119/209] refactored streaming and writing of voldemort admin data printing --- src/java/voldemort/VoldemortAdminTool.java | 312 ++++++++++----------- 1 file changed, 152 insertions(+), 160 deletions(-) diff --git a/src/java/voldemort/VoldemortAdminTool.java b/src/java/voldemort/VoldemortAdminTool.java index 4603230ec3..756a650c5d 100644 --- a/src/java/voldemort/VoldemortAdminTool.java +++ b/src/java/voldemort/VoldemortAdminTool.java @@ -28,10 +28,12 @@ import java.io.FileOutputStream; import java.io.FileWriter; import java.io.IOException; +import java.io.OutputStream; import java.io.OutputStreamWriter; import java.io.PrintStream; import java.io.StringReader; import java.io.StringWriter; +import java.io.Writer; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -993,20 +995,91 @@ private static void executeFetchEntries(Integer nodeId, + Joiner.on(", ").join(partitionIdList) + " of " + store); } - Iterator>> entriesIterator = adminClient.fetchEntries(nodeId, - store, - partitionIdList, - null, - false); + final Iterator>> entriesIterator = adminClient.fetchEntries(nodeId, + store, + partitionIdList, + null, + false); File outputFile = null; if(directory != null) { outputFile = new File(directory, store + ".entries"); } if(useAscii) { - writeEntriesAscii(entriesIterator, outputFile, storeDefinition); + // k-v serializer + SerializerDefinition keySerializerDef = storeDefinition.getKeySerializer(); + SerializerDefinition valueSerializerDef = storeDefinition.getValueSerializer(); + SerializerFactory serializerFactory = new DefaultSerializerFactory(); + @SuppressWarnings("unchecked") + final Serializer keySerializer = (Serializer) serializerFactory.getSerializer(keySerializerDef); + @SuppressWarnings("unchecked") + final Serializer valueSerializer = (Serializer) serializerFactory.getSerializer(valueSerializerDef); + + // compression strategy + final CompressionStrategy keyCompressionStrategy; + final CompressionStrategy valueCompressionStrategy; + if(keySerializerDef != null && keySerializerDef.hasCompression()) { + keyCompressionStrategy = new CompressionStrategyFactory().get(keySerializerDef.getCompression()); + } else { + keyCompressionStrategy = null; + } + if(valueSerializerDef != null && valueSerializerDef.hasCompression()) { + valueCompressionStrategy = new CompressionStrategyFactory().get(valueSerializerDef.getCompression()); + } else { + valueCompressionStrategy = null; + } + + writeAscii(outputFile, new Writable() { + + @Override + public void writeTo(BufferedWriter out) throws IOException { + final StringWriter stringWriter = new StringWriter(); + final JsonGenerator generator = new JsonFactory(new ObjectMapper()).createJsonGenerator(stringWriter); + + while(entriesIterator.hasNext()) { + Pair> kvPair = entriesIterator.next(); + byte[] keyBytes = kvPair.getFirst().get(); + byte[] valueBytes = kvPair.getSecond().getValue(); + VectorClock version = (VectorClock) kvPair.getSecond().getVersion(); + + Object keyObject = keySerializer.toObject((null == keyCompressionStrategy) ? keyBytes + : keyCompressionStrategy.inflate(keyBytes)); + Object valueObject = valueSerializer.toObject((null == valueCompressionStrategy) ? valueBytes + : valueCompressionStrategy.inflate(valueBytes)); + generator.writeObject(keyObject); + stringWriter.write(' '); + stringWriter.write(version.toString()); + generator.writeObject(valueObject); + + StringBuffer buf = stringWriter.getBuffer(); + if(buf.charAt(0) == ' ') { + buf.setCharAt(0, '\n'); + } + out.write(buf.toString()); + buf.setLength(0); + } + out.write('\n'); + } + }); } else { - writeEntriesBinary(entriesIterator, outputFile); + writeBinary(outputFile, new Printable() { + + @Override + public void printTo(DataOutputStream out) throws IOException { + while(entriesIterator.hasNext()) { + Pair> kvPair = entriesIterator.next(); + byte[] keyBytes = kvPair.getFirst().get(); + byte[] versionBytes = ((VectorClock) kvPair.getSecond().getVersion()).toBytes(); + byte[] valueBytes = kvPair.getSecond().getValue(); + out.writeInt(keyBytes.length); + out.write(keyBytes); + out.writeInt(versionBytes.length); + out.write(versionBytes); + out.writeInt(valueBytes.length); + out.write(valueBytes); + } + } + }); } if(outputFile != null) @@ -1098,93 +1171,6 @@ protected Pair> computeNext() { }; } - private static void writeEntriesAscii(Iterator>> iterator, - File outputFile, - StoreDefinition storeDefinition) throws IOException { - BufferedWriter writer = null; - CompressionStrategy keyCompressionStrategy = null; - CompressionStrategy valueCompressionStrategy = null; - - if(outputFile != null) { - writer = new BufferedWriter(new FileWriter(outputFile)); - } else { - writer = new BufferedWriter(new OutputStreamWriter(System.out)); - } - SerializerFactory serializerFactory = new DefaultSerializerFactory(); - StringWriter stringWriter = new StringWriter(); - JsonGenerator generator = new JsonFactory(new ObjectMapper()).createJsonGenerator(stringWriter); - - SerializerDefinition keySerializerDef = storeDefinition.getKeySerializer(); - if(null != keySerializerDef && keySerializerDef.hasCompression()) { - keyCompressionStrategy = new CompressionStrategyFactory().get(keySerializerDef.getCompression()); - } - - SerializerDefinition valueSerializerDef = storeDefinition.getValueSerializer(); - if(null != valueSerializerDef && valueSerializerDef.hasCompression()) { - valueCompressionStrategy = new CompressionStrategyFactory().get(valueSerializerDef.getCompression()); - } - - @SuppressWarnings("unchecked") - Serializer keySerializer = (Serializer) serializerFactory.getSerializer(storeDefinition.getKeySerializer()); - @SuppressWarnings("unchecked") - Serializer valueSerializer = (Serializer) serializerFactory.getSerializer(storeDefinition.getValueSerializer()); - - try { - while(iterator.hasNext()) { - Pair> kvPair = iterator.next(); - byte[] keyBytes = kvPair.getFirst().get(); - VectorClock version = (VectorClock) kvPair.getSecond().getVersion(); - byte[] valueBytes = kvPair.getSecond().getValue(); - - Object keyObject = keySerializer.toObject((null == keyCompressionStrategy) ? keyBytes - : keyCompressionStrategy.inflate(keyBytes)); - Object valueObject = valueSerializer.toObject((null == valueCompressionStrategy) ? valueBytes - : valueCompressionStrategy.inflate(valueBytes)); - - generator.writeObject(keyObject); - stringWriter.write(' '); - stringWriter.write(version.toString()); - generator.writeObject(valueObject); - - StringBuffer buf = stringWriter.getBuffer(); - if(buf.charAt(0) == ' ') { - buf.setCharAt(0, '\n'); - } - writer.write(buf.toString()); - buf.setLength(0); - } - writer.write('\n'); - } finally { - writer.close(); - } - } - - private static void writeEntriesBinary(Iterator>> iterator, - File outputFile) throws IOException { - DataOutputStream dos = null; - if(outputFile != null) { - dos = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(outputFile))); - } else { - dos = new DataOutputStream(new BufferedOutputStream(System.out)); - } - try { - while(iterator.hasNext()) { - Pair> kvPair = iterator.next(); - byte[] keyBytes = kvPair.getFirst().get(); - byte[] versionBytes = ((VectorClock) kvPair.getSecond().getVersion()).toBytes(); - byte[] valueBytes = kvPair.getSecond().getValue(); - dos.writeInt(keyBytes.length); - dos.write(keyBytes); - dos.writeInt(versionBytes.length); - dos.write(versionBytes); - dos.writeInt(valueBytes.length); - dos.write(valueBytes); - } - } finally { - dos.close(); - } - } - private static void executeFetchKeys(Integer nodeId, AdminClient adminClient, List partitionIdList, @@ -1232,20 +1218,65 @@ private static void executeFetchKeys(Integer nodeId, + Joiner.on(", ").join(partitionIdList) + " of " + store); } - Iterator keyIterator = adminClient.fetchKeys(nodeId, - store, - partitionIdList, - null, - false); + final Iterator keyIterator = adminClient.fetchKeys(nodeId, + store, + partitionIdList, + null, + false); File outputFile = null; if(directory != null) { outputFile = new File(directory, store + ".keys"); } if(useAscii) { - writeKeysAscii(keyIterator, outputFile, storeDefinition); + final SerializerDefinition serializerDef = storeDefinition.getKeySerializer(); + final SerializerFactory serializerFactory = new DefaultSerializerFactory(); + @SuppressWarnings("unchecked") + final Serializer serializer = (Serializer) serializerFactory.getSerializer(serializerDef); + + final CompressionStrategy keysCompressionStrategy; + if(serializerDef != null && serializerDef.hasCompression()) { + keysCompressionStrategy = new CompressionStrategyFactory().get(serializerDef.getCompression()); + } else { + keysCompressionStrategy = null; + } + + writeAscii(outputFile, new Writable() { + + @Override + public void writeTo(BufferedWriter out) throws IOException { + final StringWriter stringWriter = new StringWriter(); + final JsonGenerator generator = new JsonFactory(new ObjectMapper()).createJsonGenerator(stringWriter); + + while(keyIterator.hasNext()) { + // Ugly hack to be able to separate text by newlines + // vs. spaces + byte[] keyBytes = keyIterator.next().get(); + Object keyObject = serializer.toObject((null == keysCompressionStrategy) ? keyBytes + : keysCompressionStrategy.inflate(keyBytes)); + generator.writeObject(keyObject); + StringBuffer buf = stringWriter.getBuffer(); + if(buf.charAt(0) == ' ') { + buf.setCharAt(0, '\n'); + } + out.write(buf.toString()); + buf.setLength(0); + } + out.write('\n'); + } + }); } else { - writeKeysBinary(keyIterator, outputFile); + writeBinary(outputFile, new Printable() { + + @Override + public void printTo(DataOutputStream out) throws IOException { + while(keyIterator.hasNext()) { + byte[] keyBytes = keyIterator.next().get(); + out.writeInt(keyBytes.length); + out.write(keyBytes); + } + } + }); } if(outputFile != null) @@ -1253,74 +1284,35 @@ private static void executeFetchKeys(Integer nodeId, } } - private static void writeKeysAscii(Iterator keyIterator, - File outputFile, - StoreDefinition storeDefinition) throws IOException { - BufferedWriter writer = null; - CompressionStrategy keysCompressionStrategy = null; - FileWriter fileWriter = null; - if(outputFile != null) { - fileWriter = new FileWriter(outputFile); - writer = new BufferedWriter(fileWriter); - } else { - writer = new BufferedWriter(new OutputStreamWriter(System.out)); - } + private abstract static class Printable { - SerializerDefinition serializerDef = storeDefinition.getKeySerializer(); - if(null != serializerDef && serializerDef.hasCompression()) { - keysCompressionStrategy = new CompressionStrategyFactory().get(serializerDef.getCompression()); - } + public abstract void printTo(DataOutputStream out) throws IOException; + } + + private abstract static class Writable { + + public abstract void writeTo(BufferedWriter out) throws IOException; + } - SerializerFactory serializerFactory = new DefaultSerializerFactory(); - StringWriter stringWriter = new StringWriter(); - JsonGenerator generator = new JsonFactory(new ObjectMapper()).createJsonGenerator(stringWriter); - @SuppressWarnings("unchecked") - Serializer serializer = (Serializer) serializerFactory.getSerializer(storeDefinition.getKeySerializer()); + private static void writeBinary(File outputFile, Printable printable) throws IOException { + OutputStream outputStream = (outputFile == null) ? System.out + : (new FileOutputStream(outputFile)); + DataOutputStream dataOutputStream = new DataOutputStream(new BufferedOutputStream(outputStream)); try { - while(keyIterator.hasNext()) { - // Ugly hack to be able to separate text by newlines vs. spaces - byte[] keyBytes = keyIterator.next().get(); - Object keyObject = serializer.toObject((null == keysCompressionStrategy) ? keyBytes - : keysCompressionStrategy.inflate(keyBytes)); - generator.writeObject(keyObject); - StringBuffer buf = stringWriter.getBuffer(); - if(buf.charAt(0) == ' ') { - buf.setCharAt(0, '\n'); - } - writer.write(buf.toString()); - buf.setLength(0); - } - writer.write('\n'); + printable.printTo(dataOutputStream); } finally { - if(fileWriter != null) { - fileWriter.close(); - } - writer.close(); + dataOutputStream.close(); } } - private static void writeKeysBinary(Iterator keyIterator, File outputFile) - throws IOException { - DataOutputStream dos = null; - FileOutputStream outputStream = null; - if(outputFile != null) { - outputStream = new FileOutputStream(outputFile); - dos = new DataOutputStream(new BufferedOutputStream(outputStream)); - } else { - dos = new DataOutputStream(new BufferedOutputStream(System.out)); - } - + private static void writeAscii(File outputFile, Writable writable) throws IOException { + Writer writer = (outputFile == null) ? (new OutputStreamWriter(System.out)) + : (new FileWriter(outputFile)); + BufferedWriter bufferedWriter = new BufferedWriter(writer); try { - while(keyIterator.hasNext()) { - byte[] keyBytes = keyIterator.next().get(); - dos.writeInt(keyBytes.length); - dos.write(keyBytes); - } + writable.writeTo(bufferedWriter); } finally { - if(outputStream != null) { - outputStream.close(); - } - dos.close(); + bufferedWriter.close(); } } From b18840b360bbb29b5c9d029ae98876a0b5e76ed1 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Tue, 3 Jul 2012 14:16:20 -0700 Subject: [PATCH 120/209] Added admin option to query keys on specified nodes --- src/java/voldemort/VoldemortAdminTool.java | 123 +++++++++ .../client/protocol/admin/AdminClient.java | 64 +++++ .../client/AdminServiceBasicTest.java | 242 ++++++++++++++++++ 3 files changed, 429 insertions(+) diff --git a/src/java/voldemort/VoldemortAdminTool.java b/src/java/voldemort/VoldemortAdminTool.java index 756a650c5d..1af5334fd9 100644 --- a/src/java/voldemort/VoldemortAdminTool.java +++ b/src/java/voldemort/VoldemortAdminTool.java @@ -34,6 +34,8 @@ import java.io.StringReader; import java.io.StringWriter; import java.io.Writer; +import java.util.ArrayList; +import java.util.Date; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -55,6 +57,7 @@ import voldemort.serialization.Serializer; import voldemort.serialization.SerializerDefinition; import voldemort.serialization.SerializerFactory; +import voldemort.serialization.StringSerializer; import voldemort.server.rebalance.RebalancerState; import voldemort.store.StoreDefinition; import voldemort.store.compress.CompressionStrategy; @@ -130,6 +133,11 @@ public static void main(String[] args) throws Exception { .describedAs("store-names") .withValuesSeparatedBy(',') .ofType(String.class); + parser.accepts("store", "Store name for querying keys") + .withRequiredArg() + .describedAs("store-name") + .withValuesSeparatedBy(',') + .ofType(String.class); parser.accepts("add-stores", "Add stores in this stores.xml") .withRequiredArg() .describedAs("stores.xml containing just the new stores") @@ -226,6 +234,11 @@ public static void main(String[] args) throws Exception { .withRequiredArg() .describedAs("size-in-mb") .ofType(Long.class); + parser.accepts("query-keys", "Get values of keys on specific nodes") + .withRequiredArg() + .describedAs("query-keys") + .withValuesSeparatedBy(',') + .ofType(String.class); OptionSet options = parser.parse(args); @@ -323,6 +336,9 @@ public static void main(String[] args) throws Exception { } ops += "f"; } + if(options.has("query-keys")) { + ops += "q"; + } if(ops.length() < 1) { Utils.croak("At least one of (delete-partitions, restore, add-node, fetch-entries, " + "fetch-keys, add-stores, delete-store, update-entries, get-metadata, ro-metadata, " @@ -496,6 +512,16 @@ public static void main(String[] args) throws Exception { long reserveMB = (Long) options.valueOf("reserve-memory"); adminClient.reserveMemory(nodeId, storeNames, reserveMB); } + if(ops.contains("q")) { + List keyList = null; + String storeName = (String) options.valueOf("store"); + if(storeName == null) { + throw new VoldemortException("Must specify store name using --store option (NOT --stores)"); + } + if(options.hasArgument("query-keys")) + keyList = (List) options.valuesOf("query-keys"); + executeQueryKeys(nodeId, adminClient, storeName, keyList); + } } catch(Exception e) { e.printStackTrace(); Utils.croak(e.getMessage()); @@ -599,6 +625,8 @@ public static void printHelp(PrintStream stream, OptionParser parser) throws IOE stream.println("\t\t./bin/voldemort-admin-tool.sh --fetch-entries --url [url] --node [node-id]"); stream.println("\t9) Update entries for a set of stores using the output from a binary dump fetch entries"); stream.println("\t\t./bin/voldemort-admin-tool.sh --update-entries [folder path from output of --fetch-entries --outdir] --url [url] --node [node-id] --stores [comma-separated list of store names]"); + stream.println("\t10) Query a store for a set of keys on a specific node. Notice that the --store option is not prural"); + stream.println("\t\t./bin/voldemort-admin-tool.sh --query-keys [comma-separated list of keys] --url [url] --node [node-id] --store [store name]"); stream.println(); stream.println("READ-ONLY OPERATIONS"); stream.println("\t1) Retrieve metadata information of read-only data for a particular node and all stores"); @@ -1336,4 +1364,99 @@ private static void executeDeletePartitions(Integer nodeId, adminClient.deletePartitions(nodeId, store, partitionIdList, null); } } + + private static void executeQueryKeys(Integer nodeId, + AdminClient adminClient, + String storeName, + List keys) throws IOException { + Serializer serializer = new StringSerializer(); + List listKeys = new ArrayList(); + for(String key: keys) { + listKeys.add(new ByteArray(serializer.toBytes(key))); + } + final Iterator>, Exception>>> iterator = adminClient.queryKeys(nodeId.intValue(), + storeName, + listKeys.iterator()); + List storeDefinitionList = adminClient.getRemoteStoreDefList(nodeId) + .getValue(); + StoreDefinition storeDefinition = null; + for(StoreDefinition storeDef: storeDefinitionList) { + if(storeDef.getName().equals(storeName)) + storeDefinition = storeDef; + } + + // k-v serializer + SerializerDefinition keySerializerDef = storeDefinition.getKeySerializer(); + SerializerDefinition valueSerializerDef = storeDefinition.getValueSerializer(); + SerializerFactory serializerFactory = new DefaultSerializerFactory(); + @SuppressWarnings("unchecked") + final Serializer keySerializer = (Serializer) serializerFactory.getSerializer(keySerializerDef); + @SuppressWarnings("unchecked") + final Serializer valueSerializer = (Serializer) serializerFactory.getSerializer(valueSerializerDef); + + // compression strategy + final CompressionStrategy keyCompressionStrategy; + final CompressionStrategy valueCompressionStrategy; + if(keySerializerDef != null && keySerializerDef.hasCompression()) { + keyCompressionStrategy = new CompressionStrategyFactory().get(keySerializerDef.getCompression()); + } else { + keyCompressionStrategy = null; + } + if(valueSerializerDef != null && valueSerializerDef.hasCompression()) { + valueCompressionStrategy = new CompressionStrategyFactory().get(valueSerializerDef.getCompression()); + } else { + valueCompressionStrategy = null; + } + + // write to stdout + writeAscii(null, new Writable() { + + @Override + public void writeTo(BufferedWriter out) throws IOException { + final StringWriter stringWriter = new StringWriter(); + final JsonGenerator generator = new JsonFactory(new ObjectMapper()).createJsonGenerator(stringWriter); + + while(iterator.hasNext()) { + Pair>, Exception>> kvPair = iterator.next(); + // unserialize and write key + byte[] keyBytes = kvPair.getFirst().get(); + Object keyObject = keySerializer.toObject((null == keyCompressionStrategy) ? keyBytes + : keyCompressionStrategy.inflate(keyBytes)); + generator.writeObject(keyObject); + + // iterate through, unserialize and write values + List> values = kvPair.getSecond().getFirst(); + if(values != null) { + for(Versioned versioned: values) { + VectorClock version = (VectorClock) versioned.getVersion(); + byte[] valueBytes = versioned.getValue(); + Object valueObject = valueSerializer.toObject((null == valueCompressionStrategy) ? valueBytes + : valueCompressionStrategy.inflate(valueBytes)); + + stringWriter.write(", "); + stringWriter.write(version.toString()); + stringWriter.write('['); + stringWriter.write(new Date(version.getTimestamp()).toString()); + stringWriter.write(']'); + generator.writeObject(valueObject); + + } + } + // write out exception + if(kvPair.getSecond().getSecond() != null) { + stringWriter.write(", "); + stringWriter.write(kvPair.getSecond().getSecond().toString()); + } + + StringBuffer buf = stringWriter.getBuffer(); + if(buf.charAt(0) == ' ') { + buf.setCharAt(0, '\n'); + } + out.write(buf.toString()); + buf.setLength(0); + } + out.write('\n'); + } + }); + } } diff --git a/src/java/voldemort/client/protocol/admin/AdminClient.java b/src/java/voldemort/client/protocol/admin/AdminClient.java index 21c0116904..179b5ee959 100644 --- a/src/java/voldemort/client/protocol/admin/AdminClient.java +++ b/src/java/voldemort/client/protocol/admin/AdminClient.java @@ -58,10 +58,12 @@ import voldemort.cluster.Node; import voldemort.routing.RoutingStrategy; import voldemort.routing.RoutingStrategyFactory; +import voldemort.server.RequestRoutingType; import voldemort.server.protocol.admin.AsyncOperationStatus; import voldemort.server.rebalance.RebalancerState; import voldemort.server.rebalance.VoldemortRebalancingException; import voldemort.store.ErrorCodeMapper; +import voldemort.store.Store; import voldemort.store.StoreDefinition; import voldemort.store.metadata.MetadataStore; import voldemort.store.metadata.MetadataStore.VoldemortState; @@ -73,6 +75,7 @@ import voldemort.store.slop.Slop; import voldemort.store.slop.Slop.Operation; import voldemort.store.socket.SocketDestination; +import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; import voldemort.store.views.ViewStorageConfiguration; import voldemort.utils.ByteArray; import voldemort.utils.ByteUtils; @@ -510,6 +513,67 @@ public Pair> computeNext() { } + /** + * Fetch key/value tuples belonging to a node with given key values + * + *

+ * Entries are being queried synchronously as the iteration happens + * i.e. the whole result set is not buffered in memory. + * + * @param nodeId Id of the node to fetch from + * @param storeName Name of the store + * @param keys An Iterable of keys + * @return An iterator which allows entries to be streamed as they're being + * iterated over. + */ + public Iterator>, Exception>>> queryKeys(int nodeId, + String storeName, + final Iterator keys) { + + Node node = this.getAdminClientCluster().getNodeById(nodeId); + ClientConfig clientConfig = new ClientConfig(); + final Store store; + final ClientRequestExecutorPool clientPool = new ClientRequestExecutorPool(clientConfig.getSelectors(), + clientConfig.getMaxConnectionsPerNode(), + clientConfig.getConnectionTimeout(TimeUnit.MILLISECONDS), + clientConfig.getSocketTimeout(TimeUnit.MILLISECONDS), + clientConfig.getSocketBufferSize(), + clientConfig.getSocketKeepAlive()); + try { + store = clientPool.create(storeName, + node.getHost(), + node.getSocketPort(), + clientConfig.getRequestFormatType(), + RequestRoutingType.IGNORE_CHECKS); + + } catch(Exception e) { + clientPool.close(); + throw new VoldemortException(e); + } + + return new AbstractIterator>, Exception>>>() { + + @Override + public Pair>, Exception>> computeNext() { + ByteArray key; + Exception exception = null; + List> value = null; + if(!keys.hasNext()) { + clientPool.close(); + return endOfData(); + } else { + key = keys.next(); + } + try { + value = store.get(key, null); + } catch(Exception e) { + exception = e; + } + return Pair.create(key, Pair.create(value, exception)); + } + }; + } + /** * Legacy interface for fetching entries. See * {@link AdminClient#fetchKeys(int, String, HashMap, VoldemortFilter, boolean, Cluster, long)} diff --git a/test/unit/voldemort/client/AdminServiceBasicTest.java b/test/unit/voldemort/client/AdminServiceBasicTest.java index 8343bd6855..3adaf6aac0 100644 --- a/test/unit/voldemort/client/AdminServiceBasicTest.java +++ b/test/unit/voldemort/client/AdminServiceBasicTest.java @@ -21,6 +21,7 @@ import java.io.FileOutputStream; import java.io.FileWriter; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.HashMap; @@ -54,6 +55,7 @@ import voldemort.routing.RoutingStrategyType; import voldemort.serialization.SerializerDefinition; import voldemort.server.VoldemortServer; +import voldemort.store.InvalidMetadataException; import voldemort.store.Store; import voldemort.store.StoreDefinition; import voldemort.store.StoreDefinitionBuilder; @@ -67,6 +69,7 @@ import voldemort.store.socket.SocketStoreFactory; import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; import voldemort.utils.ByteArray; +import voldemort.utils.ByteUtils; import voldemort.utils.Pair; import voldemort.utils.RebalanceUtils; import voldemort.utils.Utils; @@ -1255,6 +1258,245 @@ public void testFetch() { } + @Test + public void testQuery() { + HashMap belongToAndInsideServer0 = new HashMap(); + HashMap belongToAndInsideServer1 = new HashMap(); + HashMap notBelongServer0ButInsideServer0 = new HashMap(); + HashMap belongToServer0ButOutsideBoth = new HashMap(); + HashMap notBelongToServer0AndOutsideBoth = new HashMap(); + + Store store0 = getStore(0, testStoreName); + Store store1 = getStore(1, testStoreName); + + HashMap entrySet = null; + Iterator keys = null; + RoutingStrategy strategy = servers[0].getMetadataStore().getRoutingStrategy(testStoreName); + while(true) { + ByteArray key; + byte[] value; + if(keys == null || !keys.hasNext()) { + entrySet = ServerTestUtils.createRandomKeyValuePairs(100); + keys = entrySet.keySet().iterator(); + } + key = keys.next(); + value = entrySet.get(key); + List routedNodes = strategy.routeRequest(key.get()); + boolean keyShouldBeInNode0 = false; + boolean keyShouldBeInNode1 = false; + for(Node node: routedNodes) { + keyShouldBeInNode0 = keyShouldBeInNode0 || (node.getId() == 0); + keyShouldBeInNode1 = keyShouldBeInNode1 || (node.getId() == 1); + } + + if(belongToAndInsideServer0.size() < 10) { + if(keyShouldBeInNode0) { + belongToAndInsideServer0.put(key, value); + store0.put(key, new Versioned(value), null); + } + } else if(belongToAndInsideServer1.size() < 10) { + if(keyShouldBeInNode1) { + belongToAndInsideServer1.put(key, value); + store1.put(key, new Versioned(value), null); + } + } else if(notBelongServer0ButInsideServer0.size() < 5) { + if(!keyShouldBeInNode0) { + notBelongServer0ButInsideServer0.put(key, value); + store0.put(key, new Versioned(value), null); + } + } else if(belongToServer0ButOutsideBoth.size() < 5) { + if(keyShouldBeInNode0) { + belongToServer0ButOutsideBoth.put(key, value); + } + } else if(notBelongToServer0AndOutsideBoth.size() < 5) { + if(!keyShouldBeInNode0) { + notBelongToServer0AndOutsideBoth.put(key, value); + } + } else { + break; + } + } + + ArrayList belongToAndInsideServer0Keys = new ArrayList(belongToAndInsideServer0.keySet()); + ArrayList belongToAndInsideServer1Keys = new ArrayList(belongToAndInsideServer1.keySet()); + ArrayList notBelongServer0ButInsideServer0Keys = new ArrayList(notBelongServer0ButInsideServer0.keySet()); + ArrayList belongToServer0ButOutsideBothKeys = new ArrayList(belongToServer0ButOutsideBoth.keySet()); + ArrayList notBelongToServer0AndOutsideBothKeys = new ArrayList(notBelongToServer0AndOutsideBoth.keySet()); + + List queryKeys; + Iterator>, Exception>>> results; + Pair>, Exception>> entry; + // test one key on store 0 + queryKeys = new ArrayList(); + queryKeys.add(belongToAndInsideServer0Keys.get(0)); + results = getAdminClient().queryKeys(0, testStoreName, queryKeys.iterator()); + assertTrue("Results should not be empty", results.hasNext()); + entry = results.next(); + assertEquals(queryKeys.get(0), entry.getFirst()); + assertNull("There should not be exception in response", entry.getSecond().getSecond()); + assertEquals("There should be only 1 value in versioned list", 1, entry.getSecond() + .getFirst() + .size()); + assertEquals("Two byte[] should be equal", + 0, + ByteUtils.compare(belongToAndInsideServer0.get(queryKeys.get(0)), + entry.getSecond().getFirst().get(0).getValue())); + assertFalse("There should be only one result", results.hasNext()); + + // test one key belongs to but not exists in server 0 + queryKeys = new ArrayList(); + queryKeys.add(belongToServer0ButOutsideBothKeys.get(0)); + results = getAdminClient().queryKeys(0, testStoreName, queryKeys.iterator()); + assertTrue("Results should not be empty", results.hasNext()); + entry = results.next(); + assertFalse("There should not be more results", results.hasNext()); + assertEquals("Not the right key", queryKeys.get(0), entry.getFirst()); + assertNotNull("Response should be non-null", entry.getSecond()); + assertEquals("Value should be empty list", 0, entry.getSecond().getFirst().size()); + assertNull("There should not be exception", entry.getSecond().getSecond()); + + // test one key not exist and does not belong to server 0 + queryKeys = new ArrayList(); + queryKeys.add(notBelongToServer0AndOutsideBothKeys.get(0)); + results = getAdminClient().queryKeys(0, testStoreName, queryKeys.iterator()); + assertTrue("Results should not be empty", results.hasNext()); + entry = results.next(); + assertFalse("There should not be more results", results.hasNext()); + assertEquals("Not the right key", queryKeys.get(0), entry.getFirst()); + assertNotNull("Response should be non-null", entry.getSecond()); + assertNull("Value should be null", entry.getSecond().getFirst()); + assertTrue("There should be InvalidMetadataException exception", + entry.getSecond().getSecond() instanceof InvalidMetadataException); + + // test one key that exists on server 0 but does not belong to server 0 + queryKeys = new ArrayList(); + queryKeys.add(notBelongServer0ButInsideServer0Keys.get(0)); + results = getAdminClient().queryKeys(0, testStoreName, queryKeys.iterator()); + assertTrue("Results should not be empty", results.hasNext()); + entry = results.next(); + assertFalse("There should not be more results", results.hasNext()); + assertEquals("Not the right key", queryKeys.get(0), entry.getFirst()); + assertNotNull("Response should be non-null", entry.getSecond()); + assertNull("Value should be null", entry.getSecond().getFirst()); + assertTrue("There should be InvalidMetadataException exception", + entry.getSecond().getSecond() instanceof InvalidMetadataException); + + // test one key deleted + store0.delete(belongToAndInsideServer0Keys.get(4), null); + queryKeys = new ArrayList(); + queryKeys.add(belongToAndInsideServer0Keys.get(4)); + results = getAdminClient().queryKeys(0, testStoreName, queryKeys.iterator()); + assertTrue("Results should not be empty", results.hasNext()); + entry = results.next(); + assertFalse("There should not be more results", results.hasNext()); + assertEquals("Not the right key", queryKeys.get(0), entry.getFirst()); + assertNotNull("Response should be non-null", entry.getSecond()); + assertEquals("Value should be empty list", 0, entry.getSecond().getFirst().size()); + assertNull("There should not be exception", entry.getSecond().getSecond()); + + // test empty request + queryKeys = new ArrayList(); + results = getAdminClient().queryKeys(0, testStoreName, queryKeys.iterator()); + assertFalse("Results should be empty", results.hasNext()); + + // test null key + queryKeys = new ArrayList(); + queryKeys.add(null); + assertEquals(1, queryKeys.size()); + results = getAdminClient().queryKeys(0, testStoreName, queryKeys.iterator()); + assertTrue("Results should not be empty", results.hasNext()); + entry = results.next(); + assertFalse("There should not be more results", results.hasNext()); + assertNotNull("Response should be non-null", entry.getSecond()); + assertNull("Value should be null", entry.getSecond().getFirst()); + assertTrue("There should be IllegalArgumentException exception", + entry.getSecond().getSecond() instanceof IllegalArgumentException); + + // test multiple keys (3) on store 1 + queryKeys = new ArrayList(); + queryKeys.add(belongToAndInsideServer1Keys.get(0)); + queryKeys.add(belongToAndInsideServer1Keys.get(1)); + queryKeys.add(belongToAndInsideServer1Keys.get(2)); + results = getAdminClient().queryKeys(1, testStoreName, queryKeys.iterator()); + assertTrue("Results should not be empty", results.hasNext()); + Map>> entries = new HashMap>>(); + int resultCount = 0; + while(results.hasNext()) { + resultCount++; + entry = results.next(); + assertNull("There should not be exception in response", entry.getSecond().getSecond()); + assertNotNull("Value should not be null for Key: ", entry.getSecond().getFirst()); + entries.put(entry.getFirst(), entry.getSecond().getFirst()); + } + assertEquals("There should 3 and only 3 results", 3, resultCount); + for(ByteArray key: queryKeys) { + // this loop and the count ensure one-to-one mapping + assertNotNull("This key should exist in the results: " + key, entries.get(key)); + assertEquals("Two byte[] should be equal for key: " + key, + 0, + ByteUtils.compare(belongToAndInsideServer1.get(key), + entries.get(key).get(0).getValue())); + } + + // test multiple keys, mixed situation + // key 0: Exists and belongs to + // key 1: Exists but does not belong to + // key 2: Does not exist but belongs to + // key 3: Does not belong and not exist + // key 4: Same situation with key0 + // key 5: Deleted + // key 6: Same situation with key2 + store0.delete(belongToAndInsideServer0Keys.get(5), null); + queryKeys = new ArrayList(); + queryKeys.add(belongToAndInsideServer0Keys.get(2)); + queryKeys.add(notBelongServer0ButInsideServer0Keys.get(1)); + queryKeys.add(belongToServer0ButOutsideBothKeys.get(1)); + queryKeys.add(notBelongToServer0AndOutsideBothKeys.get(1)); + queryKeys.add(belongToAndInsideServer0Keys.get(3)); + queryKeys.add(belongToAndInsideServer0Keys.get(5)); + queryKeys.add(notBelongServer0ButInsideServer0Keys.get(2)); + results = getAdminClient().queryKeys(0, testStoreName, queryKeys.iterator()); + // key 0 + entry = results.next(); + assertEquals(0, ByteUtils.compare(queryKeys.get(0).get(), entry.getFirst().get())); + assertEquals(0, ByteUtils.compare(belongToAndInsideServer0.get(queryKeys.get(0)), + entry.getSecond().getFirst().get(0).getValue())); + assertNull(entry.getSecond().getSecond()); + // key 1 + entry = results.next(); + assertEquals(0, ByteUtils.compare(queryKeys.get(1).get(), entry.getFirst().get())); + assertTrue("There should be InvalidMetadataException exception", + entry.getSecond().getSecond() instanceof InvalidMetadataException); + // key 2 + entry = results.next(); + assertEquals(0, ByteUtils.compare(queryKeys.get(2).get(), entry.getFirst().get())); + assertEquals(0, entry.getSecond().getFirst().size()); + assertNull(entry.getSecond().getSecond()); + // key 3 + entry = results.next(); + assertEquals(0, ByteUtils.compare(queryKeys.get(3).get(), entry.getFirst().get())); + assertTrue("There should be InvalidMetadataException exception", + entry.getSecond().getSecond() instanceof InvalidMetadataException); + // key 4 + entry = results.next(); + assertEquals(0, ByteUtils.compare(queryKeys.get(4).get(), entry.getFirst().get())); + assertEquals(0, ByteUtils.compare(belongToAndInsideServer0.get(queryKeys.get(4)), + entry.getSecond().getFirst().get(0).getValue())); + assertNull(entry.getSecond().getSecond()); + // key 5 + entry = results.next(); + assertEquals(0, ByteUtils.compare(queryKeys.get(5).get(), entry.getFirst().get())); + assertEquals(0, entry.getSecond().getFirst().size()); + assertNull(entry.getSecond().getSecond()); + // key 6 + entry = results.next(); + assertEquals(0, ByteUtils.compare(queryKeys.get(6).get(), entry.getFirst().get())); + assertTrue("There should be InvalidMetadataException exception", + entry.getSecond().getSecond() instanceof InvalidMetadataException); + // no more keys + assertFalse(results.hasNext()); + } + @Test public void testUpdate() { final HashMap entrySet = ServerTestUtils.createRandomKeyValuePairs(TEST_STREAM_KEYS_SIZE); From ef91db78ea7b9546fca781b8b7277de8f3c7b920 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Thu, 30 Aug 2012 13:32:05 -0700 Subject: [PATCH 121/209] Deleted multiple store detection for --store option --- src/java/voldemort/VoldemortAdminTool.java | 1 - 1 file changed, 1 deletion(-) diff --git a/src/java/voldemort/VoldemortAdminTool.java b/src/java/voldemort/VoldemortAdminTool.java index 1af5334fd9..8e7c23dd2b 100644 --- a/src/java/voldemort/VoldemortAdminTool.java +++ b/src/java/voldemort/VoldemortAdminTool.java @@ -136,7 +136,6 @@ public static void main(String[] args) throws Exception { parser.accepts("store", "Store name for querying keys") .withRequiredArg() .describedAs("store-name") - .withValuesSeparatedBy(',') .ofType(String.class); parser.accepts("add-stores", "Add stores in this stores.xml") .withRequiredArg() From a33b280fc8068284ea1f7bcc3a6b47d5db65db80 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Wed, 5 Sep 2012 17:03:31 -0700 Subject: [PATCH 122/209] Reuses --stores option and added multiple store query support. Fixed bug that prevents queryKeys, fetchEntries, fetchKeys to print results beyond the first store --- src/java/voldemort/VoldemortAdminTool.java | 206 ++++++++++++--------- 1 file changed, 116 insertions(+), 90 deletions(-) diff --git a/src/java/voldemort/VoldemortAdminTool.java b/src/java/voldemort/VoldemortAdminTool.java index 8e7c23dd2b..080d2f3e34 100644 --- a/src/java/voldemort/VoldemortAdminTool.java +++ b/src/java/voldemort/VoldemortAdminTool.java @@ -27,6 +27,7 @@ import java.io.FileNotFoundException; import java.io.FileOutputStream; import java.io.FileWriter; +import java.io.FilterOutputStream; import java.io.IOException; import java.io.OutputStream; import java.io.OutputStreamWriter; @@ -512,14 +513,11 @@ public static void main(String[] args) throws Exception { adminClient.reserveMemory(nodeId, storeNames, reserveMB); } if(ops.contains("q")) { - List keyList = null; - String storeName = (String) options.valueOf("store"); - if(storeName == null) { - throw new VoldemortException("Must specify store name using --store option (NOT --stores)"); + List keyList = (List) options.valuesOf("query-keys"); + if(storeNames == null || storeNames.size() == 0) { + throw new VoldemortException("Must specify store name using --stores option"); } - if(options.hasArgument("query-keys")) - keyList = (List) options.valuesOf("query-keys"); - executeQueryKeys(nodeId, adminClient, storeName, keyList); + executeQueryKeys(nodeId, adminClient, storeNames, keyList); } } catch(Exception e) { e.printStackTrace(); @@ -1322,8 +1320,18 @@ private abstract static class Writable { } private static void writeBinary(File outputFile, Printable printable) throws IOException { - OutputStream outputStream = (outputFile == null) ? System.out - : (new FileOutputStream(outputFile)); + OutputStream outputStream = null; + if(outputFile == null) { + outputStream = new FilterOutputStream(System.out) { + + @Override + public void close() throws IOException { + flush(); + } + }; + } else { + outputStream = new FileOutputStream(outputFile); + } DataOutputStream dataOutputStream = new DataOutputStream(new BufferedOutputStream(outputStream)); try { printable.printTo(dataOutputStream); @@ -1333,8 +1341,18 @@ private static void writeBinary(File outputFile, Printable printable) throws IOE } private static void writeAscii(File outputFile, Writable writable) throws IOException { - Writer writer = (outputFile == null) ? (new OutputStreamWriter(System.out)) - : (new FileWriter(outputFile)); + Writer writer = null; + if(outputFile == null) { + writer = new OutputStreamWriter(new FilterOutputStream(System.out) { + + @Override + public void close() throws IOException { + flush(); + } + }); + } else { + writer = new FileWriter(outputFile); + } BufferedWriter bufferedWriter = new BufferedWriter(writer); try { writable.writeTo(bufferedWriter); @@ -1364,98 +1382,106 @@ private static void executeDeletePartitions(Integer nodeId, } } - private static void executeQueryKeys(Integer nodeId, + private static void executeQueryKeys(final Integer nodeId, AdminClient adminClient, - String storeName, + List storeNames, List keys) throws IOException { Serializer serializer = new StringSerializer(); List listKeys = new ArrayList(); for(String key: keys) { listKeys.add(new ByteArray(serializer.toBytes(key))); } - final Iterator>, Exception>>> iterator = adminClient.queryKeys(nodeId.intValue(), - storeName, - listKeys.iterator()); - List storeDefinitionList = adminClient.getRemoteStoreDefList(nodeId) - .getValue(); - StoreDefinition storeDefinition = null; - for(StoreDefinition storeDef: storeDefinitionList) { - if(storeDef.getName().equals(storeName)) - storeDefinition = storeDef; - } - - // k-v serializer - SerializerDefinition keySerializerDef = storeDefinition.getKeySerializer(); - SerializerDefinition valueSerializerDef = storeDefinition.getValueSerializer(); - SerializerFactory serializerFactory = new DefaultSerializerFactory(); - @SuppressWarnings("unchecked") - final Serializer keySerializer = (Serializer) serializerFactory.getSerializer(keySerializerDef); - @SuppressWarnings("unchecked") - final Serializer valueSerializer = (Serializer) serializerFactory.getSerializer(valueSerializerDef); - - // compression strategy - final CompressionStrategy keyCompressionStrategy; - final CompressionStrategy valueCompressionStrategy; - if(keySerializerDef != null && keySerializerDef.hasCompression()) { - keyCompressionStrategy = new CompressionStrategyFactory().get(keySerializerDef.getCompression()); - } else { - keyCompressionStrategy = null; - } - if(valueSerializerDef != null && valueSerializerDef.hasCompression()) { - valueCompressionStrategy = new CompressionStrategyFactory().get(valueSerializerDef.getCompression()); - } else { - valueCompressionStrategy = null; - } - - // write to stdout - writeAscii(null, new Writable() { + for(final String storeName: storeNames) { + final Iterator>, Exception>>> iterator = adminClient.queryKeys(nodeId.intValue(), + storeName, + listKeys.iterator()); + List storeDefinitionList = adminClient.getRemoteStoreDefList(nodeId) + .getValue(); + StoreDefinition storeDefinition = null; + for(StoreDefinition storeDef: storeDefinitionList) { + if(storeDef.getName().equals(storeName)) + storeDefinition = storeDef; + } - @Override - public void writeTo(BufferedWriter out) throws IOException { - final StringWriter stringWriter = new StringWriter(); - final JsonGenerator generator = new JsonFactory(new ObjectMapper()).createJsonGenerator(stringWriter); - - while(iterator.hasNext()) { - Pair>, Exception>> kvPair = iterator.next(); - // unserialize and write key - byte[] keyBytes = kvPair.getFirst().get(); - Object keyObject = keySerializer.toObject((null == keyCompressionStrategy) ? keyBytes - : keyCompressionStrategy.inflate(keyBytes)); - generator.writeObject(keyObject); - - // iterate through, unserialize and write values - List> values = kvPair.getSecond().getFirst(); - if(values != null) { - for(Versioned versioned: values) { - VectorClock version = (VectorClock) versioned.getVersion(); - byte[] valueBytes = versioned.getValue(); - Object valueObject = valueSerializer.toObject((null == valueCompressionStrategy) ? valueBytes - : valueCompressionStrategy.inflate(valueBytes)); + // k-v serializer + SerializerDefinition keySerializerDef = storeDefinition.getKeySerializer(); + SerializerDefinition valueSerializerDef = storeDefinition.getValueSerializer(); + SerializerFactory serializerFactory = new DefaultSerializerFactory(); + @SuppressWarnings("unchecked") + final Serializer keySerializer = (Serializer) serializerFactory.getSerializer(keySerializerDef); + @SuppressWarnings("unchecked") + final Serializer valueSerializer = (Serializer) serializerFactory.getSerializer(valueSerializerDef); + + // compression strategy + final CompressionStrategy keyCompressionStrategy; + final CompressionStrategy valueCompressionStrategy; + if(keySerializerDef != null && keySerializerDef.hasCompression()) { + keyCompressionStrategy = new CompressionStrategyFactory().get(keySerializerDef.getCompression()); + } else { + keyCompressionStrategy = null; + } + if(valueSerializerDef != null && valueSerializerDef.hasCompression()) { + valueCompressionStrategy = new CompressionStrategyFactory().get(valueSerializerDef.getCompression()); + } else { + valueCompressionStrategy = null; + } + // write to stdout + writeAscii(null, new Writable() { + + @Override + public void writeTo(BufferedWriter out) throws IOException { + final StringWriter stringWriter = new StringWriter(); + final JsonGenerator generator = new JsonFactory(new ObjectMapper()).createJsonGenerator(stringWriter); + stringWriter.write("Querying keys in node " + nodeId + " of " + storeName + + "\n"); + + while(iterator.hasNext()) { + Pair>, Exception>> kvPair = iterator.next(); + // unserialize and write key + byte[] keyBytes = kvPair.getFirst().get(); + Object keyObject = keySerializer.toObject((null == keyCompressionStrategy) ? keyBytes + : keyCompressionStrategy.inflate(keyBytes)); + generator.writeObject(keyObject); + + // iterate through, unserialize and write values + List> values = kvPair.getSecond().getFirst(); + if(values != null) { + if(values.size() == 0) { + stringWriter.write(", null"); + } + for(Versioned versioned: values) { + VectorClock version = (VectorClock) versioned.getVersion(); + byte[] valueBytes = versioned.getValue(); + Object valueObject = valueSerializer.toObject((null == valueCompressionStrategy) ? valueBytes + : valueCompressionStrategy.inflate(valueBytes)); + + stringWriter.write(", "); + stringWriter.write(version.toString()); + stringWriter.write('['); + stringWriter.write(new Date(version.getTimestamp()).toString()); + stringWriter.write(']'); + generator.writeObject(valueObject); + } + } else { + stringWriter.write(", null"); + } + // write out exception + if(kvPair.getSecond().getSecond() != null) { stringWriter.write(", "); - stringWriter.write(version.toString()); - stringWriter.write('['); - stringWriter.write(new Date(version.getTimestamp()).toString()); - stringWriter.write(']'); - generator.writeObject(valueObject); - + stringWriter.write(kvPair.getSecond().getSecond().toString()); } - } - // write out exception - if(kvPair.getSecond().getSecond() != null) { - stringWriter.write(", "); - stringWriter.write(kvPair.getSecond().getSecond().toString()); - } - StringBuffer buf = stringWriter.getBuffer(); - if(buf.charAt(0) == ' ') { - buf.setCharAt(0, '\n'); + StringBuffer buf = stringWriter.getBuffer(); + if(buf.charAt(0) == ' ') { + buf.setCharAt(0, '\n'); + } + out.write(buf.toString()); + buf.setLength(0); } - out.write(buf.toString()); - buf.setLength(0); + out.write('\n'); } - out.write('\n'); - } - }); + }); + } } } From 878af0451185d5b4572767fae37eb70018b0b6c3 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Thu, 30 Aug 2012 17:36:28 -0700 Subject: [PATCH 123/209] Readded peter's change --- .../server/niosocket/AsyncRequestHandler.java | 11 +++-- .../vold/VoldemortNativeRequestHandler.java | 34 ++++++++------ .../voldemort/store/bdb/BdbStorageEngine.java | 10 ++--- .../store/routed/PipelineRoutedStore.java | 45 +++++++++++-------- .../routed/action/AbstractReadRepair.java | 11 +++-- .../action/PerformParallelDeleteRequests.java | 6 +-- .../action/PerformParallelPutRequests.java | 4 +- .../action/PerformParallelRequests.java | 16 ++++--- .../action/PerformSerialGetAllRequests.java | 19 ++++---- .../action/PerformSerialPutRequests.java | 12 ++--- .../routed/action/PerformSerialRequests.java | 8 ++-- src/java/voldemort/utils/ByteArray.java | 14 ++++++ 12 files changed, 116 insertions(+), 74 deletions(-) diff --git a/src/java/voldemort/server/niosocket/AsyncRequestHandler.java b/src/java/voldemort/server/niosocket/AsyncRequestHandler.java index 27772e7740..d6af05b642 100644 --- a/src/java/voldemort/server/niosocket/AsyncRequestHandler.java +++ b/src/java/voldemort/server/niosocket/AsyncRequestHandler.java @@ -127,13 +127,16 @@ protected void read(SelectionKey selectionKey) throws IOException { if(logger.isTraceEnabled()) logger.trace("Starting execution for " + socketChannel.socket()); - streamRequestHandler = requestHandler.handleRequest(new DataInputStream(inputStream), - new DataOutputStream(outputStream)); + DataInputStream dataInputStream = new DataInputStream(inputStream); + DataOutputStream dataOutputStream = new DataOutputStream(outputStream); + + streamRequestHandler = requestHandler.handleRequest(dataInputStream, + dataOutputStream); if(logger.isDebugEnabled()) { logger.debug("AsyncRequestHandler:read finished request from " + socketChannel.socket().getRemoteSocketAddress() + " handlerRef: " - + System.identityHashCode(streamRequestHandler) + " at time: " + + System.identityHashCode(dataInputStream) + " at time: " + System.currentTimeMillis() + " elapsed time: " + (System.nanoTime() - startNs) + " ns"); } @@ -307,7 +310,7 @@ private StreamRequestHandlerState handleStreamRequestInternal(SelectionKey selec if(logger.isDebugEnabled()) { logger.debug("Handled request from " + socketChannel.socket().getRemoteSocketAddress() + " handlerRef: " - + System.identityHashCode(streamRequestHandler) + " at time: " + + System.identityHashCode(dataInputStream) + " at time: " + System.currentTimeMillis() + " elapsed time: " + (System.nanoTime() - startNs) + " ns"); } diff --git a/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java b/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java index fa70a4b3eb..cadce8b8bb 100644 --- a/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java +++ b/src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java @@ -135,7 +135,8 @@ private void handleGetVersion(DataInputStream inputStream, if(logger.isDebugEnabled()) { logger.debug("GETVERSIONS started at: " + startTimeMs + " handlerRef: " - + System.identityHashCode(this) + " key: " + key + " " + + System.identityHashCode(inputStream) + " key: " + + ByteUtils.toHexString(key.get()) + " " + (System.nanoTime() - startTimeNs) + " ns, keySize: " + key.length() + "clocks: " + clockStr); } @@ -317,7 +318,7 @@ private void handleGet(DataInputStream inputStream, } writeResults(outputStream, results); if(logger.isDebugEnabled()) { - debugLogReturnValue(key, results, startTimeMs, startTimeNs, "GET"); + debugLogReturnValue(inputStream, key, results, startTimeMs, startTimeNs, "GET"); } } @@ -374,7 +375,8 @@ private void handleGetAll(DataInputStream inputStream, writeResults(outputStream, entry.getValue()); if(logger.isDebugEnabled()) { - debugLogReturnValue(entry.getKey(), + debugLogReturnValue(inputStream, + entry.getKey(), entry.getValue(), startTimeMs, startTimeNs, @@ -386,7 +388,8 @@ private void handleGetAll(DataInputStream inputStream, logger.debug("GETALL end"); } - private void debugLogReturnValue(ByteArray key, + private void debugLogReturnValue(DataInputStream input, + ByteArray key, List> values, long startTimeMs, long startTimeNs, @@ -406,12 +409,13 @@ private void debugLogReturnValue(ByteArray key, valueHashStr += "]"; versionsStr += "]"; - logger.debug(getType + " handlerRef: " + System.identityHashCode(this) + " start time: " - + startTimeMs + " key: " + key + " elapsed time: " - + (System.nanoTime() - startTimeNs) + " ns, keySize: " + key.length() - + " numResults: " + values.size() + " totalResultSize: " + totalValueSize - + " resultSizes: " + valueSizeStr + " resultHashes: " + valueHashStr - + " versions: " + versionsStr + " current time: " + System.currentTimeMillis()); + logger.debug(getType + " handlerRef: " + System.identityHashCode(input) + " start time: " + + startTimeMs + " key: " + ByteUtils.toHexString(key.get()) + + " elapsed time: " + (System.nanoTime() - startTimeNs) + " ns, keySize: " + + key.length() + " numResults: " + values.size() + " totalResultSize: " + + totalValueSize + " resultSizes: " + valueSizeStr + " resultHashes: " + + valueHashStr + " versions: " + versionsStr + " current time: " + + System.currentTimeMillis()); } private void handlePut(DataInputStream inputStream, @@ -447,9 +451,10 @@ private void handlePut(DataInputStream inputStream, if(logger.isDebugEnabled()) { logger.debug("PUT started at: " + startTimeMs + " handlerRef: " - + System.identityHashCode(this) + " key: " + key + " " + + System.identityHashCode(inputStream) + " key: " + + ByteUtils.toHexString(key.get()) + " " + (System.nanoTime() - startTimeNs) + " ns, keySize: " + key.length() - + " valueHash: " + value.hashCode() + " valueSize: " + valueSize + + " valueHash: " + value.hashCode() + " valueSize: " + value.length + " clockSize: " + clock.sizeInBytes() + " time: " + System.currentTimeMillis()); } @@ -480,8 +485,9 @@ private void handleDelete(DataInputStream inputStream, } if(logger.isDebugEnabled()) { - logger.debug("DELETE started at: " + startTimeMs + " key: " + key + " handlerRef: " - + System.identityHashCode(this) + " time: " + logger.debug("DELETE started at: " + startTimeMs + " key: " + + ByteUtils.toHexString(key.get()) + " handlerRef: " + + System.identityHashCode(inputStream) + " time: " + (System.nanoTime() - startTimeNs) + " ns, keySize: " + key.length() + " clockSize: " + version.sizeInBytes()); } diff --git a/src/java/voldemort/store/bdb/BdbStorageEngine.java b/src/java/voldemort/store/bdb/BdbStorageEngine.java index 1b2013ca46..b74e31cfe6 100644 --- a/src/java/voldemort/store/bdb/BdbStorageEngine.java +++ b/src/java/voldemort/store/bdb/BdbStorageEngine.java @@ -281,7 +281,7 @@ public Map>> getAll(Iterable keys, for(ByteArray key: keys) { if(logger.isTraceEnabled()) - keyStr += key + " "; + keyStr += ByteUtils.toHexString(key.get()) + " "; List> values = get(cursor, key, readLockMode, versionedSerializer); if(!values.isEmpty()) @@ -324,7 +324,7 @@ private static List get(Cursor cursor, } if(logger.isTraceEnabled()) { - logger.trace("Completed GET from key " + key + " in " + logger.trace("Completed GET from key " + ByteUtils.toHexString(key.get()) + " in " + (System.nanoTime() - startTimeNs) + " ns at " + System.currentTimeMillis()); } @@ -390,7 +390,7 @@ else if(occurred == Occurred.AFTER) } if(logger.isTraceEnabled()) { - logger.trace("Completed PUT to key " + key + " (keyRef: " + logger.trace("Completed PUT to key " + ByteUtils.toHexString(key.get()) + " (keyRef: " + System.identityHashCode(key) + " value " + value + " in " + (System.nanoTime() - startTimeNs) + " ns at " + System.currentTimeMillis()); @@ -431,8 +431,8 @@ public boolean delete(ByteArray key, Version version) throws PersistenceFailureE } finally { if(logger.isTraceEnabled()) { - logger.trace("Completed DELETE of key " + key + " (keyRef: " - + System.identityHashCode(key) + ") in " + logger.trace("Completed DELETE of key " + ByteUtils.toHexString(key.get()) + + " (keyRef: " + System.identityHashCode(key) + ") in " + (System.nanoTime() - startTimeNs) + " ns at " + System.currentTimeMillis()); } diff --git a/src/java/voldemort/store/routed/PipelineRoutedStore.java b/src/java/voldemort/store/routed/PipelineRoutedStore.java index 1dffd8111f..b444f947a9 100644 --- a/src/java/voldemort/store/routed/PipelineRoutedStore.java +++ b/src/java/voldemort/store/routed/PipelineRoutedStore.java @@ -254,9 +254,10 @@ public List> request(Store store) { } if(logger.isDebugEnabled()) { - logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + key - + " keyRef: " + System.identityHashCode(key) + "; started at " - + startTimeMs + " took " + (System.nanoTime() - startTimeNs) + " values: " + logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + + ByteUtils.toHexString(key.get()) + " keyRef: " + + System.identityHashCode(key) + "; started at " + startTimeMs + " took " + + (System.nanoTime() - startTimeNs) + " values: " + formatNodeValuesFromGet(pipelineData.getResponses())); } @@ -358,9 +359,10 @@ public Map>> getAll(Iterable keys, throw pipelineData.getFatalError(); if(logger.isDebugEnabled()) { - logger.debug("Finished " + pipeline.getOperation().getSimpleName() + "for keys " + keys - + " keyRef: " + System.identityHashCode(keys) + "; started at " - + startTimeMs + " took " + (System.nanoTime() - startTimeNs) + " values: " + logger.debug("Finished " + pipeline.getOperation().getSimpleName() + "for keys " + + ByteArray.toHexStrings(keys) + " keyRef: " + + System.identityHashCode(keys) + "; started at " + startTimeMs + " took " + + (System.nanoTime() - startTimeNs) + " values: " + formatNodeValuesFromGetAll(pipelineData.getResponses())); } @@ -372,8 +374,9 @@ private String formatNodeValuesFromGetAll(List, Map StringBuilder builder = new StringBuilder(); builder.append("{"); for(Response, Map>>> r: list) { - builder.append("(nodeId=" + r.getNode().getId() + ", key=" + r.getKey() - + ", retrieved= " + r.getValue() + ")"); + builder.append("(nodeId=" + r.getNode().getId() + ", keys=" + + ByteArray.toHexStrings(r.getKey()) + ", retrieved= " + r.getValue() + + ")"); builder.append(", "); } builder.append("}"); @@ -472,9 +475,10 @@ public List request(Store store) { results.addAll(response.getValue()); if(logger.isDebugEnabled()) { - logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + key - + " keyRef: " + System.identityHashCode(key) + "; started at " - + startTimeMs + " took " + (System.nanoTime() - startTimeNs) + " values: " + logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + + ByteUtils.toHexString(key.get()) + " keyRef: " + + System.identityHashCode(key) + "; started at " + startTimeMs + " took " + + (System.nanoTime() - startTimeNs) + " values: " + formatNodeValuesFromGetVersions(pipelineData.getResponses())); } @@ -486,8 +490,9 @@ private String formatNodeValuesFromGetVersions(List> r: results) { - builder.append("(nodeId=" + r.getNode().getId() + ", key=" + r.getKey() - + ", retrieved= " + r.getValue() + "), "); + builder.append("(nodeId=" + r.getNode().getId() + ", key=" + + ByteUtils.toHexString(r.getKey().get()) + ", retrieved= " + + r.getValue() + "), "); } builder.append("}"); @@ -578,8 +583,9 @@ public boolean delete(final ByteArray key, final Version version) throws Voldemo if(logger.isDebugEnabled()) { logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " - + key.get() + " keyRef: " + System.identityHashCode(key) + "; started at " - + startTimeMs + " took " + (System.nanoTime() - startTimeNs)); + + ByteUtils.toHexString(key.get()) + " keyRef: " + + System.identityHashCode(key) + "; started at " + startTimeMs + " took " + + (System.nanoTime() - startTimeNs)); } if(pipelineData.getFatalError() != null) @@ -703,10 +709,11 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) } if(logger.isDebugEnabled()) { - logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + key - + " keyRef: " + System.identityHashCode(key) + "; started at " - + startTimeMs + " took " + (System.nanoTime() - startTimeNs) + " value: " - + versioned.getValue() + " (size: " + versioned.getValue().length + ")"); + logger.debug("Finished " + pipeline.getOperation().getSimpleName() + " for key " + + ByteUtils.toHexString(key.get()) + " keyRef: " + + System.identityHashCode(key) + "; started at " + startTimeMs + " took " + + (System.nanoTime() - startTimeNs) + " value: " + versioned.getValue() + + " (size: " + versioned.getValue().length + ")"); } if(pipelineData.getFatalError() != null) diff --git a/src/java/voldemort/store/routed/action/AbstractReadRepair.java b/src/java/voldemort/store/routed/action/AbstractReadRepair.java index 6e2ea19d6f..ac2dac7829 100644 --- a/src/java/voldemort/store/routed/action/AbstractReadRepair.java +++ b/src/java/voldemort/store/routed/action/AbstractReadRepair.java @@ -24,10 +24,11 @@ import voldemort.store.nonblockingstore.NonblockingStore; import voldemort.store.routed.NodeValue; import voldemort.store.routed.Pipeline; +import voldemort.store.routed.Pipeline.Event; import voldemort.store.routed.PipelineData; import voldemort.store.routed.ReadRepairer; -import voldemort.store.routed.Pipeline.Event; import voldemort.utils.ByteArray; +import voldemort.utils.ByteUtils; import voldemort.versioning.VectorClock; import voldemort.versioning.Versioned; @@ -101,7 +102,8 @@ public void execute(Pipeline pipeline) { try { if(logger.isDebugEnabled()) logger.debug("Doing read repair on node " + v.getNodeId() + " for key '" - + v.getKey() + "' with version " + v.getVersion() + "."); + + ByteUtils.toHexString(v.getKey().get()) + "' with version " + + v.getVersion() + "."); NonblockingStore store = nonblockingStores.get(v.getNodeId()); store.submitPutRequest(v.getKey(), v.getVersioned(), null, null, timeoutMs); @@ -110,8 +112,9 @@ public void execute(Pipeline pipeline) { logger.debug("Read repair cancelled due to application level exception on node " + v.getNodeId() + " for key '" - + v.getKey() - + "' with version " + v.getVersion() + ": " + e.getMessage()); + + ByteUtils.toHexString(v.getKey().get()) + + "' with version " + + v.getVersion() + ": " + e.getMessage()); } catch(Exception e) { logger.debug("Read repair failed: ", e); } diff --git a/src/java/voldemort/store/routed/action/PerformParallelDeleteRequests.java b/src/java/voldemort/store/routed/action/PerformParallelDeleteRequests.java index 7f30c90a9b..634b470805 100644 --- a/src/java/voldemort/store/routed/action/PerformParallelDeleteRequests.java +++ b/src/java/voldemort/store/routed/action/PerformParallelDeleteRequests.java @@ -93,9 +93,9 @@ public void execute(final Pipeline pipeline) { public void requestComplete(Object result, long requestTime) { if(logger.isTraceEnabled()) - logger.info(pipeline.getOperation().getSimpleName() - + " response received (" + requestTime + " ms.) from node " - + node.getId()); + logger.trace(pipeline.getOperation().getSimpleName() + + " response received (" + requestTime + " ms.) from node " + + node.getId()); Response response = new Response(node, key, diff --git a/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java b/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java index a8373418b5..b2e3bc09be 100644 --- a/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java +++ b/src/java/voldemort/store/routed/action/PerformParallelPutRequests.java @@ -42,6 +42,7 @@ import voldemort.store.slop.HintedHandoff; import voldemort.store.slop.Slop; import voldemort.utils.ByteArray; +import voldemort.utils.ByteUtils; import voldemort.utils.Time; import voldemort.versioning.ObsoleteVersionException; import voldemort.versioning.Versioned; @@ -130,7 +131,8 @@ public void requestComplete(Object result, long requestTime) { responses.put(node.getId(), response); if(logger.isDebugEnabled()) - logger.debug("Finished secondary PUT for key " + key + " (keyRef: " + logger.debug("Finished secondary PUT for key " + + ByteUtils.toHexString(key.get()) + " (keyRef: " + System.identityHashCode(key) + "); took " + requestTime + " ms on node " + node.getId() + "(" + node.getHost() + ")"); diff --git a/src/java/voldemort/store/routed/action/PerformParallelRequests.java b/src/java/voldemort/store/routed/action/PerformParallelRequests.java index 73399d75b0..18385a5f7f 100644 --- a/src/java/voldemort/store/routed/action/PerformParallelRequests.java +++ b/src/java/voldemort/store/routed/action/PerformParallelRequests.java @@ -38,6 +38,7 @@ import voldemort.store.routed.Pipeline.Operation; import voldemort.store.routed.Response; import voldemort.utils.ByteArray; +import voldemort.utils.ByteUtils; import voldemort.utils.Utils; public class PerformParallelRequests> extends @@ -111,10 +112,10 @@ public void requestComplete(Object result, long requestTime) { requestTime); if(logger.isDebugEnabled()) logger.debug("Finished " + pipeline.getOperation().getSimpleName() - + " for key " + key + " (keyRef: " - + System.identityHashCode(key) + "); started at " + startMs - + " took " + requestTime + " ms on node " + node.getId() + "(" - + node.getHost() + ")"); + + " for key " + ByteUtils.toHexString(key.get()) + + " (keyRef: " + System.identityHashCode(key) + + "); started at " + startMs + " took " + requestTime + + " ms on node " + node.getId() + "(" + node.getHost() + ")"); responses.put(node.getId(), response); latch.countDown(); @@ -174,9 +175,10 @@ else if(pipeline.getOperation() == Operation.GET_VERSIONS) } if(logger.isDebugEnabled()) - logger.debug("GET for key " + key + " (keyRef: " + System.identityHashCode(key) - + "); successes: " + pipelineData.getSuccesses() + " preferred: " - + preferred + " required: " + required); + logger.debug("GET for key " + ByteUtils.toHexString(key.get()) + " (keyRef: " + + System.identityHashCode(key) + "); successes: " + + pipelineData.getSuccesses() + " preferred: " + preferred + " required: " + + required); if(pipelineData.getSuccesses() < required) { if(insufficientSuccessesEvent != null) { diff --git a/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java b/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java index f1719a8f55..0d8b551d4b 100644 --- a/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java +++ b/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java @@ -80,9 +80,10 @@ public void execute(Pipeline pipeline) { MutableInt successCount = pipelineData.getSuccessCount(key); if(logger.isDebugEnabled()) - logger.debug("GETALL for key " + key + " (keyRef: " + System.identityHashCode(key) - + ") successes: " + successCount.intValue() + " preferred: " - + preferred + " required: " + required); + logger.debug("GETALL for key " + ByteUtils.toHexString(key.get()) + " (keyRef: " + + System.identityHashCode(key) + ") successes: " + + successCount.intValue() + " preferred: " + preferred + " required: " + + required); if(successCount.intValue() >= preferred) { if(pipelineData.getZonesRequired() != null && pipelineData.getZonesRequired() > 0) { @@ -138,11 +139,11 @@ public void execute(Pipeline pipeline) { failureDetector.recordSuccess(response.getNode(), response.getRequestTime()); if(logger.isDebugEnabled()) - logger.debug("GET for key " + key + " (keyRef: " - + System.identityHashCode(key) + ") successes: " - + successCount.intValue() + " preferred: " + preferred - + " required: " + required + " new GET success on node " - + node.getId()); + logger.debug("GET for key " + ByteUtils.toHexString(key.get()) + + " (keyRef: " + System.identityHashCode(key) + + ") successes: " + successCount.intValue() + " preferred: " + + preferred + " required: " + required + + " new GET success on node " + node.getId()); HashSet zoneResponses = null; if(pipelineData.getKeyToZoneResponse().containsKey(key)) { @@ -201,4 +202,4 @@ public void execute(Pipeline pipeline) { pipeline.addEvent(completeEvent); } -} \ No newline at end of file +} diff --git a/src/java/voldemort/store/routed/action/PerformSerialPutRequests.java b/src/java/voldemort/store/routed/action/PerformSerialPutRequests.java index 7b7b3d65fd..08a25c7bc7 100644 --- a/src/java/voldemort/store/routed/action/PerformSerialPutRequests.java +++ b/src/java/voldemort/store/routed/action/PerformSerialPutRequests.java @@ -29,6 +29,7 @@ import voldemort.store.routed.Pipeline.Event; import voldemort.store.routed.PutPipelineData; import voldemort.utils.ByteArray; +import voldemort.utils.ByteUtils; import voldemort.utils.Time; import voldemort.versioning.VectorClock; import voldemort.versioning.Versioned; @@ -173,7 +174,8 @@ public void execute(Pipeline pipeline) { } else { if(logger.isDebugEnabled()) - logger.debug("Finished master PUT for key " + key + " (keyRef: " + logger.debug("Finished master PUT for key " + + ByteUtils.toHexString(key.get()) + " (keyRef: " + System.identityHashCode(key) + "); started at " + startMasterMs + " took " + (System.nanoTime() - startMasterNs) + " ns on node " @@ -185,10 +187,10 @@ public void execute(Pipeline pipeline) { } } else { if(logger.isDebugEnabled()) - logger.debug("Finished master PUT for key " + key + " (keyRef: " - + System.identityHashCode(key) + "); started at " + startMasterMs - + " took " + (System.nanoTime() - startMasterNs) + " ns on node " - + (node == null ? "NULL" : node.getId()) + "(" + logger.debug("Finished master PUT for key " + ByteUtils.toHexString(key.get()) + + " (keyRef: " + System.identityHashCode(key) + "); started at " + + startMasterMs + " took " + (System.nanoTime() - startMasterNs) + + " ns on node " + (node == null ? "NULL" : node.getId()) + "(" + (node == null ? "NULL" : node.getHost()) + ")"); pipeline.addEvent(masterDeterminedEvent); diff --git a/src/java/voldemort/store/routed/action/PerformSerialRequests.java b/src/java/voldemort/store/routed/action/PerformSerialRequests.java index 4c447174d8..6feafd140c 100644 --- a/src/java/voldemort/store/routed/action/PerformSerialRequests.java +++ b/src/java/voldemort/store/routed/action/PerformSerialRequests.java @@ -31,6 +31,7 @@ import voldemort.store.routed.Pipeline.Event; import voldemort.store.routed.Response; import voldemort.utils.ByteArray; +import voldemort.utils.ByteUtils; import voldemort.utils.Time; public class PerformSerialRequests> extends @@ -99,9 +100,10 @@ public void execute(Pipeline pipeline) { ((System.nanoTime() - start) / Time.NS_PER_MS)); if(logger.isDebugEnabled()) - logger.debug(pipeline.getOperation().getSimpleName() + " for key " + key - + " successes: " + pipelineData.getSuccesses() + " preferred: " - + preferred + " required: " + required + " new " + logger.debug(pipeline.getOperation().getSimpleName() + " for key " + + ByteUtils.toHexString(key.get()) + " successes: " + + pipelineData.getSuccesses() + " preferred: " + preferred + + " required: " + required + " new " + pipeline.getOperation().getSimpleName() + " success on node " + node.getId()); diff --git a/src/java/voldemort/utils/ByteArray.java b/src/java/voldemort/utils/ByteArray.java index 71dc609b2d..c6ccf86ada 100644 --- a/src/java/voldemort/utils/ByteArray.java +++ b/src/java/voldemort/utils/ByteArray.java @@ -1,6 +1,7 @@ package voldemort.utils; import java.io.Serializable; +import java.util.ArrayList; import java.util.Arrays; /** @@ -43,6 +44,19 @@ public String toString() { return Arrays.toString(underlying); } + /** + * Translate the each ByteArray in an iterable into a hexidecimal string + * + * @param arrays The array of bytes to translate + * @return An iterable of converted strings + */ + public static Iterable toHexStrings(Iterable arrays) { + ArrayList ret = new ArrayList(); + for(ByteArray array: arrays) + ret.add(ByteUtils.toHexString(array.get())); + return ret; + } + public int length() { return underlying.length; } From bca4a12e937859dbee15f9cfa6d91fec183d0deb Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Thu, 6 Sep 2012 13:40:02 -0700 Subject: [PATCH 124/209] Added possible fix for rebalance tests random failures --- src/java/voldemort/store/metadata/MetadataStore.java | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/src/java/voldemort/store/metadata/MetadataStore.java b/src/java/voldemort/store/metadata/MetadataStore.java index 5a82acd99e..ace349a23e 100644 --- a/src/java/voldemort/store/metadata/MetadataStore.java +++ b/src/java/voldemort/store/metadata/MetadataStore.java @@ -191,8 +191,9 @@ public synchronized void put(String key, Versioned value) { public void put(String key, Object value) { if(METADATA_KEYS.contains(key)) { VectorClock version = (VectorClock) get(key, null).get(0).getVersion(); - put(key, new Versioned(value, version.incremented(getNodeId(), - System.currentTimeMillis()))); + put(key, + new Versioned(value, version.incremented(getNodeId(), + System.currentTimeMillis()))); } else { throw new VoldemortException("Unhandled Key:" + key + " for MetadataStore put()"); } @@ -316,7 +317,12 @@ public VoldemortState getServerState() { } public RebalancerState getRebalancerState() { - return (RebalancerState) metadataCache.get(REBALANCING_STEAL_INFO).getValue(); + readLock.lock(); + try { + return (RebalancerState) metadataCache.get(REBALANCING_STEAL_INFO).getValue(); + } finally { + readLock.unlock(); + } } @SuppressWarnings("unchecked") From 31bab1966660fbac44e5d9539eeb1a8290af51e4 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Thu, 6 Sep 2012 14:27:09 -0700 Subject: [PATCH 125/209] Corrected help message for a33b280fc8 --- src/java/voldemort/VoldemortAdminTool.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/java/voldemort/VoldemortAdminTool.java b/src/java/voldemort/VoldemortAdminTool.java index 080d2f3e34..69d004b93c 100644 --- a/src/java/voldemort/VoldemortAdminTool.java +++ b/src/java/voldemort/VoldemortAdminTool.java @@ -622,8 +622,8 @@ public static void printHelp(PrintStream stream, OptionParser parser) throws IOE stream.println("\t\t./bin/voldemort-admin-tool.sh --fetch-entries --url [url] --node [node-id]"); stream.println("\t9) Update entries for a set of stores using the output from a binary dump fetch entries"); stream.println("\t\t./bin/voldemort-admin-tool.sh --update-entries [folder path from output of --fetch-entries --outdir] --url [url] --node [node-id] --stores [comma-separated list of store names]"); - stream.println("\t10) Query a store for a set of keys on a specific node. Notice that the --store option is not prural"); - stream.println("\t\t./bin/voldemort-admin-tool.sh --query-keys [comma-separated list of keys] --url [url] --node [node-id] --store [store name]"); + stream.println("\t10) Query stores for a set of keys on a specific node."); + stream.println("\t\t./bin/voldemort-admin-tool.sh --query-keys [comma-separated list of keys] --url [url] --node [node-id] --stores [comma-separated list of store names]"); stream.println(); stream.println("READ-ONLY OPERATIONS"); stream.println("\t1) Retrieve metadata information of read-only data for a particular node and all stores"); From 440832e87686741a5d173b3d8785cada2b655f1d Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Mon, 10 Sep 2012 23:10:11 -0700 Subject: [PATCH 126/209] Code cleanup, bug fixes for system stores and auto-rebootstrapper --- src/java/voldemort/client/ClientConfig.java | 3 +- src/java/voldemort/client/SystemStore.java | 19 +++--- .../client/protocol/admin/AdminClient.java | 21 ++++-- .../AsyncMetadataVersionManager.java | 68 +++++++++++-------- .../FailureDetectorConfig.java | 3 +- .../server/storage/StorageService.java | 28 -------- ...FileBackedCachingStorageConfiguration.java | 1 - .../FileBackedCachingStorageEngine.java | 4 +- .../voldemort/store/logging/LoggingStore.java | 5 +- .../action/ConfigureNodesLocalHost.java | 4 ++ .../utils/MetadataVersionStoreUtils.java | 22 ++++-- .../FailureDetectorPerformanceTest.java | 4 +- .../RoutedStoreParallelismTest.java | 2 +- test/unit/voldemort/client/ClientJmxTest.java | 9 ++- .../client/EndToEndRebootstrapTest.java | 4 +- test/unit/voldemort/cluster/TestCluster.java | 4 +- .../BannagePeriodFailureDetectorTest.java | 4 +- .../ThresholdFailureDetectorTest.java | 11 ++- .../scheduled/StreamingSlopPusherTest.java | 12 ++-- .../store/routed/HintedHandoffTest.java | 2 +- .../store/routed/ReadRepairerTest.java | 2 +- .../store/routed/RoutedStoreTest.java | 2 +- .../routed/action/AbstractActionTest.java | 4 +- .../action/ConfigureNodesLocalHostTest.java | 2 +- 24 files changed, 128 insertions(+), 112 deletions(-) diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index 66b85c915a..0091110301 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -626,8 +626,9 @@ public int getClientZoneId() { return this.clientZoneId; } - public void enableDefaultClient(boolean enableDefault) { + public ClientConfig enableDefaultClient(boolean enableDefault) { this.useDefaultClient = enableDefault; + return this; } public boolean isDefaultClientEnabled() { diff --git a/src/java/voldemort/client/SystemStore.java b/src/java/voldemort/client/SystemStore.java index 5ae4913550..814265b732 100644 --- a/src/java/voldemort/client/SystemStore.java +++ b/src/java/voldemort/client/SystemStore.java @@ -46,7 +46,11 @@ public SystemStore(String storeName, .setClientZoneId(clientZoneID); this.systemStoreFactory = new SystemStoreClientFactory(config); this.storeName = storeName; - this.sysStore = this.systemStoreFactory.getSystemStore(this.storeName, clusterXml, fd); + try { + this.sysStore = this.systemStoreFactory.getSystemStore(this.storeName, clusterXml, fd); + } catch(Exception e) { + logger.debug("Error while creating a system store client for store : " + this.storeName); + } } public Version putSysStore(K key, V value) { @@ -61,9 +65,8 @@ public Version putSysStore(K key, V value) { this.sysStore.put(key, versioned, null); version = versioned.getVersion(); } catch(Exception e) { - logger.info("Exception caught during putSysStore: " + e); if(logger.isDebugEnabled()) { - e.printStackTrace(); + logger.debug("Exception caught during putSysStore: " + e); } } return version; @@ -76,9 +79,8 @@ public Version putSysStore(K key, Versioned value) { this.sysStore.put(key, value, null); version = value.getVersion(); } catch(Exception e) { - logger.info("Exception caught during putSysStore: " + e); if(logger.isDebugEnabled()) { - e.printStackTrace(); + logger.debug("Exception caught during putSysStore: " + e); } } return version; @@ -89,6 +91,7 @@ public Versioned getSysStore(K key) { Versioned versioned = null; try { List> items = this.sysStore.get(key, null); + if(items.size() == 1) versioned = items.get(0); else if(items.size() > 1) @@ -100,9 +103,8 @@ else if(items.size() > 1) else logger.debug("Got null value"); } catch(Exception e) { - logger.info("Exception caught during getSysStore: " + e); if(logger.isDebugEnabled()) { - e.printStackTrace(); + logger.debug("Exception caught during getSysStore: " + e); } } return versioned; @@ -119,9 +121,8 @@ public V getValueSysStore(K key) { value = versioned.getValue(); } } catch(Exception e) { - logger.info("Exception caught during getSysStore: " + e); if(logger.isDebugEnabled()) { - e.printStackTrace(); + logger.debug("Exception caught during getSysStore: " + e); } } return value; diff --git a/src/java/voldemort/client/protocol/admin/AdminClient.java b/src/java/voldemort/client/protocol/admin/AdminClient.java index e54afba1de..e17193a0a5 100644 --- a/src/java/voldemort/client/protocol/admin/AdminClient.java +++ b/src/java/voldemort/client/protocol/admin/AdminClient.java @@ -213,9 +213,13 @@ public AdminClient(String bootstrapURL, AdminClientConfig adminClientConfig, int private void initSystemStoreClient(String bootstrapURL, int zoneID) { String[] bootstrapUrls = new String[1]; bootstrapUrls[0] = bootstrapURL; - this.sysStoreVersion = new SystemStore(SystemStoreConstants.SystemStoreName.voldsys$_metadata_version_persistence.name(), - bootstrapUrls, - zoneID); + try { + this.sysStoreVersion = new SystemStore(SystemStoreConstants.SystemStoreName.voldsys$_metadata_version_persistence.name(), + bootstrapUrls, + zoneID); + } catch(Exception e) { + logger.debug("Error while creating a system store client for metadata version store."); + } } /** @@ -226,12 +230,15 @@ private void initSystemStoreClient(String bootstrapURL, int zoneID) { */ public void updateMetadataversion(String versionKey) { Properties props = MetadataVersionStoreUtils.getProperties(this.sysStoreVersion); - if(props.getProperty(versionKey) != null) { + if(props != null && props.getProperty(versionKey) != null) { logger.debug("Version obtained = " + props.getProperty(versionKey)); long newValue = Long.parseLong(props.getProperty(versionKey)) + 1; props.setProperty(versionKey, Long.toString(newValue)); } else { logger.debug("Current version is null. Assuming version 0."); + if(props == null) { + props = new Properties(); + } props.setProperty(versionKey, "0"); } MetadataVersionStoreUtils.setProperties(this.sysStoreVersion, props); @@ -2403,7 +2410,11 @@ public void rebalanceStateChange(Cluster existingCluster, * metadata */ if(changeClusterMetadata) { - updateMetadataversion(CLUSTER_VERSION_KEY); + try { + updateMetadataversion(CLUSTER_VERSION_KEY); + } catch(Exception e) { + logger.info("Exception occurred while setting cluster metadata version during Rebalance state change !!!"); + } } } catch(Exception e) { diff --git a/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java b/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java index a9d5029e8b..fe55c4c8d4 100644 --- a/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java +++ b/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java @@ -94,7 +94,7 @@ public Long fetchNewVersion(String versionKey, Long curVersion, Properties versi // Swallow all exceptions here (we dont want to fail the client). catch(Exception e) { - logger.info("Could not retrieve Metadata Version. Exception : " + e); + logger.debug("Could not retrieve Metadata Version."); } return null; @@ -102,43 +102,51 @@ public Long fetchNewVersion(String versionKey, Long curVersion, Properties versi public void run() { - // Get the properties object from the system store (containing versions) - Properties versionProps = MetadataVersionStoreUtils.getProperties(this.sysRepository.getMetadataVersionStore()); - Long newClusterVersion = fetchNewVersion(CLUSTER_VERSION_KEY, - currentClusterVersion, - versionProps); - - // If nothing has been updated, continue - if(newClusterVersion != null) { - - logger.info("Metadata version mismatch detected."); + try { + // Get the properties object from the system store (containing + // versions) + Properties versionProps = MetadataVersionStoreUtils.getProperties(this.sysRepository.getMetadataVersionStore()); + Long newClusterVersion = fetchNewVersion(CLUSTER_VERSION_KEY, + currentClusterVersion, + versionProps); - // Determine a random delta delay between 0 to DELTA_MAX to sleep - int delta = randomGenerator.nextInt(DELTA_MAX); + // If nothing has been updated, continue + if(newClusterVersion != null) { - try { - logger.info("Sleeping for delta : " + delta + " (ms) before re-bootstrapping."); - Thread.sleep(delta); - } catch(InterruptedException e) { - // do nothing, continue. - } + logger.info("Metadata version mismatch detected."); - /* - * Do another check for mismatch here since the versions might have - * been updated while we were sleeping - */ - if(!newClusterVersion.equals(currentClusterVersion)) { + // Determine a random delta delay between 0 to DELTA_MAX to + // sleep + int delta = randomGenerator.nextInt(DELTA_MAX); try { - logger.info("Updating cluster version"); - currentClusterVersion = newClusterVersion; + logger.info("Sleeping for delta : " + delta + " (ms) before re-bootstrapping."); + Thread.sleep(delta); + } catch(InterruptedException e) { + // do nothing, continue. + } - this.storeClientThunk.call(); - } catch(Exception e) { - e.printStackTrace(); - logger.info(e.getMessage()); + /* + * Do another check for mismatch here since the versions might + * have been updated while we were sleeping + */ + if(!newClusterVersion.equals(currentClusterVersion)) { + + try { + logger.info("Updating cluster version"); + currentClusterVersion = newClusterVersion; + + this.storeClientThunk.call(); + } catch(Exception e) { + if(logger.isDebugEnabled()) { + e.printStackTrace(); + logger.debug(e.getMessage()); + } + } } } + } catch(Exception e) { + logger.debug("Could not retrieve metadata versions from the server."); } } diff --git a/src/java/voldemort/cluster/failuredetector/FailureDetectorConfig.java b/src/java/voldemort/cluster/failuredetector/FailureDetectorConfig.java index be389cdd7c..5856d22254 100644 --- a/src/java/voldemort/cluster/failuredetector/FailureDetectorConfig.java +++ b/src/java/voldemort/cluster/failuredetector/FailureDetectorConfig.java @@ -548,7 +548,6 @@ public FailureDetectorConfig setRequestLengthThreshold(long requestLengthThresho */ public synchronized Cluster getCluster() { - System.err.println("Cluster = " + cluster); return this.cluster; } @@ -577,6 +576,7 @@ public synchronized FailureDetectorConfig setCluster(Cluster cluster) { */ public synchronized Collection getNodes() { + System.err.println("DEPRECATED !!! Please use getCluster().getNodes() method instead !"); return ImmutableSet.copyOf(this.cluster.getNodes()); } @@ -593,6 +593,7 @@ public synchronized Collection getNodes() { */ public synchronized FailureDetectorConfig setNodes(Collection nodes) { + System.err.println("DEPRECATED !!! Please use setCluster method instead !"); Utils.notNull(nodes); this.nodes = new HashSet(nodes); return this; diff --git a/src/java/voldemort/server/storage/StorageService.java b/src/java/voldemort/server/storage/StorageService.java index daacf7e8a2..940f5f49f1 100644 --- a/src/java/voldemort/server/storage/StorageService.java +++ b/src/java/voldemort/server/storage/StorageService.java @@ -372,7 +372,6 @@ protected void initializeMetadataVersions(List storeDefs) { for(String propName: props.stringPropertyNames()) { finalVersionList.append(propName + "=" + props.getProperty(propName) + "\n"); } - System.err.println(finalVersionList); versionStore.put(metadataVersionsKey, new Versioned(finalVersionList.toString().getBytes(), newClock), null); @@ -424,33 +423,6 @@ public void registerSystemEngine(StorageEngine engine store = new LoggingStore(store, cluster.getName(), SystemTime.INSTANCE); - /* TODO: Do we really need rebalancing for system stores? */ - if(voldemortConfig.isEnableRebalanceService()) { - store = new RedirectingStore(store, - metadata, - storeRepository, - failureDetector, - storeFactory); - if(voldemortConfig.isJmxEnabled()) { - MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer(); - ObjectName name = null; - if(this.voldemortConfig.isEnableJmxClusterName()) - name = JmxUtils.createObjectName(cluster.getName() - + "." - + JmxUtils.getPackageName(RedirectingStore.class), - store.getName()); - else - name = JmxUtils.createObjectName(JmxUtils.getPackageName(RedirectingStore.class), - store.getName()); - - synchronized(mbeanServer) { - if(mbeanServer.isRegistered(name)) - JmxUtils.unregisterMbean(mbeanServer, name); - JmxUtils.registerMbean(mbeanServer, JmxUtils.createModelMBean(store), name); - } - - } - } if(voldemortConfig.isMetadataCheckingEnabled()) store = new InvalidMetadataCheckingStore(metadata.getNodeId(), store, metadata); diff --git a/src/java/voldemort/store/configuration/FileBackedCachingStorageConfiguration.java b/src/java/voldemort/store/configuration/FileBackedCachingStorageConfiguration.java index c43c781e3a..d9570c393f 100644 --- a/src/java/voldemort/store/configuration/FileBackedCachingStorageConfiguration.java +++ b/src/java/voldemort/store/configuration/FileBackedCachingStorageConfiguration.java @@ -12,7 +12,6 @@ public class FileBackedCachingStorageConfiguration implements StorageConfigurati public FileBackedCachingStorageConfiguration(VoldemortConfig config) { this.inputPath = config.getMetadataDirectory(); - System.err.println("Created a new File backed caching engine config ..."); } public StorageEngine getStore(String name) { diff --git a/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java b/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java index 0b4d0961ba..62830af188 100644 --- a/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java +++ b/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java @@ -54,7 +54,9 @@ public FileBackedCachingStorageEngine(String name, String inputDirectory) { this.inputPath = inputDirectory + "/" + name; this.metadataMap = new HashMap(); this.loadData(); - logger.debug("Created a new File backed caching engine. File location = " + inputPath); + if(logger.isDebugEnabled()) { + logger.debug("Created a new File backed caching engine. File location = " + inputPath); + } } private File getVersionFile() { diff --git a/src/java/voldemort/store/logging/LoggingStore.java b/src/java/voldemort/store/logging/LoggingStore.java index 753834f8ee..cac0acebb9 100644 --- a/src/java/voldemort/store/logging/LoggingStore.java +++ b/src/java/voldemort/store/logging/LoggingStore.java @@ -128,9 +128,8 @@ public void put(K key, Versioned value, T transform) throws VoldemortExceptio private void printTimedMessage(String operation, boolean success, long startNs) { if(logger.isDebugEnabled()) { double elapsedMs = (time.getNanoseconds() - startNs) / (double) Time.NS_PER_MS; - logger.debug(instanceName + operation + " " + getName() - + " " + (success ? "successful" : "unsuccessful") + " in " - + elapsedMs + " ms"); + logger.debug(instanceName + operation + " " + getName() + " " + + (success ? "successful" : "unsuccessful") + " in " + elapsedMs + " ms"); } } diff --git a/src/java/voldemort/store/routed/action/ConfigureNodesLocalHost.java b/src/java/voldemort/store/routed/action/ConfigureNodesLocalHost.java index 4eb2237380..443a538b25 100644 --- a/src/java/voldemort/store/routed/action/ConfigureNodesLocalHost.java +++ b/src/java/voldemort/store/routed/action/ConfigureNodesLocalHost.java @@ -47,6 +47,10 @@ public List getNodes(ByteArray key) { try { nodes = super.getNodes(key); + if(nodes == null) { + return null; + } + String currentHost = InetAddress.getLocalHost().getHostName(); for(Node n: nodes) { if(currentHost.contains(n.getHost()) || n.getHost().contains(currentHost)) { diff --git a/src/java/voldemort/utils/MetadataVersionStoreUtils.java b/src/java/voldemort/utils/MetadataVersionStoreUtils.java index 5809a9c5c9..dc6d917052 100644 --- a/src/java/voldemort/utils/MetadataVersionStoreUtils.java +++ b/src/java/voldemort/utils/MetadataVersionStoreUtils.java @@ -44,14 +44,22 @@ public static Properties getProperties(SystemStore versionStore) * @param props The Properties object to write to the System store */ public static void setProperties(SystemStore versionStore, Properties props) { - StringBuilder finalVersionList = new StringBuilder(); - for(String propName: props.stringPropertyNames()) { - if(finalVersionList.length() == 0) { - finalVersionList.append(propName + "=" + props.getProperty(propName)); - } else { - finalVersionList.append("\n" + propName + "=" + props.getProperty(propName)); + if(props == null) { + return; + } + + try { + StringBuilder finalVersionList = new StringBuilder(); + for(String propName: props.stringPropertyNames()) { + if(finalVersionList.length() == 0) { + finalVersionList.append(propName + "=" + props.getProperty(propName)); + } else { + finalVersionList.append("\n" + propName + "=" + props.getProperty(propName)); + } } + versionStore.putSysStore(VERSIONS_METADATA_KEY, finalVersionList.toString()); + } catch(Exception e) { + logger.debug("Got exception in setting properties : " + e.getMessage()); } - versionStore.putSysStore(VERSIONS_METADATA_KEY, finalVersionList.toString()); } } diff --git a/test/integration/voldemort/cluster/failuredetector/FailureDetectorPerformanceTest.java b/test/integration/voldemort/cluster/failuredetector/FailureDetectorPerformanceTest.java index f8775b9ff9..d9d76a08f4 100644 --- a/test/integration/voldemort/cluster/failuredetector/FailureDetectorPerformanceTest.java +++ b/test/integration/voldemort/cluster/failuredetector/FailureDetectorPerformanceTest.java @@ -16,9 +16,9 @@ package voldemort.cluster.failuredetector; -import static voldemort.cluster.failuredetector.MutableStoreVerifier.create; import static voldemort.VoldemortTestConstants.getNineNodeCluster; import static voldemort.cluster.failuredetector.FailureDetectorUtils.create; +import static voldemort.cluster.failuredetector.MutableStoreVerifier.create; import java.io.IOException; @@ -71,7 +71,7 @@ protected FailureDetectorPerformanceTest(String[] args) { failureDetectorConfig.getThresholdInterval()); Cluster cluster = getNineNodeCluster(); - failureDetectorConfig.setNodes(cluster.getNodes()) + failureDetectorConfig.setCluster(cluster) .setStoreVerifier(create(cluster.getNodes())) .setAsyncRecoveryInterval(asyncScanInterval) .setBannagePeriod(bannagePeriod) diff --git a/test/integration/voldemort/performance/RoutedStoreParallelismTest.java b/test/integration/voldemort/performance/RoutedStoreParallelismTest.java index 0397416613..aecfd0394a 100644 --- a/test/integration/voldemort/performance/RoutedStoreParallelismTest.java +++ b/test/integration/voldemort/performance/RoutedStoreParallelismTest.java @@ -195,7 +195,7 @@ public static void main(String[] args) throws Throwable { } FailureDetectorConfig failureDetectorConfig = new FailureDetectorConfig().setImplementationClassName(BannagePeriodFailureDetector.class.getName()) - .setNodes(cluster.getNodes()) + .setCluster(cluster) .setStoreVerifier(MutableStoreVerifier.create(stores)); FailureDetector failureDetector = FailureDetectorUtils.create(failureDetectorConfig, false); diff --git a/test/unit/voldemort/client/ClientJmxTest.java b/test/unit/voldemort/client/ClientJmxTest.java index cb67bb1ff7..1e5529c09e 100644 --- a/test/unit/voldemort/client/ClientJmxTest.java +++ b/test/unit/voldemort/client/ClientJmxTest.java @@ -74,7 +74,8 @@ public void tearDown() throws Exception { protected StoreClientFactory getFactory(String... bootstrapUrls) { return new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrls) .setEnableLazy(false) - .setEnableJmx(true)); + .setEnableJmx(true) + .enableDefaultClient(true)); } protected StoreClientFactory getFactoryWithClientContext(String clientContext, @@ -82,7 +83,8 @@ protected StoreClientFactory getFactoryWithClientContext(String clientContext, return new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrls) .setEnableLazy(false) .setClientContextName(clientContext) - .setEnableJmx(true)); + .setEnableJmx(true) + .enableDefaultClient(true)); } @Test @@ -280,7 +282,8 @@ protected StoreClientFactory getFactoryWithSerializer(SerializerFactory factory, String... bootstrapUrls) { return new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrls) .setEnableLazy(false) - .setSerializerFactory(factory)); + .setSerializerFactory(factory) + .enableDefaultClient(true)); } @Override diff --git a/test/unit/voldemort/client/EndToEndRebootstrapTest.java b/test/unit/voldemort/client/EndToEndRebootstrapTest.java index 024e6a2873..2202bfb41e 100644 --- a/test/unit/voldemort/client/EndToEndRebootstrapTest.java +++ b/test/unit/voldemort/client/EndToEndRebootstrapTest.java @@ -115,7 +115,9 @@ public void testEndToEndRebootstrap() { String value = "SF"; String bootstrapTime = ""; String newBootstrapTime = ""; - AdminClient adminClient = new AdminClient(bootStrapUrls[0], new AdminClientConfig()); + AdminClient adminClient = new AdminClient(bootStrapUrls[0], + new AdminClientConfig(), + CLIENT_ZONE_ID); try { storeClient.put(key, value); diff --git a/test/unit/voldemort/cluster/TestCluster.java b/test/unit/voldemort/cluster/TestCluster.java index 3e516005b8..1229c43d5f 100644 --- a/test/unit/voldemort/cluster/TestCluster.java +++ b/test/unit/voldemort/cluster/TestCluster.java @@ -16,8 +16,8 @@ package voldemort.cluster; -import static voldemort.cluster.failuredetector.MutableStoreVerifier.create; import static voldemort.cluster.failuredetector.FailureDetectorUtils.create; +import static voldemort.cluster.failuredetector.MutableStoreVerifier.create; import java.util.Arrays; import java.util.Collection; @@ -69,7 +69,7 @@ public void setUp() throws Exception { FailureDetectorConfig failureDetectorConfig = new FailureDetectorConfig().setImplementationClassName(failureDetectorClass.getName()) .setBannagePeriod(1000) - .setNodes(cluster.getNodes()) + .setCluster(cluster) .setStoreVerifier(create(cluster.getNodes())) .setTime(time); diff --git a/test/unit/voldemort/cluster/failuredetector/BannagePeriodFailureDetectorTest.java b/test/unit/voldemort/cluster/failuredetector/BannagePeriodFailureDetectorTest.java index 7acd98423e..ef2fd3252d 100644 --- a/test/unit/voldemort/cluster/failuredetector/BannagePeriodFailureDetectorTest.java +++ b/test/unit/voldemort/cluster/failuredetector/BannagePeriodFailureDetectorTest.java @@ -20,8 +20,8 @@ import static org.junit.Assert.assertTrue; import static voldemort.FailureDetectorTestUtils.recordException; import static voldemort.FailureDetectorTestUtils.recordSuccess; -import static voldemort.cluster.failuredetector.MutableStoreVerifier.create; import static voldemort.cluster.failuredetector.FailureDetectorUtils.create; +import static voldemort.cluster.failuredetector.MutableStoreVerifier.create; import java.net.ConnectException; import java.net.NoRouteToHostException; @@ -42,7 +42,7 @@ public class BannagePeriodFailureDetectorTest extends AbstractFailureDetectorTes public FailureDetector createFailureDetector() throws Exception { FailureDetectorConfig failureDetectorConfig = new FailureDetectorConfig().setImplementationClassName(BannagePeriodFailureDetector.class.getName()) .setBannagePeriod(BANNAGE_MILLIS) - .setNodes(cluster.getNodes()) + .setCluster(cluster) .setStoreVerifier(create(cluster.getNodes())) .setTime(time); return create(failureDetectorConfig, true); diff --git a/test/unit/voldemort/cluster/failuredetector/ThresholdFailureDetectorTest.java b/test/unit/voldemort/cluster/failuredetector/ThresholdFailureDetectorTest.java index 7a9e409846..25aa19ebc5 100644 --- a/test/unit/voldemort/cluster/failuredetector/ThresholdFailureDetectorTest.java +++ b/test/unit/voldemort/cluster/failuredetector/ThresholdFailureDetectorTest.java @@ -20,9 +20,9 @@ import static org.junit.Assert.assertTrue; import static voldemort.FailureDetectorTestUtils.recordException; import static voldemort.FailureDetectorTestUtils.recordSuccess; -import static voldemort.cluster.failuredetector.MutableStoreVerifier.create; -import static voldemort.cluster.failuredetector.FailureDetectorUtils.create; import static voldemort.VoldemortTestConstants.getTenNodeCluster; +import static voldemort.cluster.failuredetector.FailureDetectorUtils.create; +import static voldemort.cluster.failuredetector.MutableStoreVerifier.create; import java.net.ConnectException; import java.net.NoRouteToHostException; @@ -48,7 +48,7 @@ public FailureDetector createFailureDetector() throws Exception { .setBannagePeriod(BANNAGE_MILLIS) .setAsyncRecoveryInterval(250) .setThresholdInterval(500) - .setNodes(cluster.getNodes()) + .setCluster(cluster) .setStoreVerifier(storeVerifier) .setTime(time); return create(failureDetectorConfig, true); @@ -179,6 +179,11 @@ public void testChangeMetadata() throws Exception { new UnreachableStoreException("intentionalerror", new ConnectException("intentionalerror"))); + /** + * Update the failure detector state with the new cluster + */ + failureDetector.getConfig().setCluster(this.cluster); + assertEquals(false, failureDetector.isAvailable(node)); Thread.sleep(failureDetector.getConfig().getAsyncRecoveryInterval() * 2); assertEquals(true, failureDetector.isAvailable(node)); diff --git a/test/unit/voldemort/scheduled/StreamingSlopPusherTest.java b/test/unit/voldemort/scheduled/StreamingSlopPusherTest.java index f0a045face..2c3a732f5a 100644 --- a/test/unit/voldemort/scheduled/StreamingSlopPusherTest.java +++ b/test/unit/voldemort/scheduled/StreamingSlopPusherTest.java @@ -136,7 +136,7 @@ public void testFailedServer() throws IOException, InterruptedException { StreamingSlopPusherJob pusher = new StreamingSlopPusherJob(getVoldemortServer(0).getStoreRepository(), getVoldemortServer(0).getMetadataStore(), - new BannagePeriodFailureDetector(new FailureDetectorConfig().setNodes(cluster.getNodes()) + new BannagePeriodFailureDetector(new FailureDetectorConfig().setCluster(cluster) .setStoreVerifier(new ServerStoreVerifier(socketStoreFactory, metadataStore, configs[0]))), @@ -270,7 +270,7 @@ public void testOutOfOrder() throws InterruptedException, IOException { StreamingSlopPusherJob pusher = new StreamingSlopPusherJob(getVoldemortServer(0).getStoreRepository(), getVoldemortServer(0).getMetadataStore(), - new BannagePeriodFailureDetector(new FailureDetectorConfig().setNodes(cluster.getNodes()) + new BannagePeriodFailureDetector(new FailureDetectorConfig().setCluster(cluster) .setStoreVerifier(new ServerStoreVerifier(socketStoreFactory, metadataStore, configs[0]))), @@ -321,7 +321,7 @@ public void testNormalPush() throws InterruptedException, IOException { StreamingSlopPusherJob pusher = new StreamingSlopPusherJob(getVoldemortServer(0).getStoreRepository(), getVoldemortServer(0).getMetadataStore(), - new BannagePeriodFailureDetector(new FailureDetectorConfig().setNodes(cluster.getNodes()) + new BannagePeriodFailureDetector(new FailureDetectorConfig().setCluster(cluster) .setStoreVerifier(new ServerStoreVerifier(socketStoreFactory, metadataStore, configs[0]))), @@ -437,14 +437,14 @@ public void testNormalPushBothWays() throws InterruptedException, IOException { StreamingSlopPusherJob pusher0 = new StreamingSlopPusherJob(getVoldemortServer(0).getStoreRepository(), getVoldemortServer(0).getMetadataStore(), - new BannagePeriodFailureDetector(new FailureDetectorConfig().setNodes(cluster.getNodes()) + new BannagePeriodFailureDetector(new FailureDetectorConfig().setCluster(cluster) .setStoreVerifier(new ServerStoreVerifier(socketStoreFactory, metadataStore, configs[0]))), configs[0], new ScanPermitWrapper(1)), pusher1 = new StreamingSlopPusherJob(getVoldemortServer(1).getStoreRepository(), getVoldemortServer(1).getMetadataStore(), - new BannagePeriodFailureDetector(new FailureDetectorConfig().setNodes(cluster.getNodes()) + new BannagePeriodFailureDetector(new FailureDetectorConfig().setCluster(cluster) .setStoreVerifier(new ServerStoreVerifier(socketStoreFactory, metadataStore, configs[1]))), @@ -546,7 +546,7 @@ public void testServerReplacementWithoutBounce() throws IOException, Interrupted StreamingSlopPusherJob pusher = new StreamingSlopPusherJob(getVoldemortServer(0).getStoreRepository(), getVoldemortServer(0).getMetadataStore(), - new BannagePeriodFailureDetector(new FailureDetectorConfig().setNodes(cluster.getNodes()) + new BannagePeriodFailureDetector(new FailureDetectorConfig().setCluster(cluster) .setStoreVerifier(new ServerStoreVerifier(socketStoreFactory, metadataStore, configs[0]))), diff --git a/test/unit/voldemort/store/routed/HintedHandoffTest.java b/test/unit/voldemort/store/routed/HintedHandoffTest.java index 7f6d7d695e..a2c5ee197a 100644 --- a/test/unit/voldemort/store/routed/HintedHandoffTest.java +++ b/test/unit/voldemort/store/routed/HintedHandoffTest.java @@ -442,7 +442,7 @@ private void setFailureDetector(Map> s FailureDetectorConfig failureDetectorConfig = new FailureDetectorConfig(); failureDetectorConfig.setImplementationClassName(failureDetectorCls.getName()); failureDetectorConfig.setBannagePeriod(500); - failureDetectorConfig.setNodes(cluster.getNodes()); + failureDetectorConfig.setCluster(cluster); failureDetectorConfig.setStoreVerifier(MutableStoreVerifier.create(subStores)); failureDetector = FailureDetectorUtils.create(failureDetectorConfig, false); diff --git a/test/unit/voldemort/store/routed/ReadRepairerTest.java b/test/unit/voldemort/store/routed/ReadRepairerTest.java index c7d64e2e9a..61e41981c8 100644 --- a/test/unit/voldemort/store/routed/ReadRepairerTest.java +++ b/test/unit/voldemort/store/routed/ReadRepairerTest.java @@ -148,7 +148,7 @@ public void testMissingKeysAreAddedToNodeWhenDoingReadRepair() throws Exception FailureDetectorConfig failureDetectorConfig = new FailureDetectorConfig().setImplementationClassName(failureDetectorClass.getName()) .setBannagePeriod(1000) - .setNodes(cluster.getNodes()) + .setCluster(cluster) .setStoreVerifier(create(subStores)) .setTime(time); diff --git a/test/unit/voldemort/store/routed/RoutedStoreTest.java b/test/unit/voldemort/store/routed/RoutedStoreTest.java index 1b1ed1428e..a7fbcf22b6 100644 --- a/test/unit/voldemort/store/routed/RoutedStoreTest.java +++ b/test/unit/voldemort/store/routed/RoutedStoreTest.java @@ -1481,7 +1481,7 @@ private void setFailureDetector(Map> s FailureDetectorConfig failureDetectorConfig = new FailureDetectorConfig().setImplementationClassName(failureDetectorClass.getName()) .setBannagePeriod(BANNAGE_PERIOD) - .setNodes(cluster.getNodes()) + .setCluster(cluster) .setStoreVerifier(create(subStores)); failureDetector = create(failureDetectorConfig, false); } diff --git a/test/unit/voldemort/store/routed/action/AbstractActionTest.java b/test/unit/voldemort/store/routed/action/AbstractActionTest.java index 68e408f7e5..269b50366f 100644 --- a/test/unit/voldemort/store/routed/action/AbstractActionTest.java +++ b/test/unit/voldemort/store/routed/action/AbstractActionTest.java @@ -41,9 +41,9 @@ public class AbstractActionTest { @Before public void setUp() throws Exception { cluster = VoldemortTestConstants.getThreeNodeCluster(); - failureDetector = new BannagePeriodFailureDetector(new FailureDetectorConfig().setNodes(cluster.getNodes())); + failureDetector = new BannagePeriodFailureDetector(new FailureDetectorConfig().setCluster(cluster)); clusterWithZones = VoldemortTestConstants.getFourNodeClusterWithZones(); - failureDetectorWithZones = new BannagePeriodFailureDetector(new FailureDetectorConfig().setNodes(clusterWithZones.getNodes())); + failureDetectorWithZones = new BannagePeriodFailureDetector(new FailureDetectorConfig().setCluster(clusterWithZones)); storeDef = new StoreDefinitionsMapper().readStoreList(new StringReader(VoldemortTestConstants.getSingleStoreWithZonesXml())) .get(0); } diff --git a/test/unit/voldemort/store/routed/action/ConfigureNodesLocalHostTest.java b/test/unit/voldemort/store/routed/action/ConfigureNodesLocalHostTest.java index e89d43e2b6..97ceff026b 100644 --- a/test/unit/voldemort/store/routed/action/ConfigureNodesLocalHostTest.java +++ b/test/unit/voldemort/store/routed/action/ConfigureNodesLocalHostTest.java @@ -57,7 +57,7 @@ private Node node(int id, String hostName, int... tags) { public void testConfigureNodesLocalHost() throws Exception { List nodes = getTestNodes(); Cluster cluster = new Cluster("test-route-all-local-pref-cluster", nodes); - FailureDetector failureDetector = new ThresholdFailureDetector(new FailureDetectorConfig().setNodes(nodes)); + FailureDetector failureDetector = new ThresholdFailureDetector(new FailureDetectorConfig().setCluster(cluster)); RoutingStrategy routingStrategy = new RouteToAllLocalPrefStrategy(cluster.getNodes()); BasicPipelineData pipelineData = new BasicPipelineData(); ConfigureNodesLocalHost> action = new ConfigureNodesLocalHost>(pipelineData, From 5cb124957686a4e84216b4f92a14b8dcde0c4199 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Tue, 11 Sep 2012 10:36:02 -0700 Subject: [PATCH 127/209] Added a instantInit flag to LazyClientStore, removed DefaultSocketStoreClientTest, returning DefaultStoreClient for Http protocol --- .../client/AbstractStoreClientFactory.java | 3 + .../voldemort/client/LazyStoreClient.java | 32 ++++-- .../client/DefaultSocketStoreClientTest.java | 104 ------------------ .../voldemort/client/LazyStoreClientTest.java | 19 ++-- 4 files changed, 35 insertions(+), 123 deletions(-) delete mode 100644 test/unit/voldemort/client/DefaultSocketStoreClientTest.java diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index 5fe98ec046..2ca94503a0 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -157,6 +157,9 @@ public StoreClient getStoreClient(String storeName, StoreClient client = null; if(this.config.isDefaultClientEnabled()) { client = new DefaultStoreClient(storeName, resolver, this, 3); + } else if(this.bootstrapUrls.length > 0 + && this.bootstrapUrls[0].getScheme().equals(HttpStoreClientFactory.URL_SCHEME)) { + client = new DefaultStoreClient(storeName, resolver, this, 3); } else { SchedulerService service = new SchedulerService(config.getAsyncJobThreadPoolSize(), diff --git a/src/java/voldemort/client/LazyStoreClient.java b/src/java/voldemort/client/LazyStoreClient.java index d6f72eb722..e7524d2d29 100644 --- a/src/java/voldemort/client/LazyStoreClient.java +++ b/src/java/voldemort/client/LazyStoreClient.java @@ -41,19 +41,29 @@ public class LazyStoreClient implements StoreClient { private StoreClient storeClient; public LazyStoreClient(Callable> storeClientThunk) { + this(storeClientThunk, true); + } + + /** + * A Hybrid store client which tries to do immediate bootstrap. In case of + * an exception, we fallback to the lazy way of doing initialization. + * + * @param storeClientThunk The callback invoked for doing the actual + * bootstrap + * @param instantInit A boolean flag when set indicates that we should try + * to immediately bootstrap + */ + public LazyStoreClient(Callable> storeClientThunk, boolean instantInit) { this.storeClientThunk = storeClientThunk; - /* - * Although it says Lazy Store Client, we try to bootstrap during - * initialization if we can. If the server isn't up at this time, it - * will be done lazily during the next client API call. - */ - try { - storeClient = initStoreClient(); - } catch(Exception e) { - storeClient = null; - e.printStackTrace(); - logger.info("Could not bootstrap right away. Trying on the next call ... "); + if(instantInit) { + try { + storeClient = initStoreClient(); + } catch(Exception e) { + storeClient = null; + e.printStackTrace(); + logger.info("Could not bootstrap right away. Trying on the next call ... "); + } } } diff --git a/test/unit/voldemort/client/DefaultSocketStoreClientTest.java b/test/unit/voldemort/client/DefaultSocketStoreClientTest.java deleted file mode 100644 index d5cce3cac7..0000000000 --- a/test/unit/voldemort/client/DefaultSocketStoreClientTest.java +++ /dev/null @@ -1,104 +0,0 @@ -package voldemort.client; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.fail; - -import java.io.File; -import java.util.List; -import java.util.Properties; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.junit.Before; -import org.junit.Test; - -import voldemort.ServerTestUtils; -import voldemort.TestUtils; -import voldemort.client.protocol.admin.AdminClient; -import voldemort.cluster.Cluster; -import voldemort.server.VoldemortServer; -import voldemort.store.StoreDefinition; -import voldemort.store.socket.SocketStoreFactory; -import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; -import voldemort.utils.Time; -import voldemort.versioning.ObsoleteVersionException; -import voldemort.versioning.VectorClock; -import voldemort.versioning.Versioned; -import voldemort.xml.StoreDefinitionsMapper; - -public class DefaultSocketStoreClientTest { - - private static String testStoreName = "test-replication-memory"; - private static String storesXmlfile = "test/common/voldemort/config/stores.xml"; - private SocketStoreFactory socketStoreFactory = new ClientRequestExecutorPool(2, - 10000, - 100000, - 32 * 1024); - private static AtomicBoolean running = new AtomicBoolean(true); - private List storeDefs; - private VoldemortServer[] servers; - private Cluster cluster; - private AdminClient adminClient; - - public static String socketUrl = "tcp://localhost:6667"; - protected StoreClient client; - protected int nodeId; - protected Time time; - - @Before - public void setUp() throws Exception { - cluster = ServerTestUtils.getLocalCluster(2, new int[][] { { 0, 1, 2, 3 }, { 4, 5, 6, 7 } }); - - servers = new VoldemortServer[2]; - storeDefs = new StoreDefinitionsMapper().readStoreList(new File(storesXmlfile)); - - servers[0] = ServerTestUtils.startVoldemortServer(socketStoreFactory, - ServerTestUtils.createServerConfig(true, - 0, - TestUtils.createTempDir() - .getAbsolutePath(), - null, - storesXmlfile, - new Properties()), - cluster); - servers[1] = ServerTestUtils.startVoldemortServer(socketStoreFactory, - ServerTestUtils.createServerConfig(true, - 1, - TestUtils.createTempDir() - .getAbsolutePath(), - null, - storesXmlfile, - new Properties()), - cluster); - - adminClient = ServerTestUtils.getAdminClient(cluster); - } - - @Test - public void test() { - client.put("k", Versioned.value("v")); - Versioned v = client.get("k"); - assertEquals("GET should return the version set by PUT.", "v", v.getValue()); - VectorClock expected = new VectorClock(); - expected.incrementVersion(nodeId, time.getMilliseconds()); - assertEquals("The version should be incremented after a put.", expected, v.getVersion()); - try { - client.put("k", Versioned.value("v")); - fail("Put of obsolete version should throw exception."); - } catch(ObsoleteVersionException e) { - // this is good - } - // PUT of a concurrent version should succeed - client.put("k", - new Versioned("v2", - new VectorClock().incremented(nodeId + 1, - time.getMilliseconds()))); - assertEquals("GET should return the new value set by PUT.", "v2", client.getValue("k")); - assertEquals("GET should return the new version set by PUT.", - expected.incremented(nodeId + 1, time.getMilliseconds()), - client.get("k").getVersion()); - client.delete("k"); - assertNull("After a successful delete(k), get(k) should return null.", client.get("k")); - } - -} diff --git a/test/unit/voldemort/client/LazyStoreClientTest.java b/test/unit/voldemort/client/LazyStoreClientTest.java index feea5481a8..2baec6be10 100644 --- a/test/unit/voldemort/client/LazyStoreClientTest.java +++ b/test/unit/voldemort/client/LazyStoreClientTest.java @@ -1,12 +1,12 @@ /* * Copyright 2008-2011 LinkedIn, Inc - * + * * Licensed under the Apache License, Version 2.0 (the "License"); you may not * use this file except in compliance with the License. You may obtain a copy of * the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the @@ -16,16 +16,19 @@ package voldemort.client; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import java.util.concurrent.Callable; + import org.junit.Before; import org.junit.Test; + import voldemort.serialization.Serializer; import voldemort.serialization.StringSerializer; import voldemort.utils.SystemTime; -import java.util.concurrent.Callable; - -import static org.mockito.Mockito.*; - /** */ public class LazyStoreClientTest extends DefaultStoreClientTest { @@ -69,6 +72,6 @@ private LazyStoreClient newLazyStoreClient(final StoreClientFact public StoreClient call() throws Exception { return factory.getStoreClient("test"); } - }); + }, false); } } From 07be42763f6e6783b959814d87fa62ca77f65c81 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Wed, 12 Sep 2012 09:04:13 -0700 Subject: [PATCH 128/209] Bug fix: initializing system store client in AdminClient during every operation to account for cluster.xml changes --- .../client/protocol/admin/AdminClient.java | 44 ++++++++++++++----- 1 file changed, 34 insertions(+), 10 deletions(-) diff --git a/src/java/voldemort/client/protocol/admin/AdminClient.java b/src/java/voldemort/client/protocol/admin/AdminClient.java index e17193a0a5..f2a6915e46 100644 --- a/src/java/voldemort/client/protocol/admin/AdminClient.java +++ b/src/java/voldemort/client/protocol/admin/AdminClient.java @@ -138,6 +138,8 @@ public class AdminClient { private Cluster currentCluster; private SystemStore sysStoreVersion = null; + private String[] cachedBootstrapURLs = null; + private int cachedZoneID = -1; /** * Create an instance of AdminClient given a URL of a node in the cluster. @@ -160,7 +162,7 @@ public AdminClient(String bootstrapURL, AdminClientConfig adminClientConfig) { this.networkClassLoader = new NetworkClassLoader(Thread.currentThread() .getContextClassLoader()); this.adminClientConfig = adminClientConfig; - initSystemStoreClient(bootstrapURL, DEFAULT_ZONE_ID); + cacheSystemStoreParams(bootstrapURL, DEFAULT_ZONE_ID); } /** @@ -187,7 +189,7 @@ public AdminClient(Cluster cluster, AdminClientConfig adminClientConfig) { Node node = cluster.getNodeById(0); String bootstrapURL = "tcp://" + node.getHost() + ":" + node.getSocketPort(); - initSystemStoreClient(bootstrapURL, DEFAULT_ZONE_ID); + cacheSystemStoreParams(bootstrapURL, DEFAULT_ZONE_ID); } /** @@ -207,18 +209,38 @@ public AdminClient(Cluster cluster, AdminClientConfig adminClientConfig) { */ public AdminClient(String bootstrapURL, AdminClientConfig adminClientConfig, int zoneID) { this(bootstrapURL, adminClientConfig); - initSystemStoreClient(bootstrapURL, zoneID); + cacheSystemStoreParams(bootstrapURL, zoneID); } - private void initSystemStoreClient(String bootstrapURL, int zoneID) { + /** + * Cache the paramater values for the internal system store client. These + * cached values are used every time the system store client needs to be + * initialized (useful when the cluster.xml changes). + * + * @param bootstrapURL The URL to bootstrap from + * @param zoneID Indicates the primary zone of the sytem store client + */ + private void cacheSystemStoreParams(String bootstrapURL, int zoneID) { String[] bootstrapUrls = new String[1]; bootstrapUrls[0] = bootstrapURL; - try { - this.sysStoreVersion = new SystemStore(SystemStoreConstants.SystemStoreName.voldsys$_metadata_version_persistence.name(), - bootstrapUrls, - zoneID); - } catch(Exception e) { - logger.debug("Error while creating a system store client for metadata version store."); + this.cachedBootstrapURLs = bootstrapUrls; + this.cachedZoneID = zoneID; + } + + /** + * Create a system store client based on the cached bootstrap URLs and Zone + * ID + */ + public void initSystemStoreClient() { + if(this.cachedBootstrapURLs != null && this.cachedZoneID >= 0) { + try { + this.sysStoreVersion = new SystemStore(SystemStoreConstants.SystemStoreName.voldsys$_metadata_version_persistence.name(), + this.cachedBootstrapURLs, + this.cachedZoneID); + } catch(Exception e) { + logger.debug("Error while creating a system store client for metadata version store."); + } + } } @@ -229,6 +251,7 @@ private void initSystemStoreClient(String bootstrapURL, int zoneID) { * incremented */ public void updateMetadataversion(String versionKey) { + initSystemStoreClient(); Properties props = MetadataVersionStoreUtils.getProperties(this.sysStoreVersion); if(props != null && props.getProperty(versionKey) != null) { logger.debug("Version obtained = " + props.getProperty(versionKey)); @@ -251,6 +274,7 @@ public void updateMetadataversion(String versionKey) { * nodes in the cluster */ public void setMetadataversion(Properties newProperties) { + initSystemStoreClient(); MetadataVersionStoreUtils.setProperties(this.sysStoreVersion, newProperties); } From c54100d5f18c28caf8e5e753a448bd31a128292b Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Wed, 12 Sep 2012 10:31:05 -0700 Subject: [PATCH 129/209] Updating release version to 0.96 in build.properties --- build.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.properties b/build.properties index f363ea1bff..72a2407ac9 100644 --- a/build.properties +++ b/build.properties @@ -37,4 +37,4 @@ tomcat.manager.password=tomcat tomcat.context=/voldemort ## Release -curr.release=0.90.1 +curr.release=0.96 From 6b9260f5d1ec57812a925d9ab90361a29cc275f2 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Thu, 13 Sep 2012 11:03:42 -0700 Subject: [PATCH 130/209] Modified behavior of getall to comply javadoc when key does not exist in store --- .../action/PerformParallelGetAllRequests.java | 16 ++++---- .../action/PerformSerialGetAllRequests.java | 10 +++-- test/unit/voldemort/server/EndToEndTest.java | 4 +- .../voldemort/store/AbstractStoreTest.java | 16 +------- .../store/routed/GetallNodeReachTest.java | 37 ++++++------------- .../store/routed/RoutedStoreTest.java | 8 ++-- 6 files changed, 31 insertions(+), 60 deletions(-) diff --git a/src/java/voldemort/store/routed/action/PerformParallelGetAllRequests.java b/src/java/voldemort/store/routed/action/PerformParallelGetAllRequests.java index 357f30c97f..e44cfdc9d3 100644 --- a/src/java/voldemort/store/routed/action/PerformParallelGetAllRequests.java +++ b/src/java/voldemort/store/routed/action/PerformParallelGetAllRequests.java @@ -16,7 +16,6 @@ package voldemort.store.routed.action; -import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; import java.util.List; @@ -138,19 +137,18 @@ public void requestComplete(Object result, long requestTime) { successCount.increment(); List> retrieved = values.get(key); - if(retrieved == null) { - retrieved = new ArrayList>(); - } /* * retrieved can be null if there are no values for the key * provided */ - List> existing = pipelineData.getResult().get(key); + if(retrieved != null) { + List> existing = pipelineData.getResult().get(key); - if(existing == null) - pipelineData.getResult().put(key, Lists.newArrayList(retrieved)); - else - existing.addAll(retrieved); + if(existing == null) + pipelineData.getResult().put(key, Lists.newArrayList(retrieved)); + else + existing.addAll(retrieved); + } HashSet zoneResponses = null; if(pipelineData.getKeyToZoneResponse().containsKey(key)) { diff --git a/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java b/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java index 0d8b551d4b..01c52d06d2 100644 --- a/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java +++ b/src/java/voldemort/store/routed/action/PerformSerialGetAllRequests.java @@ -121,10 +121,12 @@ public void execute(Pipeline pipeline) { else values = store.get(key, transforms.get(key)); - if(result.get(key) == null) - result.put(key, Lists.newArrayList(values)); - else - result.get(key).addAll(values); + if(values.size() != 0) { + if(result.get(key) == null) + result.put(key, Lists.newArrayList(values)); + else + result.get(key).addAll(values); + } Map>> map = new HashMap>>(); map.put(key, values); diff --git a/test/unit/voldemort/server/EndToEndTest.java b/test/unit/voldemort/server/EndToEndTest.java index f8e9c798c8..7616e2ae61 100644 --- a/test/unit/voldemort/server/EndToEndTest.java +++ b/test/unit/voldemort/server/EndToEndTest.java @@ -1,8 +1,8 @@ package voldemort.server; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; import java.io.IOException; import java.util.ArrayList; @@ -120,7 +120,7 @@ public void testSanity() { assertEquals("getAll works as expected", "Moscow", capitals.get("Russia").getValue()); assertEquals("getAll works as expected", "Kiev", capitals.get("Ukraine").getValue()); - assertTrue("getAll works as expected", capitals.get("Japan") == null); + assertFalse("getAll works as expected", capitals.containsKey("Japan")); storeClient.delete("Ukraine"); assertNull("delete works as expected", storeClient.get("Ukraine")); diff --git a/test/unit/voldemort/store/AbstractStoreTest.java b/test/unit/voldemort/store/AbstractStoreTest.java index e2adf72f01..c157d1d1be 100644 --- a/test/unit/voldemort/store/AbstractStoreTest.java +++ b/test/unit/voldemort/store/AbstractStoreTest.java @@ -312,21 +312,7 @@ public void testGetAll() throws Exception { public void testGetAllWithAbsentKeys() throws Exception { Store store = getStore(); Map>> result = store.getAll(getKeys(3), null); - boolean resultZero = (result.size() == 0); - boolean resultEmpty = true; - if(!resultZero) { - if(result.get(result.keySet().toArray()[0]).size() != 0) { - resultEmpty = false; - } - if(result.get(result.keySet().toArray()[1]).size() != 0) { - resultEmpty = false; - } - if(result.get(result.keySet().toArray()[2]).size() != 0) { - resultEmpty = false; - } - } - assertTrue(resultZero || resultEmpty); - + assertEquals(0, result.size()); } @Test diff --git a/test/unit/voldemort/store/routed/GetallNodeReachTest.java b/test/unit/voldemort/store/routed/GetallNodeReachTest.java index 869c4a0ed1..39d28f43a0 100644 --- a/test/unit/voldemort/store/routed/GetallNodeReachTest.java +++ b/test/unit/voldemort/store/routed/GetallNodeReachTest.java @@ -2,7 +2,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; import static voldemort.VoldemortTestConstants.getEightNodeClusterWithZones; import static voldemort.VoldemortTestConstants.getFourNodeClusterWithZones; @@ -96,16 +95,15 @@ public void testGetallTouchOneZone() throws Exception { assertEquals(2, store.getAll(keys011, null) .get(TestUtils.toByteArray("k011_zone0_only")) .size()); - assertEquals(0, store.getAll(keys100, null) - .get(TestUtils.toByteArray("k100_zone1_only")) - .size()); + assertFalse(store.getAll(keys100, null) + .containsKey(TestUtils.toByteArray("k100_zone1_only"))); /* test multiple keys getall */ List keys = new ArrayList(); keys.add(TestUtils.toByteArray("k011_zone0_only")); keys.add(TestUtils.toByteArray("k100_zone1_only")); Map>> result = store.getAll(keys, null); assertEquals(2, result.get(TestUtils.toByteArray("k011_zone0_only")).size()); - assertEquals(0, result.get(TestUtils.toByteArray("k100_zone1_only")).size()); + assertFalse(result.containsKey(TestUtils.toByteArray("k100_zone1_only"))); } @Test @@ -161,13 +159,9 @@ public void testGetall_211() throws Exception { keys.add(TestUtils.toByteArray("k111")); Map>> result = store.getAll(keys, null); assertFalse(result.containsKey(TestUtils.toByteArray("not_included"))); - assertEquals(0, result.get(TestUtils.toByteArray("k000")).size()); - assertEquals(1, result.get(TestUtils.toByteArray("k001")).size()); - assertEquals(0, result.get(TestUtils.toByteArray("k010")).size()); + assertFalse(result.containsKey(TestUtils.toByteArray("k000"))); assertEquals(1, result.get(TestUtils.toByteArray("k011")).size()); - assertEquals(0, result.get(TestUtils.toByteArray("k100")).size()); - assertEquals(1, result.get(TestUtils.toByteArray("k101")).size()); - assertEquals(0, result.get(TestUtils.toByteArray("k110")).size()); + assertFalse(result.containsKey(TestUtils.toByteArray("k100"))); assertEquals(1, result.get(TestUtils.toByteArray("k111")).size()); } @@ -227,21 +221,12 @@ public void testGetall_211_zoneCountRead_1() throws Exception { Map>> result = store.getAll(keys, null); assertFalse(result.containsKey(TestUtils.toByteArray("not_included"))); /* client will first try all the nodes in local zone */ - assertEquals(0, result.get(TestUtils.toByteArray("k000")).size()); - assertEquals(1, result.get(TestUtils.toByteArray("k001")).size()); - // don't know which one of node 0 or 1 comes second on preferece list - // for key - // k*10 or k*01 - // therefore it can be 1 or 0 beside existence on node 2 - int size = -1; - size = result.get(TestUtils.toByteArray("k010")).size(); - assertTrue(size == 1 || size == 0); + assertFalse(result.containsKey(TestUtils.toByteArray("k000"))); + assertEquals(1, result.get(TestUtils.toByteArray("k011")).size()); + assertFalse(result.containsKey(TestUtils.toByteArray("not_included"))); + assertFalse(result.containsKey(TestUtils.toByteArray("k000"))); assertEquals(1, result.get(TestUtils.toByteArray("k011")).size()); assertEquals(1, result.get(TestUtils.toByteArray("k100")).size()); - size = result.get(TestUtils.toByteArray("k101")).size(); - assertTrue(size == 1 || size == 2); - size = result.get(TestUtils.toByteArray("k110")).size(); - assertTrue(size == 1 || size == 2); assertEquals(2, result.get(TestUtils.toByteArray("k111")).size()); } @@ -297,9 +282,9 @@ public void testGetall_322() throws Exception { keys.add(TestUtils.toByteArray("k1111_1111")); Map>> result = store.getAll(keys, null); assertFalse(result.containsKey(TestUtils.toByteArray("not_included"))); - assertEquals(0, result.get(TestUtils.toByteArray("k0000_0000")).size()); + assertFalse(result.containsKey(TestUtils.toByteArray("k0000_0000"))); assertEquals(2, result.get(TestUtils.toByteArray("k0000_1111")).size()); - assertEquals(0, result.get(TestUtils.toByteArray("k1111_0000")).size()); + assertFalse(result.containsKey(TestUtils.toByteArray("k1111_0000"))); assertEquals(2, result.get(TestUtils.toByteArray("k1111_1111")).size()); } } diff --git a/test/unit/voldemort/store/routed/RoutedStoreTest.java b/test/unit/voldemort/store/routed/RoutedStoreTest.java index 1b1ed1428e..975420379f 100644 --- a/test/unit/voldemort/store/routed/RoutedStoreTest.java +++ b/test/unit/voldemort/store/routed/RoutedStoreTest.java @@ -457,8 +457,8 @@ public void testZoneRouting() throws Exception { new ByteArray("test2".getBytes())); Map>> values = s1.getAll(keys, null); - List> results = values.get(new ByteArray("test".getBytes())); - assertEquals("\'test\' did not get deleted.", 0, results.size()); + assertFalse("'test' did not get deleted.", + values.containsKey(new ByteArray("test".getBytes()))); ByteUtils.compare(values.get(new ByteArray("test2".getBytes())).get(0).getValue(), new byte[] { 1 }); @@ -517,8 +517,8 @@ public void testZoneRouting() throws Exception { } values = s2.getAll(keys, null); - results = values.get(new ByteArray("test".getBytes())); - assertEquals("\'test\' did not get deleted.", 0, results.size()); + assertFalse("'test' did not get deleted.", + values.containsKey(new ByteArray("test".getBytes()))); ByteUtils.compare(values.get(new ByteArray("test2".getBytes())).get(0).getValue(), new byte[] { 1 }); From 91ab1d80e39ff549427506ceac4f7f8db4bf9f98 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Thu, 13 Sep 2012 18:00:37 -0700 Subject: [PATCH 131/209] Bug fix in AsyncMetadataVersionManager. Client config parameters added for System store. Changed AdminClient to use a Timestamp instead of a counter for metadata version --- src/java/voldemort/VoldemortAdminTool.java | 4 + .../client/AbstractStoreClientFactory.java | 6 +- src/java/voldemort/client/ClientConfig.java | 104 +++++++++++++++++- src/java/voldemort/client/ClientInfo.java | 19 ++++ src/java/voldemort/client/SystemStore.java | 89 +++++++++++++-- .../client/SystemStoreClientFactory.java | 14 --- .../client/SystemStoreRepository.java | 3 +- src/java/voldemort/client/ZenStoreClient.java | 52 +++++---- .../client/protocol/admin/AdminClient.java | 9 +- .../AsyncMetadataVersionManager.java | 61 +++++----- .../scheduler/ClientRegistryRefresher.java | 38 ++++--- .../BannagePeriodFailureDetector.java | 1 + .../FileBackedCachingStorageEngine.java | 9 ++ .../store/system/SystemStoreTest.java | 1 + 14 files changed, 309 insertions(+), 101 deletions(-) delete mode 100644 src/java/voldemort/client/SystemStoreClientFactory.java diff --git a/src/java/voldemort/VoldemortAdminTool.java b/src/java/voldemort/VoldemortAdminTool.java index a267858d60..00b9a8cb20 100644 --- a/src/java/voldemort/VoldemortAdminTool.java +++ b/src/java/voldemort/VoldemortAdminTool.java @@ -583,6 +583,10 @@ private static void synchronizeMetadataVersion(AdminClient adminClient, int base Properties props = new Properties(); try { props.load(new ByteArrayInputStream(valueObject.getBytes())); + if(props.size() == 0) { + System.err.println("The specified node does not have any versions metadata ! Exiting ..."); + System.exit(-1); + } adminClient.setMetadataversion(props); System.out.println("Metadata versions synchronized successfully."); } catch(IOException e) { diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index 2ca94503a0..0b21d7e4e4 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -193,10 +193,10 @@ public Store getRawStore(String storeName, String clusterXmlString, FailureDetector fd) { - if(logger.isDebugEnabled()) { - logger.debug("Client zone-id [" + clientZoneId - + "] Attempting to obtain metadata for store [" + storeName + "] "); + logger.info("Client zone-id [" + clientZoneId + + "] Attempting to obtain metadata for store [" + storeName + "] "); + if(logger.isDebugEnabled()) { for(URI uri: bootstrapUrls) { logger.debug("Client Bootstrap url [" + uri + "]"); } diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index 0091110301..59081b113d 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -89,6 +89,14 @@ public class ClientConfig { private volatile int clientRegistryRefreshInterval = 3600 * 12; private volatile int asyncJobThreadPoolSize = 2; + /* SystemStore client config */ + private volatile int sysMaxConnectionsPerNode = 2; + private volatile int sysRoutingTimeout = 5000; + private volatile int sysSocketTimeout = 5000; + private volatile int sysConnectionTimeout = 1500; + private volatile boolean sysEnableJmx = false; + private volatile boolean sysEnablePipelineRoutedStore = true; + public ClientConfig() {} /* Propery names for propery-based configuration */ @@ -130,9 +138,15 @@ public ClientConfig() {} public static final String MAX_BOOTSTRAP_RETRIES = "max_bootstrap_retries"; public static final String CLIENT_CONTEXT_NAME = "voldemort_client_context"; public static final String ASYNC_CHECK_METADATA_INTERVAL = "check_metadata_interval"; - private static final String USE_DEFAULT_CLIENT = "use_default_client"; + public static final String USE_DEFAULT_CLIENT = "use_default_client"; public static final String CLIENT_REGISTRY_REFRESH_INTERVAL = "client_registry_refresh_interval"; public static final String ASYNC_JOB_THREAD_POOL_SIZE = "async_job_thread_pool_size"; + public static final String SYS_MAX_CONNECTIONS_PER_NODE = "sys_max_connections_per_node"; + public static final String SYS_ROUTING_TIMEOUT_MS = "sys_routing_timeout_ms"; + public static final String SYS_CONNECTION_TIMEOUT_MS = "sys_connection_timeout_ms"; + public static final String SYS_SOCKET_TIMEOUT_MS = "sys_socket_timeout_ms"; + public static final String SYS_ENABLE_JMX = "sys_enable_jmx"; + public static final String SYS_ENABLE_PIPELINE_ROUTED_STORE = "sys_enable_pipeline_routed_store"; /** * Instantiate the client config using a properties file @@ -310,6 +324,94 @@ private void setProperties(Properties properties) { if(props.containsKey(ASYNC_JOB_THREAD_POOL_SIZE)) { this.setAsyncJobThreadPoolSize(props.getInt(ASYNC_JOB_THREAD_POOL_SIZE)); } + + /* Check for system store paramaters if any */ + if(props.containsKey(SYS_MAX_CONNECTIONS_PER_NODE)) { + this.setSysMaxConnectionsPerNode(props.getInt(SYS_MAX_CONNECTIONS_PER_NODE)); + } + + if(props.containsKey(SYS_ROUTING_TIMEOUT_MS)) { + this.setSysRoutingTimeout(props.getInt(SYS_ROUTING_TIMEOUT_MS)); + } + + if(props.containsKey(SYS_SOCKET_TIMEOUT_MS)) { + this.setSysSocketTimeout(props.getInt(SYS_SOCKET_TIMEOUT_MS)); + } + + if(props.containsKey(SYS_CONNECTION_TIMEOUT_MS)) { + this.setSysConnectionTimeout(props.getInt(SYS_CONNECTION_TIMEOUT_MS)); + } + + if(props.containsKey(SYS_ENABLE_JMX)) { + this.setSysEnableJmx(props.getBoolean(SYS_ENABLE_JMX)); + } + + if(props.containsKey(SYS_ENABLE_PIPELINE_ROUTED_STORE)) { + this.setSysEnablePipelineRoutedStore(props.getBoolean(SYS_ENABLE_PIPELINE_ROUTED_STORE)); + } + + } + + private ClientConfig setSysMaxConnectionsPerNode(int maxConnectionsPerNode) { + if(maxConnectionsPerNode <= 0) + throw new IllegalArgumentException("Value must be greater than zero."); + this.sysMaxConnectionsPerNode = maxConnectionsPerNode; + return this; + } + + public int getSysMaxConnectionsPerNode() { + return this.sysMaxConnectionsPerNode; + } + + private ClientConfig setSysRoutingTimeout(int sysRoutingTimeout) { + if(sysRoutingTimeout <= 0) + throw new IllegalArgumentException("Value must be greater than zero."); + this.sysRoutingTimeout = sysRoutingTimeout; + return this; + } + + public int getSysRoutingTimeout() { + return this.sysRoutingTimeout; + } + + private ClientConfig setSysSocketTimeout(int sysSocketTimeout) { + if(sysSocketTimeout <= 0) + throw new IllegalArgumentException("Value must be greater than zero."); + this.sysSocketTimeout = sysSocketTimeout; + return this; + } + + public int getSysSocketTimeout() { + return this.sysSocketTimeout; + } + + private ClientConfig setSysConnectionTimeout(int sysConnectionTimeout) { + if(sysConnectionTimeout <= 0) + throw new IllegalArgumentException("Value must be greater than zero."); + this.sysConnectionTimeout = sysConnectionTimeout; + return this; + } + + public int getSysConnectionTimeout() { + return this.sysConnectionTimeout; + } + + public boolean getSysEnableJmx() { + return this.sysEnableJmx; + } + + public ClientConfig setSysEnableJmx(boolean sysEnableJmx) { + this.sysEnableJmx = sysEnableJmx; + return this; + } + + public boolean getSysEnablePipelineRoutedStore() { + return this.sysEnablePipelineRoutedStore; + } + + public ClientConfig setSysEnablePipelineRoutedStore(boolean sysEnablePipelineRoutedStore) { + this.sysEnablePipelineRoutedStore = sysEnablePipelineRoutedStore; + return this; } public int getMaxConnectionsPerNode() { diff --git a/src/java/voldemort/client/ClientInfo.java b/src/java/voldemort/client/ClientInfo.java index d2dfcfb00f..0f28941176 100644 --- a/src/java/voldemort/client/ClientInfo.java +++ b/src/java/voldemort/client/ClientInfo.java @@ -48,6 +48,7 @@ public class ClientInfo implements Serializable { private long updateTime; private String releaseVersion; private ClientConfig config; + private long clusterMetadataVersion; public ClientInfo(String storeName, String clientContext, @@ -64,6 +65,7 @@ public ClientInfo(String storeName, this.updateTime = bootstrapTime; this.releaseVersion = version; this.config = config; + this.clusterMetadataVersion = 0; if(logger.isDebugEnabled()) { logger.debug(this.toString()); @@ -161,6 +163,10 @@ public synchronized ClientConfig getClientConfig() { return this.config; } + public synchronized void setClusterMetadataVersion(long newVersion) { + this.clusterMetadataVersion = newVersion; + } + /** * At the moment we're not checking if the Config objects are similar. TODO: * reevaluate in the future. @@ -195,6 +201,7 @@ public synchronized String toString() { builder.append("storeName=").append(storeName).append("\n"); builder.append("updateTime=").append(updateTime).append("\n"); builder.append("releaseVersion=").append(releaseVersion).append("\n"); + builder.append("clusterMetadataVersion=").append(clusterMetadataVersion).append("\n"); /** * Append the Client Config information. Right now we only track the @@ -222,6 +229,18 @@ public synchronized String toString() { builder.append("failuredetector_implementation=") .append(this.config.getFailureDetectorImplementation()) .append("\n"); + builder.append("failuredetector_threshold=") + .append(this.config.getFailureDetectorThreshold()) + .append("\n"); + builder.append("failuredetector_threshold_count_minimum=") + .append(this.config.getFailureDetectorThresholdCountMinimum()) + .append("\n"); + builder.append("failuredetector_threshold_interval=") + .append(this.config.getFailureDetectorThresholdInterval()) + .append("\n"); + builder.append("failuredetector_threshold_async_recovery_interval=") + .append(this.config.getFailureDetectorAsyncRecoveryInterval()) + .append("\n"); return builder.toString(); } diff --git a/src/java/voldemort/client/SystemStore.java b/src/java/voldemort/client/SystemStore.java index 814265b732..4303002a75 100644 --- a/src/java/voldemort/client/SystemStore.java +++ b/src/java/voldemort/client/SystemStore.java @@ -14,22 +14,91 @@ import voldemort.versioning.Version; import voldemort.versioning.Versioned; +/* + * A client interface for interacting with System stores (managed by the + * cluster). The naming convention is kept consistent with SocketStore (which is + * also a client interface). + */ public class SystemStore { private final Logger logger = Logger.getLogger(SystemStore.class); - private final SocketStoreClientFactory systemStoreFactory; + private final SocketStoreClientFactory socketStoreFactory; private final String storeName; private volatile Store sysStore; + /** + * Wrapper for the actual SystemStore constructor. Used when we dont have + * custom Cluster XML, failure detector or a base Voldemort Client config to + * be used with this system store client. + * + * @param storeName Name of the system store + * @param bootstrapUrls Bootstrap URLs used to connect to + * @param clientZoneID Primary zone ID for this system store client + * (determines routing strategy) + */ public SystemStore(String storeName, String[] bootstrapUrls, int clientZoneID) { - this(storeName, bootstrapUrls, clientZoneID, null, null); + this(storeName, bootstrapUrls, clientZoneID, null, null, new ClientConfig()); } + /** + * Wrapper for the actual SystemStore constructor. Used when we dont have + * custom Cluster XML or failure detector to be used with this system store + * client. + * + * @param storeName Name of the system store + * @param bootstrapUrls Bootstrap URLs used to connect to + * @param clientZoneID Primary zone ID for this system store client + * (determines routing strategy) + * @param baseConfig Base Voldemort Client config which specifies properties + * for this system store client + */ + public SystemStore(String storeName, + String[] bootstrapUrls, + int clientZoneID, + ClientConfig baseConfig) { + this(storeName, bootstrapUrls, clientZoneID, null, null, baseConfig); + } + + /** + * SystemStore Constructor wrapper for the actual constructor. Used when we + * dont want to specify a base Voldemort Client Config. + * + * @param storeName Name of the system store + * @param bootstrapUrls Bootstrap URLs used to connect to + * @param clientZoneID Primary zone ID for this system store client + * (determines routing strategy) + * @param clusterXml Custom ClusterXml to be used for this system store + * client + * @param fd Failure Detector to be used with this system store client + */ public SystemStore(String storeName, String[] bootstrapUrls, int clientZoneID, String clusterXml, FailureDetector fd) { + this(storeName, bootstrapUrls, clientZoneID, clusterXml, fd, new ClientConfig()); + } + + /** + * SystemStore Constructor that creates a system store client which can be + * used to interact with the system stores managed by the cluster + * + * @param storeName Name of the system store + * @param bootstrapUrls Bootstrap URLs used to connect to + * @param clientZoneID Primary zone ID for this system store client + * (determines routing strategy) + * @param clusterXml Custom ClusterXml to be used for this system store + * client + * @param fd Failure Detector to be used with this system store client + * @param baseConfig Base Voldemort Client config which specifies properties + * for this system store client + */ + public SystemStore(String storeName, + String[] bootstrapUrls, + int clientZoneID, + String clusterXml, + FailureDetector fd, + ClientConfig baseConfig) { String prefix = storeName.substring(0, SystemStoreConstants.NAME_PREFIX.length()); if(!SystemStoreConstants.NAME_PREFIX.equals(prefix)) throw new VoldemortException("Illegal system store : " + storeName); @@ -37,17 +106,17 @@ public SystemStore(String storeName, ClientConfig config = new ClientConfig(); config.setSelectors(1) .setBootstrapUrls(bootstrapUrls) - .setMaxConnectionsPerNode(2) - .setConnectionTimeout(1500, TimeUnit.MILLISECONDS) - .setSocketTimeout(5000, TimeUnit.MILLISECONDS) - .setRoutingTimeout(5000, TimeUnit.MILLISECONDS) - .setEnableJmx(false) - .setEnablePipelineRoutedStore(true) + .setMaxConnectionsPerNode(baseConfig.getSysMaxConnectionsPerNode()) + .setConnectionTimeout(baseConfig.getSysConnectionTimeout(), TimeUnit.MILLISECONDS) + .setSocketTimeout(baseConfig.getSysSocketTimeout(), TimeUnit.MILLISECONDS) + .setRoutingTimeout(baseConfig.getSysRoutingTimeout(), TimeUnit.MILLISECONDS) + .setEnableJmx(baseConfig.getSysEnableJmx()) + .setEnablePipelineRoutedStore(baseConfig.getSysEnablePipelineRoutedStore()) .setClientZoneId(clientZoneID); - this.systemStoreFactory = new SystemStoreClientFactory(config); + this.socketStoreFactory = new SocketStoreClientFactory(config); this.storeName = storeName; try { - this.sysStore = this.systemStoreFactory.getSystemStore(this.storeName, clusterXml, fd); + this.sysStore = this.socketStoreFactory.getSystemStore(this.storeName, clusterXml, fd); } catch(Exception e) { logger.debug("Error while creating a system store client for store : " + this.storeName); } diff --git a/src/java/voldemort/client/SystemStoreClientFactory.java b/src/java/voldemort/client/SystemStoreClientFactory.java deleted file mode 100644 index 2cbd832464..0000000000 --- a/src/java/voldemort/client/SystemStoreClientFactory.java +++ /dev/null @@ -1,14 +0,0 @@ -package voldemort.client; - -public class SystemStoreClientFactory extends SocketStoreClientFactory { - - public SystemStoreClientFactory(ClientConfig config) { - super(config); - } - - @Override - public int getNextJmxId() { - // for system store, we don't increment jmx id - return getCurrentJmxId(); - } -} diff --git a/src/java/voldemort/client/SystemStoreRepository.java b/src/java/voldemort/client/SystemStoreRepository.java index e8e19f5d41..38289a91b2 100644 --- a/src/java/voldemort/client/SystemStoreRepository.java +++ b/src/java/voldemort/client/SystemStoreRepository.java @@ -29,7 +29,8 @@ public void createSystemStores(ClientConfig config, String clusterXml, FailureDe config.getBootstrapUrls(), config.getClientZoneId(), clusterXml, - fd); + fd, + config); this.sysStoreMap.put(storeName.name(), sysStore); } } diff --git a/src/java/voldemort/client/ZenStoreClient.java b/src/java/voldemort/client/ZenStoreClient.java index afe82b4f04..6db6492cbf 100644 --- a/src/java/voldemort/client/ZenStoreClient.java +++ b/src/java/voldemort/client/ZenStoreClient.java @@ -65,7 +65,8 @@ public class ZenStoreClient extends DefaultStoreClient { private final SchedulerService scheduler; private ClientInfo clientInfo; private String clusterXml; - private AsyncMetadataVersionManager asyncCheckMetadata = null; + private AsyncMetadataVersionManager asyncMetadataManager = null; + private ClientRegistryRefresher clientRegistryRefresher = null; public ZenStoreClient(String storeName, InconsistencyResolver> resolver, @@ -111,23 +112,24 @@ public ZenStoreClient(String storeName, // Initialize the background thread for checking metadata version if(config != null) { - asyncCheckMetadata = scheduleMetadataChecker(clientId.toString(), - config.getAsyncMetadataRefreshInMs()); + asyncMetadataManager = scheduleAsyncMetadataVersionManager(clientId.toString(), + config.getAsyncMetadataRefreshInMs()); } - registerClient(clientId, config.getClientRegistryUpdateInSecs()); + clientRegistryRefresher = registerClient(clientId, config.getClientRegistryUpdateInSecs()); logger.info("Voldemort client created: " + clientId + "\n" + clientInfo); } - private void registerClient(String jobId, int interval) { + private ClientRegistryRefresher registerClient(String jobId, int interval) { + ClientRegistryRefresher refresher = null; if(this.sysRepository.getClientRegistryStore() != null) { try { Version version = this.sysRepository.getClientRegistryStore() .putSysStore(clientId, clientInfo.toString()); - ClientRegistryRefresher refresher = new ClientRegistryRefresher(this.sysRepository, - clientId, - clientInfo, - version); + refresher = new ClientRegistryRefresher(this.sysRepository, + clientId, + clientInfo, + version); GregorianCalendar cal = new GregorianCalendar(); cal.add(Calendar.SECOND, interval); @@ -148,10 +150,12 @@ private void registerClient(String jobId, int interval) { logger.warn(SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name() + "not found. Unable to registry with voldemort cluster."); } + return refresher; } - private AsyncMetadataVersionManager scheduleMetadataChecker(String jobId, long interval) { - AsyncMetadataVersionManager asyncCheckMetadata = null; + private AsyncMetadataVersionManager scheduleAsyncMetadataVersionManager(String jobId, + long interval) { + AsyncMetadataVersionManager asyncMetadataManager = null; SystemStore versionStore = this.sysRepository.getVersionStore(); if(versionStore == null) { logger.warn("Metadata version system store not found. Cannot run Metadata version check thread."); @@ -166,14 +170,14 @@ public Void call() throws Exception { } }; - asyncCheckMetadata = new AsyncMetadataVersionManager(this.sysRepository, - rebootstrapCallback); + asyncMetadataManager = new AsyncMetadataVersionManager(this.sysRepository, + rebootstrapCallback); // schedule the job to run every 'checkInterval' period, starting // now if(scheduler != null) { - scheduler.schedule(jobId + asyncCheckMetadata.getClass().getName(), - asyncCheckMetadata, + scheduler.schedule(jobId + asyncMetadataManager.getClass().getName(), + asyncMetadataManager, new Date(), interval); logger.info("Metadata version check thread started. Frequency = Every " + interval @@ -182,7 +186,7 @@ public Void call() throws Exception { logger.warn("Metadata version check thread won't start because the scheduler service is not configured."); } } - return asyncCheckMetadata; + return asyncMetadataManager; } @Override @@ -209,16 +213,26 @@ public void bootStrap() { * Metadata exception). This will prevent another bootstrap via the * Async metadata checker */ - if(asyncCheckMetadata != null) { - asyncCheckMetadata.updateMetadataVersions(); + if(asyncMetadataManager != null) { + asyncMetadataManager.updateMetadataVersions(); } /* * Every time we bootstrap, update the bootstrap time */ if(this.clientInfo != null) { + if(this.asyncMetadataManager != null) { + this.clientInfo.setClusterMetadataVersion(this.asyncMetadataManager.getClusterMetadataVersion()); + } this.clientInfo.setBootstrapTime(System.currentTimeMillis()); } + + if(this.clientRegistryRefresher == null) { + logger.error("Unable to publish the client registry after bootstrap. Client Registry Refresher is NULL."); + } else { + logger.info("Publishing client registry after Bootstrap."); + this.clientRegistryRefresher.publishRegistry(); + } } public String getClientId() { @@ -228,7 +242,7 @@ public String getClientId() { @JmxGetter(name = "getClusterMetadataVersion") public String getClusterMetadataVersion() { String result = "Current Cluster Metadata Version : " - + this.asyncCheckMetadata.getClusterMetadataVersion(); + + this.asyncMetadataManager.getClusterMetadataVersion(); return result; } diff --git a/src/java/voldemort/client/protocol/admin/AdminClient.java b/src/java/voldemort/client/protocol/admin/AdminClient.java index f2a6915e46..a2dcfe2d80 100644 --- a/src/java/voldemort/client/protocol/admin/AdminClient.java +++ b/src/java/voldemort/client/protocol/admin/AdminClient.java @@ -245,7 +245,8 @@ public void initSystemStoreClient() { } /** - * Increment the metadata version for the given key (cluster or store) + * Update the metadata version for the given key (cluster or store). The new + * value set is the current timestamp. * * @param versionKey The metadata key for which Version should be * incremented @@ -253,17 +254,17 @@ public void initSystemStoreClient() { public void updateMetadataversion(String versionKey) { initSystemStoreClient(); Properties props = MetadataVersionStoreUtils.getProperties(this.sysStoreVersion); + long newValue = 0; if(props != null && props.getProperty(versionKey) != null) { logger.debug("Version obtained = " + props.getProperty(versionKey)); - long newValue = Long.parseLong(props.getProperty(versionKey)) + 1; - props.setProperty(versionKey, Long.toString(newValue)); + newValue = System.currentTimeMillis(); } else { logger.debug("Current version is null. Assuming version 0."); if(props == null) { props = new Properties(); } - props.setProperty(versionKey, "0"); } + props.setProperty(versionKey, Long.toString(newValue)); MetadataVersionStoreUtils.setProperties(this.sysStoreVersion, props); } diff --git a/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java b/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java index fe55c4c8d4..f90862dc12 100644 --- a/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java +++ b/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java @@ -18,6 +18,9 @@ * init if the initial version turns out to be null, it means that no change has * been done to that store since it was created. In this case, we assume version * '0'. + * + * At the moment, this only tracks the cluster.xml changes. TODO: Extend this to + * track other stuff (like stores.xml) */ public class AsyncMetadataVersionManager implements Runnable { @@ -84,7 +87,11 @@ public Long fetchNewVersion(String versionKey, Long curVersion, Properties versi logger.debug("MetadataVersion check => Obtained " + versionKey + " version : " + newVersion); - if(!newVersion.equals(curVersion)) { + /* + * Check if the new version is greater than the current one. We + * should not re-bootstrap on a stale version. + */ + if(newVersion > curVersion) { return newVersion; } } else { @@ -103,8 +110,10 @@ public Long fetchNewVersion(String versionKey, Long curVersion, Properties versi public void run() { try { - // Get the properties object from the system store (containing - // versions) + /* + * Get the properties object from the system store (containing + * versions) + */ Properties versionProps = MetadataVersionStoreUtils.getProperties(this.sysRepository.getMetadataVersionStore()); Long newClusterVersion = fetchNewVersion(CLUSTER_VERSION_KEY, currentClusterVersion, @@ -112,39 +121,20 @@ public void run() { // If nothing has been updated, continue if(newClusterVersion != null) { - - logger.info("Metadata version mismatch detected."); - - // Determine a random delta delay between 0 to DELTA_MAX to - // sleep - int delta = randomGenerator.nextInt(DELTA_MAX); - + logger.info("Metadata version mismatch detected. Re-bootstrapping !!!"); try { - logger.info("Sleeping for delta : " + delta + " (ms) before re-bootstrapping."); - Thread.sleep(delta); - } catch(InterruptedException e) { - // do nothing, continue. - } - - /* - * Do another check for mismatch here since the versions might - * have been updated while we were sleeping - */ - if(!newClusterVersion.equals(currentClusterVersion)) { - - try { - logger.info("Updating cluster version"); - currentClusterVersion = newClusterVersion; - - this.storeClientThunk.call(); - } catch(Exception e) { - if(logger.isDebugEnabled()) { - e.printStackTrace(); - logger.debug(e.getMessage()); - } + logger.info("Updating cluster version"); + currentClusterVersion = newClusterVersion; + + this.storeClientThunk.call(); + } catch(Exception e) { + if(logger.isDebugEnabled()) { + e.printStackTrace(); + logger.debug(e.getMessage()); } } } + } catch(Exception e) { logger.debug("Could not retrieve metadata versions from the server."); } @@ -155,9 +145,12 @@ public Long getClusterMetadataVersion() { return this.currentClusterVersion; } - // Fetch the latest versions for cluster and store + // Fetch the latest versions for cluster metadata public void updateMetadataVersions() { Properties versionProps = MetadataVersionStoreUtils.getProperties(this.sysRepository.getMetadataVersionStore()); - this.currentClusterVersion = fetchNewVersion(CLUSTER_VERSION_KEY, null, versionProps); + Long newVersion = fetchNewVersion(CLUSTER_VERSION_KEY, null, versionProps); + if(newVersion != null) { + this.currentClusterVersion = newVersion; + } } } diff --git a/src/java/voldemort/client/scheduler/ClientRegistryRefresher.java b/src/java/voldemort/client/scheduler/ClientRegistryRefresher.java index e92ad00e56..cbffd3bfa9 100644 --- a/src/java/voldemort/client/scheduler/ClientRegistryRefresher.java +++ b/src/java/voldemort/client/scheduler/ClientRegistryRefresher.java @@ -51,22 +51,26 @@ public ClientRegistryRefresher(SystemStoreRepository sysRepository, logger.info("Initial version obtained from client registry: " + version); } - public void run() { - - if(hadConflict) { - // if we previously had a conflict during update, we will try to get - // a newer version before update this time. This case shall not - // happen under regular circumstances. But it is just avoid update - // keeping failing when strange situations occur. - lastVersion = this.systemStoreRepository.getClientRegistryStore() - .getSysStore(clientId) - .getVersion(); - hadConflict = false; - } - clientInfo.setUpdateTime(System.currentTimeMillis()); - logger.info("updating client registry with the following info for client: " + clientId - + "\n" + clientInfo); + /* + * Procedure to publish client registry info in the system store. + */ + public synchronized void publishRegistry() { try { + if(hadConflict) { + /* + * if we previously had a conflict during update, we will try to + * get a newer version before update this time. This case shall + * not happen under regular circumstances. But it is just avoid + * update keeping failing when strange situations occur. + */ + lastVersion = this.systemStoreRepository.getClientRegistryStore() + .getSysStore(clientId) + .getVersion(); + hadConflict = false; + } + clientInfo.setUpdateTime(System.currentTimeMillis()); + logger.info("updating client registry with the following info for client: " + clientId + + "\n" + clientInfo); lastVersion = this.systemStoreRepository.getClientRegistryStore() .putSysStore(clientId, new Versioned(clientInfo.toString(), @@ -84,4 +88,8 @@ public void run() { + e); } } + + public void run() { + publishRegistry(); + } } diff --git a/src/java/voldemort/cluster/failuredetector/BannagePeriodFailureDetector.java b/src/java/voldemort/cluster/failuredetector/BannagePeriodFailureDetector.java index 218992fa59..62966c9dca 100644 --- a/src/java/voldemort/cluster/failuredetector/BannagePeriodFailureDetector.java +++ b/src/java/voldemort/cluster/failuredetector/BannagePeriodFailureDetector.java @@ -49,6 +49,7 @@ */ @JmxManaged(description = "Detects the availability of the nodes on which a Voldemort cluster runs") +@Deprecated public class BannagePeriodFailureDetector extends AbstractFailureDetector { public BannagePeriodFailureDetector(FailureDetectorConfig failureDetectorConfig) { diff --git a/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java b/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java index 62830af188..ac9216e4cf 100644 --- a/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java +++ b/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java @@ -31,6 +31,15 @@ import voldemort.versioning.Version; import voldemort.versioning.Versioned; +/* + * A Storage Engine used to persist the keys and values in a human readable + * format on disk. The data is primarily served off of the cache. After each + * put, the entire cache state is flushed to the backing file. The data is UTF-8 + * serialized when writing to the file in order to make it human readable. + * + * The primary purpose of this storage engine is for maintaining the cluster + * metadata which is characterized by low QPS and not latency sensitive. + */ public class FileBackedCachingStorageEngine implements StorageEngine { private final static Logger logger = Logger.getLogger(FileBackedCachingStorageEngine.class); diff --git a/test/unit/voldemort/store/system/SystemStoreTest.java b/test/unit/voldemort/store/system/SystemStoreTest.java index a9f5e67ab2..4f6b12ffbf 100644 --- a/test/unit/voldemort/store/system/SystemStoreTest.java +++ b/test/unit/voldemort/store/system/SystemStoreTest.java @@ -118,6 +118,7 @@ public void testCustomClusterXmlStore() { @Test public void testIllegalSystemStore() { try { + @SuppressWarnings("unused") SystemStore sysVersionStore = new SystemStore("test-store", bootStrapUrls, this.CLIENT_ZONE_ID, From 78ae5d5c916f30724cf5262e7561e5de9079f9c2 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Mon, 17 Sep 2012 15:22:57 -0700 Subject: [PATCH 132/209] Removed unnecessary variables from AsyncMetadataVersionManager --- .../client/scheduler/AsyncMetadataVersionManager.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java b/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java index f90862dc12..972cadbf1e 100644 --- a/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java +++ b/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java @@ -1,7 +1,6 @@ package voldemort.client.scheduler; import java.util.Properties; -import java.util.Random; import java.util.concurrent.Callable; import org.apache.log4j.Logger; @@ -32,11 +31,6 @@ public class AsyncMetadataVersionManager implements Runnable { private Long currentClusterVersion; private final Callable storeClientThunk; private final SystemStoreRepository sysRepository; - - // Random delta generator - private final int DELTA_MAX = 2000; - private final Random randomGenerator = new Random(System.currentTimeMillis()); - public boolean isActive = false; public AsyncMetadataVersionManager(SystemStoreRepository sysRepository, From c26373f1ae114dd2bb84d433386dc7776d390df0 Mon Sep 17 00:00:00 2001 From: Zhongjie Wu Date: Wed, 19 Sep 2012 11:40:28 -0700 Subject: [PATCH 133/209] increased sleep time in RoutedStoreTest --- .../store/routed/RoutedStoreTest.java | 57 +++++++++---------- 1 file changed, 28 insertions(+), 29 deletions(-) diff --git a/test/unit/voldemort/store/routed/RoutedStoreTest.java b/test/unit/voldemort/store/routed/RoutedStoreTest.java index 975420379f..2489df1cc7 100644 --- a/test/unit/voldemort/store/routed/RoutedStoreTest.java +++ b/test/unit/voldemort/store/routed/RoutedStoreTest.java @@ -96,7 +96,7 @@ public class RoutedStoreTest extends AbstractByteArrayStoreTest { public static final int BANNAGE_PERIOD = 1000; - public static final int SLEEPY_TIME = 81; + public static final int SLEEPY_TIME = 200; public static final int OPERATION_TIMEOUT = 60; private Cluster cluster; @@ -204,7 +204,7 @@ else if(count < failing + sleepy) routedStoreThreadPool = Executors.newFixedThreadPool(threads); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(isPipelineRoutedStoreEnabled, routedStoreThreadPool, - new TimeoutConfig(1000L, + new TimeoutConfig(BANNAGE_PERIOD, false)); return routedStoreFactory.create(cluster, storeDef, subStores, true, failureDetector); @@ -415,7 +415,7 @@ public void testZoneRouting() throws Exception { s1.put(new ByteArray("test".getBytes()), versioned, null); } finally { long elapsed = (System.nanoTime() - start) / Time.NS_PER_MS; - assertTrue(elapsed + " < " + 81, elapsed < 81); + assertTrue(elapsed + " < " + SLEEPY_TIME, elapsed < SLEEPY_TIME); } // Putting extra key to test getAll s1.put(new ByteArray("test2".getBytes()), versioned, null); @@ -425,7 +425,7 @@ public void testZoneRouting() throws Exception { s1.get(new ByteArray("test".getBytes()), null); } finally { long elapsed = (System.nanoTime() - start) / Time.NS_PER_MS; - assertTrue(elapsed + " < " + 130, elapsed < 130); + assertTrue(elapsed + " < " + SLEEPY_TIME, elapsed < SLEEPY_TIME); } start = System.nanoTime(); @@ -436,23 +436,21 @@ public void testZoneRouting() throws Exception { } } finally { long elapsed = (System.nanoTime() - start) / Time.NS_PER_MS; - assertTrue(elapsed + " < " + 81, elapsed < 81); + assertTrue(elapsed + " < " + SLEEPY_TIME, elapsed < SLEEPY_TIME); } - // make sure the failure detector adds back any previously failed nodes - Thread.sleep(BANNAGE_PERIOD * 2); start = System.nanoTime(); try { s1.delete(new ByteArray("test".getBytes()), versioned.getVersion()); } finally { long elapsed = (System.nanoTime() - start) / Time.NS_PER_MS; - assertTrue(elapsed + " < " + 81, elapsed < 81); + assertTrue(elapsed + " < " + SLEEPY_TIME, elapsed < SLEEPY_TIME); } // make sure sleepy stores processed the delete before checking, // otherwise, we might be bailing // out of the test too early for the delete to be processed. - Thread.sleep(SLEEPY_TIME); + Thread.sleep(SLEEPY_TIME * 2); List keys = Lists.newArrayList(new ByteArray("test".getBytes()), new ByteArray("test2".getBytes())); @@ -474,7 +472,7 @@ public void testZoneRouting() throws Exception { zoneReplicationFactor, RoutingStrategyType.ZONE_STRATEGY, SLEEPY_TIME, - 1000, + BANNAGE_PERIOD, new VoldemortException()); start = System.nanoTime(); @@ -483,7 +481,7 @@ public void testZoneRouting() throws Exception { s2.put(new ByteArray("test".getBytes()), versioned, null); } finally { long elapsed = (System.nanoTime() - start) / Time.NS_PER_MS; - assertTrue(elapsed + " > " + 81, elapsed >= 81); + assertTrue(elapsed + " > " + SLEEPY_TIME, elapsed >= SLEEPY_TIME); } s2.put(new ByteArray("test2".getBytes()), versioned, null); @@ -534,8 +532,8 @@ public void testZoneRouting() throws Exception { null, zoneReplicationFactor, RoutingStrategyType.ZONE_STRATEGY, - 81, - 1000, + SLEEPY_TIME, + BANNAGE_PERIOD, new VoldemortException()); start = System.nanoTime(); @@ -543,7 +541,7 @@ public void testZoneRouting() throws Exception { s3.put(new ByteArray("test".getBytes()), versioned, null); } finally { long elapsed = (System.nanoTime() - start) / Time.NS_PER_MS; - assertTrue(elapsed + " < " + 81, elapsed < 81); + assertTrue(elapsed + " < " + SLEEPY_TIME, elapsed < SLEEPY_TIME); } // Putting extra key to test getAll s3.put(new ByteArray("test2".getBytes()), versioned, null); @@ -556,7 +554,7 @@ public void testZoneRouting() throws Exception { } } finally { long elapsed = (System.nanoTime() - start) / Time.NS_PER_MS; - assertTrue(elapsed + " < " + 81, elapsed < 81); + assertTrue(elapsed + " < " + SLEEPY_TIME, elapsed < SLEEPY_TIME); } start = System.nanoTime(); @@ -564,7 +562,7 @@ public void testZoneRouting() throws Exception { s3.get(new ByteArray("test".getBytes()), null); } finally { long elapsed = (System.nanoTime() - start) / Time.NS_PER_MS; - assertTrue(elapsed + " < " + 81, elapsed < 81); + assertTrue(elapsed + " < " + SLEEPY_TIME, elapsed < SLEEPY_TIME); } start = System.nanoTime(); @@ -572,7 +570,7 @@ public void testZoneRouting() throws Exception { s3.delete(new ByteArray("test".getBytes()), versioned.getVersion()); } finally { long elapsed = (System.nanoTime() - start) / Time.NS_PER_MS; - assertTrue(elapsed + " < " + 81, elapsed < 81); + assertTrue(elapsed + " < " + SLEEPY_TIME, elapsed < SLEEPY_TIME); } // Basic put with zone read = 1, zone write = 1 and failures in other @@ -587,8 +585,8 @@ public void testZoneRouting() throws Exception { null, zoneReplicationFactor, RoutingStrategyType.ZONE_STRATEGY, - 81, - 1000, + SLEEPY_TIME, + BANNAGE_PERIOD, new VoldemortException()); try { @@ -875,7 +873,7 @@ public void testGetAllWithFailingStore() throws Exception { routedStoreThreadPool = Executors.newFixedThreadPool(1); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(isPipelineRoutedStoreEnabled, routedStoreThreadPool, - new TimeoutConfig(1000L, + new TimeoutConfig(BANNAGE_PERIOD, false)); RoutedStore routedStore = routedStoreFactory.create(cluster, @@ -932,7 +930,7 @@ public void testGetAllWithMorePreferredReadsThanNodes() throws Exception { routedStoreThreadPool = Executors.newFixedThreadPool(1); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(isPipelineRoutedStoreEnabled, routedStoreThreadPool, - new TimeoutConfig(1000L, + new TimeoutConfig(BANNAGE_PERIOD, false)); RoutedStore routedStore = routedStoreFactory.create(cluster, @@ -1040,7 +1038,7 @@ public void testPutWithOneNodeDownAndOneNodeSlow() throws Exception { routedStoreThreadPool = Executors.newFixedThreadPool(1); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(isPipelineRoutedStoreEnabled, routedStoreThreadPool, - new TimeoutConfig(1000L, + new TimeoutConfig(BANNAGE_PERIOD, false)); RoutedStore routedStore = routedStoreFactory.create(cluster, @@ -1247,7 +1245,7 @@ public void testNoReadRepair() throws Exception { routedStoreThreadPool = Executors.newFixedThreadPool(1); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(isPipelineRoutedStoreEnabled, routedStoreThreadPool, - new TimeoutConfig(1000L, + new TimeoutConfig(BANNAGE_PERIOD, false)); RoutedStore routedStore = routedStoreFactory.create(cluster, @@ -1392,7 +1390,7 @@ public void testPutDeleteZoneRouting() throws Exception { Store subStore = null; if(sleepy != null && sleepy.contains(n.getId())) - subStore = new SleepyStore(81, + subStore = new SleepyStore(SLEEPY_TIME, new InMemoryStorageEngine("test")); else subStore = new InMemoryStorageEngine("test"); @@ -1416,7 +1414,8 @@ public void testPutDeleteZoneRouting() throws Exception { routedStoreThreadPool = Executors.newFixedThreadPool(8); RoutedStoreFactory routedStoreFactory = new RoutedStoreFactory(true, routedStoreThreadPool, - new TimeoutConfig(60, false)); + new TimeoutConfig(OPERATION_TIMEOUT, + false)); Store s1 = routedStoreFactory.create(cluster, storeDef, @@ -1433,10 +1432,10 @@ public void testPutDeleteZoneRouting() throws Exception { s1.put(new ByteArray("test".getBytes()), versioned, null); } finally { elapsed = (System.nanoTime() - start) / Time.NS_PER_MS; - assertTrue(elapsed + " < " + 81, elapsed < 81); + assertTrue(elapsed + " < " + SLEEPY_TIME, elapsed < SLEEPY_TIME); } - Thread.sleep(81 - elapsed); + Thread.sleep(SLEEPY_TIME - elapsed); for(Node node: nodesRoutedTo) { assertEquals(subStores.get(node.getId()) @@ -1451,10 +1450,10 @@ public void testPutDeleteZoneRouting() throws Exception { s1.delete(new ByteArray("test".getBytes()), versioned.getVersion()); } finally { elapsed = (System.nanoTime() - start) / Time.NS_PER_MS; - assertTrue(elapsed + " < " + 81, elapsed < 81); + assertTrue(elapsed + " < " + SLEEPY_TIME, elapsed < SLEEPY_TIME); } - Thread.sleep(81 - elapsed); + Thread.sleep(SLEEPY_TIME - elapsed); for(Node node: nodesRoutedTo) { assertEquals(subStores.get(node.getId()) From 87914feed43853c07bc4f834070e5241c705966a Mon Sep 17 00:00:00 2001 From: Brent Miller Date: Wed, 19 Sep 2012 19:25:48 -0700 Subject: [PATCH 134/209] [https://github.com/voldemort/voldemort/issues/108] allow getting byte and object arrays using voldemort-shell.sh --- src/java/voldemort/VoldemortClientShell.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/java/voldemort/VoldemortClientShell.java b/src/java/voldemort/VoldemortClientShell.java index 6c364cac84..93bd7d6a29 100644 --- a/src/java/voldemort/VoldemortClientShell.java +++ b/src/java/voldemort/VoldemortClientShell.java @@ -381,6 +381,12 @@ private static void printObject(Object o) { System.out.print(", "); } System.out.print('}'); + } else if(o instanceof Object[]) { + Object[] a = (Object[]) o; + System.out.print( Arrays.deepToString(a) ); + } else if(o instanceof byte[]) { + byte[] a = (byte[]) o; + System.out.print( Arrays.toString(a) ); } else { System.out.print(o); } From 6889d34bcf012a360f1d526e30a9d205fca41dc8 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Thu, 20 Sep 2012 23:34:43 -0700 Subject: [PATCH 135/209] Final review comments correction for autobootstrapper: Copyright, documentation and variable naming convention --- .../client/AbstractStoreClientFactory.java | 5 +- src/java/voldemort/client/ClientConfig.java | 27 +++---- src/java/voldemort/client/ClientInfo.java | 27 +++---- src/java/voldemort/client/SystemStore.java | 23 +++++- .../client/SystemStoreRepository.java | 24 ++++-- src/java/voldemort/client/ZenStoreClient.java | 18 ++--- .../AsyncMetadataVersionManager.java | 33 ++++++-- .../FailureDetectorConfig.java | 16 +--- .../routing/RouteToAllLocalPrefStrategy.java | 25 ++++++ ...FileBackedCachingStorageConfiguration.java | 25 +++++- .../FileBackedCachingStorageEngine.java | 34 ++++++-- .../store/metadata/MetadataStore.java | 2 +- .../store/routed/PipelineRoutedStore.java | 47 +++++------ .../routed/action/ConfigureNodesByZone.java | 2 +- .../routed/action/ConfigureNodesDefault.java | 18 ++++- .../action/ConfigureNodesLocalHost.java | 18 ++++- .../action/ConfigureNodesLocalHostByZone.java | 18 ++++- .../store/system/SystemStoreConstants.java | 20 +---- .../utils/MetadataVersionStoreUtils.java | 23 ++++++ test/unit/voldemort/client/ClientJmxTest.java | 16 ++++ .../voldemort/client/ClientRegistryTest.java | 78 ++++++++++++++----- .../client/EndToEndRebootstrapTest.java | 47 +++++++++-- .../FileBackedCachingStorageEngineTest.java | 22 ++++++ .../action/ConfigureNodesLocalHostTest.java | 27 ++++++- .../AsyncMetadataVersionManagerTest.java | 29 +++++++ .../store/system/SystemStoreTest.java | 23 ++++++ 26 files changed, 498 insertions(+), 149 deletions(-) diff --git a/src/java/voldemort/client/AbstractStoreClientFactory.java b/src/java/voldemort/client/AbstractStoreClientFactory.java index e996937ab7..08502196cb 100644 --- a/src/java/voldemort/client/AbstractStoreClientFactory.java +++ b/src/java/voldemort/client/AbstractStoreClientFactory.java @@ -345,6 +345,7 @@ protected abstract FailureDetector initFailureDetector(final ClientConfig config public FailureDetector getFailureDetector() { if(this.cluster == null) { + logger.info("Cluster is null ! Getting cluster.xml again for setting up FailureDetector."); String clusterXml = bootstrapMetadataWithRetries(MetadataStore.CLUSTER_KEY, bootstrapUrls); this.cluster = clusterMapper.readCluster(new StringReader(clusterXml), false); @@ -354,11 +355,11 @@ public FailureDetector getFailureDetector() { FailureDetector result = failureDetector; if(result == null) { - logger.debug("Failure detector is null. Creating a new FD."); synchronized(this) { // second check: avoids double initialization result = failureDetector; if(result == null) { + logger.info("Failure detector is null. Creating a new FD."); failureDetector = result = initFailureDetector(config, this.cluster); if(isJmxEnabled) { JmxUtils.registerMbean(failureDetector, @@ -373,7 +374,7 @@ public FailureDetector getFailureDetector() { /* * The existing failure detector might have an old state */ - logger.debug("Failure detector already exists. Updating the state and flushing cached verifier stores."); + logger.info("Failure detector already exists. Updating the state and flushing cached verifier stores."); synchronized(this) { failureDetector.getConfig().setCluster(this.cluster); failureDetector.getConfig().getStoreVerifier().flushCachedStores(); diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index 59081b113d..66a3ec6d4d 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -84,9 +84,9 @@ public class ClientConfig { private volatile String clientContextName = ""; /* 5 second check interval, in ms */ - private volatile long asyncCheckMetadataInterval = 5000; + private volatile long asyncCheckMetadataIntervalInMs = 5000; /* 12 hr refresh internval, in seconds */ - private volatile int clientRegistryRefreshInterval = 3600 * 12; + private volatile int clientRegistryRefreshIntervalInSecs = 3600 * 12; private volatile int asyncJobThreadPoolSize = 2; /* SystemStore client config */ @@ -136,10 +136,10 @@ public ClientConfig() {} public static final String FAILUREDETECTOR_CATASTROPHIC_ERROR_TYPES_PROPERTY = "failuredetector_catastrophic_error_types"; public static final String FAILUREDETECTOR_REQUEST_LENGTH_THRESHOLD_PROPERTY = "failuredetector_request_length_threshold"; public static final String MAX_BOOTSTRAP_RETRIES = "max_bootstrap_retries"; - public static final String CLIENT_CONTEXT_NAME = "voldemort_client_context"; - public static final String ASYNC_CHECK_METADATA_INTERVAL = "check_metadata_interval"; + public static final String CLIENT_CONTEXT_NAME = "voldemort_client_context_name"; + public static final String ASYNC_CHECK_METADATA_INTERVAL = "check_metadata_interval_ms"; public static final String USE_DEFAULT_CLIENT = "use_default_client"; - public static final String CLIENT_REGISTRY_REFRESH_INTERVAL = "client_registry_refresh_interval"; + public static final String CLIENT_REGISTRY_REFRESH_INTERVAL = "client_registry_refresh_interval_seconds"; public static final String ASYNC_JOB_THREAD_POOL_SIZE = "async_job_thread_pool_size"; public static final String SYS_MAX_CONNECTIONS_PER_NODE = "sys_max_connections_per_node"; public static final String SYS_ROUTING_TIMEOUT_MS = "sys_routing_timeout_ms"; @@ -318,7 +318,7 @@ private void setProperties(Properties properties) { } if(props.containsKey(CLIENT_REGISTRY_REFRESH_INTERVAL)) { - this.setClientRegistryUpdateInSecs(props.getInt(CLIENT_REGISTRY_REFRESH_INTERVAL)); + this.setClientRegistryUpdateIntervalInSecs(props.getInt(CLIENT_REGISTRY_REFRESH_INTERVAL)); } if(props.containsKey(ASYNC_JOB_THREAD_POOL_SIZE)) { @@ -852,7 +852,7 @@ public ClientConfig setClientContextName(String clientContextName) { } public long getAsyncMetadataRefreshInMs() { - return asyncCheckMetadataInterval; + return asyncCheckMetadataIntervalInMs; } /** @@ -862,22 +862,23 @@ public long getAsyncMetadataRefreshInMs() { */ public ClientConfig setAsyncMetadataRefreshInMs(long asyncCheckMetadataInterval) { - this.asyncCheckMetadataInterval = asyncCheckMetadataInterval; + this.asyncCheckMetadataIntervalInMs = asyncCheckMetadataInterval; return this; } - public int getClientRegistryUpdateInSecs() { - return this.clientRegistryRefreshInterval; + public int getClientRegistryUpdateIntervalInSecs() { + return this.clientRegistryRefreshIntervalInSecs; } /** * Set the interval on which client refreshes its corresponding entry of the * client registry on the servers * - * @param clientRegistryRefreshInterval The refresh interval in seconds + * @param clientRegistryRefreshIntervalInSecs The refresh interval in + * seconds */ - public ClientConfig setClientRegistryUpdateInSecs(int clientRegistryRefrshInterval) { - this.clientRegistryRefreshInterval = clientRegistryRefrshInterval; + public ClientConfig setClientRegistryUpdateIntervalInSecs(int clientRegistryRefrshInterval) { + this.clientRegistryRefreshIntervalInSecs = clientRegistryRefrshInterval; return this; } diff --git a/src/java/voldemort/client/ClientInfo.java b/src/java/voldemort/client/ClientInfo.java index 0f28941176..f84e1e0c77 100644 --- a/src/java/voldemort/client/ClientInfo.java +++ b/src/java/voldemort/client/ClientInfo.java @@ -32,20 +32,17 @@ */ public class ClientInfo implements Serializable { - /** - * - */ private static final long serialVersionUID = 1L; protected static final Logger logger = Logger.getLogger(ClientInfo.class); - private long bootstrapTime; + private long bootstrapTimestampMs; private String storeName; private String context; private int sequence; private String localHostName; private String deploymentPath; - private long updateTime; + private long updateTimestampMs; private String releaseVersion; private ClientConfig config; private long clusterMetadataVersion; @@ -56,13 +53,13 @@ public ClientInfo(String storeName, long bootstrapTime, String version, ClientConfig config) { - this.bootstrapTime = bootstrapTime; + this.bootstrapTimestampMs = bootstrapTime; this.storeName = storeName; this.context = clientContext; this.sequence = clientSequence; this.localHostName = createHostName(); this.deploymentPath = createDeploymentPath(); - this.updateTime = bootstrapTime; + this.updateTimestampMs = bootstrapTime; this.releaseVersion = version; this.config = config; this.clusterMetadataVersion = 0; @@ -104,11 +101,11 @@ public synchronized String getStoreName() { } public synchronized void setBootstrapTime(long bootstrapTime) { - this.bootstrapTime = bootstrapTime; + this.bootstrapTimestampMs = bootstrapTime; } public synchronized long getBootstrapTime() { - return bootstrapTime; + return bootstrapTimestampMs; } public synchronized void setContext(String clientContext) { @@ -144,11 +141,11 @@ public synchronized String getLocalHostName() { } public synchronized void setUpdateTime(long updateTime) { - this.updateTime = updateTime; + this.updateTimestampMs = updateTime; } public synchronized long getUpdateTime() { - return this.updateTime; + return this.updateTimestampMs; } public synchronized void setReleaseVersion(String version) { @@ -180,26 +177,26 @@ public boolean equals(Object object) { if(!object.getClass().equals(ClientInfo.class)) return false; ClientInfo clientInfo = (ClientInfo) object; - return (this.bootstrapTime == clientInfo.bootstrapTime) + return (this.bootstrapTimestampMs == clientInfo.bootstrapTimestampMs) && (this.context.equals(clientInfo.context)) && (this.deploymentPath.equals(clientInfo.deploymentPath)) && (this.localHostName.equals(clientInfo.localHostName)) && (this.sequence == clientInfo.sequence) && (this.storeName.equals(clientInfo.storeName)) - && (this.updateTime == clientInfo.updateTime) + && (this.updateTimestampMs == clientInfo.updateTimestampMs) && (this.releaseVersion == clientInfo.releaseVersion); } @Override public synchronized String toString() { StringBuilder builder = new StringBuilder(); - builder.append("bootstrapTime=").append(bootstrapTime).append("\n"); + builder.append("bootstrapTime=").append(bootstrapTimestampMs).append("\n"); builder.append("context=").append(context).append("\n"); builder.append("deploymentPath=").append(deploymentPath).append("\n"); builder.append("localHostName=").append(localHostName).append("\n"); builder.append("sequence=").append(sequence).append("\n"); builder.append("storeName=").append(storeName).append("\n"); - builder.append("updateTime=").append(updateTime).append("\n"); + builder.append("updateTime=").append(updateTimestampMs).append("\n"); builder.append("releaseVersion=").append(releaseVersion).append("\n"); builder.append("clusterMetadataVersion=").append(clusterMetadataVersion).append("\n"); diff --git a/src/java/voldemort/client/SystemStore.java b/src/java/voldemort/client/SystemStore.java index 4303002a75..4a79c91b25 100644 --- a/src/java/voldemort/client/SystemStore.java +++ b/src/java/voldemort/client/SystemStore.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2012 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.client; import java.util.List; @@ -14,10 +30,15 @@ import voldemort.versioning.Version; import voldemort.versioning.Versioned; -/* +/** * A client interface for interacting with System stores (managed by the * cluster). The naming convention is kept consistent with SocketStore (which is * also a client interface). + * + * @author csoman + * + * @param Type of Key + * @param Type of Value */ public class SystemStore { diff --git a/src/java/voldemort/client/SystemStoreRepository.java b/src/java/voldemort/client/SystemStoreRepository.java index 38289a91b2..6f9cd42015 100644 --- a/src/java/voldemort/client/SystemStoreRepository.java +++ b/src/java/voldemort/client/SystemStoreRepository.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2009 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.client; import java.util.concurrent.ConcurrentHashMap; @@ -5,7 +21,7 @@ import voldemort.cluster.failuredetector.FailureDetector; import voldemort.store.system.SystemStoreConstants; -/* +/** * A repository that creates and maintains all the system stores in one place. * The purpose is to act as a source of truth for all the system stores, since * they can be recreated dynamically (in case cluster.xml changes). @@ -35,12 +51,6 @@ public void createSystemStores(ClientConfig config, String clusterXml, FailureDe } } - public SystemStore getVersionStore() { - String name = SystemStoreConstants.SystemStoreName.voldsys$_metadata_version.name(); - SystemStore sysVersionStore = sysStoreMap.get(name); - return sysVersionStore; - } - public SystemStore getClientRegistryStore() { String name = SystemStoreConstants.SystemStoreName.voldsys$_client_registry.name(); SystemStore sysRegistryStore = sysStoreMap.get(name); diff --git a/src/java/voldemort/client/ZenStoreClient.java b/src/java/voldemort/client/ZenStoreClient.java index 6db6492cbf..ba18b5b13e 100644 --- a/src/java/voldemort/client/ZenStoreClient.java +++ b/src/java/voldemort/client/ZenStoreClient.java @@ -53,9 +53,6 @@ @JmxManaged(description = "A voldemort client") public class ZenStoreClient extends DefaultStoreClient { - private static final int ASYNC_THREADS_COUNT = 2; - private static final boolean ALLOW_INTERRUPT_ASYNC = true; - private final Logger logger = Logger.getLogger(ZenStoreClient.class); private final AbstractStoreClientFactory abstractStoreFactory; @@ -116,11 +113,12 @@ public ZenStoreClient(String storeName, config.getAsyncMetadataRefreshInMs()); } - clientRegistryRefresher = registerClient(clientId, config.getClientRegistryUpdateInSecs()); + clientRegistryRefresher = registerClient(clientId, + config.getClientRegistryUpdateIntervalInSecs()); logger.info("Voldemort client created: " + clientId + "\n" + clientInfo); } - private ClientRegistryRefresher registerClient(String jobId, int interval) { + private ClientRegistryRefresher registerClient(String jobId, int intervalInSecs) { ClientRegistryRefresher refresher = null; if(this.sysRepository.getClientRegistryStore() != null) { try { @@ -131,15 +129,16 @@ private ClientRegistryRefresher registerClient(String jobId, int interval) { clientInfo, version); GregorianCalendar cal = new GregorianCalendar(); - cal.add(Calendar.SECOND, interval); + cal.add(Calendar.SECOND, intervalInSecs); if(scheduler != null) { scheduler.schedule(jobId + refresher.getClass().getName(), refresher, cal.getTime(), - TimeUnit.MILLISECONDS.convert(interval, TimeUnit.SECONDS)); + TimeUnit.MILLISECONDS.convert(intervalInSecs, + TimeUnit.SECONDS)); logger.info("Client registry refresher thread started, refresh interval: " - + interval + " seconds"); + + intervalInSecs + " seconds"); } else { logger.warn("Client registry won't run because scheduler service is not configured"); } @@ -156,7 +155,7 @@ private ClientRegistryRefresher registerClient(String jobId, int interval) { private AsyncMetadataVersionManager scheduleAsyncMetadataVersionManager(String jobId, long interval) { AsyncMetadataVersionManager asyncMetadataManager = null; - SystemStore versionStore = this.sysRepository.getVersionStore(); + SystemStore versionStore = this.sysRepository.getMetadataVersionStore(); if(versionStore == null) { logger.warn("Metadata version system store not found. Cannot run Metadata version check thread."); } else { @@ -200,6 +199,7 @@ public void bootStrap() { */ clusterXml = abstractStoreFactory.bootstrapMetadataWithRetries(MetadataStore.CLUSTER_KEY); + // Get client store this.store = abstractStoreFactory.getRawStore(storeName, resolver, null, clusterXml, null); // Create system stores diff --git a/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java b/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java index 972cadbf1e..71987c70a3 100644 --- a/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java +++ b/src/java/voldemort/client/scheduler/AsyncMetadataVersionManager.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2012 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.client.scheduler; import java.util.Properties; @@ -8,7 +24,7 @@ import voldemort.client.SystemStoreRepository; import voldemort.utils.MetadataVersionStoreUtils; -/* +/** * The AsyncMetadataVersionManager is used to track the Metadata version on the * cluster and if necessary Re-bootstrap the client. * @@ -20,6 +36,9 @@ * * At the moment, this only tracks the cluster.xml changes. TODO: Extend this to * track other stuff (like stores.xml) + * + * @author csoman + * */ public class AsyncMetadataVersionManager implements Runnable { @@ -30,15 +49,15 @@ public class AsyncMetadataVersionManager implements Runnable { private final Logger logger = Logger.getLogger(this.getClass()); private Long currentClusterVersion; private final Callable storeClientThunk; - private final SystemStoreRepository sysRepository; + private final SystemStoreRepository systemStoreRepository; public boolean isActive = false; public AsyncMetadataVersionManager(SystemStoreRepository sysRepository, Callable storeClientThunk) { - this.sysRepository = sysRepository; + this.systemStoreRepository = sysRepository; // Get the properties object from the system store (containing versions) - Properties versionProps = MetadataVersionStoreUtils.getProperties(this.sysRepository.getMetadataVersionStore()); + Properties versionProps = MetadataVersionStoreUtils.getProperties(this.systemStoreRepository.getMetadataVersionStore()); try { this.currentClusterVersion = getCurrentVersion(CLUSTER_VERSION_KEY, versionProps); @@ -108,14 +127,14 @@ public void run() { * Get the properties object from the system store (containing * versions) */ - Properties versionProps = MetadataVersionStoreUtils.getProperties(this.sysRepository.getMetadataVersionStore()); + Properties versionProps = MetadataVersionStoreUtils.getProperties(this.systemStoreRepository.getMetadataVersionStore()); Long newClusterVersion = fetchNewVersion(CLUSTER_VERSION_KEY, currentClusterVersion, versionProps); // If nothing has been updated, continue if(newClusterVersion != null) { - logger.info("Metadata version mismatch detected. Re-bootstrapping !!!"); + logger.info("Metadata version mismatch detected. Re-bootstrapping!"); try { logger.info("Updating cluster version"); currentClusterVersion = newClusterVersion; @@ -141,7 +160,7 @@ public Long getClusterMetadataVersion() { // Fetch the latest versions for cluster metadata public void updateMetadataVersions() { - Properties versionProps = MetadataVersionStoreUtils.getProperties(this.sysRepository.getMetadataVersionStore()); + Properties versionProps = MetadataVersionStoreUtils.getProperties(this.systemStoreRepository.getMetadataVersionStore()); Long newVersion = fetchNewVersion(CLUSTER_VERSION_KEY, null, versionProps); if(newVersion != null) { this.currentClusterVersion = newVersion; diff --git a/src/java/voldemort/cluster/failuredetector/FailureDetectorConfig.java b/src/java/voldemort/cluster/failuredetector/FailureDetectorConfig.java index 5856d22254..6fc76aa1a1 100644 --- a/src/java/voldemort/cluster/failuredetector/FailureDetectorConfig.java +++ b/src/java/voldemort/cluster/failuredetector/FailureDetectorConfig.java @@ -547,7 +547,7 @@ public FailureDetectorConfig setRequestLengthThreshold(long requestLengthThresho * topology */ - public synchronized Cluster getCluster() { + public Cluster getCluster() { return this.cluster; } @@ -558,7 +558,7 @@ public synchronized Cluster getCluster() { * non-null */ - public synchronized FailureDetectorConfig setCluster(Cluster cluster) { + public FailureDetectorConfig setCluster(Cluster cluster) { Utils.notNull(cluster); this.cluster = cluster; return this; @@ -569,14 +569,10 @@ public synchronized FailureDetectorConfig setCluster(Cluster cluster) { * detector configuration. * * @return Collection of Node instances, usually determined from the Cluster - * - * EDIT: this should be deprecated in the future. We should not be - * making copies of the cluster state. It results in a series of - * inconsistency bugs. */ + @Deprecated public synchronized Collection getNodes() { - System.err.println("DEPRECATED !!! Please use getCluster().getNodes() method instead !"); return ImmutableSet.copyOf(this.cluster.getNodes()); } @@ -586,14 +582,10 @@ public synchronized Collection getNodes() { * * @param nodes Collection of Node instances, usually determined from the * Cluster; must be non-null - * - * EDIT: this should be deprecated in the future. We should not be - * making copies of the cluster state. It results in a series of - * inconsistency bugs. */ + @Deprecated public synchronized FailureDetectorConfig setNodes(Collection nodes) { - System.err.println("DEPRECATED !!! Please use setCluster method instead !"); Utils.notNull(nodes); this.nodes = new HashSet(nodes); return this; diff --git a/src/java/voldemort/routing/RouteToAllLocalPrefStrategy.java b/src/java/voldemort/routing/RouteToAllLocalPrefStrategy.java index 240aff1d49..f721312e7f 100644 --- a/src/java/voldemort/routing/RouteToAllLocalPrefStrategy.java +++ b/src/java/voldemort/routing/RouteToAllLocalPrefStrategy.java @@ -1,9 +1,34 @@ +/* + * Copyright 2008-2009 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.routing; import java.util.Collection; import voldemort.cluster.Node; +/** + * A class that denotes a route to all strategy with local preference. This + * class is meant to be consistent with the routing hierarchy convention. It + * simply returns the list of all nodes (just like RouteToAllStrategy) but is + * used to indicate that extra processing will be done down the pipeline. + * + * @author csoman + * + */ public class RouteToAllLocalPrefStrategy extends RouteToAllStrategy { public RouteToAllLocalPrefStrategy(Collection nodes) { diff --git a/src/java/voldemort/store/configuration/FileBackedCachingStorageConfiguration.java b/src/java/voldemort/store/configuration/FileBackedCachingStorageConfiguration.java index 069a6907c1..8cd93aa82c 100644 --- a/src/java/voldemort/store/configuration/FileBackedCachingStorageConfiguration.java +++ b/src/java/voldemort/store/configuration/FileBackedCachingStorageConfiguration.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2012 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.store.configuration; import voldemort.server.VoldemortConfig; @@ -6,9 +22,16 @@ import voldemort.store.StoreDefinition; import voldemort.utils.ByteArray; +/** + * Storage configuration class for FileBackedCachingStorageEngine + * + * @author csoman + * + */ + public class FileBackedCachingStorageConfiguration implements StorageConfiguration { - public static final String TYPE_NAME = "file"; + public static final String TYPE_NAME = "file-backed-cache"; private final String inputPath; public FileBackedCachingStorageConfiguration(VoldemortConfig config) { diff --git a/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java b/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java index ac9216e4cf..38817ce272 100644 --- a/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java +++ b/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2012 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.store.configuration; import java.io.BufferedReader; @@ -31,7 +47,7 @@ import voldemort.versioning.Version; import voldemort.versioning.Versioned; -/* +/** * A Storage Engine used to persist the keys and values in a human readable * format on disk. The data is primarily served off of the cache. After each * put, the entire cache state is flushed to the backing file. The data is UTF-8 @@ -39,11 +55,15 @@ * * The primary purpose of this storage engine is for maintaining the cluster * metadata which is characterized by low QPS and not latency sensitive. + * + * @author csoman + * */ public class FileBackedCachingStorageEngine implements StorageEngine { private final static Logger logger = Logger.getLogger(FileBackedCachingStorageEngine.class); private static final CharSequence NEW_PROPERTY_SEPARATOR = "[name="; + private static final String NEW_LINE = System.getProperty("line.separator"); private final String inputPath; private final String inputDirectory; @@ -60,7 +80,7 @@ public FileBackedCachingStorageEngine(String name, String inputDirectory) { + " does not exist or can not be read."); } - this.inputPath = inputDirectory + "/" + name; + this.inputPath = inputDirectory + System.getProperty("file.separator") + name; this.metadataMap = new HashMap(); this.loadData(); if(logger.isDebugEnabled()) { @@ -112,7 +132,7 @@ private void loadData() { String line = reader.readLine(); while(line != null) { - if(line.contains(NEW_PROPERTY_SEPARATOR)) { + if(line.startsWith(NEW_PROPERTY_SEPARATOR.toString())) { String key = null; StringBuilder value = new StringBuilder(); String parts[] = line.split("="); @@ -124,11 +144,11 @@ private void loadData() { // Now read the value block ! while((line = reader.readLine()) != null && line.length() != 0 - && !line.contains(NEW_PROPERTY_SEPARATOR)) { + && !line.startsWith(NEW_PROPERTY_SEPARATOR.toString())) { if(value.length() == 0) { value.append(line); } else { - value.append("\n" + line); + value.append(NEW_LINE + line); } } @@ -152,9 +172,9 @@ private synchronized void flushData() { try { writer = new BufferedWriter(new FileWriter(new File(this.inputPath))); for(String key: this.metadataMap.keySet()) { - writer.write(NEW_PROPERTY_SEPARATOR + key.toString() + "]\n"); + writer.write(NEW_PROPERTY_SEPARATOR + key.toString() + "]" + NEW_LINE); writer.write(this.metadataMap.get(key).toString()); - writer.write("\n\n"); + writer.write("" + NEW_LINE + "" + NEW_LINE); } writer.flush(); } catch(IOException e) { diff --git a/src/java/voldemort/store/metadata/MetadataStore.java b/src/java/voldemort/store/metadata/MetadataStore.java index 93fd015d49..97a31aba64 100644 --- a/src/java/voldemort/store/metadata/MetadataStore.java +++ b/src/java/voldemort/store/metadata/MetadataStore.java @@ -74,7 +74,7 @@ public class MetadataStore implements StorageEngine { public static final String CLUSTER_KEY = "cluster.xml"; public static final String STORES_KEY = "stores.xml"; - public static final String SYSTEM_STORES_KEY = "systemStores"; + public static final String SYSTEM_STORES_KEY = "system.stores"; public static final String SERVER_STATE_KEY = "server.state"; public static final String NODE_ID_KEY = "node.id"; public static final String REBALANCING_STEAL_INFO = "rebalancing.steal.info.key"; diff --git a/src/java/voldemort/store/routed/PipelineRoutedStore.java b/src/java/voldemort/store/routed/PipelineRoutedStore.java index e6284d2e4b..968ff5a89e 100644 --- a/src/java/voldemort/store/routed/PipelineRoutedStore.java +++ b/src/java/voldemort/store/routed/PipelineRoutedStore.java @@ -157,29 +157,24 @@ public PipelineRoutedStore(String name, } private ConfigureNodesType obtainNodeConfigurationType(Integer zonesRequired) { - // If Zone and local preference required - if(zonesRequired != null - && routingStrategy.getType().equals(RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY)) - return ConfigureNodesType.BYZONE_LOCAL; - - // If only local preference required - else if(zonesRequired == null - && routingStrategy.getType().equals(RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY)) - return ConfigureNodesType.DEFAULT_LOCAL; - - // If only Zone required - else if(zonesRequired != null - && !routingStrategy.getType() - .equals(RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY)) - return ConfigureNodesType.BYZONE; - - // Default case - return ConfigureNodesType.DEFAULT; + if(zonesRequired != null) { + if(routingStrategy.getType().equals(RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY)) { + return ConfigureNodesType.BYZONE_LOCAL; + } else { + return ConfigureNodesType.BYZONE; + } + } else { + if(routingStrategy.getType().equals(RoutingStrategyType.TO_ALL_LOCAL_PREF_STRATEGY)) { + return ConfigureNodesType.DEFAULT_LOCAL; + } + } + + return ConfigureNodesType.DEFAULT; } - private AbstractConfigureNodes>, BasicPipelineData>>> getNodeConfiguration(BasicPipelineData>> pipelineData, - ByteArray key) { + private AbstractConfigureNodes>, BasicPipelineData>>> makeNodeConfigurationForGet(BasicPipelineData>> pipelineData, + ByteArray key) { switch(obtainNodeConfigurationType(pipelineData.getZonesRequired())) { case DEFAULT: return new ConfigureNodesDefault>, BasicPipelineData>>>(pipelineData, @@ -250,8 +245,8 @@ public List> request(Store store) { // Get the correct type of configure nodes action depending on the store // requirements - AbstractConfigureNodes>, BasicPipelineData>>> configureNodes = getNodeConfiguration(pipelineData, - key); + AbstractConfigureNodes>, BasicPipelineData>>> configureNodes = makeNodeConfigurationForGet(pipelineData, + key); pipeline.addEventAction(Event.STARTED, configureNodes); @@ -675,8 +670,8 @@ public boolean isHintedHandoffEnabled() { return slopStores != null; } - private AbstractConfigureNodes putNodeConfiguration(PutPipelineData pipelineData, - ByteArray key) { + private AbstractConfigureNodes makeNodeConfigurationForPut(PutPipelineData pipelineData, + ByteArray key) { switch(obtainNodeConfigurationType(pipelineData.getZonesRequired())) { case DEFAULT: return new ConfigureNodesDefault(pipelineData, @@ -744,8 +739,8 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) // Get the correct type of configure nodes action depending on the store // requirements - AbstractConfigureNodes configureNodes = putNodeConfiguration(pipelineData, - key); + AbstractConfigureNodes configureNodes = makeNodeConfigurationForPut(pipelineData, + key); if(isHintedHandoffEnabled()) hintedHandoff = new HintedHandoff(failureDetector, diff --git a/src/java/voldemort/store/routed/action/ConfigureNodesByZone.java b/src/java/voldemort/store/routed/action/ConfigureNodesByZone.java index 7a194f649b..6f1d15a1f0 100644 --- a/src/java/voldemort/store/routed/action/ConfigureNodesByZone.java +++ b/src/java/voldemort/store/routed/action/ConfigureNodesByZone.java @@ -34,7 +34,7 @@ import voldemort.utils.ByteArray; import voldemort.utils.ByteUtils; -/* +/** * Configure the Nodes obtained via the routing strategy based on the zone * information. Local zone nodes first, followed by the corresponding nodes from * each of the other zones, ordered by proximity. diff --git a/src/java/voldemort/store/routed/action/ConfigureNodesDefault.java b/src/java/voldemort/store/routed/action/ConfigureNodesDefault.java index b05379d74f..0df936339b 100644 --- a/src/java/voldemort/store/routed/action/ConfigureNodesDefault.java +++ b/src/java/voldemort/store/routed/action/ConfigureNodesDefault.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2009 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.store.routed.action; import java.util.List; @@ -12,7 +28,7 @@ import voldemort.utils.ByteArray; import voldemort.utils.ByteUtils; -/* +/** * Default Configure Nodes that does not reorder the list of nodes obtained via * the routing strategy */ diff --git a/src/java/voldemort/store/routed/action/ConfigureNodesLocalHost.java b/src/java/voldemort/store/routed/action/ConfigureNodesLocalHost.java index 443a538b25..f3c88b962b 100644 --- a/src/java/voldemort/store/routed/action/ConfigureNodesLocalHost.java +++ b/src/java/voldemort/store/routed/action/ConfigureNodesLocalHost.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2009 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.store.routed.action; import java.net.InetAddress; @@ -15,7 +31,7 @@ import voldemort.store.routed.Pipeline.Event; import voldemort.utils.ByteArray; -/* +/** * Use the default node list returned via the routing strategy. However give * preference to the current node, if it is part of the preflist returned from * the routing strategy. diff --git a/src/java/voldemort/store/routed/action/ConfigureNodesLocalHostByZone.java b/src/java/voldemort/store/routed/action/ConfigureNodesLocalHostByZone.java index ceb5e688eb..8d34795f31 100644 --- a/src/java/voldemort/store/routed/action/ConfigureNodesLocalHostByZone.java +++ b/src/java/voldemort/store/routed/action/ConfigureNodesLocalHostByZone.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2009 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.store.routed.action; import java.net.InetAddress; @@ -15,7 +31,7 @@ import voldemort.store.routed.Pipeline.Operation; import voldemort.utils.ByteArray; -/* +/** * Use the zone aware node list returned via the routing strategy. However give * preference to the current node, if it is part of the preflist returned from * the routing strategy. diff --git a/src/java/voldemort/store/system/SystemStoreConstants.java b/src/java/voldemort/store/system/SystemStoreConstants.java index f46cc779c2..0013b47d7a 100644 --- a/src/java/voldemort/store/system/SystemStoreConstants.java +++ b/src/java/voldemort/store/system/SystemStoreConstants.java @@ -34,7 +34,6 @@ public class SystemStoreConstants { public static enum SystemStoreName { voldsys$_client_registry, - voldsys$_metadata_version, voldsys$_metadata_version_persistence; } @@ -61,28 +60,11 @@ public static enum SystemStoreName { + " 7" + " " - + " " - + " voldsys$_metadata_version" - + " local-pref-all-routing" - + " proximity-handoff" - + " memory" - + " client" - + " 1" - + " 1" - + " 1" - + " " - + " string" - + " " - + " " - + " string" - + " " - + " " - + " " + " voldsys$_metadata_version_persistence" + " local-pref-all-routing" + " proximity-handoff" - + " file" + + " file-backed-cache" + " client" + " 1" + " 1" diff --git a/src/java/voldemort/utils/MetadataVersionStoreUtils.java b/src/java/voldemort/utils/MetadataVersionStoreUtils.java index dc6d917052..e320692d27 100644 --- a/src/java/voldemort/utils/MetadataVersionStoreUtils.java +++ b/src/java/voldemort/utils/MetadataVersionStoreUtils.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2012 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.utils; import java.io.ByteArrayInputStream; @@ -7,6 +23,13 @@ import voldemort.client.SystemStore; +/** + * A Utils class that facilitates conversion between the string containing + * metadata versions and the corresponding Properties object. + * + * @author csoman + * + */ public class MetadataVersionStoreUtils { public static final String VERSIONS_METADATA_KEY = "metadata-versions"; diff --git a/test/unit/voldemort/client/ClientJmxTest.java b/test/unit/voldemort/client/ClientJmxTest.java index 1e5529c09e..57f65d22c5 100644 --- a/test/unit/voldemort/client/ClientJmxTest.java +++ b/test/unit/voldemort/client/ClientJmxTest.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2012 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.client; import java.lang.management.ManagementFactory; diff --git a/test/unit/voldemort/client/ClientRegistryTest.java b/test/unit/voldemort/client/ClientRegistryTest.java index 7ab2c479e1..87cb8b8512 100644 --- a/test/unit/voldemort/client/ClientRegistryTest.java +++ b/test/unit/voldemort/client/ClientRegistryTest.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2012 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.client; import java.io.ByteArrayInputStream; @@ -39,7 +55,7 @@ public class ClientRegistryTest extends TestCase { public static final String STORES_XML_FILE = "test/common/voldemort/config/stores.xml"; public static final String CLIENT_CONTEXT_NAME = "testClientRegistryHappyPath"; public static final String CLIENT_CONTEXT_NAME2 = "testClientRegistryUnhappyPath"; - public static final int CLIENT_REGISTRY_REFRSH_INTERVAL = 1; + public static final int CLIENT_REGISTRY_REFRESH_INTERVAL = 1; public static final int TOTAL_SERVERS = 2; private SocketStoreFactory socketStoreFactory = new ClientRequestExecutorPool(TOTAL_SERVERS, @@ -87,6 +103,10 @@ public void tearDown() throws Exception { this.clearRegistryContent(); } + /* + * Tests that the client registry is populated correctly and that we can + * query using Admin Tool. + */ @Test public void testHappyPath() { List emptyPartitionList = Lists.newArrayList(); @@ -96,7 +116,7 @@ public void testHappyPath() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[0]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateIntervalInSecs(CLIENT_REGISTRY_REFRESH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); StoreClient client1 = socketFactory.getStoreClient(TEST_STORE_NAME); @@ -131,7 +151,7 @@ public void testHappyPath() { assertEquals(1, infoList.size()); try { - Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 1000 * 5); + Thread.sleep(CLIENT_REGISTRY_REFRESH_INTERVAL * 1000 * 5); } catch(InterruptedException e) {} // now the periodical update has gone through, it shall be higher than // the bootstrap time @@ -149,6 +169,10 @@ public void testHappyPath() { socketFactory.close(); } + /* + * Test happy path for 2 clients created by the same factory, pointing to + * the same store + */ @Test public void testTwoClients() { List emptyPartitionList = Lists.newArrayList(); @@ -158,7 +182,7 @@ public void testTwoClients() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[0]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateIntervalInSecs(CLIENT_REGISTRY_REFRESH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); StoreClient client1 = socketFactory.getStoreClient(TEST_STORE_NAME); @@ -215,7 +239,7 @@ public void testTwoClients() { assertEquals(infoList.size(), 2); try { - Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 1000 * 5); + Thread.sleep(CLIENT_REGISTRY_REFRESH_INTERVAL * 1000 * 5); } catch(InterruptedException e) {} // now the periodical update has gone through, it shall be higher than // the bootstrap time @@ -233,6 +257,10 @@ public void testTwoClients() { socketFactory.close(); } + /* + * Tests client registry for 2 clients created using the same factory, + * pointing to different stores + */ @Test public void testTwoStores() { List emptyPartitionList = Lists.newArrayList(); @@ -242,7 +270,7 @@ public void testTwoStores() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[0]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateIntervalInSecs(CLIENT_REGISTRY_REFRESH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory = new SocketStoreClientFactory(clientConfig); StoreClient client1 = socketFactory.getStoreClient(TEST_STORE_NAME); @@ -316,7 +344,7 @@ public void testTwoStores() { } try { - Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 1000 * 5); + Thread.sleep(CLIENT_REGISTRY_REFRESH_INTERVAL * 1000 * 5); } catch(InterruptedException e) {} // now the periodical update has gone through, it shall be higher than // the bootstrap time @@ -334,6 +362,10 @@ public void testTwoStores() { socketFactory.close(); } + /* + * Tests client registry for 2 clients created using the different + * factories, pointing to different stores + */ @Test public void testTwoFactories() { List emptyPartitionList = Lists.newArrayList(); @@ -343,7 +375,7 @@ public void testTwoFactories() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[0]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateIntervalInSecs(CLIENT_REGISTRY_REFRESH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory1 = new SocketStoreClientFactory(clientConfig); @@ -353,7 +385,7 @@ public void testTwoFactories() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[0]) .setClientContextName(CLIENT_CONTEXT_NAME2) - .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateIntervalInSecs(CLIENT_REGISTRY_REFRESH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory2 = new SocketStoreClientFactory(clientConfig2); @@ -448,7 +480,7 @@ public void testTwoFactories() { } try { - Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 1000 * 5); + Thread.sleep(CLIENT_REGISTRY_REFRESH_INTERVAL * 1000 * 5); } catch(InterruptedException e) {} // now the periodical update has gone through, it shall be higher than // the bootstrap time @@ -467,6 +499,9 @@ public void testTwoFactories() { socketFactory2.close(); } + /* + * Tests client registry for in the presence of 1 server failure. + */ @Test public void testOneServerFailure() { // bring down one server before starting up the clients @@ -479,7 +514,7 @@ public void testOneServerFailure() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[1]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateIntervalInSecs(CLIENT_REGISTRY_REFRESH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory1 = new SocketStoreClientFactory(clientConfig); @@ -489,7 +524,7 @@ public void testOneServerFailure() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[1]) .setClientContextName(CLIENT_CONTEXT_NAME2) - .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateIntervalInSecs(CLIENT_REGISTRY_REFRESH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory2 = new SocketStoreClientFactory(clientConfig2); @@ -542,7 +577,7 @@ public void testOneServerFailure() { } try { - Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 1000 * 5); + Thread.sleep(CLIENT_REGISTRY_REFRESH_INTERVAL * 1000 * 5); } catch(InterruptedException e) {} // now the periodical update has gone through, it shall be higher than // the bootstrap time @@ -561,6 +596,9 @@ public void testOneServerFailure() { socketFactory2.close(); } + /* + * Test repeated client-registry setup due to client bounce. + */ @Test public void testRepeatRegistrationSameFactory() { @@ -571,7 +609,7 @@ public void testRepeatRegistrationSameFactory() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[1]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateIntervalInSecs(CLIENT_REGISTRY_REFRESH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory1 = new SocketStoreClientFactory(clientConfig); @@ -581,7 +619,7 @@ public void testRepeatRegistrationSameFactory() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[1]) .setClientContextName(CLIENT_CONTEXT_NAME2) - .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateIntervalInSecs(CLIENT_REGISTRY_REFRESH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory2 = new SocketStoreClientFactory(clientConfig2); @@ -607,6 +645,10 @@ public void testRepeatRegistrationSameFactory() { socketFactory2.close(); } + /* + * Test repeated client-registry setup due to client bounce and via a + * different factory. + */ @Test public void testRepeatRegistrationDifferentFactories() { long client1LastBootstrapTime = 0; @@ -620,7 +662,7 @@ public void testRepeatRegistrationDifferentFactories() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[1]) .setClientContextName(CLIENT_CONTEXT_NAME) - .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateIntervalInSecs(CLIENT_REGISTRY_REFRESH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory1 = new SocketStoreClientFactory(clientConfig); @@ -630,7 +672,7 @@ public void testRepeatRegistrationDifferentFactories() { .setBootstrapUrls(SERVER_LOCAL_URL + serverPorts[1]) .setClientContextName(CLIENT_CONTEXT_NAME2) - .setClientRegistryUpdateInSecs(CLIENT_REGISTRY_REFRSH_INTERVAL) + .setClientRegistryUpdateIntervalInSecs(CLIENT_REGISTRY_REFRESH_INTERVAL) .setEnableLazy(false); SocketStoreClientFactory socketFactory2 = new SocketStoreClientFactory(clientConfig2); @@ -685,7 +727,7 @@ public void testRepeatRegistrationDifferentFactories() { } try { - Thread.sleep(CLIENT_REGISTRY_REFRSH_INTERVAL * 1000 * 5); + Thread.sleep(CLIENT_REGISTRY_REFRESH_INTERVAL * 1000 * 5); } catch(InterruptedException e) {} // now the periodical update has gone through, it shall be higher // than diff --git a/test/unit/voldemort/client/EndToEndRebootstrapTest.java b/test/unit/voldemort/client/EndToEndRebootstrapTest.java index 2202bfb41e..a2ee510627 100644 --- a/test/unit/voldemort/client/EndToEndRebootstrapTest.java +++ b/test/unit/voldemort/client/EndToEndRebootstrapTest.java @@ -1,10 +1,30 @@ +/* + * Copyright 2008-2009 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.client; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + import java.io.ByteArrayInputStream; import java.util.Properties; -import junit.framework.TestCase; - import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -25,7 +45,14 @@ import voldemort.utils.SystemTime; import voldemort.xml.ClusterMapper; -public class EndToEndRebootstrapTest extends TestCase { +/** + * Test class to verify that the Zenstore client rebootstraps when needed (on + * change of cluster.xml) + * + * @author csoman + * + */ +public class EndToEndRebootstrapTest { private static final String STORE_NAME = "test-replication-persistent"; private static final String CLUSTER_KEY = "cluster.xml"; @@ -44,7 +71,6 @@ public class EndToEndRebootstrapTest extends TestCase { public static String socketUrl = ""; protected final int CLIENT_ZONE_ID = 0; - @Override @Before public void setUp() throws Exception { cluster = ServerTestUtils.getLocalCluster(2, new int[][] { { 0, 1, 2, 3 }, { 4, 5, 6, 7 } }); @@ -75,7 +101,7 @@ public void setUp() throws Exception { Node node = cluster.getNodeById(0); String bootstrapUrl = "tcp://" + node.getHost() + ":" + node.getSocketPort(); ClientConfig clientConfig = new ClientConfig(); - clientConfig.setClientRegistryUpdateInSecs(5); + clientConfig.setClientRegistryUpdateIntervalInSecs(5); clientConfig.setAsyncMetadataRefreshInMs(5000); clientConfig.setBootstrapUrls(bootstrapUrl); SocketStoreClientFactory storeClientFactory = new SocketStoreClientFactory(clientConfig); @@ -100,13 +126,22 @@ public void setUp() throws Exception { } - @Override @After public void tearDown() throws Exception { servers[0].stop(); servers[1].stop(); } + /* + * Test to validate that the client bootstraps on metadata change. First do + * some operations to validate that the client is correctly initialized. + * Then update the cluster.xml using the Admin Tool (which should update the + * metadata version as well). Verify that the client bootstraps after this + * update. + * + * Whether the client has automatically bootstrapped is verified by checking + * the new bootstrap time in the client registry. + */ @Test public void testEndToEndRebootstrap() { try { diff --git a/test/unit/voldemort/store/configuration/FileBackedCachingStorageEngineTest.java b/test/unit/voldemort/store/configuration/FileBackedCachingStorageEngineTest.java index 46186d28fa..27eff256dc 100644 --- a/test/unit/voldemort/store/configuration/FileBackedCachingStorageEngineTest.java +++ b/test/unit/voldemort/store/configuration/FileBackedCachingStorageEngineTest.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2009 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.store.configuration; import static voldemort.TestUtils.getClock; @@ -19,6 +35,12 @@ import voldemort.versioning.VectorClock; import voldemort.versioning.Versioned; +/** + * A testclass for verifying the FileBackedCachingStorageEngine + * + * @author csoman + * + */ public class FileBackedCachingStorageEngineTest extends AbstractStoreTest { diff --git a/test/unit/voldemort/store/routed/action/ConfigureNodesLocalHostTest.java b/test/unit/voldemort/store/routed/action/ConfigureNodesLocalHostTest.java index 97ceff026b..7e3eb11157 100644 --- a/test/unit/voldemort/store/routed/action/ConfigureNodesLocalHostTest.java +++ b/test/unit/voldemort/store/routed/action/ConfigureNodesLocalHostTest.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2009 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.store.routed.action; import static org.junit.Assert.assertEquals; @@ -26,6 +42,12 @@ import com.google.common.collect.ImmutableList; +/** + * Test class to verify the ConfigureNodesLocalHost strategy + * + * @author csoman + * + */ public class ConfigureNodesLocalHostTest { protected final ByteArray aKey = TestUtils.toByteArray("vold"); @@ -35,7 +57,6 @@ private List getTestNodes() { try { currentHost = InetAddress.getLocalHost().getHostName(); } catch(UnknownHostException e) { - // TODO Auto-generated catch block e.printStackTrace(); } return ImmutableList.of(node(0, "some-node-1", 2, 7, 14), @@ -53,6 +74,10 @@ private Node node(int id, String hostName, int... tags) { return new Node(id, hostName, 8080, 6666, 6667, list); } + /* + * Checks to see that the local host is obtained as the first node in the + * list returned by ConfigureNodesLocalHost + */ @Test public void testConfigureNodesLocalHost() throws Exception { List nodes = getTestNodes(); diff --git a/test/unit/voldemort/store/system/AsyncMetadataVersionManagerTest.java b/test/unit/voldemort/store/system/AsyncMetadataVersionManagerTest.java index c28f9f518e..1972254411 100644 --- a/test/unit/voldemort/store/system/AsyncMetadataVersionManagerTest.java +++ b/test/unit/voldemort/store/system/AsyncMetadataVersionManagerTest.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2009 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.store.system; import static org.junit.Assert.assertEquals; @@ -23,6 +39,12 @@ import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; import voldemort.utils.SystemTime; +/** + * Test class to verify the AsyncMetadataVersionManager + * + * @author csoman + * + */ public class AsyncMetadataVersionManagerTest { private static String storesXmlfile = "test/common/voldemort/config/stores.xml"; @@ -88,6 +110,13 @@ public void tearDown() throws Exception { servers[1].stop(); } + /* + * Validates that the AsyncMetadataVersionManager correctly identifies the + * version update. This is done by initializing the base metadata version + * (for cluster.xml), starting the AsyncMetadataVersionManager and then + * updating the version to a new value. For the test to succeed the callback + * has to be invoked correctly by the asynchronous manager. + */ @Test public void testBasicAsyncBehaviour() { String storeVersionKey = "cluster.xml"; diff --git a/test/unit/voldemort/store/system/SystemStoreTest.java b/test/unit/voldemort/store/system/SystemStoreTest.java index 4f6b12ffbf..a2ca3eef0b 100644 --- a/test/unit/voldemort/store/system/SystemStoreTest.java +++ b/test/unit/voldemort/store/system/SystemStoreTest.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2009 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + package voldemort.store.system; import static org.junit.Assert.assertEquals; @@ -21,6 +37,13 @@ import voldemort.store.socket.SocketStoreFactory; import voldemort.store.socket.clientrequest.ClientRequestExecutorPool; +/** + * Test class to verify the SystemStore (used to interact with the system + * metadata stores managed by the cluster). + * + * @author csoman + * + */ public class SystemStoreTest { private static String storesXmlfile = "test/common/voldemort/config/stores.xml"; From 5b70b50599dc345c325e739aebbc1346a8a6d369 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Fri, 21 Sep 2012 14:50:14 -0700 Subject: [PATCH 136/209] Updating release notes for 0.96 open source release --- release_notes.txt | 54 ++++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 53 insertions(+), 1 deletion(-) diff --git a/release_notes.txt b/release_notes.txt index 62c1867d70..d0562eefef 100644 --- a/release_notes.txt +++ b/release_notes.txt @@ -1,4 +1,56 @@ -Release 0.90.1 on 10/10/2011 +Release 0.96 on 09/05/2012 + +Changes made since 0.90.1 + + * Monitoring: + * Append cluster name to various mbeans for better stats display + * Implement average throughput in bytes + * Add BDB JE stats + * Add 95th and 99th latency tracking + * Add stats for ClientRequestExecutorPool + * Add error/exception count and max getall count + * BDB+ Data cleanup Monitoring changes + * Rebalancing: + * Donor-based rebalancing and post cleanup (see https://github.com/voldemort/voldemort/wiki/Voldemort-Donor-Based-Rebalancing for more details) + * Rebalancing integration testing framework (under test/integration/voldemort/rebalance/) + * Generate multiple cluster.xml files based on the number specified when running the tool and choose the cluster with the smallest std dev as the final-cluster.xml + * Add status output to log for updateEntries (used by rebalancing) + * Read-only pipeline: + * Add hftp and webhdfs support + * Read-only bandwidth dynamic throttler + * Add minimum throttle limit per store + * Add rollback capability to the Admin tool + * Voldemort-backed stack and index linked list impl + * Change client requests to not process responses after timeout + * Modified client request executor timeout to not factor in the NIO selector timeout + * Added BDB native backup capabalities, checksum verification and incremental backups (well tested, but not yet used in production) + * Add additional client-side tracing for debugging and consistency analytics + * Clean up logging during exception at client-side + * Security exception handling + * Add snappy to CompressionStrategyFactory + * Add configurable option to interrupt service being unscheduled + * Add logging support for tracking ScanPermit owners (for debugging purposes) + * Add a jmx terminate operation for async jobs + * Add zone option for restore from replicas + * Changing the enable.nio.connector to true by default + * Better disconnection handling for python client + * Split junit tests into a long and a short test suites + * Add separate timeouts for different operations (put, get, delete, and getAll + * Allow getAll to return partial results upon timeout + * Improved cluster generation tool + * Added log4j properties folder for junit test + * Bug fixes: + * httpclient 3.x to httpclient 4.x + * Fix NPE in listing read-only store versions + * Fixed 2 failure detector bugs during rebalancing or node swapping + * Fixed a thread leak issue in StreamingSlopPusher + * Fixed a NIO bug + * Fixed a bug in TimeBasedInconsistency resolver. + * Fixed race condition in client socket close + * Fixed a potential deadlock issue in ScanPermitWrapper + * Fixed a bug where a read returns null (on rare occations) when being concurrent with a write + * Fixed a performance bug in HdfsFetcher when hftp is used + Changes made since 0.90 From c13fbeb5efbcf40dd1132d9861519ea0c33e493a Mon Sep 17 00:00:00 2001 From: Abhinay Nagpal Date: Fri, 31 Aug 2012 10:24:26 -0700 Subject: [PATCH 137/209] Avro build and push support removes azkaban dependancies and refactors out all classes from batch commons --- .classpath | 6 ++++-- .settings/org.eclipse.jdt.core.prefs | 8 ++++---- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/.classpath b/.classpath index 1c153f728f..c74248be75 100644 --- a/.classpath +++ b/.classpath @@ -51,8 +51,10 @@ - - + + + + diff --git a/.settings/org.eclipse.jdt.core.prefs b/.settings/org.eclipse.jdt.core.prefs index cbe13973a0..8d8bb87867 100644 --- a/.settings/org.eclipse.jdt.core.prefs +++ b/.settings/org.eclipse.jdt.core.prefs @@ -1,4 +1,4 @@ -#Fri Dec 30 14:37:10 PST 2011 +#Thu Aug 30 10:43:57 PDT 2012 eclipse.preferences.version=1 org.eclipse.jdt.core.codeComplete.argumentPrefixes= org.eclipse.jdt.core.codeComplete.argumentSuffixes= @@ -11,9 +11,9 @@ org.eclipse.jdt.core.codeComplete.staticFieldSuffixes= org.eclipse.jdt.core.codeComplete.staticFinalFieldPrefixes= org.eclipse.jdt.core.codeComplete.staticFinalFieldSuffixes= org.eclipse.jdt.core.compiler.codegen.inlineJsrBytecode=enabled -org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.5 +org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.6 org.eclipse.jdt.core.compiler.codegen.unusedLocal=preserve -org.eclipse.jdt.core.compiler.compliance=1.5 +org.eclipse.jdt.core.compiler.compliance=1.6 org.eclipse.jdt.core.compiler.debug.lineNumber=generate org.eclipse.jdt.core.compiler.debug.localVariable=generate org.eclipse.jdt.core.compiler.debug.sourceFile=generate @@ -77,7 +77,7 @@ org.eclipse.jdt.core.compiler.problem.unusedParameterWhenOverridingConcrete=disa org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=warning org.eclipse.jdt.core.compiler.problem.unusedWarningToken=warning org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=warning -org.eclipse.jdt.core.compiler.source=1.5 +org.eclipse.jdt.core.compiler.source=1.6 org.eclipse.jdt.core.formatter.align_type_members_on_columns=false org.eclipse.jdt.core.formatter.alignment_for_arguments_in_allocation_expression=82 org.eclipse.jdt.core.formatter.alignment_for_arguments_in_enum_constant=82 From 9a78082336c2b687682c07333418916d192028a7 Mon Sep 17 00:00:00 2001 From: Abhinay Nagpal Date: Wed, 5 Sep 2012 10:19:02 -0700 Subject: [PATCH 138/209] Refactoring reducer logic --- .classpath | 1 + .../store/readonly/mr/HadoopStoreBuilder.java | 239 +++++++++++++++- .../mr/HadoopStoreBuilderReducer.java | 255 ++--------------- .../HadoopStoreBuilderReducerPerBucket.java | 265 ++---------------- 4 files changed, 265 insertions(+), 495 deletions(-) diff --git a/.classpath b/.classpath index c74248be75..9daadee954 100644 --- a/.classpath +++ b/.classpath @@ -56,5 +56,6 @@ + diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/HadoopStoreBuilder.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/HadoopStoreBuilder.java index 22b1f711ba..fd7bde983f 100644 --- a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/HadoopStoreBuilder.java +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/HadoopStoreBuilder.java @@ -17,10 +17,15 @@ package voldemort.store.readonly.mr; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; +import org.apache.avro.Schema; +import org.apache.avro.mapred.AvroJob; +import org.apache.avro.mapred.AvroOutputFormat; +import org.apache.avro.mapred.Pair; import org.apache.commons.codec.binary.Hex; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -36,6 +41,7 @@ import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.mapred.JobClient; import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.OutputFormat; import org.apache.hadoop.mapred.RunningJob; import org.apache.hadoop.mapred.SequenceFileOutputFormat; import org.apache.log4j.Logger; @@ -48,6 +54,7 @@ import voldemort.store.readonly.ReadOnlyStorageMetadata; import voldemort.store.readonly.checksum.CheckSum; import voldemort.store.readonly.checksum.CheckSum.CheckSumType; +import voldemort.store.readonly.disk.KeyValueWriter; import voldemort.utils.Utils; import voldemort.xml.ClusterMapper; import voldemort.xml.StoreDefinitionsMapper; @@ -66,7 +73,7 @@ public class HadoopStoreBuilder { private static final Logger logger = Logger.getLogger(HadoopStoreBuilder.class); private final Configuration config; - private final Class> mapperClass; + private final Class mapperClass; @SuppressWarnings("unchecked") private final Class inputFormatClass; private final Cluster cluster; @@ -99,7 +106,7 @@ public class HadoopStoreBuilder { @SuppressWarnings("unchecked") @Deprecated public HadoopStoreBuilder(Configuration conf, - Class> mapperClass, + Class mapperClass, Class inputFormatClass, Cluster cluster, StoreDefinition storeDef, @@ -135,7 +142,7 @@ public HadoopStoreBuilder(Configuration conf, */ @SuppressWarnings("unchecked") public HadoopStoreBuilder(Configuration conf, - Class> mapperClass, + Class mapperClass, Class inputFormatClass, Cluster cluster, StoreDefinition storeDef, @@ -175,7 +182,7 @@ public HadoopStoreBuilder(Configuration conf, */ @SuppressWarnings("unchecked") public HadoopStoreBuilder(Configuration conf, - Class> mapperClass, + Class mapperClass, Class inputFormatClass, Cluster cluster, StoreDefinition storeDef, @@ -218,7 +225,7 @@ public HadoopStoreBuilder(Configuration conf, */ @SuppressWarnings("unchecked") public HadoopStoreBuilder(Configuration conf, - Class> mapperClass, + Class mapperClass, Class inputFormatClass, Cluster cluster, StoreDefinition storeDef, @@ -265,7 +272,7 @@ public HadoopStoreBuilder(Configuration conf, */ @SuppressWarnings("unchecked") public HadoopStoreBuilder(Configuration conf, - Class> mapperClass, + Class mapperClass, Class inputFormatClass, Cluster cluster, StoreDefinition storeDef, @@ -391,14 +398,14 @@ public void build() { if(saveKeys) { if(reducerPerBucket) { logger.info("Number of collisions in the job - " - + counters.getCounter(HadoopStoreBuilderReducerPerBucket.CollisionCounter.NUM_COLLISIONS)); + + counters.getCounter(KeyValueWriter.CollisionCounter.NUM_COLLISIONS)); logger.info("Maximum number of collisions for one entry - " - + counters.getCounter(HadoopStoreBuilderReducerPerBucket.CollisionCounter.MAX_COLLISIONS)); + + counters.getCounter(KeyValueWriter.CollisionCounter.MAX_COLLISIONS)); } else { logger.info("Number of collisions in the job - " - + counters.getCounter(HadoopStoreBuilderReducer.CollisionCounter.NUM_COLLISIONS)); + + counters.getCounter(KeyValueWriter.CollisionCounter.NUM_COLLISIONS)); logger.info("Maximum number of collisions for one entry - " - + counters.getCounter(HadoopStoreBuilderReducer.CollisionCounter.MAX_COLLISIONS)); + + counters.getCounter(KeyValueWriter.CollisionCounter.MAX_COLLISIONS)); } } @@ -490,6 +497,218 @@ public boolean accept(Path arg0) { } + /** + * Run the job + */ + public void buildAvro() { + try { + JobConf conf = new JobConf(config); + conf.setInt("io.file.buffer.size", DEFAULT_BUFFER_SIZE); + conf.set("cluster.xml", new ClusterMapper().writeCluster(cluster)); + conf.set("stores.xml", + new StoreDefinitionsMapper().writeStoreList(Collections.singletonList(storeDef))); + conf.setBoolean("save.keys", saveKeys); + conf.setBoolean("reducer.per.bucket", reducerPerBucket); + conf.setPartitionerClass(AvroStoreBuilderPartitioner.class); + // conf.setMapperClass(mapperClass); + conf.setMapOutputKeyClass(ByteBuffer.class); + conf.setMapOutputValueClass(ByteBuffer.class); + + conf.setInputFormat(inputFormatClass); + + conf.setOutputFormat((Class) AvroOutputFormat.class); + conf.setOutputKeyClass(ByteBuffer.class); + conf.setOutputValueClass(ByteBuffer.class); + conf.setJarByClass(getClass()); + conf.setReduceSpeculativeExecution(false); + FileInputFormat.setInputPaths(conf, inputPath); + conf.set("final.output.dir", outputDir.toString()); + conf.set("checksum.type", CheckSum.toString(checkSumType)); + FileOutputFormat.setOutputPath(conf, tempDir); + + FileSystem outputFs = outputDir.getFileSystem(conf); + if(outputFs.exists(outputDir)) { + throw new IOException("Final output directory already exists."); + } + + // delete output dir if it already exists + FileSystem tempFs = tempDir.getFileSystem(conf); + tempFs.delete(tempDir, true); + + long size = sizeOfPath(tempFs, inputPath); + logger.info("Data size = " + size + ", replication factor = " + + storeDef.getReplicationFactor() + ", numNodes = " + + cluster.getNumberOfNodes() + ", chunk size = " + chunkSizeBytes); + + // Derive "rough" number of chunks and reducers + int numReducers; + if(saveKeys) { + + if(this.numChunks == -1) { + this.numChunks = Math.max((int) (storeDef.getReplicationFactor() * size + / cluster.getNumberOfPartitions() + / storeDef.getReplicationFactor() / chunkSizeBytes), + 1); + } else { + logger.info("Overriding chunk size byte and taking num chunks (" + + this.numChunks + ") directly"); + } + + if(reducerPerBucket) { + numReducers = cluster.getNumberOfPartitions() * storeDef.getReplicationFactor(); + } else { + numReducers = cluster.getNumberOfPartitions() * storeDef.getReplicationFactor() + * numChunks; + } + } else { + + if(this.numChunks == -1) { + this.numChunks = Math.max((int) (storeDef.getReplicationFactor() * size + / cluster.getNumberOfPartitions() / chunkSizeBytes), + 1); + } else { + logger.info("Overriding chunk size byte and taking num chunks (" + + this.numChunks + ") directly"); + } + + if(reducerPerBucket) { + numReducers = cluster.getNumberOfPartitions(); + } else { + numReducers = cluster.getNumberOfPartitions() * numChunks; + } + } + conf.setInt("num.chunks", numChunks); + conf.setNumReduceTasks(numReducers); + + conf.setSpeculativeExecution(false); + + System.out.println(config.get("avro.rec.schema")); + AvroJob.setInputSchema(conf, Schema.parse(config.get("avro.rec.schema"))); + + AvroJob.setOutputSchema(conf, + Pair.getPairSchema(Schema.create(Schema.Type.BYTES), + Schema.create(Schema.Type.BYTES))); + + AvroJob.setMapperClass(conf, mapperClass); + + if(reducerPerBucket) { + conf.setReducerClass(AvroStoreBuilderReducerPerBucket.class); + } else { + conf.setReducerClass(AvroStoreBuilderReducer.class); + } + + logger.info("Number of chunks: " + numChunks + ", number of reducers: " + numReducers + + ", save keys: " + saveKeys + ", reducerPerBucket: " + reducerPerBucket); + logger.info("Building store..."); + + RunningJob job = JobClient.runJob(conf); + + // Once the job has completed log the counter + Counters counters = job.getCounters(); + + if(saveKeys) { + if(reducerPerBucket) { + logger.info("Number of collisions in the job - " + + counters.getCounter(KeyValueWriter.CollisionCounter.NUM_COLLISIONS)); + logger.info("Maximum number of collisions for one entry - " + + counters.getCounter(KeyValueWriter.CollisionCounter.MAX_COLLISIONS)); + } else { + logger.info("Number of collisions in the job - " + + counters.getCounter(KeyValueWriter.CollisionCounter.NUM_COLLISIONS)); + logger.info("Maximum number of collisions for one entry - " + + counters.getCounter(KeyValueWriter.CollisionCounter.MAX_COLLISIONS)); + } + } + + // Do a CheckSumOfCheckSum - Similar to HDFS + CheckSum checkSumGenerator = CheckSum.getInstance(this.checkSumType); + if(!this.checkSumType.equals(CheckSumType.NONE) && checkSumGenerator == null) { + throw new VoldemortException("Could not generate checksum digest for type " + + this.checkSumType); + } + + // Check if all folder exists and with format file + for(Node node: cluster.getNodes()) { + + ReadOnlyStorageMetadata metadata = new ReadOnlyStorageMetadata(); + + if(saveKeys) { + metadata.add(ReadOnlyStorageMetadata.FORMAT, + ReadOnlyStorageFormat.READONLY_V2.getCode()); + } else { + metadata.add(ReadOnlyStorageMetadata.FORMAT, + ReadOnlyStorageFormat.READONLY_V1.getCode()); + } + + Path nodePath = new Path(outputDir.toString(), "node-" + node.getId()); + + if(!outputFs.exists(nodePath)) { + logger.info("No data generated for node " + node.getId() + + ". Generating empty folder"); + outputFs.mkdirs(nodePath); // Create empty folder + } + + if(checkSumType != CheckSumType.NONE) { + + FileStatus[] storeFiles = outputFs.listStatus(nodePath, new PathFilter() { + + @Override + public boolean accept(Path arg0) { + if(arg0.getName().endsWith("checksum") + && !arg0.getName().startsWith(".")) { + return true; + } + return false; + } + }); + + if(storeFiles != null && storeFiles.length > 0) { + Arrays.sort(storeFiles, new IndexFileLastComparator()); + FSDataInputStream input = null; + + for(FileStatus file: storeFiles) { + try { + input = outputFs.open(file.getPath()); + byte fileCheckSum[] = new byte[CheckSum.checkSumLength(this.checkSumType)]; + input.read(fileCheckSum); + logger.debug("Checksum for file " + file.toString() + " - " + + new String(Hex.encodeHex(fileCheckSum))); + checkSumGenerator.update(fileCheckSum); + } catch(Exception e) { + logger.error("Error while reading checksum file " + e.getMessage(), + e); + } finally { + if(input != null) + input.close(); + } + outputFs.delete(file.getPath(), false); + } + + metadata.add(ReadOnlyStorageMetadata.CHECKSUM_TYPE, + CheckSum.toString(checkSumType)); + + String checkSum = new String(Hex.encodeHex(checkSumGenerator.getCheckSum())); + logger.info("Checksum for node " + node.getId() + " - " + checkSum); + + metadata.add(ReadOnlyStorageMetadata.CHECKSUM, checkSum); + } + } + + // Write metadata + FSDataOutputStream metadataStream = outputFs.create(new Path(nodePath, ".metadata")); + metadataStream.write(metadata.toJsonString().getBytes()); + metadataStream.flush(); + metadataStream.close(); + + } + + } catch(Exception e) { + logger.error("Error in Store builder", e); + throw new VoldemortException(e); + } + + } + /** * A comparator that sorts index files last. This is required to maintain * the order while calculating checksum diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/HadoopStoreBuilderReducer.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/HadoopStoreBuilderReducer.java index 5e1cb8297f..0b3a496f0d 100644 --- a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/HadoopStoreBuilderReducer.java +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/HadoopStoreBuilderReducer.java @@ -16,64 +16,29 @@ package voldemort.store.readonly.mr; -import java.io.ByteArrayOutputStream; -import java.io.DataOutputStream; import java.io.IOException; import java.util.Iterator; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapred.FileOutputFormat; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reducer; import org.apache.hadoop.mapred.Reporter; -import org.apache.log4j.Logger; -import voldemort.VoldemortException; -import voldemort.store.readonly.ReadOnlyUtils; -import voldemort.store.readonly.checksum.CheckSum; -import voldemort.store.readonly.checksum.CheckSum.CheckSumType; -import voldemort.utils.ByteUtils; +import voldemort.store.readonly.disk.HadoopStoreWriter; +import voldemort.store.readonly.disk.KeyValueWriter; +import azkaban.common.utils.Utils; /** * Take key md5s and value bytes and build a read-only store from these values */ @SuppressWarnings("deprecation") -public class HadoopStoreBuilderReducer extends AbstractStoreBuilderConfigurable implements - Reducer { +public class HadoopStoreBuilderReducer implements Reducer { - private static final Logger logger = Logger.getLogger(HadoopStoreBuilderReducer.class); - - private DataOutputStream indexFileStream = null; - private DataOutputStream valueFileStream = null; - private int position; - private String taskId = null; - - private int nodeId = -1; - private int partitionId = -1; - private int chunkId = -1; - private int replicaType = -1; - - private Path taskIndexFileName; - private Path taskValueFileName; - - private JobConf conf; - private CheckSumType checkSumType; - private CheckSum checkSumDigestIndex; - private CheckSum checkSumDigestValue; - - private String outputDir; - - private FileSystem fs; - - protected static enum CollisionCounter { - NUM_COLLISIONS, - MAX_COLLISIONS; - } + String keyValueWriterClass; + @SuppressWarnings("rawtypes") + KeyValueWriter writer; /** * Reduce should get sorted MD5 of Voldemort key ( either 16 bytes if saving @@ -82,220 +47,36 @@ protected static enum CollisionCounter { * partition-id, replica-type, [key-size, value-size, key, value]* if saving * keys is enabled */ + @SuppressWarnings("unchecked") public void reduce(BytesWritable key, Iterator iterator, OutputCollector output, Reporter reporter) throws IOException { - // Write key and position - this.indexFileStream.write(key.get(), 0, key.getSize()); - this.indexFileStream.writeInt(this.position); - - // Run key through checksum digest - if(this.checkSumDigestIndex != null) { - this.checkSumDigestIndex.update(key.get(), 0, key.getSize()); - this.checkSumDigestIndex.update(this.position); - } - - short numTuples = 0; - ByteArrayOutputStream stream = new ByteArrayOutputStream(); - DataOutputStream valueStream = new DataOutputStream(stream); - - while(iterator.hasNext()) { - BytesWritable writable = iterator.next(); - byte[] valueBytes = writable.get(); - int offsetTillNow = 0; - - // Read node Id - if(this.nodeId == -1) - this.nodeId = ByteUtils.readInt(valueBytes, offsetTillNow); - offsetTillNow += ByteUtils.SIZE_OF_INT; - - // Read partition id - if(this.partitionId == -1) - this.partitionId = ByteUtils.readInt(valueBytes, offsetTillNow); - offsetTillNow += ByteUtils.SIZE_OF_INT; - - // Read chunk id - if(this.chunkId == -1) - this.chunkId = ReadOnlyUtils.chunk(key.get(), getNumChunks()); - - // Read replica type - if(getSaveKeys()) { - if(this.replicaType == -1) - this.replicaType = (int) ByteUtils.readBytes(valueBytes, - offsetTillNow, - ByteUtils.SIZE_OF_BYTE); - offsetTillNow += ByteUtils.SIZE_OF_BYTE; - } - - int valueLength = writable.getSize() - offsetTillNow; - if(getSaveKeys()) { - // Write ( key_length, value_length, key, - // value ) - valueStream.write(valueBytes, offsetTillNow, valueLength); - } else { - // Write (value_length + value) - valueStream.writeInt(valueLength); - valueStream.write(valueBytes, offsetTillNow, valueLength); - } - - numTuples++; - - // If we have multiple values for this md5 that is a collision, - // throw an exception--either the data itself has duplicates, there - // are trillions of keys, or someone is attempting something - // malicious ( We obviously expect collisions when we save keys ) - if(!getSaveKeys() && numTuples > 1) - throw new VoldemortException("Duplicate keys detected for md5 sum " - + ByteUtils.toHexString(ByteUtils.copy(key.get(), - 0, - key.getSize()))); - - } - - if(numTuples < 0) { - // Overflow - throw new VoldemortException("Found too many collisions: chunk " + chunkId - + " has exceeded " + Short.MAX_VALUE + " collisions."); - } else if(numTuples > 1) { - // Update number of collisions + max keys per collision - reporter.incrCounter(CollisionCounter.NUM_COLLISIONS, 1); - - long numCollisions = reporter.getCounter(CollisionCounter.MAX_COLLISIONS).getCounter(); - if(numTuples > numCollisions) { - reporter.incrCounter(CollisionCounter.MAX_COLLISIONS, numTuples - numCollisions); - } - } - - // Flush the value - valueStream.flush(); - byte[] value = stream.toByteArray(); - - // Start writing to file now - // First, if save keys flag set the number of keys - if(getSaveKeys()) { - - this.valueFileStream.writeShort(numTuples); - this.position += ByteUtils.SIZE_OF_SHORT; - - if(this.checkSumDigestValue != null) { - this.checkSumDigestValue.update(numTuples); - } - } - - this.valueFileStream.write(value); - this.position += value.length; - - if(this.checkSumDigestValue != null) { - this.checkSumDigestValue.update(value); - } - - if(this.position < 0) - throw new VoldemortException("Chunk overflow exception: chunk " + chunkId - + " has exceeded " + Integer.MAX_VALUE + " bytes."); + writer.write(key, iterator, reporter); } @Override public void configure(JobConf job) { - super.configure(job); - try { - this.conf = job; - this.position = 0; - this.outputDir = job.get("final.output.dir"); - this.taskId = job.get("mapred.task.id"); - this.checkSumType = CheckSum.fromString(job.get("checksum.type")); - this.checkSumDigestIndex = CheckSum.getInstance(checkSumType); - this.checkSumDigestValue = CheckSum.getInstance(checkSumType); - this.taskIndexFileName = new Path(FileOutputFormat.getOutputPath(job), getStoreName() - + "." - + this.taskId - + ".index"); - this.taskValueFileName = new Path(FileOutputFormat.getOutputPath(job), getStoreName() - + "." - + this.taskId - + ".data"); - - if(this.fs == null) - this.fs = this.taskIndexFileName.getFileSystem(job); + try { - this.indexFileStream = fs.create(this.taskIndexFileName); - this.valueFileStream = fs.create(this.taskValueFileName); + keyValueWriterClass = job.get("writer.class"); + if(keyValueWriterClass != null) + writer = (KeyValueWriter) Utils.callConstructor(keyValueWriterClass); + else + writer = new HadoopStoreWriter(); - logger.info("Opening " + this.taskIndexFileName + " and " + this.taskValueFileName - + " for writing."); + writer.conf(job); - } catch(IOException e) { + } catch(Exception e) { throw new RuntimeException("Failed to open Input/OutputStream", e); } } @Override public void close() throws IOException { - - this.indexFileStream.close(); - this.valueFileStream.close(); - - if(this.nodeId == -1 || this.chunkId == -1 || this.partitionId == -1) { - // Issue 258 - No data was read in the reduce phase, do not create - // any output - return; - } - - // If the replica type read was not valid, shout out - if(getSaveKeys() && this.replicaType == -1) { - throw new RuntimeException("Could not read the replica type correctly for node " - + nodeId + " ( partition - " + this.partitionId + " )"); - } - - String fileNamePrefix = null; - if(getSaveKeys()) { - fileNamePrefix = new String(Integer.toString(this.partitionId) + "_" - + Integer.toString(this.replicaType) + "_" - + Integer.toString(this.chunkId)); - } else { - fileNamePrefix = new String(Integer.toString(this.partitionId) + "_" - + Integer.toString(this.chunkId)); - } - - // Initialize the node directory - Path nodeDir = new Path(this.outputDir, "node-" + this.nodeId); - - // Create output directory, if it doesn't exist - FileSystem outputFs = nodeDir.getFileSystem(this.conf); - outputFs.mkdirs(nodeDir); - - // Write the checksum and output files - if(this.checkSumType != CheckSumType.NONE) { - - if(this.checkSumDigestIndex != null && this.checkSumDigestValue != null) { - Path checkSumIndexFile = new Path(nodeDir, fileNamePrefix + ".index.checksum"); - Path checkSumValueFile = new Path(nodeDir, fileNamePrefix + ".data.checksum"); - - FSDataOutputStream output = outputFs.create(checkSumIndexFile); - output.write(this.checkSumDigestIndex.getCheckSum()); - output.close(); - - output = outputFs.create(checkSumValueFile); - output.write(this.checkSumDigestValue.getCheckSum()); - output.close(); - } else { - throw new RuntimeException("Failed to open checksum digest for node " + nodeId - + " ( partition - " + this.partitionId + ", chunk - " - + chunkId + " )"); - } - } - - // Generate the final chunk files - Path indexFile = new Path(nodeDir, fileNamePrefix + ".index"); - Path valueFile = new Path(nodeDir, fileNamePrefix + ".data"); - - logger.info("Moving " + this.taskIndexFileName + " to " + indexFile); - outputFs.rename(taskIndexFileName, indexFile); - logger.info("Moving " + this.taskValueFileName + " to " + valueFile); - outputFs.rename(this.taskValueFileName, valueFile); - + writer.close(); } } diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/HadoopStoreBuilderReducerPerBucket.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/HadoopStoreBuilderReducerPerBucket.java index 4b76f5662b..d38af9d9a4 100644 --- a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/HadoopStoreBuilderReducerPerBucket.java +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/HadoopStoreBuilderReducerPerBucket.java @@ -16,28 +16,20 @@ package voldemort.store.readonly.mr; -import java.io.ByteArrayOutputStream; -import java.io.DataOutputStream; import java.io.IOException; import java.util.Iterator; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapred.FileOutputFormat; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reducer; import org.apache.hadoop.mapred.Reporter; import org.apache.log4j.Logger; -import voldemort.VoldemortException; -import voldemort.store.readonly.ReadOnlyUtils; -import voldemort.store.readonly.checksum.CheckSum; -import voldemort.store.readonly.checksum.CheckSum.CheckSumType; -import voldemort.utils.ByteUtils; +import voldemort.store.readonly.disk.HadoopStoreWriterPerBucket; +import voldemort.store.readonly.disk.KeyValueWriter; +import azkaban.common.utils.Utils; /** * Take key md5s and value bytes and build a read-only store from these values @@ -48,31 +40,9 @@ public class HadoopStoreBuilderReducerPerBucket extends AbstractStoreBuilderConf private static final Logger logger = Logger.getLogger(HadoopStoreBuilderReducerPerBucket.class); - private DataOutputStream[] indexFileStream = null; - private DataOutputStream[] valueFileStream = null; - private int[] position; - private String taskId = null; - - private int nodeId = -1; - private int partitionId = -1; - private int replicaType = -1; - - private Path[] taskIndexFileName; - private Path[] taskValueFileName; - - private JobConf conf; - private CheckSumType checkSumType; - private CheckSum[] checkSumDigestIndex; - private CheckSum[] checkSumDigestValue; - - private String outputDir; - - private FileSystem fs; - - protected static enum CollisionCounter { - NUM_COLLISIONS, - MAX_COLLISIONS; - } + String keyValueWriterClass; + @SuppressWarnings("rawtypes") + KeyValueWriter writer; /** * Reduce should get sorted MD5 of Voldemort key ( either 16 bytes if saving @@ -81,237 +51,36 @@ protected static enum CollisionCounter { * partition-id, replica-type, [key-size, value-size, key, value]* if saving * keys is enabled */ + @SuppressWarnings("unchecked") public void reduce(BytesWritable key, Iterator iterator, OutputCollector output, Reporter reporter) throws IOException { - // Read chunk id - int chunkId = ReadOnlyUtils.chunk(key.get(), getNumChunks()); - - // Write key and position - this.indexFileStream[chunkId].write(key.get(), 0, key.getSize()); - this.indexFileStream[chunkId].writeInt(this.position[chunkId]); - - // Run key through checksum digest - if(this.checkSumDigestIndex[chunkId] != null) { - this.checkSumDigestIndex[chunkId].update(key.get(), 0, key.getSize()); - this.checkSumDigestIndex[chunkId].update(this.position[chunkId]); - } - - short numTuples = 0; - ByteArrayOutputStream stream = new ByteArrayOutputStream(); - DataOutputStream valueStream = new DataOutputStream(stream); - - while(iterator.hasNext()) { - BytesWritable writable = iterator.next(); - byte[] valueBytes = writable.get(); - int offsetTillNow = 0; - - // Read node Id - if(this.nodeId == -1) - this.nodeId = ByteUtils.readInt(valueBytes, offsetTillNow); - offsetTillNow += ByteUtils.SIZE_OF_INT; - - // Read partition id - if(this.partitionId == -1) - this.partitionId = ByteUtils.readInt(valueBytes, offsetTillNow); - offsetTillNow += ByteUtils.SIZE_OF_INT; - - // Read replica type - if(getSaveKeys()) { - if(this.replicaType == -1) - this.replicaType = (int) ByteUtils.readBytes(valueBytes, - offsetTillNow, - ByteUtils.SIZE_OF_BYTE); - offsetTillNow += ByteUtils.SIZE_OF_BYTE; - } - - int valueLength = writable.getSize() - offsetTillNow; - if(getSaveKeys()) { - // Write ( key_length, value_length, key, - // value ) - valueStream.write(valueBytes, offsetTillNow, valueLength); - } else { - // Write (value_length + value) - valueStream.writeInt(valueLength); - valueStream.write(valueBytes, offsetTillNow, valueLength); - } - - numTuples++; - - // If we have multiple values for this md5 that is a collision, - // throw an exception--either the data itself has duplicates, there - // are trillions of keys, or someone is attempting something - // malicious ( We obviously expect collisions when we save keys ) - if(!getSaveKeys() && numTuples > 1) - throw new VoldemortException("Duplicate keys detected for md5 sum " - + ByteUtils.toHexString(ByteUtils.copy(key.get(), - 0, - key.getSize()))); - - } - - if(numTuples < 0) { - // Overflow - throw new VoldemortException("Found too many collisions: chunk " + chunkId - + " has exceeded " + Short.MAX_VALUE + " collisions."); - } else if(numTuples > 1) { - // Update number of collisions + max keys per collision - reporter.incrCounter(CollisionCounter.NUM_COLLISIONS, 1); - - long numCollisions = reporter.getCounter(CollisionCounter.MAX_COLLISIONS).getCounter(); - if(numTuples > numCollisions) { - reporter.incrCounter(CollisionCounter.MAX_COLLISIONS, numTuples - numCollisions); - } - } - - // Flush the value - valueStream.flush(); - byte[] value = stream.toByteArray(); - - // Start writing to file now - // First, if save keys flag set the number of keys - if(getSaveKeys()) { - - this.valueFileStream[chunkId].writeShort(numTuples); - this.position[chunkId] += ByteUtils.SIZE_OF_SHORT; - - if(this.checkSumDigestValue[chunkId] != null) { - this.checkSumDigestValue[chunkId].update(numTuples); - } - } - - this.valueFileStream[chunkId].write(value); - this.position[chunkId] += value.length; - - if(this.checkSumDigestValue[chunkId] != null) { - this.checkSumDigestValue[chunkId].update(value); - } - - if(this.position[chunkId] < 0) - throw new VoldemortException("Chunk overflow exception: chunk " + chunkId - + " has exceeded " + Integer.MAX_VALUE + " bytes."); + writer.write(key, iterator, reporter); } @Override public void configure(JobConf job) { - super.configure(job); - try { - this.conf = job; - this.outputDir = job.get("final.output.dir"); - this.taskId = job.get("mapred.task.id"); - this.checkSumType = CheckSum.fromString(job.get("checksum.type")); - - this.checkSumDigestIndex = new CheckSum[getNumChunks()]; - this.checkSumDigestValue = new CheckSum[getNumChunks()]; - this.position = new int[getNumChunks()]; - this.taskIndexFileName = new Path[getNumChunks()]; - this.taskValueFileName = new Path[getNumChunks()]; - this.indexFileStream = new DataOutputStream[getNumChunks()]; - this.valueFileStream = new DataOutputStream[getNumChunks()]; - - for(int chunkId = 0; chunkId < getNumChunks(); chunkId++) { - - this.checkSumDigestIndex[chunkId] = CheckSum.getInstance(checkSumType); - this.checkSumDigestValue[chunkId] = CheckSum.getInstance(checkSumType); - this.position[chunkId] = 0; - - this.taskIndexFileName[chunkId] = new Path(FileOutputFormat.getOutputPath(job), - getStoreName() + "." - + Integer.toString(chunkId) - + "_" + this.taskId + ".index"); - this.taskValueFileName[chunkId] = new Path(FileOutputFormat.getOutputPath(job), - getStoreName() + "." - + Integer.toString(chunkId) - + "_" + this.taskId + ".data"); - if(this.fs == null) - this.fs = this.taskIndexFileName[chunkId].getFileSystem(job); + try { - this.indexFileStream[chunkId] = fs.create(this.taskIndexFileName[chunkId]); - this.valueFileStream[chunkId] = fs.create(this.taskValueFileName[chunkId]); + keyValueWriterClass = job.get("writer.class"); + if(keyValueWriterClass != null) + writer = (KeyValueWriter) Utils.callConstructor(keyValueWriterClass); + else + writer = new HadoopStoreWriterPerBucket(); - logger.info("Opening " + this.taskIndexFileName[chunkId] + " and " - + this.taskValueFileName[chunkId] + " for writing."); - } + writer.conf(job); - } catch(IOException e) { + } catch(Exception e) { throw new RuntimeException("Failed to open Input/OutputStream", e); } } @Override public void close() throws IOException { - - for(int chunkId = 0; chunkId < getNumChunks(); chunkId++) { - this.indexFileStream[chunkId].close(); - this.valueFileStream[chunkId].close(); - } - - if(this.nodeId == -1 || this.partitionId == -1) { - // Issue 258 - No data was read in the reduce phase, do not create - // any output - return; - } - - // If the replica type read was not valid, shout out - if(getSaveKeys() && this.replicaType == -1) { - throw new RuntimeException("Could not read the replica type correctly for node " - + nodeId + " ( partition - " + this.partitionId + " )"); - } - - String fileNamePrefix = null; - if(getSaveKeys()) { - fileNamePrefix = new String(Integer.toString(this.partitionId) + "_" - + Integer.toString(this.replicaType) + "_"); - } else { - fileNamePrefix = new String(Integer.toString(this.partitionId) + "_"); - } - - // Initialize the node directory - Path nodeDir = new Path(this.outputDir, "node-" + this.nodeId); - - // Create output directory, if it doesn't exist - FileSystem outputFs = nodeDir.getFileSystem(this.conf); - outputFs.mkdirs(nodeDir); - - // Write the checksum and output files - for(int chunkId = 0; chunkId < getNumChunks(); chunkId++) { - - String chunkFileName = fileNamePrefix + Integer.toString(chunkId); - if(this.checkSumType != CheckSumType.NONE) { - - if(this.checkSumDigestIndex[chunkId] != null - && this.checkSumDigestValue[chunkId] != null) { - Path checkSumIndexFile = new Path(nodeDir, chunkFileName + ".index.checksum"); - Path checkSumValueFile = new Path(nodeDir, chunkFileName + ".data.checksum"); - - FSDataOutputStream output = outputFs.create(checkSumIndexFile); - output.write(this.checkSumDigestIndex[chunkId].getCheckSum()); - output.close(); - - output = outputFs.create(checkSumValueFile); - output.write(this.checkSumDigestValue[chunkId].getCheckSum()); - output.close(); - } else { - throw new RuntimeException("Failed to open checksum digest for node " + nodeId - + " ( partition - " + this.partitionId - + ", chunk - " + chunkId + " )"); - } - } - - // Generate the final chunk files - Path indexFile = new Path(nodeDir, chunkFileName + ".index"); - Path valueFile = new Path(nodeDir, chunkFileName + ".data"); - - logger.info("Moving " + this.taskIndexFileName[chunkId] + " to " + indexFile); - fs.rename(taskIndexFileName[chunkId], indexFile); - logger.info("Moving " + this.taskValueFileName[chunkId] + " to " + valueFile); - fs.rename(this.taskValueFileName[chunkId], valueFile); - - } - + writer.close(); } } From 65e8bee4949583de0cefe4874af0e7cb2edd3e93 Mon Sep 17 00:00:00 2001 From: Abhinay Nagpal Date: Fri, 7 Sep 2012 09:47:02 -0700 Subject: [PATCH 139/209] Added support for specifying the key and value field in the avro record --- .../readonly/disk/HadoopStoreWriter.java | 321 ++++++ .../disk/HadoopStoreWriterPerBucket.java | 342 ++++++ .../store/readonly/disk/KeyValueWriter.java | 23 + .../readonly/mr/AvroStoreBuilderMapper.java | 296 ++++++ .../mr/AvroStoreBuilderPartitioner.java | 154 +++ .../readonly/mr/AvroStoreBuilderReducer.java | 92 ++ .../mr/AvroStoreBuilderReducerPerBucket.java | 94 ++ .../store/readonly/mr/IdentityJsonMapper.java | 20 + .../readonly/mr/IdentityJsonReducer.java | 22 + .../mr/VoldemortStoreBuilderMapper.java | 73 ++ .../mr/azkaban/AbstractHadoopJob.java | 268 +++++ .../AbstractVoldemortBatchCopyJob.java | 116 +++ .../store/readonly/mr/azkaban/Job.java | 59 ++ .../azkaban/StoreBuilderTransformation.java | 14 + .../azkaban/UndefinedPropertyException.java | 11 + .../mr/azkaban/VoldemortBatchIndexJob.java | 405 +++++++ .../mr/azkaban/VoldemortBuildAndPushJob.java | 792 ++++++++++++++ .../VoldemortMultiStoreBuildAndPushJob.java | 834 +++++++++++++++ .../mr/azkaban/VoldemortRollbackJob.java | 116 +++ .../mr/azkaban/VoldemortStoreBuilderJob.java | 450 ++++++++ .../readonly/mr/azkaban/VoldemortSwapJob.java | 200 ++++ .../mr/azkaban/VoldemortSwapperUtils.java | 72 ++ .../mr/serialization/JsonConfigurable.java | 76 ++ .../JsonDeserializerComparator.java | 118 +++ .../readonly/mr/serialization/JsonMapper.java | 58 ++ .../mr/serialization/JsonOutputCollector.java | 44 + .../mr/serialization/JsonReducer.java | 89 ++ .../JsonSequenceFileInputFormat.java | 110 ++ .../JsonSequenceFileOutputFormat.java | 101 ++ .../store/readonly/mr/utils/AvroUtils.java | 105 ++ .../store/readonly/mr/utils/EmailMessage.java | 186 ++++ .../store/readonly/mr/utils/HadoopUtils.java | 985 ++++++++++++++++++ .../store/readonly/mr/utils/JsonSchema.java | 49 + .../readonly/mr/utils/VoldemortUtils.java | 131 +++ lib/azkaban-common-0.05.jar | Bin 0 -> 58108 bytes lib/joda-time-1.6.jar | Bin 0 -> 534827 bytes lib/mail-1.4.1.jar | Bin 0 -> 371264 bytes 37 files changed, 6826 insertions(+) create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/HadoopStoreWriter.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/HadoopStoreWriterPerBucket.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/KeyValueWriter.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderMapper.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderPartitioner.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderReducer.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderReducerPerBucket.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/IdentityJsonMapper.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/IdentityJsonReducer.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/VoldemortStoreBuilderMapper.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/AbstractHadoopJob.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/AbstractVoldemortBatchCopyJob.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/Job.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/StoreBuilderTransformation.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/UndefinedPropertyException.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortBatchIndexJob.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortBuildAndPushJob.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortMultiStoreBuildAndPushJob.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortRollbackJob.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortStoreBuilderJob.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortSwapJob.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortSwapperUtils.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonConfigurable.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonDeserializerComparator.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonMapper.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonOutputCollector.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonReducer.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonSequenceFileInputFormat.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/serialization/JsonSequenceFileOutputFormat.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/AvroUtils.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/EmailMessage.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/HadoopUtils.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/JsonSchema.java create mode 100644 contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/utils/VoldemortUtils.java create mode 100644 lib/azkaban-common-0.05.jar create mode 100644 lib/joda-time-1.6.jar create mode 100644 lib/mail-1.4.1.jar diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/HadoopStoreWriter.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/HadoopStoreWriter.java new file mode 100644 index 0000000000..b0cfb63479 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/HadoopStoreWriter.java @@ -0,0 +1,321 @@ +package voldemort.store.readonly.disk; + +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.StringReader; +import java.util.Iterator; +import java.util.List; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.mapred.FileOutputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Reporter; +import org.apache.log4j.Logger; + +import voldemort.VoldemortException; +import voldemort.cluster.Cluster; +import voldemort.store.StoreDefinition; +import voldemort.store.readonly.ReadOnlyUtils; +import voldemort.store.readonly.checksum.CheckSum; +import voldemort.store.readonly.checksum.CheckSum.CheckSumType; +import voldemort.utils.ByteUtils; +import voldemort.xml.ClusterMapper; +import voldemort.xml.StoreDefinitionsMapper; + +public class HadoopStoreWriter implements KeyValueWriter { + + private static final Logger logger = Logger.getLogger(HadoopStoreWriter.class); + + private DataOutputStream indexFileStream = null; + private DataOutputStream valueFileStream = null; + private int position; + private String taskId = null; + + private int nodeId = -1; + private int partitionId = -1; + private int chunkId = -1; + private int replicaType = -1; + + private Path taskIndexFileName; + private Path taskValueFileName; + + private JobConf conf; + private CheckSumType checkSumType; + private CheckSum checkSumDigestIndex; + private CheckSum checkSumDigestValue; + + private String outputDir; + + private FileSystem fs; + + private int numChunks; + private Cluster cluster; + private StoreDefinition storeDef; + private boolean saveKeys; + private boolean reducerPerBucket; + + public Cluster getCluster() { + checkNotNull(cluster); + return cluster; + } + + public boolean getSaveKeys() { + return this.saveKeys; + } + + public boolean getReducerPerBucket() { + return this.reducerPerBucket; + } + + public StoreDefinition getStoreDef() { + checkNotNull(storeDef); + return storeDef; + } + + public String getStoreName() { + checkNotNull(storeDef); + return storeDef.getName(); + } + + private final void checkNotNull(Object o) { + if(o == null) + throw new VoldemortException("Not configured yet!"); + } + + public int getNumChunks() { + return this.numChunks; + } + + @Override + public void conf(JobConf job) { + + conf = job; + try { + + this.cluster = new ClusterMapper().readCluster(new StringReader(conf.get("cluster.xml"))); + List storeDefs = new StoreDefinitionsMapper().readStoreList(new StringReader(conf.get("stores.xml"))); + if(storeDefs.size() != 1) + throw new IllegalStateException("Expected to find only a single store, but found multiple!"); + this.storeDef = storeDefs.get(0); + + this.numChunks = conf.getInt("num.chunks", -1); + if(this.numChunks < 1) + throw new VoldemortException("num.chunks not specified in the job conf."); + this.saveKeys = conf.getBoolean("save.keys", false); + this.reducerPerBucket = conf.getBoolean("reducer.per.bucket", false); + this.conf = job; + this.position = 0; + this.outputDir = job.get("final.output.dir"); + this.taskId = job.get("mapred.task.id"); + this.checkSumType = CheckSum.fromString(job.get("checksum.type")); + this.checkSumDigestIndex = CheckSum.getInstance(checkSumType); + this.checkSumDigestValue = CheckSum.getInstance(checkSumType); + + this.taskIndexFileName = new Path(FileOutputFormat.getOutputPath(job), getStoreName() + + "." + + this.taskId + + ".index"); + this.taskValueFileName = new Path(FileOutputFormat.getOutputPath(job), getStoreName() + + "." + + this.taskId + + ".data"); + + if(this.fs == null) + this.fs = this.taskIndexFileName.getFileSystem(job); + + this.indexFileStream = fs.create(this.taskIndexFileName); + this.valueFileStream = fs.create(this.taskValueFileName); + + logger.info("Opening " + this.taskIndexFileName + " and " + this.taskValueFileName + + " for writing."); + + } catch(IOException e) { + throw new RuntimeException("Failed to open Input/OutputStream", e); + } + + } + + @Override + public void write(BytesWritable key, Iterator iterator, Reporter reporter) + throws IOException { + + // Write key and position + this.indexFileStream.write(key.get(), 0, key.getSize()); + this.indexFileStream.writeInt(this.position); + + // Run key through checksum digest + if(this.checkSumDigestIndex != null) { + this.checkSumDigestIndex.update(key.get(), 0, key.getSize()); + this.checkSumDigestIndex.update(this.position); + } + + short numTuples = 0; + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + DataOutputStream valueStream = new DataOutputStream(stream); + + while(iterator.hasNext()) { + BytesWritable writable = iterator.next(); + byte[] valueBytes = writable.get(); + int offsetTillNow = 0; + + // Read node Id + if(this.nodeId == -1) + this.nodeId = ByteUtils.readInt(valueBytes, offsetTillNow); + offsetTillNow += ByteUtils.SIZE_OF_INT; + + // Read partition id + if(this.partitionId == -1) + this.partitionId = ByteUtils.readInt(valueBytes, offsetTillNow); + offsetTillNow += ByteUtils.SIZE_OF_INT; + + // Read chunk id + if(this.chunkId == -1) + this.chunkId = ReadOnlyUtils.chunk(key.get(), getNumChunks()); + + // Read replica type + if(getSaveKeys()) { + if(this.replicaType == -1) + this.replicaType = (int) ByteUtils.readBytes(valueBytes, + offsetTillNow, + ByteUtils.SIZE_OF_BYTE); + offsetTillNow += ByteUtils.SIZE_OF_BYTE; + } + + int valueLength = writable.getSize() - offsetTillNow; + if(getSaveKeys()) { + // Write ( key_length, value_length, key, + // value ) + valueStream.write(valueBytes, offsetTillNow, valueLength); + } else { + // Write (value_length + value) + valueStream.writeInt(valueLength); + valueStream.write(valueBytes, offsetTillNow, valueLength); + } + + numTuples++; + + // If we have multiple values for this md5 that is a collision, + // throw an exception--either the data itself has duplicates, there + // are trillions of keys, or someone is attempting something + // malicious ( We obviously expect collisions when we save keys ) + if(!getSaveKeys() && numTuples > 1) + throw new VoldemortException("Duplicate keys detected for md5 sum " + + ByteUtils.toHexString(ByteUtils.copy(key.get(), + 0, + key.getSize()))); + + } + + if(numTuples < 0) { + // Overflow + throw new VoldemortException("Found too many collisions: chunk " + chunkId + + " has exceeded " + Short.MAX_VALUE + " collisions."); + } else if(numTuples > 1) { + // Update number of collisions + max keys per collision + reporter.incrCounter(CollisionCounter.NUM_COLLISIONS, 1); + + long numCollisions = reporter.getCounter(CollisionCounter.MAX_COLLISIONS).getCounter(); + if(numTuples > numCollisions) { + reporter.incrCounter(CollisionCounter.MAX_COLLISIONS, numTuples - numCollisions); + } + } + + // Flush the value + valueStream.flush(); + byte[] value = stream.toByteArray(); + + // Start writing to file now + // First, if save keys flag set the number of keys + if(getSaveKeys()) { + + this.valueFileStream.writeShort(numTuples); + this.position += ByteUtils.SIZE_OF_SHORT; + + if(this.checkSumDigestValue != null) { + this.checkSumDigestValue.update(numTuples); + } + } + + this.valueFileStream.write(value); + this.position += value.length; + + if(this.checkSumDigestValue != null) { + this.checkSumDigestValue.update(value); + } + + if(this.position < 0) + throw new VoldemortException("Chunk overflow exception: chunk " + chunkId + + " has exceeded " + Integer.MAX_VALUE + " bytes."); + } + + @Override + public void close() throws IOException { + + this.indexFileStream.close(); + this.valueFileStream.close(); + + if(this.nodeId == -1 || this.chunkId == -1 || this.partitionId == -1) { + // Issue 258 - No data was read in the reduce phase, do not create + // any output + return; + } + + // If the replica type read was not valid, shout out + if(getSaveKeys() && this.replicaType == -1) { + throw new RuntimeException("Could not read the replica type correctly for node " + + nodeId + " ( partition - " + this.partitionId + " )"); + } + + String fileNamePrefix = null; + if(getSaveKeys()) { + fileNamePrefix = new String(Integer.toString(this.partitionId) + "_" + + Integer.toString(this.replicaType) + "_" + + Integer.toString(this.chunkId)); + } else { + fileNamePrefix = new String(Integer.toString(this.partitionId) + "_" + + Integer.toString(this.chunkId)); + } + + // Initialize the node directory + Path nodeDir = new Path(this.outputDir, "node-" + this.nodeId); + + // Create output directory, if it doesn't exist + FileSystem outputFs = nodeDir.getFileSystem(this.conf); + outputFs.mkdirs(nodeDir); + + // Write the checksum and output files + if(this.checkSumType != CheckSumType.NONE) { + + if(this.checkSumDigestIndex != null && this.checkSumDigestValue != null) { + Path checkSumIndexFile = new Path(nodeDir, fileNamePrefix + ".index.checksum"); + Path checkSumValueFile = new Path(nodeDir, fileNamePrefix + ".data.checksum"); + + FSDataOutputStream output = outputFs.create(checkSumIndexFile); + output.write(this.checkSumDigestIndex.getCheckSum()); + output.close(); + + output = outputFs.create(checkSumValueFile); + output.write(this.checkSumDigestValue.getCheckSum()); + output.close(); + } else { + throw new RuntimeException("Failed to open checksum digest for node " + nodeId + + " ( partition - " + this.partitionId + ", chunk - " + + chunkId + " )"); + } + } + + // Generate the final chunk files + Path indexFile = new Path(nodeDir, fileNamePrefix + ".index"); + Path valueFile = new Path(nodeDir, fileNamePrefix + ".data"); + + logger.info("Moving " + this.taskIndexFileName + " to " + indexFile); + outputFs.rename(taskIndexFileName, indexFile); + logger.info("Moving " + this.taskValueFileName + " to " + valueFile); + outputFs.rename(this.taskValueFileName, valueFile); + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/HadoopStoreWriterPerBucket.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/HadoopStoreWriterPerBucket.java new file mode 100644 index 0000000000..3df92b85ba --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/HadoopStoreWriterPerBucket.java @@ -0,0 +1,342 @@ +package voldemort.store.readonly.disk; + +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.StringReader; +import java.util.Iterator; +import java.util.List; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.mapred.FileOutputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Reporter; +import org.apache.log4j.Logger; + +import voldemort.VoldemortException; +import voldemort.cluster.Cluster; +import voldemort.store.StoreDefinition; +import voldemort.store.readonly.ReadOnlyUtils; +import voldemort.store.readonly.checksum.CheckSum; +import voldemort.store.readonly.checksum.CheckSum.CheckSumType; +import voldemort.utils.ByteUtils; +import voldemort.xml.ClusterMapper; +import voldemort.xml.StoreDefinitionsMapper; + +public class HadoopStoreWriterPerBucket implements KeyValueWriter { + + private static final Logger logger = Logger.getLogger(HadoopStoreWriterPerBucket.class); + + private DataOutputStream[] indexFileStream = null; + private DataOutputStream[] valueFileStream = null; + private int[] position; + private String taskId = null; + + private int nodeId = -1; + private int partitionId = -1; + private int replicaType = -1; + + private Path[] taskIndexFileName; + private Path[] taskValueFileName; + + private JobConf conf; + private CheckSumType checkSumType; + private CheckSum[] checkSumDigestIndex; + private CheckSum[] checkSumDigestValue; + + private String outputDir; + + private FileSystem fs; + + @Override + public void conf(JobConf job) { + + JobConf conf = job; + try { + + this.cluster = new ClusterMapper().readCluster(new StringReader(conf.get("cluster.xml"))); + List storeDefs = new StoreDefinitionsMapper().readStoreList(new StringReader(conf.get("stores.xml"))); + if(storeDefs.size() != 1) + throw new IllegalStateException("Expected to find only a single store, but found multiple!"); + this.storeDef = storeDefs.get(0); + + this.numChunks = conf.getInt("num.chunks", -1); + if(this.numChunks < 1) + throw new VoldemortException("num.chunks not specified in the job conf."); + + this.saveKeys = conf.getBoolean("save.keys", false); + this.reducerPerBucket = conf.getBoolean("reducer.per.bucket", false); + this.conf = job; + this.outputDir = job.get("final.output.dir"); + this.taskId = job.get("mapred.task.id"); + this.checkSumType = CheckSum.fromString(job.get("checksum.type")); + + this.checkSumDigestIndex = new CheckSum[getNumChunks()]; + this.checkSumDigestValue = new CheckSum[getNumChunks()]; + this.position = new int[getNumChunks()]; + this.taskIndexFileName = new Path[getNumChunks()]; + this.taskValueFileName = new Path[getNumChunks()]; + this.indexFileStream = new DataOutputStream[getNumChunks()]; + this.valueFileStream = new DataOutputStream[getNumChunks()]; + + for(int chunkId = 0; chunkId < getNumChunks(); chunkId++) { + + this.checkSumDigestIndex[chunkId] = CheckSum.getInstance(checkSumType); + this.checkSumDigestValue[chunkId] = CheckSum.getInstance(checkSumType); + this.position[chunkId] = 0; + + this.taskIndexFileName[chunkId] = new Path(FileOutputFormat.getOutputPath(job), + getStoreName() + "." + + Integer.toString(chunkId) + + "_" + this.taskId + ".index"); + this.taskValueFileName[chunkId] = new Path(FileOutputFormat.getOutputPath(job), + getStoreName() + "." + + Integer.toString(chunkId) + + "_" + this.taskId + ".data"); + + if(this.fs == null) + this.fs = this.taskIndexFileName[chunkId].getFileSystem(job); + + this.indexFileStream[chunkId] = fs.create(this.taskIndexFileName[chunkId]); + this.valueFileStream[chunkId] = fs.create(this.taskValueFileName[chunkId]); + + logger.info("Opening " + this.taskIndexFileName[chunkId] + " and " + + this.taskValueFileName[chunkId] + " for writing."); + } + + } catch(IOException e) { + // throw new RuntimeException("Failed to open Input/OutputStream", + // e); + e.printStackTrace(); + } + + } + + @Override + public void write(BytesWritable key, Iterator iterator, Reporter reporter) + throws IOException { + + // Read chunk id + int chunkId = ReadOnlyUtils.chunk(key.get(), getNumChunks()); + + // Write key and position + this.indexFileStream[chunkId].write(key.get(), 0, key.getSize()); + this.indexFileStream[chunkId].writeInt(this.position[chunkId]); + + // Run key through checksum digest + if(this.checkSumDigestIndex[chunkId] != null) { + this.checkSumDigestIndex[chunkId].update(key.get(), 0, key.getSize()); + this.checkSumDigestIndex[chunkId].update(this.position[chunkId]); + } + + short numTuples = 0; + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + DataOutputStream valueStream = new DataOutputStream(stream); + + while(iterator.hasNext()) { + BytesWritable writable = iterator.next(); + byte[] valueBytes = writable.get(); + int offsetTillNow = 0; + + // Read node Id + if(this.nodeId == -1) + this.nodeId = ByteUtils.readInt(valueBytes, offsetTillNow); + offsetTillNow += ByteUtils.SIZE_OF_INT; + + // Read partition id + if(this.partitionId == -1) + this.partitionId = ByteUtils.readInt(valueBytes, offsetTillNow); + offsetTillNow += ByteUtils.SIZE_OF_INT; + + // Read replica type + if(getSaveKeys()) { + if(this.replicaType == -1) + this.replicaType = (int) ByteUtils.readBytes(valueBytes, + offsetTillNow, + ByteUtils.SIZE_OF_BYTE); + offsetTillNow += ByteUtils.SIZE_OF_BYTE; + } + + int valueLength = writable.getSize() - offsetTillNow; + if(getSaveKeys()) { + // Write ( key_length, value_length, key, + // value ) + valueStream.write(valueBytes, offsetTillNow, valueLength); + } else { + // Write (value_length + value) + valueStream.writeInt(valueLength); + valueStream.write(valueBytes, offsetTillNow, valueLength); + } + + numTuples++; + + // If we have multiple values for this md5 that is a collision, + // throw an exception--either the data itself has duplicates, there + // are trillions of keys, or someone is attempting something + // malicious ( We obviously expect collisions when we save keys ) + if(!getSaveKeys() && numTuples > 1) + throw new VoldemortException("Duplicate keys detected for md5 sum " + + ByteUtils.toHexString(ByteUtils.copy(key.get(), + 0, + key.getSize()))); + + } + + if(numTuples < 0) { + // Overflow + throw new VoldemortException("Found too many collisions: chunk " + chunkId + + " has exceeded " + Short.MAX_VALUE + " collisions."); + } else if(numTuples > 1) { + // Update number of collisions + max keys per collision + reporter.incrCounter(CollisionCounter.NUM_COLLISIONS, 1); + + long numCollisions = reporter.getCounter(CollisionCounter.MAX_COLLISIONS).getCounter(); + if(numTuples > numCollisions) { + reporter.incrCounter(CollisionCounter.MAX_COLLISIONS, numTuples - numCollisions); + } + } + + // Flush the value + valueStream.flush(); + byte[] value = stream.toByteArray(); + + // Start writing to file now + // First, if save keys flag set the number of keys + if(getSaveKeys()) { + + this.valueFileStream[chunkId].writeShort(numTuples); + this.position[chunkId] += ByteUtils.SIZE_OF_SHORT; + + if(this.checkSumDigestValue[chunkId] != null) { + this.checkSumDigestValue[chunkId].update(numTuples); + } + } + + this.valueFileStream[chunkId].write(value); + this.position[chunkId] += value.length; + + if(this.checkSumDigestValue[chunkId] != null) { + this.checkSumDigestValue[chunkId].update(value); + } + + if(this.position[chunkId] < 0) + throw new VoldemortException("Chunk overflow exception: chunk " + chunkId + + " has exceeded " + Integer.MAX_VALUE + " bytes."); + + } + + @Override + public void close() throws IOException { + + for(int chunkId = 0; chunkId < getNumChunks(); chunkId++) { + this.indexFileStream[chunkId].close(); + this.valueFileStream[chunkId].close(); + } + + if(this.nodeId == -1 || this.partitionId == -1) { + // Issue 258 - No data was read in the reduce phase, do not create + // any output + return; + } + + // If the replica type read was not valid, shout out + if(getSaveKeys() && this.replicaType == -1) { + throw new RuntimeException("Could not read the replica type correctly for node " + + nodeId + " ( partition - " + this.partitionId + " )"); + } + + String fileNamePrefix = null; + if(getSaveKeys()) { + fileNamePrefix = new String(Integer.toString(this.partitionId) + "_" + + Integer.toString(this.replicaType) + "_"); + } else { + fileNamePrefix = new String(Integer.toString(this.partitionId) + "_"); + } + + // Initialize the node directory + Path nodeDir = new Path(this.outputDir, "node-" + this.nodeId); + + // Create output directory, if it doesn't exist + FileSystem outputFs = nodeDir.getFileSystem(this.conf); + outputFs.mkdirs(nodeDir); + + // Write the checksum and output files + for(int chunkId = 0; chunkId < getNumChunks(); chunkId++) { + + String chunkFileName = fileNamePrefix + Integer.toString(chunkId); + if(this.checkSumType != CheckSumType.NONE) { + + if(this.checkSumDigestIndex[chunkId] != null + && this.checkSumDigestValue[chunkId] != null) { + Path checkSumIndexFile = new Path(nodeDir, chunkFileName + ".index.checksum"); + Path checkSumValueFile = new Path(nodeDir, chunkFileName + ".data.checksum"); + + FSDataOutputStream output = outputFs.create(checkSumIndexFile); + output.write(this.checkSumDigestIndex[chunkId].getCheckSum()); + output.close(); + + output = outputFs.create(checkSumValueFile); + output.write(this.checkSumDigestValue[chunkId].getCheckSum()); + output.close(); + } else { + throw new RuntimeException("Failed to open checksum digest for node " + nodeId + + " ( partition - " + this.partitionId + + ", chunk - " + chunkId + " )"); + } + } + + // Generate the final chunk files + Path indexFile = new Path(nodeDir, chunkFileName + ".index"); + Path valueFile = new Path(nodeDir, chunkFileName + ".data"); + + logger.info("Moving " + this.taskIndexFileName[chunkId] + " to " + indexFile); + fs.rename(taskIndexFileName[chunkId], indexFile); + logger.info("Moving " + this.taskValueFileName[chunkId] + " to " + valueFile); + fs.rename(this.taskValueFileName[chunkId], valueFile); + + } + + } + + private int numChunks; + private Cluster cluster; + private StoreDefinition storeDef; + private boolean saveKeys; + private boolean reducerPerBucket; + + public Cluster getCluster() { + checkNotNull(cluster); + return cluster; + } + + public boolean getSaveKeys() { + return this.saveKeys; + } + + public boolean getReducerPerBucket() { + return this.reducerPerBucket; + } + + public StoreDefinition getStoreDef() { + checkNotNull(storeDef); + return storeDef; + } + + public String getStoreName() { + checkNotNull(storeDef); + return storeDef.getName(); + } + + private final void checkNotNull(Object o) { + if(o == null) + throw new VoldemortException("Not configured yet!"); + } + + public int getNumChunks() { + return this.numChunks; + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/KeyValueWriter.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/KeyValueWriter.java new file mode 100644 index 0000000000..ddc11c69e2 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/disk/KeyValueWriter.java @@ -0,0 +1,23 @@ +package voldemort.store.readonly.disk; + +import java.io.IOException; +import java.util.Iterator; + +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Reporter; + +public interface KeyValueWriter { + + public static enum CollisionCounter { + + NUM_COLLISIONS, + MAX_COLLISIONS; + } + + public void conf(JobConf job); + + public void write(K key, Iterator iterator, Reporter reporter) throws IOException; + + public void close() throws IOException; + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderMapper.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderMapper.java new file mode 100644 index 0000000000..05b14179e5 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderMapper.java @@ -0,0 +1,296 @@ +package voldemort.store.readonly.mr; + +import java.io.IOException; +import java.io.StringReader; +import java.nio.ByteBuffer; +import java.security.MessageDigest; +import java.util.List; + +import org.apache.avro.generic.GenericData; +import org.apache.avro.mapred.AvroCollector; +import org.apache.avro.mapred.AvroMapper; +import org.apache.avro.mapred.Pair; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.JobConfigurable; +import org.apache.hadoop.mapred.Reporter; + +import voldemort.VoldemortException; +import voldemort.cluster.Cluster; +import voldemort.cluster.Node; +import voldemort.routing.ConsistentRoutingStrategy; +import voldemort.serialization.DefaultSerializerFactory; +import voldemort.serialization.SerializerDefinition; +import voldemort.serialization.SerializerFactory; +import voldemort.serialization.avro.AvroGenericSerializer; +import voldemort.store.StoreDefinition; +import voldemort.store.compress.CompressionStrategy; +import voldemort.store.compress.CompressionStrategyFactory; +import voldemort.store.readonly.mr.azkaban.StoreBuilderTransformation; +import voldemort.store.readonly.mr.utils.HadoopUtils; +import voldemort.utils.ByteUtils; +import voldemort.xml.ClusterMapper; +import voldemort.xml.StoreDefinitionsMapper; +import azkaban.common.utils.Props; +import azkaban.common.utils.Utils; + +public class AvroStoreBuilderMapper extends + AvroMapper> implements JobConfigurable { + + protected MessageDigest md5er; + protected ConsistentRoutingStrategy routingStrategy; + protected AvroGenericSerializer keySerializer; + protected AvroGenericSerializer valueSerializer; + + private String keySchema; + private String valSchema; + + private String keyField; + private String valField; + + private String _keySelection; + private String _valSelection; + + private StoreBuilderTransformation _keyTrans; + private StoreBuilderTransformation _valTrans; + + private CompressionStrategy valueCompressor; + private CompressionStrategy keyCompressor; + private SerializerDefinition keySerializerDefinition; + private SerializerDefinition valueSerializerDefinition; + + // Path path = new Path(fileName); + FSDataOutputStream outputStream; + + /** + * Create the voldemort key and value from the input key and value and map + * it out for each of the responsible voldemort nodes + * + * The output key is the md5 of the serialized key returned by makeKey(). + * The output value is the node_id & partition_id of the responsible node + * followed by serialized value returned by makeValue() OR if we have + * setKeys flag on the serialized key and serialized value + */ + @Override + public void map(GenericData.Record record, + AvroCollector> collector, + Reporter reporter) throws IOException { + + byte[] keyBytes = keySerializer.toBytes(record.get(keyField)); + byte[] valBytes = valueSerializer.toBytes(record.get(valField)); + + // Compress key and values if required + if(keySerializerDefinition.hasCompression()) { + keyBytes = keyCompressor.deflate(keyBytes); + } + + if(valueSerializerDefinition.hasCompression()) { + valBytes = valueCompressor.deflate(valBytes); + } + + // Get the output byte arrays ready to populate + byte[] outputValue; + BytesWritable outputKey; + + // Leave initial offset for (a) node id (b) partition id + // since they are written later + int offsetTillNow = 2 * ByteUtils.SIZE_OF_INT; + + if(getSaveKeys()) { + + // In order - 4 ( for node id ) + 4 ( partition id ) + 1 ( + // replica + // type - primary | secondary | tertiary... ] + 4 ( key size ) + // size ) + 4 ( value size ) + key + value + outputValue = new byte[valBytes.length + keyBytes.length + ByteUtils.SIZE_OF_BYTE + 4 + * ByteUtils.SIZE_OF_INT]; + + // Write key length - leave byte for replica type + offsetTillNow += ByteUtils.SIZE_OF_BYTE; + ByteUtils.writeInt(outputValue, keyBytes.length, offsetTillNow); + + // Write value length + offsetTillNow += ByteUtils.SIZE_OF_INT; + ByteUtils.writeInt(outputValue, valBytes.length, offsetTillNow); + + // Write key + offsetTillNow += ByteUtils.SIZE_OF_INT; + System.arraycopy(keyBytes, 0, outputValue, offsetTillNow, keyBytes.length); + + // Write value + offsetTillNow += keyBytes.length; + System.arraycopy(valBytes, 0, outputValue, offsetTillNow, valBytes.length); + + // Generate MR key - upper 8 bytes of 16 byte md5 + outputKey = new BytesWritable(ByteUtils.copy(md5er.digest(keyBytes), + 0, + 2 * ByteUtils.SIZE_OF_INT)); + + } else { + + // In order - 4 ( for node id ) + 4 ( partition id ) + value + outputValue = new byte[valBytes.length + 2 * ByteUtils.SIZE_OF_INT]; + + // Write value + System.arraycopy(valBytes, 0, outputValue, offsetTillNow, valBytes.length); + + // Generate MR key - 16 byte md5 + outputKey = new BytesWritable(md5er.digest(keyBytes)); + + } + + // Generate partition and node list this key is destined for + List partitionList = routingStrategy.getPartitionList(keyBytes); + Node[] partitionToNode = routingStrategy.getPartitionToNode(); + + for(int replicaType = 0; replicaType < partitionList.size(); replicaType++) { + + // Node id + ByteUtils.writeInt(outputValue, + partitionToNode[partitionList.get(replicaType)].getId(), + 0); + + if(getSaveKeys()) { + // Primary partition id + ByteUtils.writeInt(outputValue, partitionList.get(0), ByteUtils.SIZE_OF_INT); + + // Replica type + ByteUtils.writeBytes(outputValue, + replicaType, + 2 * ByteUtils.SIZE_OF_INT, + ByteUtils.SIZE_OF_BYTE); + } else { + // Partition id + ByteUtils.writeInt(outputValue, + partitionList.get(replicaType), + ByteUtils.SIZE_OF_INT); + } + BytesWritable outputVal = new BytesWritable(outputValue); + + // System.out.println("collect length (K/V): "+ + // outputKey.getLength()+ " , " + outputVal.getLength()); + ByteBuffer keyBuffer = null, valueBuffer = null; + + byte[] md5KeyBytes = outputKey.getBytes(); + keyBuffer = ByteBuffer.allocate(md5KeyBytes.length); + keyBuffer.put(md5KeyBytes); + keyBuffer.rewind(); + + valueBuffer = ByteBuffer.allocate(outputValue.length); + valueBuffer.put(outputValue); + valueBuffer.rewind(); + + Pair p = new Pair(keyBuffer, + valueBuffer); + + collector.collect(p); + } + md5er.reset(); + } + + @Override + public void configure(JobConf conf) { + + super.setConf(conf); + // from parent code + + md5er = ByteUtils.getDigest("md5"); + + this.cluster = new ClusterMapper().readCluster(new StringReader(conf.get("cluster.xml"))); + List storeDefs = new StoreDefinitionsMapper().readStoreList(new StringReader(conf.get("stores.xml"))); + if(storeDefs.size() != 1) + throw new IllegalStateException("Expected to find only a single store, but found multiple!"); + this.storeDef = storeDefs.get(0); + + this.numChunks = conf.getInt("num.chunks", -1); + if(this.numChunks < 1) + throw new VoldemortException("num.chunks not specified in the job conf."); + + this.saveKeys = conf.getBoolean("save.keys", true); + this.reducerPerBucket = conf.getBoolean("reducer.per.bucket", false); + + keySerializerDefinition = getStoreDef().getKeySerializer(); + valueSerializerDefinition = getStoreDef().getValueSerializer(); + + try { + SerializerFactory factory = new DefaultSerializerFactory(); + + if(conf.get("serializer.factory") != null) { + factory = (SerializerFactory) Class.forName(conf.get("serializer.factory")) + .newInstance(); + } + + keyField = conf.get("avro.key.field"); + valField = conf.get("avro.value.field"); + + keySchema = conf.get("avro.key.schema"); + valSchema = conf.get("avro.val.schema"); + + // hadoop.job.valueSchema + keySerializer = new AvroGenericSerializer(keySchema); + valueSerializer = new AvroGenericSerializer(valSchema); + } catch(Exception e) { + throw new RuntimeException(e); + } + + keyCompressor = new CompressionStrategyFactory().get(keySerializerDefinition.getCompression()); + valueCompressor = new CompressionStrategyFactory().get(valueSerializerDefinition.getCompression()); + + routingStrategy = new ConsistentRoutingStrategy(getCluster().getNodes(), + getStoreDef().getReplicationFactor()); + + // / + Props props = HadoopUtils.getPropsFromJob(conf); + + _keySelection = props.getString("key.selection", null); + _valSelection = props.getString("value.selection", null); + + String _keyTransClass = props.getString("key.transformation.class", null); + String _valueTransClass = props.getString("value.transformation.class", null); + + if(_keyTransClass != null) + _keyTrans = (StoreBuilderTransformation) Utils.callConstructor(_keyTransClass); + if(_valueTransClass != null) + _valTrans = (StoreBuilderTransformation) Utils.callConstructor(_valueTransClass); + } + + private int numChunks; + private Cluster cluster; + private StoreDefinition storeDef; + private boolean saveKeys; + private boolean reducerPerBucket; + + public Cluster getCluster() { + checkNotNull(cluster); + return cluster; + } + + public boolean getSaveKeys() { + return this.saveKeys; + } + + public boolean getReducerPerBucket() { + return this.reducerPerBucket; + } + + public StoreDefinition getStoreDef() { + checkNotNull(storeDef); + return storeDef; + } + + public String getStoreName() { + checkNotNull(storeDef); + return storeDef.getName(); + } + + private final void checkNotNull(Object o) { + if(o == null) + throw new VoldemortException("Not configured yet!"); + } + + public int getNumChunks() { + return this.numChunks; + } + +} \ No newline at end of file diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderPartitioner.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderPartitioner.java new file mode 100644 index 0000000000..6c1c5a9dc2 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderPartitioner.java @@ -0,0 +1,154 @@ +package voldemort.store.readonly.mr; + +/* + * Copyright 2008-2009 LinkedIn, Inc + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +import java.io.IOException; +import java.io.StringReader; +import java.nio.ByteBuffer; +import java.util.List; + +import org.apache.avro.mapred.AvroKey; +import org.apache.avro.mapred.AvroValue; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Partitioner; + +import voldemort.VoldemortException; +import voldemort.cluster.Cluster; +import voldemort.store.StoreDefinition; +import voldemort.store.readonly.ReadOnlyUtils; +import voldemort.utils.ByteUtils; +import voldemort.xml.ClusterMapper; +import voldemort.xml.StoreDefinitionsMapper; + +/** + * A Partitioner that splits data so that all data for the same nodeId, chunkId + * combination ends up in the same reduce (and hence in the same store chunk) + */ +@SuppressWarnings("deprecation") +public class AvroStoreBuilderPartitioner implements + Partitioner, AvroValue> { + + @Override + public int getPartition(AvroKey key, AvroValue value, int numReduceTasks) { + + byte[] keyBytes = null, valueBytes; + + keyBytes = new byte[key.datum().remaining()]; + key.datum().get(keyBytes); + + valueBytes = new byte[value.datum().remaining()]; + value.datum().get(valueBytes); + + BytesWritable outputKey = new BytesWritable(keyBytes); + BytesWritable outputVal = new BytesWritable(valueBytes); + + ByteBuffer keyBuffer = null, valueBuffer = null; + + keyBuffer = ByteBuffer.allocate(keyBytes.length); + keyBuffer.put(keyBytes); + keyBuffer.rewind(); + + valueBuffer = ByteBuffer.allocate(valueBytes.length); + valueBuffer.put(valueBytes); + valueBuffer.rewind(); + + key.datum(keyBuffer); + value.datum(valueBuffer); + + int partitionId = ByteUtils.readInt(valueBytes, ByteUtils.SIZE_OF_INT); + int chunkId = ReadOnlyUtils.chunk(keyBytes, getNumChunks()); + if(getSaveKeys()) { + int replicaType = (int) ByteUtils.readBytes(valueBytes, + 2 * ByteUtils.SIZE_OF_INT, + ByteUtils.SIZE_OF_BYTE); + if(getReducerPerBucket()) { + return (partitionId * getStoreDef().getReplicationFactor() + replicaType) + % numReduceTasks; + } else { + return ((partitionId * getStoreDef().getReplicationFactor() * getNumChunks()) + + (replicaType * getNumChunks()) + chunkId) + % numReduceTasks; + } + } else { + if(getReducerPerBucket()) { + return partitionId % numReduceTasks; + } else { + return (partitionId * getNumChunks() + chunkId) % numReduceTasks; + } + + } + } + + private int numChunks; + private Cluster cluster; + private StoreDefinition storeDef; + private boolean saveKeys; + private boolean reducerPerBucket; + + @Override + public void configure(JobConf conf) { + this.cluster = new ClusterMapper().readCluster(new StringReader(conf.get("cluster.xml"))); + List storeDefs = new StoreDefinitionsMapper().readStoreList(new StringReader(conf.get("stores.xml"))); + if(storeDefs.size() != 1) + throw new IllegalStateException("Expected to find only a single store, but found multiple!"); + this.storeDef = storeDefs.get(0); + + this.numChunks = conf.getInt("num.chunks", -1); + if(this.numChunks < 1) + throw new VoldemortException("num.chunks not specified in the job conf."); + + this.saveKeys = conf.getBoolean("save.keys", false); + this.reducerPerBucket = conf.getBoolean("reducer.per.bucket", false); + } + + @SuppressWarnings("unused") + public void close() throws IOException {} + + public Cluster getCluster() { + checkNotNull(cluster); + return cluster; + } + + public boolean getSaveKeys() { + return this.saveKeys; + } + + public boolean getReducerPerBucket() { + return this.reducerPerBucket; + } + + public StoreDefinition getStoreDef() { + checkNotNull(storeDef); + return storeDef; + } + + public String getStoreName() { + checkNotNull(storeDef); + return storeDef.getName(); + } + + private final void checkNotNull(Object o) { + if(o == null) + throw new VoldemortException("Not configured yet!"); + } + + public int getNumChunks() { + return this.numChunks; + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderReducer.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderReducer.java new file mode 100644 index 0000000000..6a8be8662b --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderReducer.java @@ -0,0 +1,92 @@ +package voldemort.store.readonly.mr; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Iterator; + +import org.apache.avro.mapred.AvroKey; +import org.apache.avro.mapred.AvroValue; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.JobConfigurable; +import org.apache.hadoop.mapred.OutputCollector; +import org.apache.hadoop.mapred.Reducer; +import org.apache.hadoop.mapred.Reporter; + +import voldemort.store.readonly.disk.HadoopStoreWriter; +import voldemort.store.readonly.disk.KeyValueWriter; +import azkaban.common.utils.Utils; + +public class AvroStoreBuilderReducer implements + Reducer, AvroValue, Text, Text>, JobConfigurable, Closeable { + + String keyValueWriterClass; + @SuppressWarnings("rawtypes") + KeyValueWriter writer; + + @SuppressWarnings("unchecked") + @Override + public void reduce(AvroKey keyAvro, + Iterator> iterator, + OutputCollector collector, + Reporter reporter) throws IOException { + + ByteBuffer keyBuffer = keyAvro.datum(); + keyBuffer.rewind(); + + byte[] keyBytes = null, valueBytes; + + keyBytes = new byte[keyBuffer.remaining()]; + keyBuffer.get(keyBytes); + + BytesWritable key = new BytesWritable(keyBytes); + + ArrayList valueList = new ArrayList(); + + while(iterator.hasNext()) { + ByteBuffer writable = iterator.next().datum(); + writable.rewind(); + // BytesWritable writable = iterator.next(); + valueBytes = null; + valueBytes = new byte[writable.remaining()]; + writable.get(valueBytes); + + BytesWritable value = new BytesWritable(valueBytes); + valueList.add(value); + + } + + writer.write(key, valueList.iterator(), reporter); + + } + + @Override + public void configure(JobConf job) { + + JobConf conf = job; + try { + + keyValueWriterClass = conf.get("writer.class"); + if(keyValueWriterClass != null) + writer = (KeyValueWriter) Utils.callConstructor(keyValueWriterClass); + else + writer = new HadoopStoreWriter(); + + writer.conf(job); + + } catch(Exception e) { + // throw new RuntimeException("Failed to open Input/OutputStream", + // e); + e.printStackTrace(); + } + } + + @Override + public void close() throws IOException { + + writer.close(); + } +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderReducerPerBucket.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderReducerPerBucket.java new file mode 100644 index 0000000000..d1f53d678b --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/AvroStoreBuilderReducerPerBucket.java @@ -0,0 +1,94 @@ +package voldemort.store.readonly.mr; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Iterator; + +import org.apache.avro.mapred.AvroKey; +import org.apache.avro.mapred.AvroValue; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.Closeable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.JobConfigurable; +import org.apache.hadoop.mapred.OutputCollector; +import org.apache.hadoop.mapred.Reducer; +import org.apache.hadoop.mapred.Reporter; +import org.apache.log4j.Logger; + +import voldemort.store.readonly.disk.HadoopStoreWriterPerBucket; +import voldemort.store.readonly.disk.KeyValueWriter; +import azkaban.common.utils.Utils; + +public class AvroStoreBuilderReducerPerBucket implements + Reducer, AvroValue, Text, Text>, JobConfigurable, Closeable { + + private static final Logger logger = Logger.getLogger(AvroStoreBuilderReducerPerBucket.class); + + String keyValueWriterClass; + @SuppressWarnings("rawtypes") + KeyValueWriter writer; + + @Override + public void reduce(AvroKey keyAvro, + Iterator> iterator, + OutputCollector collector, + Reporter reporter) throws IOException { + + ByteBuffer keyBuffer = keyAvro.datum(); + keyBuffer.rewind(); + + byte[] keyBytes = null, valueBytes; + + keyBytes = new byte[keyBuffer.remaining()]; + keyBuffer.get(keyBytes); + + BytesWritable key = new BytesWritable(keyBytes); + + ArrayList valueList = new ArrayList(); + + while(iterator.hasNext()) { + ByteBuffer writable = iterator.next().datum(); + writable.rewind(); + // BytesWritable writable = iterator.next(); + valueBytes = null; + valueBytes = new byte[writable.remaining()]; + writable.get(valueBytes); + + BytesWritable value = new BytesWritable(valueBytes); + valueList.add(value); + + } + + writer.write(key, valueList.iterator(), reporter); + + } + + @Override + public void configure(JobConf job) { + + JobConf conf = job; + try { + + keyValueWriterClass = conf.get("writer.class"); + if(keyValueWriterClass != null) + writer = (KeyValueWriter) Utils.callConstructor(keyValueWriterClass); + else + writer = new HadoopStoreWriterPerBucket(); + + writer.conf(job); + + } catch(Exception e) { + // throw new RuntimeException("Failed to open Input/OutputStream", + // e); + e.printStackTrace(); + } + } + + @Override + public void close() throws IOException { + + writer.close(); + } +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/IdentityJsonMapper.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/IdentityJsonMapper.java new file mode 100644 index 0000000000..3ea7783ca7 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/IdentityJsonMapper.java @@ -0,0 +1,20 @@ +package voldemort.store.readonly.mr; + +import java.io.IOException; + +import org.apache.hadoop.mapred.OutputCollector; +import org.apache.hadoop.mapred.Reporter; + +import voldemort.store.readonly.mr.serialization.JsonMapper; + +public class IdentityJsonMapper extends JsonMapper { + + @Override + public void mapObjects(Object key, + Object value, + OutputCollector output, + Reporter reporter) throws IOException { + output.collect(key, value); + } + +} \ No newline at end of file diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/IdentityJsonReducer.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/IdentityJsonReducer.java new file mode 100644 index 0000000000..e4c7f3a21e --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/IdentityJsonReducer.java @@ -0,0 +1,22 @@ +package voldemort.store.readonly.mr; + +import java.io.IOException; +import java.util.Iterator; + +import org.apache.hadoop.mapred.OutputCollector; +import org.apache.hadoop.mapred.Reporter; + +import voldemort.store.readonly.mr.serialization.JsonReducer; + +public class IdentityJsonReducer extends JsonReducer { + + @Override + public void reduceObjects(Object key, + Iterator values, + OutputCollector collector, + Reporter reporter) throws IOException { + while(values.hasNext()) { + collector.collect(key, values.next()); + } + } +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/VoldemortStoreBuilderMapper.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/VoldemortStoreBuilderMapper.java new file mode 100644 index 0000000000..d0473d4bfa --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/VoldemortStoreBuilderMapper.java @@ -0,0 +1,73 @@ +package voldemort.store.readonly.mr; + +import java.util.Map; + +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.mapred.JobConf; + +import voldemort.serialization.json.JsonTypeSerializer; +import voldemort.store.readonly.mr.azkaban.StoreBuilderTransformation; +import voldemort.store.readonly.mr.utils.HadoopUtils; +import azkaban.common.utils.Props; +import azkaban.common.utils.Utils; + +public class VoldemortStoreBuilderMapper extends AbstractHadoopStoreBuilderMapper { + + private String _keySelection; + private String _valSelection; + private JsonTypeSerializer _inputKeySerializer; + private JsonTypeSerializer _inputValueSerializer; + private StoreBuilderTransformation _keyTrans; + private StoreBuilderTransformation _valTrans; + + @Override + public Object makeKey(Object key, Object value) { + return makeResult((BytesWritable) key, _inputKeySerializer, _keySelection, _keyTrans); + } + + @Override + public Object makeValue(Object key, Object value) { + return makeResult((BytesWritable) value, _inputValueSerializer, _valSelection, _valTrans); + } + + private Object makeResult(BytesWritable writable, + JsonTypeSerializer serializer, + String selection, + StoreBuilderTransformation trans) { + Object obj = serializer.toObject(writable.get()); + if(selection != null) { + Map m = (Map) obj; + obj = m.get(selection); + } + + if(trans != null) + obj = trans.transform(obj); + + return obj; + } + + @Override + public void configure(JobConf conf) { + super.configure(conf); + Props props = HadoopUtils.getPropsFromJob(conf); + + _keySelection = props.getString("key.selection", null); + _valSelection = props.getString("value.selection", null); + _inputKeySerializer = getSchemaFromJob(conf, "mapper.input.key.schema"); + _inputValueSerializer = getSchemaFromJob(conf, "mapper.input.value.schema"); + String _keyTransClass = props.getString("key.transformation.class", null); + String _valueTransClass = props.getString("value.transformation.class", null); + + if(_keyTransClass != null) + _keyTrans = (StoreBuilderTransformation) Utils.callConstructor(_keyTransClass); + if(_valueTransClass != null) + _valTrans = (StoreBuilderTransformation) Utils.callConstructor(_valueTransClass); + } + + protected JsonTypeSerializer getSchemaFromJob(JobConf conf, String key) { + if(conf.get(key) == null) + throw new IllegalArgumentException("Missing required parameter '" + key + "' on job."); + return new JsonTypeSerializer(conf.get(key)); + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/AbstractHadoopJob.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/AbstractHadoopJob.java new file mode 100644 index 0000000000..afd8bc34e3 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/AbstractHadoopJob.java @@ -0,0 +1,268 @@ +package voldemort.store.readonly.mr.azkaban; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Arrays; +import java.util.List; + +import org.apache.hadoop.filecache.DistributedCache; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.mapred.Counters; +import org.apache.hadoop.mapred.Counters.Counter; +import org.apache.hadoop.mapred.FileOutputFormat; +import org.apache.hadoop.mapred.JobClient; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Mapper; +import org.apache.hadoop.mapred.Reducer; +import org.apache.hadoop.mapred.RunningJob; +import org.joda.time.DateTime; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; + +import voldemort.store.readonly.mr.IdentityJsonReducer; +import voldemort.store.readonly.mr.utils.HadoopUtils; +import azkaban.common.jobs.AbstractJob; +import azkaban.common.utils.Props; + +/** + * An abstract Base class for Hadoop Jobs + * + * @author bbansal + * + */ +public abstract class AbstractHadoopJob extends AbstractJob { + + public static String COMMON_FILE_DATE_PATTERN = "yyyy-MM-dd-HH-mm"; + public static final String HADOOP_PREFIX = "hadoop-conf."; + public static final String LATEST_SUFFIX = "#LATEST"; + public static final String CURRENT_SUFFIX = "#CURRENT"; + private final Props _props; + private RunningJob _runningJob; + + public AbstractHadoopJob(String name, Props props) { + super(name); + this._props = props; + } + + public void run(JobConf conf) throws Exception { + _runningJob = new JobClient(conf).submitJob(conf); + info("See " + _runningJob.getTrackingURL() + " for details."); + _runningJob.waitForCompletion(); + + if(!_runningJob.isSuccessful()) { + throw new Exception("Hadoop job:" + getId() + " failed!"); + } + + // dump all counters + Counters counters = _runningJob.getCounters(); + for(String groupName: counters.getGroupNames()) { + Counters.Group group = counters.getGroup(groupName); + info("Group: " + group.getDisplayName()); + for(Counter counter: group) + info(counter.getDisplayName() + ":\t" + counter.getValue()); + } + } + + public JobConf createJobConf(Class mapperClass) throws IOException, + URISyntaxException { + JobConf conf = createJobConf(mapperClass, IdentityJsonReducer.class); + conf.setNumReduceTasks(0); + + return conf; + } + + public JobConf createJobConf(Class mapperClass, + Class reducerClass, + Class combinerClass) throws IOException, + URISyntaxException { + JobConf conf = createJobConf(mapperClass, reducerClass); + conf.setCombinerClass(combinerClass); + + return conf; + } + + public JobConf createJobConf(Class mapperClass, + Class reducerClass) throws IOException, + URISyntaxException { + JobConf conf = new JobConf(); + // set custom class loader with custom find resource strategy. + + conf.setJobName(getId()); + conf.setMapperClass(mapperClass); + conf.setReducerClass(reducerClass); + + String hadoop_ugi = _props.getString("hadoop.job.ugi", null); + if(hadoop_ugi != null) { + conf.set("hadoop.job.ugi", hadoop_ugi); + } + + if(_props.getBoolean("is.local", false)) { + conf.set("mapred.job.tracker", "local"); + conf.set("fs.default.name", "file:///"); + conf.set("mapred.local.dir", "/tmp/map-red"); + + info("Running locally, no hadoop jar set."); + } else { + setClassLoaderAndJar(conf, getClass()); + info("Setting hadoop jar file for class:" + getClass() + " to " + conf.getJar()); + info("*************************************************************************"); + info(" Running on Real Hadoop Cluster(" + conf.get("mapred.job.tracker") + + ") "); + info("*************************************************************************"); + } + + // set JVM options if present + if(_props.containsKey("mapred.child.java.opts")) { + conf.set("mapred.child.java.opts", _props.getString("mapred.child.java.opts")); + info("mapred.child.java.opts set to " + _props.getString("mapred.child.java.opts")); + } + + // set input and output paths if they are present + if(_props.containsKey("input.paths")) { + List inputPaths = _props.getStringList("input.paths"); + if(inputPaths.size() == 0) + throw new IllegalArgumentException("Must specify at least one value for property 'input.paths'"); + for(String path: inputPaths) { + // Implied stuff, but good implied stuff + if(path.endsWith(LATEST_SUFFIX)) { + FileSystem fs = FileSystem.get(conf); + + PathFilter filter = new PathFilter() { + + @Override + public boolean accept(Path arg0) { + return !arg0.getName().startsWith("_") + && !arg0.getName().startsWith("."); + } + }; + + String latestPath = path.substring(0, path.length() - LATEST_SUFFIX.length()); + FileStatus[] statuses = fs.listStatus(new Path(latestPath), filter); + + Arrays.sort(statuses); + + path = statuses[statuses.length - 1].getPath().toString(); + System.out.println("Using latest folder: " + path); + } + HadoopUtils.addAllSubPaths(conf, new Path(path)); + } + } + + if(_props.containsKey("output.path")) { + String location = _props.get("output.path"); + if(location.endsWith("#CURRENT")) { + DateTimeFormatter format = DateTimeFormat.forPattern(COMMON_FILE_DATE_PATTERN); + String destPath = format.print(new DateTime()); + location = location.substring(0, location.length() - "#CURRENT".length()) + + destPath; + System.out.println("Store location set to " + location); + } + + FileOutputFormat.setOutputPath(conf, new Path(location)); + // For testing purpose only remove output file if exists + if(_props.getBoolean("force.output.overwrite", false)) { + FileSystem fs = FileOutputFormat.getOutputPath(conf).getFileSystem(conf); + fs.delete(FileOutputFormat.getOutputPath(conf), true); + } + } + + // Adds External jars to hadoop classpath + String externalJarList = _props.getString("hadoop.external.jarFiles", null); + if(externalJarList != null) { + String[] jarFiles = externalJarList.split(","); + for(String jarFile: jarFiles) { + info("Adding extenral jar File:" + jarFile); + DistributedCache.addFileToClassPath(new Path(jarFile), conf); + } + } + + // Adds distributed cache files + String cacheFileList = _props.getString("hadoop.cache.files", null); + if(cacheFileList != null) { + String[] cacheFiles = cacheFileList.split(","); + for(String cacheFile: cacheFiles) { + info("Adding Distributed Cache File:" + cacheFile); + DistributedCache.addCacheFile(new URI(cacheFile), conf); + } + } + + // Adds distributed cache files + String archiveFileList = _props.getString("hadoop.cache.archives", null); + if(archiveFileList != null) { + String[] archiveFiles = archiveFileList.split(","); + for(String archiveFile: archiveFiles) { + info("Adding Distributed Cache Archive File:" + archiveFile); + DistributedCache.addCacheArchive(new URI(archiveFile), conf); + } + } + + String hadoopCacheJarDir = _props.getString("hdfs.default.classpath.dir", null); + if(hadoopCacheJarDir != null) { + FileSystem fs = FileSystem.get(conf); + if(fs != null) { + FileStatus[] status = fs.listStatus(new Path(hadoopCacheJarDir)); + + if(status != null) { + for(int i = 0; i < status.length; ++i) { + if(!status[i].isDir()) { + Path path = new Path(hadoopCacheJarDir, status[i].getPath().getName()); + info("Adding Jar to Distributed Cache Archive File:" + path); + + DistributedCache.addFileToClassPath(path, conf); + } + } + } else { + info("hdfs.default.classpath.dir " + hadoopCacheJarDir + " is empty."); + } + } else { + info("hdfs.default.classpath.dir " + hadoopCacheJarDir + + " filesystem doesn't exist"); + } + } + + // May want to add this to HadoopUtils, but will await refactoring + for(String key: getProps().keySet()) { + String lowerCase = key.toLowerCase(); + if(lowerCase.startsWith(HADOOP_PREFIX)) { + String newKey = key.substring(HADOOP_PREFIX.length()); + conf.set(newKey, getProps().get(key)); + } + } + + HadoopUtils.setPropsInJob(conf, getProps()); + return conf; + } + + public Props getProps() { + return this._props; + } + + public void cancel() throws Exception { + if(_runningJob != null) + _runningJob.killJob(); + } + + public double getProgress() throws IOException { + if(_runningJob == null) + return 0.0; + else + return (double) (_runningJob.mapProgress() + _runningJob.reduceProgress()) / 2.0d; + } + + public Counters getCounters() throws IOException { + return _runningJob.getCounters(); + } + + public static void setClassLoaderAndJar(JobConf conf, Class jobClass) { + conf.setClassLoader(Thread.currentThread().getContextClassLoader()); + String jar = HadoopUtils.findContainingJar(jobClass, Thread.currentThread() + .getContextClassLoader()); + if(jar != null) { + conf.setJar(jar); + } + } +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/AbstractVoldemortBatchCopyJob.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/AbstractVoldemortBatchCopyJob.java new file mode 100644 index 0000000000..3e6860ed04 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/AbstractVoldemortBatchCopyJob.java @@ -0,0 +1,116 @@ +package voldemort.store.readonly.mr.azkaban; + +import java.io.IOException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.JobConf; + +import voldemort.cluster.Cluster; +import voldemort.cluster.Node; +import voldemort.store.readonly.mr.utils.HadoopUtils; +import azkaban.common.jobs.AbstractJob; +import azkaban.common.utils.Props; + +/** + * A test job that throws an exception + * + * @author bbansal Required Properties + *
    + *
  • voldemort.cluster.file
  • + *
  • voldemort.store.name
  • + *
  • input.path
  • + *
  • dest.path
  • + *
  • source.host
  • + *
  • dest.host
  • + *
+ */ +public abstract class AbstractVoldemortBatchCopyJob extends AbstractJob { + + private final Props _props; + + public AbstractVoldemortBatchCopyJob(String name, Props props) throws IOException { + super(name); + _props = props; + } + + public void run() throws Exception { + JobConf conf = new JobConf(); + HadoopUtils.copyInAllProps(_props, conf); + + Cluster cluster = HadoopUtils.readCluster(_props.get("voldemort.cluster.file"), conf); + final String storeName = _props.get("voldemort.store.name"); + final Path inputDir = new Path(_props.get("input.path")); + + ExecutorService executors = Executors.newFixedThreadPool(cluster.getNumberOfNodes()); + final Semaphore semaphore = new Semaphore(0, false); + final AtomicInteger countSuccess = new AtomicInteger(0); + final boolean[] succeeded = new boolean[cluster.getNumberOfNodes()]; + final String destinationDir = _props.get("dest.path"); + final String sourceHost = _props.getString("src.host", "localhost"); + + for(final Node node: cluster.getNodes()) { + + executors.execute(new Runnable() { + + public void run() { + int id = node.getId(); + String indexFile = inputDir + "/" + storeName + ".index" + "_" + + Integer.toString(id); + String dataFile = inputDir + "/" + storeName + ".data" + "_" + + Integer.toString(id); + + String host = node.getHost(); + try { + // copyFileToLocal(sourceHost, + // indexFile, + // host, + // VoldemortSwapperUtils.getIndexDestinationFile(node.getId(), + // destinationDir)); + // copyFileToLocal(sourceHost, + // dataFile, + // host, + // VoldemortSwapperUtils.getDataDestinationFile(node.getId(), + // destinationDir)); + + succeeded[node.getId()] = true; + countSuccess.incrementAndGet(); + } catch(Exception e) { + error("copy to Remote node failed for node:" + node.getId(), e); + } + + semaphore.release(); + } + }); + } + + // wait for all operations to complete + semaphore.acquire(cluster.getNumberOfNodes()); + + try { + if(countSuccess.get() == cluster.getNumberOfNodes() + || _props.getBoolean("swap.partial.index", false)) { + int counter = 0; + // lets try to swap only the successful nodes + for(Node node: cluster.getNodes()) { + // data refresh succeeded + if(succeeded[node.getId()]) { + VoldemortSwapperUtils.doSwap(storeName, node, destinationDir); + counter++; + } + } + info(counter + " node out of " + cluster.getNumberOfNodes() + + " refreshed with fresh index/data for store '" + storeName + "'"); + } else { + error("Failed to copy Index Files for the entire cluster."); + } + } finally { + // stop all executors Now + executors.shutdown(); + } + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/Job.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/Job.java new file mode 100644 index 0000000000..91ba74370e --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/Job.java @@ -0,0 +1,59 @@ +package voldemort.store.readonly.mr.azkaban; + +import java.util.Properties; + +/** + * This interface defines a Raw Job interface. Each job defines + *
    + *
  • Job Type : {HADOOP, UNIX, JAVA, SUCCESS_TEST, CONTROLLER}
  • + *
  • Job ID/Name : {String}
  • + *
  • Arguments: Key/Value Map for Strings
  • + *
+ * + * A job is required to have a constructor Job(String jobId, Props props) + */ + +public interface Job { + + /** + * Returns a unique(should be checked in xml) string name/id for the Job. + * + * @return + */ + public String getId(); + + /** + * Run the job. In general this method can only be run once. Must either + * succeed or throw an exception. + */ + public void run() throws Exception; + + /** + * Best effort attempt to cancel the job. + * + * @throws Exception If cancel fails + */ + public void cancel() throws Exception; + + /** + * Returns a progress report between [0 - 1.0] to indicate the percentage + * complete + * + * @throws Exception If getting progress fails + */ + public double getProgress() throws Exception; + + /** + * Get the generated properties from this job. + * + * @return + */ + public Properties getJobGeneratedProperties(); + + /** + * Determine if the job was cancelled. + * + * @return + */ + public boolean isCanceled(); +} \ No newline at end of file diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/StoreBuilderTransformation.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/StoreBuilderTransformation.java new file mode 100644 index 0000000000..6703f6879a --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/StoreBuilderTransformation.java @@ -0,0 +1,14 @@ +package voldemort.store.readonly.mr.azkaban; + +/** + * An interface to use for processing rows in the voldemort store builder + * + * @author jkreps + * + */ +public interface StoreBuilderTransformation +{ + + public Object transform(Object obj); + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/UndefinedPropertyException.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/UndefinedPropertyException.java new file mode 100644 index 0000000000..cbfe09504b --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/UndefinedPropertyException.java @@ -0,0 +1,11 @@ +package voldemort.store.readonly.mr.azkaban; + +public class UndefinedPropertyException extends RuntimeException { + + private static final long serialVersionUID = 1; + + public UndefinedPropertyException(String message) { + super(message); + } + +} diff --git a/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortBatchIndexJob.java b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortBatchIndexJob.java new file mode 100644 index 0000000000..cbe1d28f78 --- /dev/null +++ b/contrib/hadoop-store-builder/src/java/voldemort/store/readonly/mr/azkaban/VoldemortBatchIndexJob.java @@ -0,0 +1,405 @@ +package voldemort.store.readonly.mr.azkaban; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URISyntaxException; +import java.util.Iterator; +import java.util.List; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.FileInputFormat; +import org.apache.hadoop.mapred.FileOutputFormat; +import org.apache.hadoop.mapred.JobClient; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.Mapper; +import org.apache.hadoop.mapred.OutputCollector; +import org.apache.hadoop.mapred.Reducer; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapred.SequenceFileInputFormat; +import org.apache.hadoop.mapred.SequenceFileOutputFormat; +import org.apache.hadoop.mapred.lib.HashPartitioner; +import org.apache.log4j.Logger; + +import voldemort.cluster.Cluster; +import voldemort.cluster.Node; +import voldemort.routing.ConsistentRoutingStrategy; +import voldemort.serialization.DefaultSerializerFactory; +import voldemort.serialization.Serializer; +import voldemort.store.StoreDefinition; +import voldemort.store.readonly.mr.serialization.JsonConfigurable; +import voldemort.store.readonly.mr.utils.HadoopUtils; +import voldemort.utils.ByteUtils; +import azkaban.common.utils.Props; + +/** + * Creates Index and value files using Voldemort hash keys for easy batch + * update. + *

+ * Creates two files + *