Skip to content

Commit

Permalink
Uplift Lucene to 4.10.4
Browse files Browse the repository at this point in the history
Update Clouseau to use the latest available Lucene version from
the 4.x series.  This commit brings many bug fixes while it
maintains the backward compatibility for the existing indexes.
4.10.4 has 45 API changes and 116 bug fixes since 4.6.1, for the
details see the change log [1].

[1] https://lucene.apache.org/core/4_10_4/changes/Changes.html
  • Loading branch information
pgj committed Jun 2, 2023
1 parent c59127d commit 29df361
Show file tree
Hide file tree
Showing 7 changed files with 134 additions and 138 deletions.
2 changes: 1 addition & 1 deletion pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@

<properties>
<encoding>UTF-8</encoding>
<lucene-version>4.6.1</lucene-version>
<lucene-version>4.10.4</lucene-version>
<maven.compiler.source>1.6</maven.compiler.source>
<maven.compiler.target>1.6</maven.compiler.target>
<scala.version>2.9.1</scala.version>
Expand Down
36 changes: 24 additions & 12 deletions src/main/scala/com/cloudant/clouseau/ClouseauTypeFactory.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,15 +17,15 @@ import org.slf4j.Logger
import org.slf4j.LoggerFactory
import org.apache.lucene.document.Field._
import org.apache.lucene.document._
import org.apache.lucene.facet.FacetsConfig
import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetField
import org.apache.lucene.index.FieldInfo.IndexOptions
import org.apache.lucene.search._
import org.apache.lucene.util.BytesRef
import scala.collection.immutable.Map
import scala.collection.JavaConversions._
import scalang._
import org.jboss.netty.buffer.ChannelBuffer
import org.apache.lucene.util.BytesRef
import org.apache.lucene.facet.params.FacetIndexingParams
import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetFields
import org.apache.lucene.facet.taxonomy.CategoryPath
import scala.collection.mutable.ArrayBuffer

