Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


uschindler commented on code in PR #13570:
URL: https://github.com/apache/lucene/pull/13570#discussion_r1677539871


##
lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java:
##
@@ -256,15 +267,15 @@ default IOException convertMapFailedIOException(
 }
   }
 
-  private static MMapIndexInputProvider lookupProvider() {
+  private static  MMapIndexInputProvider lookupProvider() {

Review Comment:
   like here, move this to `attachment()`



##
lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java:
##
@@ -199,20 +203,27 @@ public IndexInput openInput(String name, IOContext 
context) throws IOException {
 ensureOpen();
 ensureCanRead(name);
 Path path = directory.resolve(name);
-return PROVIDER.openInput(path, context, chunkSizePower, 
preload.test(name, context));
+return PROVIDER.openInput(
+path, context, chunkSizePower, preload.test(name, context), 
attachment);
   }
 
   // visible for tests:
-  static final MMapIndexInputProvider PROVIDER;
+  static final MMapIndexInputProvider PROVIDER;
 
-  interface MMapIndexInputProvider {
-IndexInput openInput(Path path, IOContext context, int chunkSizePower, 
boolean preload)
+  interface MMapIndexInputProvider {
+IndexInput openInput(
+Path path, IOContext context, int chunkSizePower, boolean preload, A 
attachment)
 throws IOException;
 
 long getDefaultMaxChunkSize();
 
 boolean supportsMadvise();
 
+/** An optional attachment of the provider, that will be passed to 
openInput. */
+default Optional attachment() {

Review Comment:
   I think this can just return null by default and let's remove the Optional. 
Optional would only make sense if we also pass the optional on openInput().



##
lucene/core/src/java21/org/apache/lucene/store/RefCountedSharedArena.java:
##
@@ -0,0 +1,112 @@
+/*
+ * 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.store;
+
+import java.lang.foreign.Arena;
+import java.lang.foreign.MemorySegment;
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+
+@SuppressWarnings("preview")
+final class RefCountedSharedArena implements Arena {
+
+  static final int OPEN = 0;
+  static final int CLOSED = -1;
+
+  static final VarHandle STATE;
+
+  static {

Review Comment:
   You want to use the VarHandle approach to spare instances, because 
AtomicReference would do the same?



##
lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java:
##
@@ -199,20 +203,27 @@ public IndexInput openInput(String name, IOContext 
context) throws IOException {
 ensureOpen();
 ensureCanRead(name);
 Path path = directory.resolve(name);
-return PROVIDER.openInput(path, context, chunkSizePower, 
preload.test(name, context));
+return PROVIDER.openInput(
+path, context, chunkSizePower, preload.test(name, context), 
attachment);
   }
 
   // visible for tests:
-  static final MMapIndexInputProvider PROVIDER;
+  static final MMapIndexInputProvider PROVIDER;
 
-  interface MMapIndexInputProvider {
-IndexInput openInput(Path path, IOContext context, int chunkSizePower, 
boolean preload)
+  interface MMapIndexInputProvider {
+IndexInput openInput(
+Path path, IOContext context, int chunkSizePower, boolean preload, A 
attachment)
 throws IOException;
 
 long getDefaultMaxChunkSize();
 
 boolean supportsMadvise();
 
+/** An optional attachment of the provider, that will be passed to 
openInput. */
+default Optional attachment() {

Review Comment:
   We could also use `default  Optional attachment()` as signature here 
so we don't need to make the whole class typed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@l

Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


uschindler commented on code in PR #13570:
URL: https://github.com/apache/lucene/pull/13570#discussion_r1677544947


##
lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java:
##
@@ -199,20 +203,27 @@ public IndexInput openInput(String name, IOContext 
context) throws IOException {
 ensureOpen();
 ensureCanRead(name);
 Path path = directory.resolve(name);
-return PROVIDER.openInput(path, context, chunkSizePower, 
preload.test(name, context));
+return PROVIDER.openInput(
+path, context, chunkSizePower, preload.test(name, context), 
attachment);
   }
 
   // visible for tests:
-  static final MMapIndexInputProvider PROVIDER;
+  static final MMapIndexInputProvider PROVIDER;
 
-  interface MMapIndexInputProvider {
-IndexInput openInput(Path path, IOContext context, int chunkSizePower, 
boolean preload)
+  interface MMapIndexInputProvider {
+IndexInput openInput(
+Path path, IOContext context, int chunkSizePower, boolean preload, A 
attachment)
 throws IOException;
 
 long getDefaultMaxChunkSize();
 
 boolean supportsMadvise();
 
+/** An optional attachment of the provider, that will be passed to 
openInput. */
+default Optional attachment() {

Review Comment:
   or without optional: `default  A attachment()`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


ChrisHegarty commented on code in PR #13570:
URL: https://github.com/apache/lucene/pull/13570#discussion_r1677621308


##
lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java:
##
@@ -199,20 +203,27 @@ public IndexInput openInput(String name, IOContext 
context) throws IOException {
 ensureOpen();
 ensureCanRead(name);
 Path path = directory.resolve(name);
-return PROVIDER.openInput(path, context, chunkSizePower, 
preload.test(name, context));
+return PROVIDER.openInput(
+path, context, chunkSizePower, preload.test(name, context), 
attachment);
   }
 
   // visible for tests:
-  static final MMapIndexInputProvider PROVIDER;
+  static final MMapIndexInputProvider PROVIDER;
 
-  interface MMapIndexInputProvider {
-IndexInput openInput(Path path, IOContext context, int chunkSizePower, 
boolean preload)
+  interface MMapIndexInputProvider {
+IndexInput openInput(
+Path path, IOContext context, int chunkSizePower, boolean preload, A 
attachment)
 throws IOException;
 
 long getDefaultMaxChunkSize();
 
 boolean supportsMadvise();
 
+/** An optional attachment of the provider, that will be passed to 
openInput. */
+default Optional attachment() {

Review Comment:
   The alternative, of course, is to just drop the use of generics here and use 
`Object`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


ChrisHegarty commented on code in PR #13570:
URL: https://github.com/apache/lucene/pull/13570#discussion_r1677620883


##
lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java:
##
@@ -199,20 +203,27 @@ public IndexInput openInput(String name, IOContext 
context) throws IOException {
 ensureOpen();
 ensureCanRead(name);
 Path path = directory.resolve(name);
-return PROVIDER.openInput(path, context, chunkSizePower, 
preload.test(name, context));
+return PROVIDER.openInput(
+path, context, chunkSizePower, preload.test(name, context), 
attachment);
   }
 
   // visible for tests:
-  static final MMapIndexInputProvider PROVIDER;
+  static final MMapIndexInputProvider PROVIDER;
 
-  interface MMapIndexInputProvider {
-IndexInput openInput(Path path, IOContext context, int chunkSizePower, 
boolean preload)
+  interface MMapIndexInputProvider {
+IndexInput openInput(
+Path path, IOContext context, int chunkSizePower, boolean preload, A 
attachment)
 throws IOException;
 
 long getDefaultMaxChunkSize();
 
 boolean supportsMadvise();
 
+/** An optional attachment of the provider, that will be passed to 
openInput. */
+default Optional attachment() {

Review Comment:
   The use of generics here is a little less than ideal. Defining the type 
parameter at the class-level allows the attachment type to be the same across 
the two instance methods, `openInput` and `attachment`. Whereas a type param on 
each of the individual methods means that the type could be different.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Add levels to DocValues skipper index [lucene]

2024-07-15 Thread via GitHub


iverase commented on code in PR #13563:
URL: https://github.com/apache/lucene/pull/13563#discussion_r1677624311


##
lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java:
##
@@ -1792,61 +1794,88 @@ public DocValuesSkipper getSkipper(FieldInfo field) 
throws IOException {
 if (input.length() > 0) {
   input.prefetch(0, 1);
 }
+// TODO: should we write to disk the actual max level for this segment?
 return new DocValuesSkipper() {
-  int minDocID = -1;
-  int maxDocID = -1;
-  long minValue, maxValue;
-  int docCount;
+  final int[] minDocID = new int[SKIP_INDEX_MAX_LEVEL];
+  final int[] maxDocID = new int[SKIP_INDEX_MAX_LEVEL];
+
+  {
+for (int i = 0; i < SKIP_INDEX_MAX_LEVEL; i++) {
+  minDocID[i] = maxDocID[i] = -1;
+}
+  }
+
+  final long[] minValue = new long[SKIP_INDEX_MAX_LEVEL];
+  final long[] maxValue = new long[SKIP_INDEX_MAX_LEVEL];
+  final int[] docCount = new int[SKIP_INDEX_MAX_LEVEL];
+  int levels;
 
   @Override
   public void advance(int target) throws IOException {
 if (target > entry.maxDocId) {
-  minDocID = DocIdSetIterator.NO_MORE_DOCS;
-  maxDocID = DocIdSetIterator.NO_MORE_DOCS;
+  // skipper is exhausted
+  for (int i = 0; i < SKIP_INDEX_MAX_LEVEL; i++) {
+minDocID[i] = maxDocID[i] = DocIdSetIterator.NO_MORE_DOCS;
+  }
 } else {
+  // find next interval
+  assert target > maxDocID[0] : "target must be bigger that current 
interval";
   while (true) {
-maxDocID = input.readInt();
-if (maxDocID >= target) {
-  minDocID = input.readInt();
-  maxValue = input.readLong();
-  minValue = input.readLong();
-  docCount = input.readInt();
+levels = input.readByte();

Review Comment:
   This is done in purpose as I am thinking in this data structure as a skip 
list, no a tree, so any given interval at any level should only be evaluated 
once and therefore only appear once while reading the index.
   
   The idea is that users only see new levels the first time they get updated. 
At that point, user can decide if they want to skip it or not. If they don't 
skip it, then they go inside that level but they should not have to evaluate it 
again and therefore it should not appear in lower levels. 
   
   I think otherwise, we will be evaluating the same interval many times?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


stefanvodita commented on code in PR #13568:
URL: https://github.com/apache/lucene/pull/13568#discussion_r1677574763


##
lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java:
##
@@ -300,35 +302,28 @@ public DrillSidewaysResult(
 }
   }
 
-  private static class CallableCollector implements Callable {
-
-private final int pos;
+  private static class CallableCollector implements Callable {

Review Comment:
   I find it a bit strange to parametrise with `Object` and always return 
`null`. Would parametrising with `Void` make sense?



##
lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java:
##
@@ -398,130 +497,56 @@ private  ConcurrentDrillSidewaysResult 
searchSequentially(
 }
 Query[] drillDownQueries = query.getDrillDownQueries();
 
-int numDims = drillDownDims.size();
-
-FacetsCollectorManager drillDownCollectorManager = 
createDrillDownFacetsCollectorManager();
-
-FacetsCollectorManager[] drillSidewaysFacetsCollectorManagers =
-new FacetsCollectorManager[numDims];
-for (int i = 0; i < numDims; i++) {
-  drillSidewaysFacetsCollectorManagers[i] = 
createDrillSidewaysFacetsCollectorManager();
-}
-
 DrillSidewaysQuery dsq =
 new DrillSidewaysQuery(
 baseQuery,
-drillDownCollectorManager,
-drillSidewaysFacetsCollectorManagers,
+// drillDownCollectorOwner,
+// Don't pass drill down collector because drill down is collected 
by IndexSearcher
+// itself.
+// TODO: deprecate drillDown collection in DrillSidewaysQuery?

Review Comment:
   Overall, the changes in this file make me think we would write things 
differently if we didn't have to maintain the API for the other faceting 
implementations. Is that so?



##
lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java:
##
@@ -398,130 +497,56 @@ private  ConcurrentDrillSidewaysResult 
searchSequentially(
 }
 Query[] drillDownQueries = query.getDrillDownQueries();
 
-int numDims = drillDownDims.size();
-
-FacetsCollectorManager drillDownCollectorManager = 
createDrillDownFacetsCollectorManager();
-
-FacetsCollectorManager[] drillSidewaysFacetsCollectorManagers =
-new FacetsCollectorManager[numDims];
-for (int i = 0; i < numDims; i++) {
-  drillSidewaysFacetsCollectorManagers[i] = 
createDrillSidewaysFacetsCollectorManager();
-}
-
 DrillSidewaysQuery dsq =
 new DrillSidewaysQuery(
 baseQuery,
-drillDownCollectorManager,
-drillSidewaysFacetsCollectorManagers,
+// drillDownCollectorOwner,
+// Don't pass drill down collector because drill down is collected 
by IndexSearcher
+// itself.
+// TODO: deprecate drillDown collection in DrillSidewaysQuery?
+null,
+drillSidewaysCollectorOwners,
 drillDownQueries,
 scoreSubDocsAtOnce());
 
-R collectorResult = searcher.search(dsq, hitCollectorManager);
-
-FacetsCollector drillDownCollector;
-if (drillDownCollectorManager != null) {
-  drillDownCollector = 
drillDownCollectorManager.reduce(dsq.managedDrillDownCollectors);
-} else {
-  drillDownCollector = null;
-}
-
-FacetsCollector[] drillSidewaysCollectors = new FacetsCollector[numDims];
-int numSlices = dsq.managedDrillSidewaysCollectors.size();
-
-for (int dim = 0; dim < numDims; dim++) {
-  List facetsCollectorsForDim = new 
ArrayList<>(numSlices);
-
-  for (int slice = 0; slice < numSlices; slice++) {
-
facetsCollectorsForDim.add(dsq.managedDrillSidewaysCollectors.get(slice)[dim]);
-  }
-
-  drillSidewaysCollectors[dim] =
-  
drillSidewaysFacetsCollectorManagers[dim].reduce(facetsCollectorsForDim);
-}
-
-String[] drillSidewaysDims = drillDownDims.keySet().toArray(new String[0]);
-
-return new ConcurrentDrillSidewaysResult<>(
-buildFacetsResult(drillDownCollector, drillSidewaysCollectors, 
drillSidewaysDims),
-null,
-collectorResult,
-drillDownCollector,
-drillSidewaysCollectors,
-drillSidewaysDims);
+searcher.searchNoReduce(dsq, drillDownCollectorOwner);
   }
 
-  @SuppressWarnings("unchecked")
-  private  ConcurrentDrillSidewaysResult searchConcurrently(
-  final DrillDownQuery query, final CollectorManager 
hitCollectorManager)
+  private void searchConcurrently(
+  final DrillDownQuery query,
+  final CollectorOwner drillDownCollectorOwner,
+  final List> drillSidewaysCollectorOwners)
   throws IOException {
 
 final Map drillDownDims = query.getDims();
 final List callableCollectors = new 
ArrayList<>(drillDownDims.size() + 1);
 
-// Add the main DrillDownQuery
-FacetsCollectorManager drillDownFacetsCollectorManager =
-createDrillDownFacetsCollectorManager();
-CollectorManager mainCo

Re: [PR] Add levels to DocValues skipper index [lucene]

2024-07-15 Thread via GitHub


jpountz commented on code in PR #13563:
URL: https://github.com/apache/lucene/pull/13563#discussion_r1677636214


##
lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java:
##
@@ -1792,61 +1794,88 @@ public DocValuesSkipper getSkipper(FieldInfo field) 
throws IOException {
 if (input.length() > 0) {
   input.prefetch(0, 1);
 }
+// TODO: should we write to disk the actual max level for this segment?
 return new DocValuesSkipper() {
-  int minDocID = -1;
-  int maxDocID = -1;
-  long minValue, maxValue;
-  int docCount;
+  final int[] minDocID = new int[SKIP_INDEX_MAX_LEVEL];
+  final int[] maxDocID = new int[SKIP_INDEX_MAX_LEVEL];
+
+  {
+for (int i = 0; i < SKIP_INDEX_MAX_LEVEL; i++) {
+  minDocID[i] = maxDocID[i] = -1;
+}
+  }
+
+  final long[] minValue = new long[SKIP_INDEX_MAX_LEVEL];
+  final long[] maxValue = new long[SKIP_INDEX_MAX_LEVEL];
+  final int[] docCount = new int[SKIP_INDEX_MAX_LEVEL];
+  int levels;
 
   @Override
   public void advance(int target) throws IOException {
 if (target > entry.maxDocId) {
-  minDocID = DocIdSetIterator.NO_MORE_DOCS;
-  maxDocID = DocIdSetIterator.NO_MORE_DOCS;
+  // skipper is exhausted
+  for (int i = 0; i < SKIP_INDEX_MAX_LEVEL; i++) {
+minDocID[i] = maxDocID[i] = DocIdSetIterator.NO_MORE_DOCS;
+  }
 } else {
+  // find next interval
+  assert target > maxDocID[0] : "target must be bigger that current 
interval";
   while (true) {
-maxDocID = input.readInt();
-if (maxDocID >= target) {
-  minDocID = input.readInt();
-  maxValue = input.readLong();
-  minValue = input.readLong();
-  docCount = input.readInt();
+levels = input.readByte();

Review Comment:
   I see. I need to think more about it. It makes sense to me for top-level 
queries which would visit the full doc ID range anyway. But if the query is 
part of a conjunction, then the leading clause of the conjunction could advance 
this clause to an arbitrary doc in the doc ID space, and I wonder if we're 
losing potential efficiency by not making the higher levels visible.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Add levels to DocValues skipper index [lucene]

2024-07-15 Thread via GitHub


iverase commented on code in PR #13563:
URL: https://github.com/apache/lucene/pull/13563#discussion_r1677637583


##
lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesConsumer.java:
##
@@ -207,65 +210,120 @@ void accumulate(long value) {
   maxValue = Math.max(maxValue, value);
 }
 
+void accumulate(SkipAccumulator other) {
+  maxDocID = other.maxDocID;
+  minValue = Math.min(minValue, other.minValue);
+  maxValue = Math.max(maxValue, other.maxValue);
+  docCount += other.docCount;
+}
+
 void nextDoc(int docID) {
   maxDocID = docID;
   ++docCount;
 }
 
-void writeTo(DataOutput output) throws IOException {
-  output.writeInt(maxDocID);
-  output.writeInt(minDocID);
-  output.writeLong(maxValue);
-  output.writeLong(minValue);
-  output.writeInt(docCount);
+public static SkipAccumulator merge(List list, int index, 
int length) {
+  SkipAccumulator acc = new SkipAccumulator(list.get(index).minDocID);
+  for (int i = 0; i < length; i++) {
+acc.accumulate(list.get(index + i));
+  }
+  return acc;
 }
   }
 
   private void writeSkipIndex(FieldInfo field, DocValuesProducer 
valuesProducer)
   throws IOException {
 assert field.hasDocValuesSkipIndex();
-// TODO: This disk compression once we introduce levels
-long start = data.getFilePointer();
-SortedNumericDocValues values = valuesProducer.getSortedNumeric(field);
+final long start = data.getFilePointer();
+final SortedNumericDocValues values = 
valuesProducer.getSortedNumeric(field);
 long globalMaxValue = Long.MIN_VALUE;
 long globalMinValue = Long.MAX_VALUE;
 int globalDocCount = 0;
 int maxDocId = -1;
+List accumulators = new ArrayList<>();
 SkipAccumulator accumulator = null;
-int counter = 0;
 for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc 
= values.nextDoc()) {
-  if (counter == 0) {
+  if (accumulator == null) {
 accumulator = new SkipAccumulator(doc);
+accumulators.add(accumulator);
   }
   accumulator.nextDoc(doc);
   for (int i = 0, end = values.docValueCount(); i < end; ++i) {
 accumulator.accumulate(values.nextValue());
   }
-  if (++counter == skipIndexIntervalSize) {
+  if (accumulator.docCount == skipIndexIntervalSize) {
 globalMaxValue = Math.max(globalMaxValue, accumulator.maxValue);
 globalMinValue = Math.min(globalMinValue, accumulator.minValue);
 globalDocCount += accumulator.docCount;
 maxDocId = accumulator.maxDocID;
-accumulator.writeTo(data);
-counter = 0;
+accumulator = null;
+if (accumulators.size()
+== SKIP_INDEX_NUMBER_INTERVALS_PER_LEVEL[SKIP_INDEX_MAX_LEVEL - 
1]) {
+  writeLevels(accumulators);
+  accumulators.clear();
+}
   }
 }
 
-if (counter > 0) {
-  globalMaxValue = Math.max(globalMaxValue, accumulator.maxValue);
-  globalMinValue = Math.min(globalMinValue, accumulator.minValue);
-  globalDocCount += accumulator.docCount;
-  maxDocId = accumulator.maxDocID;
-  accumulator.writeTo(data);
+if (accumulators.isEmpty() == false) {
+  if (accumulator != null) {
+globalMaxValue = Math.max(globalMaxValue, accumulator.maxValue);
+globalMinValue = Math.min(globalMinValue, accumulator.minValue);
+globalDocCount += accumulator.docCount;
+maxDocId = accumulator.maxDocID;
+  }
+  writeLevels(accumulators);
 }
 meta.writeLong(start); // record the start in meta
 meta.writeLong(data.getFilePointer() - start); // record the length
+assert globalDocCount == 0 || globalMaxValue >= globalMinValue;
 meta.writeLong(globalMaxValue);
 meta.writeLong(globalMinValue);
+assert globalDocCount <= maxDocId + 1;
 meta.writeInt(globalDocCount);
 meta.writeInt(maxDocId);
   }
 
+  private void writeLevels(List accumulators) throws 
IOException {
+for (int index = 0; index < accumulators.size(); index++) {
+  // compute how many levels we need to write for the current accumulator
+  final int levels = getLevels(index, accumulators.size());
+  // build the levels
+  final SkipAccumulator[] accLevels = new SkipAccumulator[levels];
+  for (int level = 0; level < levels; level++) {
+accLevels[level] =
+SkipAccumulator.merge(
+accumulators, index, 
SKIP_INDEX_NUMBER_INTERVALS_PER_LEVEL[level]);

Review Comment:
   I was hoping this approach would be a bit more heap friendly as we will only 
have to create those accumulators for a short period of time and not having a 
list per level. 
   
   But yes, it might make sense to build all the levels upfront using previous 
levels.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above

Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


ChrisHegarty commented on code in PR #13570:
URL: https://github.com/apache/lucene/pull/13570#discussion_r1677637602


##
lucene/core/src/java21/org/apache/lucene/store/RefCountedSharedArena.java:
##
@@ -0,0 +1,112 @@
+/*
+ * 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.store;
+
+import java.lang.foreign.Arena;
+import java.lang.foreign.MemorySegment;
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+
+@SuppressWarnings("preview")
+final class RefCountedSharedArena implements Arena {
+
+  static final int OPEN = 0;
+  static final int CLOSED = -1;
+
+  static final VarHandle STATE;
+
+  static {

Review Comment:
   There are certainly different approaches here. Conceptually, I think of ref 
counting as a simple `int` counter, then how to operate on that in a 
thread-safe way - that's how I got to what is in the PR.   Of course, now that 
I look at it, it could be rewritten with a `AtomicInteger`, since the CAS 
operations we need are there. And we don't operate on the state outside of that.
   
   I updated to use AtomicInteger, since the code is a little more readably, 
and I don't think that the instance creation will be a problem.  I'm on the 
fence on this one, happy to keep or revert it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Add levels to DocValues skipper index [lucene]

2024-07-15 Thread via GitHub


iverase commented on code in PR #13563:
URL: https://github.com/apache/lucene/pull/13563#discussion_r1677661302


##
lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java:
##
@@ -1792,61 +1794,88 @@ public DocValuesSkipper getSkipper(FieldInfo field) 
throws IOException {
 if (input.length() > 0) {
   input.prefetch(0, 1);
 }
+// TODO: should we write to disk the actual max level for this segment?
 return new DocValuesSkipper() {
-  int minDocID = -1;
-  int maxDocID = -1;
-  long minValue, maxValue;
-  int docCount;
+  final int[] minDocID = new int[SKIP_INDEX_MAX_LEVEL];
+  final int[] maxDocID = new int[SKIP_INDEX_MAX_LEVEL];
+
+  {
+for (int i = 0; i < SKIP_INDEX_MAX_LEVEL; i++) {
+  minDocID[i] = maxDocID[i] = -1;
+}
+  }
+
+  final long[] minValue = new long[SKIP_INDEX_MAX_LEVEL];
+  final long[] maxValue = new long[SKIP_INDEX_MAX_LEVEL];
+  final int[] docCount = new int[SKIP_INDEX_MAX_LEVEL];
+  int levels;
 
   @Override
   public void advance(int target) throws IOException {
 if (target > entry.maxDocId) {
-  minDocID = DocIdSetIterator.NO_MORE_DOCS;
-  maxDocID = DocIdSetIterator.NO_MORE_DOCS;
+  // skipper is exhausted
+  for (int i = 0; i < SKIP_INDEX_MAX_LEVEL; i++) {
+minDocID[i] = maxDocID[i] = DocIdSetIterator.NO_MORE_DOCS;
+  }
 } else {
+  // find next interval
+  assert target > maxDocID[0] : "target must be bigger that current 
interval";
   while (true) {
-maxDocID = input.readInt();
-if (maxDocID >= target) {
-  minDocID = input.readInt();
-  maxValue = input.readLong();
-  minValue = input.readLong();
-  docCount = input.readInt();
+levels = input.readByte();

Review Comment:
   mmm, advancing doc ids is different to exposing levels through the API. It 
is related to the implementation of `#advance(int docID)`. We are currently 
reading the next interval when we advance which can be inefficient on advancing 
to an arbitrary doc ID. 
   
   We can probably do better there 
   
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


epotyom commented on PR #13568:
URL: https://github.com/apache/lucene/pull/13568#issuecomment-2228412566

   @stefanvodita thank you for reviewing! 
   
   > Do you have any benchmark results to share? Hooking this into luceneutil 
might be tricky. I think even a vague sense of where this is better/worse than 
the existing facets would help.
   
   Our local test results show that throughput and CPU utilization are the 
same, but sandbox module has better latency. Although that doesn't mean that 
throughput and CPU util are the same in all cases - moving counting to a 
separate entity does add some overhead, as well as merging results in `reduce` 
call. We could also overlook some other efficiency improvements from the facets 
module.
   
   I've started looking into adding performance test tasks to luceneutil, but I 
think it requires some changes in current tasks as well, as we only seem to 
measure counting itself, but not query execution? And in the sandbox module we 
can't measure it separately. I'll continue looking into it.
   
   > Could this go into 9x? Have you tried?
   
   Yes, it is backwards compatible with 9x, there are some conflicts to solve 
when back porting though. I can share the commits if/when we want to backport.
   
   > This to make sure - do we need to mark things @lucene.experimental if 
they're in the sandbox?
   
   I checked a few other classes in sandbox module, they don't have the tag, so 
I suppose it goes without saying? But I can add it just in case, WDYT?
   
   > As I was saying, I haven't had a good look at the tests yet, but there're 
only a few for a very big PR. Are we sure we're covering everything? I like 
per-class unit tests when we can have them, although I understand that is a lot 
of work.
   
   Ah no, we only have some basic unit tests right now, we can definitely 
improve them, but I'd suggest we do it as a follow up step when/if the change 
is merged. It should be safe enough to do it that way as this is a sandbox 
module. As for non-sandbox changes, I believe existing unit tests cover 
existing APIs, and for newly added public methods we also have some coverage, 
and we can improve it later.
   
   > This replaces taxonomy and SSDV facets both, right? Together with point 2 
then - what would become obsolete if this goes ahead, i.e. will this be 
strictly better than the other facets or will we end up with 3 implementations 
instead of 2?
   
   "replaces" is a strong word :-) Firstly, the change doesn't touch indexing 
changes, so users still have to choose between side car taxonomy index and 
SSDV. As for searching, both taxonomy and SSDV based facets can be computed 
with the new module, although we haven't implemented SSDV support yet. There 
are quite a few other TODO items as well, both for performance and features, so 
I think it is too early to talk about replacing existing facets module.
   
   Even when the TODOs are implemented, we might never want to actually 
deprecate existing facets module. Except for potential performance impact for 
other users, there are other limitations. For example, in current facets module 
we collect document IDs first, which allows deciding which facets to compute 
based on collected docs. Not sure if anyone does it, but someone might? We can 
extend the new module to support that case too by first collecting doc IDs, and 
then iterating through all docs and computing facets. It can still be faster to 
do it with the new module, as it requires single doc ID loop for all facets 
rather than a loop per each Facets. But I guess what I'm saying is that it 
might be too early to discuss replacing existing facets module at this stage.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13568:
URL: https://github.com/apache/lucene/pull/13568#discussion_r166951


##
lucene/core/src/java/org/apache/lucene/search/CollectorOwner.java:
##
@@ -0,0 +1,86 @@
+/*
+ * 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.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Like {@link CollectorManager}, but it owns the collectors its manager 
creates. Benefit is that

Review Comment:
   `CollectorManager` is an interface, so unless we convert it to an abstract 
class (which might not even be possible if some classes implement 
CollectorManager and some other interface(s)), we can't do that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13568:
URL: https://github.com/apache/lucene/pull/13568#discussion_r1677782365


##
lucene/core/src/java/org/apache/lucene/search/CollectorOwner.java:
##
@@ -0,0 +1,86 @@
+/*
+ * 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.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Like {@link CollectorManager}, but it owns the collectors its manager 
creates. Benefit is that
+ * clients of the class don't have to worry about keeping the list of 
collectors, as well as about
+ * making the collectors type (C) compatible when reduce is called.
+ *
+ * @lucene.experimental
+ */
+public final class CollectorOwner {
+
+  private final CollectorManager manager;
+
+  private T result;
+  private boolean reduced;
+
+  // TODO: Normally, for IndexSearcher, we don't need parallelized write 
access to the list
+  //  because we create new collectors sequentially. But drill sideways 
creates new collectors in
+  //  DrillSidewaysQuery#Weight#bulkScorer which is already called 
concurrently.
+  //  I think making the list sychronized here is not a huge concern, at the 
same time, do we want
+  // to do something about it?
+  //  e.g. have boolean property in constructor that makes it threads friendly 
when set?
+  private final List collectors = Collections.synchronizedList(new 
ArrayList<>());
+
+  public CollectorOwner(CollectorManager manager) {
+this.manager = manager;
+  }
+
+  /** Return a new {@link Collector}. This must return a different instance on 
each call. */
+  public C newCollector() throws IOException {
+C collector = manager.newCollector();
+collectors.add(collector);
+return collector;
+  }
+
+  public C getCollector(int i) {
+return collectors.get(i);
+  }
+
+  /**
+   * Reduce the results of individual collectors into a meaningful result. For 
instance a {@link
+   * TopDocsCollector} would compute the {@link TopDocsCollector#topDocs() top 
docs} of each
+   * collector and then merge them using {@link TopDocs#merge(int, 
TopDocs[])}. This method must be
+   * called after collection is finished on all provided collectors.
+   */
+  public T reduce() throws IOException {
+result = manager.reduce(collectors);
+reduced = true;
+return result;
+  }
+
+  public static  CollectorOwner 
hire(CollectorManager manager) {

Review Comment:
   I kind of liked  `CollectorOwner.hire(manager)` more than using diamond 
operator `new CollectorOwner<>(manager)`, but there are no strong reasons, I 
can remove it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] SparseFixedBitSet#firstDoc: reduce number of `indices` iterations for a bit set that is not fully built yet. [lucene]

2024-07-15 Thread via GitHub


msokolov commented on PR #13559:
URL: https://github.com/apache/lucene/pull/13559#issuecomment-2228466558

   Another idea -- would it help your use case? -- would be to support 
`nextSetBit(start, end)` . We could do this without adding any additional 
tracking in existing SparseBitSet methods


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] SparseFixedBitSet#firstDoc: reduce number of `indices` iterations for a bit set that is not fully built yet. [lucene]

2024-07-15 Thread via GitHub


msokolov commented on code in PR #13559:
URL: https://github.com/apache/lucene/pull/13559#discussion_r1677810270


##
lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java:
##
@@ -51,6 +51,7 @@ private static int blockCount(int length) {
   }
 
   final long[] indices;
+  int maxIndicesIndex = -1;
   final long[][] bits;
   final int length;
   int nonZeroLongCount;

Review Comment:
   would it be possible to limit the scan based on `nonZeroLongCount` rather 
than adding `maxIndicesIndex`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] SparseFixedBitSet#firstDoc: reduce number of `indices` iterations for a bit set that is not fully built yet. [lucene]

2024-07-15 Thread via GitHub


epotyom commented on PR #13559:
URL: https://github.com/apache/lucene/pull/13559#issuecomment-2228524276

   @msokolov thanks for looking into it!
   
   > I wonder if DocIdSetBuilder would help? I guess it doesn't support 
accessing as a BitSet while under construction though.
   
   Yes, I think this is exactly the reason we don't use it.
   
   > Still maybe a similar class inspired from that, or even a fork of 
SparseFixedBitset could be helpful, although I do wonder how often SPFBS has a 
large number of trailing zeros.
   
   Yeah I was curious if making in SparseFixedBitset makes sense, but I haven't 
found similar use cases in Lucene code. Also looking at benchmark results, it 
is probably a case for creating a fork.
   
   > Another idea -- would it help your use case? -- would be to support 
nextSetBit(start, end) . We could do this without adding any additional 
tracking in existing SparseBitSet methods. Basically instead of asking SBS to 
keep track, the client might be able to do so?
   
   Ah, I like this idea! Working on it. Not sure if it worth adding an abstract 
method to the base BitSet class, so I'll start with SparseFixedBitSet itself 
and see how it goes.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] SparseFixedBitSet#firstDoc: reduce number of `indices` iterations for a bit set that is not fully built yet. [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13559:
URL: https://github.com/apache/lucene/pull/13559#discussion_r1677851990


##
lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java:
##
@@ -51,6 +51,7 @@ private static int blockCount(int length) {
   }
 
   final long[] indices;
+  int maxIndicesIndex = -1;
   final long[][] bits;
   final int length;
   int nonZeroLongCount;

Review Comment:
   Hmm, when we call `nextSetBit(int i)` I think we need to somehow know how 
many non-zero hits are *after* `i`, not just overall `nonZeroLongCount`; or 
maybe I'm missing something?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] SparseFixedBitSet#firstDoc: reduce number of `indices` iterations for a bit set that is not fully built yet. [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13559:
URL: https://github.com/apache/lucene/pull/13559#discussion_r1677851990


##
lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java:
##
@@ -51,6 +51,7 @@ private static int blockCount(int length) {
   }
 
   final long[] indices;
+  int maxIndicesIndex = -1;
   final long[][] bits;
   final int length;
   int nonZeroLongCount;

Review Comment:
   Hmm, when we call `nextSetBit(int i)` I think we need to somehow know how 
many non-zero bits are *after* `i`, not just overall `nonZeroLongCount`; or 
maybe I'm missing something?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] SparseFixedBitSet#firstDoc: reduce number of `indices` iterations for a bit set that is not fully built yet. [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13559:
URL: https://github.com/apache/lucene/pull/13559#discussion_r1677851990


##
lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java:
##
@@ -51,6 +51,7 @@ private static int blockCount(int length) {
   }
 
   final long[] indices;
+  int maxIndicesIndex = -1;
   final long[][] bits;
   final int length;
   int nonZeroLongCount;

Review Comment:
   Hmm, when we call `nextSetBit(int i)` I think we need to somehow know how 
many non-zero longs are *after* `i`, not just overall `nonZeroLongCount`; or 
maybe I'm missing something?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13568:
URL: https://github.com/apache/lucene/pull/13568#discussion_r1677862194


##
lucene/core/src/java/org/apache/lucene/search/CollectorOwner.java:
##
@@ -0,0 +1,86 @@
+/*
+ * 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.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Like {@link CollectorManager}, but it owns the collectors its manager 
creates. Benefit is that
+ * clients of the class don't have to worry about keeping the list of 
collectors, as well as about
+ * making the collectors type (C) compatible when reduce is called.
+ *
+ * @lucene.experimental
+ */
+public final class CollectorOwner {
+
+  private final CollectorManager manager;
+
+  private T result;
+  private boolean reduced;
+
+  // TODO: Normally, for IndexSearcher, we don't need parallelized write 
access to the list
+  //  because we create new collectors sequentially. But drill sideways 
creates new collectors in
+  //  DrillSidewaysQuery#Weight#bulkScorer which is already called 
concurrently.
+  //  I think making the list sychronized here is not a huge concern, at the 
same time, do we want
+  // to do something about it?
+  //  e.g. have boolean property in constructor that makes it threads friendly 
when set?
+  private final List collectors = Collections.synchronizedList(new 
ArrayList<>());
+
+  public CollectorOwner(CollectorManager manager) {
+this.manager = manager;
+  }
+
+  /** Return a new {@link Collector}. This must return a different instance on 
each call. */
+  public C newCollector() throws IOException {
+C collector = manager.newCollector();
+collectors.add(collector);
+return collector;
+  }
+
+  public C getCollector(int i) {
+return collectors.get(i);
+  }
+
+  /**
+   * Reduce the results of individual collectors into a meaningful result. For 
instance a {@link
+   * TopDocsCollector} would compute the {@link TopDocsCollector#topDocs() top 
docs} of each
+   * collector and then merge them using {@link TopDocs#merge(int, 
TopDocs[])}. This method must be
+   * called after collection is finished on all provided collectors.
+   */
+  public T reduce() throws IOException {
+result = manager.reduce(collectors);
+reduced = true;
+return result;
+  }
+
+  public static  CollectorOwner 
hire(CollectorManager manager) {
+// TODO: can we guarantee that the manager didn't create any Collectors 
yet?
+//  Or should we expect the new owner to be able to reduce only the work 
the manager has done
+// after it was hired?
+return new CollectorOwner<>(manager);
+  }
+
+  public T getResult() {
+if (reduced == false) {

Review Comment:
   Good question, I think we can do that. The reason it is currently separated 
is that I added `getResults` method later, and I was not sure if that is a good 
idea to have it at all; but now I believe that having `getResults` does make 
lives easier, and I don't see strong reasons to not have it. I'll merge them in 
the next iteration.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13568:
URL: https://github.com/apache/lucene/pull/13568#discussion_r1677870287


##
lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java:
##
@@ -115,6 +116,69 @@ public final LongValuesSource toLongValuesSource() {
 return new LongDoubleValuesSource(this);
   }
 
+  public final LongValuesSource toSortableLongDoubleValuesSource() {
+return new SortableLongDoubleValuesSource(this);
+  }
+
+  private static class SortableLongDoubleValuesSource extends LongValuesSource 
{
+
+private final DoubleValuesSource inner;
+
+private SortableLongDoubleValuesSource(DoubleValuesSource inner) {
+  this.inner = inner;
+}
+
+@Override
+public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) 
throws IOException {
+  DoubleValues in = inner.getValues(ctx, scores);
+
+  return new LongValues() {
+@Override
+public long longValue() throws IOException {
+  return NumericUtils.doubleToSortableLong(in.doubleValue());
+}
+
+@Override
+public boolean advanceExact(int doc) throws IOException {
+  return in.advanceExact(doc);
+}
+  };
+}
+
+@Override
+public boolean needsScores() {
+  return inner.needsScores();
+}
+
+@Override
+public int hashCode() {
+  return Objects.hash(inner);

Review Comment:
   I think the only benefit is that `Objects.hash` handles null, but not sure 
if we want to support nulls here? +cc @Shradha26 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13568:
URL: https://github.com/apache/lucene/pull/13568#discussion_r1677872262


##
lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java:
##
@@ -630,27 +630,47 @@ private TopFieldDocs searchAfter(
*/
   public  T search(Query query, CollectorManager 
collectorManager)
   throws IOException {
-final C firstCollector = collectorManager.newCollector();
+CollectorOwner collectorOwner = 
CollectorOwner.hire(collectorManager);
+final C firstCollector = collectorOwner.newCollector();
 query = rewrite(query, firstCollector.scoreMode().needsScores());
 final Weight weight = createWeight(query, firstCollector.scoreMode(), 1);
-return search(weight, collectorManager, firstCollector);
+search(weight, collectorOwner, firstCollector);
+return collectorOwner.reduce();
   }
 
-  private  T search(
-  Weight weight, CollectorManager collectorManager, C 
firstCollector) throws IOException {
+  /**
+   * Lower-level search API. Search all leaves using the given {@link 
CollectorOwner}, without
+   * calling {@link CollectorOwner#reduce()} so that clients can reduce and 
read results themselves.
+   *
+   * TODO: CollectorOwner has getResults method now (not yet used by 
anything), so maybe it's ok
+   * to call reduce in this method?
+   *
+   * @lucene.experimental
+   */
+  public  void searchNoReduce(Query query, 
CollectorOwner collectorOwner)
+  throws IOException {
+final C firstCollector = collectorOwner.newCollector();
+query = rewrite(query, firstCollector.scoreMode().needsScores());
+final Weight weight = createWeight(query, firstCollector.scoreMode(), 1);
+search(weight, collectorOwner, firstCollector);
+  }
+
+  private  void search(
+  Weight weight, CollectorOwner collectorOwner, C firstCollector) 
throws IOException {

Review Comment:
   Oh, will do - sorry!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] SparseFixedBitSet#firstDoc: reduce number of `indices` iterations for a bit set that is not fully built yet. [lucene]

2024-07-15 Thread via GitHub


msokolov commented on code in PR #13559:
URL: https://github.com/apache/lucene/pull/13559#discussion_r1677885744


##
lucene/core/src/java/org/apache/lucene/util/SparseFixedBitSet.java:
##
@@ -51,6 +51,7 @@ private static int blockCount(int length) {
   }
 
   final long[] indices;
+  int maxIndicesIndex = -1;
   final long[][] bits;
   final int length;
   int nonZeroLongCount;

Review Comment:
   oh you're right! Somehow I assumed we would know how many non-zero longs 
were before `i` but now I see that's not the case.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13568:
URL: https://github.com/apache/lucene/pull/13568#discussion_r1677923025


##
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/ordinal_iterators/SortOrdinalIterator.java:
##
@@ -0,0 +1,85 @@
+/*
+ * 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.sandbox.facet.ordinal_iterators;
+
+import java.io.IOException;
+import org.apache.lucene.sandbox.facet.abstracts.GetOrd;
+import org.apache.lucene.sandbox.facet.abstracts.OrdToComparable;
+import org.apache.lucene.sandbox.facet.abstracts.OrdinalIterator;
+import org.apache.lucene.util.InPlaceMergeSorter;
+
+/**
+ * {@link OrdinalIterator} that consumes incoming ordinals, sorts them by 
Comparable, and returns in
+ * sorted order.
+ */
+public class SortOrdinalIterator & GetOrd> implements 
OrdinalIterator {

Review Comment:
   Oh, I'll remove this class in the next iteration - I don't think we actually 
need it. The reason is that when we sort results instead of computing topN, we 
expect to keep all source ordinals, so it seems to be easier to just get an 
array of ordinals and sort the array. We can add a static util method to 
ComparableUtils for that I think.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13568:
URL: https://github.com/apache/lucene/pull/13568#discussion_r1677925857


##
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/ordinal_iterators/SortOrdinalIterator.java:
##
@@ -0,0 +1,85 @@
+/*
+ * 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.sandbox.facet.ordinal_iterators;
+
+import java.io.IOException;
+import org.apache.lucene.sandbox.facet.abstracts.GetOrd;
+import org.apache.lucene.sandbox.facet.abstracts.OrdToComparable;
+import org.apache.lucene.sandbox.facet.abstracts.OrdinalIterator;
+import org.apache.lucene.util.InPlaceMergeSorter;
+
+/**
+ * {@link OrdinalIterator} that consumes incoming ordinals, sorts them by 
Comparable, and returns in
+ * sorted order.
+ */
+public class SortOrdinalIterator & GetOrd> implements 
OrdinalIterator {
+
+  private final OrdToComparable ordToComparable;
+  private final OrdinalIterator sourceOrds;
+  private int[] result;
+  private int currentIndex;
+
+  /**
+   * @param sourceOrds source ordinals
+   * @param ordToComparable object that creates Comparable for provided facet 
ordinal. If null,
+   * ordinals are sorted in natural order (ascending).
+   */
+  public SortOrdinalIterator(OrdinalIterator sourceOrds, OrdToComparable 
ordToComparable) {
+this.sourceOrds = sourceOrds;
+this.ordToComparable = ordToComparable;
+  }
+
+  private void sort() throws IOException {
+assert result == null;
+result = sourceOrds.toArray();
+// TODO: it doesn't really work - we need List.

Review Comment:
   Thanks for heads up - will remove the class as per the comment above!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


magibney commented on code in PR #13570:
URL: https://github.com/apache/lucene/pull/13570#discussion_r1677916943


##
lucene/core/src/java21/org/apache/lucene/store/RefCountedSharedArena.java:
##
@@ -0,0 +1,102 @@
+/*
+ * 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.store;
+
+import java.lang.foreign.Arena;
+import java.lang.foreign.MemorySegment;
+import java.util.concurrent.atomic.AtomicInteger;
+
+@SuppressWarnings("preview")
+final class RefCountedSharedArena implements Arena {
+
+  static final int OPEN = 0;
+  static final int CLOSED = -1;
+
+  private final String segmentName;
+  private final Runnable removeFromMap;
+  private final Arena arena;
+
+  AtomicInteger state;
+
+  RefCountedSharedArena(String segmentName, Runnable removeFromMap) {
+this.segmentName = segmentName;
+this.removeFromMap = removeFromMap;
+this.arena = Arena.ofShared();
+this.state = new AtomicInteger(OPEN);
+  }
+
+  // for debugging
+  String getSegmentName() {
+return segmentName;
+  }
+
+  boolean acquire() {
+int value;
+while (true) {
+  value = state.get();
+  if (value < OPEN) {
+return false;
+  }
+  if (state.compareAndSet(value, value + 1)) {
+return true;
+  }
+}
+  }
+
+  void release() {
+int value;
+while (true) {
+  value = state.get();
+  if (value <= OPEN) {
+throw new IllegalStateException("already closed");
+  }
+  if (state.compareAndSet(value, value - 1)) {

Review Comment:
   Outer CAS could be replaced by `decrementAndGet()`, removing the need for a 
`while` loop? 



##
lucene/core/src/java21/org/apache/lucene/store/RefCountedSharedArena.java:
##
@@ -0,0 +1,102 @@
+/*
+ * 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.store;
+
+import java.lang.foreign.Arena;
+import java.lang.foreign.MemorySegment;
+import java.util.concurrent.atomic.AtomicInteger;
+
+@SuppressWarnings("preview")
+final class RefCountedSharedArena implements Arena {
+
+  static final int OPEN = 0;
+  static final int CLOSED = -1;
+
+  private final String segmentName;
+  private final Runnable removeFromMap;
+  private final Arena arena;
+
+  AtomicInteger state;

Review Comment:
   any reason for `state` to not be `final`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


uschindler commented on code in PR #13570:
URL: https://github.com/apache/lucene/pull/13570#discussion_r1677946847


##
lucene/core/src/java21/org/apache/lucene/store/RefCountedSharedArena.java:
##
@@ -0,0 +1,102 @@
+/*
+ * 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.store;
+
+import java.lang.foreign.Arena;
+import java.lang.foreign.MemorySegment;
+import java.util.concurrent.atomic.AtomicInteger;
+
+@SuppressWarnings("preview")
+final class RefCountedSharedArena implements Arena {
+
+  static final int OPEN = 0;
+  static final int CLOSED = -1;
+
+  private final String segmentName;
+  private final Runnable removeFromMap;
+  private final Arena arena;
+
+  AtomicInteger state;

Review Comment:
   +1
   
   I think that's a relic from the previous plain integer with VarHandle.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


uschindler commented on code in PR #13570:
URL: https://github.com/apache/lucene/pull/13570#discussion_r1677959677


##
lucene/core/src/java21/org/apache/lucene/store/RefCountedSharedArena.java:
##
@@ -0,0 +1,102 @@
+/*
+ * 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.store;
+
+import java.lang.foreign.Arena;
+import java.lang.foreign.MemorySegment;
+import java.util.concurrent.atomic.AtomicInteger;
+
+@SuppressWarnings("preview")
+final class RefCountedSharedArena implements Arena {
+
+  static final int OPEN = 0;
+  static final int CLOSED = -1;
+
+  private final String segmentName;
+  private final Runnable removeFromMap;
+  private final Arena arena;
+
+  AtomicInteger state;

Review Comment:
   Could be private, too!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


uschindler commented on code in PR #13570:
URL: https://github.com/apache/lucene/pull/13570#discussion_r1677959677


##
lucene/core/src/java21/org/apache/lucene/store/RefCountedSharedArena.java:
##
@@ -0,0 +1,102 @@
+/*
+ * 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.store;
+
+import java.lang.foreign.Arena;
+import java.lang.foreign.MemorySegment;
+import java.util.concurrent.atomic.AtomicInteger;
+
+@SuppressWarnings("preview")
+final class RefCountedSharedArena implements Arena {
+
+  static final int OPEN = 0;
+  static final int CLOSED = -1;
+
+  private final String segmentName;
+  private final Runnable removeFromMap;
+  private final Arena arena;
+
+  AtomicInteger state;

Review Comment:
   It should be private, too!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Add IntervalsSource for range and regexp queries (#13562) [lucene]

2024-07-15 Thread via GitHub


mayya-sharipova merged PR #13569:
URL: https://github.com/apache/lucene/pull/13569


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


uschindler commented on PR #13570:
URL: https://github.com/apache/lucene/pull/13570#issuecomment-2228727994

   Hi,
   we also get Updates on the JDK fixes: 
https://github.com/openjdk/jdk/pull/20158 (see issue about more details and 
some improvements Solr and/or Elasticsearch/Opensearch should do).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


ChrisHegarty commented on code in PR #13570:
URL: https://github.com/apache/lucene/pull/13570#discussion_r1677994244


##
lucene/core/src/java21/org/apache/lucene/store/RefCountedSharedArena.java:
##
@@ -0,0 +1,102 @@
+/*
+ * 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.store;
+
+import java.lang.foreign.Arena;
+import java.lang.foreign.MemorySegment;
+import java.util.concurrent.atomic.AtomicInteger;
+
+@SuppressWarnings("preview")
+final class RefCountedSharedArena implements Arena {
+
+  static final int OPEN = 0;
+  static final int CLOSED = -1;
+
+  private final String segmentName;
+  private final Runnable removeFromMap;
+  private final Arena arena;
+
+  AtomicInteger state;

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [I] Examine performance of individual data accessor methods of MemorySegmentIndexInput when IndexInputs are closed in other threads (deoptimizations,...) [lucene]

2024-07-15 Thread via GitHub


uschindler commented on issue #13325:
URL: https://github.com/apache/lucene/issues/13325#issuecomment-2228749316

   We also have another PR doing the same than @magibney's: 
https://github.com/apache/lucene/pull/13570
   
   We also got some updates on the JDK front. Next to the fixes here (use 
confined segments for READONCE and grouping shared arenas) there are a few 
suggestions:
   
   I have the feeling that Elasticsearch does not see the issue like Solr does 
under high load because the concurrency is different. @ChrisHegarty Is it true 
that Elasticsearch refreshes indexes in a sepate thread, so all close() calls 
are from a few threads only? In large Solr the installations the problem is 
mainly that every core looks like having its own threads that refresh indexes. 
Maybe it would be a good idea in Apache Solr to somehow have a maintenance 
thread which is responsible for refreshing indexes. The original patch by 
@magibney already showed that a static singleton lock around `Arena#close` 
looks like it helps. This in fact makes all closes sequential.
   
   @dsmiley Maybe that's another solution to prevent this issue from happening: 
Introduce a global "maintenance task" in Solr which is responsible to reopen 
indexes. We could include that also in the SearcherManager tool? (cc 
@mikemccand).
   
   I think with all those fixes here we have fixed most of the overhead.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


magibney commented on PR #13570:
URL: https://github.com/apache/lucene/pull/13570#issuecomment-2228755150

   To expand a bit on the concern I raised above:
   
   IIUC, in order for this to work properly (guaranteed to not potentially leak 
virtual memory address space) it depends on segment filename usage patterns and 
segment lifecycle. I think we may either need to:
   1. provide a way to configure specific MMapDirectory instances to bypass 
this Arena pooling, or
   2. disallow use of MMapDirectory for anything other than Lucene index files 
(because of the potential for collision of filename patterns that could be 
parsed as "segments", potentially allowing an Arena to live forever, 
accumulating an unlimited number of associated files.
   
   Am I missing something -- is this not a concern?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


uschindler commented on PR #13570:
URL: https://github.com/apache/lucene/pull/13570#issuecomment-2228783249

   > To expand a bit on the concern I raised above:
   > 
   > IIUC, in order for this to work properly (guaranteed to not potentially 
leak virtual memory address space) it depends on segment filename usage 
patterns and segment lifecycle. I think we may either need to:
   > 
   > 1. provide a way to configure specific MMapDirectory instances to bypass 
this Arena pooling, or
   > 2. disallow use of MMapDirectory for anything other than Lucene index 
files (because of the potential for collision of filename patterns that could 
be parsed as "segments", potentially allowing an Arena to live forever, 
accumulating an unlimited number of associated files.
   > 
   > Am I missing something -- is this not a concern?
   
   In addition we could add a separate ctor option to disable grouping.
   
   In addition I would go for option #2: Like for the segment main file it 
should also not use grouped segments for any other file that is not a correct 
segment number (it could validate the filename using the base32 hash starting 
with "_" or whatever this is).
   
   Maybe add a protected method to MMapDirectory that checks if a filename is a 
segement file) and pass this one as `Predicate` to the provider 
interface.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


ChrisHegarty commented on code in PR #13570:
URL: https://github.com/apache/lucene/pull/13570#discussion_r1678024394


##
lucene/core/src/java21/org/apache/lucene/store/RefCountedSharedArena.java:
##
@@ -0,0 +1,102 @@
+/*
+ * 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.store;
+
+import java.lang.foreign.Arena;
+import java.lang.foreign.MemorySegment;
+import java.util.concurrent.atomic.AtomicInteger;
+
+@SuppressWarnings("preview")
+final class RefCountedSharedArena implements Arena {
+
+  static final int OPEN = 0;
+  static final int CLOSED = -1;
+
+  private final String segmentName;
+  private final Runnable removeFromMap;
+  private final Arena arena;
+
+  AtomicInteger state;
+
+  RefCountedSharedArena(String segmentName, Runnable removeFromMap) {
+this.segmentName = segmentName;
+this.removeFromMap = removeFromMap;
+this.arena = Arena.ofShared();
+this.state = new AtomicInteger(OPEN);
+  }
+
+  // for debugging
+  String getSegmentName() {
+return segmentName;
+  }
+
+  boolean acquire() {
+int value;
+while (true) {
+  value = state.get();
+  if (value < OPEN) {
+return false;
+  }
+  if (state.compareAndSet(value, value + 1)) {
+return true;
+  }
+}
+  }
+
+  void release() {
+int value;
+while (true) {
+  value = state.get();
+  if (value <= OPEN) {
+throw new IllegalStateException("already closed");
+  }
+  if (state.compareAndSet(value, value - 1)) {

Review Comment:
   That's simpler, thanks. Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


uschindler commented on PR #13570:
URL: https://github.com/apache/lucene/pull/13570#issuecomment-2228809498

   We may need some other logic: We have some files that can change without the 
segment name changing? How do we handle *.del od docValues update files? If 
they correspond to the same segment, they would only be closed when actually 
the whole segment is closed after full refresh.
   
   This should be easily testable on windows: Files that have mmapped areas 
can't be deleted, so we may cause issues when closing deletion files.
   
   But on the other hand: Aren't ".del" files not READONCE!?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


magibney commented on PR #13570:
URL: https://github.com/apache/lucene/pull/13570#issuecomment-2228833300

   heh, was just writing to suggest something similar 
(`Function>`). SGTM.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Group memory arenas by segment to reduce costly `Arena.close()` [lucene]

2024-07-15 Thread via GitHub


magibney closed pull request #13555: Group memory arenas by segment to reduce 
costly `Arena.close()`
URL: https://github.com/apache/lucene/pull/13555


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Group memory arenas by segment to reduce costly `Arena.close()` [lucene]

2024-07-15 Thread via GitHub


magibney commented on PR #13555:
URL: https://github.com/apache/lucene/pull/13555#issuecomment-2228844256

   Closing in favor of #13570 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [I] Examine performance of individual data accessor methods of MemorySegmentIndexInput when IndexInputs are closed in other threads (deoptimizations,...) [lucene]

2024-07-15 Thread via GitHub


uschindler commented on issue #13325:
URL: https://github.com/apache/lucene/issues/13325#issuecomment-2228936957

   Yeah: https://github.com/openjdk/jdk/pull/20158#issuecomment-2228916752


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [I] Significant drop in recall for 8 bit Scalar Quantizer [lucene]

2024-07-15 Thread via GitHub


benwtrent commented on issue #13519:
URL: https://github.com/apache/lucene/issues/13519#issuecomment-2228966111

   @MilindShyani OK, I did some more benchmarking. 
   
   I tried switching to `+` & with your full correction term and recall 
significantly dropped to `0.518`
   
   I tried the follow permutations and they were all the same recall of `0.877`.
   
   `(dx - dxq) * (dxq - SIGNED_CORRECTION * alpha)`
   
   `(dx - dxq) * (dxq - SIGNED_CORRECTION * alpha - alpha * minQuantile)`
   
   `(dx - dxq) * (dxq - SIGNED_CORRECTION * alpha + alpha * minQuantile)`
   
   This makes me think the additional term of `alpha * minQuantize` is bringing 
nothing to the table, even for non-unit vectors (where I would expect quantiles 
& alpha to be more extreme). 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Introduces efSearch as a separate parameter in KNN{Byte:Float}VectorQuery [lucene]

2024-07-15 Thread via GitHub


shatejas commented on PR #13407:
URL: https://github.com/apache/lucene/pull/13407#issuecomment-2228981677

   @benwtrent 
   
   > Its not necessary and without it things are simpler (less code and less 
API features is a good thing by itself)
   
   This shouldn't come at the cost of not providing the core parameters of a 
widely used algorithm
   
   > Leaking the internal storage mechanism in the query API directly is 
frustrating.
   
   This is more of a compute mechanism with ability to improve recall and not 
the storage mechanism, it leverages how its stored but not a storage mechanism 
in itself. Not providing those is limiting the users their ability to improve 
the results. Forcing them to have a work around (in this case bumping up the 
`k`) is even more frustrating as there will be some amount of logic that needs 
to added on each client end.
   
   @msokolov 
   
   Thanks for linking those! Most of the discussion around removing the 
parameter is based on the fact that its in on an abstract layer which cannot be 
reused. 
   
   I do believe that there is a possibility of having an implementation which 
can make it abstract enough for other algorithms and provide flexibility to 
have parameters. It doesn't have to be either-or. 
   
   To conclude this discussion, I wouldn't want to introduce a non-optimal way 
to accomplish something when it isn't a blocker. The optimal way unfortunately 
will require significant refactor to accomplish both the goals.
   
   Closing this for now


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Introduces efSearch as a separate parameter in KNN{Byte:Float}VectorQuery [lucene]

2024-07-15 Thread via GitHub


shatejas closed pull request #13407: Introduces efSearch as a separate 
parameter in KNN{Byte:Float}VectorQuery
URL: https://github.com/apache/lucene/pull/13407


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13568:
URL: https://github.com/apache/lucene/pull/13568#discussion_r1678175501


##
lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/SandboxFacetTestCase.java:
##
@@ -0,0 +1,407 @@
+/*
+ * 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.sandbox.facet;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
+import org.apache.lucene.facet.FacetsConfig;
+import org.apache.lucene.facet.LabelAndValue;
+import org.apache.lucene.facet.taxonomy.FacetLabel;
+import org.apache.lucene.facet.taxonomy.TaxonomyFacetLabels;
+import org.apache.lucene.facet.taxonomy.TaxonomyFacetLabels.FacetLabelReader;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.sandbox.facet.abstracts.OrdLabelBiMap;
+import org.apache.lucene.sandbox.facet.abstracts.OrdToComparable;
+import org.apache.lucene.sandbox.facet.abstracts.OrdinalIterator;
+import org.apache.lucene.sandbox.facet.ordinal_iterators.TopnOrdinalIterator;
+import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder;
+import 
org.apache.lucene.sandbox.facet.taxonomy.TaxonomyChildrenOrdinalIterator;
+import org.apache.lucene.sandbox.facet.taxonomy.TaxonomyOrdLabelBiMap;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.tests.util.LuceneTestCase;
+import org.apache.lucene.tests.util.TestUtil;
+import org.apache.lucene.util.BytesRef;
+
+public abstract class SandboxFacetTestCase extends LuceneTestCase {
+  // we don't have access to overall count for all facets from count recorder,
+  // and we can't compute it as a SUM of values for each facet ID because we 
need to respect cases
+  // where
+  // the same doc belongs to multiple facets (e.g. overlapping ranges and
+  // multi value fields). We can add an extra range that includes everything,
+  // or consider supporting overall count in CountFacetRecorder. But it is not 
exactly the value
+  // we can get now, as this value wouldn't respect top-n cutoff. Is this 
value a must have facets

Review Comment:
I was under impression that it has to be sum value only for docs from top 
N, I think I misread javadoc for `FacetResult#value`, it says "Total number of 
documents containing a value for this path, **even those not included in the 
topN**.". So we actually *can* implement computing the value, just need to 
change range facets and add a facet ordinal that means "some range from the 
list".
   
   At the same time I'd suggest that we do it as a follow up task, I think it's 
ok to not have full feature parity in the first release? If you're ok with it, 
I'll just update the comment and add a TODO item for now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13568:
URL: https://github.com/apache/lucene/pull/13568#discussion_r1678177857


##
lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java:
##
@@ -300,35 +302,28 @@ public DrillSidewaysResult(
 }
   }
 
-  private static class CallableCollector implements Callable {
-
-private final int pos;
+  private static class CallableCollector implements Callable {

Review Comment:
   TIL there is `Void` - Thanks :-) 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13568:
URL: https://github.com/apache/lucene/pull/13568#discussion_r1678181693


##
lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java:
##
@@ -349,45 +344,149 @@ private DrillDownQuery getDrillDownQuery(
   public  ConcurrentDrillSidewaysResult search(
   final DrillDownQuery query, final CollectorManager 
hitCollectorManager)
   throws IOException {
+// Main query
+FacetsCollectorManager drillDownFacetsCollectorManager =
+createDrillDownFacetsCollectorManager();
+final CollectorOwner mainCollectorOwner;
+if (drillDownFacetsCollectorManager != null) {
+  // Make sure we populate a facet collector corresponding to the base 
query if desired:
+  mainCollectorOwner =
+  CollectorOwner.hire(
+  new MultiCollectorManager(drillDownFacetsCollectorManager, 
hitCollectorManager));
+} else {
+  mainCollectorOwner = CollectorOwner.hire(hitCollectorManager);
+}
+// Drill sideways dimensions
+final List> drillSidewaysCollectorOwners;
+if (query.getDims().isEmpty() == false) {
+  drillSidewaysCollectorOwners = new ArrayList<>(query.getDims().size());
+  for (int i = 0; i < query.getDims().size(); i++) {
+drillSidewaysCollectorOwners.add(
+CollectorOwner.hire(createDrillSidewaysFacetsCollectorManager()));
+  }
+} else {
+  drillSidewaysCollectorOwners = null;
+}
+// Execute query
 if (executor != null) {
-  return searchConcurrently(query, hitCollectorManager);
+  searchConcurrently(query, mainCollectorOwner, 
drillSidewaysCollectorOwners);
+} else {
+  searchSequentially(query, mainCollectorOwner, 
drillSidewaysCollectorOwners);
+}
+
+// Collect results
+final FacetsCollector facetsCollectorResult;
+final R hitCollectorResult;
+if (drillDownFacetsCollectorManager != null) {
+  // drill down collected using MultiCollector
+  // Extract the results:
+  Object[] drillDownResult = (Object[]) mainCollectorOwner.reduce();
+  facetsCollectorResult = (FacetsCollector) drillDownResult[0];
+  hitCollectorResult = (R) drillDownResult[1];
+} else {
+  facetsCollectorResult = null;
+  hitCollectorResult = (R) mainCollectorOwner.reduce();
+}
+
+// Getting results for drill sideways dimensions (if any)
+final String[] drillSidewaysDims;
+final FacetsCollector[] drillSidewaysCollectors;
+if (query.getDims().isEmpty() == false) {
+  drillSidewaysDims = query.getDims().keySet().toArray(new String[0]);
+  int numDims = query.getDims().size();
+  assert drillSidewaysCollectorOwners != null;
+  assert drillSidewaysCollectorOwners.size() == numDims;
+  drillSidewaysCollectors = new FacetsCollector[numDims];
+  for (int dim = 0; dim < numDims; dim++) {
+drillSidewaysCollectors[dim] =
+(FacetsCollector) drillSidewaysCollectorOwners.get(dim).reduce();
+  }
 } else {
-  return searchSequentially(query, hitCollectorManager);
+  drillSidewaysDims = null;
+  drillSidewaysCollectors = null;
 }
+
+return new ConcurrentDrillSidewaysResult<>(
+buildFacetsResult(facetsCollectorResult, drillSidewaysCollectors, 
drillSidewaysDims),
+null,
+hitCollectorResult,
+facetsCollectorResult,
+drillSidewaysCollectors,
+drillSidewaysDims);
   }
 
-  @SuppressWarnings("unchecked")
-  private  ConcurrentDrillSidewaysResult searchSequentially(
-  final DrillDownQuery query, final CollectorManager 
hitCollectorManager)
+  /**
+   * Search using DrillDownQuery with custom collectors. This method can be 
used with any {@link
+   * CollectorOwner}s. It doesn't return anything because it is expected that 
you read results from
+   * provided {@link CollectorOwner}s.
+   *
+   * To read the results, run {@link CollectorOwner#reduce()} for drill 
down and all drill
+   * sideways dimensions.
+   *
+   * If {@code doReduce} is set to true, this method itself calls {@link
+   * CollectorOwner#reduce()}. Note that results of the call are not returned 
by this method, so you
+   * can only do that if there is some other way of accessing results from the 
reduce call.
+   *
+   * Note: use {@link Collections#unmodifiableList(List)} to wrap {@code
+   * drillSidewaysCollectorOwners} to convince compiler that it is safe to use 
List here.
+   *
+   * TODO: Class CollectorOwner was created so that we can ignore 
CollectorManager type C,
+   * because we want each dimensions to be able to use their own types. 
Alternatively, we can use
+   * typesafe heterogeneous container and provide CollectorManager type for 
each dimension to this
+   * method? I do like CollectorOwner approach as it seems more intuitive?
+   *
+   * TODO: deprecate doReduce - always reduce, {@link 
CollectorOwner#getResult()} can be used by

Review Comment:
   I'll remove `doReduce` param and change `Collect

Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13568:
URL: https://github.com/apache/lucene/pull/13568#discussion_r1678184630


##
lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java:
##
@@ -398,130 +497,56 @@ private  ConcurrentDrillSidewaysResult 
searchSequentially(
 }
 Query[] drillDownQueries = query.getDrillDownQueries();
 
-int numDims = drillDownDims.size();
-
-FacetsCollectorManager drillDownCollectorManager = 
createDrillDownFacetsCollectorManager();
-
-FacetsCollectorManager[] drillSidewaysFacetsCollectorManagers =
-new FacetsCollectorManager[numDims];
-for (int i = 0; i < numDims; i++) {
-  drillSidewaysFacetsCollectorManagers[i] = 
createDrillSidewaysFacetsCollectorManager();
-}
-
 DrillSidewaysQuery dsq =
 new DrillSidewaysQuery(
 baseQuery,
-drillDownCollectorManager,
-drillSidewaysFacetsCollectorManagers,
+// drillDownCollectorOwner,
+// Don't pass drill down collector because drill down is collected 
by IndexSearcher
+// itself.
+// TODO: deprecate drillDown collection in DrillSidewaysQuery?

Review Comment:
   I don't think we would write it completely differently - we could deprecate 
some methods that use FacetCollectors in the future maybe? But there are some 
followup items, like this TODO, which I think we can look into later.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13568:
URL: https://github.com/apache/lucene/pull/13568#discussion_r1678185585


##
lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java:
##
@@ -398,130 +497,56 @@ private  ConcurrentDrillSidewaysResult 
searchSequentially(
 }
 Query[] drillDownQueries = query.getDrillDownQueries();
 
-int numDims = drillDownDims.size();
-
-FacetsCollectorManager drillDownCollectorManager = 
createDrillDownFacetsCollectorManager();
-
-FacetsCollectorManager[] drillSidewaysFacetsCollectorManagers =
-new FacetsCollectorManager[numDims];
-for (int i = 0; i < numDims; i++) {
-  drillSidewaysFacetsCollectorManagers[i] = 
createDrillSidewaysFacetsCollectorManager();
-}
-
 DrillSidewaysQuery dsq =
 new DrillSidewaysQuery(
 baseQuery,
-drillDownCollectorManager,
-drillSidewaysFacetsCollectorManagers,
+// drillDownCollectorOwner,
+// Don't pass drill down collector because drill down is collected 
by IndexSearcher
+// itself.
+// TODO: deprecate drillDown collection in DrillSidewaysQuery?
+null,
+drillSidewaysCollectorOwners,
 drillDownQueries,
 scoreSubDocsAtOnce());
 
-R collectorResult = searcher.search(dsq, hitCollectorManager);
-
-FacetsCollector drillDownCollector;
-if (drillDownCollectorManager != null) {
-  drillDownCollector = 
drillDownCollectorManager.reduce(dsq.managedDrillDownCollectors);
-} else {
-  drillDownCollector = null;
-}
-
-FacetsCollector[] drillSidewaysCollectors = new FacetsCollector[numDims];
-int numSlices = dsq.managedDrillSidewaysCollectors.size();
-
-for (int dim = 0; dim < numDims; dim++) {
-  List facetsCollectorsForDim = new 
ArrayList<>(numSlices);
-
-  for (int slice = 0; slice < numSlices; slice++) {
-
facetsCollectorsForDim.add(dsq.managedDrillSidewaysCollectors.get(slice)[dim]);
-  }
-
-  drillSidewaysCollectors[dim] =
-  
drillSidewaysFacetsCollectorManagers[dim].reduce(facetsCollectorsForDim);
-}
-
-String[] drillSidewaysDims = drillDownDims.keySet().toArray(new String[0]);
-
-return new ConcurrentDrillSidewaysResult<>(
-buildFacetsResult(drillDownCollector, drillSidewaysCollectors, 
drillSidewaysDims),
-null,
-collectorResult,
-drillDownCollector,
-drillSidewaysCollectors,
-drillSidewaysDims);
+searcher.searchNoReduce(dsq, drillDownCollectorOwner);
   }
 
-  @SuppressWarnings("unchecked")
-  private  ConcurrentDrillSidewaysResult searchConcurrently(
-  final DrillDownQuery query, final CollectorManager 
hitCollectorManager)
+  private void searchConcurrently(
+  final DrillDownQuery query,
+  final CollectorOwner drillDownCollectorOwner,
+  final List> drillSidewaysCollectorOwners)
   throws IOException {
 
 final Map drillDownDims = query.getDims();
 final List callableCollectors = new 
ArrayList<>(drillDownDims.size() + 1);
 
-// Add the main DrillDownQuery
-FacetsCollectorManager drillDownFacetsCollectorManager =
-createDrillDownFacetsCollectorManager();
-CollectorManager mainCollectorManager;
-if (drillDownFacetsCollectorManager != null) {
-  // Make sure we populate a facet collector corresponding to the base 
query if desired:
-  mainCollectorManager =
-  new MultiCollectorManager(drillDownFacetsCollectorManager, 
hitCollectorManager);
-} else {
-  mainCollectorManager = hitCollectorManager;
-}
-callableCollectors.add(new CallableCollector(-1, searcher, query, 
mainCollectorManager));
+callableCollectors.add(new CallableCollector(-1, searcher, query, 
drillDownCollectorOwner));
 int i = 0;
 final Query[] filters = query.getDrillDownQueries();
-for (String dim : drillDownDims.keySet())
+for (String dim : drillDownDims.keySet()) {
   callableCollectors.add(
   new CallableCollector(
-  i++,
+  i,
   searcher,
   getDrillDownQuery(query, filters, dim),
-  createDrillSidewaysFacetsCollectorManager()));
-
-final FacetsCollector mainFacetsCollector;
-final FacetsCollector[] facetsCollectors = new 
FacetsCollector[drillDownDims.size()];
-final R collectorResult;
+  drillSidewaysCollectorOwners.get(i)));
+  i++; // TODO: refactor maybe?

Review Comment:
   Ah yes, I think we can just remove the TODO - don't remember what 
refactoring I had in mind TBH.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13568:
URL: https://github.com/apache/lucene/pull/13568#discussion_r1678226435


##
lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java:
##
@@ -195,11 +156,11 @@ public ScorerSupplier scorerSupplier(LeafReaderContext 
context) throws IOExcepti
 // a null scorer in this case, but we need to make sure #finish gets 
called on all facet
 // collectors since IndexSearcher won't handle this for us:
 if (baseScorerSupplier == null || nullCount > 1) {
-  if (drillDownCollector != null) {
-drillDownCollector.finish();
+  if (drillDownLeafCollector != null) {

Review Comment:
   The problem is that `#finish` is a method of `LeafCollector`, not 
`Collector`. The way it used to work is that `FacetCollector` implements both 
`Collector` and `LeafCollector`, so `drillDownCollector.finish()` used to 
finish work for the current leaf.
   
   With this change we have to differentiate between the two interfaces, and 
call `#finish` on the leaf collector rather than Collector itself.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13568:
URL: https://github.com/apache/lucene/pull/13568#discussion_r1678240834


##
lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestRangeFacet.java:
##
@@ -0,0 +1,1654 @@
+/*
+ * 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.sandbox.facet;
+
+import static org.apache.lucene.facet.FacetsConfig.DEFAULT_INDEX_FIELD_NAME;
+
+import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
+import java.io.IOException;
+import java.util.List;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleDocValuesField;
+import org.apache.lucene.document.DoublePoint;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedNumericDocValuesField;
+import org.apache.lucene.facet.DrillDownQuery;
+import org.apache.lucene.facet.DrillSideways;
+import org.apache.lucene.facet.FacetField;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.FacetsConfig;
+import org.apache.lucene.facet.LabelAndValue;
+import org.apache.lucene.facet.MultiDoubleValuesSource;
+import org.apache.lucene.facet.MultiLongValuesSource;
+import org.apache.lucene.facet.range.DoubleRange;
+import org.apache.lucene.facet.range.LongRange;
+import org.apache.lucene.facet.range.Range;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
+import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.sandbox.facet.abstracts.OrdLabelBiMap;
+import org.apache.lucene.sandbox.facet.ranges.DoubleRangeFacetCutter;
+import org.apache.lucene.sandbox.facet.ranges.LongRangeFacetCutter;
+import org.apache.lucene.sandbox.facet.ranges.RangeOrdLabelBiMap;
+import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder;
+import org.apache.lucene.sandbox.facet.taxonomy.TaxonomyFacetsCutter;
+import org.apache.lucene.search.CollectorOwner;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LongValuesSource;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.MultiCollectorManager;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.tests.index.RandomIndexWriter;
+import org.apache.lucene.tests.search.DummyTotalHitCountCollector;
+import org.apache.lucene.tests.util.TestUtil;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * Test sandbox facet ranges. Mostly test cases from LongRangeFacetCounts 
adopted for sandbox
+ * faceting.
+ */
+public class TestRangeFacet extends SandboxFacetTestCase {
+
+  public void testBasicLong() throws Exception {
+Directory d = newDirectory();
+RandomIndexWriter w = new RandomIndexWriter(random(), d);
+Document doc = new Document();
+NumericDocValuesField field = new NumericDocValuesField("field", 0L);
+doc.add(field);
+for (long l = 0; l < 100; l++) {
+  field.setLongValue(l);
+  w.addDocument(doc);
+}
+
+// Also add Long.MAX_VALUE
+field.setLongValue(Long.MAX_VALUE);
+w.addDocument(doc);
+
+IndexReader r = w.getReader();
+w.close();
+
+IndexSearcher s = newSearcher(r);
+LongRange[] inputRanges =
+new LongRange[] {
+  new LongRange("less than 10", 0L, true, 10L, false),
+  new LongRange("less than or equal to 10", 0L, true, 10L, true),
+  new LongRange("over 90", 90L, false, 100L, false),
+  new LongRange("90 or above", 90L, true, 100L, false),
+  new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, true),
+};
+
+MultiLongValuesSource valuesSource = 
MultiLongValuesSource.fromLongField("field");
+LongRangeFacetCutter longRangeFacetCutter =
+LongRangeFacetCutter.create("field", valuesSource, inputR

Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13568:
URL: https://github.com/apache/lucene/pull/13568#discussion_r1678246773


##
lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestRangeFacet.java:
##
@@ -0,0 +1,1654 @@
+/*
+ * 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.sandbox.facet;
+
+import static org.apache.lucene.facet.FacetsConfig.DEFAULT_INDEX_FIELD_NAME;
+
+import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
+import java.io.IOException;
+import java.util.List;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleDocValuesField;
+import org.apache.lucene.document.DoublePoint;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedNumericDocValuesField;
+import org.apache.lucene.facet.DrillDownQuery;
+import org.apache.lucene.facet.DrillSideways;
+import org.apache.lucene.facet.FacetField;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.FacetsConfig;
+import org.apache.lucene.facet.LabelAndValue;
+import org.apache.lucene.facet.MultiDoubleValuesSource;
+import org.apache.lucene.facet.MultiLongValuesSource;
+import org.apache.lucene.facet.range.DoubleRange;
+import org.apache.lucene.facet.range.LongRange;
+import org.apache.lucene.facet.range.Range;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
+import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.sandbox.facet.abstracts.OrdLabelBiMap;
+import org.apache.lucene.sandbox.facet.ranges.DoubleRangeFacetCutter;
+import org.apache.lucene.sandbox.facet.ranges.LongRangeFacetCutter;
+import org.apache.lucene.sandbox.facet.ranges.RangeOrdLabelBiMap;
+import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder;
+import org.apache.lucene.sandbox.facet.taxonomy.TaxonomyFacetsCutter;
+import org.apache.lucene.search.CollectorOwner;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LongValuesSource;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.MultiCollectorManager;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.tests.index.RandomIndexWriter;
+import org.apache.lucene.tests.search.DummyTotalHitCountCollector;
+import org.apache.lucene.tests.util.TestUtil;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * Test sandbox facet ranges. Mostly test cases from LongRangeFacetCounts 
adopted for sandbox
+ * faceting.
+ */
+public class TestRangeFacet extends SandboxFacetTestCase {
+
+  public void testBasicLong() throws Exception {
+Directory d = newDirectory();
+RandomIndexWriter w = new RandomIndexWriter(random(), d);
+Document doc = new Document();
+NumericDocValuesField field = new NumericDocValuesField("field", 0L);
+doc.add(field);
+for (long l = 0; l < 100; l++) {
+  field.setLongValue(l);
+  w.addDocument(doc);
+}
+
+// Also add Long.MAX_VALUE
+field.setLongValue(Long.MAX_VALUE);
+w.addDocument(doc);
+
+IndexReader r = w.getReader();
+w.close();
+
+IndexSearcher s = newSearcher(r);
+LongRange[] inputRanges =
+new LongRange[] {
+  new LongRange("less than 10", 0L, true, 10L, false),
+  new LongRange("less than or equal to 10", 0L, true, 10L, true),
+  new LongRange("over 90", 90L, false, 100L, false),
+  new LongRange("90 or above", 90L, true, 100L, false),
+  new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, true),
+};
+
+MultiLongValuesSource valuesSource = 
MultiLongValuesSource.fromLongField("field");
+LongRangeFacetCutter longRangeFacetCutter =
+LongRangeFacetCutter.create("field", valuesSource, inputR

Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13568:
URL: https://github.com/apache/lucene/pull/13568#discussion_r1678247836


##
lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestRangeFacet.java:
##
@@ -0,0 +1,1654 @@
+/*
+ * 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.sandbox.facet;
+
+import static org.apache.lucene.facet.FacetsConfig.DEFAULT_INDEX_FIELD_NAME;
+
+import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
+import java.io.IOException;
+import java.util.List;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleDocValuesField;
+import org.apache.lucene.document.DoublePoint;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedNumericDocValuesField;
+import org.apache.lucene.facet.DrillDownQuery;
+import org.apache.lucene.facet.DrillSideways;
+import org.apache.lucene.facet.FacetField;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.FacetsConfig;
+import org.apache.lucene.facet.LabelAndValue;
+import org.apache.lucene.facet.MultiDoubleValuesSource;
+import org.apache.lucene.facet.MultiLongValuesSource;
+import org.apache.lucene.facet.range.DoubleRange;
+import org.apache.lucene.facet.range.LongRange;
+import org.apache.lucene.facet.range.Range;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
+import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.sandbox.facet.abstracts.OrdLabelBiMap;
+import org.apache.lucene.sandbox.facet.ranges.DoubleRangeFacetCutter;
+import org.apache.lucene.sandbox.facet.ranges.LongRangeFacetCutter;
+import org.apache.lucene.sandbox.facet.ranges.RangeOrdLabelBiMap;
+import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder;
+import org.apache.lucene.sandbox.facet.taxonomy.TaxonomyFacetsCutter;
+import org.apache.lucene.search.CollectorOwner;
+import org.apache.lucene.search.DoubleValues;
+import org.apache.lucene.search.DoubleValuesSource;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LongValuesSource;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.MultiCollectorManager;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.tests.index.RandomIndexWriter;
+import org.apache.lucene.tests.search.DummyTotalHitCountCollector;
+import org.apache.lucene.tests.util.TestUtil;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * Test sandbox facet ranges. Mostly test cases from LongRangeFacetCounts 
adopted for sandbox
+ * faceting.
+ */
+public class TestRangeFacet extends SandboxFacetTestCase {
+
+  public void testBasicLong() throws Exception {
+Directory d = newDirectory();
+RandomIndexWriter w = new RandomIndexWriter(random(), d);
+Document doc = new Document();
+NumericDocValuesField field = new NumericDocValuesField("field", 0L);
+doc.add(field);
+for (long l = 0; l < 100; l++) {
+  field.setLongValue(l);
+  w.addDocument(doc);
+}
+
+// Also add Long.MAX_VALUE
+field.setLongValue(Long.MAX_VALUE);
+w.addDocument(doc);
+
+IndexReader r = w.getReader();
+w.close();
+
+IndexSearcher s = newSearcher(r);
+LongRange[] inputRanges =
+new LongRange[] {
+  new LongRange("less than 10", 0L, true, 10L, false),
+  new LongRange("less than or equal to 10", 0L, true, 10L, true),
+  new LongRange("over 90", 90L, false, 100L, false),
+  new LongRange("90 or above", 90L, true, 100L, false),
+  new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, true),
+};
+
+MultiLongValuesSource valuesSource = 
MultiLongValuesSource.fromLongField("field");
+LongRangeFacetCutter longRangeFacetCutter =
+LongRangeFacetCutter.create("field", valuesSource, inputR

Re: [PR] Compute facets while collecting [lucene]

2024-07-15 Thread via GitHub


epotyom commented on code in PR #13568:
URL: https://github.com/apache/lucene/pull/13568#discussion_r1678265251


##
lucene/sandbox/src/test/org/apache/lucene/sandbox/facet/TestTaxonomyFacet.java:
##
@@ -0,0 +1,163 @@
+/*
+ * 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.sandbox.facet;
+
+import static org.apache.lucene.facet.FacetsConfig.DEFAULT_INDEX_FIELD_NAME;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.facet.DrillDownQuery;
+import org.apache.lucene.facet.FacetField;
+import org.apache.lucene.facet.FacetsConfig;
+import org.apache.lucene.facet.LabelAndValue;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
+import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder;
+import org.apache.lucene.sandbox.facet.taxonomy.TaxonomyFacetsCutter;
+import org.apache.lucene.sandbox.facet.taxonomy.TaxonomyOrdLabelBiMap;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.tests.index.RandomIndexWriter;
+import org.apache.lucene.util.IOUtils;
+
+/** Test for associations */
+public class TestTaxonomyFacet extends SandboxFacetTestCase {
+
+  public void testConstants() {
+// It is essential for TaxonomyOrdLabelBiMap that invalid ordinal is the 
same as for
+// TaxonomyReader
+assertEquals(TaxonomyOrdLabelBiMap.INVALID_ORD, 
TaxonomyReader.INVALID_ORDINAL);

Review Comment:
   I agree, at the same time I'm not sure what what would be a better way to 
implement it? I do think that we need different constants for that as they have 
different scope - `OrdLabelBiMap.INVALID_ORD` is invalid ordinal for any facet 
implementation for sandbox facets, and `TaxonomyReader.INVALID_ORDINAL` is 
invalid ordinal for taxonomy facets only, for the main `facets` module.
   
   At the same time I think it is wasteful to implement TaxonomyOrdLabelBiMap 
as if these constants were different.
   
   But I'm open for suggestions!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


dsmiley commented on code in PR #13570:
URL: https://github.com/apache/lucene/pull/13570#discussion_r1678448829


##
lucene/core/src/java/org/apache/lucene/store/MMapDirectory.java:
##
@@ -199,20 +202,27 @@ public IndexInput openInput(String name, IOContext 
context) throws IOException {
 ensureOpen();
 ensureCanRead(name);
 Path path = directory.resolve(name);
-return PROVIDER.openInput(path, context, chunkSizePower, 
preload.test(name, context));
+return PROVIDER.openInput(
+path, context, chunkSizePower, preload.test(name, context), 
attachment);
   }
 
   // visible for tests:
-  static final MMapIndexInputProvider PROVIDER;
+  static final MMapIndexInputProvider PROVIDER;
 
-  interface MMapIndexInputProvider {
-IndexInput openInput(Path path, IOContext context, int chunkSizePower, 
boolean preload)
+  interface MMapIndexInputProvider {
+IndexInput openInput(
+Path path, IOContext context, int chunkSizePower, boolean preload, A 
attachment)
 throws IOException;
 
 long getDefaultMaxChunkSize();
 
 boolean supportsMadvise();
 
+/** An optional attachment of the provider, that will be passed to 
openInput. */

Review Comment:
   Without looking at the actual implementation, it wasn't clear what 
"attachment" is, or what's expected of this method.  Apparently this should 
return a *new* "attachment", and it's called on MMapDirectory's constructor 
(thus once, not per openInput).  Renaming "createX" would clarify.  
   
   Are we being overly generic?  We can call it what it is -- arenaBySegment.  



##
lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInputProvider.java:
##
@@ -125,4 +134,31 @@ private final MemorySegment[] map(
 }
 return segments;
   }
+
+  public ConcurrentHashMap attachment() {

Review Comment:
   hm; I'd expect an Override annotation



##
lucene/core/src/java21/org/apache/lucene/store/RefCountedSharedArena.java:
##
@@ -0,0 +1,95 @@
+/*
+ * 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.store;
+
+import java.lang.foreign.Arena;
+import java.lang.foreign.MemorySegment;
+import java.util.concurrent.atomic.AtomicInteger;
+
+@SuppressWarnings("preview")
+final class RefCountedSharedArena implements Arena {
+
+  static final int OPEN = 0;
+  static final int CLOSED = -1;
+
+  private final String segmentName;
+  private final Runnable removeFromMap;

Review Comment:
   I would call this onClose



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


dsmiley commented on code in PR #13570:
URL: https://github.com/apache/lucene/pull/13570#discussion_r1678459190


##
lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInputProvider.java:
##
@@ -125,4 +134,31 @@ private final MemorySegment[] map(
 }
 return segments;
   }
+
+  public ConcurrentHashMap attachment() {
+return new ConcurrentHashMap<>();
+  }
+
+  /**
+   * Gets an arena for the give path, potentially aggregating files from the 
same segment into a
+   * single ref counted shared arena. A ref counted shared arena, if created 
will be added to the
+   * given arenas map.
+   */
+  static Arena getSharedArena(Path p, ConcurrentHashMap arenas) {
+String filename = p.getFileName().toString();
+String segmentName = IndexFileNames.parseSegmentName(filename);
+if (filename.length() == segmentName.length()) {
+  // no segment found; just use a shared segment
+  return Arena.ofShared();

Review Comment:
   A global shared arena seems dangerous.  Shouldn't we _at least_ have an 
Arena instance for the Index (i.e. for the Directory instance?).  You could use 
the empty string as a key into the map, accomplishing this trivially.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [I] Examine performance of individual data accessor methods of MemorySegmentIndexInput when IndexInputs are closed in other threads (deoptimizations,...) [lucene]

2024-07-15 Thread via GitHub


dsmiley commented on issue #13325:
URL: https://github.com/apache/lucene/issues/13325#issuecomment-2229550743

   What does "refreshing indexes" mean exactly?  Solr does not have special 
threads for opening/closing indexes; they will typically happen on the common 
Jetty request thread pool.  If the suggestion is _intended_ to ensure that only 
one thread open/close an index, I don't see why that would be a "maintenance 
task" but I do see it could easily be done simply by ensuring only one thread 
does this work on-demand.  But I'd be concerned indexes would open slowly if 
you have many.  Any way; feels like a work-around to the JDK bug and not 
something desirable in its own right.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Add a clear button for log content [lucene]

2024-07-15 Thread via GitHub


github-actions[bot] commented on PR #13516:
URL: https://github.com/apache/lucene/pull/13516#issuecomment-2229712938

   This PR has not had activity in the past 2 weeks, labeling it as stale. If 
the PR is waiting for review, notify the d...@lucene.apache.org list. Thank you 
for your contribution!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Simplifying text area stream in Luke- ticket 12809 [lucene]

2024-07-15 Thread via GitHub


github-actions[bot] commented on PR #12869:
URL: https://github.com/apache/lucene/pull/12869#issuecomment-2229714723

   This PR has not had activity in the past 2 weeks, labeling it as stale. If 
the PR is waiting for review, notify the d...@lucene.apache.org list. Thank you 
for your contribution!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



[PR] New JMH benchmark method - vdot8s that implement int8 dotProduct in C… [lucene]

2024-07-15 Thread via GitHub


goankur opened a new pull request, #13572:
URL: https://github.com/apache/lucene/pull/13572

   
    Credit: 
https://www.elastic.co/search-labs/blog/vector-similarity-computations-ludicrous-speed
   
   Implement vectorized dot product in native C code using Neon intrinsics
   
   ### Description
   
   `DO NOT MERGE`
   Rough implementation created for discussion with Lucene community. 
   
   `TODO:` Add more notes 
   
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [I] Significant drop in recall for 8 bit Scalar Quantizer [lucene]

2024-07-15 Thread via GitHub


MilindShyani commented on issue #13519:
URL: https://github.com/apache/lucene/issues/13519#issuecomment-2229864587

   Hi @benwtrent, I am trying to carefully check the math vs code. The first 
thing I noticed is that my definitions of dxs and dxq don't necessarily match 
with yours. That might be the origin for the sign error. I will uniformize the 
notation and get back.
   
   But meanwhile, in the latest code 
https://github.com/apache/lucene/compare/main...benwtrent:fix-8-bit?expand=1 I 
find that 
   `float dxs = scale * dxc;` but scale is defined as 127/max-min. Shouldn't 
the 127 actually be 255?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] New JMH benchmark method - vdot8s that implement int8 dotProduct in C… [lucene]

2024-07-15 Thread via GitHub


rmuir commented on PR #13572:
URL: https://github.com/apache/lucene/pull/13572#issuecomment-2230045004

   Do we even need to use intrinsics? function is so simple that the compiler 
seems to do the right thing, e.g. use `SDOT` dot production instruction, given 
the correct flags:
   
   https://godbolt.org/z/KG1dPnrqn
   
   
https://developer.arm.com/documentation/102651/a/What-are-dot-product-intructions-


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] New JMH benchmark method - vdot8s that implement int8 dotProduct in C… [lucene]

2024-07-15 Thread via GitHub


rmuir commented on PR #13572:
URL: https://github.com/apache/lucene/pull/13572#issuecomment-2230061569

   I haven't benchmarked, just seems `SDOT` is the one to optimize for, and GCC 
can both recognize the code shape and autovectorize to it without hassle. 
   
   my cheap 2021 phone has `asimddp` feature in /proc/cpuinfo, dot product 
support seems widespread.
   
   You can use it directly via intrinsic, too, no need to use add/multiply 
intrinsic: 
https://arm-software.github.io/acle/neon_intrinsics/advsimd.html#dot-product
   
   But unless it is really faster than what GCC does with simple C, no need.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] New JMH benchmark method - vdot8s that implement int8 dotProduct in C… [lucene]

2024-07-15 Thread via GitHub


rmuir commented on code in PR #13572:
URL: https://github.com/apache/lucene/pull/13572#discussion_r1678790088


##
lucene/core/build.gradle:
##
@@ -14,10 +14,43 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+plugins {
+  id "c"
+}
 
 apply plugin: 'java-library'
+apply plugin: 'c'
 
 description = 'Lucene core library'
+model {
+  toolChains {
+gcc(Gcc) {
+  target("linux_aarch64"){
+cppCompiler.withArguments { args ->
+  args << "-O3 --shared"

Review Comment:
   I think here is the problem, pass `-march` so it can use NEON/SVE dot 
product instructions for a more interesting comparison



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] New JMH benchmark method - vdot8s that implement int8 dotProduct in C… [lucene]

2024-07-15 Thread via GitHub


rmuir commented on code in PR #13572:
URL: https://github.com/apache/lucene/pull/13572#discussion_r1678799350


##
lucene/core/build.gradle:
##
@@ -14,10 +14,43 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+plugins {
+  id "c"
+}
 
 apply plugin: 'java-library'
+apply plugin: 'c'
 
 description = 'Lucene core library'
+model {
+  toolChains {
+gcc(Gcc) {
+  target("linux_aarch64"){
+cppCompiler.withArguments { args ->
+  args << "-O3 --shared"

Review Comment:
   try something like `-O3 -march=armv8.4-a -funroll-loops` to test NEON. The 
documentation says that cortex/neoverse can do 4 of these in parallel, and GCC 
appears at a glance to unroll the SDOTs: https://godbolt.org/z/drbvW3TE1



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


uschindler commented on code in PR #13570:
URL: https://github.com/apache/lucene/pull/13570#discussion_r1678835899


##
lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInputProvider.java:
##
@@ -125,4 +134,31 @@ private final MemorySegment[] map(
 }
 return segments;
   }
+
+  public ConcurrentHashMap attachment() {
+return new ConcurrentHashMap<>();
+  }
+
+  /**
+   * Gets an arena for the give path, potentially aggregating files from the 
same segment into a
+   * single ref counted shared arena. A ref counted shared arena, if created 
will be added to the
+   * given arenas map.
+   */
+  static Arena getSharedArena(Path p, ConcurrentHashMap arenas) {
+String filename = p.getFileName().toString();
+String segmentName = IndexFileNames.parseSegmentName(filename);
+if (filename.length() == segmentName.length()) {
+  // no segment found; just use a shared segment
+  return Arena.ofShared();

Review Comment:
   This is a new Arena so it's private to that file. It's basically a fall ack 
to current behaviour.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [PR] Aggregate files from the same segment into a single Arena [lucene]

2024-07-15 Thread via GitHub


uschindler commented on code in PR #13570:
URL: https://github.com/apache/lucene/pull/13570#discussion_r1678835899


##
lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInputProvider.java:
##
@@ -125,4 +134,31 @@ private final MemorySegment[] map(
 }
 return segments;
   }
+
+  public ConcurrentHashMap attachment() {
+return new ConcurrentHashMap<>();
+  }
+
+  /**
+   * Gets an arena for the give path, potentially aggregating files from the 
same segment into a
+   * single ref counted shared arena. A ref counted shared arena, if created 
will be added to the
+   * given arenas map.
+   */
+  static Arena getSharedArena(Path p, ConcurrentHashMap arenas) {
+String filename = p.getFileName().toString();
+String segmentName = IndexFileNames.parseSegmentName(filename);
+if (filename.length() == segmentName.length()) {
+  // no segment found; just use a shared segment
+  return Arena.ofShared();

Review Comment:
   This is a new Arena so it's private to that file. It's basically a fallback 
to current behaviour without PR 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [I] Examine performance of individual data accessor methods of MemorySegmentIndexInput when IndexInputs are closed in other threads (deoptimizations,...) [lucene]

2024-07-15 Thread via GitHub


uschindler commented on issue #13325:
URL: https://github.com/apache/lucene/issues/13325#issuecomment-2230135534

   It is not a JDK bug, it is expected behaviour. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org



Re: [I] Examine performance of individual data accessor methods of MemorySegmentIndexInput when IndexInputs are closed in other threads (deoptimizations,...) [lucene]

2024-07-15 Thread via GitHub


uschindler commented on issue #13325:
URL: https://github.com/apache/lucene/issues/13325#issuecomment-2230139147

   Basically the idea is to close indexes asynchronous in a single thread doing 
this to not block main threads.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org