Browse Source

Core: Remove DocSetCache.

This class was unused.

Close #7582
Adrien Grand 11 years ago
parent
commit
4ca2dd0a0a

+ 1 - 10
src/main/java/org/elasticsearch/index/cache/IndexCache.java

@@ -29,7 +29,6 @@ import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.index.AbstractIndexComponent;
 import org.elasticsearch.index.AbstractIndexComponent;
 import org.elasticsearch.index.Index;
 import org.elasticsearch.index.Index;
-import org.elasticsearch.index.cache.docset.DocSetCache;
 import org.elasticsearch.index.cache.filter.FilterCache;
 import org.elasticsearch.index.cache.filter.FilterCache;
 import org.elasticsearch.index.cache.fixedbitset.FixedBitSetFilterCache;
 import org.elasticsearch.index.cache.fixedbitset.FixedBitSetFilterCache;
 import org.elasticsearch.index.cache.query.parser.QueryParserCache;
 import org.elasticsearch.index.cache.query.parser.QueryParserCache;
@@ -42,17 +41,15 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
 
 
     private final FilterCache filterCache;
     private final FilterCache filterCache;
     private final QueryParserCache queryParserCache;
     private final QueryParserCache queryParserCache;
-    private final DocSetCache docSetCache;
     private final FixedBitSetFilterCache fixedBitSetFilterCache;
     private final FixedBitSetFilterCache fixedBitSetFilterCache;
 
 
     private ClusterService clusterService;
     private ClusterService clusterService;
 
 
     @Inject
     @Inject
-    public IndexCache(Index index, @IndexSettings Settings indexSettings, FilterCache filterCache, QueryParserCache queryParserCache, DocSetCache docSetCache, FixedBitSetFilterCache fixedBitSetFilterCache) {
+    public IndexCache(Index index, @IndexSettings Settings indexSettings, FilterCache filterCache, QueryParserCache queryParserCache, FixedBitSetFilterCache fixedBitSetFilterCache) {
         super(index, indexSettings);
         super(index, indexSettings);
         this.filterCache = filterCache;
         this.filterCache = filterCache;
         this.queryParserCache = queryParserCache;
         this.queryParserCache = queryParserCache;
-        this.docSetCache = docSetCache;
         this.fixedBitSetFilterCache = fixedBitSetFilterCache;
         this.fixedBitSetFilterCache = fixedBitSetFilterCache;
     }
     }
 
 
@@ -68,10 +65,6 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
         return filterCache;
         return filterCache;
     }
     }
 
 
-    public DocSetCache docSet() {
-        return this.docSetCache;
-    }
-
     public QueryParserCache queryParserCache() {
     public QueryParserCache queryParserCache() {
         return this.queryParserCache;
         return this.queryParserCache;
     }
     }
