You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-user@lucene.apache.org by wangjing <pp...@gmail.com> on 2012/06/27 11:40:17 UTC

about .frq file format in doc

http://lucene.apache.org/core/3_6_0/fileformats.html#Frequencies

The .frq file contains the lists of documents which contain each term,
along with the frequency of the term in that document (except when
frequencies are omitted: IndexOptions.DOCS_ONLY).

BUT in my example,the frq record the frequency of the term in FIELD?

my app is
public void createLuceneIndex() {
		try {
			File file = new File(path);
			FSDirectory directory = FSDirectory.open(file);
			// 用来创建索引
			IndexWriterConfig conf = new IndexWriterConfig(Version.LUCENE_36,
					new StandardAnalyzer(Version.LUCENE_36));
			// 避免重复运用时每次都附加index导致查询出多条记录
			conf.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
			IndexWriter iw = new IndexWriter(directory, conf);
			iw.setInfoStream(System.out);
			Document doc = new Document();
			// Store.YES 保存 可以查询 可以打印内容
			Field storeYes = new Field("content1", "my my my my storeyes test",
					Store.YES, Index.ANALYZED);
			// Store.NO 不保存 可以查询 不可打印内容 由于不保存内容所以节省空间
			Field storeNo = new Field("content2", "my storeno test", Store.YES,
					Index.ANALYZED);

			doc.add(storeYes);
			doc.add(storeNo);

			Document doc1 = new Document();
			Field storeYes1 = new Field("doc1content1",
					"doc1 my storeyes test", Store.YES, Index.ANALYZED);
			doc1.add(storeYes1);

			Document doc2 = new Document();
			Field storeYesdoc2 = new Field("doc2content1",
					"doc2 my storeyes test", Store.YES, Index.ANALYZED);
			Field storeNodoc2 = new Field("doc2content2",
					"doc2 my storeno test", Store.YES, Index.ANALYZED);
			doc2.add(storeYesdoc2);
			doc2.add(storeNodoc2);

			iw.addDocument(doc);
			iw.addDocument(doc1);
			iw.addDocument(doc2);
			System.out.println("doc :" + doc);
			System.out.println("doc1 :" + doc1);
			System.out.println("doc2 :" + doc2);
			iw.forceMerge(1);
			iw.close();
		} catch (CorruptIndexException e) {
			// TODO Auto-generated catch block
			e.printStackTrace();
		} catch (LockObtainFailedException e) {
			// TODO Auto-generated catch block
			e.printStackTrace();
		} catch (IOException e) {
			// TODO Auto-generated catch block
			e.printStackTrace();
		}
	}


THE HEX in .frq file is
00 04 01 01 01 01 01 03  03 03 03 05 05 05 05 05
05 05 05

in my  analysis
1   [0,4] [0,1] [0,1] [0,1] [0,1]
[0,1][1,1][1,1][1,1][1,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1]
2   [0,4] [0,1] [0,1] [0,1] [0,1] [0,1]
[1,1][1,1][1,1][1,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1]
3   [0,4][1][1][1][1][1][3][3][3][3][5] [5][5][5][5][5][5][5]

---------------------------------------------------------------------
To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-user-help@lucene.apache.org


Re: about .frq file format in doc

Posted by Li Li <fa...@gmail.com>.
lastDocID represent last document which contains this term.
because it will reuse this FormatPostingsDocsConsumer. so you need
clear all member variables in finish method

