Make Releasable extend AutoCloseable.

Java7's AutoCloseable allows to manage resources more nicely using
try-with-resources statements. Since the semantics of our Releasable interface
are very close to a Closeable, let's switch to it.

Close #5689
This commit is contained in:
Adrien Grand 2014-04-04 11:50:39 +02:00
parent e688f445ad
commit e589301806
117 changed files with 401 additions and 436 deletions

View File

@ -164,7 +164,7 @@ public class TransportIndicesStatusAction extends TransportBroadcastOperationAct
shardStatus.docs.maxDoc = searcher.reader().maxDoc(); shardStatus.docs.maxDoc = searcher.reader().maxDoc();
shardStatus.docs.deletedDocs = searcher.reader().numDeletedDocs(); shardStatus.docs.deletedDocs = searcher.reader().numDeletedDocs();
} finally { } finally {
searcher.release(); searcher.close();
} }
shardStatus.mergeStats = indexShard.mergeScheduler().stats(); shardStatus.mergeStats = indexShard.mergeScheduler().stats();

View File

@ -202,7 +202,7 @@ public class TransportValidateQueryAction extends TransportBroadcastOperationAct
valid = false; valid = false;
error = e.getMessage(); error = e.getMessage();
} finally { } finally {
SearchContext.current().release(); SearchContext.current().close();
SearchContext.removeCurrent(); SearchContext.removeCurrent();
} }
} }

View File

@ -199,7 +199,7 @@ public class TransportCountAction extends TransportBroadcastOperationAction<Coun
} }
} finally { } finally {
// this will also release the index searcher // this will also release the index searcher
context.release(); context.close();
SearchContext.removeCurrent(); SearchContext.removeCurrent();
} }
} }

View File

@ -149,7 +149,7 @@ public class TransportExplainAction extends TransportShardSingleOperationAction<
} catch (IOException e) { } catch (IOException e) {
throw new ElasticsearchException("Could not explain", e); throw new ElasticsearchException("Could not explain", e);
} finally { } finally {
context.release(); context.close();
SearchContext.removeCurrent(); SearchContext.removeCurrent();
} }
} }

View File

@ -170,7 +170,7 @@ public class TransportSuggestAction extends TransportBroadcastOperationAction<Su
} catch (Throwable ex) { } catch (Throwable ex) {
throw new ElasticsearchException("failed to execute suggest", ex); throw new ElasticsearchException("failed to execute suggest", ex);
} finally { } finally {
searcher.release(); searcher.close();
if (parser != null) { if (parser != null) {
parser.close(); parser.close();
} }

View File

@ -39,8 +39,7 @@ public class ReleasablePagedBytesReference extends PagedBytesReference implement
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
Releasables.release(bytearray); Releasables.close(bytearray);
return true;
} }
} }

View File

@ -22,9 +22,9 @@ package org.elasticsearch.common.lease;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
/** /**
* * Specialization of {@link AutoCloseable} that may only throw an {@link ElasticsearchException}.
*/ */
public interface Releasable { public interface Releasable extends AutoCloseable {
boolean release() throws ElasticsearchException; void close() throws ElasticsearchException;
} }

View File

@ -19,6 +19,8 @@
package org.elasticsearch.common.lease; package org.elasticsearch.common.lease;
import org.elasticsearch.ElasticsearchException;
import java.util.Arrays; import java.util.Arrays;
/** Utility methods to work with {@link Releasable}s. */ /** Utility methods to work with {@link Releasable}s. */
@ -35,12 +37,12 @@ public enum Releasables {
throw new RuntimeException(t); throw new RuntimeException(t);
} }
private static void release(Iterable<? extends Releasable> releasables, boolean ignoreException) { private static void close(Iterable<? extends Releasable> releasables, boolean ignoreException) {
Throwable th = null; Throwable th = null;
for (Releasable releasable : releasables) { for (Releasable releasable : releasables) {
if (releasable != null) { if (releasable != null) {
try { try {
releasable.release(); releasable.close();
} catch (Throwable t) { } catch (Throwable t) {
if (th == null) { if (th == null) {
th = t; th = t;
@ -54,36 +56,70 @@ public enum Releasables {
} }
/** Release the provided {@link Releasable}s. */ /** Release the provided {@link Releasable}s. */
public static void release(Iterable<? extends Releasable> releasables) { public static void close(Iterable<? extends Releasable> releasables) {
release(releasables, false); close(releasables, false);
} }
/** Release the provided {@link Releasable}s. */ /** Release the provided {@link Releasable}s. */
public static void release(Releasable... releasables) { public static void close(Releasable... releasables) {
release(Arrays.asList(releasables)); close(Arrays.asList(releasables));
} }
/** Release the provided {@link Releasable}s, ignoring exceptions. */ /** Release the provided {@link Releasable}s, ignoring exceptions. */
public static void releaseWhileHandlingException(Iterable<Releasable> releasables) { public static void closeWhileHandlingException(Iterable<Releasable> releasables) {
release(releasables, true); close(releasables, true);
} }
/** Release the provided {@link Releasable}s, ignoring exceptions. */ /** Release the provided {@link Releasable}s, ignoring exceptions. */
public static void releaseWhileHandlingException(Releasable... releasables) { public static void closeWhileHandlingException(Releasable... releasables) {
releaseWhileHandlingException(Arrays.asList(releasables)); closeWhileHandlingException(Arrays.asList(releasables));
} }
/** Release the provided {@link Releasable}s, ignoring exceptions if <code>success</code> is <tt>false</tt>. */ /** Release the provided {@link Releasable}s, ignoring exceptions if <code>success</code> is <tt>false</tt>. */
public static void release(boolean success, Iterable<Releasable> releasables) { public static void close(boolean success, Iterable<Releasable> releasables) {
if (success) { if (success) {
release(releasables); close(releasables);
} else { } else {
releaseWhileHandlingException(releasables); closeWhileHandlingException(releasables);
} }
} }
/** Release the provided {@link Releasable}s, ignoring exceptions if <code>success</code> is <tt>false</tt>. */ /** Release the provided {@link Releasable}s, ignoring exceptions if <code>success</code> is <tt>false</tt>. */
public static void release(boolean success, Releasable... releasables) { public static void close(boolean success, Releasable... releasables) {
release(success, Arrays.asList(releasables)); close(success, Arrays.asList(releasables));
}
/** Wrap several releasables into a single one. This is typically useful for use with try-with-resources: for example let's assume
* that you store in a list several resources that you would like to see released after execution of the try block:
*
* <pre>
* List&lt;Releasable&gt; resources = ...;
* try (Releasable releasable = Releasables.wrap(resources)) {
* // do something
* }
* // the resources will be released when reaching here
* </pre>
*/
public static Releasable wrap(final Iterable<Releasable> releasables) {
return new Releasable() {
@Override
public void close() throws ElasticsearchException {
Releasables.close(releasables);
}
};
}
/** @see #wrap(Iterable) */
public static Releasable wrap(final Releasable... releasables) {
return new Releasable() {
@Override
public void close() throws ElasticsearchException {
Releasables.close(releasables);
}
};
} }
} }

View File

@ -107,15 +107,14 @@ public class FreqTermsEnum extends FilterableTermsEnum implements Releasable {
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
try { try {
Releasables.release(cachedTermOrds, termDocFreqs, termsTotalFreqs); Releasables.close(cachedTermOrds, termDocFreqs, termsTotalFreqs);
} finally { } finally {
cachedTermOrds = null; cachedTermOrds = null;
termDocFreqs = null; termDocFreqs = null;
termsTotalFreqs = null; termsTotalFreqs = null;
} }
return true;
} }
} }

View File

@ -37,6 +37,6 @@ public class ReleaseChannelFutureListener implements ChannelFutureListener {
@Override @Override
public void operationComplete(ChannelFuture future) throws Exception { public void operationComplete(ChannelFuture future) throws Exception {
releasable.release(); releasable.close();
} }
} }

View File

@ -87,7 +87,7 @@ public class DequeRecycler<T> extends AbstractRecycler<T> {
} }
@Override @Override
public boolean release() { public void close() {
if (value == null) { if (value == null) {
throw new ElasticsearchIllegalStateException("recycler entry already released..."); throw new ElasticsearchIllegalStateException("recycler entry already released...");
} }
@ -101,7 +101,6 @@ public class DequeRecycler<T> extends AbstractRecycler<T> {
} }
value = null; value = null;
afterRelease(recycle); afterRelease(recycle);
return true;
} }
} }
} }

View File

@ -58,12 +58,11 @@ public class NoneRecycler<T> extends AbstractRecycler<T> {
} }
@Override @Override
public boolean release() { public void close() {
if (value == null) { if (value == null) {
throw new ElasticsearchIllegalStateException("recycler entry already released..."); throw new ElasticsearchIllegalStateException("recycler entry already released...");
} }
value = null; value = null;
return true;
} }
} }
} }

View File

@ -131,7 +131,7 @@ public enum Recyclers {
} }
/** /**
* Wrap the provided recycler so that calls to {@link Recycler#obtain()} and {@link Recycler.V#release()} are protected by * Wrap the provided recycler so that calls to {@link Recycler#obtain()} and {@link Recycler.V#close()} are protected by
* a lock. * a lock.
*/ */
public static <T> Recycler<T> locked(final Recycler<T> recycler) { public static <T> Recycler<T> locked(final Recycler<T> recycler) {
@ -167,9 +167,9 @@ public enum Recyclers {
return new Recycler.V<T>() { return new Recycler.V<T>() {
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
synchronized (lock) { synchronized (lock) {
return delegate.release(); delegate.close();
} }
} }

View File

@ -31,10 +31,9 @@ abstract class AbstractArray implements Releasable {
} }
@Override @Override
public boolean release() { public void close() {
assert !released : "double release"; assert !released : "double release";
released = true; released = true;
return true; // nothing to release by default
} }
} }

View File

@ -155,19 +155,18 @@ abstract class AbstractBigArray extends AbstractArray {
protected final void releasePage(int page) { protected final void releasePage(int page) {
if (recycler != null) { if (recycler != null) {
cache[page].release(); cache[page].close();
cache[page] = null; cache[page] = null;
} }
} }
@Override @Override
public final boolean release() { public final void close() {
super.release(); super.close();
if (recycler != null) { if (recycler != null) {
Releasables.release(cache); Releasables.close(cache);
cache = null; cache = null;
} }
return true;
} }
} }

