From 9785e684d183fbebe99c35e517a3e60333b5f5b6 Mon Sep 17 00:00:00 2001 From: Chinmay Soman Date: Fri, 8 Mar 2013 14:23:49 -0800 Subject: [PATCH] A working implementation of the Coordinator and thin client. Includes following things: - Creating AbstractStore and AbstractStorageEngine to refactor the corresponding Store and StorageEngine interfaces. - Refactored the fat client to accomodate dynamic per call timeout. - Isolated Fat client wrapper to safeguard multitenancy - Autobootstrap mechanism added to the Coordinator service - Basic HTTP request/response parsing and Error handling --- .../store/krati/KratiStorageEngine.java | 56 ++-- src/java/voldemort/client/ClientConfig.java | 9 + .../CoordinatorPipelineFactory.java | 24 +- .../coordinator/CoordinatorService.java | 62 +++- .../DynamicTimeoutStoreClient.java | 177 ++++++++++++ .../coordinator/FatClientWrapper.java | 133 ++++----- .../coordinator/GetRequestExecutor.java | 102 +++++-- .../coordinator/HttpResponseHandler.java | 77 ----- .../coordinator/NoopHttpRequestHandler.java | 40 ++- .../coordinator/PutRequestExecutor.java | 131 +++++++++ .../voldemort/coordinator/R2StoreWrapper.java | 221 -------------- .../voldemort/coordinator/RESTClient.java | 29 +- .../coordinator/RESTErrorHandler.java | 51 ++-- .../coordinator/SampleRESTClient.java | 19 +- .../coordinator/VectorClockWrapper.java | 22 ++ .../VoldemortHttpRequestHandler.java | 269 ++++++++++-------- .../store/AbstractStorageEngine.java | 57 ++++ src/java/voldemort/store/AbstractStore.java | 75 +++++ src/java/voldemort/store/DelegatingStore.java | 37 ++- src/java/voldemort/store/Store.java | 45 +++ .../store/VoldemortRequestWrapper.java | 80 ++++++ .../voldemort/store/bdb/BdbStorageEngine.java | 44 ++- .../PartitionPrefixedBdbStorageEngine.java | 4 + .../store/compress/CompressingStore.java | 15 +- .../ConfigurationStorageEngine.java | 44 +-- .../FileBackedCachingStorageEngine.java | 55 ++-- src/java/voldemort/store/gzip/GzipStore.java | 3 +- src/java/voldemort/store/http/HttpStore.java | 48 ++-- .../voldemort/store/logging/LoggingStore.java | 30 ++ .../store/memory/InMemoryStorageEngine.java | 53 ++-- .../store/metadata/MetadataStore.java | 36 ++- .../store/mysql/MysqlStorageEngine.java | 66 ++--- .../store/readonly/ReadOnlyStorageEngine.java | 45 ++- .../retention/RetentionEnforcingStore.java | 2 + .../store/routed/PipelineRoutedStore.java | 82 +++++- .../voldemort/store/routed/RoutedStore.java | 12 +- .../store/routed/ThreadPoolRoutedStore.java | 13 + .../serialized/SerializingStorageEngine.java | 15 + .../store/serialized/SerializingStore.java | 16 +- .../store/slop/SlopStorageEngine.java | 22 +- .../voldemort/store/socket/SocketStore.java | 48 ++-- .../store/stats/StatTrackingStore.java | 96 +++++++ .../InconsistencyResolvingStore.java | 22 ++ .../versioned/VersionIncrementingStore.java | 3 +- .../store/views/ViewStorageEngine.java | 36 +-- .../failuredetector/MutableStoreVerifier.java | 22 ++ .../voldemort/store/DoNothingStore.java | 44 +-- .../voldemort/store/FailingReadsStore.java | 20 +- test/common/voldemort/store/FailingStore.java | 20 +- .../store/RandomlyFailingDelegatingStore.java | 23 ++ .../store/noop/NoopStorageEngine.java | 51 +--- .../store/pausable/PausableStorageEngine.java | 34 +-- .../store/slow/SlowStorageEngine.java | 30 +- 53 files changed, 1698 insertions(+), 1072 deletions(-) create mode 100644 src/java/voldemort/coordinator/DynamicTimeoutStoreClient.java delete mode 100644 src/java/voldemort/coordinator/HttpResponseHandler.java create mode 100644 src/java/voldemort/coordinator/PutRequestExecutor.java delete mode 100644 src/java/voldemort/coordinator/R2StoreWrapper.java create mode 100644 src/java/voldemort/store/AbstractStorageEngine.java create mode 100644 src/java/voldemort/store/AbstractStore.java create mode 100644 src/java/voldemort/store/VoldemortRequestWrapper.java diff --git a/contrib/krati/src/java/voldemort/store/krati/KratiStorageEngine.java b/contrib/krati/src/java/voldemort/store/krati/KratiStorageEngine.java index 73082fecd3..c30eea305f 100644 --- a/contrib/krati/src/java/voldemort/store/krati/KratiStorageEngine.java +++ b/contrib/krati/src/java/voldemort/store/krati/KratiStorageEngine.java @@ -20,25 +20,21 @@ import org.apache.log4j.Logger; import voldemort.VoldemortException; -import voldemort.store.NoSuchCapabilityException; -import voldemort.store.StorageEngine; -import voldemort.store.StoreCapabilityType; +import voldemort.store.AbstractStorageEngine; import voldemort.store.StoreUtils; import voldemort.utils.ByteArray; import voldemort.utils.ClosableIterator; import voldemort.utils.Pair; import voldemort.utils.StripedLock; -import voldemort.utils.Utils; import voldemort.versioning.ObsoleteVersionException; import voldemort.versioning.Occurred; import voldemort.versioning.VectorClock; import voldemort.versioning.Version; import voldemort.versioning.Versioned; -public class KratiStorageEngine implements StorageEngine { +public class KratiStorageEngine extends AbstractStorageEngine { private static final Logger logger = Logger.getLogger(KratiStorageEngine.class); - private final String name; private final DynamicDataStore datastore; private final StripedLock locks; @@ -49,7 +45,7 @@ public KratiStorageEngine(String name, double hashLoadFactor, int initLevel, File dataDirectory) { - this.name = Utils.notNull(name); + super(name); try { this.datastore = new DynamicDataStore(dataDirectory, initLevel, @@ -64,16 +60,7 @@ public KratiStorageEngine(String name, } - public Object getCapability(StoreCapabilityType capability) { - throw new NoSuchCapabilityException(capability, getName()); - } - - public String getName() { - return this.name; - } - - public void close() throws VoldemortException {} - + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { @@ -81,19 +68,22 @@ public Map>> getAll(Iterable keys, return StoreUtils.getAll(this, keys, null); } + @Override public List getVersions(ByteArray key) { return StoreUtils.getVersions(get(key, null)); } + @Override public void truncate() { try { datastore.clear(); } catch(Exception e) { - logger.error("Failed to truncate store '" + name + "': ", e); - throw new VoldemortException("Failed to truncate store '" + name + "'."); + logger.error("Failed to truncate store '" + getName() + "': ", e); + throw new VoldemortException("Failed to truncate store '" + getName() + "'."); } } + @Override public List> get(ByteArray key, byte[] transforms) throws VoldemortException { StoreUtils.assertValidKey(key); try { @@ -104,6 +94,7 @@ public List> get(ByteArray key, byte[] transforms) throws Vold } } + @Override public ClosableIterator>> entries() { List>> returnedList = new ArrayList>>(); DataArray array = datastore.getDataArray(); @@ -143,18 +134,22 @@ public ClosableIterator>> entries() { return new KratiClosableIterator(returnedList); } + @Override public ClosableIterator keys() { return StoreUtils.keys(entries()); } + @Override public ClosableIterator>> entries(int partition) { throw new UnsupportedOperationException("Partition based entries scan not supported for this storage type"); } + @Override public ClosableIterator keys(int partition) { throw new UnsupportedOperationException("Partition based key scan not supported for this storage type"); } + @Override public boolean delete(ByteArray key, Version maxVersion) throws VoldemortException { StoreUtils.assertValidKey(key); @@ -197,6 +192,7 @@ public boolean delete(ByteArray key, Version maxVersion) throws VoldemortExcepti } } + @Override public void put(ByteArray key, Versioned value, byte[] transforms) throws VoldemortException { StoreUtils.assertValidKey(key); @@ -298,40 +294,26 @@ public KratiClosableIterator(List>> list) { iter = list.iterator(); } + @Override public void close() { // Nothing to close here } + @Override public boolean hasNext() { return iter.hasNext(); } + @Override public Pair> next() { return iter.next(); } + @Override public void remove() { Pair> currentPair = next(); delete(currentPair.getFirst(), currentPair.getSecond().getVersion()); } } - - public boolean isPartitionAware() { - return false; - } - - public boolean isPartitionScanSupported() { - return false; - } - - @Override - public boolean beginBatchModifications() { - return false; - } - - @Override - public boolean endBatchModifications() { - return false; - } } diff --git a/src/java/voldemort/client/ClientConfig.java b/src/java/voldemort/client/ClientConfig.java index e9408b7fd4..eb62ccc74b 100644 --- a/src/java/voldemort/client/ClientConfig.java +++ b/src/java/voldemort/client/ClientConfig.java @@ -1104,4 +1104,13 @@ public ClientConfig setEnableInconsistencyResolvingLayer(boolean enableInconsist this.enableInconsistencyResolvingLayer = enableInconsistencyResolvingLayer; return this; } + + public String toString() { + StringBuilder clientConfigInfo = new StringBuilder(); + clientConfigInfo.append("Max connections per node: " + this.maxConnectionsPerNode + "\n"); + clientConfigInfo.append("Connection timeout : " + this.connectionTimeoutMs + "\n"); + clientConfigInfo.append("Socket timeout : " + this.socketTimeoutMs + "\n"); + clientConfigInfo.append("Routing timeout : " + this.routingTimeoutMs + "\n"); + return clientConfigInfo.toString(); + } } diff --git a/src/java/voldemort/coordinator/CoordinatorPipelineFactory.java b/src/java/voldemort/coordinator/CoordinatorPipelineFactory.java index 45bd7a58f8..1646f02130 100644 --- a/src/java/voldemort/coordinator/CoordinatorPipelineFactory.java +++ b/src/java/voldemort/coordinator/CoordinatorPipelineFactory.java @@ -1,11 +1,9 @@ -package voldemort.coordinator; - /* - * Copyright 2009 Red Hat, Inc. + * Copyright 2008-2013 LinkedIn, Inc * - * Red Hat licenses this file to you 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: + * 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 * @@ -16,6 +14,8 @@ * the License. */ +package voldemort.coordinator; + import static org.jboss.netty.channel.Channels.pipeline; import java.util.Map; @@ -27,6 +27,11 @@ import org.jboss.netty.handler.codec.http.HttpRequestDecoder; import org.jboss.netty.handler.codec.http.HttpResponseEncoder; +/** + * A PipelineFactory implementation to setup the Netty Pipeline in the + * Coordinator + * + */ public class CoordinatorPipelineFactory implements ChannelPipelineFactory { private boolean noop = false; @@ -42,14 +47,7 @@ public ChannelPipeline getPipeline() throws Exception { // Create a default pipeline implementation. ChannelPipeline pipeline = pipeline(); - // Uncomment the following line if you want HTTPS - // SSLEngine engine = - // SecureChatSslContextFactory.getServerContext().createSSLEngine(); - // engine.setUseClientMode(false); - // pipeline.addLast("ssl", new SslHandler(engine)); - pipeline.addLast("decoder", new HttpRequestDecoder()); - // Uncomment the following line if you don't want to handle HttpChunks. pipeline.addLast("aggregator", new HttpChunkAggregator(1048576)); pipeline.addLast("encoder", new HttpResponseEncoder()); // Remove the following line if you don't want automatic content diff --git a/src/java/voldemort/coordinator/CoordinatorService.java b/src/java/voldemort/coordinator/CoordinatorService.java index 7ece75e880..95c6a2c92a 100644 --- a/src/java/voldemort/coordinator/CoordinatorService.java +++ b/src/java/voldemort/coordinator/CoordinatorService.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2013 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.coordinator; import java.io.File; @@ -36,6 +52,11 @@ import com.google.common.base.Joiner; +/** + * A Netty based HTTP service that accepts REST requests from the Voldemort thin + * clients and invokes the corresponding Fat client API. + * + */ public class CoordinatorService { private static boolean noop = false; @@ -49,13 +70,24 @@ public class CoordinatorService { public final static Schema CLIENT_CONFIGS_AVRO_SCHEMA = Schema.parse("{ \"name\": \"clientConfigs\", \"type\":\"array\"," + "\"items\": { \"name\": \"clientConfig\", \"type\": \"map\", \"values\":\"string\" }}}"); private static final String STORE_NAME_KEY = "store_name"; - + private static String CLIENT_CONFIG_AVRO_FILE_PATH = ""; + + /** + * Initializes all the Fat clients (1 per store) for the cluster that this + * Coordinator talks to. This is invoked once during startup and then every + * time the Metadata manager detects changes to the cluster and stores + * metadata. + */ private static void initializeFatClients() { StoreDefinitionsMapper storeMapper = new StoreDefinitionsMapper(); + + // Fetch the state once and use this to initialize all the Fat clients String storesXml = storeClientFactory.bootstrapMetadataWithRetries(MetadataStore.STORES_KEY); + String clusterXml = storeClientFactory.bootstrapMetadataWithRetries(MetadataStore.CLUSTER_KEY); + List storeDefList = storeMapper.readStoreList(new StringReader(storesXml), false); - Map fatClientConfigMap = readClientConfig("/home/csoman/Downloads/clientConfigs.avro", + Map fatClientConfigMap = readClientConfig(CLIENT_CONFIG_AVRO_FILE_PATH, bootstrapURLs); // For now Simply create the map of store definition to // FatClientWrappers @@ -70,7 +102,9 @@ private static void initializeFatClients() { logger.info("Using config: " + fatClientConfigMap.get(storeName)); fatClientMap.put(storeName, new FatClientWrapper(storeName, bootstrapURLs, - fatClientConfigMap.get(storeName))); + fatClientConfigMap.get(storeName), + storesXml, + clusterXml)); } @@ -78,20 +112,23 @@ private static void initializeFatClients() { public static void main(String[] args) { - if(args.length < 1) { - System.err.println("Missing argument: "); + if(args.length < 2) { + System.err.println("Missing argument: "); System.exit(-1); } - if(args.length == 2) { - if(args[1].equals("noop")) { + if(args.length == 3) { + if(args[2].equals("noop")) { noop = true; } } - // Initialize the Voldemort Metadata + // Initialize Config bootstrapURLs = new String[1]; bootstrapURLs[0] = args[0]; + CLIENT_CONFIG_AVRO_FILE_PATH = args[1]; + + // Initialize the Voldemort Metadata ClientConfig clientConfig = new ClientConfig(); clientConfig.setBootstrapUrls(bootstrapURLs); storeClientFactory = new SocketStoreClientFactory(clientConfig); @@ -107,6 +144,7 @@ public static void main(String[] args) { // Create a callback for re-bootstrapping the client Callable rebootstrapCallback = new Callable() { + @Override public Void call() throws Exception { initializeFatClients(); return null; @@ -136,6 +174,14 @@ public Void call() throws Exception { bootstrap.bind(new InetSocketAddress(8080)); } + /** + * A function to parse the specified Avro file in order to obtain the config + * for each fat client managed by this coordinator. + * + * @param configFilePath Path of the Avro file containing fat client configs + * @param bootstrapURLs The server URLs used during bootstrap + * @return Map of store name to the corresponding fat client config + */ @SuppressWarnings("unchecked") private static Map readClientConfig(String configFilePath, String[] bootstrapURLs) { diff --git a/src/java/voldemort/coordinator/DynamicTimeoutStoreClient.java b/src/java/voldemort/coordinator/DynamicTimeoutStoreClient.java new file mode 100644 index 0000000000..5c2f840ec7 --- /dev/null +++ b/src/java/voldemort/coordinator/DynamicTimeoutStoreClient.java @@ -0,0 +1,177 @@ +/* + * Copyright 2008-2013 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.coordinator; + +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.log4j.Logger; + +import voldemort.VoldemortException; +import voldemort.client.AbstractStoreClientFactory; +import voldemort.client.DefaultStoreClient; +import voldemort.client.StoreClientFactory; +import voldemort.store.InvalidMetadataException; +import voldemort.store.StoreTimeoutException; +import voldemort.store.VoldemortRequestWrapper; +import voldemort.versioning.ObsoleteVersionException; +import voldemort.versioning.VectorClock; +import voldemort.versioning.Version; +import voldemort.versioning.Versioned; + +import com.google.common.collect.Maps; + +/** + * A special store client to invoke Voldemort operations with the following new + * features: 1) Per call timeout facility 2) Ability to disable resolution per + * call + * + * @param Type of the Key + * @param Type of the Value + */ +public class DynamicTimeoutStoreClient extends DefaultStoreClient { + + private final Logger logger = Logger.getLogger(DynamicTimeoutStoreClient.class); + + /** + * + * @param storeName Name of the store this client connects to + * @param storeFactory Reference to the factory used to create this client + * @param maxMetadataRefreshAttempts Number of retries to retrieve the state + * @param storesXml The storesXml used during bootstrap + * @param clusterXml The clusterXml used during bootstrap + */ + public DynamicTimeoutStoreClient(String storeName, + StoreClientFactory storeFactory, + int maxMetadataRefreshAttempts, + String storesXml, + String clusterXml) { + this.storeName = storeName; + this.storeFactory = storeFactory; + this.metadataRefreshAttempts = maxMetadataRefreshAttempts; + bootStrap(clusterXml, storesXml); + } + + // Bootstrap using the given cluster xml and stores xml + // The super class bootStrap() method is used to handle the + // InvalidMetadataException + public void bootStrap(String customClusterXml, String customStoresXml) { + AbstractStoreClientFactory factory = (AbstractStoreClientFactory) this.storeFactory; + this.store = factory.getRawStore(storeName, null, customStoresXml, customClusterXml, null); + } + + public Versioned getWithCustomTimeout(VoldemortRequestWrapper requestWrapper) { + validateTimeout(requestWrapper.getRoutingTimeout()); + for(int attempts = 0; attempts < this.metadataRefreshAttempts; attempts++) { + try { + List> items = store.get(requestWrapper); + return getItemOrThrow(requestWrapper.getKey(), requestWrapper.getValue(), items); + } catch(InvalidMetadataException e) { + logger.info("Received invalid metadata exception during get [ " + e.getMessage() + + " ] on store '" + storeName + "'. Rebootstrapping"); + bootStrap(); + } + } + throw new VoldemortException(this.metadataRefreshAttempts + + " metadata refresh attempts failed."); + } + + public Version putWithCustomTimeout(VoldemortRequestWrapper requestWrapper) { + validateTimeout(requestWrapper.getRoutingTimeout()); + Versioned versioned; + long startTime = System.currentTimeMillis(); + + // We use the full timeout for doing the Get. In this, we're being + // optimistic that the subsequent put might be faster all the steps + // might finish within the alloted time + versioned = getWithCustomTimeout(new VoldemortRequestWrapper(requestWrapper.getKey(), + requestWrapper.getRoutingTimeout(), + true)); + + long endTime = System.currentTimeMillis(); + if(versioned == null) + versioned = Versioned.value(requestWrapper.getRawValue(), new VectorClock()); + else + versioned.setObject(requestWrapper.getRawValue()); + + // This should not happen unless there's a bug in the + // getWithCustomTimeout + if((endTime - startTime) > requestWrapper.getRoutingTimeout()) { + throw new StoreTimeoutException("PUT request timed out"); + } + + return putVersionedWithCustomTimeout(new VoldemortRequestWrapper(requestWrapper.getKey(), + versioned, + (requestWrapper.getRoutingTimeout() - (endTime - startTime)))); + } + + public Version putVersionedWithCustomTimeout(VoldemortRequestWrapper requestWrapper) + throws ObsoleteVersionException { + validateTimeout(requestWrapper.getRoutingTimeout()); + for(int attempts = 0; attempts < this.metadataRefreshAttempts; attempts++) { + try { + store.put(requestWrapper); + return requestWrapper.getValue().getVersion(); + } catch(InvalidMetadataException e) { + logger.info("Received invalid metadata exception during put [ " + e.getMessage() + + " ] on store '" + storeName + "'. Rebootstrapping"); + bootStrap(); + } + } + throw new VoldemortException(this.metadataRefreshAttempts + + " metadata refresh attempts failed."); + } + + public Map> getAllWithCustomTimeout(Iterable keys, + long getAllOpTimeoutInMs, + boolean resolveConflicts) { + validateTimeout(getAllOpTimeoutInMs); + Map>> items = null; + for(int attempts = 0;; attempts++) { + if(attempts >= this.metadataRefreshAttempts) + throw new VoldemortException(this.metadataRefreshAttempts + + " metadata refresh attempts failed."); + try { + items = store.getAll(new VoldemortRequestWrapper(keys, + getAllOpTimeoutInMs, + resolveConflicts)); + break; + } catch(InvalidMetadataException e) { + logger.info("Received invalid metadata exception during getAll [ " + + e.getMessage() + " ] on store '" + storeName + "'. Rebootstrapping"); + bootStrap(); + } + } + Map> result = Maps.newHashMapWithExpectedSize(items.size()); + + for(Entry>> mapEntry: items.entrySet()) { + Versioned value = getItemOrThrow(mapEntry.getKey(), null, mapEntry.getValue()); + result.put(mapEntry.getKey(), value); + } + return result; + } + + // Make sure that the timeout specified is valid + private void validateTimeout(long opTimeoutInMs) { + if(opTimeoutInMs <= 0) { + throw new IllegalArgumentException("Illegal parameter: Timeout is too low: " + + opTimeoutInMs); + } + } + +} diff --git a/src/java/voldemort/coordinator/FatClientWrapper.java b/src/java/voldemort/coordinator/FatClientWrapper.java index 215aa02bed..a56ded29f6 100644 --- a/src/java/voldemort/coordinator/FatClientWrapper.java +++ b/src/java/voldemort/coordinator/FatClientWrapper.java @@ -1,9 +1,20 @@ -package voldemort.coordinator; +/* + * Copyright 2008-2013 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 static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_LENGTH; -import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK; -import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1; +package voldemort.coordinator; import java.util.concurrent.ExecutorService; import java.util.concurrent.RejectedExecutionException; @@ -14,36 +25,40 @@ import java.util.concurrent.TimeUnit; import org.apache.log4j.Logger; -import org.jboss.netty.buffer.ChannelBuffer; -import org.jboss.netty.buffer.ChannelBuffers; -import org.jboss.netty.channel.ChannelFuture; -import org.jboss.netty.channel.ChannelFutureListener; import org.jboss.netty.channel.MessageEvent; -import org.jboss.netty.handler.codec.http.DefaultHttpResponse; -import org.jboss.netty.handler.codec.http.HttpResponse; import voldemort.client.ClientConfig; -import voldemort.client.DefaultStoreClient; import voldemort.client.SocketStoreClientFactory; -import voldemort.utils.ByteArray; -import voldemort.versioning.ObsoleteVersionException; - +import voldemort.store.VoldemortRequestWrapper; + +/** + * A Wrapper class to provide asynchronous API for calling the fat client + * methods. These methods will be invoked by the Netty request handler instead + * of invoking the Fat Client methods on its own + * + */ public class FatClientWrapper { private ExecutorService fatClientExecutor; private SocketStoreClientFactory storeClientFactory; - private DefaultStoreClient storeClient; private final Logger logger = Logger.getLogger(FatClientWrapper.class); + private DynamicTimeoutStoreClient dynamicTimeoutClient; /** - * A Wrapper class to provide asynchronous API for calling the fat client - * methods. These methods will be invoked by the Netty request handler - * instead of invoking the Fat Client methods on its own * - * @param storeName: Store to connect to via this fat client - * @param bootstrapURLs: Bootstrap URLs for the intended cluster + * @param storeName Store to connect to via this fat client + * @param bootstrapURLs Bootstrap URLs for the intended cluster + * @param clientConfig The config used to bootstrap the fat client + * @param storesXml Stores XML used to bootstrap the fat client + * @param clusterXml Cluster XML used to bootstrap the fat client */ - public FatClientWrapper(String storeName, String[] bootstrapURLs, ClientConfig clientConfig) { + public FatClientWrapper(String storeName, + String[] bootstrapURLs, + ClientConfig clientConfig, + String storesXml, + String clusterXml) { + + // TODO: Import this from Config this.fatClientExecutor = new ThreadPoolExecutor(20, // Core pool size 20, // Max pool size 60, // Keepalive @@ -79,24 +94,32 @@ public void rejectedExecution(Runnable r, // this.fatClientRequestQueue = new SynchronousQueue(); this.storeClientFactory = new SocketStoreClientFactory(clientConfig); - this.storeClient = (DefaultStoreClient) this.storeClientFactory.getStoreClient(storeName); + this.dynamicTimeoutClient = new DynamicTimeoutStoreClient(storeName, + this.storeClientFactory, + 1, + storesXml, + clusterXml); } /** * Interface to do get from the Fat client * + * * @param key: ByteArray representation of the key to get received from the * thin client * @param getRequest: MessageEvent to write the response on. + * @param operationTimeoutInMs The timeout value for this operation + * @param resolveConflicts Determines whether the default resolver should be + * used in case of conflicts */ - void submitGetRequest(final ByteArray key, final MessageEvent getRequest) { + void submitGetRequest(final VoldemortRequestWrapper getRequestObject, + final MessageEvent getRequest) { try { - this.fatClientExecutor.submit(new GetRequestExecutor(key, - null, + this.fatClientExecutor.submit(new GetRequestExecutor(getRequestObject, getRequest, - this.storeClient)); + this.dynamicTimeoutClient)); // Keep track of this request for monitoring // this.fatClientRequestQueue.add(f); @@ -111,11 +134,15 @@ void submitGetRequest(final ByteArray key, final MessageEvent getRequest) { * @param key: ByteArray representation of the key to put * @param value: value corresponding to the key to put * @param putRequest: MessageEvent to write the response on. + * @param operationTimeoutInMs The timeout value for this operation */ - void submitPutRequest(final ByteArray key, final byte[] value, final MessageEvent putRequest) { + void submitPutRequest(final VoldemortRequestWrapper putRequestObject, + final MessageEvent putRequest) { try { - this.fatClientExecutor.submit(new PutRequestExecutor(key, value, putRequest)); + this.fatClientExecutor.submit(new PutRequestExecutor(putRequestObject, + putRequest, + this.dynamicTimeoutClient)); // Keep track of this request for monitoring // this.fatClientRequestQueue.add(f); @@ -129,52 +156,4 @@ private void handleRejectedException(MessageEvent getRequest) { // client } - private class PutRequestExecutor implements Runnable { - - private ByteArray key; - private byte[] value; - private MessageEvent putRequest; - private ChannelBuffer responseContent; - - public PutRequestExecutor(ByteArray key, byte[] value, MessageEvent request) { - this.key = key; - this.value = value; - this.putRequest = request; - } - - private void writeResponse() { - // 1. Create the Response object - HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK); - - // 2. Set the right headers - // response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8"); - response.setHeader(CONTENT_TYPE, "application/json"); - // response.setChunked(true); - - // 3. Copy the data into the payload - response.setContent(responseContent); - response.setHeader(CONTENT_LENGTH, response.getContent().readableBytes()); - - // Write the response to the Netty Channel - ChannelFuture future = this.putRequest.getChannel().write(response); - - // Close the non-keep-alive connection after the write operation is - // done. - future.addListener(ChannelFutureListener.CLOSE); - } - - @Override - public void run() { - - try { - storeClient.put(key, value); - logger.info("Put successful !"); - } catch(ObsoleteVersionException oe) { - // Ideally propagate the exception ! - } - this.responseContent = ChannelBuffers.EMPTY_BUFFER; - writeResponse(); - } - - } } diff --git a/src/java/voldemort/coordinator/GetRequestExecutor.java b/src/java/voldemort/coordinator/GetRequestExecutor.java index a9d1933b09..91c7d0bd11 100644 --- a/src/java/voldemort/coordinator/GetRequestExecutor.java +++ b/src/java/voldemort/coordinator/GetRequestExecutor.java @@ -1,11 +1,30 @@ +/* + * Copyright 2008-2013 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.coordinator; import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_LENGTH; import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TRANSFER_ENCODING; import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE; import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.ETAG; +import static org.jboss.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST; +import static org.jboss.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR; import static org.jboss.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND; import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK; +import static org.jboss.netty.handler.codec.http.HttpResponseStatus.REQUEST_TIMEOUT; import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1; import java.io.IOException; @@ -22,28 +41,41 @@ import org.jboss.netty.handler.codec.http.DefaultHttpResponse; import org.jboss.netty.handler.codec.http.HttpResponse; -import voldemort.client.DefaultStoreClient; -import voldemort.utils.ByteArray; +import voldemort.VoldemortException; +import voldemort.store.StoreTimeoutException; +import voldemort.store.VoldemortRequestWrapper; import voldemort.versioning.VectorClock; import voldemort.versioning.Versioned; +/** + * A Runnable class that uses the specified Fat client to perform a Voldemort + * GET operation. This is invoked by a FatClientWrapper thread to satisfy a + * corresponding REST GET request. + * + */ public class GetRequestExecutor implements Runnable { - private ByteArray key; - private Versioned defaultValue; private MessageEvent getRequestMessageEvent; private ChannelBuffer responseContent; - DefaultStoreClient storeClient; + DynamicTimeoutStoreClient storeClient; private final Logger logger = Logger.getLogger(GetRequestExecutor.class); - - public GetRequestExecutor(ByteArray key, - Versioned defaultValue, + private final VoldemortRequestWrapper getRequestObject; + + /** + * + * @param getRequestObject The request object containing key and timeout + * values + * @param requestEvent Reference to the MessageEvent for the response / + * error + * @param storeClient Reference to the fat client for performing this Get + * operation + */ + public GetRequestExecutor(VoldemortRequestWrapper getRequestObject, MessageEvent requestEvent, - DefaultStoreClient storeClient) { - this.key = key; - this.defaultValue = defaultValue; + DynamicTimeoutStoreClient storeClient) { this.getRequestMessageEvent = requestEvent; this.storeClient = storeClient; + this.getRequestObject = getRequestObject; } public void writeResponse(Versioned responseVersioned) { @@ -75,7 +107,6 @@ public void writeResponse(Versioned responseVersioned) { HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK); // 2. Set the right headers - // response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8"); response.setHeader(CONTENT_TYPE, "application/json"); response.setHeader(CONTENT_TRANSFER_ENCODING, "binary"); response.setHeader(ETAG, eTag); @@ -95,23 +126,44 @@ public void writeResponse(Versioned responseVersioned) { } - public void setResponseContent(Versioned responseVersioned) {} - @Override public void run() { - Versioned responseVersioned = storeClient.get(this.key); - logger.info("Get successful !"); - if(responseVersioned == null) { - if(this.defaultValue != null) { - responseVersioned = this.defaultValue; - } else { - RESTErrorHandler.handleError(NOT_FOUND, - this.getRequestMessageEvent, - false, - "Requested Key does not exist"); + try { + Versioned responseVersioned = storeClient.getWithCustomTimeout(this.getRequestObject); + logger.info("Get successful !"); + if(responseVersioned == null) { + if(this.getRequestObject.getValue() != null) { + responseVersioned = this.getRequestObject.getValue(); + } else { + RESTErrorHandler.handleError(NOT_FOUND, + this.getRequestMessageEvent, + false, + "Requested Key does not exist"); + } } + writeResponse(responseVersioned); + } catch(IllegalArgumentException illegalArgsException) { + String errorDescription = "PUT Failed !!! Illegal Arguments : " + + illegalArgsException.getMessage(); + logger.error(errorDescription); + RESTErrorHandler.handleError(BAD_REQUEST, + this.getRequestMessageEvent, + false, + errorDescription); + } catch(StoreTimeoutException timeoutException) { + String errorDescription = "GET Request timed out: " + timeoutException.getMessage(); + logger.error(errorDescription); + RESTErrorHandler.handleError(REQUEST_TIMEOUT, + this.getRequestMessageEvent, + false, + errorDescription); + } catch(VoldemortException ve) { + String errorDescription = "Voldemort Exception: " + ve.getMessage(); + RESTErrorHandler.handleError(INTERNAL_SERVER_ERROR, + this.getRequestMessageEvent, + false, + errorDescription); } - writeResponse(responseVersioned); } } \ No newline at end of file diff --git a/src/java/voldemort/coordinator/HttpResponseHandler.java b/src/java/voldemort/coordinator/HttpResponseHandler.java deleted file mode 100644 index 69660314a9..0000000000 --- a/src/java/voldemort/coordinator/HttpResponseHandler.java +++ /dev/null @@ -1,77 +0,0 @@ -package voldemort.coordinator; -/* - * Copyright 2009 Red Hat, Inc. - * - * Red Hat licenses this file to you 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 org.jboss.netty.buffer.ChannelBuffer; -import org.jboss.netty.channel.ChannelHandlerContext; -import org.jboss.netty.channel.MessageEvent; -import org.jboss.netty.channel.SimpleChannelUpstreamHandler; -import org.jboss.netty.handler.codec.http.HttpChunk; -import org.jboss.netty.handler.codec.http.HttpResponse; -import org.jboss.netty.util.CharsetUtil; - -/** - * @author The Netty Project - * @author Andy Taylor (andy.taylor@jboss.org) - * @author Trustin Lee - * - * @version $Rev: 2189 $, $Date: 2010-02-19 18:02:57 +0900 (Fri, 19 Feb 2010) $ - */ -public class HttpResponseHandler extends SimpleChannelUpstreamHandler { - - private boolean readingChunks; - - @Override - public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Exception { - if (!readingChunks) { - HttpResponse response = (HttpResponse) e.getMessage(); - - System.out.println("STATUS: " + response.getStatus()); - System.out.println("VERSION: " + response.getProtocolVersion()); - System.out.println(); - - if (!response.getHeaderNames().isEmpty()) { - for (String name: response.getHeaderNames()) { - for (String value: response.getHeaders(name)) { - System.out.println("HEADER: " + name + " = " + value); - } - } - System.out.println(); - } - - if (response.getStatus().getCode() == 200 && response.isChunked()) { - readingChunks = true; - System.out.println("CHUNKED CONTENT {"); - } else { - ChannelBuffer content = response.getContent(); - if (content.readable()) { - System.out.println("CONTENT {"); - System.out.println(content.toString(CharsetUtil.UTF_8)); - System.out.println("} END OF CONTENT"); - } - } - } else { - HttpChunk chunk = (HttpChunk) e.getMessage(); - if (chunk.isLast()) { - readingChunks = false; - System.out.println("} END OF CHUNKED CONTENT"); - } else { - System.out.print(chunk.getContent().toString(CharsetUtil.UTF_8)); - System.out.flush(); - } - } - } -} diff --git a/src/java/voldemort/coordinator/NoopHttpRequestHandler.java b/src/java/voldemort/coordinator/NoopHttpRequestHandler.java index d29de914ab..804fae9b32 100644 --- a/src/java/voldemort/coordinator/NoopHttpRequestHandler.java +++ b/src/java/voldemort/coordinator/NoopHttpRequestHandler.java @@ -1,32 +1,58 @@ +/* + * Copyright 2008-2013 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.coordinator; import org.jboss.netty.buffer.ChannelBuffers; import org.jboss.netty.channel.ChannelHandlerContext; import org.jboss.netty.channel.MessageEvent; +import voldemort.common.VoldemortOpCode; +import voldemort.store.VoldemortRequestWrapper; import voldemort.versioning.Versioned; +/** + * A class that does a Noop after handling a REST request from the thin client. + * This is used for benchmarking purposes. + * + * + */ public class NoopHttpRequestHandler extends VoldemortHttpRequestHandler { public NoopHttpRequestHandler() {} @Override public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Exception { - OP_TYPE operation = getOperationType(this.request.getMethod()); + byte operationType = getOperationType(this.request.getMethod()); - switch(operation) { - case GET: - GetRequestExecutor getExecutor = new GetRequestExecutor(null, null, e, null); + switch(operationType) { + case VoldemortOpCode.GET_OP_CODE: + GetRequestExecutor getExecutor = new GetRequestExecutor(new VoldemortRequestWrapper(null, + 0l, + false), + e, + null); Versioned responseVersioned = null; byte[] nullByteArray = new byte[1]; nullByteArray[0] = 0; responseVersioned = new Versioned(nullByteArray); - - getExecutor.setResponseContent(responseVersioned); getExecutor.writeResponse(responseVersioned); break; - case PUT: + case VoldemortOpCode.PUT_OP_CODE: this.responseContent = ChannelBuffers.EMPTY_BUFFER; break; default: diff --git a/src/java/voldemort/coordinator/PutRequestExecutor.java b/src/java/voldemort/coordinator/PutRequestExecutor.java new file mode 100644 index 0000000000..a63a451a18 --- /dev/null +++ b/src/java/voldemort/coordinator/PutRequestExecutor.java @@ -0,0 +1,131 @@ +/* + * Copyright 2008-2013 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.coordinator; + +import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_LENGTH; +import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE; +import static org.jboss.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST; +import static org.jboss.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR; +import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK; +import static org.jboss.netty.handler.codec.http.HttpResponseStatus.PRECONDITION_FAILED; +import static org.jboss.netty.handler.codec.http.HttpResponseStatus.REQUEST_TIMEOUT; +import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1; + +import org.apache.log4j.Logger; +import org.jboss.netty.channel.ChannelFuture; +import org.jboss.netty.channel.ChannelFutureListener; +import org.jboss.netty.channel.MessageEvent; +import org.jboss.netty.handler.codec.http.DefaultHttpResponse; +import org.jboss.netty.handler.codec.http.HttpResponse; + +import voldemort.VoldemortException; +import voldemort.store.StoreTimeoutException; +import voldemort.store.VoldemortRequestWrapper; +import voldemort.versioning.ObsoleteVersionException; + +/** + * A Runnable class that uses the specified Fat client to perform a Voldemort + * PUT operation. This is invoked by a FatClientWrapper thread to satisfy a + * corresponding REST POST (PUT) request. + * + */ +public class PutRequestExecutor implements Runnable { + + private MessageEvent putRequestMessageEvent; + DynamicTimeoutStoreClient storeClient; + private final Logger logger = Logger.getLogger(PutRequestExecutor.class); + private final VoldemortRequestWrapper putRequestObject; + + /** + * + * @param putRequestObject The request object containing key and timeout + * values + * @param requestEvent Reference to the MessageEvent for the response / + * error + * @param storeClient Reference to the fat client for performing this Get + * operation + */ + public PutRequestExecutor(VoldemortRequestWrapper putRequestObject, + MessageEvent requestEvent, + DynamicTimeoutStoreClient storeClient) { + this.putRequestMessageEvent = requestEvent; + this.storeClient = storeClient; + this.putRequestObject = putRequestObject; + } + + private void writeResponse() { + // 1. Create the Response object + HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK); + + // 2. Set the right headers + response.setHeader(CONTENT_TYPE, "application/json"); + + // 3. Copy the data into the payload + // response.setContent(responseContent); + response.setHeader(CONTENT_LENGTH, 0); + + // Write the response to the Netty Channel + ChannelFuture future = this.putRequestMessageEvent.getChannel().write(response); + + // Close the non-keep-alive connection after the write operation is + // done. + future.addListener(ChannelFutureListener.CLOSE); + } + + @Override + public void run() { + + try { + this.storeClient.putWithCustomTimeout(putRequestObject); + logger.info("Put successful !"); + } catch(IllegalArgumentException illegalArgsException) { + String errorDescription = "PUT Failed !!! Illegal Arguments : " + + illegalArgsException.getMessage(); + logger.error(errorDescription); + RESTErrorHandler.handleError(BAD_REQUEST, + this.putRequestMessageEvent, + false, + errorDescription); + } catch(ObsoleteVersionException oe) { + String errorDescription = "PUT Failed !!! Obsolete version exception: " + + oe.getMessage(); + logger.error(errorDescription); + RESTErrorHandler.handleError(PRECONDITION_FAILED, + this.putRequestMessageEvent, + false, + errorDescription); + + } catch(StoreTimeoutException timeoutException) { + String errorDescription = "GET Request timed out: " + timeoutException.getMessage(); + logger.error(errorDescription); + RESTErrorHandler.handleError(REQUEST_TIMEOUT, + this.putRequestMessageEvent, + false, + errorDescription); + + } catch(VoldemortException ve) { + String errorDescription = "Voldemort Exception: " + ve.getMessage(); + RESTErrorHandler.handleError(INTERNAL_SERVER_ERROR, + this.putRequestMessageEvent, + false, + errorDescription); + } + + writeResponse(); + } + +} \ No newline at end of file diff --git a/src/java/voldemort/coordinator/R2StoreWrapper.java b/src/java/voldemort/coordinator/R2StoreWrapper.java deleted file mode 100644 index d2726c06e7..0000000000 --- a/src/java/voldemort/coordinator/R2StoreWrapper.java +++ /dev/null @@ -1,221 +0,0 @@ -package voldemort.coordinator; - -import java.io.ByteArrayOutputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.net.HttpURLConnection; -import java.net.URI; -import java.net.URL; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; - -import org.apache.commons.codec.binary.Base64; -import org.codehaus.jackson.map.ObjectMapper; - -import voldemort.VoldemortException; -import voldemort.store.Store; -import voldemort.store.StoreCapabilityType; -import voldemort.store.StoreUtils; -import voldemort.utils.ByteArray; -import voldemort.versioning.VectorClock; -import voldemort.versioning.Version; -import voldemort.versioning.Versioned; - -import com.linkedin.common.callback.FutureCallback; -import com.linkedin.common.util.None; -import com.linkedin.data.ByteString; -import com.linkedin.r2.message.rest.RestRequest; -import com.linkedin.r2.message.rest.RestRequestBuilder; -import com.linkedin.r2.message.rest.RestResponse; -import com.linkedin.r2.transport.common.Client; -import com.linkedin.r2.transport.common.bridge.client.TransportClient; -import com.linkedin.r2.transport.common.bridge.client.TransportClientAdapter; -import com.linkedin.r2.transport.http.client.HttpClientFactory; - -public class R2StoreWrapper implements Store { - - private static final String GET = "GET"; - private static final String PUT = "PUT"; - private URL url = null; - HttpURLConnection conn = null; - private HttpClientFactory _clientFactory; - private Client client = null; - private String baseURL; - - public R2StoreWrapper(String baseURL) { - try { - _clientFactory = new HttpClientFactory(); - final TransportClient transportClient = _clientFactory.getClient(new HashMap()); - client = new TransportClientAdapter(transportClient); - this.baseURL = baseURL; - } catch(Exception e) { - e.printStackTrace(); - } - - } - - @Override - public void close() throws VoldemortException { - final FutureCallback callback = new FutureCallback(); - client.shutdown(callback); - try { - callback.get(); - } catch(InterruptedException e) { - e.printStackTrace(); - } catch(ExecutionException e) { - e.printStackTrace(); - } - } - - @Override - public boolean delete(ByteArray arg0, Version arg1) throws VoldemortException { - // TODO Auto-generated method stub - return false; - } - - @Override - public List> get(ByteArray key, byte[] transforms) throws VoldemortException { - - List> resultList = new ArrayList>(); - - try { - // Create the byte[] array - ByteArrayOutputStream outputBytes = new ByteArrayOutputStream(); - DataOutputStream outputStream = new DataOutputStream(outputBytes); - writeGetRequest(outputStream, key); - - String base64Key = new String(Base64.encodeBase64(key.get())); - RestRequestBuilder rb = new RestRequestBuilder(new URI(this.baseURL + "/test/" - + base64Key)); - // RestRequestBuilder rb = new RestRequestBuilder(new - // URI(this.baseURL + "/" + base64Key)); - - rb.setMethod(GET); - rb.setEntity(outputBytes.toByteArray()); - rb.setHeader("Accept", "application/json"); - - RestRequest request = rb.build(); - Future f = client.restRequest(request); - - // This will block - RestResponse response = f.get(); - final ByteString entity = response.getEntity(); - String eTag = response.getHeader("ETag"); - String lastModified = response.getHeader("Last-Modified"); - if(entity != null) { - resultList = readResults(entity, eTag, lastModified); - } else { - System.out.println("NOTHING!"); - } - - } catch(VoldemortException ve) { - throw ve; - } catch(Exception e) { - e.printStackTrace(); - } - - return resultList; - } - - @Override - public void put(ByteArray key, Versioned value, byte[] transform) - throws VoldemortException { - try { - // Create the byte[] array - ByteArrayOutputStream outputBytes = new ByteArrayOutputStream(); - DataOutputStream outputStream = new DataOutputStream(outputBytes); - - // Write the value in the payload - byte[] payload = value.getValue(); - outputStream.write(payload); - - // Create the REST request with this byte array - String base64Key = new String(Base64.encodeBase64(key.get())); - RestRequestBuilder rb = new RestRequestBuilder(new URI(this.baseURL + "/test/" - + base64Key)); - // RestRequestBuilder rb = new RestRequestBuilder(new - // URI(this.baseURL + "/" + base64Key)); - - rb.setMethod(PUT); - rb.setEntity(outputBytes.toByteArray()); - rb.setHeader("Content-Type", "application/json"); - rb.setHeader("Content-Length", "" + payload.length); - - RestRequest request = rb.build(); - Future f = client.restRequest(request); - - // This will block - RestResponse response = f.get(); - final ByteString entity = response.getEntity(); - if(entity != null) { - // System.out.println(entity.asString("UTF-8")); - } else { - System.out.println("NOTHING!"); - } - } catch(VoldemortException ve) { - throw ve; - } catch(Exception e) { - e.printStackTrace(); - } - } - - private void writeGetRequest(DataOutputStream outputStream, ByteArray key) throws IOException { - StoreUtils.assertValidKey(key); - outputStream.writeInt(key.length()); - outputStream.write(key.get()); - } - - private List> readResults(ByteString entity, String eTag, String lastModified) - throws IOException { - - ObjectMapper mapper = new ObjectMapper(); - System.out.println("Received etag : " + eTag); - System.out.println("Received last modified date : " + lastModified); - VectorClockWrapper vcWrapper = mapper.readValue(eTag, VectorClockWrapper.class); - List> results = new ArrayList>(2); - - byte[] bytes = new byte[entity.length()]; - entity.copyBytes(bytes, 0); - VectorClock clock = new VectorClock(vcWrapper.getVersions(), vcWrapper.getTimestamp()); - results.add(new Versioned(bytes, clock)); - return results; - } - - private void writePutRequest(DataOutputStream outputStream, ByteArray key, byte[] value) - throws IOException { - writeGetRequest(outputStream, key); - outputStream.writeInt(value.length); - outputStream.write(value); - } - - @Override - public Map>> getAll(Iterable arg0, - Map arg1) - throws VoldemortException { - // TODO Auto-generated method stub - return null; - } - - @Override - public Object getCapability(StoreCapabilityType arg0) { - // TODO Auto-generated method stub - return null; - } - - @Override - public String getName() { - // TODO Auto-generated method stub - return null; - } - - @Override - public List getVersions(ByteArray arg0) { - // TODO Auto-generated method stub - return null; - } - -} diff --git a/src/java/voldemort/coordinator/RESTClient.java b/src/java/voldemort/coordinator/RESTClient.java index ca4695e66e..9c46457158 100644 --- a/src/java/voldemort/coordinator/RESTClient.java +++ b/src/java/voldemort/coordinator/RESTClient.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2013 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.coordinator; import java.util.List; @@ -33,11 +49,19 @@ public class RESTClient implements StoreClient { private SerializerFactory serializerFactory = new DefaultSerializerFactory(); private StoreDefinition storeDef; + /** + * A REST ful equivalent of the DefaultStoreClient. This uses the R2Store to + * interact with the RESTful Coordinator + * + * @param bootstrapURL The bootstrap URL of the Voldemort cluster + * @param storeName Name of the store to interact with + */ public RESTClient(String bootstrapURL, String storeName) { String baseURL = "http://" + bootstrapURL.split(":")[1].substring(2) + ":8080"; // The lowest layer : Transporting request to coordinator - Store store = (Store) new R2StoreWrapper(baseURL); + Store store = (Store) new R2Store(baseURL, + "R2Store"); // TODO // Get the store definition so that we can learn the Serialization @@ -48,6 +72,9 @@ public RESTClient(String bootstrapURL, String storeName) { // Add compression layer // Add Serialization layer + + // Set the following values although we don't need them + // TODO: Fix this, so that we only need to set the needed parameters storeDef = new StoreDefinitionBuilder().setName(storeName) .setType("bdb") .setKeySerializer(new SerializerDefinition("string")) diff --git a/src/java/voldemort/coordinator/RESTErrorHandler.java b/src/java/voldemort/coordinator/RESTErrorHandler.java index 04f6ca77a2..074ce8d1ca 100644 --- a/src/java/voldemort/coordinator/RESTErrorHandler.java +++ b/src/java/voldemort/coordinator/RESTErrorHandler.java @@ -1,8 +1,22 @@ +/* + * Copyright 2008-2013 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.coordinator; import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR; import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1; import org.apache.log4j.Logger; @@ -15,6 +29,11 @@ import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.jboss.netty.util.CharsetUtil; +/** + * A Generic class used to propagate the error back to the client over the Netty + * channel + * + */ public class RESTErrorHandler { private static final Logger logger = Logger.getLogger(RESTErrorHandler.class); @@ -39,32 +58,4 @@ public static void handleError(HttpResponseStatus status, future.addListener(ChannelFutureListener.CLOSE); } } - - public static void handleBadRequestError(MessageEvent e, boolean keepAlive) { - // 1. Create the Response object - HttpResponse response = new DefaultHttpResponse(HTTP_1_1, BAD_REQUEST); - - // Write the response to the Netty Channel - ChannelFuture future = e.getChannel().write(response); - - // Close the non-keep-alive connection after the write operation is - // done. - if(!keepAlive) { - future.addListener(ChannelFutureListener.CLOSE); - } - } - - public static void handleInternalServerError(MessageEvent e, boolean keepAlive) { - // 1. Create the Response object - HttpResponse response = new DefaultHttpResponse(HTTP_1_1, INTERNAL_SERVER_ERROR); - - // Write the response to the Netty Channel - ChannelFuture future = e.getChannel().write(response); - - // Close the non-keep-alive connection after the write operation is - // done. - if(!keepAlive) { - future.addListener(ChannelFutureListener.CLOSE); - } - } } diff --git a/src/java/voldemort/coordinator/SampleRESTClient.java b/src/java/voldemort/coordinator/SampleRESTClient.java index b0d6a6583d..f5fd690813 100644 --- a/src/java/voldemort/coordinator/SampleRESTClient.java +++ b/src/java/voldemort/coordinator/SampleRESTClient.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2013 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.coordinator; public class SampleRESTClient { @@ -9,8 +25,7 @@ public static void main(String[] args) { "test"); // Sample put - clientStore.put("a", - "Hola Senior !!! Bonjournooo sdafasdfsdfasadf sadfasdfasdfasdfsad fsad fsadfsadfsF!!!!"); + clientStore.put("a", "Howdy!!!!"); // Do a sample operation: System.out.println("Received response : " + clientStore.get("a")); diff --git a/src/java/voldemort/coordinator/VectorClockWrapper.java b/src/java/voldemort/coordinator/VectorClockWrapper.java index 1b2b984cf3..169bf71fb5 100644 --- a/src/java/voldemort/coordinator/VectorClockWrapper.java +++ b/src/java/voldemort/coordinator/VectorClockWrapper.java @@ -1,3 +1,19 @@ +/* + * Copyright 2008-2013 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.coordinator; import java.util.ArrayList; @@ -6,6 +22,12 @@ import voldemort.versioning.ClockEntry; import voldemort.versioning.VectorClock; +/** + * A wrapper for Vector clock used for serialization purposes. This Wrapper is + * then converted to a JSON string which in turn gets embedded in a HTTP header + * field. + * + */ public class VectorClockWrapper { private List versions; diff --git a/src/java/voldemort/coordinator/VoldemortHttpRequestHandler.java b/src/java/voldemort/coordinator/VoldemortHttpRequestHandler.java index 0ef070d547..2ff07e8810 100644 --- a/src/java/voldemort/coordinator/VoldemortHttpRequestHandler.java +++ b/src/java/voldemort/coordinator/VoldemortHttpRequestHandler.java @@ -1,42 +1,50 @@ +/* + * Copyright 2008-2013 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.coordinator; import static org.jboss.netty.handler.codec.http.HttpHeaders.isKeepAlive; -import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_LENGTH; -import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE; -import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.COOKIE; -import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.SET_COOKIE; import static org.jboss.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST; -import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK; -import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1; -import java.util.List; import java.util.Map; -import java.util.Set; import org.apache.commons.codec.binary.Base64; import org.apache.log4j.Logger; import org.jboss.netty.buffer.ChannelBuffer; -import org.jboss.netty.channel.ChannelFuture; -import org.jboss.netty.channel.ChannelFutureListener; import org.jboss.netty.channel.ChannelHandlerContext; import org.jboss.netty.channel.ExceptionEvent; import org.jboss.netty.channel.MessageEvent; import org.jboss.netty.channel.SimpleChannelUpstreamHandler; -import org.jboss.netty.handler.codec.http.Cookie; -import org.jboss.netty.handler.codec.http.CookieDecoder; -import org.jboss.netty.handler.codec.http.CookieEncoder; -import org.jboss.netty.handler.codec.http.DefaultHttpResponse; import org.jboss.netty.handler.codec.http.HttpChunk; import org.jboss.netty.handler.codec.http.HttpChunkTrailer; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpRequest; -import org.jboss.netty.handler.codec.http.HttpResponse; import org.jboss.netty.util.CharsetUtil; +import voldemort.common.VoldemortOpCode; +import voldemort.store.VoldemortRequestWrapper; import voldemort.utils.ByteArray; -import voldemort.versioning.VectorClock; -import voldemort.versioning.Versioned; +/** + * A class to handle the HTTP request and execute the same on behalf of the thin + * client. + * + * Currently, we're using a fat client to handle this request. + * + */ public class VoldemortHttpRequestHandler extends SimpleChannelUpstreamHandler { public HttpRequest request; @@ -46,11 +54,8 @@ public class VoldemortHttpRequestHandler extends SimpleChannelUpstreamHandler { public ChannelBuffer responseContent; private Map fatClientMap; private final Logger logger = Logger.getLogger(VoldemortHttpRequestHandler.class); - - public static enum OP_TYPE { - GET, - PUT - } + private String storeName = null; + private FatClientWrapper fatClientWrapper = null; // Implicit constructor defined for the derived classes public VoldemortHttpRequestHandler() {} @@ -59,74 +64,110 @@ public VoldemortHttpRequestHandler(Map fatClientMap) { this.fatClientMap = fatClientMap; } - public OP_TYPE getOperationType(HttpMethod httpMethod) { - if(httpMethod.equals(HttpMethod.PUT)) { - return OP_TYPE.PUT; + /** + * Function to parse the HTTP headers and build a Voldemort request object + * + * @param requestURI URI of the REST request + * @param operationType Message Event object used to write the response to + * @param e The REST (Voldemort) operation type + * @return true if a valid request was received. False otherwise + */ + private VoldemortRequestWrapper parseRequest(String requestURI, + MessageEvent e, + byte operationType) { + VoldemortRequestWrapper requestWrapper = null; + long operationTimeoutInMs = 1500; + boolean resolveConflicts = true; + + // Retrieve the timeout value from the REST request + String timeoutValStr = this.request.getHeader(R2Store.X_VOLD_REQUEST_TIMEOUT_MS); + if(timeoutValStr != null) { + try { + Long.parseLong(timeoutValStr); + } catch(NumberFormatException nfe) { + handleBadRequest(e, "Incorrect timeout parameter. Cannot parse this to long: " + + timeoutValStr + ". Details: " + nfe.getMessage()); + return null; + } } - return OP_TYPE.GET; - } + // Retrieve the inconsistency resolving strategy from the REST request + String inconsistencyResolverOption = this.request.getHeader(R2Store.X_VOLD_INCONSISTENCY_RESOLVER); + if(inconsistencyResolverOption != null) { + if(inconsistencyResolverOption.equalsIgnoreCase(R2Store.CUSTOM_RESOLVING_STRATEGY)) { + resolveConflicts = false; + } else if(!inconsistencyResolverOption.equalsIgnoreCase(R2Store.DEFAULT_RESOLVING_STRATEGY)) { + handleBadRequest(e, + "Invalid Inconsistency Resolving strategy specified in the Request : " + + inconsistencyResolverOption); + return null; + } + } + + // Get the store name from the REST request + storeName = getStoreName(requestURI); + this.fatClientWrapper = null; + if(storeName != null) { + this.fatClientWrapper = this.fatClientMap.get(storeName); + } + + if(storeName == null || fatClientWrapper == null) { + handleBadRequest(e, "Invalid store name. Critical error."); + return null; + } + + // Build the request object based on the operation type + switch(operationType) { + case VoldemortOpCode.GET_OP_CODE: + ByteArray getKey = readKey(requestURI); + requestWrapper = new VoldemortRequestWrapper(getKey, + operationTimeoutInMs, + resolveConflicts); + break; + case VoldemortOpCode.PUT_OP_CODE: + ChannelBuffer content = request.getContent(); + if(!content.readable()) { + handleBadRequest(e, "Contents not readable"); + return null; + } - public void writeResults(List> values) { - responseContent.writeInt(values.size()); - for(Versioned v: values) { - byte[] clock = ((VectorClock) v.getVersion()).toBytes(); - byte[] value = (byte[]) v.getValue(); - responseContent.writeInt(clock.length + value.length); - responseContent.writeBytes(clock); - responseContent.writeBytes(value); + ByteArray putKey = readKey(requestURI); + byte[] putValue = readValue(content); + requestWrapper = new VoldemortRequestWrapper(putKey, putValue, operationTimeoutInMs); + + break; + default: + handleBadRequest(e, "Illegal Operation."); + return null; } + + return requestWrapper; } @Override public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Exception { - String storeName = ""; - if(!readingChunks) { HttpRequest request = this.request = (HttpRequest) e.getMessage(); - OP_TYPE operation = getOperationType(this.request.getMethod()); + byte operationType = getOperationType(this.request.getMethod()); String requestURI = this.request.getUri(); logger.info(requestURI); - storeName = getStoreName(requestURI); - if(storeName == null) { - String errorMessage = "Invalid store name. Critical error."; - // this.responseContent = - // ChannelBuffers.copiedBuffer("Invalid store name. Critical error.".getBytes()); - logger.error(errorMessage); - RESTErrorHandler.handleError(BAD_REQUEST, e, false, errorMessage); - return; - } - if(request.isChunked()) { readingChunks = true; } else { - // TODO: Check for correct number of parameters and Decoding + VoldemortRequestWrapper requestObject = parseRequest(requestURI, e, operationType); + if(requestObject == null) { + return; + } - switch(operation) { - case GET: - ByteArray getKey = readKey(requestURI); - this.fatClientMap.get(storeName).submitGetRequest(getKey, e); + switch(operationType) { + case VoldemortOpCode.GET_OP_CODE: + this.fatClientWrapper.submitGetRequest(requestObject, e); break; - case PUT: - ChannelBuffer content = request.getContent(); - if(!content.readable()) { - String errorMessage = "Contents not readable"; - // this.responseContent = - // ChannelBuffers.copiedBuffer("Contents not readable".getBytes()); - logger.error(errorMessage); - RESTErrorHandler.handleError(BAD_REQUEST, - e, - isKeepAlive(request), - errorMessage); - return; - } - - ByteArray putKey = readKey(requestURI); - byte[] putValue = readValue(content); - this.fatClientMap.get(storeName).submitPutRequest(putKey, putValue, e); + case VoldemortOpCode.PUT_OP_CODE: + this.fatClientWrapper.submitPutRequest(requestObject, e); break; default: String errorMessage = "Illegal operation."; @@ -163,12 +204,53 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e) throws Ex } } + /** + * Send a BAD_REQUEST HTTP error back to the client with the specified + * message. + * + * @param e Message event to write the error to + * @param msg Error message + */ + private void handleBadRequest(MessageEvent e, String msg) { + String errorMessage = msg; + logger.error(errorMessage); + RESTErrorHandler.handleError(BAD_REQUEST, e, false, errorMessage); + } + + /** + * Method to determine the operation type + * + * @param httpMethod The HTTP Method object received by the Netty handler + * @return A voldemortOpCode object representing the operation type + */ + protected byte getOperationType(HttpMethod httpMethod) { + if(httpMethod.equals(HttpMethod.POST)) { + return VoldemortOpCode.PUT_OP_CODE; + } else if(httpMethod.equals(HttpMethod.GET)) { + return VoldemortOpCode.GET_OP_CODE; + } + + return -1; + } + + /** + * Method to read a value for a put operation + * + * @param content The ChannelBuffer object containing the value + * @return The byte[] array representing the value + */ private byte[] readValue(ChannelBuffer content) { byte[] value = new byte[content.capacity()]; content.readBytes(value); return value; } + /** + * Method to read a key present in the HTTP request URI + * + * @param requestURI The URI of the HTTP request + * @return the ByteArray representing the key + */ private ByteArray readKey(String requestURI) { ByteArray key = null; String[] parts = requestURI.split("/"); @@ -179,6 +261,12 @@ private ByteArray readKey(String requestURI) { return key; } + /** + * Retrieve the store name from the URI + * + * @param requestURI The URI of the HTTP request + * @return The string representing the store name + */ private String getStoreName(String requestURI) { String storeName = null; String[] parts = requestURI.split("/"); @@ -189,47 +277,6 @@ private String getStoreName(String requestURI) { return storeName; } - public void writeResponse(MessageEvent e) { - // Decide whether to close the connection or not. - boolean keepAlive = isKeepAlive(request); - - // Build the response object. - HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK); - response.setContent(this.responseContent); - // response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8"); - response.setHeader(CONTENT_TYPE, "application/pdf"); - // response.setChunked(true); - - if(keepAlive) { - // Add 'Content-Length' header only for a keep-alive connection. - response.setHeader(CONTENT_LENGTH, response.getContent().readableBytes()); - } - - // Encode the cookie. - String cookieString = request.getHeader(COOKIE); - if(cookieString != null) { - CookieDecoder cookieDecoder = new CookieDecoder(); - Set cookies = cookieDecoder.decode(cookieString); - if(!cookies.isEmpty()) { - // Reset the cookies if necessary. - CookieEncoder cookieEncoder = new CookieEncoder(true); - for(Cookie cookie: cookies) { - cookieEncoder.addCookie(cookie); - } - response.addHeader(SET_COOKIE, cookieEncoder.encode()); - } - } - - // Write the response. - ChannelFuture future = e.getChannel().write(response); - - // Close the non-keep-alive connection after the write operation is - // done. - if(!keepAlive) { - future.addListener(ChannelFutureListener.CLOSE); - } - } - @Override public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e) throws Exception { e.getCause().printStackTrace(); diff --git a/src/java/voldemort/store/AbstractStorageEngine.java b/src/java/voldemort/store/AbstractStorageEngine.java new file mode 100644 index 0000000000..5740a8ccc5 --- /dev/null +++ b/src/java/voldemort/store/AbstractStorageEngine.java @@ -0,0 +1,57 @@ +package voldemort.store; + +import voldemort.utils.ClosableIterator; +import voldemort.utils.Pair; +import voldemort.versioning.Versioned; + +public class AbstractStorageEngine extends AbstractStore implements + StorageEngine { + + public AbstractStorageEngine(String name) { + super(name); + } + + @Override + public ClosableIterator>> entries() { + return null; + } + + @Override + public ClosableIterator keys() { + return null; + } + + @Override + public ClosableIterator>> entries(int partition) { + return null; + } + + @Override + public ClosableIterator keys(int partition) { + return null; + } + + @Override + public void truncate() {} + + @Override + public boolean isPartitionAware() { + return false; + } + + @Override + public boolean isPartitionScanSupported() { + return false; + } + + @Override + public boolean beginBatchModifications() { + return false; + } + + @Override + public boolean endBatchModifications() { + return false; + } + +} diff --git a/src/java/voldemort/store/AbstractStore.java b/src/java/voldemort/store/AbstractStore.java new file mode 100644 index 0000000000..30f0f59d9c --- /dev/null +++ b/src/java/voldemort/store/AbstractStore.java @@ -0,0 +1,75 @@ +package voldemort.store; + +import java.util.List; +import java.util.Map; + +import voldemort.VoldemortException; +import voldemort.utils.Utils; +import voldemort.versioning.Version; +import voldemort.versioning.Versioned; + +public abstract class AbstractStore implements Store { + + private final String storeName; + + public AbstractStore(String name) { + this.storeName = Utils.notNull(name); + } + + @Override + public List> get(K key, T transforms) throws VoldemortException { + return null; + } + + @Override + public Map>> getAll(Iterable keys, Map transforms) + throws VoldemortException { + return null; + } + + @Override + public void put(K key, Versioned value, T transforms) throws VoldemortException {} + + @Override + public boolean delete(K key, Version version) throws VoldemortException { + return false; + } + + @Override + public String getName() { + return this.storeName; + } + + @Override + public void close() throws VoldemortException {} + + @Override + public Object getCapability(StoreCapabilityType capability) { + throw new NoSuchCapabilityException(capability, getName()); + } + + @Override + public List getVersions(K key) { + return null; + } + + @Override + public List> get(VoldemortRequestWrapper request) throws VoldemortException { + return null; + } + + @Override + public Map>> getAll(VoldemortRequestWrapper request) + throws VoldemortException { + return null; + } + + @Override + public void put(VoldemortRequestWrapper request) throws VoldemortException {} + + @Override + public boolean delete(VoldemortRequestWrapper request) throws VoldemortException { + return false; + } + +} diff --git a/src/java/voldemort/store/DelegatingStore.java b/src/java/voldemort/store/DelegatingStore.java index aba86bea79..f4ac1d535f 100644 --- a/src/java/voldemort/store/DelegatingStore.java +++ b/src/java/voldemort/store/DelegatingStore.java @@ -32,38 +32,40 @@ * * */ -public class DelegatingStore implements Store { +public class DelegatingStore extends AbstractStore { private final Store innerStore; public DelegatingStore(Store innerStore) { + super(innerStore.getName()); this.innerStore = Utils.notNull(innerStore); } + @Override public void close() throws VoldemortException { innerStore.close(); } + @Override public boolean delete(K key, Version version) throws VoldemortException { StoreUtils.assertValidKey(key); return innerStore.delete(key, version); } + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { StoreUtils.assertValidKeys(keys); return innerStore.getAll(keys, transforms); } + @Override public List> get(K key, T transform) throws VoldemortException { StoreUtils.assertValidKey(key); return innerStore.get(key, transform); } - public String getName() { - return innerStore.getName(); - } - + @Override public void put(K key, Versioned value, T transform) throws VoldemortException { StoreUtils.assertValidKey(key); innerStore.put(key, value, transform); @@ -73,6 +75,7 @@ public Store getInnerStore() { return innerStore; } + @Override public Object getCapability(StoreCapabilityType capability) { return innerStore.getCapability(capability); } @@ -82,7 +85,31 @@ public String toString() { return innerStore.toString(); } + @Override public List getVersions(K key) { return innerStore.getVersions(key); } + + @Override + public List> get(VoldemortRequestWrapper request) throws VoldemortException { + StoreUtils.assertValidKey(request.getKey()); + return innerStore.get(request); + } + + // TODO: Validate all the keys in the request object + @Override + public Map>> getAll(VoldemortRequestWrapper request) + throws VoldemortException { + return innerStore.getAll(request); + } + + @Override + public void put(VoldemortRequestWrapper request) throws VoldemortException { + innerStore.put(request); + } + + @Override + public boolean delete(VoldemortRequestWrapper request) throws VoldemortException { + return innerStore.delete(request); + } } diff --git a/src/java/voldemort/store/Store.java b/src/java/voldemort/store/Store.java index 4283353666..202a454226 100644 --- a/src/java/voldemort/store/Store.java +++ b/src/java/voldemort/store/Store.java @@ -100,6 +100,51 @@ public Map>> getAll(Iterable keys, Map transforms) */ public Object getCapability(StoreCapabilityType capability); + /** + * Get the versions associated with the given key. This is used in a put + * call to write a new value for this key + * + * @param key The key to retrieve the versions for + * @return List of Versions associated with this key. + */ public List getVersions(K key); + /** + * Get the value associated with the given key + * + * @param request Contains the key to check for and associated transforms + * @return The value associated with the key or an empty list if no values + * are found. + * @throws VoldemortException + */ + public List> get(VoldemortRequestWrapper request) throws VoldemortException; + + /** + * Get the values associated with the given keys and returns them in a Map + * of keys to a list of versioned values. Note that the returned map will + * only contain entries for the keys which have a value associated with + * them. + * + * @param requests Contains the keys to check for. + * @return A Map of keys to a list of versioned values. + * @throws VoldemortException + */ + public Map>> getAll(VoldemortRequestWrapper request) + throws VoldemortException; + + /** + * Associate the value with the key and version in this store + * + * @param request Contains the key to use along with the value and version + * to use. + */ + public void put(VoldemortRequestWrapper request) throws VoldemortException; + + /** + * Delete all entries prior to the given version + * + * @param request: Contains the key to delete and current version of the key + * @return True if anything was deleted + */ + public boolean delete(VoldemortRequestWrapper request) throws VoldemortException; } diff --git a/src/java/voldemort/store/VoldemortRequestWrapper.java b/src/java/voldemort/store/VoldemortRequestWrapper.java new file mode 100644 index 0000000000..3e879120b3 --- /dev/null +++ b/src/java/voldemort/store/VoldemortRequestWrapper.java @@ -0,0 +1,80 @@ +package voldemort.store; + +import voldemort.versioning.Version; +import voldemort.versioning.Versioned; + +public class VoldemortRequestWrapper { + + private final K key; + private final V rawValue; + private final Iterable getAllIterableKeys; + private final Versioned value; + private final Version version; + private final long routingTimeout; + private final boolean resolveConflicts; + + // To be used in get call + public VoldemortRequestWrapper(K key, long timeout, boolean resolveConflicts) { + this(key, null, null, null, null, timeout, resolveConflicts); + } + + // To be used in getAll call + public VoldemortRequestWrapper(Iterable keys, long timeout, boolean resolveConflicts) { + this(null, null, keys, null, null, timeout, resolveConflicts); + } + + // To be used in put call + public VoldemortRequestWrapper(K key, V rawValue, long timeout) { + this(key, rawValue, null, null, null, timeout, true); + } + + // To be used in versioned put call + public VoldemortRequestWrapper(K key, Versioned value, long timeout) { + this(key, null, null, value, null, timeout, true); + } + + public VoldemortRequestWrapper(K key, + V rawValue, + Iterable keys, + Versioned value, + Version version, + long timeout, + boolean resolveConflicts) { + this.key = key; + this.rawValue = rawValue; + this.getAllIterableKeys = keys; + this.routingTimeout = timeout; + this.value = value; + this.version = version; + this.resolveConflicts = resolveConflicts; + } + + public K getKey() { + return key; + } + + public Versioned getValue() { + return value; + } + + public Version getVersion() { + return version; + } + + public long getRoutingTimeout() { + return routingTimeout; + } + + public boolean resolveConflicts() { + return resolveConflicts; + } + + public Iterable getIterableKeys() { + return getAllIterableKeys; + } + + public V getRawValue() { + return rawValue; + } + +} diff --git a/src/java/voldemort/store/bdb/BdbStorageEngine.java b/src/java/voldemort/store/bdb/BdbStorageEngine.java index 5199a39e88..ca3ade7cf9 100644 --- a/src/java/voldemort/store/bdb/BdbStorageEngine.java +++ b/src/java/voldemort/store/bdb/BdbStorageEngine.java @@ -31,13 +31,11 @@ import voldemort.VoldemortException; import voldemort.annotations.jmx.JmxOperation; import voldemort.server.protocol.admin.AsyncOperationStatus; -import voldemort.store.NoSuchCapabilityException; +import voldemort.store.AbstractStorageEngine; import voldemort.store.PersistenceFailureException; -import voldemort.store.StorageEngine; import voldemort.store.StorageInitializationException; import voldemort.store.Store; import voldemort.store.StoreBinaryFormat; -import voldemort.store.StoreCapabilityType; import voldemort.store.StoreUtils; import voldemort.store.backup.NativeBackupable; import voldemort.store.bdb.stats.BdbEnvironmentStats; @@ -69,12 +67,12 @@ * A store that uses BDB for persistence * */ -public class BdbStorageEngine implements StorageEngine, NativeBackupable { +public class BdbStorageEngine extends AbstractStorageEngine implements + NativeBackupable { private static final Logger logger = Logger.getLogger(BdbStorageEngine.class); private static final Hex hexCodec = new Hex(); - private final String name; private Database bdbDatabase; private final Environment environment; private final AtomicBoolean isOpen; @@ -90,7 +88,7 @@ public BdbStorageEngine(String name, Environment environment, Database database, BdbRuntimeConfig config) { - this.name = Utils.notNull(name); + super(name); this.bdbDatabase = Utils.notNull(database); this.environment = Utils.notNull(environment); this.isOpen = new AtomicBoolean(true); @@ -103,10 +101,7 @@ public BdbStorageEngine(String name, this.checkpointerOffForBatchWrites = config.isCheckpointerOffForBatchWrites(); } - public String getName() { - return name; - } - + @Override public ClosableIterator>> entries() { try { Cursor cursor = getBdbDatabase().openCursor(null, null); @@ -121,6 +116,7 @@ public ClosableIterator>> entries() { } } + @Override public ClosableIterator keys() { try { Cursor cursor = getBdbDatabase().openCursor(null, null); @@ -135,14 +131,17 @@ public ClosableIterator keys() { } } + @Override public ClosableIterator>> entries(int partition) { throw new UnsupportedOperationException("Partition based entries scan not supported for this storage type"); } + @Override public ClosableIterator keys(int partition) { throw new UnsupportedOperationException("Partition based key scan not supported for this storage type"); } + @Override public void truncate() { if(isTruncating.compareAndSet(false, true)) { @@ -225,10 +224,12 @@ protected Database getBdbDatabase() { return bdbDatabase; } + @Override public List getVersions(ByteArray key) { return StoreUtils.getVersions(get(key, null)); } + @Override public List> get(ByteArray key, byte[] transforms) throws PersistenceFailureException { StoreUtils.assertValidKey(key); @@ -263,6 +264,7 @@ public List> get(ByteArray key, byte[] transforms) } } + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { @@ -290,6 +292,7 @@ public Map>> getAll(Iterable keys, return results; } + @Override public void put(ByteArray key, Versioned value, byte[] transforms) throws PersistenceFailureException { @@ -366,6 +369,7 @@ else if(occurred == Occurred.AFTER) } } + @Override public boolean delete(ByteArray key, Version version) throws PersistenceFailureException { StoreUtils.assertValidKey(key); @@ -443,13 +447,9 @@ public boolean delete(ByteArray key, Version version) throws PersistenceFailureE } } - public Object getCapability(StoreCapabilityType capability) { - throw new NoSuchCapabilityException(capability, getName()); - } - @Override public int hashCode() { - return name.hashCode(); + return getName().hashCode(); } @Override @@ -460,6 +460,7 @@ public boolean equals(Object o) { return s.getName().equals(s.getName()); } + @Override public void close() throws PersistenceFailureException { try { if(this.isOpen.compareAndSet(true, false)) @@ -534,12 +535,14 @@ public BdbEntriesIterator(Cursor cursor, BdbStorageEngine bdbEngine) { this.cache = new ArrayList>>(); } + @Override public boolean hasNext() { // we have a next element if there is at least one cached // element or we can make more return cache.size() > 0 || makeMore(); } + @Override public Pair> next() { if(cache.size() == 0) { if(!makeMore()) @@ -586,10 +589,12 @@ public BdbKeysIterator(Cursor cursor, BdbStorageEngine bdbEngine) { super(cursor, bdbEngine); } + @Override public boolean hasNext() { return current != null || fetchNextKey(); } + @Override public ByteArray next() { ByteArray result = null; if(current == null) { @@ -627,14 +632,7 @@ private boolean fetchNextKey() { } } - public boolean isPartitionAware() { - return false; - } - - public boolean isPartitionScanSupported() { - return false; - } - + @Override public void nativeBackup(File toDir, boolean verifyFiles, boolean isIncremental, diff --git a/src/java/voldemort/store/bdb/PartitionPrefixedBdbStorageEngine.java b/src/java/voldemort/store/bdb/PartitionPrefixedBdbStorageEngine.java index dbabb30c88..3784775300 100644 --- a/src/java/voldemort/store/bdb/PartitionPrefixedBdbStorageEngine.java +++ b/src/java/voldemort/store/bdb/PartitionPrefixedBdbStorageEngine.java @@ -144,12 +144,14 @@ public BdbPartitionEntriesIterator(Cursor cursor, int partition, BdbStorageEngin this.positioned = false; } + @Override public boolean hasNext() { // we have a next element if there is at least one cached // element or we can make more return cache.size() > 0 || makeMore(); } + @Override public Pair> next() { if(cache.size() == 0) { if(!makeMore()) @@ -216,10 +218,12 @@ public BdbPartitionKeysIterator(Cursor cursor, int partition, BdbStorageEngine b positioned = false; } + @Override public boolean hasNext() { return current != null || fetchNextKey(); } + @Override public ByteArray next() { ByteArray result = null; if(current == null) { diff --git a/src/java/voldemort/store/compress/CompressingStore.java b/src/java/voldemort/store/compress/CompressingStore.java index b1cf571908..aae39b3a2e 100644 --- a/src/java/voldemort/store/compress/CompressingStore.java +++ b/src/java/voldemort/store/compress/CompressingStore.java @@ -22,6 +22,7 @@ import java.util.Map; import voldemort.VoldemortException; +import voldemort.store.AbstractStore; import voldemort.store.Store; import voldemort.store.StoreCapabilityType; import voldemort.store.StoreUtils; @@ -45,7 +46,7 @@ * @see NoopCompressionStrategy * @see GzipCompressionStrategy */ -public class CompressingStore implements Store { +public class CompressingStore extends AbstractStore { private final Store innerStore; private final CompressionStrategy keysCompressionStrategy; @@ -54,11 +55,13 @@ public class CompressingStore implements Store { public CompressingStore(Store innerStore, CompressionStrategy keysCompressionStrategy, CompressionStrategy valuesCompressionStrategy) { + super(innerStore.getName()); this.keysCompressionStrategy = Utils.notNull(keysCompressionStrategy); this.valuesCompressionStrategy = Utils.notNull(valuesCompressionStrategy); this.innerStore = Utils.notNull(innerStore); } + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { @@ -130,11 +133,13 @@ private byte[] deflate(CompressionStrategy compressionStrategy, byte[] data) } } + @Override public List> get(ByteArray key, byte[] transforms) throws VoldemortException { StoreUtils.assertValidKey(key); return inflateValues(innerStore.get(deflateKey(key), transforms)); } + @Override public List getVersions(ByteArray key) { return innerStore.getVersions(deflateKey(key)); } @@ -147,24 +152,24 @@ private List> inflateValues(List> result) { return inflated; } + @Override public void put(ByteArray key, Versioned value, byte[] transforms) throws VoldemortException { StoreUtils.assertValidKey(key); innerStore.put(deflateKey(key), deflateValue(value), transforms); } + @Override public void close() throws VoldemortException { innerStore.close(); } + @Override public Object getCapability(StoreCapabilityType capability) { return innerStore.getCapability(capability); } - public String getName() { - return innerStore.getName(); - } - + @Override public boolean delete(ByteArray key, Version version) throws VoldemortException { StoreUtils.assertValidKey(key); return innerStore.delete(deflateKey(key), version); diff --git a/src/java/voldemort/store/configuration/ConfigurationStorageEngine.java b/src/java/voldemort/store/configuration/ConfigurationStorageEngine.java index 2efe001e17..dd35a0cb2d 100644 --- a/src/java/voldemort/store/configuration/ConfigurationStorageEngine.java +++ b/src/java/voldemort/store/configuration/ConfigurationStorageEngine.java @@ -27,7 +27,7 @@ import org.apache.log4j.Logger; import voldemort.VoldemortException; -import voldemort.store.StorageEngine; +import voldemort.store.AbstractStorageEngine; import voldemort.store.StoreCapabilityType; import voldemort.store.StoreUtils; import voldemort.store.metadata.MetadataStore; @@ -45,36 +45,35 @@ * * */ -public class ConfigurationStorageEngine implements StorageEngine { +public class ConfigurationStorageEngine extends AbstractStorageEngine { private final static Logger logger = Logger.getLogger(ConfigurationStorageEngine.class); - private final String name; private final File directory; public ConfigurationStorageEngine(String name, String directory) { - this.name = name; + super(name); this.directory = new File(directory); if(!this.directory.exists() && this.directory.canRead()) throw new IllegalArgumentException("Directory " + this.directory.getAbsolutePath() + " does not exist or can not be read."); } + @Override public ClosableIterator>> entries() { throw new VoldemortException("Iteration not supported in ConfigurationStorageEngine"); } + @Override public ClosableIterator>> entries(int partition) { throw new UnsupportedOperationException("Partition based entries scan not supported for this storage type"); } + @Override public ClosableIterator keys(int partition) { throw new UnsupportedOperationException("Partition based key scan not supported for this storage type"); } - public void close() throws VoldemortException { - - } - + @Override public synchronized boolean delete(String key, Version version) throws VoldemortException { StoreUtils.assertValidKey(key); for(File file: getDirectory(key).listFiles()) { @@ -92,12 +91,14 @@ public synchronized boolean delete(String key, Version version) throws Voldemort return false; } + @Override public synchronized List> get(String key, String transforms) throws VoldemortException { StoreUtils.assertValidKey(key); return get(key, getDirectory(key).listFiles()); } + @Override public List getVersions(String key) { List> values = get(key, (String) null); List versions = new ArrayList(values.size()); @@ -107,6 +108,7 @@ public List getVersions(String key) { return versions; } + @Override public synchronized Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { @@ -120,10 +122,7 @@ public synchronized Map>> getAll(Iterable return result; } - public String getName() { - return name; - } - + @Override public synchronized void put(String key, Versioned value, String transforms) throws VoldemortException { StoreUtils.assertValidKey(key); @@ -236,33 +235,18 @@ private File getTempDirectory() { return tempDir; } + @Override public Object getCapability(StoreCapabilityType capability) { throw new VoldemortException("No extra capability."); } + @Override public ClosableIterator keys() { throw new VoldemortException("keys iteration not supported."); } + @Override public void truncate() { throw new VoldemortException("Truncate not supported in ConfigurationStorageEngine"); } - - public boolean isPartitionAware() { - return false; - } - - public boolean isPartitionScanSupported() { - return false; - } - - @Override - public boolean beginBatchModifications() { - return false; - } - - @Override - public boolean endBatchModifications() { - return false; - } } diff --git a/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java b/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java index 50b646e094..4dce7d6023 100644 --- a/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java +++ b/src/java/voldemort/store/configuration/FileBackedCachingStorageEngine.java @@ -35,7 +35,7 @@ import org.apache.log4j.Logger; import voldemort.VoldemortException; -import voldemort.store.StorageEngine; +import voldemort.store.AbstractStorageEngine; import voldemort.store.StoreCapabilityType; import voldemort.store.StoreUtils; import voldemort.utils.ByteArray; @@ -59,7 +59,8 @@ * @author csoman * */ -public class FileBackedCachingStorageEngine implements StorageEngine { +public class FileBackedCachingStorageEngine extends + AbstractStorageEngine { private final static Logger logger = Logger.getLogger(FileBackedCachingStorageEngine.class); private static final CharSequence NEW_PROPERTY_SEPARATOR = "[name="; @@ -67,12 +68,11 @@ public class FileBackedCachingStorageEngine implements StorageEngine metadataMap; private VectorClock cachedVersion = null; public FileBackedCachingStorageEngine(String name, String inputDirectory) { - this.name = name; + super(name); this.inputDirectory = inputDirectory; File directory = new File(this.inputDirectory); if(!directory.exists() && directory.canRead()) { @@ -89,7 +89,7 @@ public FileBackedCachingStorageEngine(String name, String inputDirectory) { } private File getVersionFile() { - return new File(this.inputDirectory, this.name + ".version"); + return new File(this.inputDirectory, getName() + ".version"); } // Read the Vector clock stored in '${name}.version' file @@ -106,7 +106,7 @@ private VectorClock readVersion() { } return this.cachedVersion; } catch(Exception e) { - throw new VoldemortException("Failed to read Version for file :" + this.name, e); + throw new VoldemortException("Failed to read Version for file :" + getName(), e); } } @@ -121,7 +121,7 @@ private void writeVersion(VectorClock newClock) { } } catch(Exception e) { throw new VoldemortException("Failed to write Version for the current file :" - + this.name, e); + + getName(), e); } } @@ -190,33 +190,28 @@ private synchronized void flushData() { } } - public String getName() { - return this.name; - } - - public void close() throws VoldemortException {} - + @Override public Object getCapability(StoreCapabilityType capability) { throw new VoldemortException("No extra capability."); } + @Override public ClosableIterator>> entries() { return new FileBackedStorageIterator(this.metadataMap, this); } + @Override public ClosableIterator keys() { return StoreUtils.keys(entries()); } + @Override 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 ? + @Override public List> get(ByteArray key, byte[] transforms) throws VoldemortException { StoreUtils.assertValidKey(key); String keyString = new String(key.get()); @@ -230,6 +225,7 @@ public List> get(ByteArray key, byte[] transforms) throws Vold return found; } + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { @@ -243,6 +239,7 @@ public Map>> getAll(Iterable keys, return result; } + @Override public List getVersions(ByteArray key) { List> values = get(key, null); List versions = new ArrayList(values.size()); @@ -252,6 +249,7 @@ public List getVersions(ByteArray key) { return versions; } + @Override public void put(ByteArray key, Versioned value, byte[] transforms) throws VoldemortException { StoreUtils.assertValidKey(key); @@ -262,7 +260,7 @@ public void put(ByteArray key, Versioned value, byte[] transforms) 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); + + " exists for the current file : " + getName()); } else if(value.getVersion().compare(clock) == Occurred.CONCURRENTLY) { throw new ObsoleteVersionException("Concurrent Operation not allowed on Metadata."); } @@ -278,6 +276,7 @@ public void put(ByteArray key, Versioned value, byte[] transforms) writeVersion((VectorClock) value.getVersion()); } + @Override public boolean delete(ByteArray key, Version version) throws VoldemortException { boolean deleteSuccessful = false; StoreUtils.assertValidKey(key); @@ -307,10 +306,12 @@ public FileBackedStorageIterator(Map metadataMap, storageEngineRef = storageEngine; } + @Override public boolean hasNext() { return iterator.hasNext(); } + @Override public Pair> next() { Entry entry = iterator.next(); Pair> nextValue = null; @@ -325,10 +326,12 @@ public Pair> next() { return nextValue; } + @Override public void remove() { throw new UnsupportedOperationException("No removal y'all."); } + @Override public void close() {} } @@ -342,20 +345,4 @@ public ClosableIterator>> entries(int partitio public ClosableIterator keys(int partition) { throw new UnsupportedOperationException("Partition based keys scan not supported for this storage type"); } - - @Override - public boolean isPartitionScanSupported() { - return false; - } - - @Override - public boolean beginBatchModifications() { - return false; - } - - @Override - public boolean endBatchModifications() { - return false; - } - } diff --git a/src/java/voldemort/store/gzip/GzipStore.java b/src/java/voldemort/store/gzip/GzipStore.java index dcdc3ad465..6a36f4c97e 100644 --- a/src/java/voldemort/store/gzip/GzipStore.java +++ b/src/java/voldemort/store/gzip/GzipStore.java @@ -34,8 +34,7 @@ * * */ -public class GzipStore extends DelegatingStore implements - Store { +public class GzipStore extends DelegatingStore { public GzipStore(Store innerStore) { super(innerStore); diff --git a/src/java/voldemort/store/http/HttpStore.java b/src/java/voldemort/store/http/HttpStore.java index 9c476178df..a67aa56e91 100644 --- a/src/java/voldemort/store/http/HttpStore.java +++ b/src/java/voldemort/store/http/HttpStore.java @@ -33,9 +33,7 @@ import voldemort.VoldemortException; import voldemort.client.protocol.RequestFormat; import voldemort.server.RequestRoutingType; -import voldemort.store.NoSuchCapabilityException; -import voldemort.store.Store; -import voldemort.store.StoreCapabilityType; +import voldemort.store.AbstractStore; import voldemort.store.StoreUtils; import voldemort.store.UnreachableStoreException; import voldemort.utils.ByteArray; @@ -49,9 +47,8 @@ * the VoldemortHttpServer. * */ -public class HttpStore implements Store { +public class HttpStore extends AbstractStore { - private final String storeName; private final HttpClient httpClient; private final RequestFormat requestFormat; private final RequestRoutingType reroute; @@ -63,13 +60,14 @@ public HttpStore(String storeName, HttpClient client, RequestFormat format, boolean reroute) { - this.storeName = storeName; + super(storeName); this.httpClient = client; this.requestFormat = format; this.reroute = RequestRoutingType.getRequestRoutingType(reroute, false); this.storeUrl = "http://" + host + ":" + port + "/stores"; } + @Override public boolean delete(ByteArray key, Version version) throws VoldemortException { StoreUtils.assertValidKey(key); DataInputStream input = null; @@ -77,7 +75,7 @@ public boolean delete(ByteArray key, Version version) throws VoldemortException HttpPost method = new HttpPost(this.storeUrl); ByteArrayOutputStream outputBytes = new ByteArrayOutputStream(); requestFormat.writeDeleteRequest(new DataOutputStream(outputBytes), - storeName, + getName(), key, (VectorClock) version, reroute); @@ -85,12 +83,13 @@ public boolean delete(ByteArray key, Version version) throws VoldemortException return requestFormat.readDeleteResponse(input); } catch(IOException e) { throw new UnreachableStoreException("Could not connect to " + storeUrl + " for " - + storeName, e); + + getName(), e); } finally { IOUtils.closeQuietly(input); } } + @Override public List> get(ByteArray key, byte[] transforms) throws VoldemortException { StoreUtils.assertValidKey(key); DataInputStream input = null; @@ -98,7 +97,7 @@ public List> get(ByteArray key, byte[] transforms) throws Vold HttpPost method = new HttpPost(this.storeUrl); ByteArrayOutputStream outputBytes = new ByteArrayOutputStream(); requestFormat.writeGetRequest(new DataOutputStream(outputBytes), - storeName, + getName(), key, transforms, reroute); @@ -106,12 +105,13 @@ public List> get(ByteArray key, byte[] transforms) throws Vold return requestFormat.readGetResponse(input); } catch(IOException e) { throw new UnreachableStoreException("Could not connect to " + storeUrl + " for " - + storeName, e); + + getName(), e); } finally { IOUtils.closeQuietly(input); } } + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { @@ -121,7 +121,7 @@ public Map>> getAll(Iterable keys, HttpPost method = new HttpPost(this.storeUrl); ByteArrayOutputStream outputBytes = new ByteArrayOutputStream(); requestFormat.writeGetAllRequest(new DataOutputStream(outputBytes), - storeName, + getName(), keys, transforms, reroute); @@ -129,12 +129,13 @@ public Map>> getAll(Iterable keys, return requestFormat.readGetAllResponse(input); } catch(IOException e) { throw new UnreachableStoreException("Could not connect to " + storeUrl + " for " - + storeName, e); + + getName(), e); } finally { IOUtils.closeQuietly(input); } } + @Override public void put(ByteArray key, Versioned versioned, byte[] transforms) throws VoldemortException { StoreUtils.assertValidKey(key); @@ -143,7 +144,7 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) HttpPost method = new HttpPost(this.storeUrl); ByteArrayOutputStream outputBytes = new ByteArrayOutputStream(); requestFormat.writePutRequest(new DataOutputStream(outputBytes), - storeName, + getName(), key, versioned.getValue(), transforms, @@ -153,7 +154,7 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) requestFormat.readPutResponse(input); } catch(IOException e) { throw new UnreachableStoreException("Could not connect to " + storeUrl + " for " - + storeName, e); + + getName(), e); } finally { IOUtils.closeQuietly(input); } @@ -168,7 +169,7 @@ private DataInputStream executeRequest(HttpPost method, ByteArrayOutputStream ou if(statusCode != HttpURLConnection.HTTP_OK) { String message = response.getStatusLine().getReasonPhrase(); VoldemortIOUtils.closeQuietly(response); - throw new UnreachableStoreException("HTTP request to store " + storeName + throw new UnreachableStoreException("HTTP request to store " + getName() + " returned status code " + statusCode + " " + message); } @@ -176,20 +177,11 @@ private DataInputStream executeRequest(HttpPost method, ByteArrayOutputStream ou } catch(IOException e) { VoldemortIOUtils.closeQuietly(response); throw new UnreachableStoreException("Could not connect to " + storeUrl + " for " - + storeName, e); + + getName(), e); } } - public void close() {} - - public String getName() { - return storeName; - } - - public Object getCapability(StoreCapabilityType capability) { - throw new NoSuchCapabilityException(capability, getName()); - } - + @Override public List getVersions(ByteArray key) { StoreUtils.assertValidKey(key); DataInputStream input = null; @@ -197,14 +189,14 @@ public List getVersions(ByteArray key) { HttpPost method = new HttpPost(this.storeUrl); ByteArrayOutputStream outputBytes = new ByteArrayOutputStream(); requestFormat.writeGetVersionRequest(new DataOutputStream(outputBytes), - storeName, + getName(), key, reroute); input = executeRequest(method, outputBytes); return requestFormat.readGetVersionResponse(input); } catch(IOException e) { throw new UnreachableStoreException("Could not connect to " + storeUrl + " for " - + storeName, e); + + getName(), e); } finally { IOUtils.closeQuietly(input); } diff --git a/src/java/voldemort/store/logging/LoggingStore.java b/src/java/voldemort/store/logging/LoggingStore.java index cac0acebb9..77cf8c5389 100644 --- a/src/java/voldemort/store/logging/LoggingStore.java +++ b/src/java/voldemort/store/logging/LoggingStore.java @@ -24,6 +24,7 @@ import voldemort.store.DelegatingStore; import voldemort.store.Store; import voldemort.store.StoreCapabilityType; +import voldemort.store.VoldemortRequestWrapper; import voldemort.utils.SystemTime; import voldemort.utils.Time; import voldemort.versioning.Version; @@ -141,4 +142,33 @@ public Object getCapability(StoreCapabilityType capability) { return getInnerStore().getCapability(capability); } + @Override + public List> get(VoldemortRequestWrapper request) throws VoldemortException { + long startTimeNs = 0; + boolean succeeded = false; + if(logger.isDebugEnabled()) + startTimeNs = time.getNanoseconds(); + try { + List> l = getInnerStore().get(request); + succeeded = true; + return l; + } finally { + printTimedMessage("GET", succeeded, startTimeNs); + } + } + + @Override + public void put(VoldemortRequestWrapper request) throws VoldemortException { + long startTimeNs = 0; + boolean succeeded = false; + if(logger.isDebugEnabled()) { + startTimeNs = time.getNanoseconds(); + } + try { + getInnerStore().put(request); + succeeded = true; + } finally { + printTimedMessage("PUT", succeeded, startTimeNs); + } + } } diff --git a/src/java/voldemort/store/memory/InMemoryStorageEngine.java b/src/java/voldemort/store/memory/InMemoryStorageEngine.java index a3866c14b0..f53d0ef60b 100644 --- a/src/java/voldemort/store/memory/InMemoryStorageEngine.java +++ b/src/java/voldemort/store/memory/InMemoryStorageEngine.java @@ -26,9 +26,7 @@ import voldemort.VoldemortException; import voldemort.annotations.concurrency.NotThreadsafe; -import voldemort.store.NoSuchCapabilityException; -import voldemort.store.StorageEngine; -import voldemort.store.StoreCapabilityType; +import voldemort.store.AbstractStorageEngine; import voldemort.store.StoreUtils; import voldemort.utils.ClosableIterator; import voldemort.utils.Pair; @@ -43,23 +41,20 @@ * * */ -public class InMemoryStorageEngine implements StorageEngine { +public class InMemoryStorageEngine extends AbstractStorageEngine { private final ConcurrentMap>> map; - private final String name; public InMemoryStorageEngine(String name) { - this.name = Utils.notNull(name); + super(name); this.map = new ConcurrentHashMap>>(); } public InMemoryStorageEngine(String name, ConcurrentMap>> map) { - this.name = Utils.notNull(name); + super(name); this.map = Utils.notNull(map); } - public void close() {} - public void deleteAll() { this.map.clear(); } @@ -68,6 +63,7 @@ public boolean delete(K key) { return delete(key, null); } + @Override public boolean delete(K key, Version version) { StoreUtils.assertValidKey(key); @@ -99,10 +95,12 @@ public boolean delete(K key, Version version) { } } + @Override public List getVersions(K key) { return StoreUtils.getVersions(get(key, null)); } + @Override public List> get(K key, T transform) throws VoldemortException { StoreUtils.assertValidKey(key); List> results = map.get(key); @@ -114,12 +112,14 @@ public List> get(K key, T transform) throws VoldemortException { } } + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { StoreUtils.assertValidKeys(keys); return StoreUtils.getAll(this, keys, transforms); } + @Override public void put(K key, Versioned value, T transforms) throws VoldemortException { StoreUtils.assertValidKey(key); @@ -159,35 +159,32 @@ public void put(K key, Versioned value, T transforms) throws VoldemortExcepti } } - public Object getCapability(StoreCapabilityType capability) { - throw new NoSuchCapabilityException(capability, getName()); - } - + @Override public ClosableIterator>> entries() { return new InMemoryIterator(map); } + @Override public ClosableIterator keys() { // TODO Implement more efficient version. return StoreUtils.keys(entries()); } + @Override public ClosableIterator>> entries(int partition) { throw new UnsupportedOperationException("Partition based entries scan not supported for this storage type"); } + @Override public ClosableIterator keys(int partition) { throw new UnsupportedOperationException("Partition based key scan not supported for this storage type"); } + @Override public void truncate() { map.clear(); } - public String getName() { - return name; - } - @Override public String toString() { return toString(15); @@ -222,6 +219,7 @@ public InMemoryIterator(ConcurrentMap>> map) { this.iterator = map.entrySet().iterator(); } + @Override public boolean hasNext() { return hasNextInCurrentValues() || iterator.hasNext(); } @@ -235,6 +233,7 @@ private Pair> nextInCurrentValues() { return Pair.create(currentKey, item); } + @Override public Pair> next() { if(hasNextInCurrentValues()) { return nextInCurrentValues(); @@ -261,31 +260,15 @@ public Pair> next() { } } + @Override public void remove() { throw new UnsupportedOperationException("No removal y'all."); } + @Override public void close() { // nothing to do here } } - - public boolean isPartitionAware() { - return false; - } - - public boolean isPartitionScanSupported() { - return false; - } - - @Override - public boolean beginBatchModifications() { - return false; - } - - @Override - public boolean endBatchModifications() { - return false; - } } diff --git a/src/java/voldemort/store/metadata/MetadataStore.java b/src/java/voldemort/store/metadata/MetadataStore.java index 4ed271cf59..c6f1f1b46f 100644 --- a/src/java/voldemort/store/metadata/MetadataStore.java +++ b/src/java/voldemort/store/metadata/MetadataStore.java @@ -42,7 +42,7 @@ import voldemort.routing.RoutingStrategy; import voldemort.routing.RoutingStrategyFactory; import voldemort.server.rebalance.RebalancerState; -import voldemort.store.StorageEngine; +import voldemort.store.AbstractStorageEngine; import voldemort.store.Store; import voldemort.store.StoreCapabilityType; import voldemort.store.StoreDefinition; @@ -68,7 +68,7 @@ * Metadata is persisted as strings in inner store for ease of readability.
* Metadata Store keeps an in memory write-through-cache for performance. */ -public class MetadataStore implements StorageEngine { +public class MetadataStore extends AbstractStorageEngine { public static final String METADATA_STORE_NAME = "metadata"; @@ -119,6 +119,7 @@ public static enum VoldemortState { private static final Logger logger = Logger.getLogger(MetadataStore.class); public MetadataStore(Store innerStore, int nodeId) { + super(innerStore.getName()); this.innerStore = innerStore; this.metadataCache = new HashMap>(); this.storeNameTolisteners = new ConcurrentHashMap>(); @@ -154,6 +155,7 @@ public static MetadataStore readFromDirectory(File dir, int nodeId) { return new MetadataStore(innerStore, nodeId); } + @Override public String getName() { return METADATA_STORE_NAME; } @@ -213,6 +215,7 @@ public void put(String key, Object value) { * definitions * @throws VoldemortException */ + @Override public synchronized void put(ByteArray keyBytes, Versioned valueBytes, byte[] transforms) throws VoldemortException { String key = ByteUtils.getString(keyBytes.get(), "UTF-8"); @@ -225,10 +228,12 @@ public synchronized void put(ByteArray keyBytes, Versioned valueBytes, b this.put(key, valueObject); } + @Override public void close() throws VoldemortException { innerStore.close(); } + @Override public Object getCapability(StoreCapabilityType capability) { return innerStore.getCapability(capability); } @@ -239,6 +244,7 @@ public Object getCapability(StoreCapabilityType capability) { * bytes for cluster xml definitions * @throws VoldemortException */ + @Override public synchronized List> get(ByteArray keyBytes, byte[] transforms) throws VoldemortException { try { @@ -286,6 +292,7 @@ public synchronized void cleanAllRebalancingState() { init(getNodeId()); } + @Override public List getVersions(ByteArray key) { List> values = get(key, null); List versions = new ArrayList(values.size()); @@ -473,30 +480,37 @@ public void deleteRebalancingState(RebalancePartitionsInfo stealInfo) { } } + @Override public ClosableIterator>> entries() { throw new VoldemortException("You cannot iterate over all entries in Metadata"); } + @Override public ClosableIterator keys() { throw new VoldemortException("You cannot iterate over all keys in Metadata"); } + @Override public ClosableIterator>> entries(int partition) { throw new UnsupportedOperationException("Partition based entries scan not supported for this storage type"); } + @Override public ClosableIterator keys(int partition) { throw new UnsupportedOperationException("Partition based key scan not supported for this storage type"); } + @Override public void truncate() { throw new VoldemortException("You cannot truncate entries in Metadata"); } + @Override public boolean delete(ByteArray key, Version version) throws VoldemortException { throw new VoldemortException("You cannot delete your metadata fool !!"); } + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { @@ -650,22 +664,4 @@ private Versioned getInnerValue(String key) throws VoldemortException { throw new VoldemortException("No metadata found for required key:" + key); } - - public boolean isPartitionAware() { - return false; - } - - public boolean isPartitionScanSupported() { - return false; - } - - @Override - public boolean beginBatchModifications() { - return false; - } - - @Override - public boolean endBatchModifications() { - return false; - } } diff --git a/src/java/voldemort/store/mysql/MysqlStorageEngine.java b/src/java/voldemort/store/mysql/MysqlStorageEngine.java index bbca08676d..bf26b4be36 100644 --- a/src/java/voldemort/store/mysql/MysqlStorageEngine.java +++ b/src/java/voldemort/store/mysql/MysqlStorageEngine.java @@ -28,10 +28,8 @@ import org.apache.log4j.Logger; import voldemort.VoldemortException; -import voldemort.store.NoSuchCapabilityException; +import voldemort.store.AbstractStorageEngine; import voldemort.store.PersistenceFailureException; -import voldemort.store.StorageEngine; -import voldemort.store.StoreCapabilityType; import voldemort.store.StoreUtils; import voldemort.utils.ByteArray; import voldemort.utils.ClosableIterator; @@ -49,17 +47,16 @@ * * */ -public class MysqlStorageEngine implements StorageEngine { +public class MysqlStorageEngine extends AbstractStorageEngine { private static final Logger logger = Logger.getLogger(MysqlStorageEngine.class); private static int MYSQL_ERR_DUP_KEY = 1022; private static int MYSQL_ERR_DUP_ENTRY = 1062; - private final String name; private final DataSource datasource; public MysqlStorageEngine(String name, DataSource datasource) { - this.name = name; + super(name); this.datasource = datasource; if(!tableExists()) { @@ -112,14 +109,16 @@ public void execute(String query) { } } + @Override public ClosableIterator keys() { return StoreUtils.keys(entries()); } + @Override public void truncate() { Connection conn = null; PreparedStatement stmt = null; - String select = "delete from " + name; + String select = "delete from " + getName(); try { conn = datasource.getConnection(); stmt = conn.prepareStatement(select); @@ -132,11 +131,12 @@ public void truncate() { } } + @Override public ClosableIterator>> entries() { Connection conn = null; PreparedStatement stmt = null; ResultSet rs = null; - String select = "select key_, version_, value_ from " + name; + String select = "select key_, version_, value_ from " + getName(); try { conn = datasource.getConnection(); stmt = conn.prepareStatement(select); @@ -147,28 +147,28 @@ public ClosableIterator>> entries() { } } + @Override public ClosableIterator>> entries(int partition) { throw new UnsupportedOperationException("Partition based entries scan not supported for this storage type"); } + @Override public ClosableIterator keys(int partition) { throw new UnsupportedOperationException("Partition based key scan not supported for this storage type"); } + @Override public void close() throws PersistenceFailureException { // don't close datasource cause others could be using it } - public Object getCapability(StoreCapabilityType capability) { - throw new NoSuchCapabilityException(capability, getName()); - } - + @Override public boolean delete(ByteArray key, Version maxVersion) throws PersistenceFailureException { StoreUtils.assertValidKey(key); Connection conn = null; PreparedStatement selectStmt = null; ResultSet rs = null; - String select = "select key_, version_ from " + name + " where key_ = ? for update"; + String select = "select key_, version_ from " + getName() + " where key_ = ? for update"; try { conn = datasource.getConnection(); @@ -196,7 +196,7 @@ public boolean delete(ByteArray key, Version maxVersion) throws PersistenceFailu } private void delete(Connection connection, byte[] key, byte[] version) throws SQLException { - String delete = "delete from " + name + " where key_ = ? and version_ = ?"; + String delete = "delete from " + getName() + " where key_ = ? and version_ = ?"; PreparedStatement deleteStmt = null; try { @@ -209,6 +209,7 @@ private void delete(Connection connection, byte[] key, byte[] version) throws SQ } } + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { @@ -216,7 +217,7 @@ public Map>> getAll(Iterable keys, Connection conn = null; PreparedStatement stmt = null; ResultSet rs = null; - String select = "select version_, value_ from " + name + " where key_ = ?"; + String select = "select version_, value_ from " + getName() + " where key_ = ?"; try { conn = datasource.getConnection(); stmt = conn.prepareStatement(select); @@ -243,16 +244,14 @@ public Map>> getAll(Iterable keys, } } + @Override public List> get(ByteArray key, byte[] transforms) throws PersistenceFailureException { StoreUtils.assertValidKey(key); return StoreUtils.get(this, key, transforms); } - public String getName() { - return name; - } - + @Override public void put(ByteArray key, Versioned value, byte[] transforms) throws PersistenceFailureException { StoreUtils.assertValidKey(key); @@ -261,8 +260,9 @@ public void put(ByteArray key, Versioned value, byte[] transforms) PreparedStatement insert = null; PreparedStatement select = null; ResultSet results = null; - String insertSql = "insert into " + name + " (key_, version_, value_) values (?, ?, ?)"; - String selectSql = "select key_, version_ from " + name + " where key_ = ?"; + String insertSql = "insert into " + getName() + + " (key_, version_, value_) values (?, ?, ?)"; + String selectSql = "select key_, version_ from " + getName() + " where key_ = ?"; try { conn = datasource.getConnection(); conn.setAutoCommit(false); @@ -363,16 +363,19 @@ public MysqlClosableIterator(Connection connection, this.statement = statement; } + @Override public void close() { tryClose(rs); tryClose(statement); tryClose(connection); } + @Override public boolean hasNext() { return this.hasMore; } + @Override public Pair> next() { try { if(!this.hasMore) @@ -387,6 +390,7 @@ public Pair> next() { } } + @Override public void remove() { try { rs.deleteRow(); @@ -397,26 +401,8 @@ public void remove() { } + @Override public List getVersions(ByteArray key) { return StoreUtils.getVersions(get(key, null)); } - - public boolean isPartitionAware() { - return false; - } - - public boolean isPartitionScanSupported() { - // no reason why we cannot do this on MySQL. Will be added later - return false; - } - - @Override - public boolean beginBatchModifications() { - return false; - } - - @Override - public boolean endBatchModifications() { - return false; - } } diff --git a/src/java/voldemort/store/readonly/ReadOnlyStorageEngine.java b/src/java/voldemort/store/readonly/ReadOnlyStorageEngine.java index 713bc3733d..dece869483 100644 --- a/src/java/voldemort/store/readonly/ReadOnlyStorageEngine.java +++ b/src/java/voldemort/store/readonly/ReadOnlyStorageEngine.java @@ -34,9 +34,7 @@ import voldemort.annotations.jmx.JmxGetter; import voldemort.annotations.jmx.JmxOperation; import voldemort.routing.RoutingStrategy; -import voldemort.store.NoSuchCapabilityException; -import voldemort.store.StorageEngine; -import voldemort.store.StoreCapabilityType; +import voldemort.store.AbstractStorageEngine; import voldemort.store.StoreUtils; import voldemort.store.readonly.chunk.ChunkedFileSet; import voldemort.utils.ByteArray; @@ -54,11 +52,10 @@ * * */ -public class ReadOnlyStorageEngine implements StorageEngine { +public class ReadOnlyStorageEngine extends AbstractStorageEngine { private static Logger logger = Logger.getLogger(ReadOnlyStorageEngine.class); - private final String name; private final int numBackups, nodeId; private long currentVersionId; private final File storeDir; @@ -125,10 +122,10 @@ public ReadOnlyStorageEngine(String name, int deleteBackupMs, boolean enforceMlock) { + super(name); this.enforceMlock = enforceMlock; this.storeDir = storeDir; this.numBackups = numBackups; - this.name = Utils.notNull(name); this.searchStrategy = searchStrategy; this.routingStrategy = Utils.notNull(routingStrategy); this.nodeId = nodeId; @@ -273,6 +270,7 @@ public long getLastSwapped() { /** * Close the store. */ + @Override public void close() throws VoldemortException { logger.debug("Close called for read-only store."); this.fileModificationLock.writeLock().lock(); @@ -389,6 +387,7 @@ private void deleteBackups() { private void deleteAsync(final File file) { new Thread(new Runnable() { + @Override public void run() { try { try { @@ -466,6 +465,7 @@ public void rollback(File rollbackToDir) { } } + @Override public ClosableIterator keys() { if(!(fileSet.getReadOnlyStorageFormat().compareTo(ReadOnlyStorageFormat.READONLY_V2) == 0)) throw new UnsupportedOperationException("Iteration is not supported for " @@ -475,6 +475,7 @@ public ClosableIterator keys() { return new ChunkedFileSet.ROKeyIterator(fileSet, fileModificationLock); } + @Override public ClosableIterator>> entries() { if(!(fileSet.getReadOnlyStorageFormat().compareTo(ReadOnlyStorageFormat.READONLY_V2) == 0)) throw new UnsupportedOperationException("Iteration is not supported for " @@ -484,14 +485,17 @@ public ClosableIterator>> entries() { return new ChunkedFileSet.ROEntriesIterator(fileSet, fileModificationLock); } + @Override public ClosableIterator>> entries(int partition) { throw new UnsupportedOperationException("Partition based entries scan not supported for this storage type"); } + @Override public ClosableIterator keys(int partition) { throw new UnsupportedOperationException("Partition based key scan not supported for this storage type"); } + @Override public void truncate() { if(isOpen) close(); @@ -499,6 +503,7 @@ public void truncate() { logger.debug("Truncate successful for read-only store "); } + @Override public List> get(ByteArray key, byte[] transforms) throws VoldemortException { StoreUtils.assertValidKey(key); try { @@ -526,6 +531,7 @@ public List> get(ByteArray key, byte[] transforms) throws Vold } } + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { @@ -560,6 +566,7 @@ public Map>> getAll(Iterable keys, /** * Not supported, throws UnsupportedOperationException if called */ + @Override public boolean delete(ByteArray key, Version version) throws VoldemortException { throw new UnsupportedOperationException("Delete is not supported on this store, it is read-only."); } @@ -567,18 +574,16 @@ public boolean delete(ByteArray key, Version version) throws VoldemortException /** * Not supported, throws UnsupportedOperationException if called */ + @Override public void put(ByteArray key, Versioned value, byte[] transforms) throws VoldemortException { throw new VoldemortUnsupportedOperationalException("Put is not supported on this store, it is read-only."); } @JmxGetter(name = "name", description = "The name of the store.") + @Override public String getName() { - return name; - } - - public Object getCapability(StoreCapabilityType capability) { - throw new NoSuchCapabilityException(capability, getName()); + return super.getName(); } private final static class KeyValueLocation implements Comparable { @@ -606,6 +611,7 @@ public int getValueLocation() { return valueLocation; } + @Override public int compareTo(KeyValueLocation kvl) { if(chunk == kvl.getChunk()) { if(valueLocation == kvl.getValueLocation()) @@ -618,26 +624,13 @@ public int compareTo(KeyValueLocation kvl) { } } + @Override public List getVersions(ByteArray key) { return StoreUtils.getVersions(get(key, null)); } + @Override public boolean isPartitionAware() { return true; } - - public boolean isPartitionScanSupported() { - // this should be easy to support, will be added later - return false; - } - - @Override - public boolean beginBatchModifications() { - return false; - } - - @Override - public boolean endBatchModifications() { - return false; - } } diff --git a/src/java/voldemort/store/retention/RetentionEnforcingStore.java b/src/java/voldemort/store/retention/RetentionEnforcingStore.java index 2a6bcc7cb7..9ab151d00e 100644 --- a/src/java/voldemort/store/retention/RetentionEnforcingStore.java +++ b/src/java/voldemort/store/retention/RetentionEnforcingStore.java @@ -39,6 +39,7 @@ public RetentionEnforcingStore(Store innerStore, this.time = time; } + @Override public void updateRoutingStrategy(RoutingStrategy routingStrategyMap) { return; // no-op } @@ -47,6 +48,7 @@ public void updateRoutingStrategy(RoutingStrategy routingStrategyMap) { * Updates the store definition object and the retention time based on the * updated store definition */ + @Override public void updateStoreDefinition(StoreDefinition storeDef) { this.storeDef = storeDef; if(storeDef.hasRetentionPeriod()) diff --git a/src/java/voldemort/store/routed/PipelineRoutedStore.java b/src/java/voldemort/store/routed/PipelineRoutedStore.java index 968ff5a89e..da51f809a7 100644 --- a/src/java/voldemort/store/routed/PipelineRoutedStore.java +++ b/src/java/voldemort/store/routed/PipelineRoutedStore.java @@ -33,6 +33,7 @@ import voldemort.store.StoreDefinition; import voldemort.store.StoreRequest; import voldemort.store.StoreUtils; +import voldemort.store.VoldemortRequestWrapper; import voldemort.store.nonblockingstore.NonblockingStore; import voldemort.store.routed.Pipeline.Event; import voldemort.store.routed.Pipeline.Operation; @@ -212,7 +213,14 @@ private AbstractConfigureNodes>, BasicPipeline } + @Override public List> get(final ByteArray key, final byte[] transforms) { + return get(key, transforms, timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_OP_CODE)); + } + + public List> get(final ByteArray key, + final byte[] transforms, + final long getOpTimeout) { StoreUtils.assertValidKey(key); long startTimeMs = -1; @@ -230,13 +238,12 @@ public List> get(final ByteArray key, final byte[] transforms) pipelineData.setZonesRequired(null); pipelineData.setStats(stats); - final Pipeline pipeline = new Pipeline(Operation.GET, - timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_OP_CODE), - TimeUnit.MILLISECONDS); + final Pipeline pipeline = new Pipeline(Operation.GET, getOpTimeout, TimeUnit.MILLISECONDS); boolean allowReadRepair = repairReads && transforms == null; StoreRequest>> blockingStoreRequest = new StoreRequest>>() { + @Override public List> request(Store store) { return store.get(key, transforms); } @@ -259,7 +266,7 @@ public List> request(Store store) { failureDetector, storeDef.getPreferredReads(), storeDef.getRequiredReads(), - timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_OP_CODE), + getOpTimeout, nonblockingStores, Event.INSUFFICIENT_SUCCESSES, Event.INSUFFICIENT_ZONES)); @@ -280,7 +287,7 @@ public List> request(Store store) { new ReadRepair>>>(pipelineData, Event.COMPLETED, storeDef.getPreferredReads(), - timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_OP_CODE), + getOpTimeout, nonblockingStores, readRepairer)); @@ -344,9 +351,19 @@ private String formatNodeValuesFromGet(List>> getAll(Iterable keys, Map transforms) throws VoldemortException { + return getAll(keys, + transforms, + timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_ALL_OP_CODE)); + } + + public Map>> getAll(Iterable keys, + Map transforms, + long getAllOpTimeoutInMs) + throws VoldemortException { StoreUtils.assertValidKeys(keys); long startTimeMs = -1; @@ -367,7 +384,7 @@ public Map>> getAll(Iterable keys, pipelineData.setStats(stats); Pipeline pipeline = new Pipeline(Operation.GET_ALL, - timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_ALL_OP_CODE), + getAllOpTimeoutInMs, TimeUnit.MILLISECONDS); pipeline.addEventAction(Event.STARTED, new GetAllConfigureNodes(pipelineData, @@ -383,7 +400,7 @@ public Map>> getAll(Iterable keys, new PerformParallelGetAllRequests(pipelineData, Event.INSUFFICIENT_SUCCESSES, failureDetector, - timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_ALL_OP_CODE), + getAllOpTimeoutInMs, nonblockingStores)); pipeline.addEventAction(Event.INSUFFICIENT_SUCCESSES, new PerformSerialGetAllRequests(pipelineData, @@ -401,7 +418,7 @@ public Map>> getAll(Iterable keys, new GetAllReadRepair(pipelineData, Event.COMPLETED, storeDef.getPreferredReads(), - timeoutConfig.getOperationTimeout(VoldemortOpCode.GET_ALL_OP_CODE), + getAllOpTimeoutInMs, nonblockingStores, readRepairer)); @@ -451,6 +468,7 @@ private String formatNodeValuesFromGetAll(List, Map return builder.toString(); } + @Override public List getVersions(final ByteArray key) { StoreUtils.assertValidKey(key); @@ -474,6 +492,7 @@ public List getVersions(final ByteArray key) { StoreRequest> blockingStoreRequest = new StoreRequest>() { + @Override public List request(Store store) { return store.getVersions(key); } @@ -566,6 +585,7 @@ private String formatNodeValuesFromGetVersions(List makeNodeConfigu } + @Override public void put(ByteArray key, Versioned versioned, byte[] transforms) throws VoldemortException { + put(key, + versioned, + transforms, + timeoutConfig.getOperationTimeout(VoldemortOpCode.PUT_OP_CODE)); + } + + public void put(ByteArray key, + Versioned versioned, + byte[] transforms, + long putOpTimeoutInMs) throws VoldemortException { long startTimeMs = -1; long startTimeNs = -1; @@ -727,12 +758,10 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) else pipelineData.setZonesRequired(null); pipelineData.setStartTimeNs(System.nanoTime()); - pipelineData.setStoreName(name); + pipelineData.setStoreName(getName()); pipelineData.setStats(stats); - Pipeline pipeline = new Pipeline(Operation.PUT, - timeoutConfig.getOperationTimeout(VoldemortOpCode.PUT_OP_CODE), - TimeUnit.MILLISECONDS); + Pipeline pipeline = new Pipeline(Operation.PUT, putOpTimeoutInMs, TimeUnit.MILLISECONDS); pipeline.setEnableHintedHandoff(isHintedHandoffEnabled()); HintedHandoff hintedHandoff = null; @@ -748,7 +777,7 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) nonblockingSlopStores, handoffStrategy, pipelineData.getFailedNodes(), - timeoutConfig.getOperationTimeout(VoldemortOpCode.PUT_OP_CODE)); + putOpTimeoutInMs); pipeline.addEventAction(Event.STARTED, configureNodes); @@ -772,7 +801,7 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) failureDetector, storeDef.getPreferredWrites(), storeDef.getRequiredWrites(), - timeoutConfig.getOperationTimeout(VoldemortOpCode.PUT_OP_CODE), + putOpTimeoutInMs, nonblockingStores, hintedHandoff)); if(isHintedHandoffEnabled()) { @@ -847,4 +876,27 @@ public void close() { super.close(); } + + @Override + public List> get(VoldemortRequestWrapper request) + throws VoldemortException { + return get(request.getKey(), null, request.getRoutingTimeout()); + } + + @Override + public Map>> getAll(VoldemortRequestWrapper request) + throws VoldemortException { + return getAll(request.getIterableKeys(), null, request.getRoutingTimeout()); + } + + @Override + public void put(VoldemortRequestWrapper request) throws VoldemortException { + put(request.getKey(), request.getValue(), null, request.getRoutingTimeout()); + } + + @Override + public boolean delete(VoldemortRequestWrapper request) + throws VoldemortException { + return false; + } } diff --git a/src/java/voldemort/store/routed/RoutedStore.java b/src/java/voldemort/store/routed/RoutedStore.java index a634b16447..3c4156c510 100644 --- a/src/java/voldemort/store/routed/RoutedStore.java +++ b/src/java/voldemort/store/routed/RoutedStore.java @@ -27,6 +27,7 @@ import voldemort.cluster.failuredetector.FailureDetector; import voldemort.routing.RoutingStrategy; import voldemort.routing.RoutingStrategyFactory; +import voldemort.store.AbstractStore; import voldemort.store.NoSuchCapabilityException; import voldemort.store.Store; import voldemort.store.StoreCapabilityType; @@ -40,9 +41,8 @@ * * */ -public abstract class RoutedStore implements Store { +public abstract class RoutedStore extends AbstractStore { - protected final String name; protected final Map> innerStores; protected final boolean repairReads; protected final ReadRepairer readRepairer; @@ -61,6 +61,7 @@ protected RoutedStore(String name, TimeoutConfig timeoutConfig, FailureDetector failureDetector, Time time) { + super(name); if(storeDef.getRequiredReads() < 1) throw new IllegalArgumentException("Cannot have a storeDef.getRequiredReads() number less than 1."); if(storeDef.getRequiredWrites() < 1) @@ -74,7 +75,6 @@ protected RoutedStore(String name, if(storeDef.getPreferredWrites() > innerStores.size()) throw new IllegalArgumentException("storeDef.getPreferredWrites() is larger than the total number of nodes!"); - this.name = name; this.innerStores = new ConcurrentHashMap>(innerStores); this.repairReads = repairReads; this.readRepairer = new ReadRepairer(); @@ -90,10 +90,7 @@ public void updateRoutingStrategy(RoutingStrategy routingStrategy) { this.routingStrategy = routingStrategy; } - public String getName() { - return this.name; - } - + @Override public void close() { VoldemortException exception = null; @@ -113,6 +110,7 @@ public Map> getInnerStores() { return this.innerStores; } + @Override public Object getCapability(StoreCapabilityType capability) { switch(capability) { case ROUTING_STRATEGY: diff --git a/src/java/voldemort/store/routed/ThreadPoolRoutedStore.java b/src/java/voldemort/store/routed/ThreadPoolRoutedStore.java index 947b44f42b..e1c3c8ee90 100644 --- a/src/java/voldemort/store/routed/ThreadPoolRoutedStore.java +++ b/src/java/voldemort/store/routed/ThreadPoolRoutedStore.java @@ -68,6 +68,7 @@ public class ThreadPoolRoutedStore extends RoutedStore { private final static StoreOp> VERSIONED_OP = new StoreOp>() { + @Override public List> execute(Store store, ByteArray key, byte[] transforms) { @@ -77,6 +78,7 @@ public List> execute(Store store, private final static StoreOp VERSION_OP = new StoreOp() { + @Override public List execute(Store store, ByteArray key, byte[] transforms) { @@ -150,6 +152,7 @@ public ThreadPoolRoutedStore(String name, this.executor = threadPool; } + @Override public boolean delete(final ByteArray key, final Version version) throws VoldemortException { StoreUtils.assertValidKey(key); final List nodes = availableNodes(routingStrategy.routeRequest(key.get())); @@ -178,6 +181,7 @@ public boolean delete(final ByteArray key, final Version version) throws Voldemo for(final Node node: nodes) { this.executor.execute(new Runnable() { + @Override public void run() { long startNs = System.nanoTime(); try { @@ -238,6 +242,7 @@ public void run() { return deletedSomething.get(); } + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { @@ -414,9 +419,11 @@ public Map>> getAll(Iterable keys, return result; } + @Override public List> get(ByteArray key, final byte[] transforms) { Function>>, Void> readRepairFunction = new Function>>, Void>() { + @Override public Void apply(List>> nodeResults) { List> nodeValues = Lists.newArrayListWithExpectedSize(nodeResults.size()); for(GetResult> getResult: nodeResults) @@ -581,6 +588,7 @@ private void repairReads(List> nodeValues, boolean this.executor.execute(new Runnable() { + @Override public void run() { for(NodeValue v: toReadRepair) { try { @@ -631,6 +639,7 @@ private String formatNodeValues(List> results) { return builder.toString(); } + @Override public void put(final ByteArray key, final Versioned versioned, final byte[] transforms) throws VoldemortException { long startNs = System.nanoTime(); @@ -697,6 +706,7 @@ public void put(final ByteArray key, final Versioned versioned, final by final Node node = nodes.get(currentNode); this.executor.execute(new Runnable() { + @Override public void run() { long startNsLocal = System.nanoTime(); try { @@ -806,6 +816,7 @@ private List availableNodes(List list) { return available; } + @Override public List getVersions(ByteArray key) { return get(key, null, VERSION_OP, null); } @@ -832,6 +843,7 @@ public GetCallable(Node node, ByteArray key, byte[] transforms, StoreOp fetch this.fetcher = fetcher; } + @Override public GetResult call() throws Exception { List fetched = Collections.emptyList(); Throwable exception = null; @@ -885,6 +897,7 @@ private GetAllCallable(Node node, this.transforms = transforms; } + @Override public GetAllResult call() { Map>> retrieved = Collections.emptyMap(); Throwable exception = null; diff --git a/src/java/voldemort/store/serialized/SerializingStorageEngine.java b/src/java/voldemort/store/serialized/SerializingStorageEngine.java index b1dd126edd..8ab9843b4b 100644 --- a/src/java/voldemort/store/serialized/SerializingStorageEngine.java +++ b/src/java/voldemort/store/serialized/SerializingStorageEngine.java @@ -53,22 +53,27 @@ public static SerializingStorageEngine wrap(StorageEngi return new SerializingStorageEngine(s, k, v, t); } + @Override public ClosableIterator>> entries() { return new EntriesIterator(storageEngine.entries()); } + @Override public ClosableIterator keys() { return new KeysIterator(storageEngine.keys()); } + @Override public ClosableIterator>> entries(int partition) { return new EntriesIterator(storageEngine.entries(partition)); } + @Override public ClosableIterator keys(int partition) { return new KeysIterator(storageEngine.keys(partition)); } + @Override public void truncate() { storageEngine.truncate(); } @@ -81,10 +86,12 @@ public KeysIterator(ClosableIterator iterator) { this.iterator = iterator; } + @Override public boolean hasNext() { return iterator.hasNext(); } + @Override public K next() { ByteArray key = iterator.next(); if(key == null) @@ -92,10 +99,12 @@ public K next() { return getKeySerializer().toObject(key.get()); } + @Override public void remove() { iterator.remove(); } + @Override public void close() { iterator.close(); } @@ -109,10 +118,12 @@ public EntriesIterator(ClosableIterator>> iter this.iterator = iterator; } + @Override public boolean hasNext() { return iterator.hasNext(); } + @Override public Pair> next() { Pair> keyAndVal = iterator.next(); if(keyAndVal == null) { @@ -126,19 +137,23 @@ public Pair> next() { } + @Override public void remove() { iterator.remove(); } + @Override public void close() { iterator.close(); } } + @Override public boolean isPartitionAware() { return storageEngine.isPartitionAware(); } + @Override public boolean isPartitionScanSupported() { return storageEngine.isPartitionScanSupported(); } diff --git a/src/java/voldemort/store/serialized/SerializingStore.java b/src/java/voldemort/store/serialized/SerializingStore.java index ffc5065d94..9ed67f1df4 100644 --- a/src/java/voldemort/store/serialized/SerializingStore.java +++ b/src/java/voldemort/store/serialized/SerializingStore.java @@ -22,6 +22,7 @@ import voldemort.VoldemortException; import voldemort.serialization.Serializer; +import voldemort.store.AbstractStore; import voldemort.store.Store; import voldemort.store.StoreCapabilityType; import voldemort.store.StoreUtils; @@ -42,7 +43,7 @@ * @param The type of the value being stored * @param The type of transform */ -public class SerializingStore implements Store { +public class SerializingStore extends AbstractStore { private final Store store; private final Serializer keySerializer; @@ -53,6 +54,7 @@ public SerializingStore(Store store, Serializer keySerializer, Serializer valueSerializer, Serializer transformsSerializer) { + super(store.getName()); this.store = Utils.notNull(store); this.keySerializer = Utils.notNull(keySerializer); this.valueSerializer = Utils.notNull(valueSerializer); @@ -66,6 +68,7 @@ public static SerializingStore wrap(Store(s, k, v, t); } + @Override public boolean delete(K key, Version version) throws VoldemortException { return store.delete(keyToBytes(key), version); } @@ -99,6 +102,7 @@ private Map transformsToBytes(Map transforms) { return result; } + @Override public List> get(K key, T transforms) throws VoldemortException { List> found = store.get(keyToBytes(key), (transformsSerializer != null && transforms != null) ? transformsSerializer.toBytes(transforms) @@ -110,6 +114,7 @@ public List> get(K key, T transforms) throws VoldemortException { return results; } + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { StoreUtils.assertValidKeys(keys); @@ -129,10 +134,7 @@ public Map>> getAll(Iterable keys, Map transforms) return result; } - public String getName() { - return store.getName(); - } - + @Override public void put(K key, Versioned value, T transforms) throws VoldemortException { store.put(keyToBytes(key), new Versioned(valueSerializer.toBytes(value.getValue()), @@ -140,10 +142,12 @@ public void put(K key, Versioned value, T transforms) throws VoldemortExcepti transformToBytes(transforms)); } + @Override public List getVersions(K key) { return store.getVersions(keyToBytes(key)); } + @Override public void close() { store.close(); } @@ -156,6 +160,7 @@ protected Serializer getKeySerializer() { return keySerializer; } + @Override public Object getCapability(StoreCapabilityType capability) { switch(capability) { case KEY_SERIALIZER: @@ -166,5 +171,4 @@ public Object getCapability(StoreCapabilityType capability) { return store.getCapability(capability); } } - } diff --git a/src/java/voldemort/store/slop/SlopStorageEngine.java b/src/java/voldemort/store/slop/SlopStorageEngine.java index 796dc819c4..7f95b0b03a 100644 --- a/src/java/voldemort/store/slop/SlopStorageEngine.java +++ b/src/java/voldemort/store/slop/SlopStorageEngine.java @@ -25,6 +25,7 @@ import voldemort.serialization.ByteArraySerializer; import voldemort.serialization.IdentitySerializer; import voldemort.serialization.SlopSerializer; +import voldemort.store.AbstractStorageEngine; import voldemort.store.StorageEngine; import voldemort.store.StoreCapabilityType; import voldemort.store.serialized.SerializingStorageEngine; @@ -41,7 +42,7 @@ * last run * */ -public class SlopStorageEngine implements StorageEngine { +public class SlopStorageEngine extends AbstractStorageEngine { public static final String SLOP_STORE_NAME = "slop"; @@ -50,6 +51,7 @@ public class SlopStorageEngine implements StorageEngine slopEngine, Cluster cluster) { + super(slopEngine.getName()); this.slopEngine = slopEngine; this.slopSerializer = new SlopSerializer(); this.slopStats = new SlopStats(cluster); @@ -81,65 +83,75 @@ public StorageEngine asSlopStore() { new IdentitySerializer()); } + @Override public ClosableIterator>> entries() { return slopEngine.entries(); } + @Override public ClosableIterator keys() { return slopEngine.keys(); } + @Override public ClosableIterator>> entries(int partition) { return slopEngine.entries(partition); } + @Override public ClosableIterator keys(int partition) { return slopEngine.keys(partition); } + @Override public void truncate() { slopEngine.truncate(); } + @Override public List> get(ByteArray key, byte[] transforms) throws VoldemortException { return slopEngine.get(key, transforms); } + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { return slopEngine.getAll(keys, transforms); } + @Override public void put(ByteArray key, Versioned value, byte[] transforms) throws VoldemortException { slopEngine.put(key, value, transforms); } + @Override public boolean delete(ByteArray key, Version version) throws VoldemortException { return slopEngine.delete(key, version); } - public String getName() { - return slopEngine.getName(); - } - + @Override public void close() throws VoldemortException { slopEngine.close(); } + @Override public Object getCapability(StoreCapabilityType capability) { return slopEngine.getCapability(capability); } + @Override public List getVersions(ByteArray key) { return slopEngine.getVersions(key); } + @Override public boolean isPartitionAware() { return slopEngine.isPartitionAware(); } + @Override public boolean isPartitionScanSupported() { return slopEngine.isPartitionScanSupported(); } diff --git a/src/java/voldemort/store/socket/SocketStore.java b/src/java/voldemort/store/socket/SocketStore.java index 06433b7c24..a65673aa05 100644 --- a/src/java/voldemort/store/socket/SocketStore.java +++ b/src/java/voldemort/store/socket/SocketStore.java @@ -26,8 +26,8 @@ import voldemort.client.protocol.RequestFormat; import voldemort.client.protocol.RequestFormatFactory; import voldemort.server.RequestRoutingType; +import voldemort.store.AbstractStore; import voldemort.store.NoSuchCapabilityException; -import voldemort.store.Store; import voldemort.store.StoreCapabilityType; import voldemort.store.StoreUtils; import voldemort.store.UnreachableStoreException; @@ -59,11 +59,11 @@ * {@link ClientRequestExecutorPool pool} and adds an appropriate * {@link ClientRequest request} to be processed by the NIO thread. */ -public class SocketStore implements Store, NonblockingStore { +public class SocketStore extends AbstractStore implements + NonblockingStore { private final RequestFormatFactory requestFormatFactory = new RequestFormatFactory(); - private final String storeName; private final long timeoutMs; private final ClientRequestExecutorPool pool; private final SocketDestination destination; @@ -76,7 +76,7 @@ public SocketStore(String storeName, SocketDestination dest, ClientRequestExecutorPool pool, RequestRoutingType requestRoutingType) { - this.storeName = Utils.notNull(storeName); + super(storeName); this.timeoutMs = timeoutMs; this.pool = Utils.notNull(pool); this.destination = dest; @@ -84,12 +84,13 @@ public SocketStore(String storeName, this.requestRoutingType = requestRoutingType; } + @Override public void submitDeleteRequest(ByteArray key, Version version, NonblockingStoreCallback callback, long timeoutMs) { StoreUtils.assertValidKey(key); - DeleteClientRequest clientRequest = new DeleteClientRequest(storeName, + DeleteClientRequest clientRequest = new DeleteClientRequest(getName(), requestFormat, requestRoutingType, key, @@ -100,12 +101,13 @@ public void submitDeleteRequest(ByteArray key, requestAsync(clientRequest, callback, timeoutMs, "delete"); } + @Override public void submitGetRequest(ByteArray key, byte[] transforms, NonblockingStoreCallback callback, long timeoutMs) { StoreUtils.assertValidKey(key); - GetClientRequest clientRequest = new GetClientRequest(storeName, + GetClientRequest clientRequest = new GetClientRequest(getName(), requestFormat, requestRoutingType, key, @@ -116,12 +118,13 @@ public void submitGetRequest(ByteArray key, requestAsync(clientRequest, callback, timeoutMs, "get"); } + @Override public void submitGetAllRequest(Iterable keys, Map transforms, NonblockingStoreCallback callback, long timeoutMs) { StoreUtils.assertValidKeys(keys); - GetAllClientRequest clientRequest = new GetAllClientRequest(storeName, + GetAllClientRequest clientRequest = new GetAllClientRequest(getName(), requestFormat, requestRoutingType, keys, @@ -132,11 +135,12 @@ public void submitGetAllRequest(Iterable keys, requestAsync(clientRequest, callback, timeoutMs, "get all"); } + @Override public void submitGetVersionsRequest(ByteArray key, NonblockingStoreCallback callback, long timeoutMs) { StoreUtils.assertValidKey(key); - GetVersionsClientRequest clientRequest = new GetVersionsClientRequest(storeName, + GetVersionsClientRequest clientRequest = new GetVersionsClientRequest(getName(), requestFormat, requestRoutingType, key); @@ -146,13 +150,14 @@ public void submitGetVersionsRequest(ByteArray key, requestAsync(clientRequest, callback, timeoutMs, "get versions"); } + @Override public void submitPutRequest(ByteArray key, Versioned value, byte[] transforms, NonblockingStoreCallback callback, long timeoutMs) { StoreUtils.assertValidKey(key); - PutClientRequest clientRequest = new PutClientRequest(storeName, + PutClientRequest clientRequest = new PutClientRequest(getName(), requestFormat, requestRoutingType, key, @@ -164,9 +169,10 @@ public void submitPutRequest(ByteArray key, requestAsync(clientRequest, callback, timeoutMs, "put"); } + @Override public boolean delete(ByteArray key, Version version) throws VoldemortException { StoreUtils.assertValidKey(key); - DeleteClientRequest clientRequest = new DeleteClientRequest(storeName, + DeleteClientRequest clientRequest = new DeleteClientRequest(getName(), requestFormat, requestRoutingType, key, @@ -177,9 +183,10 @@ public boolean delete(ByteArray key, Version version) throws VoldemortException return request(clientRequest, "delete"); } + @Override public List> get(ByteArray key, byte[] transforms) throws VoldemortException { StoreUtils.assertValidKey(key); - GetClientRequest clientRequest = new GetClientRequest(storeName, + GetClientRequest clientRequest = new GetClientRequest(getName(), requestFormat, requestRoutingType, key, @@ -190,11 +197,12 @@ public List> get(ByteArray key, byte[] transforms) throws Vold return request(clientRequest, "get"); } + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { StoreUtils.assertValidKeys(keys); - GetAllClientRequest clientRequest = new GetAllClientRequest(storeName, + GetAllClientRequest clientRequest = new GetAllClientRequest(getName(), requestFormat, requestRoutingType, keys, @@ -205,9 +213,10 @@ public Map>> getAll(Iterable keys, return request(clientRequest, "getAll"); } + @Override public List getVersions(ByteArray key) { StoreUtils.assertValidKey(key); - GetVersionsClientRequest clientRequest = new GetVersionsClientRequest(storeName, + GetVersionsClientRequest clientRequest = new GetVersionsClientRequest(getName(), requestFormat, requestRoutingType, key); @@ -217,10 +226,11 @@ public List getVersions(ByteArray key) { return request(clientRequest, "getVersions"); } + @Override public void put(ByteArray key, Versioned versioned, byte[] transforms) throws VoldemortException { StoreUtils.assertValidKey(key); - PutClientRequest clientRequest = new PutClientRequest(storeName, + PutClientRequest clientRequest = new PutClientRequest(getName(), requestFormat, requestRoutingType, key, @@ -232,6 +242,7 @@ public void put(ByteArray key, Versioned versioned, byte[] transforms) request(clientRequest, "put"); } + @Override public Object getCapability(StoreCapabilityType capability) { if(StoreCapabilityType.SOCKET_POOL.equals(capability)) return this.pool; @@ -239,14 +250,6 @@ public Object getCapability(StoreCapabilityType capability) { throw new NoSuchCapabilityException(capability, getName()); } - public String getName() { - return storeName; - } - - public void close() throws VoldemortException { - // don't close the socket pool, it is shared - } - /** * This method handles submitting and then waiting for the request from the * server. It uses the ClientRequest API to actually write the request and @@ -355,5 +358,4 @@ private void requestAsync(ClientRequest delegate, String operationName) { pool.submitAsync(this.destination, delegate, callback, timeoutMs, operationName); } - } diff --git a/src/java/voldemort/store/stats/StatTrackingStore.java b/src/java/voldemort/store/stats/StatTrackingStore.java index 0f595ef168..e8c7026a10 100644 --- a/src/java/voldemort/store/stats/StatTrackingStore.java +++ b/src/java/voldemort/store/stats/StatTrackingStore.java @@ -26,6 +26,7 @@ import voldemort.store.DelegatingStore; import voldemort.store.Store; import voldemort.store.StoreCapabilityType; +import voldemort.store.VoldemortRequestWrapper; import voldemort.utils.ByteArray; import voldemort.versioning.ObsoleteVersionException; import voldemort.versioning.Version; @@ -153,4 +154,99 @@ public StoreStats getStats() { public void resetStatistics() { this.stats = new StoreStats(); } + + @Override + public List> get(VoldemortRequestWrapper request) + throws VoldemortException { + List> result = null; + long start = System.nanoTime(); + try { + result = super.get(request); + return result; + } catch(VoldemortException e) { + stats.recordTime(Tracked.EXCEPTION, System.nanoTime() - start); + throw e; + } finally { + long duration = System.nanoTime() - start; + long totalBytes = 0; + boolean returningEmpty = true; + if(result != null) { + returningEmpty = result.size() == 0; + for(Versioned bytes: result) { + totalBytes += bytes.getValue().length; + } + } + stats.recordGetTime(duration, returningEmpty, totalBytes); + } + } + + @Override + // TODO: Validate all the keys in the request object + public Map>> getAll(VoldemortRequestWrapper request) + throws VoldemortException { + Map>> result = null; + long start = System.nanoTime(); + try { + result = super.getAll(request); + return result; + } catch(VoldemortException e) { + stats.recordTime(Tracked.EXCEPTION, System.nanoTime() - start); + throw e; + } finally { + long duration = System.nanoTime() - start; + long totalBytes = 0; + int requestedValues = 0; + int returnedValues = 0; + + // Determine how many values were requested + for(ByteArray k: request.getIterableKeys()) { + requestedValues++; + } + + if(result != null) { + // Determine the number of values being returned + returnedValues = result.keySet().size(); + // Determine the total size of the response + for(List> value: result.values()) { + for(Versioned bytes: value) { + totalBytes += bytes.getValue().length; + } + } + } + + stats.recordGetAllTime(duration, requestedValues, returnedValues, totalBytes); + } + } + + @Override + public void put(VoldemortRequestWrapper request) throws VoldemortException { + long start = System.nanoTime(); + try { + super.put(request); + } catch(ObsoleteVersionException e) { + stats.recordTime(Tracked.OBSOLETE, System.nanoTime() - start); + throw e; + } catch(VoldemortException e) { + stats.recordTime(Tracked.EXCEPTION, System.nanoTime() - start); + throw e; + } finally { + stats.recordPutTimeAndSize(System.nanoTime() - start, + request.getValue().getValue().length); + } + + } + + @Override + public boolean delete(VoldemortRequestWrapper request) + throws VoldemortException { + long start = System.nanoTime(); + try { + return super.delete(request); + } catch(VoldemortException e) { + stats.recordTime(Tracked.EXCEPTION, System.nanoTime() - start); + throw e; + } finally { + stats.recordTime(Tracked.DELETE, System.nanoTime() - start); + } + } } diff --git a/src/java/voldemort/store/versioned/InconsistencyResolvingStore.java b/src/java/voldemort/store/versioned/InconsistencyResolvingStore.java index 5a1805902d..3af4cfc837 100644 --- a/src/java/voldemort/store/versioned/InconsistencyResolvingStore.java +++ b/src/java/voldemort/store/versioned/InconsistencyResolvingStore.java @@ -23,6 +23,7 @@ import voldemort.store.DelegatingStore; import voldemort.store.Store; import voldemort.store.StoreCapabilityType; +import voldemort.store.VoldemortRequestWrapper; import voldemort.versioning.InconsistencyResolver; import voldemort.versioning.Versioned; @@ -67,4 +68,25 @@ public Object getCapability(StoreCapabilityType capability) { return super.getCapability(capability); } + @Override + public List> get(VoldemortRequestWrapper request) throws VoldemortException { + if(request.resolveConflicts()) { + return resolver.resolveConflicts(super.get(request)); + } + return super.get(request); + } + + @Override + public Map>> getAll(VoldemortRequestWrapper request) + throws VoldemortException { + Map>> m = super.getAll(request); + if(request.resolveConflicts()) { + for(Map.Entry>> entry: m.entrySet()) { + m.put(entry.getKey(), resolver.resolveConflicts(entry.getValue())); + } + } + + return m; + } + } diff --git a/src/java/voldemort/store/versioned/VersionIncrementingStore.java b/src/java/voldemort/store/versioned/VersionIncrementingStore.java index 89f4a5ffe3..eb15aa3b35 100644 --- a/src/java/voldemort/store/versioned/VersionIncrementingStore.java +++ b/src/java/voldemort/store/versioned/VersionIncrementingStore.java @@ -33,8 +33,7 @@ * @param The value type * @param The transforms type */ -public class VersionIncrementingStore extends DelegatingStore implements - Store { +public class VersionIncrementingStore extends DelegatingStore { private final short nodeId; private final Time time; diff --git a/src/java/voldemort/store/views/ViewStorageEngine.java b/src/java/voldemort/store/views/ViewStorageEngine.java index 692482d476..205f3246d5 100644 --- a/src/java/voldemort/store/views/ViewStorageEngine.java +++ b/src/java/voldemort/store/views/ViewStorageEngine.java @@ -8,6 +8,7 @@ import voldemort.VoldemortException; import voldemort.annotations.Experimental; import voldemort.serialization.Serializer; +import voldemort.store.AbstractStorageEngine; import voldemort.store.StorageEngine; import voldemort.store.Store; import voldemort.store.StoreCapabilityType; @@ -29,9 +30,8 @@ * */ @Experimental -public class ViewStorageEngine implements StorageEngine { +public class ViewStorageEngine extends AbstractStorageEngine { - private final String name; private final Store serializingStore; private final StorageEngine target; private final Serializer valSerializer; @@ -50,7 +50,7 @@ public ViewStorageEngine(String name, Serializer targetValSerializer, CompressionStrategy valueCompressionStrategy, View valueTrans) { - this.name = name; + super(name); this.target = Utils.notNull(target); this.serializingStore = new SerializingStore(target, targetKeySerializer, @@ -103,10 +103,12 @@ private Versioned inflateValue(Versioned versioned) throws Volde return new Versioned(inflatedData, versioned.getVersion()); } + @Override public boolean delete(ByteArray key, Version version) throws VoldemortException { return target.delete(key, version); } + @Override public List> get(ByteArray key, byte[] transforms) throws VoldemortException { List> values = target.get(key, null); @@ -126,20 +128,19 @@ public List> get(ByteArray key, byte[] transforms) throws Vold return results; } + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { return StoreUtils.getAll(this, keys, transforms); } - public String getName() { - return name; - } - + @Override public List getVersions(ByteArray key) { return target.getVersions(key); } + @Override public void put(ByteArray key, Versioned value, byte[] transforms) throws VoldemortException { if(valueCompressionStrategy != null) @@ -153,22 +154,27 @@ public void put(ByteArray key, Versioned value, byte[] transforms) target.put(key, result, null); } + @Override public ClosableIterator>> entries() { return new ViewIterator(target.entries()); } + @Override public ClosableIterator keys() { return StoreUtils.keys(entries()); } + @Override public ClosableIterator>> entries(int partition) { return new ViewIterator(target.entries(partition)); } + @Override public ClosableIterator keys(int partition) { return StoreUtils.keys(entries(partition)); } + @Override public void truncate() { ViewIterator iterator = new ViewIterator(target.entries()); while(iterator.hasNext()) { @@ -177,6 +183,7 @@ public void truncate() { } } + @Override public Object getCapability(StoreCapabilityType capability) { if(capability == StoreCapabilityType.VIEW_TARGET) return this.target; @@ -184,7 +191,7 @@ public Object getCapability(StoreCapabilityType capability) { return null; } - public void close() throws VoldemortException {} + // public void close() throws VoldemortException {} private byte[] valueFromViewSchema(ByteArray key, byte[] value, byte[] transforms) { return this.targetValSerializer.toBytes(this.view.viewToStore(this.serializingStore, @@ -211,6 +218,7 @@ public ViewIterator(ClosableIterator>> inner) this.inner = inner; } + @Override public void close() { this.inner.close(); } @@ -226,21 +234,13 @@ protected Pair> computeNext() { } } + @Override public boolean isPartitionAware() { return target.isPartitionAware(); } + @Override public boolean isPartitionScanSupported() { return target.isPartitionScanSupported(); } - - @Override - public boolean beginBatchModifications() { - return false; - } - - @Override - public boolean endBatchModifications() { - return false; - } } diff --git a/test/common/voldemort/cluster/failuredetector/MutableStoreVerifier.java b/test/common/voldemort/cluster/failuredetector/MutableStoreVerifier.java index 6ec1fd5e22..4faa8d2df8 100644 --- a/test/common/voldemort/cluster/failuredetector/MutableStoreVerifier.java +++ b/test/common/voldemort/cluster/failuredetector/MutableStoreVerifier.java @@ -10,6 +10,7 @@ import voldemort.store.Store; import voldemort.store.StoreCapabilityType; import voldemort.store.UnreachableStoreException; +import voldemort.store.VoldemortRequestWrapper; import voldemort.utils.ByteArray; import voldemort.versioning.Version; import voldemort.versioning.Versioned; @@ -98,6 +99,27 @@ public List getVersions(ByteArray key) { public void put(ByteArray key, Versioned value, byte[] transforms) throws VoldemortException {} + @Override + public List> get(VoldemortRequestWrapper request) + throws VoldemortException { + return null; + } + + @Override + public Map>> getAll(VoldemortRequestWrapper request) + throws VoldemortException { + return null; + } + + @Override + public void put(VoldemortRequestWrapper request) + throws VoldemortException {} + + @Override + public boolean delete(VoldemortRequestWrapper request) + throws VoldemortException { + return false; + } }; } diff --git a/test/common/voldemort/store/DoNothingStore.java b/test/common/voldemort/store/DoNothingStore.java index a014980761..04d26ef15a 100644 --- a/test/common/voldemort/store/DoNothingStore.java +++ b/test/common/voldemort/store/DoNothingStore.java @@ -16,61 +16,23 @@ package voldemort.store; -import java.util.List; -import java.util.Map; - import voldemort.VoldemortException; -import voldemort.utils.Utils; import voldemort.versioning.Version; -import voldemort.versioning.Versioned; /** * A store that does no Harm :) * * */ -public class DoNothingStore implements Store { - - private final String name; +public class DoNothingStore extends AbstractStore { public DoNothingStore(String name) { - this.name = Utils.notNull(name); - } - - public void close() throws VoldemortException { - // Do nothing; - } - - public List> get(K key, T transforms) throws VoldemortException { - // do nothing - return null; - } - - public String getName() { - return name; + super(name); } + @Override public boolean delete(K key, Version value) throws VoldemortException { // Do nothing return true; } - - public void put(K key, Versioned value, T transforms) throws VoldemortException { - // Do nothing - } - - public Map>> getAll(Iterable keys, Map transforms) - throws VoldemortException { - return null; - } - - public Object getCapability(StoreCapabilityType capability) { - throw new NoSuchCapabilityException(capability, getName()); - } - - public List getVersions(K key) { - // Do nothing - return null; - } - } diff --git a/test/common/voldemort/store/FailingReadsStore.java b/test/common/voldemort/store/FailingReadsStore.java index b0a7da6779..5d95f39d64 100644 --- a/test/common/voldemort/store/FailingReadsStore.java +++ b/test/common/voldemort/store/FailingReadsStore.java @@ -8,43 +8,37 @@ import voldemort.versioning.Version; import voldemort.versioning.Versioned; -public class FailingReadsStore implements Store { +public class FailingReadsStore extends AbstractStore { - private final String name; private final InMemoryStorageEngine engine; public FailingReadsStore(String name) { - this.name = name; + super(name); this.engine = new InMemoryStorageEngine(name); } - public void close() throws VoldemortException {} - + @Override public boolean delete(K key, Version version) throws VoldemortException { return engine.delete(key, version); } + @Override public List> get(K key, T transforms) throws VoldemortException { throw new VoldemortException("Operation failed"); } + @Override public java.util.List getVersions(K key) { throw new VoldemortException("Operation failed"); } + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { throw new VoldemortException("Operation failed"); } - public Object getCapability(StoreCapabilityType capability) { - throw new NoSuchCapabilityException(capability, getName()); - } - - public String getName() { - return name; - } - + @Override public void put(K key, Versioned value, T transforms) throws VoldemortException { engine.put(key, value, transforms); } diff --git a/test/common/voldemort/store/FailingStore.java b/test/common/voldemort/store/FailingStore.java index 9bdceadf94..9c0a4551ba 100644 --- a/test/common/voldemort/store/FailingStore.java +++ b/test/common/voldemort/store/FailingStore.java @@ -20,7 +20,6 @@ import java.util.Map; import voldemort.VoldemortException; -import voldemort.utils.Utils; import voldemort.versioning.Version; import voldemort.versioning.Versioned; @@ -29,9 +28,8 @@ * * */ -public class FailingStore implements Store { +public class FailingStore extends AbstractStore { - private final String name; private final VoldemortException exception; public FailingStore(String name) { @@ -39,40 +37,38 @@ public FailingStore(String name) { } public FailingStore(String name, VoldemortException e) { - this.name = Utils.notNull(name); + super(name); this.exception = e; } + @Override public void close() throws VoldemortException { throw exception; } + @Override public List> get(K key, T transforms) throws VoldemortException { throw exception; } - public String getName() { - return name; - } - + @Override public boolean delete(K key, Version value) throws VoldemortException { throw exception; } + @Override public void put(K key, Versioned value, T transforms) throws VoldemortException { throw exception; } + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { throw exception; } + @Override public java.util.List getVersions(K key) { throw exception; } - - public Object getCapability(StoreCapabilityType capability) { - throw new NoSuchCapabilityException(capability, getName()); - } } diff --git a/test/common/voldemort/store/RandomlyFailingDelegatingStore.java b/test/common/voldemort/store/RandomlyFailingDelegatingStore.java index 502c2054c2..bb77a4ca90 100644 --- a/test/common/voldemort/store/RandomlyFailingDelegatingStore.java +++ b/test/common/voldemort/store/RandomlyFailingDelegatingStore.java @@ -16,19 +16,23 @@ public RandomlyFailingDelegatingStore(StorageEngine innerStorageEngine) this.innerStorageEngine = innerStorageEngine; } + @Override public ClosableIterator>> entries() { return new ClosableIterator>>() { ClosableIterator>> iterator = innerStorageEngine.entries(); + @Override public void close() { iterator.close(); } + @Override public boolean hasNext() { return iterator.hasNext(); } + @Override public Pair> next() { if(Math.random() > FAIL_PROBABILITY) return iterator.next(); @@ -36,23 +40,28 @@ public Pair> next() { throw new VoldemortException("Failing now !!"); } + @Override public void remove() {} }; } + @Override public ClosableIterator keys() { return new ClosableIterator() { ClosableIterator iterator = innerStorageEngine.keys(); + @Override public void close() { iterator.close(); } + @Override public boolean hasNext() { return iterator.hasNext(); } + @Override public K next() { if(Math.random() > FAIL_PROBABILITY) return iterator.next(); @@ -60,23 +69,28 @@ public K next() { throw new VoldemortException("Failing now !!"); } + @Override public void remove() {} }; } + @Override public ClosableIterator>> entries(final int partition) { return new ClosableIterator>>() { ClosableIterator>> iterator = innerStorageEngine.entries(partition); + @Override public void close() { iterator.close(); } + @Override public boolean hasNext() { return iterator.hasNext(); } + @Override public Pair> next() { if(Math.random() > FAIL_PROBABILITY) return iterator.next(); @@ -84,23 +98,28 @@ public Pair> next() { throw new VoldemortException("Failing now !!"); } + @Override public void remove() {} }; } + @Override public ClosableIterator keys(final int partition) { return new ClosableIterator() { ClosableIterator iterator = innerStorageEngine.keys(partition); + @Override public void close() { iterator.close(); } + @Override public boolean hasNext() { return iterator.hasNext(); } + @Override public K next() { if(Math.random() > FAIL_PROBABILITY) return iterator.next(); @@ -108,10 +127,12 @@ public K next() { throw new VoldemortException("Failing now !!"); } + @Override public void remove() {} }; } + @Override public void truncate() { if(Math.random() > FAIL_PROBABILITY) { innerStorageEngine.truncate(); @@ -120,10 +141,12 @@ public void truncate() { throw new VoldemortException("Failing now !!"); } + @Override public boolean isPartitionAware() { return innerStorageEngine.isPartitionAware(); } + @Override public boolean isPartitionScanSupported() { return innerStorageEngine.isPartitionScanSupported(); } diff --git a/test/integration/voldemort/store/noop/NoopStorageEngine.java b/test/integration/voldemort/store/noop/NoopStorageEngine.java index 07b6be5507..1bb6fe873a 100644 --- a/test/integration/voldemort/store/noop/NoopStorageEngine.java +++ b/test/integration/voldemort/store/noop/NoopStorageEngine.java @@ -22,8 +22,8 @@ import java.util.Map; import voldemort.VoldemortException; +import voldemort.store.AbstractStorageEngine; import voldemort.store.NoSuchCapabilityException; -import voldemort.store.StorageEngine; import voldemort.store.StoreCapabilityType; import voldemort.store.StoreUtils; import voldemort.utils.ByteArray; @@ -39,9 +39,8 @@ * knowledge of the serializer being used * */ -public class NoopStorageEngine implements StorageEngine { +public class NoopStorageEngine extends AbstractStorageEngine { - protected String name; protected boolean dataReflect; protected ByteArray key; protected Versioned value; @@ -49,44 +48,38 @@ public class NoopStorageEngine implements StorageEngine>> dataMap = new MyMap(); public NoopStorageEngine(String name, boolean reflect) { - this.name = name; + super(name); this.dataReflect = reflect; } - public ClosableIterator>> entries() { - return null; - } - - public ClosableIterator keys() { - return null; - } - + @Override public ClosableIterator>> entries(int partition) { throw new UnsupportedOperationException("Partition based entries scan not supported for this storage type"); } + @Override public ClosableIterator keys(int partition) { throw new UnsupportedOperationException("Partition based key scan not supported for this storage type"); } - public void truncate() { - - } - + @Override public List> get(ByteArray key, byte[] transforms) throws VoldemortException { return dataList; } + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { return dataMap; } + @Override public List getVersions(ByteArray key) { return StoreUtils.getVersions(get(key, null)); } + @Override public void put(ByteArray key, Versioned value, byte[] transforms) throws VoldemortException { @@ -96,16 +89,12 @@ public void put(ByteArray key, Versioned value, byte[] transforms) } } + @Override public boolean delete(ByteArray key, Version version) throws VoldemortException { return true; } - public String getName() { - return name; - } - - public void close() throws VoldemortException {} - + @Override public Object getCapability(StoreCapabilityType capability) { throw new NoSuchCapabilityException(capability, getName()); } @@ -134,22 +123,4 @@ public int size() { return value == null ? 0 : 1; } } - - public boolean isPartitionAware() { - return false; - } - - public boolean isPartitionScanSupported() { - return false; - } - - @Override - public boolean beginBatchModifications() { - return false; - } - - @Override - public boolean endBatchModifications() { - return false; - } } diff --git a/test/integration/voldemort/store/pausable/PausableStorageEngine.java b/test/integration/voldemort/store/pausable/PausableStorageEngine.java index 72819c3b59..32fba9b4b4 100644 --- a/test/integration/voldemort/store/pausable/PausableStorageEngine.java +++ b/test/integration/voldemort/store/pausable/PausableStorageEngine.java @@ -7,7 +7,7 @@ import voldemort.VoldemortException; import voldemort.annotations.jmx.JmxOperation; -import voldemort.store.StorageEngine; +import voldemort.store.AbstractStorageEngine; import voldemort.store.StoreCapabilityType; import voldemort.store.memory.InMemoryStorageEngine; import voldemort.utils.ClosableIterator; @@ -25,7 +25,7 @@ * @param The type of the value * @param The type of the transforms */ -public class PausableStorageEngine implements StorageEngine { +public class PausableStorageEngine extends AbstractStorageEngine { private static final Logger logger = Logger.getLogger(PausableStorageEngine.class); @@ -34,14 +34,16 @@ public class PausableStorageEngine implements StorageEngine { private volatile boolean paused; public PausableStorageEngine(InMemoryStorageEngine inner) { - super(); + super(inner.getName()); this.inner = inner; } + @Override public void close() throws VoldemortException { inner.close(); } + @Override public boolean delete(K key, Version version) { blockIfNecessary(); return inner.delete(key); @@ -59,59 +61,65 @@ private void blockIfNecessary() { } } + @Override public List> get(K key, T transforms) { blockIfNecessary(); return inner.get(key, transforms); } + @Override public Map>> getAll(Iterable keys, Map transforms) { blockIfNecessary(); return inner.getAll(keys, transforms); } + @Override public void put(K key, Versioned value, T transforms) { blockIfNecessary(); inner.put(key, value, transforms); } + @Override public ClosableIterator>> entries() { blockIfNecessary(); return inner.entries(); } + @Override public ClosableIterator keys() { blockIfNecessary(); return inner.keys(); } + @Override public ClosableIterator>> entries(int partition) { blockIfNecessary(); return inner.entries(partition); } + @Override public ClosableIterator keys(int partition) { blockIfNecessary(); return inner.keys(partition); } + @Override public void truncate() { blockIfNecessary(); inner.deleteAll(); } + @Override public List getVersions(K key) { blockIfNecessary(); return inner.getVersions(key); } + @Override public Object getCapability(StoreCapabilityType capability) { return inner.getCapability(capability); } - public String getName() { - return inner.getName(); - } - @JmxOperation(description = "Pause all operations on the storage engine.") public void pause() { logger.info("Pausing store '" + getName() + "'."); @@ -127,21 +135,13 @@ public void unpause() { } } + @Override public boolean isPartitionAware() { return inner.isPartitionAware(); } + @Override public boolean isPartitionScanSupported() { return inner.isPartitionScanSupported(); } - - @Override - public boolean beginBatchModifications() { - return false; - } - - @Override - public boolean endBatchModifications() { - return false; - } } diff --git a/test/integration/voldemort/store/slow/SlowStorageEngine.java b/test/integration/voldemort/store/slow/SlowStorageEngine.java index 1d3f4eb623..bc10e6001f 100644 --- a/test/integration/voldemort/store/slow/SlowStorageEngine.java +++ b/test/integration/voldemort/store/slow/SlowStorageEngine.java @@ -23,6 +23,7 @@ import voldemort.VoldemortException; import voldemort.common.OpTimeMap; import voldemort.common.VoldemortOpCode; +import voldemort.store.AbstractStorageEngine; import voldemort.store.StorageEngine; import voldemort.store.StoreCapabilityType; import voldemort.utils.ClosableIterator; @@ -47,7 +48,7 @@ * does not affect concurrentDelays. * */ -public class SlowStorageEngine implements StorageEngine { +public class SlowStorageEngine extends AbstractStorageEngine { private final StorageEngine innerStorageEngine; private final OpTimeMap queueingDelays; @@ -60,6 +61,7 @@ public SlowStorageEngine(StorageEngine innerStorageEngine) { public SlowStorageEngine(StorageEngine innerStorageEngine, OpTimeMap queueingDelays, OpTimeMap concurrentDelays) { + super(innerStorageEngine.getName()); this.innerStorageEngine = innerStorageEngine; this.queueingDelays = queueingDelays; this.concurrentDelays = concurrentDelays; @@ -92,56 +94,63 @@ public boolean delete(K key) { return delete(key, null); } + @Override public boolean delete(K key, Version version) { delayByOp(VoldemortOpCode.DELETE_OP_CODE); return innerStorageEngine.delete(key, version); } + @Override public List getVersions(K key) { delayByOp(VoldemortOpCode.GET_VERSION_OP_CODE); return innerStorageEngine.getVersions(key); } + @Override public List> get(K key, T transform) throws VoldemortException { delayByOp(VoldemortOpCode.GET_OP_CODE); return innerStorageEngine.get(key, transform); } + @Override public Map>> getAll(Iterable keys, Map transforms) throws VoldemortException { delayByOp(VoldemortOpCode.GET_ALL_OP_CODE); return innerStorageEngine.getAll(keys, transforms); } + @Override public void put(K key, Versioned value, T transforms) throws VoldemortException { delayByOp(VoldemortOpCode.PUT_OP_CODE); innerStorageEngine.put(key, value, transforms); } + @Override public ClosableIterator>> entries() { return innerStorageEngine.entries(); } + @Override public ClosableIterator keys() { return innerStorageEngine.keys(); } + @Override public void truncate() { innerStorageEngine.truncate(); } + @Override public boolean isPartitionAware() { return innerStorageEngine.isPartitionAware(); } - public String getName() { - return innerStorageEngine.getName(); - } - + @Override public void close() { innerStorageEngine.close(); } + @Override public Object getCapability(StoreCapabilityType capability) { return innerStorageEngine.getCapability(capability); } @@ -160,15 +169,4 @@ public ClosableIterator keys(int partition) { public boolean isPartitionScanSupported() { return innerStorageEngine.isPartitionScanSupported(); } - - @Override - public boolean beginBatchModifications() { - return false; - } - - @Override - public boolean endBatchModifications() { - return false; - } - }