On Thu, Jun 28, 2012 at 11:14 AM, wangjing <pp...@gmail.com> wrote:
> thanks
>
> could you help me to solve another problem,
>
> why lucene will reset lastDocID = 0 when finish add one doc?
>
> in source code FormatPostingsDocsWriter.java
>
>        @Override
>        void finish() throws IOException {
>                long skipPointer = skipListWriter.writeSkip(out);
>                // TODO: this is abstraction violation -- we should not
>                // peek up into parents terms encoding format
>                termInfo.set(df, parent.freqStart, parent.proxStart,
>                                (int) (skipPointer - parent.freqStart));
>                // TODO: we could do this incrementally
>                UnicodeUtil.UTF16toUTF8(parent.currentTerm, parent.currentTermStart,
>                                utf8);
>                if (df > 0) {
>                        parent.termsOut.add(fieldInfo.number, utf8.result, utf8.length,
>                                        termInfo);
>                }
>                lastDocID = 0;
>                df = 0;
>        }
>
> @Override
>        FormatPostingsPositionsConsumer addDoc(int docID, int termDocFreq)
>                        throws IOException {
>                System.out.println("docID:" + docID);
>                System.out.println("termDocFreq:" + termDocFreq);
>
>                final int delta = docID - lastDocID;
>                System.out.println("delta:" + delta);
>                if (docID < 0 || (df > 0 && delta <= 0))
>                        throw new CorruptIndexException("docs out of order (" + docID
>                                        + " <= " + lastDocID + " ) (out: " + out + ")");
>
>                if ((++df % skipInterval) == 0) {
>                        // TODO: abstraction violation
>                        skipListWriter.setSkipData(lastDocID, storePayloads,
>                                        posWriter.lastPayloadLength);
>                        skipListWriter.bufferSkip(df);
>                }
>                assert docID < totalNumDocs : "docID=" + docID + " totalNumDocs="
>                                + totalNumDocs;
>                lastDocID = docID;
>                if (omitTermFreqAndPositions)
>                        out.writeVInt(delta);
>                else if (1 == termDocFreq)
>
>                        out.writeVInt((delta << 1) | 1);
>                else {
>                        out.writeVInt(delta << 1);
>                        out.writeVInt(termDocFreq);
>                }
>                System.out.println("------------");
>                return posWriter;
>        }
>
>
>
> because final int delta = docID - lastDocID;  and  lastDocID set to 0;
>
> it will let  deltaDoc   always equals docID .
>
> docId is 0,1,2 but the delta is 0,1,2, not 0,1,1.
>
> It left me rather confused.
>
>
> docID:0
> termDocFreq:4
> delta:0
> ------------
> docID:0
> termDocFreq:1
> delta:0
> ------------
> docID:0
> termDocFreq:1
> delta:0
> ------------
> docID:0
> termDocFreq:1
> delta:0
> ------------
> docID:0
> termDocFreq:1
> delta:0
> ------------
> docID:0
> termDocFreq:1
> delta:0
> ------------
> docID:1
> termDocFreq:1
> delta:1
> ------------
> docID:1
> termDocFreq:1
> delta:1
> ------------
> docID:1
> termDocFreq:1
> delta:1
> ------------
> docID:1
> termDocFreq:1
> delta:1
> ------------
> docID:2
> termDocFreq:1
> delta:2
> ------------
> docID:2
> termDocFreq:1
> delta:2
> ------------
> docID:2
> termDocFreq:1
> delta:2
> ------------
> docID:2
> termDocFreq:1
> delta:2
> ------------
> docID:2
> termDocFreq:1
> delta:2
> ------------
> docID:2
> termDocFreq:1
> delta:2
> ------------
> docID:2
> termDocFreq:1
> delta:2
> ------------
> docID:2
> termDocFreq:1
> delta:2
>
>
>
>
> On Thu, Jun 28, 2012 at 12:09 AM, Simon Willnauer
> <si...@googlemail.com> wrote:
>> see definitions:
>> http://lucene.apache.org/core/3_6_0/fileformats.html#Definitions
>>
>> simon
>>
>> On Wed, Jun 27, 2012 at 6:08 PM, Simon Willnauer
>> <si...@googlemail.com> wrote:
>>> a term in this context is a (field,text) tuple - does this make sense?
>>> simon
>>>
>>> On Wed, Jun 27, 2012 at 11:40 AM, wangjing <pp...@gmail.com> wrote:
>>>> http://lucene.apache.org/core/3_6_0/fileformats.html#Frequencies
>>>>
>>>> The .frq file contains the lists of documents which contain each term,
>>>> along with the frequency of the term in that document (except when
>>>> frequencies are omitted: IndexOptions.DOCS_ONLY).
>>>>
>>>> BUT in my example,the frq record the frequency of the term in FIELD?
>>>>
>>>> my app is
>>>> public void createLuceneIndex() {
>>>>                try {
>>>>                        File file = new File(path);
>>>>                        FSDirectory directory = FSDirectory.open(file);
>>>>                        // 用来创建索引
>>>>                        IndexWriterConfig conf = new IndexWriterConfig(Version.LUCENE_36,
>>>>                                        new StandardAnalyzer(Version.LUCENE_36));
>>>>                        // 避免重复运用时每次都附加index导致查询出多条记录
>>>>                        conf.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
>>>>                        IndexWriter iw = new IndexWriter(directory, conf);
>>>>                        iw.setInfoStream(System.out);
>>>>                        Document doc = new Document();
>>>>                        // Store.YES 保存 可以查询 可以打印内容
>>>>                        Field storeYes = new Field("content1", "my my my my storeyes test",
>>>>                                        Store.YES, Index.ANALYZED);
>>>>                        // Store.NO 不保存 可以查询 不可打印内容 由于不保存内容所以节省空间
>>>>                        Field storeNo = new Field("content2", "my storeno test", Store.YES,
>>>>                                        Index.ANALYZED);
>>>>
>>>>                        doc.add(storeYes);
>>>>                        doc.add(storeNo);
>>>>
>>>>                        Document doc1 = new Document();
>>>>                        Field storeYes1 = new Field("doc1content1",
>>>>                                        "doc1 my storeyes test", Store.YES, Index.ANALYZED);
>>>>                        doc1.add(storeYes1);
>>>>
>>>>                        Document doc2 = new Document();
>>>>                        Field storeYesdoc2 = new Field("doc2content1",
>>>>                                        "doc2 my storeyes test", Store.YES, Index.ANALYZED);
>>>>                        Field storeNodoc2 = new Field("doc2content2",
>>>>                                        "doc2 my storeno test", Store.YES, Index.ANALYZED);
>>>>                        doc2.add(storeYesdoc2);
>>>>                        doc2.add(storeNodoc2);
>>>>
>>>>                        iw.addDocument(doc);
>>>>                        iw.addDocument(doc1);
>>>>                        iw.addDocument(doc2);
>>>>                        System.out.println("doc :" + doc);
>>>>                        System.out.println("doc1 :" + doc1);
>>>>                        System.out.println("doc2 :" + doc2);
>>>>                        iw.forceMerge(1);
>>>>                        iw.close();
>>>>                } catch (CorruptIndexException e) {
>>>>                        // TODO Auto-generated catch block
>>>>                        e.printStackTrace();
>>>>                } catch (LockObtainFailedException e) {
>>>>                        // TODO Auto-generated catch block
>>>>                        e.printStackTrace();
>>>>                } catch (IOException e) {
>>>>                        // TODO Auto-generated catch block
>>>>                        e.printStackTrace();
>>>>                }
>>>>        }
>>>>
>>>>
>>>> THE HEX in .frq file is
>>>> 00 04 01 01 01 01 01 03  03 03 03 05 05 05 05 05
>>>> 05 05 05
>>>>
>>>> in my  analysis
>>>> 1   [0,4] [0,1] [0,1] [0,1] [0,1]
>>>> [0,1][1,1][1,1][1,1][1,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1]
>>>> 2   [0,4] [0,1] [0,1] [0,1] [0,1] [0,1]
>>>> [1,1][1,1][1,1][1,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1]
>>>> 3   [0,4][1][1][1][1][1][3][3][3][3][5] [5][5][5][5][5][5][5]
>>>>
>>>> ---------------------------------------------------------------------
>>>> To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
>>>> For additional commands, e-mail: java-user-help@lucene.apache.org
>>>>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
> For additional commands, e-mail: java-user-help@lucene.apache.org
>