View File

@ -56,8 +56,7 @@ abstract class AbstractHash extends AbstractPagedHashMap {
} }
@Override @Override
public boolean release() { public void close() {
Releasables.release(ids); Releasables.close(ids);
return true;
} }
} }

View File

@ -96,9 +96,8 @@ public class BigArrays extends AbstractComponent {
} }
@Override @Override
public final boolean release() { public final void close() {
Releasables.release(releasable); Releasables.close(releasable);
return true;
} }
} }
@ -369,7 +368,7 @@ public class BigArrays extends AbstractComponent {
final ByteArray newArray = newByteArray(size, arr.clearOnResize); final ByteArray newArray = newByteArray(size, arr.clearOnResize);
final byte[] rawArray = ((ByteArrayWrapper) array).array; final byte[] rawArray = ((ByteArrayWrapper) array).array;
newArray.set(0, rawArray, 0, (int) Math.min(rawArray.length, newArray.size())); newArray.set(0, rawArray, 0, (int) Math.min(rawArray.length, newArray.size()));
array.release(); array.close();
return newArray; return newArray;
} }
} }
@ -451,7 +450,7 @@ public class BigArrays extends AbstractComponent {
for (long i = 0, end = Math.min(size, array.size()); i < end; ++i) { for (long i = 0, end = Math.min(size, array.size()); i < end; ++i) {
newArray.set(i, array.get(i)); newArray.set(i, array.get(i));
} }
array.release(); array.close();
return newArray; return newArray;
} }
} }
@ -500,7 +499,7 @@ public class BigArrays extends AbstractComponent {
for (long i = 0, end = Math.min(size, array.size()); i < end; ++i) { for (long i = 0, end = Math.min(size, array.size()); i < end; ++i) {
newArray.set(i, array.get(i)); newArray.set(i, array.get(i));
} }
array.release(); array.close();
return newArray; return newArray;
} }
} }
@ -546,7 +545,7 @@ public class BigArrays extends AbstractComponent {
for (long i = 0, end = Math.min(size, array.size()); i < end; ++i) { for (long i = 0, end = Math.min(size, array.size()); i < end; ++i) {
newArray.set(i, array.get(i)); newArray.set(i, array.get(i));
} }
array.release(); array.close();
return newArray; return newArray;
} }
} }
@ -630,7 +629,7 @@ public class BigArrays extends AbstractComponent {
for (long i = 0, end = Math.min(size, array.size()); i < end; ++i) { for (long i = 0, end = Math.min(size, array.size()); i < end; ++i) {
newArray.set(i, array.get(i)); newArray.set(i, array.get(i));
} }
array.release(); array.close();
return newArray; return newArray;
} }
} }

View File

@ -21,6 +21,7 @@ package org.elasticsearch.common.util;
import com.carrotsearch.hppc.hash.MurmurHash3; import com.carrotsearch.hppc.hash.MurmurHash3;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
/** /**
@ -60,7 +61,7 @@ public final class BytesRefHash extends AbstractHash {
/** /**
* Return the key at <code>0 &lte; index &lte; capacity()</code>. The result is undefined if the slot is unused. * Return the key at <code>0 &lte; index &lte; capacity()</code>. The result is undefined if the slot is unused.
* <p color="red">Beware that the content of the {@link BytesRef} may become invalid as soon as {@link #release()} is called</p> * <p color="red">Beware that the content of the {@link BytesRef} may become invalid as soon as {@link #close()} is called</p>
*/ */
public BytesRef get(long id, BytesRef dest) { public BytesRef get(long id, BytesRef dest) {
final long startOffset = startOffsets.get(id); final long startOffset = startOffsets.get(id);
@ -159,15 +160,10 @@ public final class BytesRefHash extends AbstractHash {
} }
@Override @Override
public boolean release() { public void close() {
boolean success = false; try (Releasable releasable = Releasables.wrap(bytes, hashes, startOffsets)) {
try { super.close();
super.release();
success = true;
} finally {
Releasables.release(success, bytes, hashes, startOffsets);
} }
return true;
} }
} }

View File

@ -166,9 +166,8 @@ public class DoubleObjectPagedHashMap<T> extends AbstractPagedHashMap implements
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
Releasables.release(keys, values); Releasables.close(keys, values);
return true;
} }
@Override @Override

View File