case class SearchRequest(options: Map[Symbol, Any])
Expand All @@ -52,6 +52,22 @@ object ClouseauTypeFactory extends TypeFactory {

val logger = LoggerFactory.getLogger("clouseau.tf")

// Default numeric precision step changed in Lucene 4.9, this
// wrapper is to help to preserve the original setting for
// compatibility reasons :-(
def maybeDoubleStored(stored: Store) = {
val result = new FieldType()
result.setIndexed(true)
result.setTokenized(true)
result.setOmitNorms(true)
result.setIndexOptions(IndexOptions.DOCS_ONLY)
result.setNumericType(FieldType.NumericType.DOUBLE)
result.setNumericPrecisionStep(8)
result.setStored(stored == Store.YES)
result.freeze()
result
}

def createType(name: Symbol, arity: Int, reader: TermReader): Option[Any] = (name, arity) match {
case ('open, 4) =>
Some(OpenIndexMsg(reader.readAs[Pid], reader.readAs[String], reader.readTerm))
Expand Down Expand Up @@ -113,7 +129,8 @@ object ClouseauTypeFactory extends TypeFactory {
for (field <- reader.readAs[List[Any]]) {
addFields(result, field)
}
result
val fc = new FacetsConfig()
fc.build(result)
}

private def addFields(doc: Document, field0: Any): Unit = field0 match {
Expand All @@ -129,12 +146,7 @@ object ClouseauTypeFactory extends TypeFactory {
}
doc.add(field)
if (isFacet(map) && value.nonEmpty) {
val fp = FacetIndexingParams.DEFAULT
val delim = fp.getFacetDelimChar
if (!name.contains(delim) && !value.contains(delim)) {
val facets = new SortedSetDocValuesFacetFields(fp)
facets.addFields(doc, List(new CategoryPath(name, value)))
}
doc.add(new SortedSetDocValuesFacetField(name, value))
}
case None =>
'ok
Expand All @@ -151,7 +163,7 @@ object ClouseauTypeFactory extends TypeFactory {
val map = options.toMap
toDouble(value) match {
case Some(doubleValue) =>
doc.add(new DoubleField(name, doubleValue, toStore(map)))
doc.add(new DoubleField(name, doubleValue, maybeDoubleStored(toStore(map))))
if (isFacet(map)) {
doc.add(new DoubleDocValuesField(name, doubleValue))
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import scalang._
import com.yammer.metrics.scala._
import scala.collection.JavaConversions._
import org.apache.lucene.search.FieldCache
import org.apache.lucene.util.RamUsageEstimator
import org.apache.lucene.util.Accountable

class IndexManagerService(ctx: ServiceContext[ConfigurationArgs]) extends Service(ctx) with Instrumented {

Expand Down Expand Up @@ -101,7 +101,9 @@ class IndexManagerService(ctx: ServiceContext[ConfigurationArgs]) extends Servic
val fieldCache = FieldCache.DEFAULT
var result = 0L
for (cacheEntry <- fieldCache.getCacheEntries) {
result += RamUsageEstimator.sizeOf(cacheEntry.getValue())
val value = cacheEntry.getValue().asInstanceOf[Accountable]
val bytesUsed = if (value == null) 0 else value.ramBytesUsed()
result += bytesUsed
}
result
}
Expand Down
178 changes: 77 additions & 101 deletions src/main/scala/com/cloudant/clouseau/IndexService.scala
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import org.apache.lucene.util.BytesRef
import org.apache.lucene.util.Version
import org.apache.lucene.search.IndexSearcher
import org.apache.lucene.search.BooleanClause.Occur
import org.apache.lucene.search.MultiCollector
import org.apache.lucene.queryparser.classic.QueryParser
import org.apache.lucene.queryparser.classic.ParseException
import org.apache.lucene.search.highlight.{
Expand All @@ -40,18 +41,11 @@ import collection.JavaConversions._
import com.yammer.metrics.scala._
import com.cloudant.clouseau.Utils._
import org.apache.commons.configuration.Configuration
import org.apache.lucene.facet.sortedset.{
SortedSetDocValuesReaderState,
SortedSetDocValuesAccumulator
}
import org.apache.lucene.facet.range.{
DoubleRange,
RangeAccumulator,
RangeFacetRequest
}
import org.apache.lucene.facet.search._
import org.apache.lucene.facet.DrillDownQuery
import org.apache.lucene.facet.{ Facets, FacetsCollector, FacetsConfig, FacetResult }
import org.apache.lucene.facet.sortedset.{ DefaultSortedSetDocValuesReaderState, SortedSetDocValuesFacetCounts }
import org.apache.lucene.facet.range.{ DoubleRange, DoubleRangeFacetCounts }
import org.apache.lucene.facet.taxonomy.CategoryPath
import org.apache.lucene.facet.params.{ FacetIndexingParams, FacetSearchParams }
import scala.Some
import scalang.Pid
import com.spatial4j.core.context.SpatialContext
Expand Down Expand Up @@ -302,27 +296,19 @@ class IndexService(ctx: ServiceContext[IndexServiceArgs]) extends Service(ctx) w
baseQuery
case categories: List[List[String]] =>
val drilldownQuery = new DrillDownQuery(
FacetIndexingParams.DEFAULT, baseQuery)
new FacetsConfig, baseQuery)
for (category <- categories) {
val category1 = category.toArray
val len = category1.length
try {
if (len < 3) {
drilldownQuery.add(new CategoryPath(category1: _*))
} else { //if there are multiple values OR'd them, delete this else part after updating to Apache Lucene > 4.6
val dim = category1(0)
val categoryPaths: Array[CategoryPath] = new Array[CategoryPath](len - 1)
for (i <- 1 until len) {
categoryPaths(i - 1) = new CategoryPath(Array(dim, category1(i)): _*)
}
drilldownQuery.add(categoryPaths: _*)
val dim = category.head
for (path <- category.tail) {
try {
drilldownQuery.add(dim, path)
} catch {
case e: IllegalArgumentException =>
throw new ParseException(e.getMessage)
case e: ArrayStoreException =>
throw new ParseException(category +
" contains a non-string item")
}
} catch {
case e: IllegalArgumentException =>
throw new ParseException(e.getMessage)
case e: ArrayStoreException =>
throw new ParseException(category +
" contains a non-string item")
}
}
drilldownQuery
Expand Down Expand Up @@ -352,39 +338,9 @@ class IndexService(ctx: ServiceContext[IndexServiceArgs]) extends Service(ctx) w
false, docsScoredInOrder)
}

val countsCollector = createCountsCollector(counts)

val rangesCollector = ranges match {
case None =>
null
case Some(rangeList: List[_]) =>
val rangeFacetRequests = for ((name: String, ranges: List[_]) <- rangeList) yield {
new RangeFacetRequest(name, ranges.map({
case (label: String, rangeQuery: String) =>
ctx.args.queryParser.parse(rangeQuery) match {
case q: NumericRangeQuery[_] =>
new DoubleRange(
label,
ClouseauTypeFactory.toDouble(q.getMin).get,
q.includesMin,
ClouseauTypeFactory.toDouble(q.getMax).get,
q.includesMax)
case _ =>
throw new ParseException(rangeQuery +
" was not a well-formed range specification")
}
case _ =>
throw new ParseException("invalid ranges query")
}))
}
val acc = new RangeAccumulator(rangeFacetRequests)
FacetsCollector.create(acc)
case Some(other) =>
throw new ParseException(other + " is not a valid ranges query")
}

val facetsCollector = new FacetsCollector()
val collector = MultiCollector.wrap(
hitsCollector, countsCollector, rangesCollector)
facetsCollector, hitsCollector)

searchTimer.time {
partition match {
Expand Down Expand Up @@ -412,12 +368,14 @@ class IndexService(ctx: ServiceContext[IndexServiceArgs]) extends Service(ctx) w
if (legacy) {
('ok, TopDocs(updateSeq, getTotalHits(hitsCollector), hits))
} else {
val countsFacets = pullCountsFacets(counts, facetsCollector)
val rangesFacets = pullRangesFacets(ranges, facetsCollector)
('ok, List(
('update_seq, updateSeq),
('total_hits, getTotalHits(hitsCollector)),
('hits, hits)
) ++ convertFacets('counts, countsCollector)
++ convertFacets('ranges, rangesCollector))
) ++ convertFacets('counts, countsFacets)
++ convertFacets('ranges, rangesFacets))
}
}
case error =>
Expand All @@ -441,36 +399,50 @@ class IndexService(ctx: ServiceContext[IndexServiceArgs]) extends Service(ctx) w
Nil
}

private def createCountsCollector(counts: Option[Any]): FacetsCollector = {
private def pullCountsFacets(counts: Option[Any], collector: FacetsCollector): Map[String, Facets] = {
counts match {
case None =>
null
case Some(counts: List[String]) =>
val state = try {
new SortedSetDocValuesReaderState(reader)
} catch {
case e: IllegalArgumentException =>
if (e.getMessage contains "was not indexed with SortedSetDocValues")
return null
else
throw e
}
val countFacetRequests = for (count <- counts) yield {
new CountFacetRequest(new CategoryPath(count), Int.MaxValue)
}
val facetSearchParams = new FacetSearchParams(countFacetRequests)
val acc = try {
new SortedSetDocValuesAccumulator(state, facetSearchParams)
} catch {
case e: IllegalArgumentException =>
throw new ParseException(e.getMessage)
}
FacetsCollector.create(acc)
Map.empty
case Some(countList: List[String]) =>
(for (count <- countList) yield {
val state = new DefaultSortedSetDocValuesReaderState(reader)
val facetCounts = new SortedSetDocValuesFacetCounts(state, collector)
(count, facetCounts)
}).toMap
case Some(other) =>
throw new ParseException(other + " is not a valid counts query")
}
}

private def pullRangesFacets(ranges: Option[Any], collector: FacetsCollector): Map[String, Facets] = {
ranges match {
case None =>
Map.empty
case Some(rangeList: List[_]) =>
(for ((name: String, ranges: List[_]) <- rangeList) yield {
val rangeCounts = new DoubleRangeFacetCounts(name, collector, ranges.map({
case (label: String, rangeQuery: String) =>
ctx.args.queryParser.parse(rangeQuery) match {
case q: NumericRangeQuery[_] =>
new DoubleRange(
label,
ClouseauTypeFactory.toDouble(q.getMin).get,
q.includesMin,
ClouseauTypeFactory.toDouble(q.getMax).get,
q.includesMax)
case _ =>
throw new ParseException(rangeQuery + " was not a well-formed range specification")
}
case _ =>
throw new ParseException("invalid ranges query")
}).toArray: _*)
(name, rangeCounts)
}).toMap
case Some(other) =>
throw new ParseException(other + " is not a valid ranges query")
}
}

private def group1(queryString: String, field: String, refresh: Boolean, groupSort: Any,
groupOffset: Int, groupLimit: Int): Any = parseQuery(queryString, None) match {
case query: Query =>
Expand Down Expand Up @@ -633,7 +605,8 @@ class IndexService(ctx: ServiceContext[IndexServiceArgs]) extends Service(ctx) w
case e: ParseException =>
('error, ('bad_request, e.getMessage))
case e =>
('error, e.getMessage)
val stackTrace = (for (t <- e.getStackTrace()) yield t.toString()).mkString("\n")
('error, (e.getClass, e.getMessage, stackTrace))
}

private def getSearcher(refresh: Boolean): IndexSearcher = {
Expand Down Expand Up @@ -789,20 +762,23 @@ class IndexService(ctx: ServiceContext[IndexServiceArgs]) extends Service(ctx) w
throw new ParseException("Unrecognized sort parameter: " + field)
}

private def convertFacets(name: Symbol, c: FacetsCollector): List[_] = c match {
case null =>
private def convertFacets(name: Symbol, facetsMap: Map[String, Facets]): List[_] = {
if (facetsMap.isEmpty) {
Nil
case _ =>
List((name, c.getFacetResults.map { f => convertFacet(f) }.toList))
}

private def convertFacet(facet: FacetResult): Any = {
convertFacetNode(facet.getFacetResultNode)
}

private def convertFacetNode(node: FacetResultNode): Any = {
val children = node.subResults.map { n => convertFacetNode(n) }.toList
(node.label.components.toList, node.value, children)
} else {
val facetResults = (for ((dim, facets) <- facetsMap if facets != null) yield {
val topChildren = facets.getTopChildren(10, dim)
val results = if (topChildren != null) {
(for (labelValue <- topChildren.labelValues) yield {
(List(dim, labelValue.label), labelValue.value, List())
}).toList
} else {
List()
}
(List(dim), 0, results)
}).toList
List((name, facetResults))
}
}

private def toScoreDoc(sort: Sort, after: Any): Option[ScoreDoc] = after match {
Expand Down
4 changes: 2 additions & 2 deletions src/main/scala/com/cloudant/clouseau/SupportedAnalyzers.scala
Original file line number Diff line number Diff line change
Expand Up @@ -285,9 +285,9 @@ object SupportedAnalyzers {
case "japanese" =>
options.get("stopwords") match {
case Some(stopwords: List[String]) =>
Some(new JapaneseAnalyzer(IndexService.version, null, JapaneseTokenizer.DEFAULT_MODE, stopwords, JapaneseAnalyzer.getDefaultStopTags))
Some(new JapaneseAnalyzer(null, JapaneseTokenizer.DEFAULT_MODE, stopwords, JapaneseAnalyzer.getDefaultStopTags))
case _ =>
Some(new JapaneseAnalyzer(IndexService.version))
Some(new JapaneseAnalyzer())
}
case "latvian" =>
options.get("stopwords") match {
Expand Down
8 changes: 4 additions & 4 deletions src/main/scala/com/cloudant/clouseau/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -13,16 +13,16 @@
package com.cloudant.clouseau

import org.apache.lucene.index.Term
import org.apache.lucene.util.BytesRef
import org.apache.lucene.util.{ BytesRef, BytesRefBuilder }
import org.apache.lucene.util.NumericUtils

object Utils {

def doubleToTerm(field: String, value: Double): Term = {
val bytesRef = new BytesRef
val bytesRefBuilder = new BytesRefBuilder
val asLong = NumericUtils.doubleToSortableLong(value)
NumericUtils.longToPrefixCoded(asLong, 0, bytesRef)
new Term(field, bytesRef)
NumericUtils.longToPrefixCoded(asLong, 0, bytesRefBuilder)
new Term(field, bytesRefBuilder.get())
}

implicit def stringToBytesRef(string: String): BytesRef = {
Expand Down
Loading

0 comments on commit 29df361

Please sign in to comment.