---------------------------------------------------------------------
To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-user-help@lucene.apache.org


Re: about .frq file format in doc

Posted by wangjing <pp...@gmail.com>.
thanks a lot

btw: docId is 0,1,2 but the delta is 0,1,2, not the 0,1,1.
       The delta value is useless at this moment.
       In which scene the delta value will be usefull?


On Thu, Jun 28, 2012 at 11:48 AM, Li Li <fa...@gmail.com> wrote:
> On Thu, Jun 28, 2012 at 11:14 AM, wangjing <pp...@gmail.com> wrote:
>> thanks
>>
>> could you help me to solve another problem,
>>
>> why lucene will reset lastDocID = 0 when finish add one doc?
>   it will not call finish after adding a document
>   reading the JavaDoc of FormatPostingsDocsConsumer
>     /** Called when we are done adding docs to this term */
>  abstract void finish() throws IOException;
>
>  it's calling stack:
> FreqProxTermsWriter.flush()
>   appendPostings(fieldName, state, fields, consumer);
>        docConsumer.finish();
>             FormatPostingsDocsWriter.finish()
>
>   FormatPostingsDocsWriter.addDoc() will write skiplist and docIds to
> frq and positions to prx. but skip lists are only written to memory
> data structure. finish() will actually flush them. also in finish()
> you should write terms info to tii/tis files.
>   as I said before,  lucene will reuse FormatPostingsDocsWriter, so
> you should set lastDocID and df to zero.
>>
>> in source code FormatPostingsDocsWriter.java
>>
>>        @Override
>>        void finish() throws IOException {
>>                long skipPointer = skipListWriter.writeSkip(out);
>>                // TODO: this is abstraction violation -- we should not
>>                // peek up into parents terms encoding format
>>                termInfo.set(df, parent.freqStart, parent.proxStart,
>>                                (int) (skipPointer - parent.freqStart));
>>                // TODO: we could do this incrementally
>>                UnicodeUtil.UTF16toUTF8(parent.currentTerm, parent.currentTermStart,
>>                                utf8);
>>                if (df > 0) {
>>                        parent.termsOut.add(fieldInfo.number, utf8.result, utf8.length,
>>                                        termInfo);
>>                }
>>                lastDocID = 0;
>>                df = 0;
>>        }
>>
>> @Override
>>        FormatPostingsPositionsConsumer addDoc(int docID, int termDocFreq)
>>                        throws IOException {
>>                System.out.println("docID:" + docID);
>>                System.out.println("termDocFreq:" + termDocFreq);
>>
>>                final int delta = docID - lastDocID;
>>                System.out.println("delta:" + delta);
>>                if (docID < 0 || (df > 0 && delta <= 0))
>>                        throw new CorruptIndexException("docs out of order (" + docID
>>                                        + " <= " + lastDocID + " ) (out: " + out + ")");
>>
>>                if ((++df % skipInterval) == 0) {
>>                        // TODO: abstraction violation
>>                        skipListWriter.setSkipData(lastDocID, storePayloads,
>>                                        posWriter.lastPayloadLength);
>>                        skipListWriter.bufferSkip(df);
>>                }
>>                assert docID < totalNumDocs : "docID=" + docID + " totalNumDocs="
>>                                + totalNumDocs;
>>                lastDocID = docID;
>>                if (omitTermFreqAndPositions)
>>                        out.writeVInt(delta);
>>                else if (1 == termDocFreq)
>>
>>                        out.writeVInt((delta << 1) | 1);
>>                else {
>>                        out.writeVInt(delta << 1);
>>                        out.writeVInt(termDocFreq);
>>                }
>>                System.out.println("------------");
>>                return posWriter;
>>        }
>>
>>
>>
>> because final int delta = docID - lastDocID;  and  lastDocID set to 0;
>>
>> it will let  deltaDoc   always equals docID .
>>
>> docId is 0,1,2 but the delta is 0,1,2, not 0,1,1.
>>
>> It left me rather confused.
>>
>>
>> docID:0
>> termDocFreq:4
>> delta:0
>> ------------
>> docID:0
>> termDocFreq:1
>> delta:0
>> ------------
>> docID:0
>> termDocFreq:1
>> delta:0
>> ------------
>> docID:0
>> termDocFreq:1
>> delta:0
>> ------------
>> docID:0
>> termDocFreq:1
>> delta:0
>> ------------
>> docID:0
>> termDocFreq:1
>> delta:0
>> ------------
>> docID:1
>> termDocFreq:1
>> delta:1
>> ------------
>> docID:1
>> termDocFreq:1
>> delta:1
>> ------------
>> docID:1
>> termDocFreq:1
>> delta:1
>> ------------
>> docID:1
>> termDocFreq:1
>> delta:1
>> ------------
>> docID:2
>> termDocFreq:1
>> delta:2
>> ------------
>> docID:2
>> termDocFreq:1
>> delta:2
>> ------------
>> docID:2
>> termDocFreq:1
>> delta:2
>> ------------
>> docID:2
>> termDocFreq:1
>> delta:2
>> ------------
>> docID:2
>> termDocFreq:1
>> delta:2
>> ------------
>> docID:2
>> termDocFreq:1
>> delta:2
>> ------------
>> docID:2
>> termDocFreq:1
>> delta:2
>> ------------
>> docID:2
>> termDocFreq:1
>> delta:2
>>
>>
>>
>>
>> On Thu, Jun 28, 2012 at 12:09 AM, Simon Willnauer
>> <si...@googlemail.com> wrote:
>>> see definitions:
>>> http://lucene.apache.org/core/3_6_0/fileformats.html#Definitions
>>>
>>> simon
>>>
>>> On Wed, Jun 27, 2012 at 6:08 PM, Simon Willnauer
>>> <si...@googlemail.com> wrote:
>>>> a term in this context is a (field,text) tuple - does this make sense?
>>>> simon
>>>>
>>>> On Wed, Jun 27, 2012 at 11:40 AM, wangjing <pp...@gmail.com> wrote:
>>>>> http://lucene.apache.org/core/3_6_0/fileformats.html#Frequencies
>>>>>
>>>>> The .frq file contains the lists of documents which contain each term,
>>>>> along with the frequency of the term in that document (except when
>>>>> frequencies are omitted: IndexOptions.DOCS_ONLY).
>>>>>
>>>>> BUT in my example,the frq record the frequency of the term in FIELD?
>>>>>
>>>>> my app is
>>>>> public void createLuceneIndex() {
>>>>>                try {
>>>>>                        File file = new File(path);
>>>>>                        FSDirectory directory = FSDirectory.open(file);
>>>>>                        // 用来创建索引
>>>>>                        IndexWriterConfig conf = new IndexWriterConfig(Version.LUCENE_36,
>>>>>                                        new StandardAnalyzer(Version.LUCENE_36));
>>>>>                        // 避免重复运用时每次都附加index导致查询出多条记录
>>>>>                        conf.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
>>>>>                        IndexWriter iw = new IndexWriter(directory, conf);
>>>>>                        iw.setInfoStream(System.out);
>>>>>                        Document doc = new Document();
>>>>>                        // Store.YES 保存 可以查询 可以打印内容
>>>>>                        Field storeYes = new Field("content1", "my my my my storeyes test",
>>>>>                                        Store.YES, Index.ANALYZED);
>>>>>                        // Store.NO 不保存 可以查询 不可打印内容 由于不保存内容所以节省空间
>>>>>                        Field storeNo = new Field("content2", "my storeno test", Store.YES,
>>>>>                                        Index.ANALYZED);
>>>>>
>>>>>                        doc.add(storeYes);
>>>>>                        doc.add(storeNo);
>>>>>
>>>>>                        Document doc1 = new Document();
>>>>>                        Field storeYes1 = new Field("doc1content1",
>>>>>                                        "doc1 my storeyes test", Store.YES, Index.ANALYZED);
>>>>>                        doc1.add(storeYes1);
>>>>>
>>>>>                        Document doc2 = new Document();
>>>>>                        Field storeYesdoc2 = new Field("doc2content1",
>>>>>                                        "doc2 my storeyes test", Store.YES, Index.ANALYZED);
>>>>>                        Field storeNodoc2 = new Field("doc2content2",
>>>>>                                        "doc2 my storeno test", Store.YES, Index.ANALYZED);
>>>>>                        doc2.add(storeYesdoc2);
>>>>>                        doc2.add(storeNodoc2);
>>>>>
>>>>>                        iw.addDocument(doc);
>>>>>                        iw.addDocument(doc1);
>>>>>                        iw.addDocument(doc2);
>>>>>                        System.out.println("doc :" + doc);
>>>>>                        System.out.println("doc1 :" + doc1);
>>>>>                        System.out.println("doc2 :" + doc2);
>>>>>                        iw.forceMerge(1);
>>>>>                        iw.close();
>>>>>                } catch (CorruptIndexException e) {
>>>>>                        // TODO Auto-generated catch block
>>>>>                        e.printStackTrace();
>>>>>                } catch (LockObtainFailedException e) {
>>>>>                        // TODO Auto-generated catch block
>>>>>                        e.printStackTrace();
>>>>>                } catch (IOException e) {
>>>>>                        // TODO Auto-generated catch block
>>>>>                        e.printStackTrace();
>>>>>                }
>>>>>        }
>>>>>
>>>>>
>>>>> THE HEX in .frq file is
>>>>> 00 04 01 01 01 01 01 03  03 03 03 05 05 05 05 05
>>>>> 05 05 05
>>>>>
>>>>> in my  analysis
>>>>> 1   [0,4] [0,1] [0,1] [0,1] [0,1]
>>>>> [0,1][1,1][1,1][1,1][1,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1]
>>>>> 2   [0,4] [0,1] [0,1] [0,1] [0,1] [0,1]
>>>>> [1,1][1,1][1,1][1,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1]
>>>>> 3   [0,4][1][1][1][1][1][3][3][3][3][5] [5][5][5][5][5][5][5]
>>>>>
>>>>> ---------------------------------------------------------------------
>>>>> To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
>>>>> For additional commands, e-mail: java-user-help@lucene.apache.org
>>>>>
>>
>> ---------------------------------------------------------------------
>> To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
>> For additional commands, e-mail: java-user-help@lucene.apache.org
>>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
> For additional commands, e-mail: java-user-help@lucene.apache.org
>