@ -19,7 +19,7 @@
package org.elasticsearch.common.util; package org.elasticsearch.common.util;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasable;
/** /**
* Specialized hash table implementation similar to BytesRefHash that maps * Specialized hash table implementation similar to BytesRefHash that maps
@ -119,15 +119,10 @@ public final class LongHash extends AbstractHash {
} }
@Override @Override
public boolean release() { public void close() {
boolean success = false; try (Releasable releasable = keys) {
try { super.close();
super.release();
success = true;
} finally {
Releasables.release(success, keys);
} }
return true;
} }
} }

View File

@ -166,9 +166,8 @@ public class LongObjectPagedHashMap<T> extends AbstractPagedHashMap implements I
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
Releasables.release(keys, values); Releasables.close(keys, values);
return true;
} }
@Override @Override

View File

@ -170,7 +170,7 @@ public class NettyHttpChannel extends HttpChannel {
} }
} finally { } finally {
if (!addedReleaseListener && content instanceof Releasable) { if (!addedReleaseListener && content instanceof Releasable) {
((Releasable) content).release(); ((Releasable) content).close();
} }
} }
} }

View File

@ -24,14 +24,12 @@ import org.apache.lucene.index.IndexDeletionPolicy;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.inject.name.Named; import org.elasticsearch.common.inject.name.Named;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.IndexShardComponent; import org.elasticsearch.index.shard.IndexShardComponent;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
/** /**
@ -105,7 +103,7 @@ public class SnapshotDeletionPolicy extends AbstractESDeletionPolicy {
/** /**
* Snapshots all the current commits in the index. Make sure to call * Snapshots all the current commits in the index. Make sure to call
* {@link SnapshotIndexCommits#release()} to release it. * {@link SnapshotIndexCommits#close()} to release it.
*/ */
public SnapshotIndexCommits snapshots() throws IOException { public SnapshotIndexCommits snapshots() throws IOException {
synchronized (mutex) { synchronized (mutex) {
@ -122,7 +120,7 @@ public class SnapshotDeletionPolicy extends AbstractESDeletionPolicy {
/** /**
* Returns a snapshot of the index (for the last commit point). Make * Returns a snapshot of the index (for the last commit point). Make
* sure to call {@link SnapshotIndexCommit#release()} in order to release it. * sure to call {@link SnapshotIndexCommit#close()} in order to release it.
*/ */
public SnapshotIndexCommit snapshot() throws IOException { public SnapshotIndexCommit snapshot() throws IOException {
synchronized (mutex) { synchronized (mutex) {
@ -164,7 +162,7 @@ public class SnapshotDeletionPolicy extends AbstractESDeletionPolicy {
/** /**
* Releases the version provided. Returns <tt>true</tt> if the release was successful. * Releases the version provided. Returns <tt>true</tt> if the release was successful.
*/ */
boolean release(long version) { boolean close(long version) {
synchronized (mutex) { synchronized (mutex) {
SnapshotDeletionPolicy.SnapshotHolder holder = snapshots.get(version); SnapshotDeletionPolicy.SnapshotHolder holder = snapshots.get(version);
if (holder == null) { if (holder == null) {
@ -193,12 +191,12 @@ public class SnapshotDeletionPolicy extends AbstractESDeletionPolicy {
} }
@Override @Override
public boolean release() { public void close() {
if (released) { if (released) {
return false; return;
} }
released = true; released = true;
return ((SnapshotIndexCommit) delegate).release(); ((SnapshotIndexCommit) delegate).close();
} }
} }

View File

@ -27,7 +27,7 @@ import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
/** /**
* A snapshot index commit point. While this is held and {@link #release()} * A snapshot index commit point. While this is held and {@link #close()}
* was not called, no files will be deleted that relates to this commit point * was not called, no files will be deleted that relates to this commit point
* ({@link #getFileNames()}). * ({@link #getFileNames()}).
* *
@ -57,8 +57,8 @@ public class SnapshotIndexCommit extends IndexCommitDelegate implements Releasab
* Releases the current snapshot, returning <code>true</code> if it was * Releases the current snapshot, returning <code>true</code> if it was
* actually released. * actually released.
*/ */
public boolean release() { public void close() {
return deletionPolicy.release(getGeneration()); deletionPolicy.close(getGeneration());
} }
/** /**

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.deletionpolicy; package org.elasticsearch.index.deletionpolicy;
import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
@ -47,11 +48,7 @@ public class SnapshotIndexCommits implements Iterable<SnapshotIndexCommit>, Rele
return commits.iterator(); return commits.iterator();
} }
public boolean release() { public void close() {
boolean result = false; Releasables.close(commits);
for (SnapshotIndexCommit snapshot : commits) {
result |= snapshot.release();
}
return result;
} }
} }

View File

@ -86,7 +86,7 @@ public interface Engine extends IndexShardComponent, CloseableComponent {
* API is responsible for releasing the returned seacher in a * API is responsible for releasing the returned seacher in a
* safe manner, preferably in a try/finally block. * safe manner, preferably in a try/finally block.
* *
* @see Searcher#release() * @see Searcher#close()
*/ */
Searcher acquireSearcher(String source) throws EngineException; Searcher acquireSearcher(String source) throws EngineException;
@ -197,9 +197,8 @@ public interface Engine extends IndexShardComponent, CloseableComponent {
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
// nothing to release here... // nothing to release here...
return true;
} }
} }
@ -932,7 +931,7 @@ public interface Engine extends IndexShardComponent, CloseableComponent {
public void release() { public void release() {
if (searcher != null) { if (searcher != null) {
searcher.release(); searcher.close();
} }
} }
} }

View File

@ -349,14 +349,14 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
try { try {
docIdAndVersion = Versions.loadDocIdAndVersion(searcher.reader(), get.uid()); docIdAndVersion = Versions.loadDocIdAndVersion(searcher.reader(), get.uid());
} catch (Throwable e) { } catch (Throwable e) {
Releasables.releaseWhileHandlingException(searcher); Releasables.closeWhileHandlingException(searcher);
//TODO: A better exception goes here //TODO: A better exception goes here
throw new EngineException(shardId(), "Couldn't resolve version", e); throw new EngineException(shardId(), "Couldn't resolve version", e);
} }
if (get.version() != Versions.MATCH_ANY && docIdAndVersion != null) { if (get.version() != Versions.MATCH_ANY && docIdAndVersion != null) {
if (get.versionType().isVersionConflict(docIdAndVersion.version, get.version())) { if (get.versionType().isVersionConflict(docIdAndVersion.version, get.version())) {
Releasables.release(searcher); Releasables.close(searcher);
Uid uid = Uid.createUid(get.uid().text()); Uid uid = Uid.createUid(get.uid().text());
throw new VersionConflictEngineException(shardId, uid.type(), uid.id(), docIdAndVersion.version, get.version()); throw new VersionConflictEngineException(shardId, uid.type(), uid.id(), docIdAndVersion.version, get.version());
} }
@ -366,7 +366,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
// don't release the searcher on this path, it is the responsability of the caller to call GetResult.release // don't release the searcher on this path, it is the responsability of the caller to call GetResult.release
return new GetResult(searcher, docIdAndVersion); return new GetResult(searcher, docIdAndVersion);
} else { } else {
Releasables.release(searcher); Releasables.close(searcher);
return GetResult.NOT_EXISTS; return GetResult.NOT_EXISTS;
} }
@ -1053,14 +1053,14 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
try { try {
phase1Snapshot = deletionPolicy.snapshot(); phase1Snapshot = deletionPolicy.snapshot();
} catch (Throwable e) { } catch (Throwable e) {
Releasables.releaseWhileHandlingException(onGoingRecoveries); Releasables.closeWhileHandlingException(onGoingRecoveries);
throw new RecoveryEngineException(shardId, 1, "Snapshot failed", e); throw new RecoveryEngineException(shardId, 1, "Snapshot failed", e);
} }
try { try {
recoveryHandler.phase1(phase1Snapshot); recoveryHandler.phase1(phase1Snapshot);
} catch (Throwable e) { } catch (Throwable e) {
Releasables.releaseWhileHandlingException(onGoingRecoveries, phase1Snapshot); Releasables.closeWhileHandlingException(onGoingRecoveries, phase1Snapshot);
if (closed) { if (closed) {
e = new EngineClosedException(shardId, e); e = new EngineClosedException(shardId, e);
} }
@ -1071,7 +1071,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
try { try {
phase2Snapshot = translog.snapshot(); phase2Snapshot = translog.snapshot();
} catch (Throwable e) { } catch (Throwable e) {
Releasables.releaseWhileHandlingException(onGoingRecoveries, phase1Snapshot); Releasables.closeWhileHandlingException(onGoingRecoveries, phase1Snapshot);
if (closed) { if (closed) {
e = new EngineClosedException(shardId, e); e = new EngineClosedException(shardId, e);
} }
@ -1081,7 +1081,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
try { try {
recoveryHandler.phase2(phase2Snapshot); recoveryHandler.phase2(phase2Snapshot);
} catch (Throwable e) { } catch (Throwable e) {
Releasables.releaseWhileHandlingException(onGoingRecoveries, phase1Snapshot, phase2Snapshot); Releasables.closeWhileHandlingException(onGoingRecoveries, phase1Snapshot, phase2Snapshot);
if (closed) { if (closed) {
e = new EngineClosedException(shardId, e); e = new EngineClosedException(shardId, e);
} }
@ -1098,9 +1098,9 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
} catch (Throwable e) { } catch (Throwable e) {
throw new RecoveryEngineException(shardId, 3, "Execution failed", e); throw new RecoveryEngineException(shardId, 3, "Execution failed", e);
} finally { } finally {
Releasables.release(success, onGoingRecoveries); Releasables.close(success, onGoingRecoveries);
rwl.writeLock().unlock(); rwl.writeLock().unlock();
Releasables.release(success, phase1Snapshot, phase2Snapshot, phase3Snapshot); Releasables.close(success, phase1Snapshot, phase2Snapshot, phase3Snapshot);
} }
} }
@ -1122,7 +1122,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
} }
return stats; return stats;
} finally { } finally {
searcher.release(); searcher.close();
} }
} finally { } finally {
rwl.readLock().unlock(); rwl.readLock().unlock();
@ -1158,7 +1158,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
segments.put(info.info.name, segment); segments.put(info.info.name, segment);
} }
} finally { } finally {
searcher.release(); searcher.close();
} }
// now, correlate or add the committed ones... // now, correlate or add the committed ones...
@ -1292,7 +1292,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
try { try {
return Versions.loadVersion(searcher.reader(), uid); return Versions.loadVersion(searcher.reader(), uid);
} finally { } finally {
searcher.release(); searcher.close();
} }
} }
@ -1465,25 +1465,23 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
if (!released.compareAndSet(false, true)) { if (!released.compareAndSet(false, true)) {
/* In general, searchers should never be released twice or this would break reference counting. There is one rare case /* In general, searchers should never be released twice or this would break reference counting. There is one rare case
* when it might happen though: when the request and the Reaper thread would both try to release it in a very short amount * when it might happen though: when the request and the Reaper thread would both try to release it in a very short amount
* of time, this is why we only log a warning instead of throwing an exception. * of time, this is why we only log a warning instead of throwing an exception.
*/ */
logger.warn("Searcher was released twice", new ElasticsearchIllegalStateException("Double release")); logger.warn("Searcher was released twice", new ElasticsearchIllegalStateException("Double release"));
return false; return;
} }
try { try {
manager.release(searcher); manager.release(searcher);
return true;
} catch (IOException e) { } catch (IOException e) {
return false; throw new ElasticsearchIllegalStateException("Cannot close", e);
} catch (AlreadyClosedException e) { } catch (AlreadyClosedException e) {
/* this one can happen if we already closed the /* this one can happen if we already closed the
* underlying store / directory and we call into the * underlying store / directory and we call into the
* IndexWriter to free up pending files. */ * IndexWriter to free up pending files. */
return false;
} finally { } finally {
store.decRef(); store.decRef();
} }
@ -1575,7 +1573,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
} }
} finally { } finally {
// no need to release the fullSearcher, nothing really is done... // no need to release the fullSearcher, nothing really is done...
Releasables.release(currentSearcher); Releasables.close(currentSearcher);
if (newSearcher != null && closeNewSearcher) { if (newSearcher != null && closeNewSearcher) {
IOUtils.closeWhileHandlingException(newSearcher.getIndexReader()); // ignore IOUtils.closeWhileHandlingException(newSearcher.getIndexReader()); // ignore
} }
@ -1604,9 +1602,8 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
endRecovery(); endRecovery();
return true;
} }
} }

View File

@ -271,7 +271,7 @@ public class PercolatorQueriesRegistry extends AbstractIndexShardComponent {
shardPercolateService.addedQuery(entry.getKey(), previousQuery, entry.getValue()); shardPercolateService.addedQuery(entry.getKey(), previousQuery, entry.getValue());
} }
} finally { } finally {
searcher.release(); searcher.close();
} }
} catch (Exception e) { } catch (Exception e) {
throw new PercolatorException(shardId.index(), "failed to load queries from percolator index", e); throw new PercolatorException(shardId.index(), "failed to load queries from percolator index", e);

View File

@ -128,7 +128,7 @@ public class ChildrenConstantScoreQuery extends Query {
return parentWeight; return parentWeight;
} finally { } finally {
if (releaseParentIds) { if (releaseParentIds) {
Releasables.release(parentIds); Releasables.close(parentIds);
} }
} }
@ -209,9 +209,8 @@ public class ChildrenConstantScoreQuery extends Query {
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
Releasables.release(parentIds); Releasables.close(parentIds);
return true;
} }
private final class ParentDocIdIterator extends FilteredDocIdSetIterator { private final class ParentDocIdIterator extends FilteredDocIdSetIterator {

View File

@ -166,7 +166,7 @@ public class ChildrenQuery extends Query {
scores = maxCollector.scores; scores = maxCollector.scores;
occurrences = null; occurrences = null;
} finally { } finally {
Releasables.release(maxCollector.parentIdsIndex); Releasables.close(maxCollector.parentIdsIndex);
} }
break; break;
case SUM: case SUM:
@ -177,7 +177,7 @@ public class ChildrenQuery extends Query {
scores = sumCollector.scores; scores = sumCollector.scores;
occurrences = null; occurrences = null;
} finally { } finally {
Releasables.release(sumCollector.parentIdsIndex); Releasables.close(sumCollector.parentIdsIndex);
} }
break; break;
case AVG: case AVG:
@ -188,7 +188,7 @@ public class ChildrenQuery extends Query {
scores = avgCollector.scores; scores = avgCollector.scores;
occurrences = avgCollector.occurrences; occurrences = avgCollector.occurrences;
} finally { } finally {
Releasables.release(avgCollector.parentIdsIndex); Releasables.close(avgCollector.parentIdsIndex);
} }
break; break;
default: default:
@ -197,7 +197,7 @@ public class ChildrenQuery extends Query {
int size = (int) parentIds.size(); int size = (int) parentIds.size();
if (size == 0) { if (size == 0) {
Releasables.release(parentIds, scores, occurrences); Releasables.close(parentIds, scores, occurrences);
return Queries.newMatchNoDocsQuery().createWeight(searcher); return Queries.newMatchNoDocsQuery().createWeight(searcher);
} }
@ -287,9 +287,8 @@ public class ChildrenQuery extends Query {
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
Releasables.release(parentIds, scores, occurrences); Releasables.close(parentIds, scores, occurrences);
return true;
} }
private class ParentScorer extends Scorer { private class ParentScorer extends Scorer {

View File

@ -91,12 +91,11 @@ public class CustomQueryWrappingFilter extends NoCacheFilter implements Releasab
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
// We need to clear the docIdSets, otherwise this is leaved unused // We need to clear the docIdSets, otherwise this is leaved unused
// DocIdSets around and can potentially become a memory leak. // DocIdSets around and can potentially become a memory leak.
docIdSets = null; docIdSets = null;
searcher = null; searcher = null;
return true;
} }
@Override @Override

