関口宏司のLuceneブログ

OSS検索ライブラリのLuceneおよびそのサブプロジェクト(Solr/Tika/Mahoutなど)について
LUCENE-6819: Good bye index-time boost

Lucene/Solr 7.0 のリリースの投票がまもなく始まろうとしている。

今回私が気になっているのが、LUCENE-6819 による修正である。
Deprecate index-time boosts?
https://issues.apache.org/jira/browse/LUCENE-6819

タイトルだけ見ると、「インデクシング時ブースト(重み付け)を止めよう」、もっと深読みすると「クエリ時に重み付けが指定できるからインデクシング時に重み付けしなくてもいんじゃね?」ととらえてしまう方もいるかもしれない。もちろん、インデクシング時重み付けとクエリ時重み付けでは、大きく意味や目的が異なる。[1]

本記事では、LUCENE-6819 と関連する他の修正と合わせ、Lucene 7.0 以降で fieldNorm がどのようになるか、簡単に解説しよう。

LUCENE-6819

このチケットでは、タイトル通り、インデクシング時のboostを止めよう、というものである。もう少し細かく言うと、fieldNorm という、各ドキュメントの各フィールド(ただし、omitNorms=trueのフィールドを除く)ごとに1バイトでエンコーディングされた値の、次の式からboostを取り除こう、というものである。

fieldNorm = lengthNorm * boost

ちなみにこの式は、BM25Similarity が登場する以前の、Lucene/Solr の標準であったベクトル空間モデルによるランキング計算(各ドキュメントのクエリに対する類似度計算)に強く依存している。Lucene/Solr はコサイン類似度ベースだが、上のように計算された fieldNorm を加味しており、BM25 で考慮されている文書長はとっくに加味されている。[2]

さて、これからboostを除けば、次のようになる。

fieldNorm = lengthNorm

lengthNorm はデフォルトで次のように計算される。

lengthNorm = 1 / sqrt(numTerms)

numTermsは当該フィールドの単語数である。したがって、lengthNorm は単語数が1の時は 1 / sqrt(1) = 1 / 1 = 1、単語数が2の時は 1 / sqrt(2) = 1 / 1.414213562373095… = 0.70710678118655、という具合になるので、0 < lengthNorm <= 1 となる。しかし、Lucene 6.x 以前では、boost が lengthNorm にかかった fieldNorm を保存しなければならず、boost の値はユーザの設定次第なので最悪 Float.MAX_VALUE になってしまうことも考えられる。そこで 0 から Float.MAX_VALUE の広大な値域をわずか1バイトにマッピングしているのが Lucene 6.x である(下図)。

LUCENE-6819

しかし、ほとんどのユーザーはインデクシング時重み付けなどしないので、上図グレーの部分がもったいない。boostがかけられる可能性を考えて1バイトの半分を残しているのだが、ほとんどのユーザーはboost=1なので、非常にもったいない、かつ、低精度な使い方になっている。

そこで、LUCENE-6819 では:

  • Lucene 6.x では boost 設定と参照に係わる API は @deprecated とし、boostに1でない値が投入されようとすると、「将来バージョンで使えなくなる」旨の警告が表示される。
  • Lucene 7.0 では、boost 設定と参照に係わる API から boost 引数を取り除き、fieldNorm には lengthNorm そのものが記録されるようになった。
という修正が行われた。

LUCENE-7730

LUCENE-6819 ではまだ fieldNorm の1バイトが有効活用されていない。そこでこのチケットでは、フィールド長(単語数)をSmallFloat.intoTobyte4(numTerms)でエンコードして保存するように修正された。たとえば単語数が1000のときは、SmallFloat.intoTobyte4(1000)が87となり、これが記録される。デコードにはSmallFloat.byte4ToInt()が使われる。SmallFloat.byte4ToInt(87)は984となる。ちなみに、1〜40はエンコードしても1〜40を出力する(変わらない)。長さ41から徐々にエンコードによる誤差が大きくなる(下図)。

LUCENE-7730

Lucene 6.x では boost / sqrt(numTerms) の計算結果が1バイトに圧縮されて記録されていたが、Lucene 7.0以降では numTerms がそのままエンコードされて1バイトに記録される。もはや BM25Similarity が標準なので、1 / sqrt(numTerms) を計算する意味自体がない、ということだ。

Lucene/Solr 7.0 以降も omitNorms 指定は相変わらず有効である。

ロンウイットの社内勉強会で説明に使用したスライドも公開しているので、合わせて読んで欲しい。