---------------------------------------------------------------------
To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-user-help@lucene.apache.org


Re: about .frq file format in doc

Posted by Li Li <fa...@gmail.com>.
On Thu, Jun 28, 2012 at 11:14 AM, wangjing <pp...@gmail.com> wrote:
> thanks
>
> could you help me to solve another problem,
>
> why lucene will reset lastDocID = 0 when finish add one doc?
   it will not call finish after adding a document
   reading the JavaDoc of FormatPostingsDocsConsumer
     /** Called when we are done adding docs to this term */
  abstract void finish() throws IOException;

  it's calling stack:
FreqProxTermsWriter.flush()
   appendPostings(fieldName, state, fields, consumer);
        docConsumer.finish();
             FormatPostingsDocsWriter.finish()

   FormatPostingsDocsWriter.addDoc() will write skiplist and docIds to
frq and positions to prx. but skip lists are only written to memory
data structure. finish() will actually flush them. also in finish()
you should write terms info to tii/tis files.
   as I said before,  lucene will reuse FormatPostingsDocsWriter, so
you should set lastDocID and df to zero.
>
> in source code FormatPostingsDocsWriter.java
>
>        @Override
>        void finish() throws IOException {
>                long skipPointer = skipListWriter.writeSkip(out);
>                // TODO: this is abstraction violation -- we should not
>                // peek up into parents terms encoding format
>                termInfo.set(df, parent.freqStart, parent.proxStart,
>                                (int) (skipPointer - parent.freqStart));
>                // TODO: we could do this incrementally
>                UnicodeUtil.UTF16toUTF8(parent.currentTerm, parent.currentTermStart,
>                                utf8);
>                if (df > 0) {
>                        parent.termsOut.add(fieldInfo.number, utf8.result, utf8.length,
>                                        termInfo);
>                }
>                lastDocID = 0;
>                df = 0;
>        }
>
> @Override
>        FormatPostingsPositionsConsumer addDoc(int docID, int termDocFreq)
>                        throws IOException {
>                System.out.println("docID:" + docID);
>                System.out.println("termDocFreq:" + termDocFreq);
>
>                final int delta = docID - lastDocID;
>                System.out.println("delta:" + delta);
>                if (docID < 0 || (df > 0 && delta <= 0))
>                        throw new CorruptIndexException("docs out of order (" + docID
>                                        + " <= " + lastDocID + " ) (out: " + out + ")");
>
>                if ((++df % skipInterval) == 0) {
>                        // TODO: abstraction violation
>                        skipListWriter.setSkipData(lastDocID, storePayloads,
>                                        posWriter.lastPayloadLength);
>                        skipListWriter.bufferSkip(df);
>                }
>                assert docID < totalNumDocs : "docID=" + docID + " totalNumDocs="
>                                + totalNumDocs;
>                lastDocID = docID;
>                if (omitTermFreqAndPositions)
>                        out.writeVInt(delta);
>                else if (1 == termDocFreq)
>
>                        out.writeVInt((delta << 1) | 1);
>                else {
>                        out.writeVInt(delta << 1);
>                        out.writeVInt(termDocFreq);
>                }
>                System.out.println("------------");
>                return posWriter;
>        }
>
>
>
> because final int delta = docID - lastDocID;  and  lastDocID set to 0;
>
> it will let  deltaDoc   always equals docID .
>
> docId is 0,1,2 but the delta is 0,1,2, not 0,1,1.
>
> It left me rather confused.
>
>
> docID:0
> termDocFreq:4
> delta:0
> ------------
> docID:0
> termDocFreq:1
> delta:0
> ------------
> docID:0
> termDocFreq:1
> delta:0
> ------------
> docID:0
> termDocFreq:1
> delta:0
> ------------
> docID:0
> termDocFreq:1
> delta:0
> ------------
> docID:0
> termDocFreq:1
> delta:0
> ------------
> docID:1
> termDocFreq:1
> delta:1
> ------------
> docID:1
> termDocFreq:1
> delta:1
> ------------
> docID:1
> termDocFreq:1
> delta:1
> ------------
> docID:1
> termDocFreq:1
> delta:1
> ------------
> docID:2
> termDocFreq:1
> delta:2
> ------------
> docID:2
> termDocFreq:1
> delta:2
> ------------
> docID:2
> termDocFreq:1
> delta:2
> ------------
> docID:2
> termDocFreq:1
> delta:2
> ------------
> docID:2
> termDocFreq:1
> delta:2
> ------------
> docID:2
> termDocFreq:1
> delta:2
> ------------
> docID:2
> termDocFreq:1
> delta:2
> ------------
> docID:2
> termDocFreq:1
> delta:2
>
>
>
>
> On Thu, Jun 28, 2012 at 12:09 AM, Simon Willnauer
> <si...@googlemail.com> wrote:
>> see definitions:
>> http://lucene.apache.org/core/3_6_0/fileformats.html#Definitions
>>
>> simon
>>
>> On Wed, Jun 27, 2012 at 6:08 PM, Simon Willnauer
>> <si...@googlemail.com> wrote:
>>> a term in this context is a (field,text) tuple - does this make sense?
>>> simon
>>>
>>> On Wed, Jun 27, 2012 at 11:40 AM, wangjing <pp...@gmail.com> wrote:
>>>> http://lucene.apache.org/core/3_6_0/fileformats.html#Frequencies
>>>>
>>>> The .frq file contains the lists of documents which contain each term,
>>>> along with the frequency of the term in that document (except when
>>>> frequencies are omitted: IndexOptions.DOCS_ONLY).
>>>>
>>>> BUT in my example,the frq record the frequency of the term in FIELD?
>>>>
>>>> my app is
>>>> public void createLuceneIndex() {
>>>>                try {
>>>>                        File file = new File(path);
>>>>                        FSDirectory directory = FSDirectory.open(file);
>>>>                        // 用来创建索引
>>>>                        IndexWriterConfig conf = new IndexWriterConfig(Version.LUCENE_36,
>>>>                                        new StandardAnalyzer(Version.LUCENE_36));
>>>>                        // 避免重复运用时每次都附加index导致查询出多条记录
>>>>                        conf.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
>>>>                        IndexWriter iw = new IndexWriter(directory, conf);
>>>>                        iw.setInfoStream(System.out);
>>>>                        Document doc = new Document();
>>>>                        // Store.YES 保存 可以查询 可以打印内容
>>>>                        Field storeYes = new Field("content1", "my my my my storeyes test",
>>>>                                        Store.YES, Index.ANALYZED);
>>>>                        // Store.NO 不保存 可以查询 不可打印内容 由于不保存内容所以节省空间
>>>>                        Field storeNo = new Field("content2", "my storeno test", Store.YES,
>>>>                                        Index.ANALYZED);
>>>>
>>>>                        doc.add(storeYes);
>>>>                        doc.add(storeNo);
>>>>
>>>>                        Document doc1 = new Document();
>>>>                        Field storeYes1 = new Field("doc1content1",
>>>>                                        "doc1 my storeyes test", Store.YES, Index.ANALYZED);
>>>>                        doc1.add(storeYes1);
>>>>
>>>>                        Document doc2 = new Document();
>>>>                        Field storeYesdoc2 = new Field("doc2content1",
>>>>                                        "doc2 my storeyes test", Store.YES, Index.ANALYZED);
>>>>                        Field storeNodoc2 = new Field("doc2content2",
>>>>                                        "doc2 my storeno test", Store.YES, Index.ANALYZED);
>>>>                        doc2.add(storeYesdoc2);
>>>>                        doc2.add(storeNodoc2);
>>>>
>>>>                        iw.addDocument(doc);
>>>>                        iw.addDocument(doc1);
>>>>                        iw.addDocument(doc2);
>>>>                        System.out.println("doc :" + doc);
>>>>                        System.out.println("doc1 :" + doc1);
>>>>                        System.out.println("doc2 :" + doc2);
>>>>                        iw.forceMerge(1);
>>>>                        iw.close();
>>>>                } catch (CorruptIndexException e) {
>>>>                        // TODO Auto-generated catch block
>>>>                        e.printStackTrace();
>>>>                } catch (LockObtainFailedException e) {
>>>>                        // TODO Auto-generated catch block
>>>>                        e.printStackTrace();
>>>>                } catch (IOException e) {
>>>>                        // TODO Auto-generated catch block
>>>>                        e.printStackTrace();
>>>>                }
>>>>        }
>>>>
>>>>
>>>> THE HEX in .frq file is
>>>> 00 04 01 01 01 01 01 03  03 03 03 05 05 05 05 05
>>>> 05 05 05
>>>>
>>>> in my  analysis
>>>> 1   [0,4] [0,1] [0,1] [0,1] [0,1]
>>>> [0,1][1,1][1,1][1,1][1,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1]
>>>> 2   [0,4] [0,1] [0,1] [0,1] [0,1] [0,1]
>>>> [1,1][1,1][1,1][1,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1]
>>>> 3   [0,4][1][1][1][1][1][3][3][3][3][5] [5][5][5][5][5][5][5]
>>>>
>>>> ---------------------------------------------------------------------
>>>> To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
>>>> For additional commands, e-mail: java-user-help@lucene.apache.org
>>>>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
> For additional commands, e-mail: java-user-help@lucene.apache.org
>