View File

@ -109,7 +109,7 @@ public class ParentConstantScoreQuery extends Query {
return childrenWeight; return childrenWeight;
} finally { } finally {
if (releaseParentIds) { if (releaseParentIds) {
Releasables.release(parentIds); Releasables.close(parentIds);
} }
} }
} }
@ -182,9 +182,8 @@ public class ParentConstantScoreQuery extends Query {
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
Releasables.release(parentIds); Releasables.close(parentIds);
return true;
} }
private final class ChildrenDocIdIterator extends FilteredDocIdSetIterator { private final class ChildrenDocIdIterator extends FilteredDocIdSetIterator {

View File

@ -153,7 +153,7 @@ public class ParentQuery extends Query {
} finally { } finally {
if (releaseCollectorResource) { if (releaseCollectorResource) {
// either if we run into an exception or if we return early // either if we run into an exception or if we return early
Releasables.release(collector.parentIds, collector.scores); Releasables.close(collector.parentIds, collector.scores);
} }
} }
searchContext.addReleasable(childWeight); searchContext.addReleasable(childWeight);
@ -272,9 +272,8 @@ public class ParentQuery extends Query {
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
Releasables.release(parentIds, scores, parentIdsIndexCache); Releasables.close(parentIds, scores, parentIdsIndexCache);
return true;
} }
} }

View File

@ -246,10 +246,10 @@ public class TopChildrenQuery extends Query {
ParentDoc[] _parentDocs = value.v().values().toArray(ParentDoc.class); ParentDoc[] _parentDocs = value.v().values().toArray(ParentDoc.class);
Arrays.sort(_parentDocs, PARENT_DOC_COMP); Arrays.sort(_parentDocs, PARENT_DOC_COMP);
parentDocs.v().put(keys[i], _parentDocs); parentDocs.v().put(keys[i], _parentDocs);
Releasables.release(value); Releasables.close(value);
} }
} }
Releasables.release(parentDocsPerReader); Releasables.close(parentDocsPerReader);
return parentHitsResolved; return parentHitsResolved;
} }
@ -321,9 +321,8 @@ public class TopChildrenQuery extends Query {
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
Releasables.release(parentDocs); Releasables.close(parentDocs);
return true;
} }
@Override @Override

View File

@ -495,7 +495,7 @@ public class InternalIndexShard extends AbstractIndexShardComponent implements I
try { try {
return new DocsStats(searcher.reader().numDocs(), searcher.reader().numDeletedDocs()); return new DocsStats(searcher.reader().numDocs(), searcher.reader().numDeletedDocs());
} finally { } finally {
searcher.release(); searcher.close();
} }
} }
@ -585,7 +585,7 @@ public class InternalIndexShard extends AbstractIndexShardComponent implements I
completionStats.add(completionPostingsFormat.completionStats(currentSearcher.reader(), fields)); completionStats.add(completionPostingsFormat.completionStats(currentSearcher.reader(), fields));
} }
} finally { } finally {
currentSearcher.release(); currentSearcher.close();
} }
return completionStats; return completionStats;
} }

View File

@ -90,7 +90,7 @@ public class IndexShardSnapshotAndRestoreService extends AbstractIndexShardCompo
logger.debug(sb.toString()); logger.debug(sb.toString());
} }
} finally { } finally {
snapshotIndexCommit.release(); snapshotIndexCommit.close();
} }
} catch (SnapshotFailedEngineException e) { } catch (SnapshotFailedEngineException e) {
throw e; throw e;

View File

@ -77,7 +77,7 @@ public class ShardTermVectorService extends AbstractIndexShardComponent {
} catch (Throwable ex) { } catch (Throwable ex) {
throw new ElasticsearchException("failed to execute term vector request", ex); throw new ElasticsearchException("failed to execute term vector request", ex);
} finally { } finally {
searcher.release(); searcher.close();
} }
return termVectorResponse; return termVectorResponse;
} }

View File

@ -138,8 +138,7 @@ public class FsChannelSnapshot implements Translog.Snapshot {
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
raf.decreaseRefCount(true); raf.decreaseRefCount(true);
return true;
} }
} }

View File

@ -368,7 +368,7 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
// ignore // ignore
} }
} }
Releasables.release(bytes); Releasables.close(bytes);
released = true; released = true;
return location; return location;
} catch (Throwable e) { } catch (Throwable e) {
@ -376,7 +376,7 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
} finally { } finally {
rwl.readLock().unlock(); rwl.readLock().unlock();
if (!released) { if (!released) {
Releasables.release(out.bytes()); Releasables.close(out.bytes());
} }
} }
} }

View File

@ -189,7 +189,7 @@ public class IndicesFilterCache extends AbstractComponent implements RemovalList
} }
schedule(); schedule();
} finally { } finally {
keys.release(); keys.close();
} }
} }
}); });

View File

@ -194,7 +194,7 @@ public class IndicesTTLService extends AbstractLifecycleComponent<IndicesTTLServ
} catch (Exception e) { } catch (Exception e) {
logger.warn("failed to purge", e); logger.warn("failed to purge", e);
} finally { } finally {
searcher.release(); searcher.close();
} }
} }
} }

View File

@ -120,14 +120,13 @@ class MultiDocumentPercolatorIndex implements PercolatorIndex {
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
try { try {
searcher.getIndexReader().close(); searcher.getIndexReader().close();
rootDocMemoryIndex.reset(); rootDocMemoryIndex.reset();
} catch (IOException e) { } catch (IOException e) {
throw new ElasticsearchException("failed to close IndexReader in percolator with nested doc", e); throw new ElasticsearchException("failed to close IndexReader in percolator with nested doc", e);
} }
return true;
} }
} }

View File

@ -29,6 +29,7 @@ import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.cache.recycler.CacheRecycler; import org.elasticsearch.cache.recycler.CacheRecycler;
import org.elasticsearch.cache.recycler.PageCacheRecycler; import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.lucene.HashedBytesRef; import org.elasticsearch.common.lucene.HashedBytesRef;
import org.elasticsearch.common.text.StringText; import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BigArrays;
@ -207,18 +208,13 @@ public class PercolateContext extends SearchContext {
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
try { try (Releasable releasable = Releasables.wrap(engineSearcher, docSearcher)) {
if (docSearcher != null) { if (docSearcher != null) {
IndexReader indexReader = docSearcher.reader(); IndexReader indexReader = docSearcher.reader();
fieldDataService.clear(indexReader); fieldDataService.clear(indexReader);
indexService.cache().clear(indexReader); indexService.cache().clear(indexReader);
return docSearcher.release();
} else {
return false;
} }
} finally {
engineSearcher.release();
} }
} }
@ -296,7 +292,7 @@ public class PercolateContext extends SearchContext {
// Unused: // Unused:
@Override @Override
public boolean clearAndRelease() { public void clearAndRelease() {
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }

View File

@ -227,7 +227,7 @@ public class PercolatorService extends AbstractComponent {
indexShard.readAllowed(); indexShard.readAllowed();
return action.doPercolate(request, context); return action.doPercolate(request, context);
} finally { } finally {
context.release(); context.close();
shardPercolateService.postPercolate(System.nanoTime() - startTime); shardPercolateService.postPercolate(System.nanoTime() - startTime);
} }
} }
@ -475,7 +475,7 @@ public class PercolatorService extends AbstractComponent {
} catch (Throwable e) { } catch (Throwable e) {
logger.warn("failed to execute", e); logger.warn("failed to execute", e);
} finally { } finally {
percolatorSearcher.release(); percolatorSearcher.close();
} }
return new PercolateShardResponse(count, context, request.index(), request.shardId()); return new PercolateShardResponse(count, context, request.index(), request.shardId());
} }
@ -586,7 +586,7 @@ public class PercolatorService extends AbstractComponent {
logger.debug("failed to execute", e); logger.debug("failed to execute", e);
throw new PercolateException(context.indexShard().shardId(), "failed to execute", e); throw new PercolateException(context.indexShard().shardId(), "failed to execute", e);
} finally { } finally {
percolatorSearcher.release(); percolatorSearcher.close();
} }
} }
}; };
@ -620,7 +620,7 @@ public class PercolatorService extends AbstractComponent {
logger.debug("failed to execute", e); logger.debug("failed to execute", e);
throw new PercolateException(context.indexShard().shardId(), "failed to execute", e); throw new PercolateException(context.indexShard().shardId(), "failed to execute", e);
} finally { } finally {
percolatorSearcher.release(); percolatorSearcher.close();
} }
} }
}; };
@ -764,7 +764,7 @@ public class PercolatorService extends AbstractComponent {
logger.debug("failed to execute", e); logger.debug("failed to execute", e);
throw new PercolateException(context.indexShard().shardId(), "failed to execute", e); throw new PercolateException(context.indexShard().shardId(), "failed to execute", e);
} finally { } finally {
percolatorSearcher.release(); percolatorSearcher.close();
} }
} }

View File

@ -90,14 +90,13 @@ class SingleDocumentPercolatorIndex implements PercolatorIndex {
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
try { try {
searcher.getIndexReader().close(); searcher.getIndexReader().close();
memoryIndex.reset(); memoryIndex.reset();
} catch (IOException e) { } catch (IOException e) {
throw new ElasticsearchException("failed to close percolator in-memory index", e); throw new ElasticsearchException("failed to close percolator in-memory index", e);
} }
return true;
} }
} }
} }

View File