[1] ロンウイット主催のトレーニングコース「Solr基礎」では、このような大事な基礎を丁寧に教えています。お申し込みはこちらから

[2] ロンウイット主催のトレーニングコース「Apache Mahout & Sparkではじめる機械学習」では、機械学習のさまざまなモデルやアルゴリズムを丁寧に解説しています。ベーシックな機械学習の解説にとどまらず、本記事にあるような、検索エンジンのランキングの元になるスコア計算の考え方なども含み、ソフトウェア開発の実際の現場で役に立つ内容を網羅しています。お申し込みはこちらから

| 関口宏司 | Luceneインデックス | 10:41 | comments(0) | trackbacks(0) |
1セグメント内のユニークターム数の上限の拡張(2.9.2)
これまで1セグメント内のユニークターム数は約21億個であったが、Lucene 2.9.2から約2740億件に拡張されることになった:

https://issues.apache.org/jira/browse/LUCENE-2257

インデックスファイルフォーマットにはこのような制限はないが、検索時に部分的にintでシーク位置の計算結果を保持している部分があったためintの上限の約21億が上限となっていたものである。これをlongで受け取るように変更された。longで受け取るようになったのに上限が900京でないのは、tiiのシーク位置を保持しているのはintのままで、そこからtisの位置を求めるのに128を掛けるのだが、その計算結果を受け取る部分がlongに拡張されたということであるため、上限が(intの上限)x128 = 約2740億件、となっている。

これまでこの制限に当たっていたサイトはセグメントをマージしないようにするくらいしか方法がなかったが、これでだいぶ運用が楽になるだろう。
| 関口宏司 | Luceneインデックス | 18:11 | comments(0) | trackbacks(0) |
SSDを効果的に使い検索を高速化する方法
http://old.nabble.com/Distributing-index-over-N-disks-td26504452.html

Lucene/Solrのインデックスが完成されたもので変更がない(追加/更新/削除がなく、マージもされない)ならば、SSDを用意して.frqファイルと.prxファイルを配置し、残りのインデックスファイルはHDDに配置する。そしてHDDのインデックスディレクトリにはSSDへのシンボリックリンクを配置する。

.frqファイルには各TermのDocumentごとの出現数が記録されている。.prxファイルにはさらにその詳細としてその出現毎のフィールドにおけるTermの位置情報が記録されている。いずれもomitTfがfalseのときに.fdtファイルに次いでサイズが大きく、また検索実行時に頻繁にアクセスされるため、これらのファイルを選択的にSSDに配置することで検索コストパフォーマンスを最大化できると期待できる。
| 関口宏司 | Luceneインデックス | 09:56 | comments(0) | trackbacks(0) |
インデックスの互換性(2.9〜3.0〜4.9)
Lucene 3.0で作成したインデックスはLucene 2.9では読めない。なぜならLucene 2.9でdeprecatedとなったStore.COMPRESSがLucene 3.0では削除され、インデックスのバージョン番号が上がったためである。

従来通りその逆はOKであり、Lucene 2.Xで作成したインデックスはLucene 3.0で検索に使うことができる。また、Lucene 3.0でインデックスを更新すると、新しいセグメントファイルは新フォーマットで作成される。

Lucene 3.0で作成したインデックスは、Lucene 4.9まで使える予定である。Lucene 2.9のインデックスはLucene 4.0になるとサポートされなくなる。

2.9でdeprecatedとなったStore.COMPRESSは、これまで次のように書いていたプログラムを:

doc.add( new Field( "body", bodyString, Store.COMPRESS, Index.ANALYZED ) );


次のようにCompressionToolsを使って書き換えればよい(2行必要になる):

doc.add( new Field( "body", bodyString, Store.NO, Index.ANALYZED ) );
doc.add( new Field( "body", CompressionTools.compressString(bodyString), Store.YES ) );
| 関口宏司 | Luceneインデックス | 09:29 | comments(0) | trackbacks(0) |
インデックスのサイズを見るシミュレーションプログラム
Luceneのインデックスは検索しながら更新していると、古いインデックスを参照しているSearcherがいる間はセグメントを消すことができない。そのため、大きなマージが走るときはハードディスクの容量を一時的に大量に消費する。

その様子(マージの部分のみ)をシミュレーションする(Luceneを一切使用しない)プログラムを作成した。プログラムを実行するとCSVファイルが出力される。それをEXCELで表示したのが下の図である:

インデックスのサイズ

