Skip to content

Commit

Permalink
Adding alternative iterator interface (this commit will be squashed l…
Browse files Browse the repository at this point in the history
…ater)

 - Adding iterator interface, and leaving the lower-level interface in place for the moment
 - Adding test for iterator interface
  • Loading branch information
Marco Primi committed Jan 8, 2016
1 parent 2994e44 commit 9198fcc
Show file tree
Hide file tree
Showing 3 changed files with 118 additions and 0 deletions.
66 changes: 66 additions & 0 deletions src/java/main/org/apache/zookeeper/ChildrenBatchIterator.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,66 @@
package org.apache.zookeeper;

import org.apache.zookeeper.data.PathWithStat;

import javax.naming.OperationNotSupportedException;
import java.util.Iterator;
import java.util.List;

/**
* Iterator over children nodes of a given path.
*/
class ChildrenBatchIterator implements Iterator<PathWithStat> {
private final ZooKeeper zooKeeper;
private final String path;
private final Watcher watcher;
private final int batchSize;
private Iterator<PathWithStat> currentBatchIterator;
private List<PathWithStat> currentBatch;
private long highestCZkId = -1;

ChildrenBatchIterator(ZooKeeper zooKeeper, String path, Watcher watcher, int batchSize, int minZkid)
throws KeeperException, InterruptedException {
this.zooKeeper = zooKeeper;
this.path = path;
this.watcher = watcher;
this.batchSize = batchSize;
this.highestCZkId = minZkid;

this.currentBatch = zooKeeper.getChildren(path, watcher, batchSize, minZkid);
this.currentBatchIterator = currentBatch.iterator();
}

@Override
public boolean hasNext() {
return currentBatchIterator.hasNext();
}

@Override
public PathWithStat next() {

if(!hasNext()) {
throw new RuntimeException("No more element");
}

PathWithStat returnChildren = currentBatchIterator.next();

highestCZkId = returnChildren.getStat().getCzxid();

// If we reached the end of the current batch, fetch the next one

try {
this.currentBatch = zooKeeper.getChildren(path, watcher, batchSize, highestCZkId);
} catch (Exception e) {
throw new RuntimeException("Failed to fetch next batch", e);
}

this.currentBatchIterator = currentBatch.iterator();

return returnChildren;
}

@Override
public void remove() {
throw new RuntimeException(new OperationNotSupportedException("remove not supported"));
}
}
17 changes: 17 additions & 0 deletions src/java/main/org/apache/zookeeper/ZooKeeper.java
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
Expand Down Expand Up @@ -2302,6 +2303,22 @@ public List<PathWithStat> getChildren(final String path, Watcher watcher, final
return response.getChildren();
}

/**
* Return the an iterator over the children node of the given path.
* <p>
* @param path
* @param watcher explicit watcher, set when the end of the list is reached
* @param batchSize number of children in each batch fetched by the iterator in the background
* @param minZkid The result will be filtered out to nodes having a czkid > minZkid
* @return an iterator, ordered by czkid
* @throws KeeperException If the server signals an error with a non-zero error code.
* @throws IllegalArgumentException if an invalid path is specified
*/
public Iterator<PathWithStat> getChildrenIterator(String path, Watcher watcher, int batchSize, int minZkid)
throws KeeperException, InterruptedException {
return new ChildrenBatchIterator(this, path, watcher, batchSize, minZkid);
}

/**
* Return the list of the children of the node of the given path.
* <p>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,10 @@
import org.junit.Test;

import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.UUID;

public class GetChildrenPaginatedTest extends ClientBase {
Expand Down Expand Up @@ -97,6 +99,39 @@ public void testPagination() throws Exception {
}
}

@Test(timeout = 30000)
public void testPaginationIterator() throws Exception {

final String testId = UUID.randomUUID().toString();
final String basePath = "/testPagination-" + testId;

Map<String, Stat> createdChildrenMetadata = createChildren(basePath, new Random().nextInt(50)+1, 0);

Map<String, Stat> readChildrenMetadata = new HashMap<String, Stat>();

final int batchSize = 3;


Iterator<PathWithStat> childrenIterator = zk.getChildrenIterator(basePath, null, batchSize, -1);


while(childrenIterator.hasNext()) {
PathWithStat child = childrenIterator.next();

final String nodePath = child.getPath();
final Stat nodeStat = child.getStat();

LOG.info("Read: " + nodePath + " zkId: " + nodeStat.getCzxid());
readChildrenMetadata.put(nodePath, nodeStat);
}

Assert.assertEquals(createdChildrenMetadata.keySet(), readChildrenMetadata.keySet());

for (String child : createdChildrenMetadata.keySet()) {
Assert.assertEquals(createdChildrenMetadata.get(child), readChildrenMetadata.get(child));
}
}

class FireOnlyOnceWatcher implements Watcher {
int watchFiredCount = 0;

Expand Down

0 comments on commit 9198fcc

Please sign in to comment.