You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by gs...@apache.org on 2007/10/17 22:36:21 UTC

svn commit: r585677 - in /lucene/java/trunk/contrib/benchmark: conf/indexing-flush-by-RAM-multithreaded.alg conf/indexing-flush-by-RAM.alg conf/indexing-multithreaded.alg conf/indexing.alg src/java/org/apache/lucene/benchmark/quality/trec/QueryDriver.java

Author: gsingers
Date: Wed Oct 17 13:36:20 2007
New Revision: 585677

URL: http://svn.apache.org/viewvc?rev=585677&view=rev
Log:
Added some more algorithms for testing things out, implemented basic TREC query driver based on the sample in the javadocs.

Added:
    lucene/java/trunk/contrib/benchmark/conf/indexing-flush-by-RAM-multithreaded.alg
    lucene/java/trunk/contrib/benchmark/conf/indexing-flush-by-RAM.alg
    lucene/java/trunk/contrib/benchmark/conf/indexing-multithreaded.alg
    lucene/java/trunk/contrib/benchmark/conf/indexing.alg
    lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/quality/trec/QueryDriver.java   (with props)

Added: lucene/java/trunk/contrib/benchmark/conf/indexing-flush-by-RAM-multithreaded.alg
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/benchmark/conf/indexing-flush-by-RAM-multithreaded.alg?rev=585677&view=auto
==============================================================================
--- lucene/java/trunk/contrib/benchmark/conf/indexing-flush-by-RAM-multithreaded.alg (added)
+++ lucene/java/trunk/contrib/benchmark/conf/indexing-flush-by-RAM-multithreaded.alg Wed Oct 17 13:36:20 2007
@@ -0,0 +1,70 @@
+#/**
+# * 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.
+# */
+# -------------------------------------------------------------------------------------
+# multi val params are iterated by NewRound's, added to reports, start with column name.
+
+#merge.factor=mrg:10:100:10:100:10:100:10:100
+#max.buffered=buf:10:10:100:100:10:10:100:100
+ram.flush.mb=flush:32:40:48:56:32:40:48:56
+compound=cmpnd:true:true:true:true:false:false:false:false
+
+autocommit=false
+analyzer=org.apache.lucene.analysis.standard.StandardAnalyzer
+directory=FSDirectory
+#directory=RamDirectory
+
+doc.stored=true
+doc.tokenized=true
+doc.term.vector=false
+doc.add.log.step=2000
+
+docs.dir=reuters-out
+#docs.dir=reuters-111
+
+#doc.maker=org.apache.lucene.benchmark.byTask.feeds.SimpleDocMaker
+doc.maker=org.apache.lucene.benchmark.byTask.feeds.ReutersDocMaker
+
+#query.maker=org.apache.lucene.benchmark.byTask.feeds.SimpleQueryMaker
+query.maker=org.apache.lucene.benchmark.byTask.feeds.ReutersQueryMaker
+
+# task at this depth or less would print when they start
+task.max.depth.log=2
+
+log.queries=true
+# -------------------------------------------------------------------------------------
+
+{ "Rounds"
+
+    ResetSystemErase
+
+    { "Populate"
+        CreateIndex
+        [{ "MAddDocs" AddDoc } : 5000] : 4
+        Optimize
+        CloseIndex
+    }
+
+
+    RepSumByPref MAddDocs
+
+    NewRound
+
+} : 8
+
+RepSumByNameRound
+RepSumByName
+RepSumByPrefRound MAddDocs