EXCELで表示できるよう、この図ではマージ係数を3に設定している。プログラムは以下の通りである:

public class MergeSimulator {
  
  static final String CSV_FILE = "segments.csv";
  static final int MERGE_FACTOR = 3;
  static final int MAX_BUFFERED_DOCS = 3;
  static final int NUM_DOCS = 3000;

  public static void main( String[] args ) throws FileNotFoundException {
    Directory dir = new Directory( MERGE_FACTOR );
    PrintStream ps = new PrintStream( CSV_FILE );
    dir.logHeader( ps );
    Writer writer = new Writer( MAX_BUFFERED_DOCS, dir, ps );
    for( int i = 0; i < NUM_DOCS; i++ ){
      writer.add( new Doc() );
    }
    ps.close();
  }

  static class Doc {
  }

  static class Segment {
    private final int level;
    private final int numDocs;
    private boolean deleted;
    public void delete(){ deleted = true; }
    public boolean isDeleted(){ return deleted; }
    public Segment( int numDocs ){
      this( 0, numDocs );
    }
    public Segment( int level, int numDocs ){
      this.level = level;
      this.numDocs = numDocs;
    }
    public int getSize(){ return numDocs; }
    public int getLevel(){ return level; }
  }
  
  static class Directory {
    private final int mergeFactor;
    private List> segmentsList = new ArrayList>();
    public Directory( int mergeFactor ){
      this.mergeFactor = mergeFactor;
    }
    public List> getSegmentsList(){ return segmentsList; }
    public int getNumSegments(){
      int num = 0;
      for( List segments : segmentsList ){
        if( segments == null ) continue;
        for( Segment segment : segments ){
          num++;
        }
      }
      return num;
    }
    public int getSize(){
      int size = 0;
      for( List segments : segmentsList ){
        if( segments == null ) continue;
        for( Segment segment : segments ){
          size += segment.getSize();
        }
      }
      return size;
    }
    public void addSegment( Segment segment ){
      int level = segment.getLevel();
      if( segmentsList.size() <= level )
        segmentsList.add( new ArrayList() );
      List segments = segmentsList.get( level );
      segments.add( segment );
      if( needMerge( level ) )
        merge( level );
    }
    public void merge( int level ){
      List segmentsToBeMerged = segmentsList.get( level );
      int totalDocs = 0;
      for( Segment segment : segmentsToBeMerged ){
        totalDocs += segment.numDocs;
        segment.delete();
      }
      Segment mergedSegment = new Segment( level + 1, totalDocs );
      addSegment( mergedSegment );
    }
    public boolean needMerge( int level ){
      List segments = segmentsList.get( level );
      if( segments == null ) return false;
      return segments.size() >= mergeFactor;
    }
    public void deleteMergedSegments(){
      for( List segments : segmentsList ){
        if( segments == null ) continue;
        for( Iterator ite = segments.iterator(); ite.hasNext(); ){
          Segment segment = ite.next();
          if( segment.isDeleted() ){
            ite.remove();
          }
        }
      }
    }
    public void logHeader( PrintStream ps ){
      ps.println( "number of segments, total size" );
    }
    public void log( PrintStream ps ){
      ps.println( getNumSegments() + ", " + getSize() );
    }
  }
  
  static class Writer {
    final int maxBufferedDocs;
    final Directory dir;
    final PrintStream ps;
    int docs;
    public Writer( int maxBufferedDocs, Directory dir, PrintStream ps ){
      this.maxBufferedDocs = maxBufferedDocs;
      this.dir = dir;
      this.ps = ps;
      docs = 0;
    }
    public void add( Doc doc ){
      if( ++docs >= maxBufferedDocs ){
        Segment segment = new Segment( docs );
        dir.addSegment( segment );
        dir.log( ps );
        dir.deleteMergedSegments();
        docs = 0;
      }
      else{
        dir.log( ps );
      }
    }
  }
}
| 関口宏司 | Luceneインデックス | 12:41 | comments(0) | trackbacks(0) |
並列インデックスを利用したDocumentの更新
Luceneのインデックスに登録されている既存Documentを更新する場合、LuceneではDocumentをいったん削除し、再度新規にDocument全体を追加しなければならない。このため、サイズの小さなフィールドの追加や内容変更をする場合でも、変更の必要がないサイズの大きなフィールドにまでその影響がおよぼされてしまう。そのため、SolrなどでもStoredデータを使ってフィールドの更新をできるようにしようという提案もされたりしている:

https://issues.apache.org/jira/browse/SOLR-139

