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

classic Classic list List threaded Threaded
89 messages Options
12345
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
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 <[hidden email]>
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
In reply to this post by ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
In reply to this post by ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
In reply to this post by ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
In reply to this post by ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
In reply to this post by ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
In reply to this post by ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
In reply to this post by ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
In reply to this post by ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
In reply to this post by ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
In reply to this post by ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
In reply to this post by ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
In reply to this post by ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
In reply to this post by ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
In reply to this post by ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
In reply to this post by ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
In reply to this post by ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
In reply to this post by ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
Reply | Threaded
Open this post in threaded view
|

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

ctubbsii
In reply to this post by ctubbsii
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 [hidden email] or file a JIRA ticket
with INFRA.
---
12345