@ -533,7 +533,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> {
} }
context.keepAlive(keepAlive); context.keepAlive(keepAlive);
} catch (Throwable e) { } catch (Throwable e) {
context.release(); context.close();
throw ExceptionsHelper.convertToRuntime(e); throw ExceptionsHelper.convertToRuntime(e);
} }
@ -546,7 +546,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> {
return; return;
} }
context.indexShard().searchService().onFreeContext(context); context.indexShard().searchService().onFreeContext(context);
context.release(); context.close();
} }
private void freeContext(SearchContext context) { private void freeContext(SearchContext context) {
@ -554,7 +554,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> {
if (removed != null) { if (removed != null) {
removed.indexShard().searchService().onFreeContext(removed); removed.indexShard().searchService().onFreeContext(removed);
} }
context.release(); context.close();
} }
public void freeAllScrollContexts() { public void freeAllScrollContexts() {

View File

@ -25,6 +25,7 @@ import org.apache.lucene.search.Query;
import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Scorer;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.lucene.search.XCollector; import org.elasticsearch.common.lucene.search.XCollector;
@ -106,8 +107,7 @@ public class AggregationPhase implements SearchPhase {
} }
Aggregator[] aggregators = context.aggregations().aggregators(); Aggregator[] aggregators = context.aggregations().aggregators();
boolean success = false; try (Releasable releasable = Releasables.wrap(aggregators)) {
try {
List<Aggregator> globals = new ArrayList<>(); List<Aggregator> globals = new ArrayList<>();
for (int i = 0; i < aggregators.length; i++) { for (int i = 0; i < aggregators.length; i++) {
if (aggregators[i] instanceof GlobalAggregator) { if (aggregators[i] instanceof GlobalAggregator) {
@ -136,9 +136,6 @@ public class AggregationPhase implements SearchPhase {
aggregations.add(aggregator.buildAggregation(0)); aggregations.add(aggregator.buildAggregation(0));
} }
context.queryResult().aggregations(new InternalAggregations(aggregations)); context.queryResult().aggregations(new InternalAggregations(aggregations));
success = true;
} finally {
Releasables.release(success, aggregators);
} }
} }

View File

@ -174,19 +174,14 @@ public abstract class Aggregator implements Releasable, ReaderContextAware {
/** Called upon release of the aggregator. */ /** Called upon release of the aggregator. */
@Override @Override
public boolean release() { public void close() {
boolean success = false; try (Releasable releasable = Releasables.wrap(subAggregators)) {
try { doClose();
doRelease();
success = true;
} finally {
Releasables.release(success, subAggregators);
} }
return true;
} }
/** Release instance-specific data. */ /** Release instance-specific data. */
protected void doRelease() {} protected void doClose() {}
/** /**
* Can be overriden by aggregator implementation to be called back when the collection phase ends. * Can be overriden by aggregator implementation to be called back when the collection phase ends.

View File

@ -134,7 +134,7 @@ public class AggregatorFactories {
} }
@Override @Override
public void doRelease() { public void doClose() {
final Iterable<Aggregator> aggregatorsIter = new Iterable<Aggregator>() { final Iterable<Aggregator> aggregatorsIter = new Iterable<Aggregator>() {
@Override @Override
@ -157,7 +157,7 @@ public class AggregatorFactories {
} }
}; };
Releasables.release(Iterables.concat(aggregatorsIter, Collections.singleton(aggregators))); Releasables.close(Iterables.concat(aggregatorsIter, Collections.singleton(aggregators)));
} }
}; };
} }

View File

@ -18,7 +18,7 @@
*/ */
package org.elasticsearch.search.aggregations.bucket; package org.elasticsearch.search.aggregations.bucket;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.util.LongArray; import org.elasticsearch.common.util.LongArray;
import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories; import org.elasticsearch.search.aggregations.AggregatorFactories;
@ -129,15 +129,10 @@ public abstract class BucketsAggregator extends Aggregator {
} }
@Override @Override
public final boolean release() { public final void close() {
boolean success = false; try (Releasable releasable = docCounts) {
try { super.close();
super.release();
success = true;
} finally {
Releasables.release(success, docCounts);
} }
return true;
} }
} }

View File

@ -127,8 +127,8 @@ public class GeoHashGridAggregator extends BucketsAggregator {
@Override @Override
public void doRelease() { public void doClose() {
Releasables.release(bucketOrds); Releasables.close(bucketOrds);
} }
} }

View File

@ -219,7 +219,7 @@ public class InternalGeoHashGrid extends InternalAggregation implements GeoHashG
List<Bucket> sameCellBuckets = cursor.value; List<Bucket> sameCellBuckets = cursor.value;
ordered.insertWithOverflow(sameCellBuckets.get(0).reduce(sameCellBuckets, reduceContext.bigArrays())); ordered.insertWithOverflow(sameCellBuckets.get(0).reduce(sameCellBuckets, reduceContext.bigArrays()));
} }
buckets.release(); buckets.close();
Bucket[] list = new Bucket[ordered.size()]; Bucket[] list = new Bucket[ordered.size()];
for (int i = ordered.size() - 1; i >= 0; i--) { for (int i = ordered.size() - 1; i >= 0; i--) {
list[i] = ordered.pop(); list[i] = ordered.pop();

View File

@ -129,8 +129,8 @@ public class HistogramAggregator extends BucketsAggregator {
} }
@Override @Override
public void doRelease() { public void doClose() {
Releasables.release(bucketOrds); Releasables.close(bucketOrds);
} }
public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric> { public static class Factory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric> {

View File

@ -370,7 +370,7 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
reducedBuckets.add(bucket); reducedBuckets.add(bucket);
} }
} }
bucketsByKey.release(); bucketsByKey.close();
// adding empty buckets in needed // adding empty buckets in needed
if (minDocCount == 0) { if (minDocCount == 0) {

View File

@ -103,8 +103,8 @@ public class SignificantLongTermsAggregator extends LongTermsAggregator {
} }
@Override @Override
public void doRelease() { public void doClose() {
Releasables.release(bucketOrds, termsAggFactory); Releasables.close(bucketOrds, termsAggFactory);
} }
} }

View File