このようなフィールドのちょっとした追加機能に関しては以前から要望が多かったが、Luceneの将来バージョンで機能制限付きながらも「並列インデックス(Parallel Index)」を利用したDocumentの更新をできるようにしよう、という提案がなされている:

https://issues.apache.org/jira/browse/LUCENE-1879
http://wiki.apache.org/lucene-java/ParallelIncrementalIndexing

「並列インデックス」とは、異なるフィールドを保持している複数のインデックスでかつそれぞれのインデックスで同一Documentを特定できるようにDocument IDが割り振られているインデックスを指す。ひとつのDocumentをフィールド別に複数のインデックスに分けて登録する場合、そのDocumentが同じIDを割り振られるようにするには、インデックスごとに同じ順序でDocumentを追加していく必要がある。

このような「並列インデックス」をオープンして検索に使用するために、ParallelReaderというIndexReaderがある。並列インデックスを作成し、ParallelReaderを使ってインデックスの内容を表示するサンプルプログラムを以下に示す:
public class TestParallelReader {
  
  static final String[] BODIES = {
    "This is body 0", "This is body 1", "This is body 2"
  };
  static final String F_BODY = "body";
  static final String[] PRICES = { "100", "110", "120" };
  static final String F_PRICE = "price";
  static final String[] AUTHORS = { "Erik", "Yonik", "Mike" };
  static final String F_AUTHOR = "author";
  static Analyzer analyzer = new WhitespaceAnalyzer();

  public static void main(String[] args) throws Exception {
    ParallelReader pr = new ParallelReader( true );

    Directory bodiesDir = makeIndex( F_BODY, BODIES );
    IndexReader bodiesReader = IndexReader.open( bodiesDir, true );
    pr.add( bodiesReader );
    printAllContents( pr );

    Directory pricesDir = makeIndex( F_PRICE, PRICES );
    IndexReader pricesReader = IndexReader.open( pricesDir, true );
    pr.add( pricesReader );
    printAllContents( pr );

    Directory authorsDir = makeIndex( F_AUTHOR, AUTHORS );
    IndexReader authorsReader = IndexReader.open( authorsDir, true );
    pr.add( authorsReader );
    printAllContents( pr );

    pr.close();
  }

  static Directory makeIndex( String name, String[] contents ) throws Exception {
    System.out.println( "¥n===== " + name + " フィールドをインデックスに追加 =====" );
    Directory dir = new RAMDirectory();
    IndexWriter writer = new IndexWriter( dir, analyzer, true, MaxFieldLength.LIMITED );
    for( String content : contents ){
      Document doc = new Document();
      doc.add( new Field( name, content, Store.YES, Index.ANALYZED ) );
      writer.addDocument( doc );
    }
    writer.close();
    return dir;
  }
  
  static void printAllContents( IndexReader reader ) throws Exception {
    System.out.println( "===== インデックスの内容を表示 =====" );
    IndexSearcher searcher = new IndexSearcher( reader );
    Query query = new MatchAllDocsQuery();
    TopDocs docs = searcher.search( query, 10 );
    for( ScoreDoc scoreDoc : docs.scoreDocs ){
      Document doc = searcher.doc( scoreDoc.doc );
      for( Field field : (List)doc.getFields() ){
        System.out.print( field.name() + ":" + doc.get( field.name() ) + ", " );
      }
      System.out.println();  // line break
    }
    searcher.close();
  }
}

サンプルプログラムでは、インデックスに3つのフィールドを順次追加している。通常であれば既存のインデックスのDocumentの全体を削除して再登録しなければならないが、サンプルプログラムでは新しいフィールドを新しいインデックスに登録し、ParallelReaderを使ってopenしている。実行結果は次のようになる:
===== body フィールドをインデックスに追加 =====
===== インデックスの内容を表示 =====
body:This is body 0, 
body:This is body 1, 
body:This is body 2, 

===== price フィールドをインデックスに追加 =====
===== インデックスの内容を表示 =====
body:This is body 0, price:100, 
body:This is body 1, price:110, 
body:This is body 2, price:120, 

===== author フィールドをインデックスに追加 =====
===== インデックスの内容を表示 =====
body:This is body 0, price:100, author:Erik, 
body:This is body 1, price:110, author:Yonik, 
body:This is body 2, price:120, author:Mike, 

