Browse files

Refactor TimeoutConfig

  • Loading branch information...
1 parent 10211c7 commit 35f56811cf23b5f04326a5f22be4824b1705b6bb @vinothchandar vinothchandar committed Jun 20, 2012
View
21 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));
View
56 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<VoldemortOperation, Long> 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<VoldemortOperation, Long>();
+ 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;
+ }
+
+}
View
14 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
+}
View
34 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);
View
31 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 PipelineRoutedStore(String name,
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 PipelineRoutedStore(String name,
failureDetector,
storeDef.getPreferredReads(),
storeDef.getRequiredReads(),
- timeoutConfig.getTimeoutMs(),
+ timeoutConfig.getOperationTimeout(VoldemortOperation.GET),
nonblockingStores,
Event.INSUFFICIENT_SUCCESSES,
Event.INSUFFICIENT_ZONES));
@@ -201,7 +202,7 @@ public PipelineRoutedStore(String name,
new ReadRepair<BasicPipelineData<List<Versioned<byte[]>>>>(pipelineData,
Event.COMPLETED,
storeDef.getPreferredReads(),
- timeoutConfig.getTimeoutMs(),
+ timeoutConfig.getOperationTimeout(VoldemortOperation.GET),
nonblockingStores,
readRepairer));
@@ -259,7 +260,7 @@ public PipelineRoutedStore(String name,
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 PipelineRoutedStore(String name,
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 PipelineRoutedStore(String name,
new GetAllReadRepair(pipelineData,
Event.COMPLETED,
storeDef.getPreferredReads(),
- timeoutConfig.getAllTimeoutMs(),
+ timeoutConfig.getOperationTimeout(VoldemortOperation.GETALL),
nonblockingStores,
readRepairer));
@@ -330,7 +331,7 @@ public PipelineRoutedStore(String name,
pipelineData.setZonesRequired(null);
pipelineData.setStats(stats);
Pipeline pipeline = new Pipeline(Operation.GET_VERSIONS,
- timeoutConfig.getVersionsTimeoutMs(),
+ timeoutConfig.getOperationTimeout(VoldemortOperation.GETVERSIONS),
TimeUnit.MILLISECONDS);
StoreRequest<List<Version>> blockingStoreRequest = new StoreRequest<List<Version>>() {
@@ -357,7 +358,7 @@ public PipelineRoutedStore(String name,
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<Boolean, BasicPipelineData<Boolean>>(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<byte[]> 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<byte[]> versioned, byte[] transforms)
nonblockingSlopStores,
handoffStrategy,
pipelineData.getFailedNodes(),
- timeoutConfig.putTimeoutMs());
+ timeoutConfig.getOperationTimeout(VoldemortOperation.PUT));
pipeline.addEventAction(Event.STARTED,
new ConfigureNodes<Void, PutPipelineData>(pipelineData,
@@ -549,7 +550,7 @@ public void put(ByteArray key, Versioned<byte[]> versioned, byte[] transforms)
failureDetector,
storeDef.getPreferredWrites(),
storeDef.getRequiredWrites(),
- timeoutConfig.putTimeoutMs(),
+ timeoutConfig.getOperationTimeout(VoldemortOperation.PUT),
nonblockingStores,
hintedHandoff));
if(isHintedHandoffEnabled()) {
View
2 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;
/**
View
2 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;
View
14 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 void run() {
keyToSuccessCount.put(key, new MutableInt(0));
List<Future<GetAllResult>> 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 @@ public Void apply(List<GetResult<Versioned<byte[]>>> nodeResults) {
}
List<Future<GetResult<R>>> 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) {
View
5 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
View
114 src/java/voldemort/utils/TimeoutConfig.java
@@ -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;
- }
-
-}
View
2 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
View
2 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;
View
2 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;
View
8 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);

0 comments on commit 35f5681

Please sign in to comment.