@ -115,8 +115,8 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator {
} }
@Override @Override
public void doRelease() { public void doClose() {
Releasables.release(bucketOrds, termsAggFactory); Releasables.close(bucketOrds, termsAggFactory);
} }
/** /**
@ -143,7 +143,7 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator {
final long maxOrd = ordinals.getMaxOrd(); final long maxOrd = ordinals.getMaxOrd();
if (ordinalToBucket == null || ordinalToBucket.size() < maxOrd) { if (ordinalToBucket == null || ordinalToBucket.size() < maxOrd) {
if (ordinalToBucket != null) { if (ordinalToBucket != null) {
ordinalToBucket.release(); ordinalToBucket.close();
} }
ordinalToBucket = context().bigArrays().newLongArray(BigArrays.overSize(maxOrd), false); ordinalToBucket = context().bigArrays().newLongArray(BigArrays.overSize(maxOrd), false);
} }
@ -175,8 +175,8 @@ public class SignificantStringTermsAggregator extends StringTermsAggregator {
} }
@Override @Override
public void doRelease() { public void doClose() {
Releasables.release(bucketOrds, termsAggFactory, ordinalToBucket); Releasables.close(bucketOrds, termsAggFactory, ordinalToBucket);
} }
} }

View File

@ -207,14 +207,13 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
try { try {
if (termsEnum instanceof Releasable) { if (termsEnum instanceof Releasable) {
((Releasable) termsEnum).release(); ((Releasable) termsEnum).close();
} }
} finally { } finally {
termsEnum = null; termsEnum = null;
} }
return true;
} }
} }

View File

@ -150,7 +150,7 @@ public class DoubleTerms extends InternalTerms {
ordered.insertWithOverflow(b); ordered.insertWithOverflow(b);
} }
} }
buckets.release(); buckets.close();
InternalTerms.Bucket[] list = new InternalTerms.Bucket[ordered.size()]; InternalTerms.Bucket[] list = new InternalTerms.Bucket[ordered.size()];
for (int i = ordered.size() - 1; i >= 0; i--) { for (int i = ordered.size() - 1; i >= 0; i--) {
list[i] = (Bucket) ordered.pop(); list[i] = (Bucket) ordered.pop();

View File

@ -135,8 +135,8 @@ public class DoubleTermsAggregator extends BucketsAggregator {
} }
@Override @Override
public void doRelease() { public void doClose() {
Releasables.release(bucketOrds); Releasables.close(bucketOrds);
} }
} }

View File

@ -159,8 +159,8 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
} }
@Override @Override
protected void doRelease() { protected void doClose() {
Releasables.release(bucketOrds); Releasables.close(bucketOrds);
} }
} }

View File

@ -150,7 +150,7 @@ public class LongTerms extends InternalTerms {
ordered.insertWithOverflow(b); ordered.insertWithOverflow(b);
} }
} }
buckets.release(); buckets.close();
InternalTerms.Bucket[] list = new InternalTerms.Bucket[ordered.size()]; InternalTerms.Bucket[] list = new InternalTerms.Bucket[ordered.size()];
for (int i = ordered.size() - 1; i >= 0; i--) { for (int i = ordered.size() - 1; i >= 0; i--) {
list[i] = (Bucket) ordered.pop(); list[i] = (Bucket) ordered.pop();

View File

@ -135,8 +135,8 @@ public class LongTermsAggregator extends BucketsAggregator {
} }
@Override @Override
public void doRelease() { public void doClose() {
Releasables.release(bucketOrds); Releasables.close(bucketOrds);
} }
} }

View File

@ -240,8 +240,8 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
} }
@Override @Override
public void doRelease() { public void doClose() {
Releasables.release(bucketOrds); Releasables.close(bucketOrds);
} }
/** /**
@ -267,7 +267,7 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
final long maxOrd = ordinals.getMaxOrd(); final long maxOrd = ordinals.getMaxOrd();
if (ordinalToBucket == null || ordinalToBucket.size() < maxOrd) { if (ordinalToBucket == null || ordinalToBucket.size() < maxOrd) {
if (ordinalToBucket != null) { if (ordinalToBucket != null) {
ordinalToBucket.release(); ordinalToBucket.close();
} }
ordinalToBucket = context().bigArrays().newLongArray(BigArrays.overSize(maxOrd), false); ordinalToBucket = context().bigArrays().newLongArray(BigArrays.overSize(maxOrd), false);
} }
@ -298,8 +298,8 @@ public class StringTermsAggregator extends AbstractStringTermsAggregator {
} }
@Override @Override
public void doRelease() { public void doClose() {
Releasables.release(bucketOrds, ordinalToBucket); Releasables.close(bucketOrds, ordinalToBucket);
} }
} }

View File

@ -114,8 +114,8 @@ public class AvgAggregator extends MetricsAggregator.SingleValue {
} }
@Override @Override
public void doRelease() { public void doClose() {
Releasables.release(counts, sums); Releasables.close(counts, sums);
} }
} }

View File

@ -119,7 +119,7 @@ public class CardinalityAggregator extends MetricsAggregator.SingleValue {
if (collector != null) { if (collector != null) {
try { try {
collector.postCollect(); collector.postCollect();
collector.release(); collector.close();
} finally { } finally {
collector = null; collector = null;
} }
@ -154,8 +154,8 @@ public class CardinalityAggregator extends MetricsAggregator.SingleValue {
} }
@Override @Override
protected void doRelease() { protected void doClose() {
Releasables.release(counts, collector); Releasables.close(counts, collector);
} }
private static interface Collector extends Releasable { private static interface Collector extends Releasable {
@ -190,8 +190,8 @@ public class CardinalityAggregator extends MetricsAggregator.SingleValue {
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
return true; // no-op
} }
} }
@ -249,9 +249,7 @@ public class CardinalityAggregator extends MetricsAggregator.SingleValue {
} }
final org.elasticsearch.common.hash.MurmurHash3.Hash128 hash = new org.elasticsearch.common.hash.MurmurHash3.Hash128(); final org.elasticsearch.common.hash.MurmurHash3.Hash128 hash = new org.elasticsearch.common.hash.MurmurHash3.Hash128();
final LongArray hashes = bigArrays.newLongArray(maxOrd, false); try (LongArray hashes = bigArrays.newLongArray(maxOrd, false)) {
boolean success = false;
try {
for (int ord = allVisitedOrds.nextSetBit(0); ord != -1; ord = ord + 1 < maxOrd ? allVisitedOrds.nextSetBit(ord + 1) : -1) { for (int ord = allVisitedOrds.nextSetBit(0); ord != -1; ord = ord + 1 < maxOrd ? allVisitedOrds.nextSetBit(ord + 1) : -1) {
final BytesRef value = values.getValueByOrd(ord); final BytesRef value = values.getValueByOrd(ord);
org.elasticsearch.common.hash.MurmurHash3.hash128(value.bytes, value.offset, value.length, 0, hash); org.elasticsearch.common.hash.MurmurHash3.hash128(value.bytes, value.offset, value.length, 0, hash);
@ -266,16 +264,12 @@ public class CardinalityAggregator extends MetricsAggregator.SingleValue {
} }
} }
} }
success = true;
} finally {
Releasables.release(success, hashes);
} }
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
Releasables.release(visitedOrds); Releasables.close(visitedOrds);
return true;
} }
} }

View File

@ -207,7 +207,7 @@ public final class HyperLogLogPlusPlus implements Releasable {
} }
} }
} finally { } finally {
Releasables.release(values); Releasables.close(values);
} }
} else { } else {
if (algorithm.get(thisBucket) != HYPERLOGLOG) { if (algorithm.get(thisBucket) != HYPERLOGLOG) {
@ -309,7 +309,7 @@ public final class HyperLogLogPlusPlus implements Releasable {
} }
algorithm.set(bucket); algorithm.set(bucket);
} finally { } finally {
Releasables.release(values); Releasables.close(values);
} }
} }
@ -402,9 +402,8 @@ public final class HyperLogLogPlusPlus implements Releasable {
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
Releasables.release(runLens, hashSet.sizes); Releasables.close(runLens, hashSet.sizes);
return true;
} }
/** /**
@ -506,16 +505,11 @@ public final class HyperLogLogPlusPlus implements Releasable {
out.writeVInt(p); out.writeVInt(p);
if (algorithm.get(bucket) == LINEAR_COUNTING) { if (algorithm.get(bucket) == LINEAR_COUNTING) {
out.writeBoolean(LINEAR_COUNTING); out.writeBoolean(LINEAR_COUNTING);
final IntArray hashes = hashSet.values(bucket); try (IntArray hashes = hashSet.values(bucket)) {
boolean success = false;
try {
out.writeVLong(hashes.size()); out.writeVLong(hashes.size());
for (long i = 0; i < hashes.size(); ++i) { for (long i = 0; i < hashes.size(); ++i) {
out.writeInt(hashes.get(i)); out.writeInt(hashes.get(i));
} }
success = true;
} finally {
Releasables.release(success, hashes);
} }
} else { } else {
out.writeBoolean(HYPERLOGLOG); out.writeBoolean(HYPERLOGLOG);

View File

@ -115,7 +115,7 @@ public class MaxAggregator extends MetricsAggregator.SingleValue {
} }
@Override @Override
public void doRelease() { public void doClose() {
Releasables.release(maxes); Releasables.close(maxes);
} }
} }

View File

@ -114,7 +114,7 @@ public class MinAggregator extends MetricsAggregator.SingleValue {
} }
@Override @Override
public void doRelease() { public void doClose() {
Releasables.release(mins); Releasables.close(mins);
} }
} }

View File

@ -92,8 +92,8 @@ public class PercentilesAggregator extends MetricsAggregator.MultiValue {
} }
@Override @Override
protected void doRelease() { protected void doClose() {
estimator.release(); estimator.close();
} }
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {

View File

@ -47,9 +47,8 @@ public class TDigest extends PercentilesEstimator {
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
states.release(); states.close();
return true;
} }
public void offer(double value, long bucketOrd) { public void offer(double value, long bucketOrd) {

View File

@ -156,7 +156,7 @@ public class StatsAggegator extends MetricsAggregator.MultiValue {
} }
@Override @Override
public void doRelease() { public void doClose() {
Releasables.release(counts, maxes, mins, sums); Releasables.close(counts, maxes, mins, sums);
} }
} }

View File

@ -155,8 +155,8 @@ public class ExtendedStatsAggregator extends MetricsAggregator.MultiValue {
} }
@Override @Override
public void doRelease() { public void doClose() {
Releasables.release(counts, maxes, mins, sumOfSqrs, sums); Releasables.close(counts, maxes, mins, sumOfSqrs, sums);
} }
public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> { public static class Factory extends ValuesSourceAggregatorFactory.LeafOnly<ValuesSource.Numeric> {

View File

@ -109,7 +109,7 @@ public class SumAggregator extends MetricsAggregator.SingleValue {
} }
@Override @Override
public void doRelease() { public void doClose() {
Releasables.release(sums); Releasables.close(sums);
} }
} }

View File

@ -92,8 +92,8 @@ public class ValueCountAggregator extends MetricsAggregator.SingleValue {
} }
@Override @Override
public void doRelease() { public void doClose() {
Releasables.release(counts); Releasables.close(counts);
} }
public static class Factory<VS extends ValuesSource> extends ValuesSourceAggregatorFactory.LeafOnly<VS> { public static class Factory<VS extends ValuesSource> extends ValuesSourceAggregatorFactory.LeafOnly<VS> {

View File

@ -70,7 +70,7 @@ public class CountDateHistogramFacetExecutor extends FacetExecutor {
countEntries[entryIndex++] = new InternalCountDateHistogramFacet.CountEntry(keys[i], values[i]); countEntries[entryIndex++] = new InternalCountDateHistogramFacet.CountEntry(keys[i], values[i]);
} }
} }
counts.release(); counts.close();
return new InternalCountDateHistogramFacet(facetName, comparatorType, countEntries); return new InternalCountDateHistogramFacet(facetName, comparatorType, countEntries);
} }

View File

@ -156,7 +156,7 @@ public class InternalCountDateHistogramFacet extends InternalDateHistogramFacet
countEntries[entriesIndex++] = new CountEntry(keys[i], values[i]); countEntries[entriesIndex++] = new CountEntry(keys[i], values[i]);
} }
} }
counts.release(); counts.close();
Arrays.sort(countEntries, comparatorType.comparator()); Arrays.sort(countEntries, comparatorType.comparator());

View File

@ -189,7 +189,7 @@ public class InternalFullDateHistogramFacet extends InternalDateHistogramFacet {
ordered.add(value); ordered.add(value);
} }
map.release(); map.close();
// just initialize it as already ordered facet // just initialize it as already ordered facet
InternalFullDateHistogramFacet ret = new InternalFullDateHistogramFacet(getName()); InternalFullDateHistogramFacet ret = new InternalFullDateHistogramFacet(getName());

View File

@ -73,7 +73,7 @@ public class ValueDateHistogramFacetExecutor extends FacetExecutor {
} }
} }
entries.release(); entries.close();
return new InternalFullDateHistogramFacet(facetName, comparatorType, entries1); return new InternalFullDateHistogramFacet(facetName, comparatorType, entries1);
} }

View File

@ -74,7 +74,7 @@ public class ValueScriptDateHistogramFacetExecutor extends FacetExecutor {
} }
} }
entries.release(); entries.close();
return new InternalFullDateHistogramFacet(facetName, comparatorType, entries1); return new InternalFullDateHistogramFacet(facetName, comparatorType, entries1);
} }

View File

@ -67,7 +67,7 @@ public class CountHistogramFacetExecutor extends FacetExecutor {
entries[entryIndex++] = new InternalCountHistogramFacet.CountEntry(keys[i], values[i]); entries[entryIndex++] = new InternalCountHistogramFacet.CountEntry(keys[i], values[i]);
} }
} }
counts.release(); counts.close();
return new InternalCountHistogramFacet(facetName, comparatorType, entries); return new InternalCountHistogramFacet(facetName, comparatorType, entries);
} }

View File

@ -68,7 +68,7 @@ public class FullHistogramFacetExecutor extends FacetExecutor {
fullEntries.add((InternalFullHistogramFacet.FullEntry) values[i]); fullEntries.add((InternalFullHistogramFacet.FullEntry) values[i]);
} }
} }
entries.release(); entries.close();
return new InternalFullHistogramFacet(facetName, comparatorType, fullEntries); return new InternalFullHistogramFacet(facetName, comparatorType, fullEntries);
} }

View File

@ -155,7 +155,7 @@ public class InternalCountHistogramFacet extends InternalHistogramFacet {
entries[entryIndex++] = new CountEntry(keys[i], values[i]); entries[entryIndex++] = new CountEntry(keys[i], values[i]);
} }
} }
counts.release(); counts.close();
Arrays.sort(entries, comparatorType.comparator()); Arrays.sort(entries, comparatorType.comparator());

View File

@ -186,7 +186,7 @@ public class InternalFullHistogramFacet extends InternalHistogramFacet {
ordered.add(value); ordered.add(value);
} }
map.release(); map.close();
// just initialize it as already ordered facet // just initialize it as already ordered facet
InternalFullHistogramFacet ret = new InternalFullHistogramFacet(getName()); InternalFullHistogramFacet ret = new InternalFullHistogramFacet(getName());

View File

@ -71,7 +71,7 @@ public class ScriptHistogramFacetExecutor extends FacetExecutor {
} }
} }
entries.release(); entries.close();
return new InternalFullHistogramFacet(facetName, comparatorType, entries1); return new InternalFullHistogramFacet(facetName, comparatorType, entries1);
} }

View File

@ -70,7 +70,7 @@ public class ValueHistogramFacetExecutor extends FacetExecutor {
entries1.add(value); entries1.add(value);
} }
} }
entries.release(); entries.close();
return new InternalFullHistogramFacet(facetName, comparatorType, entries1); return new InternalFullHistogramFacet(facetName, comparatorType, entries1);
} }

View File

@ -75,7 +75,7 @@ public class ValueScriptHistogramFacetExecutor extends FacetExecutor {
} }
} }
entries.release(); entries.close();
return new InternalFullHistogramFacet(facetName, comparatorType, entries1); return new InternalFullHistogramFacet(facetName, comparatorType, entries1);
} }

View File

@ -201,7 +201,7 @@ public class InternalDoubleTermsFacet extends InternalTermsFacet {
first.missing = missing; first.missing = missing;
first.total = total; first.total = total;
aggregated.release(); aggregated.close();
return first; return first;
} }

View File

@ -101,7 +101,7 @@ public class TermsDoubleFacetExecutor extends FacetExecutor {
@Override @Override
public InternalFacet buildFacet(String facetName) { public InternalFacet buildFacet(String facetName) {
if (facets.v().isEmpty()) { if (facets.v().isEmpty()) {
facets.release(); facets.close();
return new InternalDoubleTermsFacet(facetName, comparatorType, size, ImmutableList.<InternalDoubleTermsFacet.DoubleEntry>of(), missing, total); return new InternalDoubleTermsFacet(facetName, comparatorType, size, ImmutableList.<InternalDoubleTermsFacet.DoubleEntry>of(), missing, total);
} else { } else {
final boolean[] states = facets.v().allocated; final boolean[] states = facets.v().allocated;
@ -118,7 +118,7 @@ public class TermsDoubleFacetExecutor extends FacetExecutor {
for (int i = ordered.size() - 1; i >= 0; i--) { for (int i = ordered.size() - 1; i >= 0; i--) {
list[i] = (InternalDoubleTermsFacet.DoubleEntry) ordered.pop(); list[i] = (InternalDoubleTermsFacet.DoubleEntry) ordered.pop();
} }
facets.release(); facets.close();
return new InternalDoubleTermsFacet(facetName, comparatorType, size, Arrays.asList(list), missing, total); return new InternalDoubleTermsFacet(facetName, comparatorType, size, Arrays.asList(list), missing, total);
} else { } else {
BoundedTreeSet<InternalDoubleTermsFacet.DoubleEntry> ordered = new BoundedTreeSet<>(comparatorType.comparator(), shardSize); BoundedTreeSet<InternalDoubleTermsFacet.DoubleEntry> ordered = new BoundedTreeSet<>(comparatorType.comparator(), shardSize);
@ -127,7 +127,7 @@ public class TermsDoubleFacetExecutor extends FacetExecutor {
ordered.add(new InternalDoubleTermsFacet.DoubleEntry(keys[i], values[i])); ordered.add(new InternalDoubleTermsFacet.DoubleEntry(keys[i], values[i]));
} }
} }
facets.release(); facets.close();
return new InternalDoubleTermsFacet(facetName, comparatorType, size, ordered, missing, total); return new InternalDoubleTermsFacet(facetName, comparatorType, size, ordered, missing, total);
} }
} }

View File

@ -202,7 +202,7 @@ public class InternalLongTermsFacet extends InternalTermsFacet {
first.missing = missing; first.missing = missing;
first.total = total; first.total = total;
aggregated.release(); aggregated.close();
return first; return first;
} }

View File

@ -100,7 +100,7 @@ public class TermsLongFacetExecutor extends FacetExecutor {
@Override @Override
public InternalFacet buildFacet(String facetName) { public InternalFacet buildFacet(String facetName) {
if (facets.v().isEmpty()) { if (facets.v().isEmpty()) {
facets.release(); facets.close();
return new InternalLongTermsFacet(facetName, comparatorType, size, ImmutableList.<InternalLongTermsFacet.LongEntry>of(), missing, total); return new InternalLongTermsFacet(facetName, comparatorType, size, ImmutableList.<InternalLongTermsFacet.LongEntry>of(), missing, total);
} else { } else {
LongIntOpenHashMap facetEntries = facets.v(); LongIntOpenHashMap facetEntries = facets.v();
@ -118,7 +118,7 @@ public class TermsLongFacetExecutor extends FacetExecutor {
for (int i = ordered.size() - 1; i >= 0; i--) { for (int i = ordered.size() - 1; i >= 0; i--) {
list[i] = (InternalLongTermsFacet.LongEntry) ordered.pop(); list[i] = (InternalLongTermsFacet.LongEntry) ordered.pop();
} }
facets.release(); facets.close();
return new InternalLongTermsFacet(facetName, comparatorType, size, Arrays.asList(list), missing, total); return new InternalLongTermsFacet(facetName, comparatorType, size, Arrays.asList(list), missing, total);
} else { } else {
BoundedTreeSet<InternalLongTermsFacet.LongEntry> ordered = new BoundedTreeSet<>(comparatorType.comparator(), shardSize); BoundedTreeSet<InternalLongTermsFacet.LongEntry> ordered = new BoundedTreeSet<>(comparatorType.comparator(), shardSize);
@ -127,7 +127,7 @@ public class TermsLongFacetExecutor extends FacetExecutor {
ordered.add(new InternalLongTermsFacet.LongEntry(keys[i], values[i])); ordered.add(new InternalLongTermsFacet.LongEntry(keys[i], values[i]));
} }
} }
facets.release(); facets.close();
return new InternalLongTermsFacet(facetName, comparatorType, size, ordered, missing, total); return new InternalLongTermsFacet(facetName, comparatorType, size, ordered, missing, total);
} }
} }

View File

@ -218,7 +218,7 @@ public class InternalStringTermsFacet extends InternalTermsFacet {
first.missing = missing; first.missing = missing;
first.total = total; first.total = total;
aggregated.release(); aggregated.close();
return first; return first;
} }

View File

@ -80,7 +80,7 @@ public class ScriptTermsStringFieldFacetExecutor extends FacetExecutor {
@Override @Override
public InternalFacet buildFacet(String facetName) { public InternalFacet buildFacet(String facetName) {
if (facets.v().isEmpty()) { if (facets.v().isEmpty()) {
facets.release(); facets.close();
return new InternalStringTermsFacet(facetName, comparatorType, size, ImmutableList.<InternalStringTermsFacet.TermEntry>of(), missing, total); return new InternalStringTermsFacet(facetName, comparatorType, size, ImmutableList.<InternalStringTermsFacet.TermEntry>of(), missing, total);
} else { } else {
final boolean[] states = facets.v().allocated; final boolean[] states = facets.v().allocated;
@ -98,7 +98,7 @@ public class ScriptTermsStringFieldFacetExecutor extends FacetExecutor {
for (int i = ordered.size() - 1; i >= 0; i--) { for (int i = ordered.size() - 1; i >= 0; i--) {
list[i] = ((InternalStringTermsFacet.TermEntry) ordered.pop()); list[i] = ((InternalStringTermsFacet.TermEntry) ordered.pop());
} }
facets.release(); facets.close();
return new InternalStringTermsFacet(facetName, comparatorType, size, Arrays.asList(list), missing, total); return new InternalStringTermsFacet(facetName, comparatorType, size, Arrays.asList(list), missing, total);
} else { } else {
BoundedTreeSet<InternalStringTermsFacet.TermEntry> ordered = new BoundedTreeSet<>(comparatorType.comparator(), shardSize); BoundedTreeSet<InternalStringTermsFacet.TermEntry> ordered = new BoundedTreeSet<>(comparatorType.comparator(), shardSize);
@ -108,7 +108,7 @@ public class ScriptTermsStringFieldFacetExecutor extends FacetExecutor {
ordered.add(new InternalStringTermsFacet.TermEntry(key, values[i])); ordered.add(new InternalStringTermsFacet.TermEntry(key, values[i]));
} }
} }
facets.release(); facets.close();
return new InternalStringTermsFacet(facetName, comparatorType, size, ordered, missing, total); return new InternalStringTermsFacet(facetName, comparatorType, size, ordered, missing, total);
} }
} }

View File

@ -151,7 +151,7 @@ public class TermsStringOrdinalsFacetExecutor extends FacetExecutor {
list[i] = (InternalStringTermsFacet.TermEntry) ordered.pop(); list[i] = (InternalStringTermsFacet.TermEntry) ordered.pop();
} }
Releasables.release(aggregators); Releasables.close(aggregators);
return new InternalStringTermsFacet(facetName, comparatorType, size, Arrays.asList(list), missing, total); return new InternalStringTermsFacet(facetName, comparatorType, size, Arrays.asList(list), missing, total);
} }
@ -189,7 +189,7 @@ public class TermsStringOrdinalsFacetExecutor extends FacetExecutor {
} }
} }
Releasables.release(aggregators); Releasables.close(aggregators);
return new InternalStringTermsFacet(facetName, comparatorType, size, ordered, missing, total); return new InternalStringTermsFacet(facetName, comparatorType, size, ordered, missing, total);
} }
@ -210,7 +210,7 @@ public class TermsStringOrdinalsFacetExecutor extends FacetExecutor {
if (current.values.ordinals().getNumOrds() > 0) { if (current.values.ordinals().getNumOrds() > 0) {
aggregators.add(current); aggregators.add(current);
} else { } else {
Releasables.release(current); Releasables.close(current);
} }
} }
values = indexFieldData.load(context).getBytesValues(false); values = indexFieldData.load(context).getBytesValues(false);
@ -238,7 +238,7 @@ public class TermsStringOrdinalsFacetExecutor extends FacetExecutor {
if (current.values.ordinals().getNumOrds() > 0) { if (current.values.ordinals().getNumOrds() > 0) {
aggregators.add(current); aggregators.add(current);
} else { } else {
Releasables.release(current); Releasables.close(current);
} }
current = null; current = null;
} }
@ -287,9 +287,8 @@ public class TermsStringOrdinalsFacetExecutor extends FacetExecutor {
} }
@Override @Override
public boolean release() { public void close() {
Releasables.release(counts); Releasables.close(counts);
return true;
} }
} }

View File

@ -211,7 +211,7 @@ public class InternalTermsStatsDoubleFacet extends InternalTermsStatsFacet {
if (requiredSize == 0) { // all terms if (requiredSize == 0) { // all terms
DoubleEntry[] entries1 = map.v().values().toArray(DoubleEntry.class); DoubleEntry[] entries1 = map.v().values().toArray(DoubleEntry.class);
Arrays.sort(entries1, comparatorType.comparator()); Arrays.sort(entries1, comparatorType.comparator());
map.release(); map.close();
return new InternalTermsStatsDoubleFacet(getName(), comparatorType, requiredSize, Arrays.asList(entries1), missing); return new InternalTermsStatsDoubleFacet(getName(), comparatorType, requiredSize, Arrays.asList(entries1), missing);
} else { } else {
Object[] values = map.v().values; Object[] values = map.v().values;
@ -224,7 +224,7 @@ public class InternalTermsStatsDoubleFacet extends InternalTermsStatsFacet {
} }
ordered.add(value); ordered.add(value);
} }
map.release(); map.close();
return new InternalTermsStatsDoubleFacet(getName(), comparatorType, requiredSize, ordered, missing); return new InternalTermsStatsDoubleFacet(getName(), comparatorType, requiredSize, ordered, missing);
} }
} }

View File

@ -74,7 +74,7 @@ public class TermsStatsDoubleFacetExecutor extends FacetExecutor {
@Override @Override
public InternalFacet buildFacet(String facetName) { public InternalFacet buildFacet(String facetName) {
if (entries.v().isEmpty()) { if (entries.v().isEmpty()) {
entries.release(); entries.close();
return new InternalTermsStatsDoubleFacet(facetName, comparatorType, size, ImmutableList.<InternalTermsStatsDoubleFacet.DoubleEntry>of(), missing); return new InternalTermsStatsDoubleFacet(facetName, comparatorType, size, ImmutableList.<InternalTermsStatsDoubleFacet.DoubleEntry>of(), missing);
} }
if (size == 0) { // all terms if (size == 0) { // all terms
@ -87,7 +87,7 @@ public class TermsStatsDoubleFacetExecutor extends FacetExecutor {
doubleEntries.add((InternalTermsStatsDoubleFacet.DoubleEntry) values[i]); doubleEntries.add((InternalTermsStatsDoubleFacet.DoubleEntry) values[i]);
} }
} }
entries.release(); entries.close();
return new InternalTermsStatsDoubleFacet(facetName, comparatorType, 0 /* indicates all terms*/, doubleEntries, missing); return new InternalTermsStatsDoubleFacet(facetName, comparatorType, 0 /* indicates all terms*/, doubleEntries, missing);
} }
Object[] values = entries.v().values; Object[] values = entries.v().values;
@ -103,7 +103,7 @@ public class TermsStatsDoubleFacetExecutor extends FacetExecutor {
ordered.add(value); ordered.add(value);
} }
entries.release(); entries.close();
return new InternalTermsStatsDoubleFacet(facetName, comparatorType, size, ordered, missing); return new InternalTermsStatsDoubleFacet(facetName, comparatorType, size, ordered, missing);
} }