実際にこの方法を拡張してDocumentのフィールドを更新するにはいくつかの壁を乗り越える必要がある。たとえばDocument IDを複数にまたがったインデックスで同じ値に保つのは非常に難しい。Documentを削除すると、次のマージのタイミングでDocument IDはずれてしまう。現在のLuceneでは、マージはDocument数ではなくサイズ基準で発生するのがデフォルトになっているが、これではインデックスごとにマージ実行のタイミングがずれてしまうので、flush/delete/mergeのタイミングは同一に保つ必要がある。

そのためのアイディアとしては、複数のインデックスのそれぞれを担当するIndexWriterを、ひとつのマスターと複数のスレーブに分ける。そしてスレーブのIndexWriterは、マスターのIndexWriterのタイミングに合わせてマージを行ったらどうか、という提案の内容になっている。
| 関口宏司 | Luceneインデックス | 13:55 | comments(0) | trackbacks(0) |
optimize()中は元のインデックスの2倍の空き容量が必要な件
IndexReaderがオープンしているインデックスに対してoptimize()を実行するときは、ハードディスクの空き容量が元のインデックス容量の2倍必要である。たとえば、10GBのインデックスをIndexReaderがオープンしていると、optimize()には20GBの容量が必要となる。

さらにユーザからの指摘で、IndexReaderでオープン中でないインデックスも同じピーク容量が必要となるケースが報告された。それはautoCommit=falseのIndexWriterでオープンしているインデックスである。

この指摘を受け、optimize()のJavadocが変更された。
| 関口宏司 | Luceneインデックス | 18:57 | comments(0) | trackbacks(0) |
IndexWriterのautoCommitパラメータ
これはまだtrunkバージョンの話だが、IndexWriterにautoCommitモードなるものが追加された。

autoCommitはtrue|falseのどちらかをIndexWriterを作成するときのコンストラクタで指定する。デフォルトはtrueでLucene 2.1以前の動作と同じとなる。

ここでいうコミットとは、segments_Nファイルを書き込んでセグメントを確定させることを指す。

Lucene 2.0以前ではsetMaxBufferedDocs()(デフォルトは10)で指定したドキュメント数に達したときにメモリからインデックスにドキュメントが書き出されてsegments(当時は"_N"はつかなかった)にも反映されたので、この時点でIndexReaderをオープンすれば、インデックスに書き出されたドキュメントは検索にも反映された。

Lucene 2.1では、IndexWriterにflush()というメソッドが追加され、これはsetMaxBufferedDocs()にドキュメント数が達しない場合でも、flush()により強制的にメモリからインデックスにドキュメントを書き出させるものである。これにより、書き出しのタイミングコントロールがプログラムからできるようになった。

trunkバージョンではさらにautoCommit=falseを指定することができるようになり、この場合、メモリからインデックスにドキュメントが書き出されてもsegments_Nを書き換えない。そうすることによりIndexReaderにはコミット前の古いセグメントを見せることができる。そしてautoCommit=falseを指定した場合に限り、IndexWriterをclose()またはabort()することを選択できる。close()はもちろん、IndexWriterを閉じてaddDocument()を確定させる(従来どおり)。abort()はやはりtrunkバージョンで加わったメソッドで、これを呼ぶとそれまでのaddDocument()はすべて破棄され、セグメントは古いままで更新されない。
| 関口宏司 | Luceneインデックス | 15:35 | comments(0) | trackbacks(0) |
アンチウィルスソフトを使うときの注意点
先週末にアンチウィルスソフトのためにインデックスファイルが読めなくなったという報告がLuceneメーリングリストに流れていたので、簡単に状況を記しておきたい。

環境はWindowsでLucene 2.1。前の記事でも書いたsegments_NファイルがAccess DeniedでIndexWriterがFileNotFoundExceptionをスローするというもの。

報告者の報告がどうもはっきりしないが、インデックスに索引付けしようとしているファイルのひとつがウィルスに感染していたようだとある。しかしながら、「segments_NファイルがAccess Denied」という状況を考えると、segments_Nファイルの内容がウィルスパターンと酷似しており、アンチウィルスソフトがそのファイルのリードアクセスを不許可にしたのが真の原因のようだ。

このことから得られる教訓は、

  • Luceneインデックスディレクトリはアンチウィルスソフトのチェックからはずす。
  • ウィルス感染の可能性があるファイルをLuceneに登録する場合は、登録する前にその前段でウィルスチェックを終了しておく。もちろん、感染している場合は、Luceneインデックスに登録してはいけない。


である。
| 関口宏司 | Luceneインデックス | 00:38 | comments(0) | trackbacks(0) |
Lock-less commit
Lucene 2.1より、インデックスセグメントのコミット時にロックをしなくなった。

