Author: chetanm
Date: Fri Dec 15 09:08:26 2017
New Revision: 1818246

URL: http://svn.apache.org/viewvc?rev=1818246&view=rev
Log:
OAK-6353 - Use Document order traversal for reindexing performed on 
DocumentNodeStore setups

Implements a FlatFileNodeStore which uses a file with NodeState sorted
by path and provides traversal support for NodeState along with its children

Added:
    
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/ChildNodeStateProvider.java
   (with props)
    
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStoreIterator.java
   (with props)
    
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/LazyChildrenNodeState.java
   (with props)
    
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/ChildNodeStateProviderTest.java
   (with props)
    
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/CountingIterable.java
   (with props)
    
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStoreIteratorTest.java
   (with props)
    
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/TestUtils.java
   (with props)
Modified:
    
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileNodeStoreBuilder.java
    
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStore.java
    
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStoreTest.java
    
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/PathElementComparatorTest.java

Added: 
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/ChildNodeStateProvider.java
URL: 
http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/ChildNodeStateProvider.java?rev=1818246&view=auto
==============================================================================
--- 
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/ChildNodeStateProvider.java
 (added)
+++ 
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/ChildNodeStateProvider.java
 Fri Dec 15 09:08:26 2017
@@ -0,0 +1,115 @@
+/*
+ * 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.jackrabbit.oak.index.indexer.document.flatfile;
+
+import java.util.Iterator;
+
+import javax.annotation.Nonnull;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.AbstractIterator;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.PeekingIterator;
+import org.apache.jackrabbit.oak.index.indexer.document.NodeStateEntry;
+import org.apache.jackrabbit.oak.plugins.memory.MemoryChildNodeEntry;
+import org.apache.jackrabbit.oak.spi.state.ChildNodeEntry;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+
+import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.collect.Iterators.limit;
+import static com.google.common.collect.Iterators.size;
+import static com.google.common.collect.Iterators.transform;
+import static java.util.Collections.emptyIterator;
+import static org.apache.jackrabbit.oak.commons.PathUtils.getName;
+import static org.apache.jackrabbit.oak.commons.PathUtils.getParentPath;
+import static 
org.apache.jackrabbit.oak.plugins.memory.EmptyNodeState.MISSING_NODE;
+
+class ChildNodeStateProvider {
+    private final Iterable<NodeStateEntry> entries;
+    private final String path;
+    private final int checkChildLimit;
+
+    public ChildNodeStateProvider(Iterable<NodeStateEntry> entries, String 
path, int checkChildLimit) {
+        this.entries = entries;
+        this.path = path;
+        this.checkChildLimit = checkChildLimit;
+    }
+
+    public boolean hasChildNode(@Nonnull String name) {
+        return getChildNode(name).exists();
+    }
+
+    @Nonnull
+    public NodeState getChildNode(@Nonnull String name) throws 
IllegalArgumentException {
+        Optional<NodeStateEntry> o = Iterators.tryFind(limit(children(), 
checkChildLimit), p -> name.equals(name(p)));
+        return o.isPresent() ? o.get().getNodeState() : MISSING_NODE;
+    }
+
+    public long getChildNodeCount(long max) {
+        if (max == 1 && children().hasNext()) {
+            return 1;
+        }
+        return size(children());
+    }
+
+    public Iterable<String> getChildNodeNames() {
+        return () -> transform(children(), p -> name(p));
+    }
+
+    @Nonnull
+    public Iterable<? extends ChildNodeEntry> getChildNodeEntries() {
+        return () -> transform(children(), p -> new 
MemoryChildNodeEntry(name(p), p.getNodeState()));
+    }
+
+    Iterator<NodeStateEntry> children() {
+        PeekingIterator<NodeStateEntry> pitr = 
Iterators.peekingIterator(entries.iterator());
+        if (!pitr.hasNext()) {
+            return emptyIterator();
+        }
+
+        //Skip till current entry
+        while (pitr.hasNext() && !pitr.peek().getPath().equals(path)) {
+            pitr.next();
+        }
+
+        //Skip past the current find
+        checkState(pitr.hasNext() && path.equals(pitr.next().getPath()),
+                "Did not found path [%s] in leftover iterator. Possibly node 
state accessed " +
+                        "after main iterator has moved past it", path);
+
+        return new AbstractIterator<NodeStateEntry>() {
+            @Override
+            protected NodeStateEntry computeNext() {
+                if (pitr.hasNext() && isImmediateChild(pitr.peek().getPath())) 
{
+                    return pitr.next();
+                }
+                return endOfData();
+            }
+        };
+    }
+
+    private static String name(NodeStateEntry p) {
+        return getName(p.getPath());
+    }
+
+    private boolean isImmediateChild(String childPath){
+        return getParentPath(childPath).equals(path);
+    }
+}

Propchange: 
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/ChildNodeStateProvider.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: 
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileNodeStoreBuilder.java
URL: 
http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileNodeStoreBuilder.java?rev=1818246&r1=1818245&r2=1818246&view=diff
==============================================================================
--- 
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileNodeStoreBuilder.java
 (original)
+++ 
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileNodeStoreBuilder.java
 Fri Dec 15 09:08:26 2017
@@ -68,7 +68,7 @@ public class FlatFileNodeStoreBuilder {
 
     public FlatFileStore build() throws IOException {
         //TODO Check not null blobStore
-        return new FlatFileStore(createdSortedStoreFile(), new 
NodeStateEntryReader(blobStore));
+        return new FlatFileStore(createdSortedStoreFile(), new 
NodeStateEntryReader(blobStore), size(preferredPathElements));
     }
 
     private File createdSortedStoreFile() throws IOException {

Modified: 
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStore.java
URL: 
http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStore.java?rev=1818246&r1=1818245&r2=1818246&view=diff
==============================================================================
--- 
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStore.java
 (original)
+++ 
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStore.java
 Fri Dec 15 09:08:26 2017
@@ -37,14 +37,20 @@ public class FlatFileStore implements It
     private final Closer closer = Closer.create();
     private final File storeFile;
     private final NodeStateEntryReader entryReader;
+    private final int checkChildLimit;
 
-    public FlatFileStore(File storeFile, NodeStateEntryReader entryReader) {
+    public FlatFileStore(File storeFile, NodeStateEntryReader entryReader, int 
checkChildLimit) {
         this.storeFile = storeFile;
         this.entryReader = entryReader;
+        this.checkChildLimit = checkChildLimit;
     }
 
     @Override
     public Iterator<NodeStateEntry> iterator() {
+        return new FlatFileStoreIterator(createBaseIterator(), 
checkChildLimit);
+    }
+
+    private Iterator<NodeStateEntry> createBaseIterator() {
         LineIterator itr = new LineIterator(createReader());
         closer.register(itr::close);
         return new AbstractIterator<NodeStateEntry>() {

Added: 
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStoreIterator.java
URL: 
http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStoreIterator.java?rev=1818246&view=auto
==============================================================================
--- 
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStoreIterator.java
 (added)
+++ 
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStoreIterator.java
 Fri Dec 15 09:08:26 2017
@@ -0,0 +1,93 @@
+/*
+ * 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.jackrabbit.oak.index.indexer.document.flatfile;
+
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.ListIterator;
+
+import com.google.common.collect.AbstractIterator;
+import org.apache.jackrabbit.oak.index.indexer.document.NodeStateEntry;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+
+import static com.google.common.collect.Iterators.concat;
+import static com.google.common.collect.Iterators.singletonIterator;
+
+class FlatFileStoreIterator extends AbstractIterator<NodeStateEntry> 
implements Iterator<NodeStateEntry> {
+    private final Iterator<NodeStateEntry> baseItr;
+    private final LinkedList<NodeStateEntry> buffer = new LinkedList<>();
+    private NodeStateEntry current;
+    private final int checkChildLimit;
+
+    public FlatFileStoreIterator(Iterator<NodeStateEntry> baseItr, int 
checkChildLimit) {
+        this.baseItr = baseItr;
+        this.checkChildLimit = checkChildLimit;
+    }
+
+    //TODO Track max buffer size
+    int getBufferSize(){
+        return buffer.size();
+    }
+
+    @Override
+    protected NodeStateEntry computeNext() {
+        //TODO Add some checks on expected ordering
+        current = computeNextEntry();
+        return current == null ? endOfData() : current;
+    }
+
+    private NodeStateEntry computeNextEntry() {
+        if (!buffer.isEmpty()) {
+            return buffer.remove();
+        }
+        if (baseItr.hasNext()) {
+            return wrap(baseItr.next());
+        }
+        return null;
+    }
+
+    private NodeStateEntry wrap(NodeStateEntry baseEntry) {
+        NodeState state = new LazyChildrenNodeState(baseEntry.getNodeState(),
+                new ChildNodeStateProvider(getEntries(), baseEntry.getPath(), 
checkChildLimit));
+        return new NodeStateEntry(state, baseEntry.getPath());
+    }
+
+    private Iterable<NodeStateEntry> getEntries() {
+        return () -> concat(singletonIterator(current), queueIterator());
+    }
+
+    private Iterator<NodeStateEntry> queueIterator() {
+        ListIterator<NodeStateEntry> qitr = buffer.listIterator();
+        return new AbstractIterator<NodeStateEntry>() {
+            @Override
+            protected NodeStateEntry computeNext() {
+                //If queue is empty try to append by getting entry from base
+                if (!qitr.hasNext() && baseItr.hasNext()) {
+                    qitr.add(wrap(baseItr.next()));
+                    qitr.previous(); //Move back the itr again
+                }
+                if (qitr.hasNext()) {
+                    return qitr.next();
+                }
+                return endOfData();
+            }
+        };
+    }
+}

Propchange: 
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStoreIterator.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: 
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/LazyChildrenNodeState.java
URL: 
http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/LazyChildrenNodeState.java?rev=1818246&view=auto
==============================================================================
--- 
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/LazyChildrenNodeState.java
 (added)
+++ 
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/LazyChildrenNodeState.java
 Fri Dec 15 09:08:26 2017
@@ -0,0 +1,141 @@
+/*
+ * 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.jackrabbit.oak.index.indexer.document.flatfile;
+
+import javax.annotation.CheckForNull;
+import javax.annotation.Nonnull;
+
+import org.apache.jackrabbit.oak.api.PropertyState;
+import org.apache.jackrabbit.oak.spi.state.AbstractNodeState;
+import org.apache.jackrabbit.oak.spi.state.ChildNodeEntry;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+import org.apache.jackrabbit.oak.spi.state.NodeStateDiff;
+
+class LazyChildrenNodeState implements NodeState{
+    private final NodeState delegate;
+    private final ChildNodeStateProvider childProvider;
+
+    LazyChildrenNodeState(NodeState delegate, ChildNodeStateProvider 
childProvider) {
+        this.delegate = delegate;
+        this.childProvider = childProvider;
+    }
+
+    @Override
+    public boolean exists() {
+        return delegate.exists();
+    }
+
+    @Override
+    public boolean hasProperty(@Nonnull String name) {
+        return delegate.hasProperty(name);
+    }
+
+    @CheckForNull
+    @Override
+    public PropertyState getProperty(@Nonnull String name) {
+        return delegate.getProperty(name);
+    }
+
+    @Override
+    public boolean getBoolean(@Nonnull String name) {
+        return delegate.getBoolean(name);
+    }
+
+    @Override
+    public long getLong(String name) {
+        return delegate.getLong(name);
+    }
+
+    @CheckForNull
+    @Override
+    public String getString(String name) {
+        return delegate.getString(name);
+    }
+
+    @Nonnull
+    @Override
+    public Iterable<String> getStrings(@Nonnull String name) {
+        return delegate.getStrings(name);
+    }
+
+    @CheckForNull
+    @Override
+    public String getName(@Nonnull String name) {
+        return delegate.getName(name);
+    }
+
+    @Nonnull
+    @Override
+    public Iterable<String> getNames(@Nonnull String name) {
+        return delegate.getNames(name);
+    }
+
+    @Override
+    public long getPropertyCount() {
+        return delegate.getPropertyCount();
+    }
+
+    @Nonnull
+    @Override
+    public Iterable<? extends PropertyState> getProperties() {
+        return delegate.getProperties();
+    }
+
+    @Nonnull
+    @Override
+    public NodeBuilder builder() {
+        return delegate.builder();
+    }
+
+    @Override
+    public boolean compareAgainstBaseState(NodeState base, NodeStateDiff diff) 
{
+        return AbstractNodeState.compareAgainstBaseState(this, base, diff);
+    }
+
+    //~-------------------------------< child node access >
+
+    @Override
+    public boolean hasChildNode(@Nonnull String name) {
+        return childProvider.hasChildNode(name);
+    }
+
+    @Nonnull
+    @Override
+    public NodeState getChildNode(@Nonnull String name) throws 
IllegalArgumentException {
+        return childProvider.getChildNode(name);
+    }
+
+    @Override
+    public long getChildNodeCount(long max) {
+        return childProvider.getChildNodeCount(max);
+    }
+
+    @Override
+    public Iterable<String> getChildNodeNames() {
+        return childProvider.getChildNodeNames();
+    }
+
+    @Nonnull
+    @Override
+    public Iterable<? extends ChildNodeEntry> getChildNodeEntries() {
+        return childProvider.getChildNodeEntries();
+    }
+}

Propchange: 
jackrabbit/oak/trunk/oak-run/src/main/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/LazyChildrenNodeState.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/ChildNodeStateProviderTest.java
URL: 
http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/ChildNodeStateProviderTest.java?rev=1818246&view=auto
==============================================================================
--- 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/ChildNodeStateProviderTest.java
 (added)
+++ 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/ChildNodeStateProviderTest.java
 Fri Dec 15 09:08:26 2017
@@ -0,0 +1,150 @@
+/*
+ * 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.jackrabbit.oak.index.indexer.document.flatfile;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
+import org.apache.jackrabbit.oak.commons.PathUtils;
+import org.apache.jackrabbit.oak.index.indexer.document.NodeStateEntry;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+import org.junit.Test;
+
+import static com.google.common.collect.ImmutableList.copyOf;
+import static java.util.Arrays.asList;
+import static java.util.Collections.emptyList;
+import static java.util.Collections.singletonList;
+import static 
org.apache.jackrabbit.oak.index.indexer.document.flatfile.TestUtils.createList;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+public class ChildNodeStateProviderTest {
+
+    @Test
+    public void emptyCase() {
+        ChildNodeStateProvider p = new ChildNodeStateProvider(emptyList(), 
"/a", 5);
+        assertEquals(0, p.getChildNodeCount(1));
+        assertEquals(0, Iterables.size(p.getChildNodeNames()));
+        assertEquals(0, Iterables.size(p.getChildNodeEntries()));
+        assertFalse(p.hasChildNode("foo"));
+        assertFalse(p.getChildNode("foo").exists());
+    }
+
+    @Test
+    public void children() {
+        Set<String> preferred = ImmutableSet.of("jcr:content", "x");
+        CountingIterable<NodeStateEntry> citr = createList(preferred, 
asList("/a", "/a/jcr:content", "/a/c", "/a/d", "/e", "/e/f", "/g", "/h"));
+        ChildNodeStateProvider p = new ChildNodeStateProvider(citr, "/a", 100);
+
+        assertEquals(asList("jcr:content", "c", "d"), 
copyOf(childNames(p.children())));
+        assertEquals(5, citr.getCount());
+
+        citr.reset();
+        p = new ChildNodeStateProvider(citr, "/e", 100);
+        assertEquals(singletonList("f"), copyOf(childNames(p.children())));
+        assertEquals(7, citr.getCount());
+
+
+        p = new ChildNodeStateProvider(citr, "/g", 100);
+        assertEquals(emptyList(), copyOf(childNames(p.children())));
+    }
+
+    @Test
+    public void children2() {
+        Set<String> preferred = ImmutableSet.of("b");
+        CountingIterable<NodeStateEntry> citr = createList(preferred, 
asList("/a", "/a/b", "/a/b/c", "/a/b/c/d", "/e", "/e/f", "/g", "/h"));
+        ChildNodeStateProvider p = new ChildNodeStateProvider(citr, "/a", 100);
+
+        assertEquals(singletonList("b"), copyOf(childNames(p.children())));
+        assertEquals(3, citr.getCount());
+
+        citr.reset();
+        p = new ChildNodeStateProvider(citr, "/a/b", 100);
+        assertEquals(singletonList("c"), copyOf(childNames(p.children())));
+        assertEquals(4, citr.getCount());
+
+        p = new ChildNodeStateProvider(citr, "/a/b/c", 100);
+        assertEquals(singletonList("d"), copyOf(childNames(p.children())));
+
+        p = new ChildNodeStateProvider(citr, "/a/b/c/d", 100);
+        assertEquals(emptyList(), copyOf(childNames(p.children())));
+
+        p = new ChildNodeStateProvider(citr, "/h", 100);
+        assertEquals(emptyList(), copyOf(childNames(p.children())));
+    }
+
+    @Test
+    public void hasChildNode_InLimit() {
+        Set<String> preferred = ImmutableSet.of("jcr:content", "x");
+        CountingIterable<NodeStateEntry> citr = createList(preferred, 
asList("/a", "/a/jcr:content", "/a/c", "/a/d", "/e", "/e/f"));
+        ChildNodeStateProvider p = new ChildNodeStateProvider(citr, "/a", 
preferred.size());
+
+        assertTrue(p.hasChildNode("jcr:content"));
+        assertTrue(p.hasChildNode("c"));
+        assertFalse(p.hasChildNode("d"));
+    }
+
+    @Test
+    public void childCount() {
+        Set<String> preferred = ImmutableSet.of("jcr:content", "x");
+        CountingIterable<NodeStateEntry> citr = createList(preferred, 
asList("/a", "/a/jcr:content", "/a/c", "/a/d", "/e", "/e/f"));
+        ChildNodeStateProvider p = new ChildNodeStateProvider(citr, "/a", 
preferred.size());
+        assertEquals(1, p.getChildNodeCount(1));
+        assertEquals(3, p.getChildNodeCount(2));
+    }
+
+    @Test
+    public void childNames() {
+        Set<String> preferred = ImmutableSet.of("jcr:content");
+        CountingIterable<NodeStateEntry> citr = createList(preferred, 
asList("/a", "/a/jcr:content", "/a/c", "/a/d", "/e", "/e/f"));
+        ChildNodeStateProvider p = new ChildNodeStateProvider(citr, "/a", 100);
+
+        assertEquals(asList("jcr:content", "c", "d"), 
copyOf(childNames(p.children())));
+        assertEquals(5, citr.getCount());
+    }
+
+    @Test
+    public void childEntries() {
+        Set<String> preferred = ImmutableSet.of("jcr:content");
+        CountingIterable<NodeStateEntry> citr = createList(preferred, 
asList("/a", "/a/jcr:content", "/a/c", "/a/d", "/e", "/e/f"));
+        ChildNodeStateProvider p = new ChildNodeStateProvider(citr, "/a", 100);
+
+        Map<String, NodeState> children = new HashMap<>();
+        p.getChildNodeEntries().forEach(e -> children.put(e.getName(), 
e.getNodeState()));
+        assertThat(children.keySet(), containsInAnyOrder("jcr:content", "c", 
"d"));
+
+        assertEquals("/a/jcr:content", 
children.get("jcr:content").getString("path"));
+        assertEquals("/a/d", children.get("d").getString("path"));
+        assertEquals("/a/c", children.get("c").getString("path"));
+    }
+
+    private Iterator<String> childNames(Iterator<NodeStateEntry> children) {
+        return Iterators.transform(children, c -> 
PathUtils.getName(c.getPath()));
+    }
+
+}
\ No newline at end of file

Propchange: 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/ChildNodeStateProviderTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/CountingIterable.java
URL: 
http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/CountingIterable.java?rev=1818246&view=auto
==============================================================================
--- 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/CountingIterable.java
 (added)
+++ 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/CountingIterable.java
 Fri Dec 15 09:08:26 2017
@@ -0,0 +1,49 @@
+/*
+ * 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.jackrabbit.oak.index.indexer.document.flatfile;
+
+import java.util.Iterator;
+
+import com.google.common.collect.Iterators;
+
+class CountingIterable<T> implements Iterable<T> {
+    private final Iterable<T> delegate;
+    private int count;
+
+    CountingIterable(Iterable<T> delegate) {
+        this.delegate = delegate;
+    }
+
+    @Override
+    public Iterator<T> iterator() {
+        return Iterators.filter(delegate.iterator(), (e) -> {
+            count++;
+            return true;
+        });
+    }
+
+    public int getCount() {
+        return count;
+    }
+
+    public void reset() {
+        count = 0;
+    }
+}

Propchange: 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/CountingIterable.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStoreIteratorTest.java
URL: 
http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStoreIteratorTest.java?rev=1818246&view=auto
==============================================================================
--- 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStoreIteratorTest.java
 (added)
+++ 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStoreIteratorTest.java
 Fri Dec 15 09:08:26 2017
@@ -0,0 +1,101 @@
+/*
+ * 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.jackrabbit.oak.index.indexer.document.flatfile;
+
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.jackrabbit.oak.index.indexer.document.NodeStateEntry;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+import org.junit.Test;
+
+import static java.util.Arrays.asList;
+import static 
org.apache.jackrabbit.oak.index.indexer.document.flatfile.TestUtils.createList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
+
+public class FlatFileStoreIteratorTest {
+
+    @Test
+    public void simpleTraversal() {
+        Set<String> preferred = ImmutableSet.of("jcr:content");
+        CountingIterable<NodeStateEntry> citr = createList(preferred, 
asList("/a", "/a/jcr:content", "/a/jcr:content/metadata",
+                "/a/d", "/e"));
+
+        FlatFileStoreIterator fitr = new 
FlatFileStoreIterator(citr.iterator(), preferred.size());
+        NodeStateEntry a = fitr.next();
+        assertEquals("/a", a.getPath());
+
+        NodeState ns1 = a.getNodeState().getChildNode("jcr:content");
+        assertEquals("/a/jcr:content", ns1.getString("path"));
+        assertEquals(1, fitr.getBufferSize());
+
+        NodeState ns2 = ns1.getChildNode("metadata");
+        assertEquals("/a/jcr:content/metadata", ns2.getString("path"));
+        assertEquals(2, fitr.getBufferSize());
+
+        NodeStateEntry nse1 = fitr.next();
+        assertEquals("/a/jcr:content", nse1.getPath());
+
+        NodeStateEntry nse2 = fitr.next();
+        assertEquals("/a/jcr:content/metadata", nse2.getPath());
+
+        NodeStateEntry nse3 = fitr.next();
+        assertEquals("/a/d", nse3.getPath());
+        assertEquals(0, nse3.getNodeState().getChildNodeCount(100));
+
+        NodeStateEntry nse4 = fitr.next();
+        assertEquals("/e", nse4.getPath());
+        assertEquals(0, nse4.getNodeState().getChildNodeCount(100));
+
+        assertFalse(fitr.hasNext());
+    }
+
+    @Test
+    public void invalidOrderAccess() {
+        Set<String> preferred = ImmutableSet.of("jcr:content");
+        CountingIterable<NodeStateEntry> citr = createList(preferred, 
asList("/a", "/a/jcr:content", "/a/jcr:content/metadata",
+                "/a/d", "/e"));
+
+        FlatFileStoreIterator fitr = new 
FlatFileStoreIterator(citr.iterator(), preferred.size());
+        NodeStateEntry a = fitr.next();
+        assertEquals("/a", a.getPath());
+
+        NodeState ns1 = a.getNodeState().getChildNode("jcr:content");
+
+        NodeStateEntry nse1 = fitr.next();
+        assertEquals("/a/jcr:content", nse1.getPath());
+        assertEquals(1, nse1.getNodeState().getChildNodeCount(100));
+
+        //Now move past /a/jcr:content
+        NodeStateEntry nse2 = fitr.next();
+        assertEquals("/a/jcr:content/metadata", nse2.getPath());
+
+        try {
+            //Now access from /a/jcr:content node should fail
+            ns1.getChildNodeCount(100);
+            fail("Access should have failed");
+        } catch (IllegalStateException ignore) {
+
+        }
+    }
+
+}
\ No newline at end of file

Propchange: 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStoreIteratorTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStoreTest.java
URL: 
http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStoreTest.java?rev=1818246&r1=1818245&r2=1818246&view=diff
==============================================================================
--- 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStoreTest.java
 (original)
+++ 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/FlatFileStoreTest.java
 Fri Dec 15 09:08:26 2017
@@ -25,9 +25,7 @@ import java.util.Set;
 import java.util.stream.Collectors;
 import java.util.stream.StreamSupport;
 
-import com.google.common.collect.Iterables;
 import org.apache.jackrabbit.oak.index.indexer.document.NodeStateEntry;
-import org.apache.jackrabbit.oak.plugins.memory.EmptyNodeState;
 import org.apache.jackrabbit.oak.spi.blob.MemoryBlobStore;
 import org.junit.Rule;
 import org.junit.Test;
@@ -47,7 +45,7 @@ public class FlatFileStoreTest {
     @Test
     public void basicTest() throws Exception {
         List<String> paths = createTestPaths();
-        FlatFileNodeStoreBuilder builder = new 
FlatFileNodeStoreBuilder(createEntries(paths), folder.getRoot());
+        FlatFileNodeStoreBuilder builder = new 
FlatFileNodeStoreBuilder(TestUtils.createEntries(paths), folder.getRoot());
         FlatFileStore flatStore = builder.withBlobStore(new MemoryBlobStore())
                 .withPreferredPathElements(preferred)
                 .build();
@@ -56,7 +54,7 @@ public class FlatFileStoreTest {
                 .map(NodeStateEntry::getPath)
                 .collect(Collectors.toList());
 
-        List<String> sortedPaths = PathElementComparatorTest.sortPaths(paths, 
preferred);
+        List<String> sortedPaths = TestUtils.sortPaths(paths, preferred);
 
         assertEquals(sortedPaths, entryPaths);
     }
@@ -65,8 +63,4 @@ public class FlatFileStoreTest {
         return asList("/a", "/b", "/c", "/a/b w", "/a/jcr:content", "/a/b", 
"/", "/b/l");
     }
 
-    private Iterable<NodeStateEntry> createEntries(List<String> paths) {
-        return Iterables.transform(paths, p -> new 
NodeStateEntry(EmptyNodeState.EMPTY_NODE, p));
-    }
-
 }
\ No newline at end of file

Modified: 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/PathElementComparatorTest.java
URL: 
http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/PathElementComparatorTest.java?rev=1818246&r1=1818245&r2=1818246&view=diff
==============================================================================
--- 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/PathElementComparatorTest.java
 (original)
+++ 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/PathElementComparatorTest.java
 Fri Dec 15 09:08:26 2017
@@ -23,12 +23,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
-import java.util.Set;
-import java.util.stream.Collectors;
 
-import com.google.common.base.Joiner;
-import com.google.common.collect.ImmutableList;
-import org.apache.jackrabbit.oak.commons.PathUtils;
 import org.junit.Test;
 
 import static java.util.Arrays.asList;
@@ -39,7 +34,7 @@ public class PathElementComparatorTest {
 
     @Test
     public void sortPathsParentChild() {
-        List<String> sorted = sortPaths(asList("/a", "/a b", "/a/bw"));
+        List<String> sorted = TestUtils.sortPaths(asList("/a", "/a b", 
"/a/bw"));
         assertEquals(asList("/a", "/a/bw", "/a b"), sorted);
     }
 
@@ -57,7 +52,7 @@ public class PathElementComparatorTest {
     @Test
     public void preferredElements() {
         PathElementComparator c = new 
PathElementComparator(singleton("jcr:content"));
-        assertEquals(asList("/a", "/a/jcr:content", "/a/b"), 
sortPaths(asList("/a/jcr:content", "/a/b", "/a"), c));
+        assertEquals(asList("/a", "/a/jcr:content", "/a/b"), 
TestUtils.sortPaths(asList("/a/jcr:content", "/a/b", "/a"), c));
 
         assertSorted(asList("/a", "/a/jcr:content", "/a/b"),c);
         assertSorted(asList("/a", "/a/jcr:content", "/a/b", "/a/b/c", "/d", 
"/e/f", "/g"), c);
@@ -70,23 +65,8 @@ public class PathElementComparatorTest {
     private void assertSorted(List<String> sorted, 
Comparator<Iterable<String>> comparator) {
         List<String> copy = new ArrayList<>(sorted);
         Collections.shuffle(copy);
-        List<String> sortedNew = sortPaths(copy, comparator);
+        List<String> sortedNew = TestUtils.sortPaths(copy, comparator);
         assertEquals(sorted, sortedNew);
     }
 
-    static List<String> sortPaths(List<String> paths){
-        return sortPaths(paths, new PathElementComparator());
-    }
-
-    static List<String> sortPaths(List<String> paths, Set<String> 
preferredElements) {
-        return sortPaths(paths, new PathElementComparator(preferredElements));
-    }
-
-    static List<String> sortPaths(List<String> paths, 
Comparator<Iterable<String>> comparator) {
-        List<Iterable<String>> copy = paths.stream().map(p -> 
ImmutableList.copyOf(PathUtils.elements(p)))
-                .sorted(comparator).collect(Collectors.toList());
-        Joiner j = Joiner.on('/');
-        return copy.stream().map(e -> "/" + 
j.join(e)).collect(Collectors.toList());
-    }
-
 }
\ No newline at end of file

Added: 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/TestUtils.java
URL: 
http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/TestUtils.java?rev=1818246&view=auto
==============================================================================
--- 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/TestUtils.java
 (added)
+++ 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/TestUtils.java
 Fri Dec 15 09:08:26 2017
@@ -0,0 +1,74 @@
+/*
+ * 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.jackrabbit.oak.index.indexer.document.flatfile;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import org.apache.jackrabbit.oak.commons.PathUtils;
+import org.apache.jackrabbit.oak.index.indexer.document.NodeStateEntry;
+import org.apache.jackrabbit.oak.spi.state.NodeBuilder;
+import org.apache.jackrabbit.oak.spi.state.NodeState;
+
+import static com.google.common.collect.ImmutableList.copyOf;
+import static java.util.Collections.emptySet;
+import static java.util.stream.Collectors.toList;
+import static org.apache.jackrabbit.oak.commons.PathUtils.elements;
+import static 
org.apache.jackrabbit.oak.plugins.memory.EmptyNodeState.EMPTY_NODE;
+
+public class TestUtils {
+
+    static List<String> sortPaths(List<String> paths) {
+        return sortPaths(paths, emptySet());
+    }
+
+    static List<String> sortPaths(List<String> paths, Set<String> 
preferredElements) {
+        return sortPaths(paths, new PathElementComparator(preferredElements));
+    }
+
+    static List<String> sortPaths(List<String> paths, 
Comparator<Iterable<String>> comparator) {
+        List<Iterable<String>> copy = paths.stream()
+                .map(p -> copyOf(elements(p)))
+                .sorted(comparator)
+                .collect(toList());
+        Joiner j = Joiner.on('/');
+        return copy.stream().map(e -> "/" + j.join(e)).collect(toList());
+    }
+
+    static CountingIterable<NodeStateEntry> createList(Set<String> preferred, 
List<String> paths) {
+        return new CountingIterable<>(createEntries(sortPaths(paths, 
preferred)));
+    }
+
+    static Iterable<NodeStateEntry> createEntries(List<String> paths) {
+        return Iterables.transform(paths, p -> new 
NodeStateEntry(createNodeState(p), p));
+    }
+
+    private static NodeState createNodeState(String p) {
+        NodeBuilder builder = EMPTY_NODE.builder();
+        builder.setProperty("path", p);
+        return builder.getNodeState();
+    }
+}

Propchange: 
jackrabbit/oak/trunk/oak-run/src/test/java/org/apache/jackrabbit/oak/index/indexer/document/flatfile/TestUtils.java
------------------------------------------------------------------------------
    svn:eol-style = native


Reply via email to