View File

@ -211,7 +211,7 @@ public class InternalTermsStatsLongFacet extends InternalTermsStatsFacet {
if (requiredSize == 0) { // all terms if (requiredSize == 0) { // all terms
LongEntry[] entries1 = map.v().values().toArray(LongEntry.class); LongEntry[] entries1 = map.v().values().toArray(LongEntry.class);
Arrays.sort(entries1, comparatorType.comparator()); Arrays.sort(entries1, comparatorType.comparator());
map.release(); map.close();
return new InternalTermsStatsLongFacet(getName(), comparatorType, requiredSize, Arrays.asList(entries1), missing); return new InternalTermsStatsLongFacet(getName(), comparatorType, requiredSize, Arrays.asList(entries1), missing);
} else { } else {
Object[] values = map.v().values; Object[] values = map.v().values;
@ -224,7 +224,7 @@ public class InternalTermsStatsLongFacet extends InternalTermsStatsFacet {
} }
ordered.add(value); ordered.add(value);
} }
map.release(); map.close();
return new InternalTermsStatsLongFacet(getName(), comparatorType, requiredSize, ordered, missing); return new InternalTermsStatsLongFacet(getName(), comparatorType, requiredSize, ordered, missing);
} }
} }

View File

@ -75,7 +75,7 @@ public class TermsStatsLongFacetExecutor extends FacetExecutor {
@Override @Override
public InternalFacet buildFacet(String facetName) { public InternalFacet buildFacet(String facetName) {
if (entries.v().isEmpty()) { if (entries.v().isEmpty()) {
entries.release(); entries.close();
return new InternalTermsStatsLongFacet(facetName, comparatorType, size, ImmutableList.<InternalTermsStatsLongFacet.LongEntry>of(), missing); return new InternalTermsStatsLongFacet(facetName, comparatorType, size, ImmutableList.<InternalTermsStatsLongFacet.LongEntry>of(), missing);
} }
if (size == 0) { // all terms if (size == 0) { // all terms
@ -89,7 +89,7 @@ public class TermsStatsLongFacetExecutor extends FacetExecutor {
} }
} }
entries.release(); entries.close();
return new InternalTermsStatsLongFacet(facetName, comparatorType, 0 /* indicates all terms*/, longEntries, missing); return new InternalTermsStatsLongFacet(facetName, comparatorType, 0 /* indicates all terms*/, longEntries, missing);
} }
@ -106,7 +106,7 @@ public class TermsStatsLongFacetExecutor extends FacetExecutor {
} }
ordered.add(value); ordered.add(value);
} }
entries.release(); entries.close();
return new InternalTermsStatsLongFacet(facetName, comparatorType, size, ordered, missing); return new InternalTermsStatsLongFacet(facetName, comparatorType, size, ordered, missing);
} }