---------------------------------------------------------------------
To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-user-help@lucene.apache.org


Re: about .frq file format in doc

Posted by wangjing <pp...@gmail.com>.
thanks

could you help me to solve another problem,

why lucene will reset lastDocID = 0 when finish add one doc?

in source code FormatPostingsDocsWriter.java

	@Override
	void finish() throws IOException {
		long skipPointer = skipListWriter.writeSkip(out);
		// TODO: this is abstraction violation -- we should not
		// peek up into parents terms encoding format
		termInfo.set(df, parent.freqStart, parent.proxStart,
				(int) (skipPointer - parent.freqStart));
		// TODO: we could do this incrementally
		UnicodeUtil.UTF16toUTF8(parent.currentTerm, parent.currentTermStart,
				utf8);
		if (df > 0) {
			parent.termsOut.add(fieldInfo.number, utf8.result, utf8.length,
					termInfo);
		}
		lastDocID = 0;
		df = 0;
	}

@Override
	FormatPostingsPositionsConsumer addDoc(int docID, int termDocFreq)
			throws IOException {
		System.out.println("docID:" + docID);
		System.out.println("termDocFreq:" + termDocFreq);
		
		final int delta = docID - lastDocID;
		System.out.println("delta:" + delta);
		if (docID < 0 || (df > 0 && delta <= 0))
			throw new CorruptIndexException("docs out of order (" + docID
					+ " <= " + lastDocID + " ) (out: " + out + ")");
		
		if ((++df % skipInterval) == 0) {
			// TODO: abstraction violation
			skipListWriter.setSkipData(lastDocID, storePayloads,
					posWriter.lastPayloadLength);
			skipListWriter.bufferSkip(df);
		}
		assert docID < totalNumDocs : "docID=" + docID + " totalNumDocs="
				+ totalNumDocs;
		lastDocID = docID;
		if (omitTermFreqAndPositions)
			out.writeVInt(delta);
		else if (1 == termDocFreq)
			
			out.writeVInt((delta << 1) | 1);
		else {
			out.writeVInt(delta << 1);
			out.writeVInt(termDocFreq);
		}
		System.out.println("------------");
		return posWriter;
	}



