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 2016/09/30 16:59:21 UTC

[GitHub] accumulo pull request #159: ACCUMULO-1280: many changes for closing iterator...

GitHub user milleruntime opened a pull request:

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

    ACCUMULO-1280: many changes for closing iterators

    Looking for feedback on first round of changes and if there are more places close can be called.
    - made SortedKeyValueIterator extend AutoCloseable
    - in SKVI - defined default close method so it is not required by implementations
    - in SKVI - defined default closeSafely to allow calling of close in a Java 8 stream
    - implemented close method where Iterator contained a SKVI source
    - modified Scanner and Compactor to utilize AutoCloseable and try-with-resources
    
    TODO: write test to verify close on wrapped iterators will propagate through


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

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

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

    https://github.com/apache/accumulo/pull/159.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 #159
    
----
commit e67eaa7b762632f5afc7f62e4b4a35a91ea7f658
Author: milleruntime <mi...@gmail.com>
Date:   2016-09-22T14:31:24Z

    ACCUMULO-1280: many changes for closing 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 pull request #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r108463479
  
    --- Diff: server/tserver/src/test/java/org/apache/accumulo/tserver/tablet/CompactorTest.java ---
    @@ -0,0 +1,104 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.accumulo.tserver.tablet;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +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;
    +import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
    +import org.apache.accumulo.core.iterators.SortedMapIterator;
    +import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator;
    +import org.apache.accumulo.core.iterators.system.DeletingIterator;
    +import org.apache.accumulo.core.iterators.system.MultiIterator;
    +import org.junit.Test;
    +
    +import junit.framework.TestCase;
    +
    +public class CompactorTest extends TestCase {
    --- End diff --
    
    I'd remove the `extends TestCase` and just mark the test methods with `@Test`. We don't need to keep using the junit3 way of doing things.


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    Here is my full debug stacktrace where RFile is being closed:
    ```
    java.lang.Exception
            at org.apache.accumulo.core.file.rfile.RFile$LocalityGroupReader.close(RFile.java:714)
            at org.apache.accumulo.core.file.rfile.RFile$Reader.closeLocalityGroupReaders(RFile.java:1173)
            at org.apache.accumulo.core.file.rfile.RFile$Reader.close(RFile.java:1194)
            at org.apache.accumulo.tserver.tablet.TabletData.<init>(TabletData.java:161)
            at org.apache.accumulo.tserver.TabletServer$AssignmentHandler.run(TabletServer.java:2141)
            at org.apache.accumulo.tserver.TabletServer$ThriftClientHandler$2.run(TabletServer.java:1536)
    java.lang.Exception
            at org.apache.accumulo.core.file.rfile.RFile$LocalityGroupReader.close(RFile.java:714)
            at org.apache.accumulo.core.file.rfile.RFile$Reader.closeLocalityGroupReaders(RFile.java:1173)
            at org.apache.accumulo.core.file.rfile.RFile$Reader.close(RFile.java:1194)
            at org.apache.accumulo.core.iterators.system.SourceSwitchingIterator.switchSource(SourceSwitchingIterator.java:167)
            at org.apache.accumulo.core.iterators.system.SourceSwitchingIterator.readNext(SourceSwitchingIterator.java:131)
            at org.apache.accumulo.core.iterators.system.SourceSwitchingIterator.seek(SourceSwitchingIterator.java:186)
            at org.apache.accumulo.server.problems.ProblemReportingIterator.seek(ProblemReportingIterator.java:98)
            at org.apache.accumulo.core.iterators.system.MultiIterator.seek(MultiIterator.java:106)
            at org.apache.accumulo.core.iterators.WrappingIterator.seek(WrappingIterator.java:93)
            at org.apache.accumulo.core.iterators.system.StatsIterator.seek(StatsIterator.java:64)
            at org.apache.accumulo.core.iterators.WrappingIterator.seek(WrappingIterator.java:93)
            at org.apache.accumulo.core.iterators.system.DeletingIterator.seek(DeletingIterator.java:68)
            at org.apache.accumulo.core.iterators.WrappingIterator.seek(WrappingIterator.java:93)
            at org.apache.accumulo.core.iterators.SkippingIterator.seek(SkippingIterator.java:42)
            at org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator.seek(ColumnFamilySkippingIterator.java:123)
            at org.apache.accumulo.core.iterators.WrappingIterator.seek(WrappingIterator.java:93)
            at org.apache.accumulo.core.iterators.Filter.seek(Filter.java:64)
            at org.apache.accumulo.core.iterators.WrappingIterator.seek(WrappingIterator.java:93)
            at org.apache.accumulo.core.iterators.Filter.seek(Filter.java:64)
            at org.apache.accumulo.core.iterators.system.SynchronizedIterator.seek(SynchronizedIterator.java:56)
            at org.apache.accumulo.core.iterators.WrappingIterator.seek(WrappingIterator.java:93)
            at org.apache.accumulo.core.iterators.user.VersioningIterator.seek(VersioningIterator.java:81)
            at org.apache.accumulo.core.iterators.system.SourceSwitchingIterator.readNext(SourceSwitchingIterator.java:135)
            at org.apache.accumulo.core.iterators.system.SourceSwitchingIterator.seek(SourceSwitchingIterator.java:186)
            at org.apache.accumulo.tserver.tablet.Tablet.nextBatch(Tablet.java:719)
            at org.apache.accumulo.tserver.tablet.Scanner.read(Scanner.java:86)
            at org.apache.accumulo.tserver.scan.NextBatchTask.run(NextBatchTask.java:71)
            at org.apache.accumulo.tserver.TabletServerResourceManager.executeReadAhead(TabletServerResourceManager.java:729)
            at org.apache.accumulo.tserver.TabletServer$ThriftClientHandler.continueScan(TabletServer.java:542)
            at org.apache.accumulo.tserver.TabletServer$ThriftClientHandler.continueScan(TabletServer.java:531)
            at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
            at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
            at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
            at java.lang.reflect.Method.invoke(Method.java:498)
            at org.apache.accumulo.core.trace.wrappers.RpcServerInvocationHandler.invoke(RpcServerInvocationHandler.java:46)
            at org.apache.accumulo.server.rpc.RpcWrapper$1.invoke(RpcWrapper.java:74)
            at com.sun.proxy.$Proxy14.continueScan(Unknown Source)
            at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Processor$continueScan.getResult(TabletClientService.java:2332)
            at org.apache.accumulo.core.tabletserver.thrift.TabletClientService$Processor$continueScan.getResult(TabletClientService.java:2316)
            at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39)
            at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
            at org.apache.accumulo.server.rpc.TimedProcessor.process(TimedProcessor.java:63)
            at org.apache.thrift.server.AbstractNonblockingServer$FrameBuffer.invoke(AbstractNonblockingServer.java:518)
            at org.apache.accumulo.server.rpc.CustomNonBlockingServer$CustomFrameBuffer.invoke(CustomNonBlockingServer.java:106)
            at org.apache.thrift.server.Invocation.run(Invocation.java:18)
            at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
            at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
            at org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
    
    ```


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r108462721
  
    --- Diff: docs/src/main/asciidoc/chapters/iterator_design.txt ---
    @@ -182,6 +182,12 @@ early programming assignments which implement their own tree data structures. `d
     copy on its sources (the children), copies itself, attaches the copies of the children, and
     then returns itself.
     
    +==== `close`
    +
    +This method should close any resources created by the iterator.  In particular, this method should call `close`
    +on the `source` provided by the first argument of the `init` method.  By default this method will do nothing.
    --- End diff --
    
    Maybe expand this to also specifically call out SKVI's which are created via deepCopy? Something like: ".. of the `init` method[ as well as any other Iterators created via invoking `deepCopy()`]"? Phrasing could use work :)
    
    What about the guarantees on `close()` being called? I re-read Dylan's earlier comment -- something like "`close()` will be invoked in all scenarios except those resulting in JVM failure such as an `OutOfMemoryError`"?


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r87846993
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java ---
    @@ -1164,11 +1167,7 @@ private Reader(Reader r, boolean useSample) {
     
         private void closeLocalityGroupReaders() {
           for (LocalityGroupReader lgr : currentReaders) {
    -        try {
    -          lgr.close();
    -        } catch (IOException e) {
    -          log.warn("Errored out attempting to close LocalityGroupReader.", e);
    -        }
    +        lgr.close();
    --- End diff --
    
    Created https://issues.apache.org/jira/browse/ACCUMULO-4520


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    > we decided to narrow the scope of the close implementation to only be for User Iterators
    
    How are "user iterators" going to be defined?


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r108463656
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ColumnFamilyCounter.java ---
    @@ -89,4 +89,11 @@ public Value getTopValue() {
         return null;
       }
     
    +  @Override
    --- End diff --
    
    +1 can do that after?


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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/159#discussion_r84785055
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java ---
    @@ -61,39 +61,27 @@ public ScanBatch read() throws IOException, TabletClosedException {
         Batch results = null;
     
         try {
    +      scannerSemaphore.acquire();
    +    } catch (InterruptedException e) {
    +      sawException = true;
    +    }
     
    -      try {
    -        scannerSemaphore.acquire();
    -      } catch (InterruptedException e) {
    -        sawException = true;
    -      }
    -
    -      // sawException may have occurred within close, so we cannot assume that an interrupted exception was its cause
    -      if (sawException)
    -        throw new IllegalStateException("Tried to use scanner after exception occurred.");
    +    // sawException may have occurred within close, so we cannot assume that an interrupted exception was its cause
    +    if (sawException)
    +      throw new IllegalStateException("Tried to use scanner after exception occurred.");
     
    -      if (scanClosed)
    -        throw new IllegalStateException("Tried to use scanner after it was closed.");
    +    if (scanClosed)
    +      throw new IllegalStateException("Tried to use scanner after it was closed.");
     
    -      if (options.isIsolated()) {
    -        if (isolatedDataSource == null)
    -          isolatedDataSource = new ScanDataSource(tablet, options);
    -        dataSource = isolatedDataSource;
    -      } else {
    -        dataSource = new ScanDataSource(tablet, options);
    -      }
    +    if (options.isIsolated()) {
    +      if (isolatedDataSource == null)
    +        isolatedDataSource = new ScanDataSource(tablet, options);
    +      dataSource = isolatedDataSource;
    +    } else {
    +      dataSource = new ScanDataSource(tablet, options);
    +    }
     
    -      SortedKeyValueIterator<Key,Value> iter;
    -
    -      if (options.isIsolated()) {
    -        if (isolatedIter == null)
    -          isolatedIter = new SourceSwitchingIterator(dataSource, true);
    -        else
    -          isolatedDataSource.reattachFileManager();
    -        iter = isolatedIter;
    -      } else {
    -        iter = new SourceSwitchingIterator(dataSource, false);
    -      }
    +    try (SortedKeyValueIterator<Key,Value> iter = createIter(dataSource)) {
    --- End diff --
    
    Tablet servers reuse open rfiles.  I am wondering if this change or the change to ScanDataSource will cause those reused RFiles to be closed.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r88742821
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java ---
    @@ -147,4 +147,13 @@
        *              if not supported.
        */
       SortedKeyValueIterator<K,V> deepCopy(IteratorEnvironment env);
    +
    +  /**
    +   * Closes the Iterator. This should be overridden by the implementing class that has access to <tt>SortedKeyValueIterator source</tt> provided in the
    --- End diff --
    
    > OrIterator and CountingIterator throw UnsupportedOperationException on init and set the source in the constructor
    
    Ok, I see where you're coming from now. We should definitely *not* treat those as the norm. The fact that they work is an edge condition, not our recommended way of them functioning.


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    After some discussion yesterday with @ctubbsii we came up with a compromise, that I believe will benefit everyone. I can document the interface to follow the design that states "All Iterators must close their source" but maintain the a lot of the current interaction with Iterators (mainly RFile, SourceSwitchingIterator and Scanner). They ARE grandfathered in 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 issue #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    > The problem I am running into is our underlying reuse of RFiles throughout the code.
    
    One solution you suggested is to make the RFile close() methods do nothing. (Rename the current close() method or make MultiIterator's close() do nothing.)  Not very satisfying, but it could work.
    
    Another solution is approach 2 + don't close the RFiles. It seems nicer to me since it promotes inter-iterator isolation and doesn't break existing code.  I realized that the nice way of ordering the close() calls is in the reverse order of the order that the iterators were constructed/init()ed.
    
    > Retain the list of IterInfos from when Accumulo sets up the iterator stack (see IteratorUtil.loadIterators()). Before Accumulo tears down the stack, for any reason, call close() on each iterator individually, say starting from the bottom iterator and working up toward the root RFile/InMemoryMap iterators. This strategy would not break existing iterators; an iterator that does not call close() on its source is fine since Accumulo would call it. It also provides better iterator isolation.
    
    With this approach, each iterator is only responsible for closing the resources it itself opens.  No need to consider other iterators. Seems more palatable, don't you think?


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r83274406
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/OrIterator.java ---
    @@ -249,4 +268,9 @@ final public boolean hasTop() {
       public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
         throw new UnsupportedOperationException();
       }
    +
    +  @Override
    +  public void close() throws Exception {
    +    sources.forEach(s -> s.closeSafely());
    +  }
    --- End diff --
    
    I agree.  My original thinking was to define closeSafely in the interface for coveinent use with Java 1.8.  But you are right, having multiple sources is a special case so it shouldn't be in the interface.


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    Bringing the discussion on exception handling back to the top level, and adding more thoughts.
    
    ## Exceptions and Closing
    
    Suppose you're an iterator.  You call `next()` or `seek()` on your source and it throws an IOException.  I would treat this the same as if `hasTop()` returned false; no more data is available from your source.  The iterator stack will not necessarily be torn down. The bottom of the iterator stack might seek to the next range, for example (not sure if this actually happens).
    
    Accumulo should *guarantee* that `close()` will be called before an iterator is torn down, say via try-finally, except in extreme cases like OOM.  I see two approaches to doing this:
    
    1. Specify the contract that every iterator *must* `close()` its source.  This strategy works if every iterator, system and user-defined, follows the contract.  Existing iterators would need to be modified to do this, so it's a breaking change.
    
    2. Retain the list of `IterInfo`s from when Accumulo sets up the iterator stack (see `IteratorUtil.loadIterators()`). Before Accumulo tears down the stack, for any reason, call `close()` on each iterator individually, say starting from the root RFile/InMemoryMap iterators and working down (top-down vs. bottom-up order may not matter). This strategy would not break existing iterators; an iterator that does not call `close()` on its source is fine since Accumulo would call it. It also provides better iterator isolation.
    
    ## Idempotent
    
    I think `close()` *must* be idempotent; calling `close()` more than once should have the same effect as calling `close()` once. We should document the requirement. This would also suggest changing `AutoCloseable` to `Closeable`.
    
    If we don't require idempotency, then option 2 above is not viable and we should carefully engineer the implementation for *exactly-once* semantics. Seems difficult. Also, can anyone think of a use case that requires exactly-once semantics?
    
    ## Less certain ideas
    
    Because discussion is good =)
    
    ### "Reason" for Closing
    
    It would be awesome if the iterator knew *why* Accumulo is closing it. This doesn't mesh well with the `AutoCloseable` interface.  Otherwise I would suggest passing a `Reason` interface parameter to the `close()` method.  Reasons include "the client stopped requesting tuples" or "switching source" or "starting migration" or "tuple returned outside of seek fence for this tablet" or "out of fairness to other concurrent scans" or "client cancelled scan". Something like
    
    ```
    @Override
    default void close() {
      close(Reason.UNKNOWN);
    }
    
    default void close(Reason reason) {
      // default implementation, do nothing
    }
    ```
    
    It's fine if we don't do the `Reason`. It would make try-with-resources awkward. Then again, how often (if ever) would iterators be constructed in try-with-resources blocks?
    
    I'm also not sure what use cases need this. It seems helpful to provide the iterators with this information, but maybe it's not necessary.
    
    ### Put `close()` in `finalize()` for safety
    
    Yes, we know the JVM does not guarantee `finalize()` would ever be called. But it would be a nice "safety" against bugs in Accumulo that cause Accumulo to forgo calling `close()` as it normally must. In SKVI.java:
    
    ```
    @Override
    protected void finalize() throws Throwable {
      super.finalize();
      close();
    }
    ```
    
    Obviously we cannot do this if we don't require `close()` idempotency.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r108479204
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java ---
    @@ -147,4 +147,21 @@
        *              if not supported.
        */
       SortedKeyValueIterator<K,V> deepCopy(IteratorEnvironment env);
    +
    +  /**
    +   * Closes any resources that were opened by the <tt>SortedKeyValueIterator</tt>. This method does nothing by default. The implementing class must close its
    +   * <tt>SortedKeyValueIterator source</tt>. This will be provided in the <tt>init</tt> method or by the constructor.
    +   *
    +   * @exception UncheckedIOException
    --- End diff --
    
    After revisiting these changes, I am now being reminded as to why I abandoned them months ago...  @keith-turner may have an alternate solution that is far less widespread and more manageable. See https://issues.apache.org/jira/browse/ACCUMULO-1280


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    Full "mvn verify" had errors in accumulo-test :
    Tests in error: 
      BigRootTabletIT.test:58 � Accumulo org.apache.thrift.TApplicationException: In...
      LargeRowIT.run:125->test2:142->basicTest:177 � Accumulo org.apache.thrift.TApp...
    org.apache.accumulo.test.functional.MetadataMaxFilesIT.test(org.apache.accumulo.test.functional.MetadataMaxFilesIT)
      Run 1: MetadataMaxFilesIT.test:119 � TestTimedOut test timed out after 240 seconds
      Run 2: MetadataMaxFilesIT.test �  Appears to be stuck in thread Thread-26
    
    test(org.apache.accumulo.test.functional.WriteAheadLogIT)  Time elapsed: 44.258 sec  <<< ERROR!
    java.lang.RuntimeException: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server ip-172-31-55-219.ec2.internal:36366
            at org.apache.accumulo.test.functional.WriteAheadLogIT.test(WriteAheadLogIT.java:76)
    Caused by: org.apache.accumulo.core.client.impl.AccumuloServerException: Error on server ip-172-31-55-219.ec2.internal:36366
            at org.apache.accumulo.test.functional.WriteAheadLogIT.test(WriteAheadLogIT.java:76)
    Caused by: org.apache.thrift.TApplicationException: Internal error processing startMultiScan
            at org.apache.accumulo.test.functional.WriteAheadLogIT.test(WriteAheadLogIT.java:76)
    ...
    org.apache.accumulo.core.client.AccumuloException: org.apache.thrift.TApplicationException: Internal error processing waitForFlush
            at org.apache.accumulo.test.functional.BigRootTabletIT.test(BigRootTabletIT.java:58)
    Caused by: org.apache.thrift.TApplicationException: Internal error processing waitForFlush
            at org.apache.accumulo.test.functional.BigRootTabletIT.test(BigRootTabletIT.java:58)
    



---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    I ran "mvn verify -Psunny" and all tests passed.  I am running the full "mvn verify" now on AWS.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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/159#discussion_r83235022
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/system/SourceSwitchingIterator.java ---
    @@ -213,4 +221,13 @@ public void setInterruptFlag(AtomicBoolean flag) {
           source.setInterruptFlag(flag);
         }
       }
    +
    +  @Override
    +  public void close() throws Exception {
    +    copies.forEach(ssi -> ssi.closeSafely());
    --- End diff --
    
    this may need to sync on copies


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r108522399
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ColumnFamilyCounter.java ---
    @@ -89,4 +89,11 @@ public Value getTopValue() {
         return null;
       }
     
    +  @Override
    --- End diff --
    
    I would definitely be interested in clean up of the iterators.  @ctubbsii @joshelser Any ones in particular you had in mind?  I will create a ticket.


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    Revisiting these changes... rebased and running verify -Psunny.  If all goes well I will merge these changes soon.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r84754335
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/system/SourceSwitchingIterator.java ---
    @@ -162,6 +162,14 @@ private void readNext(boolean initialSeek) throws IOException {
       private boolean switchSource() throws IOException {
         if (!source.isCurrent()) {
           source = source.getNewDataSource();
    +      // if our source actually changed, then attempt to close the previous iterator
    +      try {
    +        if (iter != null) {
    +          iter.close();
    +        }
    +      } catch (Exception e) {
    +        throw new IOException(e);
    --- End diff --
    
    Fixed in ce0bd48fb8daa7eeed384d61cf7982ff49863275


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r88744008
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java ---
    @@ -147,4 +147,13 @@
        *              if not supported.
        */
       SortedKeyValueIterator<K,V> deepCopy(IteratorEnvironment env);
    +
    +  /**
    +   * Closes the Iterator. This should be overridden by the implementing class that has access to <tt>SortedKeyValueIterator source</tt> provided in the
    --- End diff --
    
    > They are very situational so there never seems to be a consistent way to handle them
    
    Well, we need to figure this out. We *need* to understand how the framework is supposed to work. If that depends on how the context in how the iterators are being invoked (which I really don't think it should), we need to encapsulate that somewhere because users will have the same question. Complicated or not, we must be clear.
    
    > In general, I think iterators should avoid throwing RuntimeException here. However, they should probably pass through any coming from a source.close(), after they are done cleaning up their own resources.
    
    Hrm. I think I see where you are coming from (if there's some DFSClient exception, we want to propagate that). What if I channel my inner @dhutchis -- I have some iterator which I'm expecting results to be sent to some external system when close() is called but I run into some I/O related exception. Should the Scan/Compaction itself fail and be retried? Do we say "oh well"?
    
    I'm starting to think that we really should have a design doc for this feature. I'm worried about this making Iterators even more complicated than they already are...


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    > That approach won't work with deep copies, or similar scenarios where the iterator "stack" grows outside the purview of the framework.
    
    Good point =)


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    I ran WriteAheadLogIT by itself and it passed. It looks like my VM just ran out of disk space.
    
    During the BigRootTabletIT... the RFile is being closed by SSI during switchSource.  Here's my debug exception stacktrace:
    ```
            at org.apache.accumulo.core.file.rfile.RFile$LocalityGroupReader.close(RFile.java:714)
            at org.apache.accumulo.core.file.rfile.RFile$Reader.closeLocalityGroupReaders(RFile.java:1173)
            at org.apache.accumulo.core.file.rfile.RFile$Reader.close(RFile.java:1194)
            at org.apache.accumulo.core.iterators.system.SourceSwitchingIterator.switchSource(SourceSwitchingIterator.java:167)
    ```
    This looks like the same issue in LargeRowIT.
    @keith-turner I think this proves your concerns about closing RFiles that are reused. Sadly, the SSI seemed like one of the system Iterators that could benefit from this change.



---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r84332426
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java ---
    @@ -163,15 +163,15 @@ private static Path createTmpDir(AccumuloConfiguration acuConf, VolumeManager fs
             try {
               if (reader != null)
                 reader.close();
    -        } catch (IOException e) {
    +        } catch (Exception e) {
    --- End diff --
    
    Should these just catch RuntimeException, then? Or does this reader still throw checked exceptions, too?


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r88687963
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java ---
    @@ -147,4 +147,13 @@
        *              if not supported.
        */
       SortedKeyValueIterator<K,V> deepCopy(IteratorEnvironment env);
    +
    +  /**
    +   * Closes the Iterator. This should be overridden by the implementing class that has access to <tt>SortedKeyValueIterator source</tt> provided in the
    --- End diff --
    
    In advance, thanks for working with me on this. I know this is difficult.
    
    > We don't want to say MUST since there may not be anything to close. 
    
    Isn't there always the `source` passed in via `init(...)` that should be closed?
    
    > I don't mention Exceptions because I wasn't sure what to say. Sometimes we eat, log and continue and sometimes we throw. Do we have best practices for handling exceptions?
    
    I would expect this would be something that you would know what to do with it :). You must have modified the code invoking `close()` on the iterator stack, so I assumed this was something that you had thought through :)
    
    > such as source provided in the init method
    
    This is still not sitting right with me. It's still coming across as "well, sometimes you need to close the `source`" without defining when that actually is. Is the following equivalent to what you're trying to phrase: "If init is called on this iterator, the provided `source` must be closed by this iterator"?


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r88679462
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java ---
    @@ -147,4 +147,13 @@
        *              if not supported.
        */
       SortedKeyValueIterator<K,V> deepCopy(IteratorEnvironment env);
    +
    +  /**
    +   * Closes the Iterator. This should be overridden by the implementing class that has access to <tt>SortedKeyValueIterator source</tt> provided in the
    --- End diff --
    
    Simplified version (still nothing about exceptions though): 
    Closes any resources that were opened by the <tt>SortedKeyValueIterator</tt>. This method does nothing by default. If the implementing class opens any kind of resources, such as <tt>source</tt> provided in the <tt>init</tt> method, than it should be overridden to close those resources.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r84969204
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/system/SourceSwitchingIterator.java ---
    @@ -162,6 +162,10 @@ private void readNext(boolean initialSeek) throws IOException {
       private boolean switchSource() throws IOException {
         if (!source.isCurrent()) {
           source = source.getNewDataSource();
    +      // if our source actually changed, then attempt to close the previous iterator
    +      if (iter != null) {
    --- End diff --
    
    Thanks for the help @keith-turner ! Merged in 4a83688


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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/159#discussion_r83235234
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java ---
    @@ -110,6 +111,19 @@ public TermSource(SortedKeyValueIterator<Key,Value> iter, Text term, boolean not
         public String getTermString() {
           return (this.term == null) ? "Iterator" : this.term.toString();
         }
    +
    +    @Override
    +    public void close() throws Exception {
    +      this.iter.close();
    +    }
    +
    +    public void closeSafely() {
    --- End diff --
    
    why override default impl in interface?


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r108473678
  
    --- Diff: core/src/test/java/org/apache/accumulo/core/iterators/system/ColumnFamilySkippingIteratorTest.java ---
    @@ -236,4 +237,45 @@ public void test3() throws Exception {
     
         // System.out.println(ci.getCount());
       }
    +
    +  private static class CloseTestIter extends SortedMapIterator {
    +
    +    int closeCallCount = 0;
    +
    +    public CloseTestIter(SortedMap<Key,Value> map) {
    +      super(map);
    +    }
    +
    +    @Override
    +    public void close() {
    +      System.out.println("Closing inner CloseIterator.");
    +      closeCallCount++;
    +    }
    +  }
    +
    +  public void testClose() throws Exception {
    +    TreeMap<Key,Value> tm = new TreeMap<>();
    +    put(tm, "r1", "cf1", "cq1", 5, "v1");
    +    put(tm, "r1", "cf1", "cq3", 5, "v2");
    +    put(tm, "r2", "cf1", "cq1", 5, "v3");
    +    put(tm, "r2", "cf2", "cq4", 5, "v4");
    +    put(tm, "r2", "cf2", "cq5", 5, "v5");
    +    put(tm, "r3", "cf3", "cq6", 5, "v6");
    +
    +    CloseTestIter closeIter = new CloseTestIter(tm);
    +    ColumnFamilySkippingIterator cfi = new ColumnFamilySkippingIterator(closeIter);
    +
    +    assertEquals(0, closeIter.closeCallCount);
    +    HashSet<ByteSequence> colfams = new HashSet<>();
    +    colfams.add(new ArrayByteSequence("cf2"));
    +    cfi.seek(new Range(), colfams, true);
    +    testAndCallnext(cfi, "r2", "cf2", "cq4", 5, "v4");
    +    testAndCallnext(cfi, "r2", "cf2", "cq5", 5, "v5");
    +    assertFalse(cfi.hasTop());
    +
    +    System.out.println("Closing ColumnFamilySkippingIterator");
    +    cfi.close();
    --- End diff --
    
    Aha! I see. Sorry about that.
    
    Do we have an IntegrationTest that checks for the TabletServer invoking the close 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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r108464160
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java ---
    @@ -163,22 +163,22 @@ private static Path createTmpDir(AccumuloConfiguration acuConf, VolumeManager fs
             try {
               if (reader != null)
                 reader.close();
    -        } catch (IOException e) {
    +        } catch (RuntimeException e) {
               log.error("{}", e.getMessage(), e);
    --- End diff --
    
    Can you update these log messages (the ones in this file that you switched from catching `IOException` to `RuntimeException`) to have a real error message (e.g. "Caught exception while closing RFile reader"), 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 pull request #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r83736440
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java ---
    @@ -163,15 +163,15 @@ private static Path createTmpDir(AccumuloConfiguration acuConf, VolumeManager fs
             try {
               if (reader != null)
                 reader.close();
    -        } catch (IOException e) {
    +        } catch (Exception e) {
    --- End diff --
    
    Are these still needed if the reader can't throw?


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    Added entry to the Iterator design section of the user manual in 752c12e



---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    Thanks for the input @dhutchis. 
    
    > Specify the contract that every iterator must close() its source. This strategy works if every iterator, system and user-defined, follows the contract. Existing iterators would need to be modified to do this, so it's a breaking change.
    
    I agree this is a sound strategy. The problem I am running into is our underlying reuse of RFiles throughout the code. This became apparent when I ran the BigRootTabletIT and IllegalStateException (Locality group reader closed) was thrown during a scan after attempting to read from a closed RFile. 
    
    I don't know how we can implement this strategy without making a fundamental change to how we use RFiles.  And if this is the case, then it would seem the risk of a change this complex, would far out weight the reward of something nice to have.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r83739144
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java ---
    @@ -1164,11 +1167,7 @@ private Reader(Reader r, boolean useSample) {
     
         private void closeLocalityGroupReaders() {
           for (LocalityGroupReader lgr : currentReaders) {
    -        try {
    -          lgr.close();
    -        } catch (IOException e) {
    -          log.warn("Errored out attempting to close LocalityGroupReader.", e);
    -        }
    +        lgr.close();
    --- End diff --
    
    If `currentReaders` were changed to an ArrayList, this could use the stream `forEach`. Not sure if that would impact performance, but it'd probably improve readability in a few places.


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    With the exception of OrIterator and Keith's concerns about Scanner reusing RFiles, I believe I have addressed all comments. I created ACCUMULO-4520 for implementation of Christopher's ideas pertaining to Java 8. Finally, I didn't create close tests for every Iterator but I think hit each of the different type of super classes with the tests I did create.  


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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/159#discussion_r84768594
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/system/SequenceFileIterator.java ---
    @@ -104,8 +104,12 @@ public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> op
       }
     
       @Override
    -  public void close() throws IOException {
    -    reader.close();
    +  public void close() {
    +    try {
    +      reader.close();
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    --- End diff --
    
    Use `UncheckedIOException`?


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r84888578
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java ---
    @@ -614,12 +614,15 @@ public DataSource getNewDataSource() {
     
           if (!isCurrent()) {
             switched = true;
    -        iter = null;
             try {
    +          if (iter != null) {
    +            iter.close();
    +          }
    +          iter = null;
               // ensure files are referenced even if iterator was never seeked before
               iterator();
    -        } catch (IOException e) {
    -          throw new RuntimeException();
    +        } catch (Exception e) {
    +          throw new RuntimeException(e);
    --- End diff --
    
    fixed in 93d86ceecef6f21eba8f60f7b6bdc29b43c1f83e


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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/159#discussion_r84777479
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/OrIterator.java ---
    @@ -219,8 +224,14 @@ public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean i
           // 1) NOT an iterator
           // 2) we have seeked into the next term (ie: seek man, get man001)
           // then ignore it as a valid source
    -      if (!(TS.iter.hasTop()) || ((TS.term != null) && (TS.term.compareTo(TS.iter.getTopKey().getColumnFamily()) != 0)))
    +      if (!(TS.iter.hasTop()) || ((TS.term != null) && (TS.term.compareTo(TS.iter.getTopKey().getColumnFamily()) != 0))) {
    +        try {
    +          TS.close();
    --- End diff --
    
    TS is closed here, but a little later its added to `sorted`.  This seems suspicious.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r84888931
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/system/MapFileIterator.java ---
    @@ -152,8 +152,12 @@ public void closeDeepCopies() throws IOException {
       }
     
       @Override
    -  public void close() throws IOException {
    -    reader.close();
    +  public void close() {
    +    try {
    +      reader.close();
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    --- End diff --
    
    fixed in 93d86ceecef6f21eba8f60f7b6bdc29b43c1f83e


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r88652597
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java ---
    @@ -147,4 +147,13 @@
        *              if not supported.
        */
       SortedKeyValueIterator<K,V> deepCopy(IteratorEnvironment env);
    +
    +  /**
    +   * Closes the Iterator. This should be overridden by the implementing class that has access to <tt>SortedKeyValueIterator source</tt> provided in the
    --- End diff --
    
    Thanks for the link but I think a "SHOULD" is the most appropriate. We don't want to say MUST since there may not be anything to close. 
    
    I think the ambiguity stems from the flexibility we have with the Iterators.  Look at the doc for init for instance: "Initializes the iterator. Data should not be read from the source in this method."  Do we want to make them more rigid?
    
    I don't mention Exceptions because I wasn't sure what to say. Sometimes we eat, log and continue and sometimes we throw.  Do we have best practices for handling exceptions?
    
    I can change the "Typically" part to be a little more specific. 


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    I tried running mvn verify -Psunny and the ReadWriteIT failed. Scans were failing because of server side errors.   Seeing lots exceptions like the following in the tablet server logs.  I suspect the RFile exception is caused by reusing closed rfiles from the rfile pool.  Not sure whats going on with the memory iterator exception.
    
    ```
    java.lang.IllegalStateException: Memory iterator is closed
            at org.apache.accumulo.tserver.InMemoryMap$MemoryIterator.getSource(InMemoryMap.java:693)
            at org.apache.accumulo.core.iterators.WrappingIterator.close(WrappingIterator.java:99)
            at org.apache.accumulo.tserver.InMemoryMap$MemoryIterator.close(InMemoryMap.java:716)
            at org.apache.accumulo.tserver.tablet.TabletMemory.returnIterators(TabletMemory.java:170)
            at org.apache.accumulo.tserver.tablet.ScanDataSource.close(ScanDataSource.java:212)
            at org.apache.accumulo.tserver.tablet.Scanner.read(Scanner.java:119)
            at org.apache.accumulo.tserver.scan.NextBatchTask.run(NextBatchTask.java:71)
            at org.apache.htrace.wrappers.TraceRunnable.run(TraceRunnable.java:57)
            at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
            at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
            at org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
            at java.lang.Thread.run(Thread.java:745)
    ```
    
    ```
    java.lang.IllegalStateException: Locality group reader closed
            at org.apache.accumulo.core.file.rfile.RFile$LocalityGroupReader.seek(RFile.java:812)
            at org.apache.accumulo.core.iterators.system.LocalityGroupIterator.seek(LocalityGroupIterator.java:138)
            at org.apache.accumulo.core.file.rfile.RFile$Reader.seek(RFile.java:1332)
            at org.apache.accumulo.server.problems.ProblemReportingIterator.seek(ProblemReportingIterator.java:98)
            at org.apache.accumulo.core.iterators.system.MultiIterator.seek(MultiIterator.java:106)
            at org.apache.accumulo.core.iterators.WrappingIterator.seek(WrappingIterator.java:93)
            at org.apache.accumulo.core.iterators.system.StatsIterator.seek(StatsIterator.java:64)
            at org.apache.accumulo.core.iterators.WrappingIterator.seek(WrappingIterator.java:93)
            at org.apache.accumulo.core.iterators.system.DeletingIterator.seek(DeletingIterator.java:68)
            at org.apache.accumulo.core.iterators.WrappingIterator.seek(WrappingIterator.java:93)
            at org.apache.accumulo.core.iterators.SkippingIterator.seek(SkippingIterator.java:42)
            at org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator.seek(ColumnFamilySkippingIterator.java:123)
            at org.apache.accumulo.core.iterators.WrappingIterator.seek(WrappingIterator.java:93)
            at org.apache.accumulo.core.iterators.Filter.seek(Filter.java:64)
            at org.apache.accumulo.core.iterators.WrappingIterator.seek(WrappingIterator.java:93)
            at org.apache.accumulo.core.iterators.Filter.seek(Filter.java:64)
            at org.apache.accumulo.core.iterators.system.SynchronizedIterator.seek(SynchronizedIterator.java:56)
            at org.apache.accumulo.core.iterators.WrappingIterator.seek(WrappingIterator.java:93)
            at org.apache.accumulo.core.iterators.user.VersioningIterator.seek(VersioningIterator.java:81)
            at org.apache.accumulo.core.iterators.system.SourceSwitchingIterator.readNext(SourceSwitchingIterator.java:135)
            at org.apache.accumulo.core.iterators.system.SourceSwitchingIterator.seek(SourceSwitchingIterator.java:186)
            at org.apache.accumulo.tserver.tablet.Tablet.nextBatch(Tablet.java:719)
            at org.apache.accumulo.tserver.tablet.Scanner.read(Scanner.java:86)
            at org.apache.accumulo.tserver.scan.NextBatchTask.run(NextBatchTask.java:71)
            at org.apache.htrace.wrappers.TraceRunnable.run(TraceRunnable.java:57)
            at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
            at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
            at org.apache.accumulo.fate.util.LoggingRunnable.run(LoggingRunnable.java:35)
            at java.lang.Thread.run(Thread.java:745)
    
    ```


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    After discussing the issues with these changes with @keith-turner we decided to narrow the scope of the close implementation to only be for User Iterators. My attempt at trying to close iterators at the system level was dangerous and unnecessary. There are already mechanisms in place for handling resources at the lower levels. 


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r83743192
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java ---
    @@ -388,9 +389,10 @@ else if (env.getIteratorScope() == IteratorScope.minc)
               }
               throw new CompactionCanceledException();
             }
    -
    +      } catch (Exception e) {
    +        throw new IOException(e);
           } finally {
    -        CompactionStats lgMajcStats = new CompactionStats(citr.getCount(), entriesCompacted);
    +        CompactionStats lgMajcStats = new CompactionStats(statsCount, entriesCompacted);
    --- End diff --
    
    Careful about suppressed exceptions here, especially when using multiple resources in a single try-with-resources, and mixing try-with-resources with a finally block. Any exception thrown in the finally block will suppress try-with-resources, and exception from closing one resource may result in others being suppressed.
    
    How do we want to handle suppressed exceptions? Do we assume they've already been logged earlier, or do we try to handle them here? Wrapping `Exception e` with `IOException` may also make it more difficult for the caller to track the suppressed exceptions attached to `e`, which is now in the `getCause()` of the `IOException`.
    
    Also, statsCount might be stale (or uninitialized) in the finally block if an exception occurs in the try block.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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/159#discussion_r84778064
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/system/SourceSwitchingIterator.java ---
    @@ -162,6 +162,10 @@ private void readNext(boolean initialSeek) throws IOException {
       private boolean switchSource() throws IOException {
         if (!source.isCurrent()) {
           source = source.getNewDataSource();
    +      // if our source actually changed, then attempt to close the previous iterator
    +      if (iter != null) {
    --- End diff --
    
    SourceSwitchingIterator has a unit test.  It would be really nice to add a test to check that iterators are closed when it switches sources.  I can help write this test if you would like, just let me know.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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/159#discussion_r84780608
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java ---
    @@ -253,6 +257,9 @@ public static double estimatePercentageLTE(VolumeManager fs, AccumuloConfigurati
           return (numLte + 1) / (double) (numKeys + 2);
     
         } finally {
    +      if (mmfi != null) {
    --- End diff --
    
    Would be better to move these lines after `cleanupIndexOp` ... or wrap with try+catch+log so that cleanupIndexOp can still run


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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/159#discussion_r84765329
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/system/MapFileIterator.java ---
    @@ -152,8 +152,12 @@ public void closeDeepCopies() throws IOException {
       }
     
       @Override
    -  public void close() throws IOException {
    -    reader.close();
    +  public void close() {
    +    try {
    +      reader.close();
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    --- End diff --
    
    I saw use of `UncheckedIOException` earlier.  Why not use that 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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r108522828
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ColumnFamilyCounter.java ---
    @@ -89,4 +89,11 @@ public Value getTopValue() {
         return null;
       }
     
    +  @Override
    --- End diff --
    
    > Any ones in particular you had in mind
    
    Negative from me. I just wouldn't be surprised if there were simplifications to be had.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r108471232
  
    --- Diff: core/src/test/java/org/apache/accumulo/core/iterators/system/ColumnFamilySkippingIteratorTest.java ---
    @@ -236,4 +237,45 @@ public void test3() throws Exception {
     
         // System.out.println(ci.getCount());
       }
    +
    +  private static class CloseTestIter extends SortedMapIterator {
    +
    +    int closeCallCount = 0;
    +
    +    public CloseTestIter(SortedMap<Key,Value> map) {
    +      super(map);
    +    }
    +
    +    @Override
    +    public void close() {
    +      System.out.println("Closing inner CloseIterator.");
    +      closeCallCount++;
    +    }
    +  }
    +
    +  public void testClose() throws Exception {
    +    TreeMap<Key,Value> tm = new TreeMap<>();
    +    put(tm, "r1", "cf1", "cq1", 5, "v1");
    +    put(tm, "r1", "cf1", "cq3", 5, "v2");
    +    put(tm, "r2", "cf1", "cq1", 5, "v3");
    +    put(tm, "r2", "cf2", "cq4", 5, "v4");
    +    put(tm, "r2", "cf2", "cq5", 5, "v5");
    +    put(tm, "r3", "cf3", "cq6", 5, "v6");
    +
    +    CloseTestIter closeIter = new CloseTestIter(tm);
    +    ColumnFamilySkippingIterator cfi = new ColumnFamilySkippingIterator(closeIter);
    +
    +    assertEquals(0, closeIter.closeCallCount);
    +    HashSet<ByteSequence> colfams = new HashSet<>();
    +    colfams.add(new ArrayByteSequence("cf2"));
    +    cfi.seek(new Range(), colfams, true);
    +    testAndCallnext(cfi, "r2", "cf2", "cq4", 5, "v4");
    +    testAndCallnext(cfi, "r2", "cf2", "cq5", 5, "v5");
    +    assertFalse(cfi.hasTop());
    +
    +    System.out.println("Closing ColumnFamilySkippingIterator");
    +    cfi.close();
    --- End diff --
    
    It looks that way but its testing whether the close gets propagated to the underlying iterator.  If the CFSKI had a noop like the MultiIterator the test would fail.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r87281103
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/file/FileSKVIterator.java ---
    @@ -35,5 +35,5 @@
       void closeDeepCopies() throws IOException;
    --- End diff --
    
    Does closeDeepCopies still need to throw an IOException, since close itself never does?


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    @ctubbsii @joshelser @keith-turner I made changes for previous discussions in e701ab6. The biggest impact occurred in RFile and FileUtil where I modified the exception handling.  FileUtil should behave the same except an Exception is caught instead of IOException. 


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

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


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r84902206
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/OrIterator.java ---
    @@ -219,8 +224,14 @@ public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean i
           // 1) NOT an iterator
           // 2) we have seeked into the next term (ie: seek man, get man001)
           // then ignore it as a valid source
    -      if (!(TS.iter.hasTop()) || ((TS.term != null) && (TS.term.compareTo(TS.iter.getTopKey().getColumnFamily()) != 0)))
    +      if (!(TS.iter.hasTop()) || ((TS.term != null) && (TS.term.compareTo(TS.iter.getTopKey().getColumnFamily()) != 0))) {
    +        try {
    +          TS.close();
    --- End diff --
    
    I agree but it seemed suspicious before I made any changes. Here is the code from before my change. I don't understand why the iter removes the element but still adds TS to the queue...
    if (!(TS.iter.hasTop()) || ((TS.term != null) && (TS.term.compareTo(TS.iter.getTopKey().getColumnFamily()) != 0)))
            iter.remove();
    
          // Otherwise, source is valid. Add it to the sources.
          sorted.add(TS);


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r83737095
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java ---
    @@ -257,26 +257,19 @@ public void setSource(final Scanner scanner) {
           tm.put(iterInfo.getPriority(), iterInfo);
         }
     
    -    SortedKeyValueIterator<Key,Value> skvi;
    -    try {
    -      skvi = IteratorUtil.loadIterators(smi, tm.values(), serverSideIteratorOptions, new ClientSideIteratorEnvironment(getSamplerConfiguration() != null,
    -          getIteratorSamplerConfigurationInternal()), false, null);
    -    } catch (IOException e) {
    -      throw new RuntimeException(e);
    -    }
    +    try (SortedKeyValueIterator<Key,Value> skvi = IteratorUtil.loadIterators(smi, tm.values(), serverSideIteratorOptions, new ClientSideIteratorEnvironment(
    +        getSamplerConfiguration() != null, getIteratorSamplerConfigurationInternal()), false, null)) {
     
    -    final Set<ByteSequence> colfs = new TreeSet<>();
    -    for (Column c : this.getFetchedColumns()) {
    -      colfs.add(new ArrayByteSequence(c.getColumnFamily()));
    -    }
    +      final Set<ByteSequence> colfs = new TreeSet<>();
    +      for (Column c : this.getFetchedColumns()) {
    +        colfs.add(new ArrayByteSequence(c.getColumnFamily()));
    +      }
     
    -    try {
           skvi.seek(range, colfs, true);
    +      return new IteratorAdapter(skvi);
    --- End diff --
    
    Won't turning this into a try-with-resources cause the new `skvi` returned in this `IteratorAdapter` to be closed?


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r83258841
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java ---
    @@ -257,26 +257,19 @@ public void setSource(final Scanner scanner) {
           tm.put(iterInfo.getPriority(), iterInfo);
         }
     
    -    SortedKeyValueIterator<Key,Value> skvi;
    -    try {
    -      skvi = IteratorUtil.loadIterators(smi, tm.values(), serverSideIteratorOptions, new ClientSideIteratorEnvironment(getSamplerConfiguration() != null,
    -          getIteratorSamplerConfigurationInternal()), false, null);
    -    } catch (IOException e) {
    -      throw new RuntimeException(e);
    -    }
    +    try (SortedKeyValueIterator<Key,Value> skvi = IteratorUtil.loadIterators(smi, tm.values(), serverSideIteratorOptions, new ClientSideIteratorEnvironment(
    +        getSamplerConfiguration() != null, getIteratorSamplerConfigurationInternal()), false, null)) {
     
    -    final Set<ByteSequence> colfs = new TreeSet<>();
    -    for (Column c : this.getFetchedColumns()) {
    -      colfs.add(new ArrayByteSequence(c.getColumnFamily()));
    -    }
    +      final Set<ByteSequence> colfs = new TreeSet<>();
    +      for (Column c : this.getFetchedColumns()) {
    +        colfs.add(new ArrayByteSequence(c.getColumnFamily()));
    +      }
     
    -    try {
           skvi.seek(range, colfs, true);
    -    } catch (IOException e) {
    +      return new IteratorAdapter(skvi);
    +    } catch (Exception e) {
    --- End diff --
    
    Quick fix is to just put a `} catch (RuntimeException e) { throw e; }` bit above the catch-all for `Exception`.


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

[GitHub] accumulo pull request #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r83427576
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/OrIterator.java ---
    @@ -249,4 +268,9 @@ final public boolean hasTop() {
       public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> options, IteratorEnvironment env) throws IOException {
         throw new UnsupportedOperationException();
       }
    +
    +  @Override
    +  public void close() throws Exception {
    +    sources.forEach(s -> s.closeSafely());
    +  }
    --- End diff --
    
    For exception handling of multiple sources, you thinking of not using forEach and going old school loop?


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r87409384
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/file/FileSKVIterator.java ---
    @@ -35,5 +35,5 @@
       void closeDeepCopies() throws IOException;
    --- End diff --
    
    Good call. Updated in fb03e2d740436515896df26d9fae3d680eec069d


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r108475855
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java ---
    @@ -147,4 +147,21 @@
        *              if not supported.
        */
       SortedKeyValueIterator<K,V> deepCopy(IteratorEnvironment env);
    +
    +  /**
    +   * Closes any resources that were opened by the <tt>SortedKeyValueIterator</tt>. This method does nothing by default. The implementing class must close its
    +   * <tt>SortedKeyValueIterator source</tt>. This will be provided in the <tt>init</tt> method or by the constructor.
    +   *
    +   * @exception UncheckedIOException
    --- End diff --
    
    I've been really mulling over whether or not to bring this up, but whatever, too late now:
    
    Is throwing `UncheckedIOException` harmful to us? I feel like this will make our server-side code more brittle (forgetting that we actually need to handle unexpected runtime situations) for the (small) gain of being able to use `AutoCloseable` (which I see as nothing but syntactic sugar). @dhutchis also touched on this, recommending that `close()` be idempotent, moving from `AutoCloseable` to just `Closeable`.
    
    For example, what if I had an Iterator which was buffering some state, when the iterator was closed, it would flush this data to some external location (read as: "performs I/O on `close()`"). We have no way to handle this scenario cleanly (users wouldn't even *know* it happened).
    
    Is there some benefit of being `AutoCloseable` that I'm not realizing? Or, am I blowing this concern out of scope of what this change is really meant to do (pointing that we need more docs as to what `close()` is limited to support)?


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    Another attempt at the proper language for the interface: 1a6cfca3e81a63d410326b32f25fd148fbede9e3


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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/159#discussion_r84780524
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java ---
    @@ -361,6 +369,9 @@ public static double estimatePercentageLTE(VolumeManager fs, AccumuloConfigurati
     
           return ret;
         } finally {
    +      if (mmfi != null) {
    --- End diff --
    
    Would be better to move these lines after `cleanupIndexOp` ... or wrap with try+catch+log so that cleanupIndexOp can still run


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r83260389
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/OrIterator.java ---
    @@ -80,6 +80,19 @@ public int compareTo(TermSource o) {
           // sorted after they have been determined to be valid.
           return this.iter.getTopKey().compareColumnQualifier(o.iter.getTopKey().getColumnQualifier());
         }
    +
    +    @Override
    +    public void close() throws Exception {
    +      this.iter.close();
    +    }
    +
    +    public void closeSafely() {
    --- End diff --
    
    I made TermSource just implement AutoCloseable.  But I could make it implement SKVI instead


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    >  If all goes well I will merge these changes soon.
    
    I don't see any doc-related changes included...


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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/159#discussion_r84783230
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java ---
    @@ -225,10 +225,18 @@ void close(boolean sawErrors) {
           fileManager = null;
         }
     
    -    if (statsIterator != null) {
    -      statsIterator.report();
    -    }
    +    try {
    +      if (statsIterator != null) {
    +        statsIterator.report();
    +        statsIterator.close();
    +      }
     
    +      if (iter != null) {
    +        iter.close();
    +      }
    +    } catch (Exception e) {
    +      throw new IOException(e);
    --- End diff --
    
    Why do this?


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r84906761
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/OrIterator.java ---
    @@ -219,8 +224,14 @@ public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean i
           // 1) NOT an iterator
           // 2) we have seeked into the next term (ie: seek man, get man001)
           // then ignore it as a valid source
    -      if (!(TS.iter.hasTop()) || ((TS.term != null) && (TS.term.compareTo(TS.iter.getTopKey().getColumnFamily()) != 0)))
    +      if (!(TS.iter.hasTop()) || ((TS.term != null) && (TS.term.compareTo(TS.iter.getTopKey().getColumnFamily()) != 0))) {
    +        try {
    +          TS.close();
    +        } catch (Exception e) {
    +          throw new IOException(e);
    --- End diff --
    
    This is holdover from removing the exception throws. I will remove it but have concerns about this class (see above 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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r83739603
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/ColumnFamilyCounter.java ---
    @@ -89,4 +89,11 @@ public Value getTopValue() {
         return null;
       }
     
    +  @Override
    --- End diff --
    
    I wonder if we could clean up some of these iterators implementations by making them use `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 pull request #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r84888609
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/system/SequenceFileIterator.java ---
    @@ -104,8 +104,12 @@ public void init(SortedKeyValueIterator<Key,Value> source, Map<String,String> op
       }
     
       @Override
    -  public void close() throws IOException {
    -    reader.close();
    +  public void close() {
    +    try {
    +      reader.close();
    +    } catch (IOException e) {
    +      throw new RuntimeException(e);
    --- End diff --
    
    fixed in 93d86ceecef6f21eba8f60f7b6bdc29b43c1f83e


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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/159#discussion_r83234698
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/OrIterator.java ---
    @@ -80,6 +80,19 @@ public int compareTo(TermSource o) {
           // sorted after they have been determined to be valid.
           return this.iter.getTopKey().compareColumnQualifier(o.iter.getTopKey().getColumnQualifier());
         }
    +
    +    @Override
    +    public void close() throws Exception {
    +      this.iter.close();
    +    }
    +
    +    public void closeSafely() {
    --- End diff --
    
    why redifine this if its defined in interface?


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r84888602
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/ScanDataSource.java ---
    @@ -225,10 +225,18 @@ void close(boolean sawErrors) {
           fileManager = null;
         }
     
    -    if (statsIterator != null) {
    -      statsIterator.report();
    -    }
    +    try {
    +      if (statsIterator != null) {
    +        statsIterator.report();
    +        statsIterator.close();
    +      }
     
    +      if (iter != null) {
    +        iter.close();
    +      }
    +    } catch (Exception e) {
    +      throw new IOException(e);
    --- End diff --
    
    fixed in 93d86ceecef6f21eba8f60f7b6bdc29b43c1f83e


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    I reverted a few changes with 92e6def901ea868d84f40d282275bce585a2db05 and mvn verify -Psunny now passes. I think the remaining files in question are Scanner and Compactor.  I am writing a test for Compactor to make sure close will only propagate down to the multi-iterator when the try with resources finishes.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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/159#discussion_r84777320
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/OrIterator.java ---
    @@ -219,8 +224,14 @@ public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean i
           // 1) NOT an iterator
           // 2) we have seeked into the next term (ie: seek man, get man001)
           // then ignore it as a valid source
    -      if (!(TS.iter.hasTop()) || ((TS.term != null) && (TS.term.compareTo(TS.iter.getTopKey().getColumnFamily()) != 0)))
    +      if (!(TS.iter.hasTop()) || ((TS.term != null) && (TS.term.compareTo(TS.iter.getTopKey().getColumnFamily()) != 0))) {
    +        try {
    +          TS.close();
    +        } catch (Exception e) {
    +          throw new IOException(e);
    --- End diff --
    
    Why have this this try/catch around close()?  The code compiles w/o 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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    The `internal error` needs to be investigated.  This indicates an unexpected error in the tablet server.  Can look in the tserver logs for that test.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r88512703
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java ---
    @@ -61,39 +61,27 @@ public ScanBatch read() throws IOException, TabletClosedException {
         Batch results = null;
     
         try {
    +      scannerSemaphore.acquire();
    +    } catch (InterruptedException e) {
    +      sawException = true;
    +    }
     
    -      try {
    -        scannerSemaphore.acquire();
    -      } catch (InterruptedException e) {
    -        sawException = true;
    -      }
    -
    -      // sawException may have occurred within close, so we cannot assume that an interrupted exception was its cause
    -      if (sawException)
    -        throw new IllegalStateException("Tried to use scanner after exception occurred.");
    +    // sawException may have occurred within close, so we cannot assume that an interrupted exception was its cause
    +    if (sawException)
    +      throw new IllegalStateException("Tried to use scanner after exception occurred.");
     
    -      if (scanClosed)
    -        throw new IllegalStateException("Tried to use scanner after it was closed.");
    +    if (scanClosed)
    +      throw new IllegalStateException("Tried to use scanner after it was closed.");
     
    -      if (options.isIsolated()) {
    -        if (isolatedDataSource == null)
    -          isolatedDataSource = new ScanDataSource(tablet, options);
    -        dataSource = isolatedDataSource;
    -      } else {
    -        dataSource = new ScanDataSource(tablet, options);
    -      }
    +    if (options.isIsolated()) {
    +      if (isolatedDataSource == null)
    +        isolatedDataSource = new ScanDataSource(tablet, options);
    +      dataSource = isolatedDataSource;
    +    } else {
    +      dataSource = new ScanDataSource(tablet, options);
    +    }
     
    -      SortedKeyValueIterator<Key,Value> iter;
    -
    -      if (options.isIsolated()) {
    -        if (isolatedIter == null)
    -          isolatedIter = new SourceSwitchingIterator(dataSource, true);
    -        else
    -          isolatedDataSource.reattachFileManager();
    -        iter = isolatedIter;
    -      } else {
    -        iter = new SourceSwitchingIterator(dataSource, false);
    -      }
    +    try (SortedKeyValueIterator<Key,Value> iter = createIter(dataSource)) {
    --- End diff --
    
    I reverted changes to ScanDataSource.  Think these changes are OK now? 


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r88543950
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java ---
    @@ -147,4 +147,13 @@
        *              if not supported.
        */
       SortedKeyValueIterator<K,V> deepCopy(IteratorEnvironment env);
    +
    +  /**
    +   * Closes the Iterator. This should be overridden by the implementing class that has access to <tt>SortedKeyValueIterator source</tt> provided in the
    --- End diff --
    
    Also, I'm referring to SHOULD and MUST as defined by https://www.ietf.org/rfc/rfc2119.txt in case it was not clear.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r108461701
  
    --- Diff: core/src/test/java/org/apache/accumulo/core/iterators/system/ColumnFamilySkippingIteratorTest.java ---
    @@ -236,4 +237,45 @@ public void test3() throws Exception {
     
         // System.out.println(ci.getCount());
       }
    +
    +  private static class CloseTestIter extends SortedMapIterator {
    +
    +    int closeCallCount = 0;
    +
    +    public CloseTestIter(SortedMap<Key,Value> map) {
    +      super(map);
    +    }
    +
    +    @Override
    +    public void close() {
    +      System.out.println("Closing inner CloseIterator.");
    +      closeCallCount++;
    +    }
    +  }
    +
    +  public void testClose() throws Exception {
    +    TreeMap<Key,Value> tm = new TreeMap<>();
    +    put(tm, "r1", "cf1", "cq1", 5, "v1");
    +    put(tm, "r1", "cf1", "cq3", 5, "v2");
    +    put(tm, "r2", "cf1", "cq1", 5, "v3");
    +    put(tm, "r2", "cf2", "cq4", 5, "v4");
    +    put(tm, "r2", "cf2", "cq5", 5, "v5");
    +    put(tm, "r3", "cf3", "cq6", 5, "v6");
    +
    +    CloseTestIter closeIter = new CloseTestIter(tm);
    +    ColumnFamilySkippingIterator cfi = new ColumnFamilySkippingIterator(closeIter);
    +
    +    assertEquals(0, closeIter.closeCallCount);
    +    HashSet<ByteSequence> colfams = new HashSet<>();
    +    colfams.add(new ArrayByteSequence("cf2"));
    +    cfi.seek(new Range(), colfams, true);
    +    testAndCallnext(cfi, "r2", "cf2", "cq4", 5, "v4");
    +    testAndCallnext(cfi, "r2", "cf2", "cq5", 5, "v5");
    +    assertFalse(cfi.hasTop());
    +
    +    System.out.println("Closing ColumnFamilySkippingIterator");
    +    cfi.close();
    --- End diff --
    
    Isn't this a bit contrived of a test? :) You're calling a method explicitly and then verifying that your methods' functions were invoked.
    
    What if you used MAC and created a close method which wrote to some externally visible location? e.g. a file or a znode.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r83740194
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/system/SourceSwitchingIterator.java ---
    @@ -162,6 +162,14 @@ private void readNext(boolean initialSeek) throws IOException {
       private boolean switchSource() throws IOException {
         if (!source.isCurrent()) {
           source = source.getNewDataSource();
    +      // if our source actually changed, then attempt to close the previous iterator
    +      try {
    +        if (iter != null) {
    +          iter.close();
    +        }
    +      } catch (Exception e) {
    +        throw new IOException(e);
    --- End diff --
    
    This is a private method. Do we need to wrap exceptions here, or should we handle it in the caller? It seems we might be converting them to IOExceptions here, only to have them converted back to RuntimeExceptions later.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r83743748
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Scanner.java ---
    @@ -120,16 +108,15 @@ public ScanBatch read() throws IOException, TabletClosedException {
           }
     
           sawException = true;
    -      dataSource.close(true);
           throw ioe;
    -    } catch (RuntimeException re) {
    +    } catch (Exception e) {
    --- End diff --
    
    Careful about catching other RuntimeExceptions and wrapping them again with yet another RuntimeException. Also, try to pick a more specific RuntimeException, if possible. Perhaps `IllegalStateException`, `IllegalArgumentException`, or `UncheckedIOException`.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r84754214
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java ---
    @@ -388,9 +389,10 @@ else if (env.getIteratorScope() == IteratorScope.minc)
               }
               throw new CompactionCanceledException();
             }
    -
    +      } catch (Exception e) {
    +        throw new IOException(e);
           } finally {
    -        CompactionStats lgMajcStats = new CompactionStats(citr.getCount(), entriesCompacted);
    +        CompactionStats lgMajcStats = new CompactionStats(statsCount, entriesCompacted);
    --- End diff --
    
    Fixed in ce0bd48fb8daa7eeed384d61cf7982ff49863275


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r83739466
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java ---
    @@ -1193,16 +1192,14 @@ public void close() throws IOException {
     
           if (sampleReaders != null) {
             for (LocalityGroupReader lgr : sampleReaders) {
    -          try {
    -            lgr.close();
    -          } catch (IOException e) {
    -            log.warn("Errored out attempting to close LocalityGroupReader.", e);
    -          }
    +          lgr.close();
             }
           }
     
           try {
             reader.close();
    +      } catch (IOException e) {
    +        throw new RuntimeException(e);
    --- End diff --
    
    Should probably use the new `UncheckedIOException` wrapper here and elsewhere when `IOException` is wrapped.


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    More specifically, we discussed not closing RFiles and in mem map.  One easy way to do this is to make the MultiIterator close do nothing.  System and user iters sit above the multi-iterator.  Below it are rfiles and in mem map iter.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r84743890
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java ---
    @@ -1193,16 +1192,14 @@ public void close() throws IOException {
     
           if (sampleReaders != null) {
             for (LocalityGroupReader lgr : sampleReaders) {
    -          try {
    -            lgr.close();
    -          } catch (IOException e) {
    -            log.warn("Errored out attempting to close LocalityGroupReader.", e);
    -          }
    +          lgr.close();
             }
           }
     
           try {
             reader.close();
    +      } catch (IOException e) {
    +        throw new RuntimeException(e);
    --- End diff --
    
    Fixed in 5aa4a99a30052c1560d46d3430886d9e5e650d61. Want me to create a ticket to utilize `UncheckedIOException` everywhere in the code for this situation?


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r88543835
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java ---
    @@ -147,4 +147,13 @@
        *              if not supported.
        */
       SortedKeyValueIterator<K,V> deepCopy(IteratorEnvironment env);
    +
    +  /**
    +   * Closes the Iterator. This should be overridden by the implementing class that has access to <tt>SortedKeyValueIterator source</tt> provided in the
    --- End diff --
    
    Thanks for the rewording, Mike. I'm still left with an ambiguous feeling after reading it.
    
    * Would recommend not using words like "Typically". If an SKVI's `source` should be closed in the `close()` method, be explicit. This also makes me wonder, are these semantics `SHOULD` or `MUST`? What happens to an SKVI I have which doesn't implement close?
    * Exceptional cases are still not covered. Should implementations of `close()` eat exceptions or propagate them as RuntimeExceptions?


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r83735883
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/iterators/SortedKeyValueIterator.java ---
    @@ -147,4 +147,13 @@
        *              if not supported.
        */
       SortedKeyValueIterator<K,V> deepCopy(IteratorEnvironment env);
    +
    +  /**
    +   * Closes the Iterator. This must be overridden by the implementing class that has access to <tt>SortedKeyValueIterator source</tt> provided in the
    --- End diff --
    
    This should say "should be overridden". Or it should say that it "is expected that". Clearly it's not a "must" situation, since the default method does exist, and allows them to avoid it. This could cause problems mixing legacy iterators with new iterators, as the new ones will be expected to be closed, but the legacy ones won't close them.


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

[GitHub] accumulo issue #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    >  > we decided to narrow the scope of the close implementation to only be for User Iterators
    
    >How are "user iterators" going to be defined?
    
    Ping on this one as I think it got lost, @milleruntime. Is `close()` still only going to be implemented for iterators in `org.apache.accumulo.core.iterator.user`?


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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/159#discussion_r84768859
  
    --- Diff: server/tserver/src/main/java/org/apache/accumulo/tserver/InMemoryMap.java ---
    @@ -614,12 +614,15 @@ public DataSource getNewDataSource() {
     
           if (!isCurrent()) {
             switched = true;
    -        iter = null;
             try {
    +          if (iter != null) {
    +            iter.close();
    +          }
    +          iter = null;
               // ensure files are referenced even if iterator was never seeked before
               iterator();
    -        } catch (IOException e) {
    -          throw new RuntimeException();
    +        } catch (Exception e) {
    +          throw new RuntimeException(e);
    --- End diff --
    
    I think this can be changed to :
    
    ```java
    } catch (IOException e) {
      throw new UncheckedIOException(e)
    }
    ```


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r84718204
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/util/FileUtil.java ---
    @@ -163,15 +163,15 @@ private static Path createTmpDir(AccumuloConfiguration acuConf, VolumeManager fs
             try {
               if (reader != null)
                 reader.close();
    -        } catch (IOException e) {
    +        } catch (Exception e) {
    --- End diff --
    
    Changed to just catch RuntimeException and I also found places where MultiIterator wasn't being closed in 38049d4d5a655915c89eeb6a5635572967d12118


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    @dhutchis wrote:
    > With this approach, each iterator is only responsible for closing the resources it itself opens. No need to consider other iterators. Seems more palatable, don't you think?
    
    That approach won't work with deep copies, or similar scenarios where the iterator "stack" grows outside the purview of the framework.


---
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 #159: ACCUMULO-1280: many changes for closing iterator...

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

    https://github.com/apache/accumulo/pull/159#discussion_r83249120
  
    --- Diff: core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java ---
    @@ -257,26 +257,19 @@ public void setSource(final Scanner scanner) {
           tm.put(iterInfo.getPriority(), iterInfo);
         }
     
    -    SortedKeyValueIterator<Key,Value> skvi;
    -    try {
    -      skvi = IteratorUtil.loadIterators(smi, tm.values(), serverSideIteratorOptions, new ClientSideIteratorEnvironment(getSamplerConfiguration() != null,
    -          getIteratorSamplerConfigurationInternal()), false, null);
    -    } catch (IOException e) {
    -      throw new RuntimeException(e);
    -    }
    +    try (SortedKeyValueIterator<Key,Value> skvi = IteratorUtil.loadIterators(smi, tm.values(), serverSideIteratorOptions, new ClientSideIteratorEnvironment(
    +        getSamplerConfiguration() != null, getIteratorSamplerConfigurationInternal()), false, null)) {
     
    -    final Set<ByteSequence> colfs = new TreeSet<>();
    -    for (Column c : this.getFetchedColumns()) {
    -      colfs.add(new ArrayByteSequence(c.getColumnFamily()));
    -    }
    +      final Set<ByteSequence> colfs = new TreeSet<>();
    +      for (Column c : this.getFetchedColumns()) {
    +        colfs.add(new ArrayByteSequence(c.getColumnFamily()));
    +      }
     
    -    try {
           skvi.seek(range, colfs, true);
    -    } catch (IOException e) {
    +      return new IteratorAdapter(skvi);
    +    } catch (Exception e) {
    --- End diff --
    
    I thought catch exceptions that extend RuntimeException and rethrow them as a different exception. Seems very undesirable.


---
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 #159: ACCUMULO-1280: many changes for closing iterators

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

    https://github.com/apache/accumulo/pull/159
  
    It would be really nice to have unit test for all of the iterators that were changed to make sure they close their source.


---
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.
---