You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by tf...@apache.org on 2017/05/08 22:07:27 UTC
[13/50] [abbrv] lucene-solr:jira/solr-10233: LUCENE-7811: add
concurrent SortedSet facets implementation
LUCENE-7811: add concurrent SortedSet facets implementation
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/de709b12
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/de709b12
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/de709b12
Branch: refs/heads/jira/solr-10233
Commit: de709b122d84d85c510dfad8f35a039c38af62b4
Parents: 2af2b55
Author: Mike McCandless <mi...@apache.org>
Authored: Thu May 4 15:16:05 2017 -0400
Committer: Mike McCandless <mi...@apache.org>
Committed: Thu May 4 15:16:05 2017 -0400
----------------------------------------------------------------------
lucene/CHANGES.txt | 8 +
.../lucene/util/SameThreadExecutorService.java | 6 +-
.../apache/lucene/facet/FacetsCollector.java | 56 +--
.../lucene/facet/FacetsCollectorManager.java | 16 +-
...ConcurrentSortedSetDocValuesFacetCounts.java | 353 +++++++++++++++++++
.../DefaultSortedSetDocValuesReaderState.java | 20 +-
.../SortedSetDocValuesFacetCounts.java | 231 +++++-------
.../sortedset/SortedSetDocValuesFacetField.java | 1 +
.../SortedSetDocValuesReaderState.java | 4 +-
.../sortedset/TestSortedSetDocValuesFacets.java | 57 ++-
.../lucene/replicator/nrt/CopyOneFile.java | 5 +-
11 files changed, 542 insertions(+), 215 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/de709b12/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 002eafc..e9aefe5 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -91,6 +91,11 @@ Other
======================= Lucene 6.6.0 =======================
+New Features
+
+* LUCENE-7811: Add a concurrent SortedSet facets implementation.
+ (Mike McCandless)
+
Bug Fixes
* LUCENE-7777: ByteBlockPool.readBytes sometimes throws
@@ -127,6 +132,9 @@ Improvements
* LUCENE-7792: OfflineSorter can now run concurrently if you pass it
an optional ExecutorService (Dawid Weiss, Mike McCandless)
+* LUCENE-7811: Sorted set facets now use sparse storage when
+ collecting hits, when appropriate. (Mike McCandless)
+
Optimizations
* LUCENE-7787: spatial-extras HeatmapFacetCounter will now short-circuit it's
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/de709b12/lucene/core/src/java/org/apache/lucene/util/SameThreadExecutorService.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/SameThreadExecutorService.java b/lucene/core/src/java/org/apache/lucene/util/SameThreadExecutorService.java
index 169b9f8..517e7e2 100644
--- a/lucene/core/src/java/org/apache/lucene/util/SameThreadExecutorService.java
+++ b/lucene/core/src/java/org/apache/lucene/util/SameThreadExecutorService.java
@@ -22,8 +22,10 @@ import java.util.concurrent.AbstractExecutorService;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.TimeUnit;
-/** An {@code ExecutorService} that executes tasks immediately in the calling thread during submit. */
-class SameThreadExecutorService extends AbstractExecutorService {
+/** An {@code ExecutorService} that executes tasks immediately in the calling thread during submit.
+ *
+ * @lucene.internal */
+public final class SameThreadExecutorService extends AbstractExecutorService {
private volatile boolean shutdown;
@Override
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/de709b12/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java b/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java
index b942f7e..9333353 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java
@@ -38,8 +38,7 @@ import org.apache.lucene.search.TopFieldDocs;
import org.apache.lucene.search.TopScoreDocCollector;
import org.apache.lucene.search.TotalHitCountCollector;
import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.BitDocIdSet;
-import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.DocIdSetBuilder;
/** Collects hits for subsequent faceting. Once you've run
* a search and collect hits into this, instantiate one of
@@ -56,25 +55,9 @@ public class FacetsCollector extends SimpleCollector implements Collector {
private float[] scores;
private final boolean keepScores;
private final List<MatchingDocs> matchingDocs = new ArrayList<>();
- private Docs docs;
+ private DocIdSetBuilder docsBuilder;
/**
- * Used during collection to record matching docs and then return a
- * {@link DocIdSet} that contains them.
- */
- protected static abstract class Docs {
-
- /** Solr constructor. */
- public Docs() {}
-
- /** Record the given document. */
- public abstract void addDoc(int docId) throws IOException;
-
- /** Return the {@link DocIdSet} which contains all the recorded docs. */
- public abstract DocIdSet getDocIdSet();
- }
-
- /**
* Holds the documents that were matched in the {@link org.apache.lucene.index.LeafReaderContext}.
* If scores were required, then {@code scores} is not null.
*/
@@ -112,27 +95,6 @@ public class FacetsCollector extends SimpleCollector implements Collector {
this.keepScores = keepScores;
}
- /**
- * Creates a {@link Docs} to record hits. The default uses {@link FixedBitSet}
- * to record hits and you can override to e.g. record the docs in your own
- * {@link DocIdSet}.
- */
- protected Docs createDocs(final int maxDoc) {
- return new Docs() {
- private final FixedBitSet bits = new FixedBitSet(maxDoc);
-
- @Override
- public void addDoc(int docId) throws IOException {
- bits.set(docId);
- }
-
- @Override
- public DocIdSet getDocIdSet() {
- return new BitDocIdSet(bits);
- }
- };
- }
-
/** True if scores were saved. */
public final boolean getKeepScores() {
return keepScores;
@@ -143,9 +105,9 @@ public class FacetsCollector extends SimpleCollector implements Collector {
* visited segment.
*/
public List<MatchingDocs> getMatchingDocs() {
- if (docs != null) {
- matchingDocs.add(new MatchingDocs(this.context, docs.getDocIdSet(), totalHits, scores));
- docs = null;
+ if (docsBuilder != null) {
+ matchingDocs.add(new MatchingDocs(this.context, docsBuilder.build(), totalHits, scores));
+ docsBuilder = null;
scores = null;
context = null;
}
@@ -155,7 +117,7 @@ public class FacetsCollector extends SimpleCollector implements Collector {
@Override
public final void collect(int doc) throws IOException {
- docs.addDoc(doc);
+ docsBuilder.grow(1).add(doc);
if (keepScores) {
if (totalHits >= scores.length) {
float[] newScores = new float[ArrayUtil.oversize(totalHits + 1, 4)];
@@ -179,10 +141,10 @@ public class FacetsCollector extends SimpleCollector implements Collector {
@Override
protected void doSetNextReader(LeafReaderContext context) throws IOException {
- if (docs != null) {
- matchingDocs.add(new MatchingDocs(this.context, docs.getDocIdSet(), totalHits, scores));
+ if (docsBuilder != null) {
+ matchingDocs.add(new MatchingDocs(this.context, docsBuilder.build(), totalHits, scores));
}
- docs = createDocs(context.reader().maxDoc());
+ docsBuilder = new DocIdSetBuilder(context.reader().maxDoc());
totalHits = 0;
if (keepScores) {
scores = new float[64]; // some initial size
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/de709b12/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollectorManager.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollectorManager.java b/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollectorManager.java
index 652436d..385a897 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollectorManager.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollectorManager.java
@@ -23,12 +23,14 @@ import java.util.Collection;
import java.util.List;
/**
- * A {@link CollectorManager} implementation which produce FacetsCollector and product a merged FacetsCollector.
+ * A {@link CollectorManager} implementation which produces FacetsCollector and produces a merged FacetsCollector.
* This is used for concurrent FacetsCollection.
*/
-class FacetsCollectorManager implements CollectorManager<FacetsCollector, FacetsCollector> {
+public class FacetsCollectorManager implements CollectorManager<FacetsCollector, FacetsCollector> {
- public final static FacetsCollector EMPTY = new FacetsCollector();
+ /** Sole constructor. */
+ public FacetsCollectorManager() {
+ }
@Override
public FacetsCollector newCollector() throws IOException {
@@ -37,10 +39,11 @@ class FacetsCollectorManager implements CollectorManager<FacetsCollector, Facets
@Override
public FacetsCollector reduce(Collection<FacetsCollector> collectors) throws IOException {
- if (collectors == null || collectors.size() == 0)
- return EMPTY;
- if (collectors.size() == 1)
+ if (collectors == null || collectors.size() == 0) {
+ return new FacetsCollector();
+ } if (collectors.size() == 1) {
return collectors.iterator().next();
+ }
return new ReducedFacetsCollector(collectors);
}
@@ -51,5 +54,4 @@ class FacetsCollectorManager implements CollectorManager<FacetsCollector, Facets
facetsCollectors.forEach(facetsCollector -> matchingDocs.addAll(facetsCollector.getMatchingDocs()));
}
}
-
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/de709b12/lucene/facet/src/java/org/apache/lucene/facet/sortedset/ConcurrentSortedSetDocValuesFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/sortedset/ConcurrentSortedSetDocValuesFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/sortedset/ConcurrentSortedSetDocValuesFacetCounts.java
new file mode 100644
index 0000000..bcb6acf
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/sortedset/ConcurrentSortedSetDocValuesFacetCounts.java
@@ -0,0 +1,353 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.facet.sortedset;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicIntegerArray;
+
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsConfig;
+import org.apache.lucene.facet.LabelAndValue;
+import org.apache.lucene.facet.TopOrdAndIntQueue;
+import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState.OrdRange;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.MultiDocValues.MultiSortedSetDocValues;
+import org.apache.lucene.index.MultiDocValues;
+import org.apache.lucene.index.ReaderUtil;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.search.ConjunctionDISI;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LongValues;
+
+/** Like {@link SortedSetDocValuesFacetCounts}, but aggregates counts concurrently across segments.
+ *
+ * @lucene.experimental */
+public class ConcurrentSortedSetDocValuesFacetCounts extends Facets {
+
+ final ExecutorService exec;
+ final SortedSetDocValuesReaderState state;
+ final SortedSetDocValues dv;
+ final String field;
+ final AtomicIntegerArray counts;
+
+ /** Returns all facet counts, same result as searching on {@link MatchAllDocsQuery} but faster. */
+ public ConcurrentSortedSetDocValuesFacetCounts(SortedSetDocValuesReaderState state, ExecutorService exec)
+ throws IOException, InterruptedException {
+ this(state, null, exec);
+ }
+
+ /** Counts all facet dimensions across the provided hits. */
+ public ConcurrentSortedSetDocValuesFacetCounts(SortedSetDocValuesReaderState state, FacetsCollector hits, ExecutorService exec)
+ throws IOException, InterruptedException {
+ this.state = state;
+ this.field = state.getField();
+ this.exec = exec;
+ dv = state.getDocValues();
+ counts = new AtomicIntegerArray(state.getSize());
+ if (hits == null) {
+ // browse only
+ countAll();
+ } else {
+ count(hits.getMatchingDocs());
+ }
+ }
+
+ @Override
+ public FacetResult getTopChildren(int topN, String dim, String... path) throws IOException {
+ if (topN <= 0) {
+ throw new IllegalArgumentException("topN must be > 0 (got: " + topN + ")");
+ }
+ if (path.length > 0) {
+ throw new IllegalArgumentException("path should be 0 length");
+ }
+ OrdRange ordRange = state.getOrdRange(dim);
+ if (ordRange == null) {
+ throw new IllegalArgumentException("dimension \"" + dim + "\" was not indexed");
+ }
+ return getDim(dim, ordRange, topN);
+ }
+
+ private final FacetResult getDim(String dim, OrdRange ordRange, int topN) throws IOException {
+
+ TopOrdAndIntQueue q = null;
+
+ int bottomCount = 0;
+
+ int dimCount = 0;
+ int childCount = 0;
+
+ TopOrdAndIntQueue.OrdAndValue reuse = null;
+ //System.out.println("getDim : " + ordRange.start + " - " + ordRange.end);
+ for(int ord=ordRange.start; ord<=ordRange.end; ord++) {
+ //System.out.println(" ord=" + ord + " count=" + counts[ord]);
+ if (counts.get(ord) > 0) {
+ dimCount += counts.get(ord);
+ childCount++;
+ if (counts.get(ord) > bottomCount) {
+ if (reuse == null) {
+ reuse = new TopOrdAndIntQueue.OrdAndValue();
+ }
+ reuse.ord = ord;
+ reuse.value = counts.get(ord);
+ if (q == null) {
+ // Lazy init, so we don't create this for the
+ // sparse case unnecessarily
+ q = new TopOrdAndIntQueue(topN);
+ }
+ reuse = q.insertWithOverflow(reuse);
+ if (q.size() == topN) {
+ bottomCount = q.top().value;
+ }
+ }
+ }
+ }
+
+ if (q == null) {
+ return null;
+ }
+
+ LabelAndValue[] labelValues = new LabelAndValue[q.size()];
+ for(int i=labelValues.length-1;i>=0;i--) {
+ TopOrdAndIntQueue.OrdAndValue ordAndValue = q.pop();
+ final BytesRef term = dv.lookupOrd(ordAndValue.ord);
+ String[] parts = FacetsConfig.stringToPath(term.utf8ToString());
+ labelValues[i] = new LabelAndValue(parts[1], ordAndValue.value);
+ }
+
+ return new FacetResult(dim, new String[0], dimCount, labelValues, childCount);
+ }
+
+ private class CountOneSegment implements Callable<Void> {
+ final LeafReader leafReader;
+ final MatchingDocs hits;
+ final MultiDocValues.OrdinalMap ordinalMap;
+ final int segOrd;
+
+ public CountOneSegment(LeafReader leafReader, MatchingDocs hits, MultiDocValues.OrdinalMap ordinalMap, int segOrd) {
+ this.leafReader = leafReader;
+ this.hits = hits;
+ this.ordinalMap = ordinalMap;
+ this.segOrd = segOrd;
+ }
+
+ @Override
+ public Void call() throws IOException {
+ SortedSetDocValues segValues = leafReader.getSortedSetDocValues(field);
+ if (segValues == null) {
+ // nothing to count here
+ return null;
+ }
+
+ // TODO: yet another option is to count all segs
+ // first, only in seg-ord space, and then do a
+ // merge-sort-PQ in the end to only "resolve to
+ // global" those seg ords that can compete, if we know
+ // we just want top K? ie, this is the same algo
+ // that'd be used for merging facets across shards
+ // (distributed faceting). but this has much higher
+ // temp ram req'ts (sum of number of ords across all
+ // segs)
+ DocIdSetIterator it;
+ if (hits == null) {
+ // count all
+ it = segValues;
+ } else {
+ it = ConjunctionDISI.intersectIterators(Arrays.asList(hits.bits.iterator(), segValues));
+ }
+
+ if (ordinalMap != null) {
+ final LongValues ordMap = ordinalMap.getGlobalOrds(segOrd);
+
+ int numSegOrds = (int) segValues.getValueCount();
+
+ if (hits != null && hits.totalHits < numSegOrds/10) {
+ // Remap every ord to global ord as we iterate:
+ for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+ int term = (int) segValues.nextOrd();
+ while (term != SortedSetDocValues.NO_MORE_ORDS) {
+ counts.incrementAndGet((int) ordMap.get(term));
+ term = (int) segValues.nextOrd();
+ }
+ }
+ } else {
+
+ // First count in seg-ord space:
+ final int[] segCounts = new int[numSegOrds];
+ for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+ int term = (int) segValues.nextOrd();
+ while (term != SortedSetDocValues.NO_MORE_ORDS) {
+ segCounts[term]++;
+ term = (int) segValues.nextOrd();
+ }
+ }
+
+ // Then, migrate to global ords:
+ for(int ord=0;ord<numSegOrds;ord++) {
+ int count = segCounts[ord];
+ if (count != 0) {
+ counts.addAndGet((int) ordMap.get(ord), count);
+ }
+ }
+ }
+ } else {
+ // No ord mapping (e.g., single segment index):
+ // just aggregate directly into counts:
+ for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+ int term = (int) segValues.nextOrd();
+ while (term != SortedSetDocValues.NO_MORE_ORDS) {
+ counts.incrementAndGet(term);
+ term = (int) segValues.nextOrd();
+ }
+ }
+ }
+
+ return null;
+ }
+ }
+
+ /** Does all the "real work" of tallying up the counts. */
+ private final void count(List<MatchingDocs> matchingDocs) throws IOException, InterruptedException {
+
+ MultiDocValues.OrdinalMap ordinalMap;
+
+ // TODO: is this right? really, we need a way to
+ // verify that this ordinalMap "matches" the leaves in
+ // matchingDocs...
+ if (dv instanceof MultiDocValues.MultiSortedSetDocValues && matchingDocs.size() > 1) {
+ ordinalMap = ((MultiSortedSetDocValues) dv).mapping;
+ } else {
+ ordinalMap = null;
+ }
+
+ IndexReader reader = state.getReader();
+ List<Future<Void>> results = new ArrayList<>();
+
+ for (MatchingDocs hits : matchingDocs) {
+ // LUCENE-5090: make sure the provided reader context "matches"
+ // the top-level reader passed to the
+ // SortedSetDocValuesReaderState, else cryptic
+ // AIOOBE can happen:
+ if (ReaderUtil.getTopLevelContext(hits.context).reader() != reader) {
+ throw new IllegalStateException("the SortedSetDocValuesReaderState provided to this class does not match the reader being searched; you must create a new SortedSetDocValuesReaderState every time you open a new IndexReader");
+ }
+
+ results.add(exec.submit(new CountOneSegment(hits.context.reader(), hits, ordinalMap, hits.context.ord)));
+ }
+
+ for (Future<Void> result : results) {
+ try {
+ result.get();
+ } catch (ExecutionException ee) {
+ // Theoretically cause can be null; guard against that.
+ Throwable cause = ee.getCause();
+ throw IOUtils.rethrowAlways(cause != null ? cause : ee);
+ }
+ }
+ }
+
+ /** Does all the "real work" of tallying up the counts. */
+ private final void countAll() throws IOException, InterruptedException {
+ //System.out.println("ssdv count");
+
+ MultiDocValues.OrdinalMap ordinalMap;
+
+ // TODO: is this right? really, we need a way to
+ // verify that this ordinalMap "matches" the leaves in
+ // matchingDocs...
+ if (dv instanceof MultiDocValues.MultiSortedSetDocValues) {
+ ordinalMap = ((MultiSortedSetDocValues) dv).mapping;
+ } else {
+ ordinalMap = null;
+ }
+
+ List<Future<Void>> results = new ArrayList<>();
+
+ for (LeafReaderContext context : state.getReader().leaves()) {
+ results.add(exec.submit(new CountOneSegment(context.reader(), null, ordinalMap, context.ord)));
+ }
+
+ for (Future<Void> result : results) {
+ try {
+ result.get();
+ } catch (ExecutionException ee) {
+ // Theoretically cause can be null; guard against that.
+ Throwable cause = ee.getCause();
+ throw IOUtils.rethrowAlways(cause != null ? cause : ee);
+ }
+ }
+ }
+
+ @Override
+ public Number getSpecificValue(String dim, String... path) throws IOException {
+ if (path.length != 1) {
+ throw new IllegalArgumentException("path must be length=1");
+ }
+ int ord = (int) dv.lookupTerm(new BytesRef(FacetsConfig.pathToString(dim, path)));
+ if (ord < 0) {
+ return -1;
+ }
+
+ return counts.get(ord);
+ }
+
+ @Override
+ public List<FacetResult> getAllDims(int topN) throws IOException {
+
+ List<FacetResult> results = new ArrayList<>();
+ for(Map.Entry<String,OrdRange> ent : state.getPrefixToOrdRange().entrySet()) {
+ FacetResult fr = getDim(ent.getKey(), ent.getValue(), topN);
+ if (fr != null) {
+ results.add(fr);
+ }
+ }
+
+ // Sort by highest count:
+ Collections.sort(results,
+ new Comparator<FacetResult>() {
+ @Override
+ public int compare(FacetResult a, FacetResult b) {
+ if (a.value.intValue() > b.value.intValue()) {
+ return -1;
+ } else if (b.value.intValue() > a.value.intValue()) {
+ return 1;
+ } else {
+ return a.dim.compareTo(b.dim);
+ }
+ }
+ });
+
+ return results;
+ }
+}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/de709b12/lucene/facet/src/java/org/apache/lucene/facet/sortedset/DefaultSortedSetDocValuesReaderState.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/sortedset/DefaultSortedSetDocValuesReaderState.java b/lucene/facet/src/java/org/apache/lucene/facet/sortedset/DefaultSortedSetDocValuesReaderState.java
index cff1cca..832ff3b 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/sortedset/DefaultSortedSetDocValuesReaderState.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/sortedset/DefaultSortedSetDocValuesReaderState.java
@@ -48,7 +48,7 @@ public class DefaultSortedSetDocValuesReaderState extends SortedSetDocValuesRead
private final int valueCount;
/** {@link IndexReader} passed to the constructor. */
- public final IndexReader origReader;
+ public final IndexReader reader;
private final Map<String,OrdinalMap> cachedOrdMaps = new HashMap<>();
@@ -64,7 +64,7 @@ public class DefaultSortedSetDocValuesReaderState extends SortedSetDocValuesRead
* field. */
public DefaultSortedSetDocValuesReaderState(IndexReader reader, String field) throws IOException {
this.field = field;
- this.origReader = reader;
+ this.reader = reader;
// We need this to create thread-safe MultiSortedSetDV
// per collector:
@@ -136,7 +136,7 @@ public class DefaultSortedSetDocValuesReaderState extends SortedSetDocValuesRead
@Override
public String toString() {
- return "DefaultSortedSetDocValuesReaderState(field=" + field + " origReader=" + origReader + ")";
+ return "DefaultSortedSetDocValuesReaderState(field=" + field + " reader=" + reader + ")";
}
/** Return top-level doc values. */
@@ -151,10 +151,10 @@ public class DefaultSortedSetDocValuesReaderState extends SortedSetDocValuesRead
map = cachedOrdMaps.get(field);
if (map == null) {
// uncached, or not a multi dv
- SortedSetDocValues dv = MultiDocValues.getSortedSetValues(origReader, field);
+ SortedSetDocValues dv = MultiDocValues.getSortedSetValues(reader, field);
if (dv instanceof MultiDocValues.MultiSortedSetDocValues) {
map = ((MultiDocValues.MultiSortedSetDocValues)dv).mapping;
- IndexReader.CacheHelper cacheHelper = origReader.getReaderCacheHelper();
+ IndexReader.CacheHelper cacheHelper = reader.getReaderCacheHelper();
if (cacheHelper != null && map.owner == cacheHelper.getKey()) {
cachedOrdMaps.put(field, map);
}
@@ -164,12 +164,12 @@ public class DefaultSortedSetDocValuesReaderState extends SortedSetDocValuesRead
}
assert map != null;
- int size = origReader.leaves().size();
+ int size = reader.leaves().size();
final SortedSetDocValues[] values = new SortedSetDocValues[size];
final int[] starts = new int[size+1];
long cost = 0;
for (int i = 0; i < size; i++) {
- LeafReaderContext context = origReader.leaves().get(i);
+ LeafReaderContext context = reader.leaves().get(i);
final LeafReader reader = context.reader();
final FieldInfo fieldInfo = reader.getFieldInfos().fieldInfo(field);
if (fieldInfo != null && fieldInfo.getDocValuesType() != DocValuesType.SORTED_SET) {
@@ -183,7 +183,7 @@ public class DefaultSortedSetDocValuesReaderState extends SortedSetDocValuesRead
starts[i] = context.docBase;
cost += v.cost();
}
- starts[size] = origReader.maxDoc();
+ starts[size] = reader.maxDoc();
return new MultiSortedSetDocValues(values, starts, map, cost);
}
@@ -206,8 +206,8 @@ public class DefaultSortedSetDocValuesReaderState extends SortedSetDocValuesRead
}
@Override
- public IndexReader getOrigReader() {
- return origReader;
+ public IndexReader getReader() {
+ return reader;
}
/** Number of unique labels. */
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/de709b12/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetCounts.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetCounts.java
index 9ba8547..2198fc0 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetCounts.java
@@ -42,7 +42,6 @@ import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.search.ConjunctionDISI;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.MatchAllDocsQuery;
-import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LongValues;
@@ -70,27 +69,25 @@ public class SortedSetDocValuesFacetCounts extends Facets {
final String field;
final int[] counts;
- /** Sparse faceting: returns any dimension that had any
- * hits, topCount labels per dimension. */
- public SortedSetDocValuesFacetCounts(SortedSetDocValuesReaderState state, FacetsCollector hits)
+ /** Returns all facet counts, same result as searching on {@link MatchAllDocsQuery} but faster. */
+ public SortedSetDocValuesFacetCounts(SortedSetDocValuesReaderState state)
throws IOException {
- this.state = state;
- this.field = state.getField();
- dv = state.getDocValues();
- counts = new int[state.getSize()];
- //System.out.println("field=" + field);
- count(hits.getMatchingDocs());
+ this(state, null);
}
- /** Returns all facet counts, same result as searching on {@link MatchAllDocsQuery} but faster. */
- public SortedSetDocValuesFacetCounts(SortedSetDocValuesReaderState state)
+ /** Counts all facet dimensions across the provided hits. */
+ public SortedSetDocValuesFacetCounts(SortedSetDocValuesReaderState state, FacetsCollector hits)
throws IOException {
this.state = state;
this.field = state.getField();
dv = state.getDocValues();
counts = new int[state.getSize()];
- //System.out.println("field=" + field);
- countAll();
+ if (hits == null) {
+ // browse only
+ countAll();
+ } else {
+ count(hits.getMatchingDocs());
+ }
}
@Override
@@ -158,6 +155,83 @@ public class SortedSetDocValuesFacetCounts extends Facets {
return new FacetResult(dim, new String[0], dimCount, labelValues, childCount);
}
+ private void countOneSegment(MultiDocValues.OrdinalMap ordinalMap, LeafReader reader, int segOrd, MatchingDocs hits) throws IOException {
+ SortedSetDocValues segValues = reader.getSortedSetDocValues(field);
+ if (segValues == null) {
+ // nothing to count
+ return;
+ }
+
+ DocIdSetIterator it;
+ if (hits == null) {
+ it = segValues;
+ } else {
+ it = ConjunctionDISI.intersectIterators(Arrays.asList(hits.bits.iterator(), segValues));
+ }
+
+ // TODO: yet another option is to count all segs
+ // first, only in seg-ord space, and then do a
+ // merge-sort-PQ in the end to only "resolve to
+ // global" those seg ords that can compete, if we know
+ // we just want top K? ie, this is the same algo
+ // that'd be used for merging facets across shards
+ // (distributed faceting). but this has much higher
+ // temp ram req'ts (sum of number of ords across all
+ // segs)
+ if (ordinalMap != null) {
+ final LongValues ordMap = ordinalMap.getGlobalOrds(segOrd);
+
+ int numSegOrds = (int) segValues.getValueCount();
+
+ if (hits != null && hits.totalHits < numSegOrds/10) {
+ //System.out.println(" remap as-we-go");
+ // Remap every ord to global ord as we iterate:
+ for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+ int term = (int) segValues.nextOrd();
+ while (term != SortedSetDocValues.NO_MORE_ORDS) {
+ //System.out.println(" segOrd=" + segOrd + " ord=" + term + " globalOrd=" + ordinalMap.getGlobalOrd(segOrd, term));
+ counts[(int) ordMap.get(term)]++;
+ term = (int) segValues.nextOrd();
+ }
+ }
+ } else {
+ //System.out.println(" count in seg ord first");
+
+ // First count in seg-ord space:
+ final int[] segCounts = new int[numSegOrds];
+ for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+ int term = (int) segValues.nextOrd();
+ while (term != SortedSetDocValues.NO_MORE_ORDS) {
+ //System.out.println(" ord=" + term);
+ segCounts[term]++;
+ term = (int) segValues.nextOrd();
+ }
+ }
+
+ // Then, migrate to global ords:
+ for(int ord=0;ord<numSegOrds;ord++) {
+ int count = segCounts[ord];
+ if (count != 0) {
+ //System.out.println(" migrate segOrd=" + segOrd + " ord=" + ord + " globalOrd=" + ordinalMap.getGlobalOrd(segOrd, ord));
+ counts[(int) ordMap.get(ord)] += count;
+ }
+ }
+ }
+ } else {
+ // No ord mapping (e.g., single segment index):
+ // just aggregate directly into counts:
+ for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+ int term = (int) segValues.nextOrd();
+ while (term != SortedSetDocValues.NO_MORE_ORDS) {
+ counts[term]++;
+ term = (int) segValues.nextOrd();
+ }
+ }
+ }
+
+
+ }
+
/** Does all the "real work" of tallying up the counts. */
private final void count(List<MatchingDocs> matchingDocs) throws IOException {
//System.out.println("ssdv count");
@@ -173,88 +247,19 @@ public class SortedSetDocValuesFacetCounts extends Facets {
ordinalMap = null;
}
- IndexReader origReader = state.getOrigReader();
+ IndexReader reader = state.getReader();
for(MatchingDocs hits : matchingDocs) {
- LeafReader reader = hits.context.reader();
- //System.out.println(" reader=" + reader);
// LUCENE-5090: make sure the provided reader context "matches"
// the top-level reader passed to the
// SortedSetDocValuesReaderState, else cryptic
// AIOOBE can happen:
- if (ReaderUtil.getTopLevelContext(hits.context).reader() != origReader) {
+ if (ReaderUtil.getTopLevelContext(hits.context).reader() != reader) {
throw new IllegalStateException("the SortedSetDocValuesReaderState provided to this class does not match the reader being searched; you must create a new SortedSetDocValuesReaderState every time you open a new IndexReader");
}
-
- SortedSetDocValues segValues = reader.getSortedSetDocValues(field);
- if (segValues == null) {
- continue;
- }
- DocIdSetIterator it = ConjunctionDISI.intersectIterators(Arrays.asList(
- hits.bits.iterator(), segValues));
-
- // TODO: yet another option is to count all segs
- // first, only in seg-ord space, and then do a
- // merge-sort-PQ in the end to only "resolve to
- // global" those seg ords that can compete, if we know
- // we just want top K? ie, this is the same algo
- // that'd be used for merging facets across shards
- // (distributed faceting). but this has much higher
- // temp ram req'ts (sum of number of ords across all
- // segs)
- if (ordinalMap != null) {
- final int segOrd = hits.context.ord;
- final LongValues ordMap = ordinalMap.getGlobalOrds(segOrd);
-
- int numSegOrds = (int) segValues.getValueCount();
-
- if (hits.totalHits < numSegOrds/10) {
- //System.out.println(" remap as-we-go");
- // Remap every ord to global ord as we iterate:
- for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
- int term = (int) segValues.nextOrd();
- while (term != SortedSetDocValues.NO_MORE_ORDS) {
- //System.out.println(" segOrd=" + segOrd + " ord=" + term + " globalOrd=" + ordinalMap.getGlobalOrd(segOrd, term));
- counts[(int) ordMap.get(term)]++;
- term = (int) segValues.nextOrd();
- }
- }
- } else {
- //System.out.println(" count in seg ord first");
-
- // First count in seg-ord space:
- final int[] segCounts = new int[numSegOrds];
- for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
- int term = (int) segValues.nextOrd();
- while (term != SortedSetDocValues.NO_MORE_ORDS) {
- //System.out.println(" ord=" + term);
- segCounts[term]++;
- term = (int) segValues.nextOrd();
- }
- }
-
- // Then, migrate to global ords:
- for(int ord=0;ord<numSegOrds;ord++) {
- int count = segCounts[ord];
- if (count != 0) {
- //System.out.println(" migrate segOrd=" + segOrd + " ord=" + ord + " globalOrd=" + ordinalMap.getGlobalOrd(segOrd, ord));
- counts[(int) ordMap.get(ord)] += count;
- }
- }
- }
- } else {
- // No ord mapping (e.g., single segment index):
- // just aggregate directly into counts:
- for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
- int term = (int) segValues.nextOrd();
- while (term != SortedSetDocValues.NO_MORE_ORDS) {
- counts[term]++;
- term = (int) segValues.nextOrd();
- }
- }
- }
+ countOneSegment(ordinalMap, hits.context.reader(), hits.context.ord, hits);
}
}
@@ -273,58 +278,8 @@ public class SortedSetDocValuesFacetCounts extends Facets {
ordinalMap = null;
}
- IndexReader origReader = state.getOrigReader();
-
- for(LeafReaderContext context : origReader.leaves()) {
-
- LeafReader reader = context.reader();
-
- SortedSetDocValues segValues = reader.getSortedSetDocValues(field);
- if (segValues == null) {
- continue;
- }
-
- Bits liveDocs = reader.getLiveDocs();
-
- if (ordinalMap != null) {
- final LongValues ordMap = ordinalMap.getGlobalOrds(context.ord);
-
- int numSegOrds = (int) segValues.getValueCount();
-
- // First count in seg-ord space:
- final int[] segCounts = new int[numSegOrds];
- int docID;
- while ((docID = segValues.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
- if (liveDocs == null || liveDocs.get(docID)) {
- int term = (int) segValues.nextOrd();
- while (term != SortedSetDocValues.NO_MORE_ORDS) {
- segCounts[term]++;
- term = (int) segValues.nextOrd();
- }
- }
- }
-
- // Then, migrate to global ords:
- for(int ord=0;ord<numSegOrds;ord++) {
- int count = segCounts[ord];
- if (count != 0) {
- counts[(int) ordMap.get(ord)] += count;
- }
- }
- } else {
- // No ord mapping (e.g., single segment index):
- // just aggregate directly into counts:
- int docID;
- while ((docID = segValues.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
- if (liveDocs == null || liveDocs.get(docID)) {
- int term = (int) segValues.nextOrd();
- while (term != SortedSetDocValues.NO_MORE_ORDS) {
- counts[term]++;
- term = (int) segValues.nextOrd();
- }
- }
- }
- }
+ for(LeafReaderContext context : state.getReader().leaves()) {
+ countOneSegment(ordinalMap, context.reader(), context.ord, null);
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/de709b12/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetField.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetField.java b/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetField.java
index 9b6c26e..dd27bc3 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetField.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetField.java
@@ -28,6 +28,7 @@ public class SortedSetDocValuesFacetField extends Field {
/** Indexed {@link FieldType}. */
public static final FieldType TYPE = new FieldType();
static {
+ // NOTE: we don't actually use these index options, because this field is "processed" by FacetsConfig.build()
TYPE.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
TYPE.freeze();
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/de709b12/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesReaderState.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesReaderState.java b/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesReaderState.java
index 546b319..1fdea85 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesReaderState.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesReaderState.java
@@ -72,9 +72,9 @@ public abstract class SortedSetDocValuesReaderState implements Accountable {
/** Returns mapping from prefix to {@link OrdRange}. */
public abstract Map<String,OrdRange> getPrefixToOrdRange();
-
+
/** Returns top-level index reader. */
- public abstract IndexReader getOrigReader();
+ public abstract IndexReader getReader();
/** Number of unique labels. */
public abstract int getSize();
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/de709b12/lucene/facet/src/test/org/apache/lucene/facet/sortedset/TestSortedSetDocValuesFacets.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/sortedset/TestSortedSetDocValuesFacets.java b/lucene/facet/src/test/org/apache/lucene/facet/sortedset/TestSortedSetDocValuesFacets.java
index 3ad6e68..805eae8 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/sortedset/TestSortedSetDocValuesFacets.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/sortedset/TestSortedSetDocValuesFacets.java
@@ -22,6 +22,10 @@ import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@@ -42,6 +46,7 @@ import org.apache.lucene.search.TopDocs;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.NamedThreadFactory;
import org.apache.lucene.util.TestUtil;
public class TestSortedSetDocValuesFacets extends FacetTestCase {
@@ -76,7 +81,8 @@ public class TestSortedSetDocValuesFacets extends FacetTestCase {
// Per-top-reader state:
SortedSetDocValuesReaderState state = new DefaultSortedSetDocValuesReaderState(searcher.getIndexReader());
- SortedSetDocValuesFacetCounts facets = getAllFacets(searcher, state);
+ ExecutorService exec = randomExecutorServiceOrNull();
+ Facets facets = getAllFacets(searcher, state, exec);
assertEquals("dim=a path=[] value=4 childCount=3\n foo (2)\n bar (1)\n zoo (1)\n", facets.getTopChildren(10, "a").toString());
assertEquals("dim=b path=[] value=1 childCount=1\n baz (1)\n", facets.getTopChildren(10, "b").toString());
@@ -88,6 +94,9 @@ public class TestSortedSetDocValuesFacets extends FacetTestCase {
TopDocs hits = searcher.search(q, 1);
assertEquals(1, hits.totalHits);
+ if (exec != null) {
+ exec.shutdownNow();
+ }
writer.close();
IOUtils.close(searcher.getIndexReader(), dir);
}
@@ -170,7 +179,8 @@ public class TestSortedSetDocValuesFacets extends FacetTestCase {
// Per-top-reader state:
SortedSetDocValuesReaderState state = new DefaultSortedSetDocValuesReaderState(searcher.getIndexReader());
- SortedSetDocValuesFacetCounts facets = getAllFacets(searcher, state);
+ ExecutorService exec = randomExecutorServiceOrNull();
+ Facets facets = getAllFacets(searcher, state, exec);
// Ask for top 10 labels for any dims that have counts:
List<FacetResult> results = facets.getAllDims(10);
@@ -191,6 +201,9 @@ public class TestSortedSetDocValuesFacets extends FacetTestCase {
assertTrue(resources.isEmpty());
}
+ if (exec != null) {
+ exec.shutdownNow();
+ }
searcher.getIndexReader().close();
dir.close();
}
@@ -223,11 +236,15 @@ public class TestSortedSetDocValuesFacets extends FacetTestCase {
// Per-top-reader state:
SortedSetDocValuesReaderState state = new DefaultSortedSetDocValuesReaderState(searcher.getIndexReader());
- SortedSetDocValuesFacetCounts facets = getAllFacets(searcher, state);
+ ExecutorService exec = randomExecutorServiceOrNull();
+ Facets facets = getAllFacets(searcher, state, exec);
// Ask for top 10 labels for any dims that have counts:
assertEquals("dim=a path=[] value=2 childCount=2\n foo1 (1)\n foo2 (1)\n", facets.getTopChildren(10, "a").toString());
+ if (exec != null) {
+ exec.shutdownNow();
+ }
searcher.getIndexReader().close();
dir.close();
}
@@ -258,6 +275,7 @@ public class TestSortedSetDocValuesFacets extends FacetTestCase {
// Per-top-reader state:
SortedSetDocValuesReaderState state = new DefaultSortedSetDocValuesReaderState(searcher.getIndexReader());
+ ExecutorService exec = randomExecutorServiceOrNull();
int iters = atLeast(100);
for(int iter=0;iter<iters;iter++) {
@@ -267,7 +285,12 @@ public class TestSortedSetDocValuesFacets extends FacetTestCase {
}
FacetsCollector fc = new FacetsCollector();
FacetsCollector.search(searcher, new TermQuery(new Term("content", searchToken)), 10, fc);
- Facets facets = new SortedSetDocValuesFacetCounts(state, fc);
+ Facets facets;
+ if (exec != null) {
+ facets = new ConcurrentSortedSetDocValuesFacetCounts(state, fc, exec);
+ } else {
+ facets = new SortedSetDocValuesFacetCounts(state, fc);
+ }
// Slow, yet hopefully bug-free, faceting:
@SuppressWarnings({"rawtypes","unchecked"}) Map<String,Integer>[] expectedCounts = new HashMap[numDims];
@@ -315,17 +338,37 @@ public class TestSortedSetDocValuesFacets extends FacetTestCase {
assertEquals(expected, actual);
}
+ if (exec != null) {
+ exec.shutdownNow();
+ }
w.close();
IOUtils.close(searcher.getIndexReader(), indexDir, taxoDir);
}
- private static SortedSetDocValuesFacetCounts getAllFacets(IndexSearcher searcher, SortedSetDocValuesReaderState state) throws IOException {
+ private static Facets getAllFacets(IndexSearcher searcher, SortedSetDocValuesReaderState state,
+ ExecutorService exec) throws IOException, InterruptedException {
if (random().nextBoolean()) {
FacetsCollector c = new FacetsCollector();
- searcher.search(new MatchAllDocsQuery(), c);
- return new SortedSetDocValuesFacetCounts(state, c);
+ searcher.search(new MatchAllDocsQuery(), c);
+ if (exec != null) {
+ return new ConcurrentSortedSetDocValuesFacetCounts(state, c, exec);
+ } else {
+ return new SortedSetDocValuesFacetCounts(state, c);
+ }
+ } else if (exec != null) {
+ return new ConcurrentSortedSetDocValuesFacetCounts(state, exec);
} else {
return new SortedSetDocValuesFacetCounts(state);
}
}
+
+ private ExecutorService randomExecutorServiceOrNull() {
+ if (random().nextBoolean()) {
+ return null;
+ } else {
+ return new ThreadPoolExecutor(1, TestUtil.nextInt(random(), 2, 6), Long.MAX_VALUE, TimeUnit.MILLISECONDS,
+ new LinkedBlockingQueue<Runnable>(),
+ new NamedThreadFactory("TestIndexSearcher"));
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/de709b12/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java
index dd9f16c..7db296b 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyOneFile.java
@@ -49,8 +49,8 @@ public class CopyOneFile implements Closeable {
out = dest.createTempOutput(name, "copy", IOContext.DEFAULT);
tmpName = out.getName();
- // last 8 bytes are checksum:
- bytesToCopy = metaData.length - 8;
+ // last 8 bytes are checksum, which we write ourselves after copying all bytes and confirming checksum:
+ bytesToCopy = metaData.length - Long.BYTES;
if (Node.VERBOSE_FILES) {
dest.message("file " + name + ": start copying to tmp file " + tmpName + " length=" + (8+bytesToCopy));
@@ -101,6 +101,7 @@ public class CopyOneFile implements Closeable {
throw new IOException("file " + name + ": checksum mismatch after file copy");
}
out.writeLong(checksum);
+ bytesCopied += Long.BYTES;
close();
if (Node.VERBOSE_FILES) {