View File

@ -216,7 +216,7 @@ public class InternalTermsStatsStringFacet extends InternalTermsStatsFacet {
if (requiredSize == 0) { // all terms if (requiredSize == 0) { // all terms
StringEntry[] entries1 = map.v().values().toArray(StringEntry.class); StringEntry[] entries1 = map.v().values().toArray(StringEntry.class);
Arrays.sort(entries1, comparatorType.comparator()); Arrays.sort(entries1, comparatorType.comparator());
map.release(); map.close();
return new InternalTermsStatsStringFacet(getName(), comparatorType, requiredSize, Arrays.asList(entries1), missing); return new InternalTermsStatsStringFacet(getName(), comparatorType, requiredSize, Arrays.asList(entries1), missing);
} else { } else {
Object[] values = map.v().values; Object[] values = map.v().values;
@ -229,7 +229,7 @@ public class InternalTermsStatsStringFacet extends InternalTermsStatsFacet {
} }
ordered.add(value); ordered.add(value);
} }
map.release(); map.close();
return new InternalTermsStatsStringFacet(getName(), comparatorType, requiredSize, ordered, missing); return new InternalTermsStatsStringFacet(getName(), comparatorType, requiredSize, ordered, missing);
} }
} }

View File

@ -77,7 +77,7 @@ public class TermsStatsStringFacetExecutor extends FacetExecutor {
@Override @Override
public InternalFacet buildFacet(String facetName) { public InternalFacet buildFacet(String facetName) {
if (entries.v().isEmpty()) { if (entries.v().isEmpty()) {
entries.release(); entries.close();
return new InternalTermsStatsStringFacet(facetName, comparatorType, size, ImmutableList.<InternalTermsStatsStringFacet.StringEntry>of(), missing); return new InternalTermsStatsStringFacet(facetName, comparatorType, size, ImmutableList.<InternalTermsStatsStringFacet.StringEntry>of(), missing);
} }
if (size == 0) { // all terms if (size == 0) { // all terms
@ -105,7 +105,7 @@ public class TermsStatsStringFacetExecutor extends FacetExecutor {
ordered.add(value); ordered.add(value);
} }
entries.release(); entries.close();
return new InternalTermsStatsStringFacet(facetName, comparatorType, size, ordered, missing); return new InternalTermsStatsStringFacet(facetName, comparatorType, size, ordered, missing);
} }

View File

@ -207,19 +207,18 @@ public class DefaultSearchContext extends SearchContext {
} }
@Override @Override
public boolean release() throws ElasticsearchException { public void close() throws ElasticsearchException {
if (scanContext != null) { if (scanContext != null) {
scanContext.clear(); scanContext.clear();
} }
// clear and scope phase we have // clear and scope phase we have
searcher.release(); searcher.release();
engineSearcher.release(); engineSearcher.close();
return true;
} }
public boolean clearAndRelease() { public void clearAndRelease() {
clearReleasables(); clearReleasables();
return release(); close();
} }
/** /**
@ -690,7 +689,7 @@ public class DefaultSearchContext extends SearchContext {
public void clearReleasables() { public void clearReleasables() {
if (clearables != null) { if (clearables != null) {
try { try {
Releasables.release(clearables); Releasables.close(clearables);
} finally { } finally {
clearables.clear(); clearables.clear();
} }

Some files were not shown because too many files have changed in this diff Show More