[GitHub] storm pull request #1798: Storm-2203 Add a getAll method to KeyValueState in...

2016-12-31 Thread aandis
Github user aandis closed the pull request at:

https://github.com/apache/storm/pull/1798


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1798: Storm-2203 Add a getAll method to KeyValueState in...

2016-12-26 Thread arunmahadevan
Github user arunmahadevan commented on a diff in the pull request:

https://github.com/apache/storm/pull/1798#discussion_r93896589
  
--- Diff: 
external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueStateIterator.java
 ---
@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF 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.
+ */
+package org.apache.storm.redis.state;
+
+import com.google.common.base.Optional;
+
+import java.util.AbstractMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import 
org.apache.storm.redis.common.container.JedisCommandsInstanceContainer;
+import org.apache.storm.redis.utils.RedisEncoder;
+import org.apache.storm.state.DefaultStateSerializer;
+import org.apache.storm.state.Serializer;
+
+import redis.clients.jedis.JedisCommands;
+import redis.clients.jedis.ScanParams;
+import redis.clients.jedis.ScanResult;
+
+/**
+ * An iterator over {@link RedisKeyValueState}
+ */
+public class RedisKeyValueStateIterator implements 
Iterator> {
+
+private final String namespace;
+private final Iterator> 
pendingPrepareIterator;
+private final Iterator> 
pendingCommitIterator;
+private final RedisEncoder decoder;
+private final JedisCommandsInstanceContainer jedisContainer;
+private final ScanParams scanParams;
+private String cursor;
+private List> cachedResult;
+private int readPosition;
+
+public RedisKeyValueStateIterator(String namespace, 
JedisCommandsInstanceContainer jedisContainer, Iterator> pendingPrepareIterator, Iterator> 
pendingCommitIterator, int chunkSize, Serializer keySerializer, 
Serializer valueSerializer) {
+this.namespace = namespace;
+this.pendingPrepareIterator = pendingPrepareIterator;
+this.pendingCommitIterator = pendingCommitIterator;
+this.jedisContainer = jedisContainer;
+this.decoder = new RedisEncoder(keySerializer, 
valueSerializer);
+this.scanParams = new ScanParams().count(chunkSize);
+this.cursor = ScanParams.SCAN_POINTER_START;
+}
+
+@Override
+public boolean hasNext() {
+if (pendingPrepareIterator != null && 
pendingPrepareIterator.hasNext()) {
+return true;
+} else if (pendingCommitIterator != null && 
pendingCommitIterator.hasNext()) {
+return true;
+} else {
+return !cursor.equals("0");
+}
+}
+
+@Override
+public Map.Entry next() {
+Map.Entry redisKeyValue = null;
+if (pendingPrepareIterator != null && 
pendingPrepareIterator.hasNext()) {
--- End diff --

see previous comment, call hasNext within next to validate and throw 
exception.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1798: Storm-2203 Add a getAll method to KeyValueState in...

2016-12-26 Thread aandis
Github user aandis commented on a diff in the pull request:

https://github.com/apache/storm/pull/1798#discussion_r93878165
  
--- Diff: 
external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueStateIterator.java
 ---
@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF 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.
+ */
+package org.apache.storm.redis.state;
+
+import com.google.common.base.Optional;
+
+import java.util.AbstractMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import 
org.apache.storm.redis.common.container.JedisCommandsInstanceContainer;
+import org.apache.storm.redis.utils.RedisEncoder;
+import org.apache.storm.state.DefaultStateSerializer;
+import org.apache.storm.state.Serializer;
+
+import redis.clients.jedis.JedisCommands;
+import redis.clients.jedis.ScanParams;
+import redis.clients.jedis.ScanResult;
+
+/**
+ * An iterator over {@link RedisKeyValueState}
+ */
+public class RedisKeyValueStateIterator implements 
Iterator> {
+
+private final String namespace;
+private final Iterator> 
pendingPrepareIterator;
+private final Iterator> 
pendingCommitIterator;
+private final RedisEncoder decoder;
+private final JedisCommandsInstanceContainer jedisContainer;
+private final ScanParams scanParams;
+private String cursor;
+private List> cachedResult;
+private int readPosition;
+
+public RedisKeyValueStateIterator(String namespace, 
JedisCommandsInstanceContainer jedisContainer, Iterator> pendingPrepareIterator, Iterator> 
pendingCommitIterator, int chunkSize, Serializer keySerializer, 
Serializer valueSerializer) {
+this.namespace = namespace;
+this.pendingPrepareIterator = pendingPrepareIterator;
+this.pendingCommitIterator = pendingCommitIterator;
+this.jedisContainer = jedisContainer;
+this.decoder = new RedisEncoder(keySerializer, 
valueSerializer);
+this.scanParams = new ScanParams().count(chunkSize);
+this.cursor = ScanParams.SCAN_POINTER_START;
+}
+
+@Override
+public boolean hasNext() {
+if (pendingPrepareIterator != null && 
pendingPrepareIterator.hasNext()) {
+return true;
+} else if (pendingCommitIterator != null && 
pendingCommitIterator.hasNext()) {
+return true;
+} else {
+return !cursor.equals("0");
+}
+}
+
+@Override
+public Map.Entry next() {
+Map.Entry redisKeyValue = null;
+if (pendingPrepareIterator != null && 
pendingPrepareIterator.hasNext()) {
--- End diff --

> Instead of repeating the checks may be assign the current pending 
iterator to a variable in hasNext() and reuse it here.

This assumes `hasNext` is always called before calling `next` which may not 
be true isn't it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1798: Storm-2203 Add a getAll method to KeyValueState in...

2016-12-26 Thread arunmahadevan
Github user arunmahadevan commented on a diff in the pull request:

https://github.com/apache/storm/pull/1798#discussion_r93876849
  
--- Diff: 
external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueStateIterator.java
 ---
@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF 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.
+ */
+package org.apache.storm.redis.state;
+
+import com.google.common.base.Optional;
+
+import java.util.AbstractMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import 
org.apache.storm.redis.common.container.JedisCommandsInstanceContainer;
+import org.apache.storm.redis.utils.RedisEncoder;
+import org.apache.storm.state.DefaultStateSerializer;
+import org.apache.storm.state.Serializer;
+
+import redis.clients.jedis.JedisCommands;
+import redis.clients.jedis.ScanParams;
+import redis.clients.jedis.ScanResult;
+
+/**
+ * An iterator over {@link RedisKeyValueState}
+ */
+public class RedisKeyValueStateIterator implements 
Iterator> {
+
+private final String namespace;
+private final Iterator> 
pendingPrepareIterator;
+private final Iterator> 
pendingCommitIterator;
+private final RedisEncoder decoder;
+private final JedisCommandsInstanceContainer jedisContainer;
+private final ScanParams scanParams;
+private String cursor;
+private List> cachedResult;
+private int readPosition;
+
+public RedisKeyValueStateIterator(String namespace, 
JedisCommandsInstanceContainer jedisContainer, Iterator> pendingPrepareIterator, Iterator> 
pendingCommitIterator, int chunkSize, Serializer keySerializer, 
Serializer valueSerializer) {
+this.namespace = namespace;
+this.pendingPrepareIterator = pendingPrepareIterator;
+this.pendingCommitIterator = pendingCommitIterator;
+this.jedisContainer = jedisContainer;
+this.decoder = new RedisEncoder(keySerializer, 
valueSerializer);
+this.scanParams = new ScanParams().count(chunkSize);
+this.cursor = ScanParams.SCAN_POINTER_START;
+}
+
+@Override
+public boolean hasNext() {
+if (pendingPrepareIterator != null && 
pendingPrepareIterator.hasNext()) {
+return true;
+} else if (pendingCommitIterator != null && 
pendingCommitIterator.hasNext()) {
+return true;
+} else {
+return !cursor.equals("0");
+}
+}
+
+@Override
+public Map.Entry next() {
--- End diff --

next should throw a `NoSuchElementException` if there are no more elements. 
https://docs.oracle.com/javase/7/docs/api/java/util/Iterator.html#next()

in `next` add a check in the beginning.
```
if (!hasNext()) {
 throw new NoSuchElementException();
}
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1798: Storm-2203 Add a getAll method to KeyValueState in...

2016-12-26 Thread arunmahadevan
Github user arunmahadevan commented on a diff in the pull request:

https://github.com/apache/storm/pull/1798#discussion_r93877316
  
--- Diff: 
external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueStateIterator.java
 ---
@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF 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.
+ */
+package org.apache.storm.redis.state;
+
+import com.google.common.base.Optional;
+
+import java.util.AbstractMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import 
org.apache.storm.redis.common.container.JedisCommandsInstanceContainer;
+import org.apache.storm.redis.utils.RedisEncoder;
+import org.apache.storm.state.DefaultStateSerializer;
+import org.apache.storm.state.Serializer;
+
+import redis.clients.jedis.JedisCommands;
+import redis.clients.jedis.ScanParams;
+import redis.clients.jedis.ScanResult;
+
+/**
+ * An iterator over {@link RedisKeyValueState}
+ */
+public class RedisKeyValueStateIterator implements 
Iterator> {
+
+private final String namespace;
+private final Iterator> 
pendingPrepareIterator;
+private final Iterator> 
pendingCommitIterator;
+private final RedisEncoder decoder;
+private final JedisCommandsInstanceContainer jedisContainer;
+private final ScanParams scanParams;
+private String cursor;
+private List> cachedResult;
+private int readPosition;
+
+public RedisKeyValueStateIterator(String namespace, 
JedisCommandsInstanceContainer jedisContainer, Iterator> pendingPrepareIterator, Iterator> 
pendingCommitIterator, int chunkSize, Serializer keySerializer, 
Serializer valueSerializer) {
+this.namespace = namespace;
+this.pendingPrepareIterator = pendingPrepareIterator;
+this.pendingCommitIterator = pendingCommitIterator;
+this.jedisContainer = jedisContainer;
+this.decoder = new RedisEncoder(keySerializer, 
valueSerializer);
+this.scanParams = new ScanParams().count(chunkSize);
+this.cursor = ScanParams.SCAN_POINTER_START;
+}
+
+@Override
+public boolean hasNext() {
+if (pendingPrepareIterator != null && 
pendingPrepareIterator.hasNext()) {
+return true;
+} else if (pendingCommitIterator != null && 
pendingCommitIterator.hasNext()) {
+return true;
+} else {
+return !cursor.equals("0");
+}
+}
+
+@Override
+public Map.Entry next() {
+Map.Entry redisKeyValue = null;
+if (pendingPrepareIterator != null && 
pendingPrepareIterator.hasNext()) {
--- End diff --

nit: Instead of repeating the checks may be assign the current pending 
iterator to a variable in hasNext() and reuse it here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1798: Storm-2203 Add a getAll method to KeyValueState in...

2016-12-15 Thread arunmahadevan
Github user arunmahadevan commented on a diff in the pull request:

https://github.com/apache/storm/pull/1798#discussion_r92644823
  
--- Diff: 
external/storm-redis/src/main/java/org/apache/storm/redis/state/RedisKeyValueState.java
 ---
@@ -161,13 +153,19 @@ public V get(K key, V defaultValue) {
 @Override
 public V delete(K key) {
 LOG.debug("delete key '{}'", key);
-String redisKey = encode(keySerializer.serialize(key));
+String redisKey = encoder.encodeKey(key);
 V curr = get(key);
-pendingPrepare.put(redisKey, TOMBSTONE);
+pendingPrepare.put(redisKey, RedisEncoder.TOMBSTONE);
 return curr;
 }
 
 @Override
+public Iterator> iterator() {
+return new RedisKeyValueStateIterator(namespace, 
jedisContainer, 
--- End diff --

You also need to consider the entries in `pendingPrepare` and 
`pendingCommit`. See the `get` api for example. `RedisKeyValueStateIterator` 
should iterate over pendingPrepare and pendingCommit before iterating over the 
entries in Redis.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1798: Storm-2203 Add a getAll method to KeyValueState in...

2016-12-15 Thread arunmahadevan
Github user arunmahadevan commented on a diff in the pull request:

https://github.com/apache/storm/pull/1798#discussion_r92643170
  
--- Diff: storm-core/src/jvm/org/apache/storm/state/KeyValueState.java ---
@@ -52,4 +55,9 @@
  * @param key   the key
  */
 V delete(K key);
+
+/**
+ * @return an iterator over all key value mappings.
+ */
+Iterator> iterator();
--- End diff --

The KeyValueState can extend `Iterable>` so that the state can 
be directly used in foreach constructs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1798: Storm-2203 Add a getAll method to KeyValueState in...

2016-12-01 Thread aandis
Github user aandis commented on a diff in the pull request:

https://github.com/apache/storm/pull/1798#discussion_r90408316
  
--- Diff: storm-core/src/jvm/org/apache/storm/state/KeyValueState.java ---
@@ -45,4 +47,9 @@
  * @return the value or defaultValue if no mapping is found
  */
 V get(K key, V defaultValue);
+
+/**
+ * @return all key value mappings as an unmodifiable map.
+ */
+Map getAll();
--- End diff --

cool. I wasn't aware `hscan` gives you a `cursor`, redis is sort of new to 
me. I am gonna look into it and will update the pr. thanks guys. :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1798: Storm-2203 Add a getAll method to KeyValueState in...

2016-12-01 Thread HeartSaVioR
Github user HeartSaVioR commented on a diff in the pull request:

https://github.com/apache/storm/pull/1798#discussion_r90406354
  
--- Diff: storm-core/src/jvm/org/apache/storm/state/KeyValueState.java ---
@@ -45,4 +47,9 @@
  * @return the value or defaultValue if no mapping is found
  */
 V get(K key, V defaultValue);
+
+/**
+ * @return all key value mappings as an unmodifiable map.
+ */
+Map getAll();
--- End diff --

You can implement custom iterator which stores cursor for `hscan` as 
internal state. Call hscan to get some elements and store to `queue` as 
internal state, pop and return it when next() is called. If queue is empty you 
can hscan again to determine there're more elements in the hash or it is end of 
elements in hash.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1798: Storm-2203 Add a getAll method to KeyValueState in...

2016-12-01 Thread arunmahadevan
Github user arunmahadevan commented on a diff in the pull request:

https://github.com/apache/storm/pull/1798#discussion_r90406042
  
--- Diff: storm-core/src/jvm/org/apache/storm/state/KeyValueState.java ---
@@ -45,4 +47,9 @@
  * @return the value or defaultValue if no mapping is found
  */
 V get(K key, V defaultValue);
+
+/**
+ * @return all key value mappings as an unmodifiable map.
+ */
+Map getAll();
--- End diff --

At a high level I think you should be able to use redis hscan to implement 
the iterator and deserialize the key/values as you return them. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1798: Storm-2203 Add a getAll method to KeyValueState in...

2016-12-01 Thread aandis
Github user aandis commented on a diff in the pull request:

https://github.com/apache/storm/pull/1798#discussion_r90405070
  
--- Diff: storm-core/src/jvm/org/apache/storm/state/KeyValueState.java ---
@@ -45,4 +47,9 @@
  * @return the value or defaultValue if no mapping is found
  */
 V get(K key, V defaultValue);
+
+/**
+ * @return all key value mappings as an unmodifiable map.
+ */
+Map getAll();
--- End diff --

> the issue with map is that you need to load the entire state into the map 
before you can return 

true. It makes sense for the interface to return an iterator. But I am not 
sure of the implementation for say, `RedisKeyValueState`. Wouldn't you _need_ 
to load the entire state and preprocess them since there's 
[deserialization](https://github.com/apache/storm/pull/1798/files#diff-2e47d8842f277cb12f6bddbe2b7f94f5R161)
 involved?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1798: Storm-2203 Add a getAll method to KeyValueState in...

2016-12-01 Thread aandis
Github user aandis commented on a diff in the pull request:

https://github.com/apache/storm/pull/1798#discussion_r90398126
  
--- Diff: storm-core/src/jvm/org/apache/storm/state/KeyValueState.java ---
@@ -45,4 +47,9 @@
  * @return the value or defaultValue if no mapping is found
  */
 V get(K key, V defaultValue);
+
+/**
+ * @return all key value mappings as an unmodifiable map.
+ */
+Map getAll();
--- End diff --

@arunmahadevan You mean an `iterator` over the original 
[state](https://github.com/apache/storm/pull/1798/files#diff-ad463cc68c1b4705bce8681f13dcd6d4R35)?
 `iterator` also has a `remove` method which shouldn't be a supported operation 
over `KeyValueState` since there's 
[delete](https://github.com/apache/storm/commit/285668742742da6316d38c1ef492c7c873b5a649#diff-4ac32a78649ca5bdd8e0ba38b7006a1eR2).
 That's the reason for creating and returning an unmodifiable map. :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1798: Storm-2203 Add a getAll method to KeyValueState in...

2016-11-30 Thread arunmahadevan
Github user arunmahadevan commented on a diff in the pull request:

https://github.com/apache/storm/pull/1798#discussion_r90301167
  
--- Diff: storm-core/src/jvm/org/apache/storm/state/KeyValueState.java ---
@@ -45,4 +47,9 @@
  * @return the value or defaultValue if no mapping is found
  */
 V get(K key, V defaultValue);
+
+/**
+ * @return all key value mappings as an unmodifiable map.
+ */
+Map getAll();
--- End diff --

Return an Iterator of the entries in the map than the map itself. That way 
it will be more manageable even if the state is huge.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1798: Storm 2203 Add a getAll method to KeyValueState in...

2016-11-25 Thread aandis
GitHub user aandis opened a pull request:

https://github.com/apache/storm/pull/1798

Storm 2203 Add a getAll method to KeyValueState interface

Adds a `getAll` to `InMemoryKeyValueState` and `RedisKeyValueState`. I'll 
add tests once the changes are approved.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/aandis/storm STORM-2203

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/storm/pull/1798.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #1798


commit 6a1b59ff50acb880465dbff536528f7e253780a1
Author: Abhishek 
Date:   2016-11-25T09:33:04Z

Add a getAll method to KeyValueState interface.

commit c244a2baf34f81523d91deb9db7b3d914fc78987
Author: Abhishek 
Date:   2016-11-25T09:33:36Z

Implement getAll for InMemoryKeyValueState

commit 5526bd35e933ec9e43cb5f6440989067125b7061
Author: Abhishek 
Date:   2016-11-25T09:33:48Z

Implement getAll for RedisKeyValueState.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---