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.deletedDocs = searcher.reader().numDeletedDocs();
} finally {
searcher.release();
searcher.close();
}
shardStatus.mergeStats = indexShard.mergeScheduler().stats();

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -22,9 +22,9 @@ package org.elasticsearch.common.lease;
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;
import org.elasticsearch.ElasticsearchException;
import java.util.Arrays;
/** Utility methods to work with {@link Releasable}s. */
@ -35,12 +37,12 @@ public enum Releasables {
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;
for (Releasable releasable : releasables) {
if (releasable != null) {
try {
releasable.release();
releasable.close();
} catch (Throwable t) {
if (th == null) {
th = t;
@ -54,36 +56,70 @@ public enum Releasables {
}
/** Release the provided {@link Releasable}s. */
public static void release(Iterable<? extends Releasable> releasables) {
release(releasables, false);
public static void close(Iterable<? extends Releasable> releasables) {
close(releasables, false);
}
/** Release the provided {@link Releasable}s. */
public static void release(Releasable... releasables) {
release(Arrays.asList(releasables));
public static void close(Releasable... releasables) {
close(Arrays.asList(releasables));
}
/** Release the provided {@link Releasable}s, ignoring exceptions. */
public static void releaseWhileHandlingException(Iterable<Releasable> releasables) {
release(releasables, true);
public static void closeWhileHandlingException(Iterable<Releasable> releasables) {
close(releasables, true);
}
/** Release the provided {@link Releasable}s, ignoring exceptions. */
public static void releaseWhileHandlingException(Releasable... releasables) {
releaseWhileHandlingException(Arrays.asList(releasables));
public static void closeWhileHandlingException(Releasable... releasables) {
closeWhileHandlingException(Arrays.asList(releasables));
}
/** 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) {
release(releasables);
close(releasables);
} else {
releaseWhileHandlingException(releasables);
closeWhileHandlingException(releasables);
}
}
/** Release the provided {@link Releasable}s, ignoring exceptions if <code>success</code> is <tt>false</tt>. */
public static void release(boolean success, Releasable... releasables) {
release(success, Arrays.asList(releasables));
public static void close(boolean success, Releasable... 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
public boolean release() throws ElasticsearchException {
public void close() throws ElasticsearchException {
try {
Releasables.release(cachedTermOrds, termDocFreqs, termsTotalFreqs);
Releasables.close(cachedTermOrds, termDocFreqs, termsTotalFreqs);
} finally {
cachedTermOrds = null;
termDocFreqs = null;
termsTotalFreqs = null;
}
return true;
}
}

View File

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

View File

@ -58,12 +58,11 @@ public class NoneRecycler<T> extends AbstractRecycler<T> {
}
@Override
public boolean release() {
public void close() {
if (value == null) {
throw new ElasticsearchIllegalStateException("recycler entry already released...");
}
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.
*/
public static <T> Recycler<T> locked(final Recycler<T> recycler) {
@ -167,9 +167,9 @@ public enum Recyclers {
return new Recycler.V<T>() {
@Override
public boolean release() throws ElasticsearchException {
public void close() throws ElasticsearchException {
synchronized (lock) {
return delegate.release();
delegate.close();
}
}

View File

@ -31,10 +31,9 @@ abstract class AbstractArray implements Releasable {
}
@Override
public boolean release() {
public void close() {
assert !released : "double release";
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) {
if (recycler != null) {
cache[page].release();
cache[page].close();
cache[page] = null;
}
}
@Override
public final boolean release() {
super.release();
public final void close() {
super.close();
if (recycler != null) {
Releasables.release(cache);
Releasables.close(cache);
cache = null;
}
return true;
}
}

View File

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

View File

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

View File

@ -21,6 +21,7 @@ package org.elasticsearch.common.util;
import com.carrotsearch.hppc.hash.MurmurHash3;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.lease.Releasable;
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.
* <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) {
final long startOffset = startOffsets.get(id);
@ -159,15 +160,10 @@ public final class BytesRefHash extends AbstractHash {
}
@Override
public boolean release() {
boolean success = false;
try {
super.release();
success = true;
} finally {
Releasables.release(success, bytes, hashes, startOffsets);
public void close() {
try (Releasable releasable = Releasables.wrap(bytes, hashes, startOffsets)) {
super.close();
}
return true;
}
}

View File

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

View File

@ -19,7 +19,7 @@
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
@ -119,15 +119,10 @@ public final class LongHash extends AbstractHash {
}
@Override
public boolean release() {
boolean success = false;
try {
super.release();
success = true;
} finally {
Releasables.release(success, keys);
public void close() {
try (Releasable releasable = keys) {
super.close();
}
return true;
}
}

View File

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

View File

@ -170,7 +170,7 @@ public class NettyHttpChannel extends HttpChannel {
}
} finally {
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.name.Named;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.IndexShardComponent;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.ConcurrentHashMap;
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
* {@link SnapshotIndexCommits#release()} to release it.
* {@link SnapshotIndexCommits#close()} to release it.
*/
public SnapshotIndexCommits snapshots() throws IOException {
synchronized (mutex) {
@ -122,7 +120,7 @@ public class SnapshotDeletionPolicy extends AbstractESDeletionPolicy {
/**
* 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 {
synchronized (mutex) {
@ -164,7 +162,7 @@ public class SnapshotDeletionPolicy extends AbstractESDeletionPolicy {
/**
* Releases the version provided. Returns <tt>true</tt> if the release was successful.
*/
boolean release(long version) {
boolean close(long version) {
synchronized (mutex) {
SnapshotDeletionPolicy.SnapshotHolder holder = snapshots.get(version);
if (holder == null) {
@ -193,12 +191,12 @@ public class SnapshotDeletionPolicy extends AbstractESDeletionPolicy {
}
@Override
public boolean release() {
public void close() {
if (released) {
return false;
return;
}
released = true;
return ((SnapshotIndexCommit) delegate).release();
((SnapshotIndexCommit) delegate).close();
}
}

View File

@ -27,7 +27,7 @@ import java.io.IOException;
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
* ({@link #getFileNames()}).
*
@ -57,8 +57,8 @@ public class SnapshotIndexCommit extends IndexCommitDelegate implements Releasab
* Releases the current snapshot, returning <code>true</code> if it was
* actually released.
*/
public boolean release() {
return deletionPolicy.release(getGeneration());
public void close() {
deletionPolicy.close(getGeneration());
}
/**

View File

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

View File

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

View File

@ -349,14 +349,14 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
try {
docIdAndVersion = Versions.loadDocIdAndVersion(searcher.reader(), get.uid());
} catch (Throwable e) {
Releasables.releaseWhileHandlingException(searcher);
Releasables.closeWhileHandlingException(searcher);
//TODO: A better exception goes here
throw new EngineException(shardId(), "Couldn't resolve version", e);
}
if (get.version() != Versions.MATCH_ANY && docIdAndVersion != null) {
if (get.versionType().isVersionConflict(docIdAndVersion.version, get.version())) {
Releasables.release(searcher);
Releasables.close(searcher);
Uid uid = Uid.createUid(get.uid().text());
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
return new GetResult(searcher, docIdAndVersion);
} else {
Releasables.release(searcher);
Releasables.close(searcher);
return GetResult.NOT_EXISTS;
}
@ -1053,14 +1053,14 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
try {
phase1Snapshot = deletionPolicy.snapshot();
} catch (Throwable e) {
Releasables.releaseWhileHandlingException(onGoingRecoveries);
Releasables.closeWhileHandlingException(onGoingRecoveries);
throw new RecoveryEngineException(shardId, 1, "Snapshot failed", e);
}
try {
recoveryHandler.phase1(phase1Snapshot);
} catch (Throwable e) {
Releasables.releaseWhileHandlingException(onGoingRecoveries, phase1Snapshot);
Releasables.closeWhileHandlingException(onGoingRecoveries, phase1Snapshot);
if (closed) {
e = new EngineClosedException(shardId, e);
}
@ -1071,7 +1071,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
try {
phase2Snapshot = translog.snapshot();
} catch (Throwable e) {
Releasables.releaseWhileHandlingException(onGoingRecoveries, phase1Snapshot);
Releasables.closeWhileHandlingException(onGoingRecoveries, phase1Snapshot);
if (closed) {
e = new EngineClosedException(shardId, e);
}
@ -1081,7 +1081,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
try {
recoveryHandler.phase2(phase2Snapshot);
} catch (Throwable e) {
Releasables.releaseWhileHandlingException(onGoingRecoveries, phase1Snapshot, phase2Snapshot);
Releasables.closeWhileHandlingException(onGoingRecoveries, phase1Snapshot, phase2Snapshot);
if (closed) {
e = new EngineClosedException(shardId, e);
}
@ -1098,9 +1098,9 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
} catch (Throwable e) {
throw new RecoveryEngineException(shardId, 3, "Execution failed", e);
} finally {
Releasables.release(success, onGoingRecoveries);
Releasables.close(success, onGoingRecoveries);
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;
} finally {
searcher.release();
searcher.close();
}
} finally {
rwl.readLock().unlock();
@ -1158,7 +1158,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
segments.put(info.info.name, segment);
}
} finally {
searcher.release();
searcher.close();
}
// now, correlate or add the committed ones...
@ -1292,7 +1292,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
try {
return Versions.loadVersion(searcher.reader(), uid);
} finally {
searcher.release();
searcher.close();
}
}
@ -1465,25 +1465,23 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
}
@Override
public boolean release() throws ElasticsearchException {
public void close() throws ElasticsearchException {
if (!released.compareAndSet(false, true)) {
/* 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
* 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"));
return false;
return;
}
try {
manager.release(searcher);
return true;
} catch (IOException e) {
return false;
throw new ElasticsearchIllegalStateException("Cannot close", e);
} catch (AlreadyClosedException e) {
/* this one can happen if we already closed the
* underlying store / directory and we call into the
* IndexWriter to free up pending files. */
return false;
} finally {
store.decRef();
}
@ -1575,7 +1573,7 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
}
} finally {
// no need to release the fullSearcher, nothing really is done...
Releasables.release(currentSearcher);
Releasables.close(currentSearcher);
if (newSearcher != null && closeNewSearcher) {
IOUtils.closeWhileHandlingException(newSearcher.getIndexReader()); // ignore
}
@ -1604,9 +1602,8 @@ public class InternalEngine extends AbstractIndexShardComponent implements Engin
}
@Override
public boolean release() throws ElasticsearchException {
public void close() throws ElasticsearchException {
endRecovery();
return true;
}
}

View File

@ -271,7 +271,7 @@ public class PercolatorQueriesRegistry extends AbstractIndexShardComponent {
shardPercolateService.addedQuery(entry.getKey(), previousQuery, entry.getValue());
}
} finally {
searcher.release();
searcher.close();
}
} catch (Exception 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;
} finally {
if (releaseParentIds) {
Releasables.release(parentIds);
Releasables.close(parentIds);
}
}
@ -209,9 +209,8 @@ public class ChildrenConstantScoreQuery extends Query {
}
@Override
public boolean release() throws ElasticsearchException {
Releasables.release(parentIds);
return true;
public void close() throws ElasticsearchException {
Releasables.close(parentIds);
}
private final class ParentDocIdIterator extends FilteredDocIdSetIterator {

View File

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

View File

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

View File

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

View File

@ -153,7 +153,7 @@ public class ParentQuery extends Query {
} finally {
if (releaseCollectorResource) {
// 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);
@ -272,9 +272,8 @@ public class ParentQuery extends Query {
}
@Override
public boolean release() throws ElasticsearchException {
Releasables.release(parentIds, scores, parentIdsIndexCache);
return true;
public void close() throws ElasticsearchException {
Releasables.close(parentIds, scores, parentIdsIndexCache);
}
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -120,14 +120,13 @@ class MultiDocumentPercolatorIndex implements PercolatorIndex {
}
@Override
public boolean release() throws ElasticsearchException {
public void close() throws ElasticsearchException {
try {
searcher.getIndexReader().close();
rootDocMemoryIndex.reset();
} catch (IOException 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.PageCacheRecycler;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.lucene.HashedBytesRef;
import org.elasticsearch.common.text.StringText;
import org.elasticsearch.common.util.BigArrays;
@ -207,18 +208,13 @@ public class PercolateContext extends SearchContext {
}
@Override
public boolean release() throws ElasticsearchException {
try {
public void close() throws ElasticsearchException {
try (Releasable releasable = Releasables.wrap(engineSearcher, docSearcher)) {
if (docSearcher != null) {
IndexReader indexReader = docSearcher.reader();
fieldDataService.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:
@Override
public boolean clearAndRelease() {
public void clearAndRelease() {
throw new UnsupportedOperationException();
}

View File

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

View File

@ -90,14 +90,13 @@ class SingleDocumentPercolatorIndex implements PercolatorIndex {
}
@Override
public boolean release() throws ElasticsearchException {
public void close() throws ElasticsearchException {
try {
searcher.getIndexReader().close();
memoryIndex.reset();
} catch (IOException 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);
} catch (Throwable e) {
context.release();
context.close();
throw ExceptionsHelper.convertToRuntime(e);
}
@ -546,7 +546,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> {
return;
}
context.indexShard().searchService().onFreeContext(context);
context.release();
context.close();
}
private void freeContext(SearchContext context) {
@ -554,7 +554,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> {
if (removed != null) {
removed.indexShard().searchService().onFreeContext(removed);
}
context.release();
context.close();
}
public void freeAllScrollContexts() {

View File

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

View File

@ -134,7 +134,7 @@ public class AggregatorFactories {
}
@Override
public void doRelease() {
public void doClose() {
final Iterable<Aggregator> aggregatorsIter = new Iterable<Aggregator>() {
@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;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.util.LongArray;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
@ -129,15 +129,10 @@ public abstract class BucketsAggregator extends Aggregator {
}
@Override
public final boolean release() {
boolean success = false;
try {
super.release();
success = true;
} finally {
Releasables.release(success, docCounts);
public final void close() {
try (Releasable releasable = docCounts) {
super.close();
}
return true;
}
}

View File

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

View File

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

View File

@ -129,8 +129,8 @@ public class HistogramAggregator extends BucketsAggregator {
}
@Override
public void doRelease() {
Releasables.release(bucketOrds);
public void doClose() {
Releasables.close(bucketOrds);
}
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);
}
}
bucketsByKey.release();
bucketsByKey.close();
// adding empty buckets in needed
if (minDocCount == 0) {

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -119,7 +119,7 @@ public class CardinalityAggregator extends MetricsAggregator.SingleValue {
if (collector != null) {
try {
collector.postCollect();
collector.release();
collector.close();
} finally {
collector = null;
}
@ -154,8 +154,8 @@ public class CardinalityAggregator extends MetricsAggregator.SingleValue {
}
@Override
protected void doRelease() {
Releasables.release(counts, collector);
protected void doClose() {
Releasables.close(counts, collector);
}
private static interface Collector extends Releasable {
@ -190,8 +190,8 @@ public class CardinalityAggregator extends MetricsAggregator.SingleValue {
}
@Override
public boolean release() throws ElasticsearchException {
return true;
public void close() throws ElasticsearchException {
// 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 LongArray hashes = bigArrays.newLongArray(maxOrd, false);
boolean success = false;
try {
try (LongArray hashes = bigArrays.newLongArray(maxOrd, false)) {
for (int ord = allVisitedOrds.nextSetBit(0); ord != -1; ord = ord + 1 < maxOrd ? allVisitedOrds.nextSetBit(ord + 1) : -1) {
final BytesRef value = values.getValueByOrd(ord);
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
public boolean release() throws ElasticsearchException {
Releasables.release(visitedOrds);
return true;
public void close() throws ElasticsearchException {
Releasables.close(visitedOrds);
}
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -92,8 +92,8 @@ public class ValueCountAggregator extends MetricsAggregator.SingleValue {
}
@Override
public void doRelease() {
Releasables.release(counts);
public void doClose() {
Releasables.close(counts);
}
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]);
}
}
counts.release();
counts.close();
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]);
}
}
counts.release();
counts.close();
Arrays.sort(countEntries, comparatorType.comparator());

View File

@ -189,7 +189,7 @@ public class InternalFullDateHistogramFacet extends InternalDateHistogramFacet {
ordered.add(value);
}
map.release();
map.close();
// just initialize it as already ordered facet
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);
}

View File

@ -74,7 +74,7 @@ public class ValueScriptDateHistogramFacetExecutor extends FacetExecutor {
}
}
entries.release();
entries.close();
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]);
}
}
counts.release();
counts.close();
return new InternalCountHistogramFacet(facetName, comparatorType, entries);
}

View File

@ -68,7 +68,7 @@ public class FullHistogramFacetExecutor extends FacetExecutor {
fullEntries.add((InternalFullHistogramFacet.FullEntry) values[i]);
}
}
entries.release();
entries.close();
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]);
}
}
counts.release();
counts.close();
Arrays.sort(entries, comparatorType.comparator());

View File

@ -186,7 +186,7 @@ public class InternalFullHistogramFacet extends InternalHistogramFacet {
ordered.add(value);
}
map.release();
map.close();
// just initialize it as already ordered facet
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);
}

View File

@ -70,7 +70,7 @@ public class ValueHistogramFacetExecutor extends FacetExecutor {
entries1.add(value);
}
}
entries.release();
entries.close();
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);
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -80,7 +80,7 @@ public class ScriptTermsStringFieldFacetExecutor extends FacetExecutor {
@Override
public InternalFacet buildFacet(String facetName) {
if (facets.v().isEmpty()) {
facets.release();
facets.close();
return new InternalStringTermsFacet(facetName, comparatorType, size, ImmutableList.<InternalStringTermsFacet.TermEntry>of(), missing, total);
} else {
final boolean[] states = facets.v().allocated;
@ -98,7 +98,7 @@ public class ScriptTermsStringFieldFacetExecutor extends FacetExecutor {
for (int i = ordered.size() - 1; i >= 0; i--) {
list[i] = ((InternalStringTermsFacet.TermEntry) ordered.pop());
}
facets.release();
facets.close();
return new InternalStringTermsFacet(facetName, comparatorType, size, Arrays.asList(list), missing, total);
} else {
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]));
}
}
facets.release();
facets.close();
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();
}
Releasables.release(aggregators);
Releasables.close(aggregators);
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);
}
@ -210,7 +210,7 @@ public class TermsStringOrdinalsFacetExecutor extends FacetExecutor {
if (current.values.ordinals().getNumOrds() > 0) {
aggregators.add(current);
} else {
Releasables.release(current);
Releasables.close(current);
}
}
values = indexFieldData.load(context).getBytesValues(false);
@ -238,7 +238,7 @@ public class TermsStringOrdinalsFacetExecutor extends FacetExecutor {
if (current.values.ordinals().getNumOrds() > 0) {
aggregators.add(current);
} else {
Releasables.release(current);
Releasables.close(current);
}
current = null;
}
@ -287,9 +287,8 @@ public class TermsStringOrdinalsFacetExecutor extends FacetExecutor {
}
@Override
public boolean release() {
Releasables.release(counts);
return true;
public void close() {
Releasables.close(counts);
}
}

View File

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

View File

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

View File

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

View File

@ -75,7 +75,7 @@ public class TermsStatsLongFacetExecutor extends FacetExecutor {
@Override
public InternalFacet buildFacet(String facetName) {
if (entries.v().isEmpty()) {
entries.release();
entries.close();
return new InternalTermsStatsLongFacet(facetName, comparatorType, size, ImmutableList.<InternalTermsStatsLongFacet.LongEntry>of(), missing);
}
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);
}
@ -106,7 +106,7 @@ public class TermsStatsLongFacetExecutor extends FacetExecutor {
}
ordered.add(value);
}
entries.release();
entries.close();
return new InternalTermsStatsLongFacet(facetName, comparatorType, size, ordered, missing);
}

View File

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

View File

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

View File

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

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