@@ -80,7 +73,6 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
     public void close() throws ElasticsearchException {
     public void close() throws ElasticsearchException {
         filterCache.close();
         filterCache.close();
         queryParserCache.close();
         queryParserCache.close();
-        docSetCache.clear("close");
         fixedBitSetFilterCache.close();
         fixedBitSetFilterCache.close();
         if (clusterService != null) {
         if (clusterService != null) {
             clusterService.remove(this);
             clusterService.remove(this);
@@ -90,7 +82,6 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
     public void clear(String reason) {
     public void clear(String reason) {
         filterCache.clear(reason);
         filterCache.clear(reason);
         queryParserCache.clear();
         queryParserCache.clear();
-        docSetCache.clear(reason);
         fixedBitSetFilterCache.clear(reason);
         fixedBitSetFilterCache.clear(reason);
     }
     }
 
 

+ 0 - 2
src/main/java/org/elasticsearch/index/cache/IndexCacheModule.java

@@ -21,7 +21,6 @@ package org.elasticsearch.index.cache;
 
 
 import org.elasticsearch.common.inject.AbstractModule;
 import org.elasticsearch.common.inject.AbstractModule;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.index.cache.docset.DocSetCacheModule;
 import org.elasticsearch.index.cache.filter.FilterCacheModule;
 import org.elasticsearch.index.cache.filter.FilterCacheModule;
 import org.elasticsearch.index.cache.fixedbitset.FixedBitSetFilterCacheModule;
 import org.elasticsearch.index.cache.fixedbitset.FixedBitSetFilterCacheModule;
 import org.elasticsearch.index.cache.query.parser.QueryParserCacheModule;
 import org.elasticsearch.index.cache.query.parser.QueryParserCacheModule;
@@ -41,7 +40,6 @@ public class IndexCacheModule extends AbstractModule {
     protected void configure() {
     protected void configure() {
         new FilterCacheModule(settings).configure(binder());
         new FilterCacheModule(settings).configure(binder());
         new QueryParserCacheModule(settings).configure(binder());
         new QueryParserCacheModule(settings).configure(binder());
-        new DocSetCacheModule(settings).configure(binder());
         new FixedBitSetFilterCacheModule(settings).configure(binder());
         new FixedBitSetFilterCacheModule(settings).configure(binder());
 
 
         bind(IndexCache.class).asEagerSingleton();
         bind(IndexCache.class).asEagerSingleton();

+ 0 - 38
src/main/java/org/elasticsearch/index/cache/docset/DocSetCache.java

@@ -1,38 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch licenses this file to you under
- * the Apache License, Version 2.0 (the "License"); you may
- * not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.elasticsearch.index.cache.docset;
-
-import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.IndexReader;
-import org.elasticsearch.common.lucene.docset.ContextDocIdSet;
-import org.elasticsearch.index.IndexComponent;
-
-/**
- */
-public interface DocSetCache extends IndexComponent {
-
-    void clear(String reason);
-
-    void clear(IndexReader reader);
-
-    ContextDocIdSet obtain(AtomicReaderContext context);
-
-    void release(ContextDocIdSet docSet);
-}

+ 0 - 44
src/main/java/org/elasticsearch/index/cache/docset/DocSetCacheModule.java

@@ -1,44 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch licenses this file to you under
- * the Apache License, Version 2.0 (the "License"); you may
- * not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.elasticsearch.index.cache.docset;
-
-import org.elasticsearch.common.inject.AbstractModule;
-import org.elasticsearch.common.inject.Scopes;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.index.cache.docset.simple.SimpleDocSetCache;
-
-/**
- *
- */
-public class DocSetCacheModule extends AbstractModule {
-
-    private final Settings settings;
-
-    public DocSetCacheModule(Settings settings) {
-        this.settings = settings;
-    }
-
-    @Override
-    protected void configure() {
-        bind(DocSetCache.class)
-                .to(settings.getAsClass("index.cache.docset.type", SimpleDocSetCache.class, "org.elasticsearch.index.cache.docset.", "DocSetCache"))
-                .in(Scopes.SINGLETON);
-    }
-}

+ 0 - 58
src/main/java/org/elasticsearch/index/cache/docset/none/NoneDocSetCache.java

@@ -1,58 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch licenses this file to you under
- * the Apache License, Version 2.0 (the "License"); you may
- * not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.elasticsearch.index.cache.docset.none;
-
-import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.util.FixedBitSet;
-import org.elasticsearch.common.inject.Inject;
-import org.elasticsearch.common.lucene.docset.ContextDocIdSet;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.index.AbstractIndexComponent;
-import org.elasticsearch.index.Index;
-import org.elasticsearch.index.cache.docset.DocSetCache;
-import org.elasticsearch.index.settings.IndexSettings;
-
-/**
- */
-public class NoneDocSetCache extends AbstractIndexComponent implements DocSetCache {
-
-    @Inject
-    public NoneDocSetCache(Index index, @IndexSettings Settings indexSettings) {
-        super(index, indexSettings);
-    }
-
-    @Override
-    public void clear(String reason) {
-    }
-
-    @Override
-    public void clear(IndexReader reader) {
-    }
-
-    @Override
-    public ContextDocIdSet obtain(AtomicReaderContext context) {
-        return new ContextDocIdSet(context, new FixedBitSet(context.reader().maxDoc()));
-    }
-
-    @Override
-    public void release(ContextDocIdSet docSet) {
-    }
-}

+ 0 - 91
src/main/java/org/elasticsearch/index/cache/docset/simple/SimpleDocSetCache.java

@@ -1,91 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch licenses this file to you under
- * the Apache License, Version 2.0 (the "License"); you may
- * not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.elasticsearch.index.cache.docset.simple;
-
-import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.SegmentReader;
-import org.apache.lucene.util.FixedBitSet;
-import org.elasticsearch.common.inject.Inject;
-import org.elasticsearch.common.lucene.docset.ContextDocIdSet;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
-import org.elasticsearch.index.AbstractIndexComponent;
-import org.elasticsearch.index.Index;
-import org.elasticsearch.index.cache.docset.DocSetCache;
-import org.elasticsearch.index.settings.IndexSettings;
-
-import java.util.Queue;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- */
-public class SimpleDocSetCache extends AbstractIndexComponent implements DocSetCache, SegmentReader.CoreClosedListener {
-
-    private final ConcurrentMap<Object, Queue<FixedBitSet>> cache;
-
-    @Inject
-    public SimpleDocSetCache(Index index, @IndexSettings Settings indexSettings) {
-        super(index, indexSettings);
-        this.cache = ConcurrentCollections.newConcurrentMap();
-    }
-
-    @Override
-    public void onClose(Object coreCacheKey) {
-        cache.remove(coreCacheKey);
-    }
-
-    @Override
-    public void clear(String reason) {
-        cache.clear();
-    }
-
-    @Override
-    public void clear(IndexReader reader) {
-        cache.remove(reader.getCoreCacheKey());
-    }
-
-    @Override
-    public ContextDocIdSet obtain(AtomicReaderContext context) {
-        Queue<FixedBitSet> docIdSets = cache.get(context.reader().getCoreCacheKey());
-        if (docIdSets == null) {
-            if (context.reader() instanceof SegmentReader) {
-                ((SegmentReader) context.reader()).addCoreClosedListener(this);
-            }
-            cache.put(context.reader().getCoreCacheKey(), ConcurrentCollections.<FixedBitSet>newQueue());
-            return new ContextDocIdSet(context, new FixedBitSet(context.reader().maxDoc()));
-        }
-        FixedBitSet docIdSet = docIdSets.poll();
-        if (docIdSet == null) {
-            docIdSet = new FixedBitSet(context.reader().maxDoc());
-        } else {
-            docIdSet.clear(0, docIdSet.length());
-        }
-        return new ContextDocIdSet(context, docIdSet);
-    }
-
-    @Override
-    public void release(ContextDocIdSet docSet) {
-        Queue<FixedBitSet> docIdSets = cache.get(docSet.context.reader().getCoreCacheKey());
-        if (docIdSets != null) {
-            docIdSets.add((FixedBitSet) docSet.docSet);
-        }
-    }
-}

+ 1 - 7
src/main/java/org/elasticsearch/percolator/PercolateContext.java

@@ -31,10 +31,9 @@ import org.elasticsearch.common.lease.Releasables;
 import org.elasticsearch.common.text.StringText;
 import org.elasticsearch.common.text.StringText;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.index.analysis.AnalysisService;
 import org.elasticsearch.index.analysis.AnalysisService;
-import org.elasticsearch.index.cache.docset.DocSetCache;
 import org.elasticsearch.index.cache.filter.FilterCache;
 import org.elasticsearch.index.cache.filter.FilterCache;
-import org.elasticsearch.index.engine.Engine;
 import org.elasticsearch.index.cache.fixedbitset.FixedBitSetFilterCache;
 import org.elasticsearch.index.cache.fixedbitset.FixedBitSetFilterCache;
+import org.elasticsearch.index.engine.Engine;
 import org.elasticsearch.index.fielddata.IndexFieldDataService;
 import org.elasticsearch.index.fielddata.IndexFieldDataService;
 import org.elasticsearch.index.fieldvisitor.JustSourceFieldsVisitor;
 import org.elasticsearch.index.fieldvisitor.JustSourceFieldsVisitor;
 import org.elasticsearch.index.mapper.FieldMapper;
 import org.elasticsearch.index.mapper.FieldMapper;
@@ -455,11 +454,6 @@ public class PercolateContext extends SearchContext {
         return indexService.fixedBitSetFilterCache();
         return indexService.fixedBitSetFilterCache();
     }
     }
 
 
-    @Override
-    public DocSetCache docSetCache() {
-        return indexService.cache().docSet();
-    }
-
     @Override
     @Override
     public long timeoutInMillis() {
     public long timeoutInMillis() {
         return -1;
         return -1;

+ 0 - 6
src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsContext.java

@@ -28,7 +28,6 @@ import org.elasticsearch.action.search.SearchType;
 import org.elasticsearch.cache.recycler.PageCacheRecycler;
 import org.elasticsearch.cache.recycler.PageCacheRecycler;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.index.analysis.AnalysisService;
 import org.elasticsearch.index.analysis.AnalysisService;
-import org.elasticsearch.index.cache.docset.DocSetCache;
 import org.elasticsearch.index.cache.filter.FilterCache;
 import org.elasticsearch.index.cache.filter.FilterCache;
 import org.elasticsearch.index.cache.fixedbitset.FixedBitSetFilterCache;
 import org.elasticsearch.index.cache.fixedbitset.FixedBitSetFilterCache;
 import org.elasticsearch.index.fielddata.IndexFieldDataService;
 import org.elasticsearch.index.fielddata.IndexFieldDataService;
@@ -336,11 +335,6 @@ public class TopHitsContext extends SearchContext {
         return context.fixedBitSetFilterCache();
         return context.fixedBitSetFilterCache();
     }
     }
 
 
-    @Override
-    public DocSetCache docSetCache() {
-        return context.docSetCache();
-    }
-
     @Override
     @Override
     public IndexFieldDataService fieldData() {
     public IndexFieldDataService fieldData() {
         return context.fieldData();
         return context.fieldData();

+ 0 - 21
src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java

@@ -22,7 +22,6 @@ package org.elasticsearch.search.internal;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.search.*;
 import org.apache.lucene.search.*;
 import org.elasticsearch.common.lease.Releasable;
 import org.elasticsearch.common.lease.Releasable;
-import org.elasticsearch.common.lease.Releasables;
 import org.elasticsearch.common.lucene.Lucene;
 import org.elasticsearch.common.lucene.Lucene;
 import org.elasticsearch.common.lucene.MinimumScoreCollector;
 import org.elasticsearch.common.lucene.MinimumScoreCollector;
 import org.elasticsearch.common.lucene.MultiCollector;
 import org.elasticsearch.common.lucene.MultiCollector;
@@ -60,9 +59,6 @@ public class ContextIndexSearcher extends IndexSearcher implements Releasable {
 
 
     private Stage currentState = Stage.NA;
     private Stage currentState = Stage.NA;
 
 
-    private boolean enableMainDocIdSetCollector;
-    private DocIdSetCollector mainDocIdSetCollector;
-
     public ContextIndexSearcher(SearchContext searchContext, Engine.Searcher searcher) {
     public ContextIndexSearcher(SearchContext searchContext, Engine.Searcher searcher) {
         super(searcher.reader());
         super(searcher.reader());
         in = searcher.searcher();
         in = searcher.searcher();
@@ -72,7 +68,6 @@ public class ContextIndexSearcher extends IndexSearcher implements Releasable {
 
 
     @Override
     @Override
     public void close() {
     public void close() {
-        Releasables.close(mainDocIdSetCollector);
     }
     }
 
 
     public void dfSource(CachedDfSource dfSource) {
     public void dfSource(CachedDfSource dfSource) {
@@ -91,14 +86,6 @@ public class ContextIndexSearcher extends IndexSearcher implements Releasable {
         queryCollectors.add(collector);
         queryCollectors.add(collector);
     }
     }
 
 
-    public DocIdSetCollector mainDocIdSetCollector() {
-        return this.mainDocIdSetCollector;
-    }
-
-    public void enableMainDocIdSetCollector() {
-        this.enableMainDocIdSetCollector = true;
-    }
-
     public void inStage(Stage stage) {
     public void inStage(Stage stage) {
         this.currentState = stage;
         this.currentState = stage;
     }
     }
@@ -152,10 +139,6 @@ public class ContextIndexSearcher extends IndexSearcher implements Releasable {
             collector = Lucene.wrapCountBasedEarlyTerminatingCollector(collector, searchContext.terminateAfter());
             collector = Lucene.wrapCountBasedEarlyTerminatingCollector(collector, searchContext.terminateAfter());
         }
         }
         if (currentState == Stage.MAIN_QUERY) {
         if (currentState == Stage.MAIN_QUERY) {
-            if (enableMainDocIdSetCollector) {
-                // TODO should we create a cache of segment->docIdSets so we won't create one each time?
-                collector = this.mainDocIdSetCollector = new DocIdSetCollector(searchContext.docSetCache(), collector);
-            }
             if (searchContext.parsedPostFilter() != null) {
             if (searchContext.parsedPostFilter() != null) {
                 // this will only get applied to the actual search collector and not
                 // this will only get applied to the actual search collector and not
                 // to any scoped collectors, also, it will only be applied to the main collector
                 // to any scoped collectors, also, it will only be applied to the main collector
@@ -192,10 +175,6 @@ public class ContextIndexSearcher extends IndexSearcher implements Releasable {
             }
             }
 
 
             if (currentState == Stage.MAIN_QUERY) {
             if (currentState == Stage.MAIN_QUERY) {
-                if (enableMainDocIdSetCollector) {
-                    enableMainDocIdSetCollector = false;
-                    mainDocIdSetCollector.postCollection();
-                }
                 if (queryCollectors != null && !queryCollectors.isEmpty()) {
                 if (queryCollectors != null && !queryCollectors.isEmpty()) {
                     for (Collector queryCollector : queryCollectors) {
                     for (Collector queryCollector : queryCollectors) {
                         if (queryCollector instanceof XCollector) {
                         if (queryCollector instanceof XCollector) {

+ 1 - 6
src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java

@@ -38,10 +38,9 @@ import org.elasticsearch.common.lucene.search.function.BoostScoreFunction;
 import org.elasticsearch.common.lucene.search.function.FunctionScoreQuery;
 import org.elasticsearch.common.lucene.search.function.FunctionScoreQuery;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.index.analysis.AnalysisService;
 import org.elasticsearch.index.analysis.AnalysisService;
-import org.elasticsearch.index.cache.docset.DocSetCache;
 import org.elasticsearch.index.cache.filter.FilterCache;
 import org.elasticsearch.index.cache.filter.FilterCache;
-import org.elasticsearch.index.engine.Engine;
 import org.elasticsearch.index.cache.fixedbitset.FixedBitSetFilterCache;
 import org.elasticsearch.index.cache.fixedbitset.FixedBitSetFilterCache;
+import org.elasticsearch.index.engine.Engine;
 import org.elasticsearch.index.fielddata.IndexFieldDataService;
 import org.elasticsearch.index.fielddata.IndexFieldDataService;
 import org.elasticsearch.index.mapper.FieldMapper;
 import org.elasticsearch.index.mapper.FieldMapper;
 import org.elasticsearch.index.mapper.FieldMappers;
 import org.elasticsearch.index.mapper.FieldMappers;
@@ -445,10 +444,6 @@ public class DefaultSearchContext extends SearchContext {
         return indexService.fixedBitSetFilterCache();
         return indexService.fixedBitSetFilterCache();
     }
     }
 
 
-    public DocSetCache docSetCache() {
-        return indexService.cache().docSet();
-    }
-
     public IndexFieldDataService fieldData() {
     public IndexFieldDataService fieldData() {
         return indexService.fieldData();
         return indexService.fieldData();
     }
     }

+ 0 - 111
src/main/java/org/elasticsearch/search/internal/DocIdSetCollector.java

@@ -1,111 +0,0 @@
-/*
- * Licensed to Elasticsearch under one or more contributor
- * license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright
- * ownership. Elasticsearch licenses this file to you under
- * the Apache License, Version 2.0 (the "License"); you may
- * not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.elasticsearch.search.internal;
-
-import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.search.Collector;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.util.FixedBitSet;
-import org.elasticsearch.common.lease.Releasable;
-import org.elasticsearch.common.lucene.docset.ContextDocIdSet;
-import org.elasticsearch.common.lucene.search.XCollector;
-import org.elasticsearch.index.cache.docset.DocSetCache;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- */
-public class DocIdSetCollector extends XCollector implements Releasable {
-
-    private final DocSetCache docSetCache;
-    private final Collector collector;
-
-    private final List<ContextDocIdSet> docSets;
-    private boolean currentHasDocs;
-    private ContextDocIdSet currentContext;
-    private FixedBitSet currentSet;
-
-    public DocIdSetCollector(DocSetCache docSetCache, Collector collector) {
-        this.docSetCache = docSetCache;
-        this.collector = collector;
-        this.docSets = new ArrayList<>();
-    }
-
-    public List<ContextDocIdSet> docSets() {
-        return docSets;
-    }
-
-    public void close() {
-        for (ContextDocIdSet docSet : docSets) {
-            docSetCache.release(docSet);
-        }
-    }
-
-    @Override
-    public void setScorer(Scorer scorer) throws IOException {
-        collector.setScorer(scorer);
-    }
-
-    @Override
-    public void collect(int doc) throws IOException {
-        collector.collect(doc);
-        currentHasDocs = true;
-        currentSet.set(doc);
-    }
-
-    @Override
-    public void setNextReader(AtomicReaderContext context) throws IOException {
-        collector.setNextReader(context);
-        if (currentContext != null) {
-            if (currentHasDocs) {
-                docSets.add(currentContext);
-            } else {
-                docSetCache.release(currentContext);
-            }
-        }
-        currentContext = docSetCache.obtain(context);
-        currentSet = (FixedBitSet) currentContext.docSet;
-        currentHasDocs = false;
-    }
-
-    @Override
-    public void postCollection() throws IOException {
-        if (collector instanceof XCollector) {
-            ((XCollector) collector).postCollection();
-        }
-        if (currentContext != null) {
-            if (currentHasDocs) {
-                docSets.add(currentContext);
-            } else {
-                docSetCache.release(currentContext);
-            }
-            currentContext = null;
-            currentSet = null;
-            currentHasDocs = false;
-        }
-    }
-
-    @Override
-    public boolean acceptsDocsOutOfOrder() {
-        return true;
-    }
-}

+ 0 - 3
src/main/java/org/elasticsearch/search/internal/SearchContext.java

@@ -32,7 +32,6 @@ import org.elasticsearch.common.lease.Releasable;
 import org.elasticsearch.common.lease.Releasables;
 import org.elasticsearch.common.lease.Releasables;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.index.analysis.AnalysisService;
 import org.elasticsearch.index.analysis.AnalysisService;
-import org.elasticsearch.index.cache.docset.DocSetCache;
 import org.elasticsearch.index.cache.filter.FilterCache;
 import org.elasticsearch.index.cache.filter.FilterCache;
 import org.elasticsearch.index.cache.fixedbitset.FixedBitSetFilterCache;
 import org.elasticsearch.index.cache.fixedbitset.FixedBitSetFilterCache;
 import org.elasticsearch.index.fielddata.IndexFieldDataService;
 import org.elasticsearch.index.fielddata.IndexFieldDataService;
@@ -211,8 +210,6 @@ public abstract class SearchContext implements Releasable {
 
 
     public abstract FixedBitSetFilterCache fixedBitSetFilterCache();
     public abstract FixedBitSetFilterCache fixedBitSetFilterCache();
 
 
-    public abstract DocSetCache docSetCache();
-
     public abstract IndexFieldDataService fieldData();
     public abstract IndexFieldDataService fieldData();
 
 
     public abstract long timeoutInMillis();
     public abstract long timeoutInMillis();

+ 0 - 6
src/test/java/org/elasticsearch/test/TestSearchContext.java

@@ -27,7 +27,6 @@ import org.elasticsearch.action.search.SearchType;
 import org.elasticsearch.cache.recycler.PageCacheRecycler;
 import org.elasticsearch.cache.recycler.PageCacheRecycler;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.common.util.BigArrays;
 import org.elasticsearch.index.analysis.AnalysisService;
 import org.elasticsearch.index.analysis.AnalysisService;
-import org.elasticsearch.index.cache.docset.DocSetCache;
 import org.elasticsearch.index.cache.filter.FilterCache;
 import org.elasticsearch.index.cache.filter.FilterCache;
 import org.elasticsearch.index.cache.fixedbitset.FixedBitSetFilterCache;
 import org.elasticsearch.index.cache.fixedbitset.FixedBitSetFilterCache;
 import org.elasticsearch.index.fielddata.IndexFieldDataService;
 import org.elasticsearch.index.fielddata.IndexFieldDataService;
@@ -330,11 +329,6 @@ public class TestSearchContext extends SearchContext {
         return fixedBitSetFilterCache;
         return fixedBitSetFilterCache;
     }
     }
 
 
-    @Override
-    public DocSetCache docSetCache() {
-        return null;
-    }
-
     @Override
     @Override
     public IndexFieldDataService fieldData() {
     public IndexFieldDataService fieldData() {
         return indexFieldDataService;
         return indexFieldDataService;