You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@accumulo.apache.org by milleruntime <gi...@git.apache.org> on 2017/04/07 17:45:38 UTC

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

GitHub user milleruntime opened a pull request:

    https://github.com/apache/accumulo/pull/244

    Discuss ACCUMULO-3079: collapsing the iterator stack to improve performance

    Applied the changes from the patch submitted by Adam Fuchs on [ACCUMULO-3079](https://issues.apache.org/jira/browse/ACCUMULO-3079) to master. This PR is to bring these changes back into discussion.  Created JMH benchmark tests [here](https://github.com/milleruntime/jmh-test/blob/master/src/main/java/org/sample/MyBenchmark.java).  I tried to isolate the tests to only the system iterators acting on data in memory.  
    
    I have attached the results from the benchmark tests. 
    [jmh-accumulo-benchmark-results.txt](https://github.com/apache/accumulo/files/906333/jmh-accumulo-benchmark-results.txt)
    
    The JMH tests can be built and run from [here](https://github.com/milleruntime/jmh-test). After cloning the repo, simply run:
    `mvn clean install`
    `java -jar target/benchmarks.jar -i 100 -f 1`

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

    $ git pull https://github.com/milleruntime/accumulo ACCUMULO-3079-all

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

    https://github.com/apache/accumulo/pull/244.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 #244
    
----
commit 747bae1c737b9df2bbd3c7a834a536696d9e304a
Author: Mike Miller <mm...@apache.org>
Date:   2017-03-31T16:43:45Z

    ACCUMULO-3079: collapsed the iterator stack to improve performance

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112592181
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerFilter.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +/**
    + * A SortedKeyValueIterator that filters entries from its source iterator.
    + *
    + * Subclasses must implement an accept method: public boolean accept(Key k, Value v);
    + *
    + * Key/Value pairs for which the accept method returns true are said to match the filter. By default, this class iterates over entries that match its filter.
    + * This iterator takes an optional "negate" boolean parameter that defaults to false. If negate is set to true, this class instead omits entries that match its
    + * filter, thus iterating over entries that do not match its filter.
    + */
    +public abstract class ServerFilter extends ServerWrappingIterator {
    +
    +  public ServerFilter(SortedKeyValueIterator<Key,Value> source) {
    +    super(source);
    +  }
    +
    +  @Override
    +  public abstract SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env);
    +
    +  @Override
    +  public void next() throws IOException {
    +    source.next();
    +    findTop();
    +  }
    +
    +  @Override
    +  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
    +    source.seek(range, columnFamilies, inclusive);
    +    findTop();
    +  }
    +
    +  /**
    +   * Iterates over the source until an acceptable key/value pair is found.
    +   */
    +  private void findTop() throws IOException {
    +    while (source.hasTop()) {
    +      Key top = source.getTopKey();
    +      if (top.isDeleted() || (accept(top, source.getTopValue()))) {
    --- End diff --
    
    Gotcha. Thanks, Adam. I was thinking that the DeletingIterator would mask all deletes propagating but forgot about the partial compaction case (again)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112967364
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java ---
    @@ -683,13 +683,6 @@ public void setInterruptFlag(AtomicBoolean flag) {
         private SourceSwitchingIterator ssi;
         private MemoryDataSource mds;
     
    -    @Override
    -    protected SortedKeyValueIterator<Key,Value> getSource() {
    -      if (closed.get())
    -        throw new IllegalStateException("Memory iterator is closed");
    -      return super.getSource();
    -    }
    -
    --- End diff --
    
    I am going to experiment to see how much it affects performance and if there is a compromise where we only have the sanity check at the start of loops.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by scubafuchs <gi...@git.apache.org>.
Github user scubafuchs commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    Improvements definitely look good. What's up with the benchmarks showing the *1000 tests being about 200x slower than the *10 tests when they are only scanning over 100x the data? Any insights?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by scubafuchs <gi...@git.apache.org>.
Github user scubafuchs commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112533546
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerWrappingIterator.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +/**
    + * A convenience class for implementing iterators that select, but do not modify, entries read from a source iterator. Default implementations exist for all
    + * methods, but {@link #deepCopy} will throw an <code>UnsupportedOperationException</code>.
    + *
    + * This iterator has some checks in place to enforce the iterator contract. Specifically, it verifies that it has a source iterator and that {@link #seek} has
    + * been called before any data is read. If either of these conditions does not hold true, an <code>IllegalStateException</code> will be thrown. In particular,
    + * this means that <code>getSource().seek</code> and <code>super.seek</code> no longer perform identical actions. Implementors should take note of this and if
    + * <code>seek</code> is overridden, ensure that <code>super.seek</code> is called before data is read.
    + */
    +public abstract class ServerWrappingIterator implements SortedKeyValueIterator<Key,Value> {
    +
    +  protected final SortedKeyValueIterator<Key,Value> source;
    +
    +  public ServerWrappingIterator(SortedKeyValueIterator<Key,Value> source) {
    +    this.source = source;
    +  }
    +
    +  @Override
    +  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Key getTopKey() {
    +    return source.getTopKey();
    +  }
    +
    +  @Override
    +  public Value getTopValue() {
    +    return source.getTopValue();
    +  }
    +
    +  @Override
    +  public boolean hasTop() {
    +    return source.hasTop();
    +  }
    +
    +  @Override
    +  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
    +    throw new UnsupportedOperationException();
    --- End diff --
    
    Same as above -- a bunch of the system iterators don't use or support the init method, so this is a common implementation.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112494665
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerSkippingIterator.java ---
    @@ -0,0 +1,47 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +public abstract class ServerSkippingIterator extends ServerWrappingIterator {
    --- End diff --
    
    Javadoc, please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    Added performance benchmarks for my Ubuntu Laptop: 
    [testresults.txt](https://github.com/apache/accumulo/files/945068/testresults.txt)



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    > Also, I would be super in-favor with including this in Accumulo. I think there are some licensing things that we would have to figure out (JMH is GPL IIRC), but I think this would be worthwhile as a benchmark is no good if no one else ever runs it :)
    
    I like the idea.  Any ideas how we would record benchmarks?  Have them run with the release plugin? Then publish with release notes?  
    
    We should also discuss alternatives to JMH.  I only chose it because it was the first one you suggested.   FYI I haven't been able to get the setup() and teardown() annotations to work.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    For the skeptics out there, I understand that when you take into consideration all the other factors of a cluster, low level system iterators only make up a small percentage of system performance considerations.  That being said, Accumulo developers have the most control over lower level code such as these system iterators so I think any improvements we can make are worthwhile. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112502766
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerFilter.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +/**
    + * A SortedKeyValueIterator that filters entries from its source iterator.
    + *
    + * Subclasses must implement an accept method: public boolean accept(Key k, Value v);
    + *
    + * Key/Value pairs for which the accept method returns true are said to match the filter. By default, this class iterates over entries that match its filter.
    + * This iterator takes an optional "negate" boolean parameter that defaults to false. If negate is set to true, this class instead omits entries that match its
    + * filter, thus iterating over entries that do not match its filter.
    + */
    +public abstract class ServerFilter extends ServerWrappingIterator {
    +
    +  public ServerFilter(SortedKeyValueIterator<Key,Value> source) {
    +    super(source);
    +  }
    +
    +  @Override
    +  public abstract SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env);
    +
    +  @Override
    +  public void next() throws IOException {
    +    source.next();
    +    findTop();
    +  }
    +
    +  @Override
    +  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
    +    source.seek(range, columnFamilies, inclusive);
    +    findTop();
    +  }
    +
    +  /**
    +   * Iterates over the source until an acceptable key/value pair is found.
    +   */
    +  private void findTop() throws IOException {
    +    while (source.hasTop()) {
    +      Key top = source.getTopKey();
    +      if (top.isDeleted() || (accept(top, source.getTopValue()))) {
    --- End diff --
    
    I think this code is just letting deletes pass through for compaction and not forcing users to think about deletes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112503133
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerFilter.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +/**
    + * A SortedKeyValueIterator that filters entries from its source iterator.
    + *
    + * Subclasses must implement an accept method: public boolean accept(Key k, Value v);
    + *
    + * Key/Value pairs for which the accept method returns true are said to match the filter. By default, this class iterates over entries that match its filter.
    + * This iterator takes an optional "negate" boolean parameter that defaults to false. If negate is set to true, this class instead omits entries that match its
    + * filter, thus iterating over entries that do not match its filter.
    + */
    +public abstract class ServerFilter extends ServerWrappingIterator {
    +
    +  public ServerFilter(SortedKeyValueIterator<Key,Value> source) {
    +    super(source);
    +  }
    +
    +  @Override
    +  public abstract SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env);
    +
    +  @Override
    +  public void next() throws IOException {
    +    source.next();
    +    findTop();
    +  }
    +
    +  @Override
    +  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
    +    source.seek(range, columnFamilies, inclusive);
    +    findTop();
    +  }
    +
    +  /**
    +   * Iterates over the source until an acceptable key/value pair is found.
    +   */
    +  private void findTop() throws IOException {
    +    while (source.hasTop()) {
    +      Key top = source.getTopKey();
    +      if (top.isDeleted() || (accept(top, source.getTopValue()))) {
    --- End diff --
    
    But the `break` is going to pop out of the `while`, no?
    
    I could see:
    
    ```java
    if (top.isDeleted()) {
      continue;
    } else if (accept(top, source.getTopValue()) {
      break;
    }
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r113061466
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java ---
    @@ -683,13 +683,6 @@ public void setInterruptFlag(AtomicBoolean flag) {
         private SourceSwitchingIterator ssi;
         private MemoryDataSource mds;
     
    -    @Override
    -    protected SortedKeyValueIterator<Key,Value> getSource() {
    -      if (closed.get())
    -        throw new IllegalStateException("Memory iterator is closed");
    -      return super.getSource();
    -    }
    -
    --- End diff --
    
    It could just be a margin of error or the JMH test I wrote: https://github.com/milleruntime/jmh-test/blob/6cd94569bbecda42181b6b7d7101e0770ef00d7f/src/main/java/org/sample/InMemoryMapBenchmark.java


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by phrocker <gi...@git.apache.org>.
Github user phrocker commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112706307
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java ---
    @@ -16,63 +16,59 @@
      */
     package org.apache.accumulo.core.iterators.system;
     
    +import org.apache.accumulo.core.data.ArrayByteSequence;
    +import org.apache.accumulo.core.data.ByteSequence;
     import org.apache.accumulo.core.data.Key;
     import org.apache.accumulo.core.data.Value;
    -import org.apache.accumulo.core.iterators.Filter;
     import org.apache.accumulo.core.iterators.IteratorEnvironment;
     import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
    +import org.apache.accumulo.core.iterators.SynchronizedServerFilter;
     import org.apache.accumulo.core.security.Authorizations;
     import org.apache.accumulo.core.security.ColumnVisibility;
     import org.apache.accumulo.core.security.VisibilityEvaluator;
     import org.apache.accumulo.core.security.VisibilityParseException;
     import org.apache.accumulo.core.util.BadArgumentException;
    -import org.apache.accumulo.core.util.TextUtil;
     import org.apache.commons.collections.map.LRUMap;
    -import org.apache.hadoop.io.Text;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -public class VisibilityFilter extends Filter {
    +public class VisibilityFilter extends SynchronizedServerFilter {
    --- End diff --
    
    ^ I second this. I think having that history embedded somewhere in comments if it is not already is incredibly useful. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    Aside from discussion of benchmark tests in Accumulo, any more input about the actual code changes?  I am currently running the tests again on my Ubuntu laptop to get a set of Linux system comparisons.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112504148
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerFilter.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +/**
    + * A SortedKeyValueIterator that filters entries from its source iterator.
    + *
    + * Subclasses must implement an accept method: public boolean accept(Key k, Value v);
    + *
    + * Key/Value pairs for which the accept method returns true are said to match the filter. By default, this class iterates over entries that match its filter.
    + * This iterator takes an optional "negate" boolean parameter that defaults to false. If negate is set to true, this class instead omits entries that match its
    + * filter, thus iterating over entries that do not match its filter.
    + */
    +public abstract class ServerFilter extends ServerWrappingIterator {
    +
    +  public ServerFilter(SortedKeyValueIterator<Key,Value> source) {
    +    super(source);
    +  }
    +
    +  @Override
    +  public abstract SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env);
    +
    +  @Override
    +  public void next() throws IOException {
    +    source.next();
    +    findTop();
    +  }
    +
    +  @Override
    +  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
    +    source.seek(range, columnFamilies, inclusive);
    +    findTop();
    +  }
    +
    +  /**
    +   * Iterates over the source until an acceptable key/value pair is found.
    +   */
    +  private void findTop() throws IOException {
    +    while (source.hasTop()) {
    +      Key top = source.getTopKey();
    +      if (top.isDeleted() || (accept(top, source.getTopValue()))) {
    --- End diff --
    
    I think breaking out of the while and leaving the delete on top of the source (not calling next) is the goal.  This allows the delete to pass through for compaction.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112554325
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerWrappingIterator.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +/**
    + * A convenience class for implementing iterators that select, but do not modify, entries read from a source iterator. Default implementations exist for all
    + * methods, but {@link #deepCopy} will throw an <code>UnsupportedOperationException</code>.
    + *
    + * This iterator has some checks in place to enforce the iterator contract. Specifically, it verifies that it has a source iterator and that {@link #seek} has
    + * been called before any data is read. If either of these conditions does not hold true, an <code>IllegalStateException</code> will be thrown. In particular,
    + * this means that <code>getSource().seek</code> and <code>super.seek</code> no longer perform identical actions. Implementors should take note of this and if
    + * <code>seek</code> is overridden, ensure that <code>super.seek</code> is called before data is read.
    + */
    +public abstract class ServerWrappingIterator implements SortedKeyValueIterator<Key,Value> {
    --- End diff --
    
    Another diff w/ Wrapping iterator is there are sanity checks every time `getSource()` is called.  ServerWrappingIterator does have these checks.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    > VM version: JDK 1.8.0_121, VM 25.121-b13
    > VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_121.jdk/Contents/Home/jre/bin/java
    
    We should definitely run these on Linux.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r113031922
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java ---
    @@ -683,13 +683,6 @@ public void setInterruptFlag(AtomicBoolean flag) {
         private SourceSwitchingIterator ssi;
         private MemoryDataSource mds;
     
    -    @Override
    -    protected SortedKeyValueIterator<Key,Value> getSource() {
    -      if (closed.get())
    -        throw new IllegalStateException("Memory iterator is closed");
    -      return super.getSource();
    -    }
    -
    --- End diff --
    
    I actually saw performance improvements when I added this method back... which I don't understand at all.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112552488
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerSkippingIterator.java ---
    @@ -0,0 +1,47 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +public abstract class ServerSkippingIterator extends ServerWrappingIterator {
    --- End diff --
    
    Specifically, do you mean because the new Server* iterators directly access the protected source?  While the original WrappingIterator had a private source that was accessed through a getter and setter.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112699772
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java ---
    @@ -746,50 +746,45 @@ public boolean hasTop() {
         @Override
         public void next() throws IOException {
    --- End diff --
    
    I don't think this method is a problem anymore for the JIT.  There were other methods in Rfile that popped up though, I listed all the ones I found here: https://issues.apache.org/jira/browse/ACCUMULO-4621?jql=project%20%3D%20ACCUMULO


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r113275311
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java ---
    @@ -683,13 +683,6 @@ public void setInterruptFlag(AtomicBoolean flag) {
         private SourceSwitchingIterator ssi;
         private MemoryDataSource mds;
     
    -    @Override
    -    protected SortedKeyValueIterator<Key,Value> getSource() {
    -      if (closed.get())
    -        throw new IllegalStateException("Memory iterator is closed");
    -      return super.getSource();
    -    }
    -
    --- End diff --
    
    I did a bit of research into the implications of removing this sanity check.  The case where the check would have triggered is when scan iterators are used after Accumulo has finished using the iterator stack.  If that's what we want to check for, this is probably not the best place to do 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by scubafuchs <gi...@git.apache.org>.
Github user scubafuchs commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    Here's a published benchmark example that I've found valuable in the past: https://home.apache.org/~mikemccand/lucenebench/


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    My attempt at proper java doc for the new server iterators in 12a0bcf


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    > What's up with the benchmarks showing the *1000 tests being about 200x slower than the *10 tests when they are only scanning over 100x the data? Any insights?
    
    The tests names are deceiving since the numbers are formatted in HEX.  So its more like 16 keys and 4096.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by scubafuchs <gi...@git.apache.org>.
Github user scubafuchs commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    Ah, that would explain 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    > @scubafuchs Would you be able to provide feedback to Josh about your original changes?
    
    Sorry, I didn't realize these were copy-paste of Adam's old changes. Let me go re-read history.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by scubafuchs <gi...@git.apache.org>.
Github user scubafuchs commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112533329
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerWrappingIterator.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +/**
    + * A convenience class for implementing iterators that select, but do not modify, entries read from a source iterator. Default implementations exist for all
    + * methods, but {@link #deepCopy} will throw an <code>UnsupportedOperationException</code>.
    + *
    + * This iterator has some checks in place to enforce the iterator contract. Specifically, it verifies that it has a source iterator and that {@link #seek} has
    + * been called before any data is read. If either of these conditions does not hold true, an <code>IllegalStateException</code> will be thrown. In particular,
    + * this means that <code>getSource().seek</code> and <code>super.seek</code> no longer perform identical actions. Implementors should take note of this and if
    + * <code>seek</code> is overridden, ensure that <code>super.seek</code> is called before data is read.
    + */
    +public abstract class ServerWrappingIterator implements SortedKeyValueIterator<Key,Value> {
    +
    +  protected final SortedKeyValueIterator<Key,Value> source;
    +
    +  public ServerWrappingIterator(SortedKeyValueIterator<Key,Value> source) {
    +    this.source = source;
    +  }
    +
    +  @Override
    +  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
    +    throw new UnsupportedOperationException();
    --- End diff --
    
    At least at the time, I believe that a bunch of the extending classes would have implement this method the same way, so it reduces the amount of code to have a common implementation here. I think this is more of a question about whether deepCopy should really be common across all uses of the SortedKeyValueIterator interface, or whether we should have organized it differently. Interesting topic, but probably shouldn't hold up this project.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112501284
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerFilter.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +/**
    + * A SortedKeyValueIterator that filters entries from its source iterator.
    + *
    + * Subclasses must implement an accept method: public boolean accept(Key k, Value v);
    + *
    + * Key/Value pairs for which the accept method returns true are said to match the filter. By default, this class iterates over entries that match its filter.
    + * This iterator takes an optional "negate" boolean parameter that defaults to false. If negate is set to true, this class instead omits entries that match its
    + * filter, thus iterating over entries that do not match its filter.
    + */
    +public abstract class ServerFilter extends ServerWrappingIterator {
    +
    +  public ServerFilter(SortedKeyValueIterator<Key,Value> source) {
    +    super(source);
    +  }
    +
    +  @Override
    +  public abstract SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env);
    +
    +  @Override
    +  public void next() throws IOException {
    +    source.next();
    +    findTop();
    +  }
    +
    +  @Override
    +  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
    +    source.seek(range, columnFamilies, inclusive);
    +    findTop();
    +  }
    +
    +  /**
    +   * Iterates over the source until an acceptable key/value pair is found.
    +   */
    +  private void findTop() throws IOException {
    +    while (source.hasTop()) {
    +      Key top = source.getTopKey();
    +      if (top.isDeleted() || (accept(top, source.getTopValue()))) {
    --- End diff --
    
    If `top.isDeleted()` returns true, then would `||` short circuit and not execute accept?  I think so.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by scubafuchs <gi...@git.apache.org>.
Github user scubafuchs commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    Thanks for resurrecting this, folks! The benchmarked performance improvement still looks pretty great -- can't wait to see it in production!


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112580134
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java ---
    @@ -58,21 +58,21 @@ protected void setSource(SortedKeyValueIterator<Key,Value> source) {
       public Key getTopKey() {
         if (seenSeek == false)
           throw new IllegalStateException("never been seeked");
    -    return getSource().getTopKey();
    +    return source.getTopKey();
    --- End diff --
    
    Interesting, the init method prevents making the source final.  Didn't see that one coming.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by scubafuchs <gi...@git.apache.org>.
Github user scubafuchs commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112532252
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerSkippingIterator.java ---
    @@ -0,0 +1,47 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +public abstract class ServerSkippingIterator extends ServerWrappingIterator {
    --- End diff --
    
    Basically, because of inheritance.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    > I'm quite astounded that the benchmarks showed these changes are faster than previous (notably, the SynchronizedFilter).
    
    It is possible some of the changes are not improvements and the changes that are useful are outperforming the negative ones.  I don't think its sensible to go through and run performance tests on each change individually but if there are things that are obvious (like SynchronizedFilter), I can remove it to see if it makes a difference.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112495020
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerWrappingIterator.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +/**
    + * A convenience class for implementing iterators that select, but do not modify, entries read from a source iterator. Default implementations exist for all
    + * methods, but {@link #deepCopy} will throw an <code>UnsupportedOperationException</code>.
    + *
    + * This iterator has some checks in place to enforce the iterator contract. Specifically, it verifies that it has a source iterator and that {@link #seek} has
    + * been called before any data is read. If either of these conditions does not hold true, an <code>IllegalStateException</code> will be thrown. In particular,
    + * this means that <code>getSource().seek</code> and <code>super.seek</code> no longer perform identical actions. Implementors should take note of this and if
    + * <code>seek</code> is overridden, ensure that <code>super.seek</code> is called before data is read.
    + */
    +public abstract class ServerWrappingIterator implements SortedKeyValueIterator<Key,Value> {
    +
    +  protected final SortedKeyValueIterator<Key,Value> source;
    +
    +  public ServerWrappingIterator(SortedKeyValueIterator<Key,Value> source) {
    +    this.source = source;
    +  }
    +
    +  @Override
    +  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
    +    throw new UnsupportedOperationException();
    +  }
    +
    +  @Override
    +  public Key getTopKey() {
    +    return source.getTopKey();
    +  }
    +
    +  @Override
    +  public Value getTopValue() {
    +    return source.getTopValue();
    +  }
    +
    +  @Override
    +  public boolean hasTop() {
    +    return source.hasTop();
    +  }
    +
    +  @Override
    +  public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
    +    throw new UnsupportedOperationException();
    --- End diff --
    
    Ditto WRT the `deepCopy()` comment


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112494213
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java ---
    @@ -746,50 +746,45 @@ public boolean hasTop() {
         @Override
         public void next() throws IOException {
    --- End diff --
    
    Why the consolidation of `next()` and `_next()` into just `next()`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112591981
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java ---
    @@ -16,63 +16,59 @@
      */
     package org.apache.accumulo.core.iterators.system;
     
    +import org.apache.accumulo.core.data.ArrayByteSequence;
    +import org.apache.accumulo.core.data.ByteSequence;
     import org.apache.accumulo.core.data.Key;
     import org.apache.accumulo.core.data.Value;
    -import org.apache.accumulo.core.iterators.Filter;
     import org.apache.accumulo.core.iterators.IteratorEnvironment;
     import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
    +import org.apache.accumulo.core.iterators.SynchronizedServerFilter;
     import org.apache.accumulo.core.security.Authorizations;
     import org.apache.accumulo.core.security.ColumnVisibility;
     import org.apache.accumulo.core.security.VisibilityEvaluator;
     import org.apache.accumulo.core.security.VisibilityParseException;
     import org.apache.accumulo.core.util.BadArgumentException;
    -import org.apache.accumulo.core.util.TextUtil;
     import org.apache.commons.collections.map.LRUMap;
    -import org.apache.hadoop.io.Text;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -public class VisibilityFilter extends Filter {
    +public class VisibilityFilter extends SynchronizedServerFilter {
    --- End diff --
    
    > We used to have a separate SynchronizedIterator on scan scope that wrapped all of the system iterators and this moves that same functionality down into the VisibilityFilter.
    
    Ok, we def need to lift your great explanation into some source code comments :)
    
    > This is only necessary when a configurable iterator does something multi-threaded.
    
    Ah yes. IMO, we should just tell people "don't do that" and move on, but that's an issue for another day :P


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    Reverted changes to RFile (they didn't seem very useful) and reverted changes to WrappingIterator since they would break user iterators in 180b9f6
    Tests still show the same level of improvements


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by scubafuchs <gi...@git.apache.org>.
Github user scubafuchs commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112560708
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerSkippingIterator.java ---
    @@ -0,0 +1,47 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +public abstract class ServerSkippingIterator extends ServerWrappingIterator {
    --- End diff --
    
    Yes, in the case of ServerSkippingIterator it's because we wanted to do the same thing that's done in ServerWrappingIterator. ServerWrappingIterator exists because direct access to a final source wasn't feasible in WrappingIterator.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    Here are updated results with @keith-turner 's additional improvements: 
    [testresults.txt](https://github.com/apache/accumulo/files/911586/testresults.txt)



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112494632
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerFilter.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +/**
    + * A SortedKeyValueIterator that filters entries from its source iterator.
    + *
    + * Subclasses must implement an accept method: public boolean accept(Key k, Value v);
    + *
    + * Key/Value pairs for which the accept method returns true are said to match the filter. By default, this class iterates over entries that match its filter.
    + * This iterator takes an optional "negate" boolean parameter that defaults to false. If negate is set to true, this class instead omits entries that match its
    + * filter, thus iterating over entries that do not match its filter.
    + */
    +public abstract class ServerFilter extends ServerWrappingIterator {
    +
    +  public ServerFilter(SortedKeyValueIterator<Key,Value> source) {
    +    super(source);
    +  }
    +
    +  @Override
    +  public abstract SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env);
    +
    +  @Override
    +  public void next() throws IOException {
    +    source.next();
    +    findTop();
    +  }
    +
    +  @Override
    +  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
    +    source.seek(range, columnFamilies, inclusive);
    +    findTop();
    +  }
    +
    +  /**
    +   * Iterates over the source until an acceptable key/value pair is found.
    +   */
    +  private void findTop() throws IOException {
    +    while (source.hasTop()) {
    +      Key top = source.getTopKey();
    +      if (top.isDeleted() || (accept(top, source.getTopValue()))) {
    --- End diff --
    
    This condition seems strange. We want to accept deletes we see?
    
    I'd have expected `if (!top.isDeleted() && accept(top, source.getTopValue()))`, avoiding the `accept()` call when we know the key is actually a delete.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    I think we have a good bunch of improvements here, that I would like to merge today.  If there is any more feedback please let me know.  I can open separate tickets for the other ideas that we discussed here (incorporating benchmarks into main repo, deprecating WrappingIterator for a Light version).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by scubafuchs <gi...@git.apache.org>.
Github user scubafuchs commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112530307
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerFilter.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +/**
    + * A SortedKeyValueIterator that filters entries from its source iterator.
    + *
    + * Subclasses must implement an accept method: public boolean accept(Key k, Value v);
    + *
    + * Key/Value pairs for which the accept method returns true are said to match the filter. By default, this class iterates over entries that match its filter.
    + * This iterator takes an optional "negate" boolean parameter that defaults to false. If negate is set to true, this class instead omits entries that match its
    + * filter, thus iterating over entries that do not match its filter.
    + */
    +public abstract class ServerFilter extends ServerWrappingIterator {
    +
    +  public ServerFilter(SortedKeyValueIterator<Key,Value> source) {
    +    super(source);
    +  }
    +
    +  @Override
    +  public abstract SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env);
    +
    +  @Override
    +  public void next() throws IOException {
    +    source.next();
    +    findTop();
    +  }
    +
    +  @Override
    +  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
    +    source.seek(range, columnFamilies, inclusive);
    +    findTop();
    +  }
    +
    +  /**
    +   * Iterates over the source until an acceptable key/value pair is found.
    +   */
    +  private void findTop() throws IOException {
    +    while (source.hasTop()) {
    +      Key top = source.getTopKey();
    +      if (top.isDeleted() || (accept(top, source.getTopValue()))) {
    --- End diff --
    
    Deletes need to pass through this filter. If we filter out a delete then we could end up un-deleting records in a partial compaction. The break happens as an indication that a top key is ready.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by scubafuchs <gi...@git.apache.org>.
Github user scubafuchs commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112561404
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java ---
    @@ -58,21 +58,21 @@ protected void setSource(SortedKeyValueIterator<Key,Value> source) {
       public Key getTopKey() {
         if (seenSeek == false)
           throw new IllegalStateException("never been seeked");
    -    return getSource().getTopKey();
    +    return source.getTopKey();
    --- End diff --
    
    In Sqrrl we use a LightWrappingIterator that does this. I'd be in favor of deprecating the WrappingIterator, but we might want to couple that with the bigger iterator rewrite project (ACCUMULO-3751).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112761301
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java ---
    @@ -683,13 +683,6 @@ public void setInterruptFlag(AtomicBoolean flag) {
         private SourceSwitchingIterator ssi;
         private MemoryDataSource mds;
     
    -    @Override
    -    protected SortedKeyValueIterator<Key,Value> getSource() {
    -      if (closed.get())
    -        throw new IllegalStateException("Memory iterator is closed");
    -      return super.getSource();
    -    }
    -
    --- End diff --
    
    I could see removing that helping performance, because its reading a volatile.  However it is a sanity check.  Might be worthwhile testing this one on its own before removing to see if there is a benefit.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112786051
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java ---
    @@ -58,21 +58,21 @@ protected void setSource(SortedKeyValueIterator<Key,Value> source) {
       public Key getTopKey() {
         if (seenSeek == false)
           throw new IllegalStateException("never been seeked");
    -    return getSource().getTopKey();
    +    return source.getTopKey();
    --- End diff --
    
    I experimented with making the VersioningIterator and Filter not call WrappingIterator.getSource() multiple times in their loops.  I saw a noticeable difference, like 5% increase for the VersioningIterator.  I change the code to call getSource() once before the loop and store the source in a local var.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112501639
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerFilter.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +/**
    + * A SortedKeyValueIterator that filters entries from its source iterator.
    + *
    + * Subclasses must implement an accept method: public boolean accept(Key k, Value v);
    + *
    + * Key/Value pairs for which the accept method returns true are said to match the filter. By default, this class iterates over entries that match its filter.
    + * This iterator takes an optional "negate" boolean parameter that defaults to false. If negate is set to true, this class instead omits entries that match its
    + * filter, thus iterating over entries that do not match its filter.
    + */
    +public abstract class ServerFilter extends ServerWrappingIterator {
    +
    +  public ServerFilter(SortedKeyValueIterator<Key,Value> source) {
    +    super(source);
    +  }
    +
    +  @Override
    +  public abstract SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env);
    +
    +  @Override
    +  public void next() throws IOException {
    +    source.next();
    +    findTop();
    +  }
    +
    +  @Override
    +  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
    +    source.seek(range, columnFamilies, inclusive);
    +    findTop();
    +  }
    +
    +  /**
    +   * Iterates over the source until an acceptable key/value pair is found.
    +   */
    +  private void findTop() throws IOException {
    +    while (source.hasTop()) {
    +      Key top = source.getTopKey();
    +      if (top.isDeleted() || (accept(top, source.getTopValue()))) {
    --- End diff --
    
    That's how I read 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    > Any ideas how we would record benchmarks? Have them run with the release plugin? Then publish with release notes?
    
    I'm hesitant to suggest publishing them with release notes as the "norm". I'd think that we could just start a new part of the website to capture this (since they are likely only developer-related). If we feel like it is worthy of mentioning in the release notes (e.g. the numbers are not misleading or context-specific), then we can do so. Otherwise, the website can just serve as the record of "results". We could expand this out to include things like ContinuousIngest numbers too (which would be awesome -- don't have to build up a list of numbers from release notes..)
    
    > We should also discuss alternatives to JMH. I only chose it because it was the first one you suggested. FYI I haven't been able to get the setup() and teardown() annotations to work.
    
    Heh. I think I played around with JMH once. For the most part, I think JMH and Caliper are pretty equivalent in terms of functionality. @matthew-dailey, didn't I talk to you about this a long time ago and you had some experience with them both?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by scubafuchs <gi...@git.apache.org>.
Github user scubafuchs commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112532047
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerWrappingIterator.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +/**
    + * A convenience class for implementing iterators that select, but do not modify, entries read from a source iterator. Default implementations exist for all
    + * methods, but {@link #deepCopy} will throw an <code>UnsupportedOperationException</code>.
    + *
    + * This iterator has some checks in place to enforce the iterator contract. Specifically, it verifies that it has a source iterator and that {@link #seek} has
    + * been called before any data is read. If either of these conditions does not hold true, an <code>IllegalStateException</code> will be thrown. In particular,
    + * this means that <code>getSource().seek</code> and <code>super.seek</code> no longer perform identical actions. Implementors should take note of this and if
    + * <code>seek</code> is overridden, ensure that <code>super.seek</code> is called before data is read.
    + */
    +public abstract class ServerWrappingIterator implements SortedKeyValueIterator<Key,Value> {
    --- End diff --
    
    Two differences: the source member is exposed directly to extending classes, and there is a constructor to set the source rather than a setSource() method. This allows the source to be final which I think maybe helps with runtime optimizations?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by scubafuchs <gi...@git.apache.org>.
Github user scubafuchs commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112535692
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java ---
    @@ -16,63 +16,59 @@
      */
     package org.apache.accumulo.core.iterators.system;
     
    +import org.apache.accumulo.core.data.ArrayByteSequence;
    +import org.apache.accumulo.core.data.ByteSequence;
     import org.apache.accumulo.core.data.Key;
     import org.apache.accumulo.core.data.Value;
    -import org.apache.accumulo.core.iterators.Filter;
     import org.apache.accumulo.core.iterators.IteratorEnvironment;
     import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
    +import org.apache.accumulo.core.iterators.SynchronizedServerFilter;
     import org.apache.accumulo.core.security.Authorizations;
     import org.apache.accumulo.core.security.ColumnVisibility;
     import org.apache.accumulo.core.security.VisibilityEvaluator;
     import org.apache.accumulo.core.security.VisibilityParseException;
     import org.apache.accumulo.core.util.BadArgumentException;
    -import org.apache.accumulo.core.util.TextUtil;
     import org.apache.commons.collections.map.LRUMap;
    -import org.apache.hadoop.io.Text;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -public class VisibilityFilter extends Filter {
    +public class VisibilityFilter extends SynchronizedServerFilter {
    --- End diff --
    
    We used to have a separate SynchronizedIterator on scan scope that wrapped all of the system iterators and this moves that same functionality down into the VisibilityFilter. The purpose of synchronization is to prevent leakage of intermediate values through methods like getTopKey and getTopValue while another thread is calling next. This is only necessary when a configurable iterator does something multi-threaded. There are at least a couple of ways to avoid the need to make this iterator synchronized: (1) make the system iterators thread safe as exposed by the top iterator in some other way, at least from a security perspective, or (2) detect and/or disallow any use of threading in configurable iterators. Maybe you can do (1) by eliminating the possibility of leaking top keys and values that are still being evaluated by the filter? Probably needs a test to make sure we don't break that in the future.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112506233
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerFilter.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +/**
    + * A SortedKeyValueIterator that filters entries from its source iterator.
    + *
    + * Subclasses must implement an accept method: public boolean accept(Key k, Value v);
    + *
    + * Key/Value pairs for which the accept method returns true are said to match the filter. By default, this class iterates over entries that match its filter.
    + * This iterator takes an optional "negate" boolean parameter that defaults to false. If negate is set to true, this class instead omits entries that match its
    + * filter, thus iterating over entries that do not match its filter.
    + */
    +public abstract class ServerFilter extends ServerWrappingIterator {
    +
    +  public ServerFilter(SortedKeyValueIterator<Key,Value> source) {
    +    super(source);
    +  }
    +
    +  @Override
    +  public abstract SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env);
    +
    +  @Override
    +  public void next() throws IOException {
    +    source.next();
    +    findTop();
    +  }
    +
    +  @Override
    +  public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
    +    source.seek(range, columnFamilies, inclusive);
    +    findTop();
    +  }
    +
    +  /**
    +   * Iterates over the source until an acceptable key/value pair is found.
    +   */
    +  private void findTop() throws IOException {
    +    while (source.hasTop()) {
    +      Key top = source.getTopKey();
    +      if (top.isDeleted() || (accept(top, source.getTopValue()))) {
    --- End diff --
    
    Hm. What about `IteratorUtil.setupSystemScanIterators(..)` though?
    
    ```java
      public static SortedKeyValueIterator<Key,Value> setupSystemScanIterators(SortedKeyValueIterator<Key,Value> source, Set<Column> cols, Authorizations auths,
          byte[] defaultVisibility) throws IOException {
        DeletingIterator delIter = new DeletingIterator(source, false);
        ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(delIter);
        ColumnQualifierFilter colFilter = new ColumnQualifierFilter(cfsi, cols);
        return new VisibilityFilter(colFilter, auths, defaultVisibility);
      }
    ```
    
    We wouldn't even get deletes at this point, right?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by scubafuchs <gi...@git.apache.org>.
Github user scubafuchs commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112529455
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java ---
    @@ -746,50 +746,45 @@ public boolean hasTop() {
         @Override
         public void next() throws IOException {
    --- End diff --
    
    A bunch of these changes are meant to reduce the call stack size and make the jit's job easier. I don't remember if the jit has trouble with optimizing this call in particular, or whether that has changed since the original tests.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112592112
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerWrappingIterator.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +/**
    + * A convenience class for implementing iterators that select, but do not modify, entries read from a source iterator. Default implementations exist for all
    + * methods, but {@link #deepCopy} will throw an <code>UnsupportedOperationException</code>.
    + *
    + * This iterator has some checks in place to enforce the iterator contract. Specifically, it verifies that it has a source iterator and that {@link #seek} has
    + * been called before any data is read. If either of these conditions does not hold true, an <code>IllegalStateException</code> will be thrown. In particular,
    + * this means that <code>getSource().seek</code> and <code>super.seek</code> no longer perform identical actions. Implementors should take note of this and if
    + * <code>seek</code> is overridden, ensure that <code>super.seek</code> is called before data is read.
    + */
    +public abstract class ServerWrappingIterator implements SortedKeyValueIterator<Key,Value> {
    --- End diff --
    
    Ok, so essentially we're creating custom variants of things that are generally "less safe" because we're controlling what the implementations are. I can buy that -- would suggest a comment to that extent as well.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by matthew-dailey <gi...@git.apache.org>.
Github user matthew-dailey commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    @joshelser we did talk about this, but I only have JMH experience.  Looking through our convos, I found [this](http://www.google.com/url?q=http%3A%2F%2Fhg.openjdk.java.net%2Fcode-tools%2Fjmh%2Ffile%2Ffa510264b3f6%2Fjmh-samples%2Fsrc%2Fmain%2Fjava%2Forg%2Fopenjdk%2Fjmh%2Fsamples%2FJMHSample_11_Loops.java%23l47&sa=D&sntz=1&usg=AFQjCNF22nEbxn2-JkEhIy2LDeM3iimXpw) and [this](https://groups.google.com/forum/#!topic/mechanical-sympathy/m4opvy4xq3U) as references about JMH being good.  The gist is that JMH is made by OpenJDK developers, so they did a pretty good job designing JMH to get around things like JIT optimizations and properly measure performance.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112494944
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerWrappingIterator.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +/**
    + * A convenience class for implementing iterators that select, but do not modify, entries read from a source iterator. Default implementations exist for all
    + * methods, but {@link #deepCopy} will throw an <code>UnsupportedOperationException</code>.
    + *
    + * This iterator has some checks in place to enforce the iterator contract. Specifically, it verifies that it has a source iterator and that {@link #seek} has
    + * been called before any data is read. If either of these conditions does not hold true, an <code>IllegalStateException</code> will be thrown. In particular,
    + * this means that <code>getSource().seek</code> and <code>super.seek</code> no longer perform identical actions. Implementors should take note of this and if
    + * <code>seek</code> is overridden, ensure that <code>super.seek</code> is called before data is read.
    + */
    +public abstract class ServerWrappingIterator implements SortedKeyValueIterator<Key,Value> {
    +
    +  protected final SortedKeyValueIterator<Key,Value> source;
    +
    +  public ServerWrappingIterator(SortedKeyValueIterator<Key,Value> source) {
    +    this.source = source;
    +  }
    +
    +  @Override
    +  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
    +    throw new UnsupportedOperationException();
    --- End diff --
    
    Why the decision to throw this exception and not let the compiler tell the user (at compile time, not run time) that they need to implement this method?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112495854
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerSkippingIterator.java ---
    @@ -0,0 +1,47 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +public abstract class ServerSkippingIterator extends ServerWrappingIterator {
    --- End diff --
    
    Similarly, to the same question as to "why do we need ServerWrappingIterator and WrappingIterator", why do we need "ServerSkippingIterator" and not just "SkippingIterator"?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    > I understand that when you take into consideration all the other factors of a cluster, low level system iterators only make up a small percentage of system performance considerations
    
    Good on you for following through here. I appreciate the testing efforts you put forward.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/accumulo/pull/244


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112496258
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/system/VisibilityFilter.java ---
    @@ -16,63 +16,59 @@
      */
     package org.apache.accumulo.core.iterators.system;
     
    +import org.apache.accumulo.core.data.ArrayByteSequence;
    +import org.apache.accumulo.core.data.ByteSequence;
     import org.apache.accumulo.core.data.Key;
     import org.apache.accumulo.core.data.Value;
    -import org.apache.accumulo.core.iterators.Filter;
     import org.apache.accumulo.core.iterators.IteratorEnvironment;
     import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
    +import org.apache.accumulo.core.iterators.SynchronizedServerFilter;
     import org.apache.accumulo.core.security.Authorizations;
     import org.apache.accumulo.core.security.ColumnVisibility;
     import org.apache.accumulo.core.security.VisibilityEvaluator;
     import org.apache.accumulo.core.security.VisibilityParseException;
     import org.apache.accumulo.core.util.BadArgumentException;
    -import org.apache.accumulo.core.util.TextUtil;
     import org.apache.commons.collections.map.LRUMap;
    -import org.apache.hadoop.io.Text;
     import org.slf4j.Logger;
     import org.slf4j.LoggerFactory;
     
    -public class VisibilityFilter extends Filter {
    +public class VisibilityFilter extends SynchronizedServerFilter {
    --- End diff --
    
    This looks to me that, for every call to this Filter we're now grabbing a lock?
    
    That sounds *really* bad to me. What's the reasoning 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112547894
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerWrappingIterator.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +/**
    + * A convenience class for implementing iterators that select, but do not modify, entries read from a source iterator. Default implementations exist for all
    + * methods, but {@link #deepCopy} will throw an <code>UnsupportedOperationException</code>.
    + *
    + * This iterator has some checks in place to enforce the iterator contract. Specifically, it verifies that it has a source iterator and that {@link #seek} has
    + * been called before any data is read. If either of these conditions does not hold true, an <code>IllegalStateException</code> will be thrown. In particular,
    + * this means that <code>getSource().seek</code> and <code>super.seek</code> no longer perform identical actions. Implementors should take note of this and if
    + * <code>seek</code> is overridden, ensure that <code>super.seek</code> is called before data is read.
    + */
    +public abstract class ServerWrappingIterator implements SortedKeyValueIterator<Key,Value> {
    +
    +  protected final SortedKeyValueIterator<Key,Value> source;
    +
    +  public ServerWrappingIterator(SortedKeyValueIterator<Key,Value> source) {
    +    this.source = source;
    +  }
    +
    +  @Override
    +  public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
    +    throw new UnsupportedOperationException();
    --- End diff --
    
    FYI WrappingIterator currently does this for deepCopy but not init


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by scubafuchs <gi...@git.apache.org>.
Github user scubafuchs commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112574461
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java ---
    @@ -58,21 +58,21 @@ protected void setSource(SortedKeyValueIterator<Key,Value> source) {
       public Key getTopKey() {
         if (seenSeek == false)
           throw new IllegalStateException("never been seeked");
    -    return getSource().getTopKey();
    +    return source.getTopKey();
    --- End diff --
    
    No, LightWrappingIterator is a user-level iterator, so it supports a no-argument constructor and non-final source to be compatible with the initialization via init method. It basically just removes some check, doesn't call getSource(), and exposes the source field as protected instead of private.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    >  Created JMH benchmark tests here.
    
    Also, I would be super in-favor with including this in Accumulo. I think there are some licensing things that we would have to figure out (JMH is GPL IIRC), but I think this would be worthwhile as a benchmark is no good if no one else ever runs 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112495202
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ServerWrappingIterator.java ---
    @@ -0,0 +1,80 @@
    +/*
    + * 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.accumulo.core.iterators;
    +
    +import java.io.IOException;
    +import java.util.Collection;
    +import java.util.Map;
    +
    +import org.apache.accumulo.core.data.ByteSequence;
    +import org.apache.accumulo.core.data.Key;
    +import org.apache.accumulo.core.data.Range;
    +import org.apache.accumulo.core.data.Value;
    +
    +/**
    + * A convenience class for implementing iterators that select, but do not modify, entries read from a source iterator. Default implementations exist for all
    + * methods, but {@link #deepCopy} will throw an <code>UnsupportedOperationException</code>.
    + *
    + * This iterator has some checks in place to enforce the iterator contract. Specifically, it verifies that it has a source iterator and that {@link #seek} has
    + * been called before any data is read. If either of these conditions does not hold true, an <code>IllegalStateException</code> will be thrown. In particular,
    + * this means that <code>getSource().seek</code> and <code>super.seek</code> no longer perform identical actions. Implementors should take note of this and if
    + * <code>seek</code> is overridden, ensure that <code>super.seek</code> is called before data is read.
    + */
    +public abstract class ServerWrappingIterator implements SortedKeyValueIterator<Key,Value> {
    --- End diff --
    
    How/why does this need to exist when compared to the `WrappingIterator`. Going off memory, these are very similar.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112568290
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java ---
    @@ -58,21 +58,21 @@ protected void setSource(SortedKeyValueIterator<Key,Value> source) {
       public Key getTopKey() {
         if (seenSeek == false)
           throw new IllegalStateException("never been seeked");
    -    return getSource().getTopKey();
    +    return source.getTopKey();
    --- End diff --
    
    Is LightWrappingIterator the same as ServerWrappingIterator in this PR?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    @scubafuchs Would you be able to provide feedback to Josh about your original changes?  


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112592302
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java ---
    @@ -746,50 +746,45 @@ public boolean hasTop() {
         @Override
         public void next() throws IOException {
    --- End diff --
    
    Ok -- @milleruntime this is one example of something that would be good to check is still the case. I *assume* some of what Adam tested on his own is still applicable but it would be good.
    
    I think we can see this sort of stuff with special JVM opts, but can't point you to it off the cuff.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112756271
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java ---
    @@ -683,13 +683,6 @@ public void setInterruptFlag(AtomicBoolean flag) {
         private SourceSwitchingIterator ssi;
         private MemoryDataSource mds;
     
    -    @Override
    -    protected SortedKeyValueIterator<Key,Value> getSource() {
    -      if (closed.get())
    -        throw new IllegalStateException("Memory iterator is closed");
    -      return super.getSource();
    -    }
    -
    --- End diff --
    
    Without the WrappingIterator changes, I don't think we gain anything from this change... we just lose the closed check.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request #244: Discuss ACCUMULO-3079: collapsing the iterator s...

Posted by keith-turner <gi...@git.apache.org>.
Github user keith-turner commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/244#discussion_r112552372
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/WrappingIterator.java ---
    @@ -58,21 +58,21 @@ protected void setSource(SortedKeyValueIterator<Key,Value> source) {
       public Key getTopKey() {
         if (seenSeek == false)
           throw new IllegalStateException("never been seeked");
    -    return getSource().getTopKey();
    +    return source.getTopKey();
    --- End diff --
    
    This could possibly break iterators that extend WrappingIterator and override getSource().  I don't think we should make this change.
    
    Maybe it would be better to deprecate WrappingIterator and introduce another version for users that more efficient.  I don't think we have a lot of wiggle room to make this one more efficient without possibly breaking user iterators.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo issue #244: Discuss ACCUMULO-3079: collapsing the iterator stack to...

Posted by milleruntime <gi...@git.apache.org>.
Github user milleruntime commented on the issue:

    https://github.com/apache/accumulo/pull/244
  
    Updated test results on Linux.  Dropping the lowest and highest gains, we average at about an 80% gain across the 8 tests. 
    [JMH Benchmarks](https://github.com/milleruntime/jmh-test/blob/master/src/main/java/org/sample/MyBenchmark.java)
    [testresults.txt](https://github.com/apache/accumulo/files/962199/testresults.txt)



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---