because final int delta = docID - lastDocID;  and  lastDocID set to 0;

it will let  deltaDoc   always equals docID .

docId is 0,1,2 but the delta is 0,1,2, not 0,1,1.

It left me rather confused.


docID:0
termDocFreq:4
delta:0
------------
docID:0
termDocFreq:1
delta:0
------------
docID:0
termDocFreq:1
delta:0
------------
docID:0
termDocFreq:1
delta:0
------------
docID:0
termDocFreq:1
delta:0
------------
docID:0
termDocFreq:1
delta:0
------------
docID:1
termDocFreq:1
delta:1
------------
docID:1
termDocFreq:1
delta:1
------------
docID:1
termDocFreq:1
delta:1
------------
docID:1
termDocFreq:1
delta:1
------------
docID:2
termDocFreq:1
delta:2
------------
docID:2
termDocFreq:1
delta:2
------------
docID:2
termDocFreq:1
delta:2
------------
docID:2
termDocFreq:1
delta:2
------------
docID:2
termDocFreq:1
delta:2
------------
docID:2
termDocFreq:1
delta:2
------------
docID:2
termDocFreq:1
delta:2
------------
docID:2
termDocFreq:1
delta:2




On Thu, Jun 28, 2012 at 12:09 AM, Simon Willnauer
<si...@googlemail.com> wrote:
> see definitions:
> http://lucene.apache.org/core/3_6_0/fileformats.html#Definitions
>
> simon
>
> On Wed, Jun 27, 2012 at 6:08 PM, Simon Willnauer
> <si...@googlemail.com> wrote:
>> a term in this context is a (field,text) tuple - does this make sense?
>> simon
>>
>> On Wed, Jun 27, 2012 at 11:40 AM, wangjing <pp...@gmail.com> wrote:
>>> http://lucene.apache.org/core/3_6_0/fileformats.html#Frequencies
>>>
>>> The .frq file contains the lists of documents which contain each term,
>>> along with the frequency of the term in that document (except when
>>> frequencies are omitted: IndexOptions.DOCS_ONLY).
>>>
>>> BUT in my example,the frq record the frequency of the term in FIELD?
>>>
>>> my app is
>>> public void createLuceneIndex() {
>>>                try {
>>>                        File file = new File(path);
>>>                        FSDirectory directory = FSDirectory.open(file);
>>>                        // 用来创建索引
>>>                        IndexWriterConfig conf = new IndexWriterConfig(Version.LUCENE_36,
>>>                                        new StandardAnalyzer(Version.LUCENE_36));
>>>                        // 避免重复运用时每次都附加index导致查询出多条记录
>>>                        conf.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
>>>                        IndexWriter iw = new IndexWriter(directory, conf);
>>>                        iw.setInfoStream(System.out);
>>>                        Document doc = new Document();
>>>                        // Store.YES 保存 可以查询 可以打印内容
>>>                        Field storeYes = new Field("content1", "my my my my storeyes test",
>>>                                        Store.YES, Index.ANALYZED);
>>>                        // Store.NO 不保存 可以查询 不可打印内容 由于不保存内容所以节省空间
>>>                        Field storeNo = new Field("content2", "my storeno test", Store.YES,
>>>                                        Index.ANALYZED);
>>>
>>>                        doc.add(storeYes);
>>>                        doc.add(storeNo);
>>>
>>>                        Document doc1 = new Document();
>>>                        Field storeYes1 = new Field("doc1content1",
>>>                                        "doc1 my storeyes test", Store.YES, Index.ANALYZED);
>>>                        doc1.add(storeYes1);
>>>
>>>                        Document doc2 = new Document();
>>>                        Field storeYesdoc2 = new Field("doc2content1",
>>>                                        "doc2 my storeyes test", Store.YES, Index.ANALYZED);
>>>                        Field storeNodoc2 = new Field("doc2content2",
>>>                                        "doc2 my storeno test", Store.YES, Index.ANALYZED);
>>>                        doc2.add(storeYesdoc2);
>>>                        doc2.add(storeNodoc2);
>>>
>>>                        iw.addDocument(doc);
>>>                        iw.addDocument(doc1);
>>>                        iw.addDocument(doc2);
>>>                        System.out.println("doc :" + doc);
>>>                        System.out.println("doc1 :" + doc1);
>>>                        System.out.println("doc2 :" + doc2);
>>>                        iw.forceMerge(1);
>>>                        iw.close();
>>>                } catch (CorruptIndexException e) {
>>>                        // TODO Auto-generated catch block
>>>                        e.printStackTrace();
>>>                } catch (LockObtainFailedException e) {
>>>                        // TODO Auto-generated catch block
>>>                        e.printStackTrace();
>>>                } catch (IOException e) {
>>>                        // TODO Auto-generated catch block
>>>                        e.printStackTrace();
>>>                }
>>>        }
>>>
>>>
>>> THE HEX in .frq file is
>>> 00 04 01 01 01 01 01 03  03 03 03 05 05 05 05 05
>>> 05 05 05
>>>
>>> in my  analysis
>>> 1   [0,4] [0,1] [0,1] [0,1] [0,1]
>>> [0,1][1,1][1,1][1,1][1,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1]
>>> 2   [0,4] [0,1] [0,1] [0,1] [0,1] [0,1]
>>> [1,1][1,1][1,1][1,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1]
>>> 3   [0,4][1][1][1][1][1][3][3][3][3][5] [5][5][5][5][5][5][5]
>>>
>>> ---------------------------------------------------------------------
>>> To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
>>> For additional commands, e-mail: java-user-help@lucene.apache.org
>>>