Added: lucene/java/trunk/contrib/benchmark/conf/indexing-flush-by-RAM.alg
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/benchmark/conf/indexing-flush-by-RAM.alg?rev=585677&view=auto
==============================================================================
--- lucene/java/trunk/contrib/benchmark/conf/indexing-flush-by-RAM.alg (added)
+++ lucene/java/trunk/contrib/benchmark/conf/indexing-flush-by-RAM.alg Wed Oct 17 13:36:20 2007
@@ -0,0 +1,70 @@
+#/**
+# * 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.
+# */
+# -------------------------------------------------------------------------------------
+# multi val params are iterated by NewRound's, added to reports, start with column name.
+
+#merge.factor=mrg:10:100:10:100:10:100:10:100
+#max.buffered=buf:10:10:100:100:10:10:100:100
+ram.flush.mb=flush:32:40:48:56:32:40:48:56
+compound=cmpnd:true:true:true:true:false:false:false:false
+
+autocommit=false
+analyzer=org.apache.lucene.analysis.standard.StandardAnalyzer
+directory=FSDirectory
+#directory=RamDirectory
+
+doc.stored=true
+doc.tokenized=true
+doc.term.vector=false
+doc.add.log.step=2000
+
+docs.dir=reuters-out
+#docs.dir=reuters-111
+
+#doc.maker=org.apache.lucene.benchmark.byTask.feeds.SimpleDocMaker
+doc.maker=org.apache.lucene.benchmark.byTask.feeds.ReutersDocMaker
+
+#query.maker=org.apache.lucene.benchmark.byTask.feeds.SimpleQueryMaker
+query.maker=org.apache.lucene.benchmark.byTask.feeds.ReutersQueryMaker
+
+# task at this depth or less would print when they start
+task.max.depth.log=2
+
+log.queries=true
+# -------------------------------------------------------------------------------------
+
+{ "Rounds"
+
+    ResetSystemErase
+
+    { "Populate"
+        CreateIndex
+        { "MAddDocs" AddDoc } : 20000
+        Optimize
+        CloseIndex
+    }
+
+
+    RepSumByPref MAddDocs
+
+    NewRound
+
+} : 8
+
+RepSumByNameRound
+RepSumByName
+RepSumByPrefRound MAddDocs

Added: lucene/java/trunk/contrib/benchmark/conf/indexing-multithreaded.alg
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/benchmark/conf/indexing-multithreaded.alg?rev=585677&view=auto
==============================================================================
--- lucene/java/trunk/contrib/benchmark/conf/indexing-multithreaded.alg (added)
+++ lucene/java/trunk/contrib/benchmark/conf/indexing-multithreaded.alg Wed Oct 17 13:36:20 2007
@@ -0,0 +1,70 @@
+#/**
+# * 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.
+# */
+# -------------------------------------------------------------------------------------
+# multi val params are iterated by NewRound's, added to reports, start with column name.
+
+merge.factor=mrg:10:100:10:100:10:100:10:100
+max.buffered=buf:10:10:100:100:10:10:100:100
+#ram.flush.mb=flush:32:40:48:56:32:40:48:56
+compound=cmpnd:true:true:true:true:false:false:false:false
+
+autocommit=false
+analyzer=org.apache.lucene.analysis.standard.StandardAnalyzer
+directory=FSDirectory
+#directory=RamDirectory
+
+doc.stored=true
+doc.tokenized=true
+doc.term.vector=false
+doc.add.log.step=2000
+
+docs.dir=reuters-out
+#docs.dir=reuters-111
+
+#doc.maker=org.apache.lucene.benchmark.byTask.feeds.SimpleDocMaker
+doc.maker=org.apache.lucene.benchmark.byTask.feeds.ReutersDocMaker
+
+#query.maker=org.apache.lucene.benchmark.byTask.feeds.SimpleQueryMaker
+query.maker=org.apache.lucene.benchmark.byTask.feeds.ReutersQueryMaker
+
+# task at this depth or less would print when they start
+task.max.depth.log=2
+
+log.queries=true
+# -------------------------------------------------------------------------------------
+
+{ "Rounds"
+
+    ResetSystemErase
+
+    { "Populate"
+        CreateIndex
+        [{ "MAddDocs" AddDoc } : 5000] : 4
+        Optimize
+        CloseIndex
+    }
+
+
+    RepSumByPref MAddDocs
+
+    NewRound
+
+} : 8
+
+RepSumByNameRound
+RepSumByName
+RepSumByPrefRound MAddDocs