インデックスセグメントのコミットとは、インデックスファイルの構成が記されているsegmentsファイルを読み書きすることである。従来はコミット動作時はcommit.lockというファイル(実際のファイル名はもっと長い)がIndexWriterやIndexReaderらによって作成され、IndexReaderが読み込みオープン中のsegmentsファイルとインデックスセグメントがIndexWriterにより書き換わったり消されることを防いできた(逆にIndexWriterにより書き込み中(マージなど)のときにIndexReaderが読まないようにする役割もある)。つまり、commit.lockでインデックスセグメントの信頼のコミットメント(約束)を取り付けていたのだ。

ローカルファイルシステムではこのしくみによりひとつのWriterと複数のReaderがインデックスを共有することになんら問題がないが、NFSやSambaなどを使って別筐体でインデックスを共有しようとしたときに問題が起こることがしばしば報告されてきた。

IndexWriterは新しいセグメントファイルを作成すると、古いセグメントファイルを削除しようとするが、古いセグメントファイルで検索が実行中の場合(オープン中のとき)、WindowsではAccess Deniedが発生する。この場合はdeletableファイルが作成され、あとで削除できるように古いセグメントファイルが登録される(Lucene本 P99参照)。Unixの場合は削除は成功するが、実際にファイルが削除されるのは、すべてのIndexReaderがファイルをクローズした時点となる。しかしNFSの場合は(Sambaはどうか不明)どうやらキャッシュが効いてしまうらしく、IndexWriterによる古いセグメントファイルの削除は成功し、NFS経由のIndexReaderがオープンしているファイルが実際に消されてしまう。そして本当に読み込もうとしたときにFileNotFoundExceptionが発生する、というのがNFSの環境下で報告されてきたエラーの原因である。これを考えると、いくらコミットロックをがんばっても解決できないのであった。

NFSは手っ取り早くLuceneアプリケーションをスケールアップさせたいときに食指が動くツールである。NFSでインデックスを共有し、ひとつのWriterがインデックスを更新し、複数のSearcherが適当な間隔でopen/closeしながら検索を行えばよさそうだからだ。しかし上記の理由から、すぐにSearcher側からFileNotFoundExceptionがスローされてしまう。

そこでその対策の第一歩として、Lucene 2.1ではコミット時のロックを廃止したのである。ではどうしたのかというと、segmentsファイルのファイル名をsegmentsという固定の名前ではなく、segments_Nという具合に変更し、Nをインクリメントする命名方法を導入した。こうすれば既存の(IndexReaderが読んでいるかもしれない)古いsegments_Nファイルを書き換えるためにコミットロックを作成する必要がなくなる。そして、IndexWriterは古いsegments_Nファイルと古いセグメントファイルを削除する(前述の通りWindowsではこれが失敗することもあるが、その場合はあとで消すようだ)。しかしそうすると、IndexReaderが古いsegments_Nの情報から取得したセグメントファイルをオープンしようとしたときにFileNotFoundExceptionが発生してしまう可能性がある。そこでLucene 2.1ではIndexReaderがその例外に遭遇したときは新しいsegments_Nファイルを探してそこからロードする再試行をするように変更された。
| 関口宏司 | Luceneインデックス | 20:58 | comments(0) | trackbacks(0) |
+ Solrによるブログ内検索
+ PROFILE
 123456
78910111213
14151617181920
21222324252627
28293031   
<< October 2018 >>
+ LINKS
検索エンジン製品 - 比較のポイント
商用検索エンジンを購入した企業担当者は読まないでください。ショックを受けますから・・・
>>製品比較 10のポイント
+ Lucene&Solrデモ
+ ThinkIT記事
+ RECOMMEND
Apache Solr入門 ―オープンソース全文検索エンジン
Apache Solr入門 ―オープンソース全文検索エンジン (JUGEMレビュー »)
関口 宏司,三部 靖夫,武田 光平,中野 猛,大谷 純
+ RECOMMEND
Lucene in Action
Lucene in Action (JUGEMレビュー »)
Erik Hatcher,Otis Gospodnetic,Mike McCandless
FastVectorHighlighterについて解説記事を寄稿しました。
+ RECOMMEND
+ SELECTED ENTRIES
+ RECENT COMMENTS
+ RECENT TRACKBACK
+ CATEGORIES
+ ARCHIVES
+ MOBILE
qrcode
+ SPONSORED LINKS