---------------------------------------------------------------------
To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-user-help@lucene.apache.org


Re: about .frq file format in doc

Posted by Simon Willnauer <si...@googlemail.com>.
see definitions:
http://lucene.apache.org/core/3_6_0/fileformats.html#Definitions

simon

On Wed, Jun 27, 2012 at 6:08 PM, Simon Willnauer
<si...@googlemail.com> wrote:
> a term in this context is a (field,text) tuple - does this make sense?
> simon
>
> On Wed, Jun 27, 2012 at 11:40 AM, wangjing <pp...@gmail.com> wrote:
>> http://lucene.apache.org/core/3_6_0/fileformats.html#Frequencies
>>
>> The .frq file contains the lists of documents which contain each term,
>> along with the frequency of the term in that document (except when
>> frequencies are omitted: IndexOptions.DOCS_ONLY).
>>
>> BUT in my example,the frq record the frequency of the term in FIELD?
>>
>> my app is
>> public void createLuceneIndex() {
>>                try {
>>                        File file = new File(path);
>>                        FSDirectory directory = FSDirectory.open(file);
>>                        // 用来创建索引
>>                        IndexWriterConfig conf = new IndexWriterConfig(Version.LUCENE_36,
>>                                        new StandardAnalyzer(Version.LUCENE_36));
>>                        // 避免重复运用时每次都附加index导致查询出多条记录
>>                        conf.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
>>                        IndexWriter iw = new IndexWriter(directory, conf);
>>                        iw.setInfoStream(System.out);
>>                        Document doc = new Document();
>>                        // Store.YES 保存 可以查询 可以打印内容
>>                        Field storeYes = new Field("content1", "my my my my storeyes test",
>>                                        Store.YES, Index.ANALYZED);
>>                        // Store.NO 不保存 可以查询 不可打印内容 由于不保存内容所以节省空间
>>                        Field storeNo = new Field("content2", "my storeno test", Store.YES,
>>                                        Index.ANALYZED);
>>
>>                        doc.add(storeYes);
>>                        doc.add(storeNo);
>>
>>                        Document doc1 = new Document();
>>                        Field storeYes1 = new Field("doc1content1",
>>                                        "doc1 my storeyes test", Store.YES, Index.ANALYZED);
>>                        doc1.add(storeYes1);
>>
>>                        Document doc2 = new Document();
>>                        Field storeYesdoc2 = new Field("doc2content1",
>>                                        "doc2 my storeyes test", Store.YES, Index.ANALYZED);
>>                        Field storeNodoc2 = new Field("doc2content2",
>>                                        "doc2 my storeno test", Store.YES, Index.ANALYZED);
>>                        doc2.add(storeYesdoc2);
>>                        doc2.add(storeNodoc2);
>>
>>                        iw.addDocument(doc);
>>                        iw.addDocument(doc1);
>>                        iw.addDocument(doc2);
>>                        System.out.println("doc :" + doc);
>>                        System.out.println("doc1 :" + doc1);
>>                        System.out.println("doc2 :" + doc2);
>>                        iw.forceMerge(1);
>>                        iw.close();
>>                } catch (CorruptIndexException e) {
>>                        // TODO Auto-generated catch block
>>                        e.printStackTrace();
>>                } catch (LockObtainFailedException e) {
>>                        // TODO Auto-generated catch block
>>                        e.printStackTrace();
>>                } catch (IOException e) {
>>                        // TODO Auto-generated catch block
>>                        e.printStackTrace();
>>                }
>>        }
>>
>>
>> THE HEX in .frq file is
>> 00 04 01 01 01 01 01 03  03 03 03 05 05 05 05 05
>> 05 05 05
>>
>> in my  analysis
>> 1   [0,4] [0,1] [0,1] [0,1] [0,1]
>> [0,1][1,1][1,1][1,1][1,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1]
>> 2   [0,4] [0,1] [0,1] [0,1] [0,1] [0,1]
>> [1,1][1,1][1,1][1,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1]
>> 3   [0,4][1][1][1][1][1][3][3][3][3][5] [5][5][5][5][5][5][5]
>>
>> ---------------------------------------------------------------------
>> To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
>> For additional commands, e-mail: java-user-help@lucene.apache.org
>>