Added: lucene/java/trunk/contrib/benchmark/conf/indexing.alg
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/benchmark/conf/indexing.alg?rev=585677&view=auto
==============================================================================
--- lucene/java/trunk/contrib/benchmark/conf/indexing.alg (added)
+++ lucene/java/trunk/contrib/benchmark/conf/indexing.alg Wed Oct 17 13:36:20 2007
@@ -0,0 +1,70 @@
+#/**
+# * 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.
+# */
+# -------------------------------------------------------------------------------------
+# multi val params are iterated by NewRound's, added to reports, start with column name.
+
+merge.factor=mrg:10:100:10:100:10:100:10:100
+max.buffered=buf:10:10:100:100:10:10:100:100
+#ram.flush.mb=flush:32:40:48:56:32:40:48:56
+compound=cmpnd:true:true:true:true:false:false:false:false
+
+autocommit=false
+analyzer=org.apache.lucene.analysis.standard.StandardAnalyzer
+directory=FSDirectory
+#directory=RamDirectory
+
+doc.stored=true
+doc.tokenized=true
+doc.term.vector=false
+doc.add.log.step=2000
+
+docs.dir=reuters-out
+#docs.dir=reuters-111
+
+#doc.maker=org.apache.lucene.benchmark.byTask.feeds.SimpleDocMaker
+doc.maker=org.apache.lucene.benchmark.byTask.feeds.ReutersDocMaker
+
+#query.maker=org.apache.lucene.benchmark.byTask.feeds.SimpleQueryMaker
+query.maker=org.apache.lucene.benchmark.byTask.feeds.ReutersQueryMaker
+
+# task at this depth or less would print when they start
+task.max.depth.log=2
+
+log.queries=true
+# -------------------------------------------------------------------------------------
+
+{ "Rounds"
+
+    ResetSystemErase
+
+    { "Populate"
+        CreateIndex
+        { "MAddDocs" AddDoc } : 20000
+        Optimize
+        CloseIndex
+    }
+
+
+    RepSumByPref MAddDocs
+
+    NewRound
+
+} : 8
+
+RepSumByNameRound
+RepSumByName
+RepSumByPrefRound MAddDocs

Added: lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/quality/trec/QueryDriver.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/quality/trec/QueryDriver.java?rev=585677&view=auto
==============================================================================
--- lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/quality/trec/QueryDriver.java (added)
+++ lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/quality/trec/QueryDriver.java Wed Oct 17 13:36:20 2007
@@ -0,0 +1,56 @@
+package org.apache.lucene.benchmark.quality.trec;
+
+import org.apache.lucene.benchmark.quality.trec.TrecJudge;
+import org.apache.lucene.benchmark.quality.trec.TrecTopicsReader;
+import org.apache.lucene.benchmark.quality.utils.SimpleQQParser;
+import org.apache.lucene.benchmark.quality.utils.SubmissionReport;
+import org.apache.lucene.benchmark.quality.*;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Searcher;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.PrintWriter;
+
+
+/**
+ *
+ *
+ **/
+public class QueryDriver {
+  public static void main(String[] args) throws Exception {
+
+    
+    File topicsFile = new File(args[0]);
+    File qrelsFile = new File(args[1]);
+    Searcher searcher = new IndexSearcher(args[3]);
+
+    int maxResults = 1000;
+    String docNameField = "docname";
+
+    PrintWriter logger = new PrintWriter(System.out, true);
+
+    // use trec utilities to read trec topics into quality queries
+    TrecTopicsReader qReader = new TrecTopicsReader();
+    QualityQuery qqs[] = qReader.readQueries(new BufferedReader(new FileReader(topicsFile)));
+
+    // prepare judge, with trec utilities that read from a QRels file
+    Judge judge = new TrecJudge(new BufferedReader(new FileReader(qrelsFile)));
+
+    // validate topics & judgments match each other
+    judge.validateData(qqs, logger);
+
+    // set the parsing of quality queries into Lucene queries.
+    QualityQueryParser qqParser = new SimpleQQParser("title", "body");
+
+    // run the benchmark
+    QualityBenchmark qrun = new QualityBenchmark(qqs, qqParser, searcher, docNameField);
+    SubmissionReport submitLog = null;
+    QualityStats stats[] = qrun.execute(maxResults, judge, submitLog, logger);
+
+    // print an avarage sum of the results
+    QualityStats avg = QualityStats.average(stats);
+    avg.log("SUMMARY", 2, logger, "  ");
+  }
+}

Propchange: lucene/java/trunk/contrib/benchmark/src/java/org/apache/lucene/benchmark/quality/trec/QueryDriver.java
------------------------------------------------------------------------------
    svn:eol-style = native