Re: about .frq file format in doc

Posted by Simon Willnauer <si...@googlemail.com>.
a term in this context is a (field,text) tuple - does this make sense?
simon

On Wed, Jun 27, 2012 at 11:40 AM, wangjing <pp...@gmail.com> wrote:
> http://lucene.apache.org/core/3_6_0/fileformats.html#Frequencies
>
> The .frq file contains the lists of documents which contain each term,
> along with the frequency of the term in that document (except when
> frequencies are omitted: IndexOptions.DOCS_ONLY).
>
> BUT in my example,the frq record the frequency of the term in FIELD?
>
> my app is
> public void createLuceneIndex() {
>                try {
>                        File file = new File(path);
>                        FSDirectory directory = FSDirectory.open(file);
>                        // 用来创建索引
>                        IndexWriterConfig conf = new IndexWriterConfig(Version.LUCENE_36,
>                                        new StandardAnalyzer(Version.LUCENE_36));
>                        // 避免重复运用时每次都附加index导致查询出多条记录
>                        conf.setOpenMode(IndexWriterConfig.OpenMode.CREATE);
>                        IndexWriter iw = new IndexWriter(directory, conf);
>                        iw.setInfoStream(System.out);
>                        Document doc = new Document();
>                        // Store.YES 保存 可以查询 可以打印内容
>                        Field storeYes = new Field("content1", "my my my my storeyes test",
>                                        Store.YES, Index.ANALYZED);
>                        // Store.NO 不保存 可以查询 不可打印内容 由于不保存内容所以节省空间
>                        Field storeNo = new Field("content2", "my storeno test", Store.YES,
>                                        Index.ANALYZED);
>
>                        doc.add(storeYes);
>                        doc.add(storeNo);
>
>                        Document doc1 = new Document();
>                        Field storeYes1 = new Field("doc1content1",
>                                        "doc1 my storeyes test", Store.YES, Index.ANALYZED);
>                        doc1.add(storeYes1);
>
>                        Document doc2 = new Document();
>                        Field storeYesdoc2 = new Field("doc2content1",
>                                        "doc2 my storeyes test", Store.YES, Index.ANALYZED);
>                        Field storeNodoc2 = new Field("doc2content2",
>                                        "doc2 my storeno test", Store.YES, Index.ANALYZED);
>                        doc2.add(storeYesdoc2);
>                        doc2.add(storeNodoc2);
>
>                        iw.addDocument(doc);
>                        iw.addDocument(doc1);
>                        iw.addDocument(doc2);
>                        System.out.println("doc :" + doc);
>                        System.out.println("doc1 :" + doc1);
>                        System.out.println("doc2 :" + doc2);
>                        iw.forceMerge(1);
>                        iw.close();
>                } catch (CorruptIndexException e) {
>                        // TODO Auto-generated catch block
>                        e.printStackTrace();
>                } catch (LockObtainFailedException e) {
>                        // TODO Auto-generated catch block
>                        e.printStackTrace();
>                } catch (IOException e) {
>                        // TODO Auto-generated catch block
>                        e.printStackTrace();
>                }
>        }
>
>
> THE HEX in .frq file is
> 00 04 01 01 01 01 01 03  03 03 03 05 05 05 05 05
> 05 05 05
>
> in my  analysis
> 1   [0,4] [0,1] [0,1] [0,1] [0,1]
> [0,1][1,1][1,1][1,1][1,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1]
> 2   [0,4] [0,1] [0,1] [0,1] [0,1] [0,1]
> [1,1][1,1][1,1][1,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1][2,1]
> 3   [0,4][1][1][1][1][1][3][3][3][3][5] [5][5][5][5][5][5][5]
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
> For additional commands, e-mail: java-user-help@lucene.apache.org
>

---------------------------------------------------------------------
To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-user-help@lucene.apache.org